From 95bd4b2853f5571f12845df83824b044720052a3 Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Tue, 6 Aug 2019 16:11:21 -0700 Subject: [PATCH 01/57] [BEAM-7738] Add external transform support to PubsubIO --- .../flink/job-server/flink_job_server.gradle | 1 + .../io/google-cloud-platform/build.gradle | 4 + .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 181 ++++++++++++- .../io/gcp/pubsub/PubsubIOExternalTest.java | 250 ++++++++++++++++++ .../apache_beam/io/external/gcp/__init__.py | 18 ++ .../apache_beam/io/external/gcp/pubsub.py | 183 +++++++++++++ 6 files changed, 635 insertions(+), 2 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java create mode 100644 sdks/python/apache_beam/io/external/gcp/__init__.py create mode 100644 sdks/python/apache_beam/io/external/gcp/pubsub.py diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index ee856e791ffee..b93b950dc0247 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -89,6 +89,7 @@ dependencies { // For resolving external transform requests runtime project(":sdks:java:io:kafka") runtime library.java.kafka_clients + runtime project(":sdks:java:io:google-cloud-platform") } // NOTE: runShadow must be used in order to run the job server. The standard run diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index 0a9b8a9195cc9..0c1befd311c94 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -69,6 +69,10 @@ dependencies { testCompile project(path: ":sdks:java:core", configuration: "shadowTest") testCompile project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntime") testCompile project(path: ":runners:direct-java", configuration: "shadow") + testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime") + testCompile project(path: ":sdks:java:testing:test-utils", configuration: "testRuntime") + // For testing Cross-language transforms + testCompile project(":runners:core-construction-java") testCompile library.java.hamcrest_core testCompile library.java.hamcrest_library testCompile library.java.junit diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 4f745ea9a4bb5..a919f44ff48f7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -20,7 +20,10 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.api.client.util.Clock; +import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; import java.io.IOException; import java.io.Serializable; @@ -39,9 +42,11 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath; @@ -53,6 +58,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ExternalTransformBuilder; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -66,6 +72,7 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; @@ -705,7 +712,8 @@ public abstract static class Read extends PTransform> abstract Builder toBuilder(); @AutoValue.Builder - abstract static class Builder { + abstract static class Builder + implements ExternalTransformBuilder> { abstract Builder setTopicProvider(ValueProvider topic); abstract Builder setPubsubClientFactory(PubsubClient.PubsubClientFactory clientFactory); @@ -733,6 +741,86 @@ abstract static class Builder { abstract Builder setClock(@Nullable Clock clock); abstract Read build(); + + @Override + public PTransform> buildExternal(External.Configuration config) { + if (config.topic != null) { + StaticValueProvider topic = StaticValueProvider.of(utf8String(config.topic)); + setTopicProvider(NestedValueProvider.of(topic, new TopicTranslator())); + } + if (config.subscription != null) { + StaticValueProvider subscription = + StaticValueProvider.of(utf8String(config.subscription)); + setSubscriptionProvider( + NestedValueProvider.of(subscription, new SubscriptionTranslator())); + } + if (config.idAttribute != null) { + String idAttribute = utf8String(config.idAttribute); + setIdAttribute(idAttribute); + } + if (config.timestampAttribute != null) { + String timestampAttribute = utf8String(config.timestampAttribute); + setTimestampAttribute(timestampAttribute); + } + setPubsubClientFactory(FACTORY); + setNeedsAttributes(config.needsAttributes); + Coder coder = ByteArrayCoder.of(); + if (config.needsAttributes) { + SimpleFunction parseFn = + (SimpleFunction) new ParsePayloadAsPubsubMessageProto(); + setParseFn(parseFn); + setCoder(coder); + } else { + setParseFn(new ParsePayloadUsingCoder<>(coder)); + setCoder(coder); + } + setNeedsMessageId(false); + return build(); + } + } + + /** Exposes {@link PubSubIO.Read} as an external transform for cross-language usage. */ + @Experimental + @AutoService(ExternalTransformRegistrar.class) + public static class External implements ExternalTransformRegistrar { + + public static final String URN = "beam:external:java:pubsub:read:v1"; + + @Override + public Map> knownBuilders() { + return ImmutableMap.of(URN, AutoValue_PubsubIO_Read.Builder.class); + } + + /** Parameters class to expose the transform to an external SDK. */ + public static class Configuration { + + // All byte arrays are UTF-8 encoded strings + @Nullable private byte[] topic; + @Nullable private byte[] subscription; + @Nullable private byte[] idAttribute; + @Nullable private byte[] timestampAttribute; + private boolean needsAttributes; + + public void setTopic(@Nullable byte[] topic) { + this.topic = topic; + } + + public void setSubscription(@Nullable byte[] subscription) { + this.subscription = subscription; + } + + public void setIdLabel(@Nullable byte[] idAttribute) { + this.idAttribute = idAttribute; + } + + public void setTimestampAttribute(@Nullable byte[] timestampAttribute) { + this.timestampAttribute = timestampAttribute; + } + + public void setWithAttributes(Long needsAttributes) { + this.needsAttributes = needsAttributes >= 1; + } + } } /** @@ -955,7 +1043,8 @@ public abstract static class Write extends PTransform, PDone> abstract Builder toBuilder(); @AutoValue.Builder - abstract static class Builder { + abstract static class Builder + implements ExternalTransformBuilder, PDone> { abstract Builder setTopicProvider(ValueProvider topicProvider); abstract Builder setPubsubClientFactory(PubsubClient.PubsubClientFactory factory); @@ -971,6 +1060,60 @@ abstract static class Builder { abstract Builder setFormatFn(SimpleFunction formatFn); abstract Write build(); + + @Override + public PTransform, PDone> buildExternal(External.Configuration config) { + if (config.topic != null) { + StaticValueProvider topic = StaticValueProvider.of(utf8String(config.topic)); + setTopicProvider(NestedValueProvider.of(topic, new TopicTranslator())); + } + if (config.idAttribute != null) { + String idAttribute = utf8String(config.idAttribute); + setIdAttribute(idAttribute); + } + if (config.timestampAttribute != null) { + String timestampAttribute = utf8String(config.timestampAttribute); + setTimestampAttribute(timestampAttribute); + } + SimpleFunction parseFn = + (SimpleFunction) new FormatPayloadFromPubsubMessageProto(); + setFormatFn(parseFn); + return build(); + } + } + + /** Exposes {@link PubSubIO.Write} as an external transform for cross-language usage. */ + @Experimental + @AutoService(ExternalTransformRegistrar.class) + public static class External implements ExternalTransformRegistrar { + + public static final String URN = "beam:external:java:pubsub:write:v1"; + + @Override + public Map> knownBuilders() { + return ImmutableMap.of(URN, AutoValue_PubsubIO_Write.Builder.class); + } + + /** Parameters class to expose the transform to an external SDK. */ + public static class Configuration { + + // All byte arrays are UTF-8 encoded strings + private byte[] topic; + @Nullable private byte[] idAttribute; + @Nullable private byte[] timestampAttribute; + + public void setTopic(byte[] topic) { + this.topic = topic; + } + + public void setIdLabel(@Nullable byte[] idAttribute) { + this.idAttribute = idAttribute; + } + + public void setTimestampAttribute(@Nullable byte[] timestampAttribute) { + this.timestampAttribute = timestampAttribute; + } + } } /** @@ -1213,6 +1356,22 @@ public T apply(PubsubMessage input) { } } + private static class ParsePayloadAsPubsubMessageProto + extends SimpleFunction { + @Override + public byte[] apply(PubsubMessage input) { + Map attributes = input.getAttributeMap(); + com.google.pubsub.v1.PubsubMessage.Builder message = + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(input.getPayload())); + // TODO(BEAM-8085) this should not be null + if (attributes != null) { + message.putAllAttributes(attributes); + } + return message.build().toByteArray(); + } + } + private static class FormatPayloadAsUtf8 extends SimpleFunction { @Override public PubsubMessage apply(String input) { @@ -1237,10 +1396,28 @@ public PubsubMessage apply(T input) { } } + private static class FormatPayloadFromPubsubMessageProto + extends SimpleFunction { + @Override + public PubsubMessage apply(byte[] input) { + try { + com.google.pubsub.v1.PubsubMessage message = + com.google.pubsub.v1.PubsubMessage.parseFrom(input); + return new PubsubMessage(message.getData().toByteArray(), message.getAttributesMap()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException("Could not decode Pubsub message", e); + } + } + } + private static class IdentityMessageFn extends SimpleFunction { @Override public PubsubMessage apply(PubsubMessage input) { return input; } } + + private static String utf8String(byte[] bytes) { + return new String(bytes, Charsets.UTF_8); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java new file mode 100644 index 0000000000000..00ede35482c94 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.pubsub; + +import static org.hamcrest.MatcherAssert.assertThat; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import javax.annotation.Nullable; +import org.apache.beam.model.expansion.v1.ExpansionApi; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.ParDoTranslation; +import org.apache.beam.runners.core.construction.PipelineTranslation; +import org.apache.beam.runners.core.construction.ReadTranslation; +import org.apache.beam.runners.core.construction.expansion.ExpansionService; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.powermock.reflect.Whitebox; + +/** Tests for building {@link PubsubIO} externally via the ExpansionService. */ +@RunWith(JUnit4.class) +public class PubsubIOExternalTest { + @Test + public void testConstructPubsubRead() throws Exception { + String topic = "projects/project-1234/topics/topic_name"; + String idAttribute = "id_foo"; + Long needsAttributes = 1L; + + ExternalTransforms.ExternalConfigurationPayload payload = + ExternalTransforms.ExternalConfigurationPayload.newBuilder() + .putConfiguration( + "topic", + ExternalTransforms.ConfigValue.newBuilder() + .addCoderUrn("beam:coder:bytes:v1") + .setPayload(ByteString.copyFrom(encodeString(topic))) + .build()) + .putConfiguration( + "id_label", + ExternalTransforms.ConfigValue.newBuilder() + .addCoderUrn("beam:coder:bytes:v1") + .setPayload(ByteString.copyFrom(encodeString(idAttribute))) + .build()) + .putConfiguration( + "with_attributes", + ExternalTransforms.ConfigValue.newBuilder() + .addCoderUrn("beam:coder:varint:v1") + .setPayload(ByteString.copyFrom(encodeVarLong(needsAttributes))) + .build()) + .build(); + + RunnerApi.Components defaultInstance = RunnerApi.Components.getDefaultInstance(); + ExpansionApi.ExpansionRequest request = + ExpansionApi.ExpansionRequest.newBuilder() + .setComponents(defaultInstance) + .setTransform( + RunnerApi.PTransform.newBuilder() + .setUniqueName("test") + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn("beam:external:java:pubsub:read:v1") + .setPayload(payload.toByteString()))) + .setNamespace("test_namespace") + .build(); + + ExpansionService expansionService = new ExpansionService(); + TestStreamObserver observer = new TestStreamObserver<>(); + expansionService.expand(request, observer); + + ExpansionApi.ExpansionResponse result = observer.result; + RunnerApi.PTransform transform = result.getTransform(); + assertThat( + transform.getSubtransformsList(), + Matchers.contains( + "test_namespacetest/PubsubUnboundedSource", "test_namespacetest/MapElements")); + assertThat(transform.getInputsCount(), Matchers.is(0)); + assertThat(transform.getOutputsCount(), Matchers.is(1)); + + RunnerApi.PTransform pubsubComposite = + result.getComponents().getTransformsOrThrow(transform.getSubtransforms(0)); + RunnerApi.PTransform pubsubRead = + result.getComponents().getTransformsOrThrow(pubsubComposite.getSubtransforms(0)); + RunnerApi.ReadPayload readPayload = + RunnerApi.ReadPayload.parseFrom(pubsubRead.getSpec().getPayload()); + PubsubUnboundedSource.PubsubSource source = + (PubsubUnboundedSource.PubsubSource) ReadTranslation.unboundedSourceFromProto(readPayload); + PubsubUnboundedSource spec = source.outer; + + assertThat( + spec.getTopicProvider() == null ? null : String.valueOf(spec.getTopicProvider()), + Matchers.is(topic)); + assertThat(spec.getIdAttribute(), Matchers.is(idAttribute)); + assertThat(spec.getNeedsAttributes(), Matchers.is(true)); + } + + @Test + public void testConstructPubsubWrite() throws Exception { + String topic = "projects/project-1234/topics/topic_name"; + String idAttribute = "id_foo"; + + ExternalTransforms.ExternalConfigurationPayload payload = + ExternalTransforms.ExternalConfigurationPayload.newBuilder() + .putConfiguration( + "topic", + ExternalTransforms.ConfigValue.newBuilder() + .addCoderUrn("beam:coder:bytes:v1") + .setPayload(ByteString.copyFrom(encodeString(topic))) + .build()) + .putConfiguration( + "id_label", + ExternalTransforms.ConfigValue.newBuilder() + .addCoderUrn("beam:coder:bytes:v1") + .setPayload(ByteString.copyFrom(encodeString(idAttribute))) + .build()) + .build(); + + Pipeline p = Pipeline.create(); + p.apply("unbounded", Create.of(1, 2, 3)).setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + String inputPCollection = + Iterables.getOnlyElement( + Iterables.getLast(pipelineProto.getComponents().getTransformsMap().values()) + .getOutputsMap() + .values()); + + ExpansionApi.ExpansionRequest request = + ExpansionApi.ExpansionRequest.newBuilder() + .setComponents(pipelineProto.getComponents()) + .setTransform( + RunnerApi.PTransform.newBuilder() + .setUniqueName("test") + .putInputs("input", inputPCollection) + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn("beam:external:java:pubsub:write:v1") + .setPayload(payload.toByteString()))) + .setNamespace("test_namespace") + .build(); + + ExpansionService expansionService = new ExpansionService(); + TestStreamObserver observer = new TestStreamObserver<>(); + expansionService.expand(request, observer); + + ExpansionApi.ExpansionResponse result = observer.result; + + RunnerApi.PTransform transform = result.getTransform(); + assertThat( + transform.getSubtransformsList(), + Matchers.contains( + "test_namespacetest/MapElements", "test_namespacetest/PubsubUnboundedSink")); + assertThat(transform.getInputsCount(), Matchers.is(1)); + assertThat(transform.getOutputsCount(), Matchers.is(0)); + + // test_namespacetest/PubsubUnboundedSink + RunnerApi.PTransform writeComposite = + result.getComponents().getTransformsOrThrow(transform.getSubtransforms(1)); + + // test_namespacetest/PubsubUnboundedSink/PubsubUnboundedSink.Writer + RunnerApi.PTransform writeComposite2 = + result.getComponents().getTransformsOrThrow(writeComposite.getSubtransforms(3)); + + // test_namespacetest/PubsubUnboundedSink/PubsubUnboundedSink.Writer/ParMultiDo(Writer) + RunnerApi.PTransform writeParDo = + result.getComponents().getTransformsOrThrow(writeComposite2.getSubtransforms(0)); + + RunnerApi.ParDoPayload parDoPayload = + RunnerApi.ParDoPayload.parseFrom(writeParDo.getSpec().getPayload()); + DoFn pubsubWriter = ParDoTranslation.getDoFn(parDoPayload); + + String idAttributeActual = (String) Whitebox.getInternalState(pubsubWriter, "idAttribute"); + + ValueProvider topicActual = + (ValueProvider) Whitebox.getInternalState(pubsubWriter, "topic"); + + assertThat(topicActual == null ? null : String.valueOf(topicActual), Matchers.is(topic)); + assertThat(idAttributeActual, Matchers.is(idAttribute)); + } + + private static byte[] encodeString(String str) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ByteArrayCoder.of().encode(utf8Bytes(str), baos); + return baos.toByteArray(); + } + + private static byte[] encodeVarLong(Long value) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + VarLongCoder.of().encode(value, baos); + return baos.toByteArray(); + } + + private static @Nullable String getTopic(@Nullable ValueProvider value) { + if (value == null) { + return null; + } + return String.valueOf(value); + } + + private static byte[] utf8Bytes(String str) { + Preconditions.checkNotNull(str, "String must not be null."); + return str.getBytes(Charsets.UTF_8); + } + + private static class TestStreamObserver implements StreamObserver { + + private T result; + + @Override + public void onNext(T t) { + result = t; + } + + @Override + public void onError(Throwable throwable) { + throw new RuntimeException("Should not happen", throwable); + } + + @Override + public void onCompleted() {} + } +} diff --git a/sdks/python/apache_beam/io/external/gcp/__init__.py b/sdks/python/apache_beam/io/external/gcp/__init__.py new file mode 100644 index 0000000000000..6569e3fe5de4d --- /dev/null +++ b/sdks/python/apache_beam/io/external/gcp/__init__.py @@ -0,0 +1,18 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import absolute_import diff --git a/sdks/python/apache_beam/io/external/gcp/pubsub.py b/sdks/python/apache_beam/io/external/gcp/pubsub.py new file mode 100644 index 0000000000000..4ff954419b2ee --- /dev/null +++ b/sdks/python/apache_beam/io/external/gcp/pubsub.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. +# + +from __future__ import absolute_import + +from apache_beam import ExternalTransform +from apache_beam import pvalue +from apache_beam.coders import BytesCoder +from apache_beam.coders import VarIntCoder +from apache_beam.coders.coders import LengthPrefixCoder +from apache_beam.io.gcp import pubsub +from apache_beam.portability.api.external_transforms_pb2 import ConfigValue +from apache_beam.portability.api.external_transforms_pb2 import ExternalConfigurationPayload +from apache_beam.transforms import Map +from apache_beam.transforms import ptransform + + +class ReadFromPubSub(ptransform.PTransform): + """An external ``PTransform`` for reading from Cloud Pub/Sub.""" + + _urn = 'beam:external:java:pubsub:read:v1' + + def __init__(self, topic=None, subscription=None, id_label=None, + with_attributes=False, timestamp_attribute=None, + expansion_service='localhost:8097'): + """Initializes ``ReadFromPubSub``. + + Args: + topic: Cloud Pub/Sub topic in the form + "projects//topics/". If provided, subscription must be + None. + subscription: Existing Cloud Pub/Sub subscription to use in the + form "projects//subscriptions/". If not + specified, a temporary subscription will be created from the specified + topic. If provided, topic must be None. + id_label: The attribute on incoming Pub/Sub messages to use as a unique + record identifier. When specified, the value of this attribute (which + can be any string that uniquely identifies the record) will be used for + deduplication of messages. If not provided, we cannot guarantee + that no duplicate data will be delivered on the Pub/Sub stream. In this + case, deduplication of the stream will be strictly best effort. + with_attributes: + True - output elements will be :class:`~PubsubMessage` objects. + False - output elements will be of type ``bytes`` (message + data only). + timestamp_attribute: Message value to use as element timestamp. If None, + uses message publishing time as the timestamp. + + Timestamp values should be in one of two formats: + + - A numerical value representing the number of milliseconds since the + Unix epoch. + - A string in RFC 3339 format, UTC timezone. 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) may be ignored. + """ + super(ReadFromPubSub, self).__init__() + self.topic = topic + self.subscription = subscription + self.id_label = id_label + self.with_attributes = with_attributes + self.timestamp_attribute = timestamp_attribute + self.expansion_service = expansion_service + + def expand(self, pbegin): + if not isinstance(pbegin, pvalue.PBegin): + raise Exception("ReadFromPubSub must be a root transform") + + args = { + 'with_attributes': _encode_bool(self.with_attributes), + } + + if self.topic is not None: + args['topic'] = _encode_str(self.topic) + + if self.subscription is not None: + args['subscription'] = _encode_str(self.subscription) + + if self.id_label is not None: + args['id_label'] = _encode_str(self.id_label) + + if self.timestamp_attribute is not None: + args['timestamp_attribute'] = _encode_str(self.timestamp_attribute) + + payload = ExternalConfigurationPayload(configuration=args) + pcoll = pbegin.apply( + ExternalTransform( + self._urn, + payload.SerializeToString(), + self.expansion_service)) + if self.with_attributes: + pcoll = pcoll | Map(pubsub.PubsubMessage._from_proto_str) + pcoll.element_type = pubsub.PubsubMessage + else: + pcoll.element_type = bytes + return pcoll + + +class WriteToPubSub(ptransform.PTransform): + """An external ``PTransform`` for writing messages to Cloud Pub/Sub.""" + + _urn = 'beam:external:java:pubsub:write:v1' + + def __init__(self, topic, with_attributes=False, id_label=None, + timestamp_attribute=None, expansion_service='localhost:8097'): + """Initializes ``WriteToPubSub``. + + Args: + topic: Cloud Pub/Sub topic in the form "/topics//". + with_attributes: + True - input elements will be :class:`~PubsubMessage` objects. + False - input elements will be of type ``bytes`` (message + data only). + id_label: 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. + timestamp_attribute: 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. + """ + super(WriteToPubSub, self).__init__() + self.topic = topic + self.with_attributes = with_attributes + self.id_label = id_label + self.timestamp_attribute = timestamp_attribute + self.expansion_service = expansion_service + + def expand(self, pvalue): + + if self.with_attributes: + pcoll = pvalue | 'ToProtobuf' >> Map(pubsub.WriteToPubSub.to_proto_str) + else: + pcoll = pvalue | 'ToProtobuf' >> Map( + lambda x: pubsub.PubsubMessage(x, {})._to_proto_str()) + pcoll.element_type = bytes + + args = { + 'topic': _encode_str(self.topic), + } + + if self.id_label is not None: + args['id_label'] = _encode_str(self.id_label) + + if self.timestamp_attribute is not None: + args['timestamp_attribute'] = _encode_str(self.timestamp_attribute) + + payload = ExternalConfigurationPayload(configuration=args) + return pcoll.apply( + ExternalTransform( + self._urn, + payload.SerializeToString(), + self.expansion_service)) + + +def _encode_str(str_obj): + encoded_str = str_obj.encode('utf-8') + coder = LengthPrefixCoder(BytesCoder()) + coder_urns = ['beam:coder:bytes:v1'] + return ConfigValue( + coder_urn=coder_urns, + payload=coder.encode(encoded_str)) + + +def _encode_bool(bool_obj): + coder = VarIntCoder() + coder_urns = ['beam:coder:varint:v1'] + return ConfigValue( + coder_urn=coder_urns, + payload=coder.encode(int(bool_obj))) From 61814dd90bd9c01157802d2d7ff527bee15058fd Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Thu, 26 Sep 2019 16:57:55 -0700 Subject: [PATCH 02/57] Adapt io.external.pubsub reader and writer to new API --- .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 52 +++--- .../io/gcp/pubsub/PubsubIOExternalTest.java | 21 +-- .../apache_beam/io/external/gcp/pubsub.py | 157 ++++++++---------- .../apache_beam/transforms/external_test.py | 12 ++ .../transforms/external_test_py3.py | 4 + .../transforms/external_test_py37.py | 2 + 6 files changed, 113 insertions(+), 135 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index a919f44ff48f7..5fddeb1459208 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -72,7 +72,6 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; @@ -745,22 +744,19 @@ abstract static class Builder @Override public PTransform> buildExternal(External.Configuration config) { if (config.topic != null) { - StaticValueProvider topic = StaticValueProvider.of(utf8String(config.topic)); + StaticValueProvider topic = StaticValueProvider.of(config.topic); setTopicProvider(NestedValueProvider.of(topic, new TopicTranslator())); } if (config.subscription != null) { - StaticValueProvider subscription = - StaticValueProvider.of(utf8String(config.subscription)); + StaticValueProvider subscription = StaticValueProvider.of(config.subscription); setSubscriptionProvider( NestedValueProvider.of(subscription, new SubscriptionTranslator())); } if (config.idAttribute != null) { - String idAttribute = utf8String(config.idAttribute); - setIdAttribute(idAttribute); + setIdAttribute(config.idAttribute); } if (config.timestampAttribute != null) { - String timestampAttribute = utf8String(config.timestampAttribute); - setTimestampAttribute(timestampAttribute); + setTimestampAttribute(config.timestampAttribute); } setPubsubClientFactory(FACTORY); setNeedsAttributes(config.needsAttributes); @@ -795,25 +791,25 @@ public Map> knownBuilders() { public static class Configuration { // All byte arrays are UTF-8 encoded strings - @Nullable private byte[] topic; - @Nullable private byte[] subscription; - @Nullable private byte[] idAttribute; - @Nullable private byte[] timestampAttribute; + @Nullable private String topic; + @Nullable private String subscription; + @Nullable private String idAttribute; + @Nullable private String timestampAttribute; private boolean needsAttributes; - public void setTopic(@Nullable byte[] topic) { + public void setTopic(@Nullable String topic) { this.topic = topic; } - public void setSubscription(@Nullable byte[] subscription) { + public void setSubscription(@Nullable String subscription) { this.subscription = subscription; } - public void setIdLabel(@Nullable byte[] idAttribute) { + public void setIdLabel(@Nullable String idAttribute) { this.idAttribute = idAttribute; } - public void setTimestampAttribute(@Nullable byte[] timestampAttribute) { + public void setTimestampAttribute(@Nullable String timestampAttribute) { this.timestampAttribute = timestampAttribute; } @@ -1064,16 +1060,14 @@ abstract static class Builder @Override public PTransform, PDone> buildExternal(External.Configuration config) { if (config.topic != null) { - StaticValueProvider topic = StaticValueProvider.of(utf8String(config.topic)); + StaticValueProvider topic = StaticValueProvider.of(config.topic); setTopicProvider(NestedValueProvider.of(topic, new TopicTranslator())); } if (config.idAttribute != null) { - String idAttribute = utf8String(config.idAttribute); - setIdAttribute(idAttribute); + setIdAttribute(config.idAttribute); } if (config.timestampAttribute != null) { - String timestampAttribute = utf8String(config.timestampAttribute); - setTimestampAttribute(timestampAttribute); + setTimestampAttribute(config.timestampAttribute); } SimpleFunction parseFn = (SimpleFunction) new FormatPayloadFromPubsubMessageProto(); @@ -1098,19 +1092,19 @@ public Map> knownBuilders() { public static class Configuration { // All byte arrays are UTF-8 encoded strings - private byte[] topic; - @Nullable private byte[] idAttribute; - @Nullable private byte[] timestampAttribute; + private String topic; + @Nullable private String idAttribute; + @Nullable private String timestampAttribute; - public void setTopic(byte[] topic) { + public void setTopic(String topic) { this.topic = topic; } - public void setIdLabel(@Nullable byte[] idAttribute) { + public void setIdLabel(@Nullable String idAttribute) { this.idAttribute = idAttribute; } - public void setTimestampAttribute(@Nullable byte[] timestampAttribute) { + public void setTimestampAttribute(@Nullable String timestampAttribute) { this.timestampAttribute = timestampAttribute; } } @@ -1416,8 +1410,4 @@ public PubsubMessage apply(PubsubMessage input) { return input; } } - - private static String utf8String(byte[] bytes) { - return new String(bytes, Charsets.UTF_8); - } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java index 00ede35482c94..ea43ec0522efb 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java @@ -30,7 +30,7 @@ import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.core.construction.expansion.ExpansionService; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; @@ -38,8 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.junit.Test; @@ -61,19 +59,19 @@ public void testConstructPubsubRead() throws Exception { .putConfiguration( "topic", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:bytes:v1") + .addCoderUrn("beam:coder:string_utf8:v1") .setPayload(ByteString.copyFrom(encodeString(topic))) .build()) .putConfiguration( "id_label", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:bytes:v1") + .addCoderUrn("beam:coder:string_utf8:v1") .setPayload(ByteString.copyFrom(encodeString(idAttribute))) .build()) .putConfiguration( "with_attributes", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:varint:v1") + .addCoderUrn("beam:coder:string_utf8:v1") .setPayload(ByteString.copyFrom(encodeVarLong(needsAttributes))) .build()) .build(); @@ -132,13 +130,13 @@ public void testConstructPubsubWrite() throws Exception { .putConfiguration( "topic", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:bytes:v1") + .addCoderUrn("beam:coder:string_utf8:v1") .setPayload(ByteString.copyFrom(encodeString(topic))) .build()) .putConfiguration( "id_label", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:bytes:v1") + .addCoderUrn("beam:coder:string_utf8:v1") .setPayload(ByteString.copyFrom(encodeString(idAttribute))) .build()) .build(); @@ -208,7 +206,7 @@ public void testConstructPubsubWrite() throws Exception { private static byte[] encodeString(String str) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ByteArrayCoder.of().encode(utf8Bytes(str), baos); + StringUtf8Coder.of().encode(str, baos); return baos.toByteArray(); } @@ -225,11 +223,6 @@ private static byte[] encodeVarLong(Long value) throws IOException { return String.valueOf(value); } - private static byte[] utf8Bytes(String str) { - Preconditions.checkNotNull(str, "String must not be null."); - return str.getBytes(Charsets.UTF_8); - } - private static class TestStreamObserver implements StreamObserver { private T result; diff --git a/sdks/python/apache_beam/io/external/gcp/pubsub.py b/sdks/python/apache_beam/io/external/gcp/pubsub.py index 4ff954419b2ee..964a646b623e5 100644 --- a/sdks/python/apache_beam/io/external/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/external/gcp/pubsub.py @@ -17,26 +17,35 @@ from __future__ import absolute_import -from apache_beam import ExternalTransform -from apache_beam import pvalue -from apache_beam.coders import BytesCoder -from apache_beam.coders import VarIntCoder -from apache_beam.coders.coders import LengthPrefixCoder -from apache_beam.io.gcp import pubsub -from apache_beam.portability.api.external_transforms_pb2 import ConfigValue -from apache_beam.portability.api.external_transforms_pb2 import ExternalConfigurationPayload -from apache_beam.transforms import Map -from apache_beam.transforms import ptransform +import typing +from past.builtins import unicode -class ReadFromPubSub(ptransform.PTransform): +from apache_beam.io.gcp import pubsub +from apache_beam.transforms import Map +from apache_beam.transforms.external import ExternalTransform +from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder + +ReadFromPubsubSchema = typing.NamedTuple( + 'ReadFromPubsubSchema', + [ + ('topic', typing.Optional[unicode]), + ('subscription', typing.Optional[unicode]), + ('id_label', typing.Optional[unicode]), + ('with_attributes', bool), + ('timestamp_attribute', typing.Optional[unicode]), + ] +) + + +class ReadFromPubSub(ExternalTransform): """An external ``PTransform`` for reading from Cloud Pub/Sub.""" - _urn = 'beam:external:java:pubsub:read:v1' + URN = 'beam:external:java:pubsub:read:v1' def __init__(self, topic=None, subscription=None, id_label=None, with_attributes=False, timestamp_attribute=None, - expansion_service='localhost:8097'): + expansion_service=None): """Initializes ``ReadFromPubSub``. Args: @@ -69,40 +78,23 @@ def __init__(self, topic=None, subscription=None, id_label=None, timestamp is optional, and digits beyond the first three (i.e., time units smaller than milliseconds) may be ignored. """ - super(ReadFromPubSub, self).__init__() - self.topic = topic - self.subscription = subscription - self.id_label = id_label + super(ReadFromPubSub, self).__init__( + self.URN, + NamedTupleBasedPayloadBuilder( + ReadFromPubsubSchema( + topic=topic, + subscription=subscription, + id_label=id_label, + with_attributes=with_attributes, + timestamp_attribute=timestamp_attribute, + ) + ), + expansion_service + ) self.with_attributes = with_attributes - self.timestamp_attribute = timestamp_attribute - self.expansion_service = expansion_service - - def expand(self, pbegin): - if not isinstance(pbegin, pvalue.PBegin): - raise Exception("ReadFromPubSub must be a root transform") - - args = { - 'with_attributes': _encode_bool(self.with_attributes), - } - if self.topic is not None: - args['topic'] = _encode_str(self.topic) - - if self.subscription is not None: - args['subscription'] = _encode_str(self.subscription) - - if self.id_label is not None: - args['id_label'] = _encode_str(self.id_label) - - if self.timestamp_attribute is not None: - args['timestamp_attribute'] = _encode_str(self.timestamp_attribute) - - payload = ExternalConfigurationPayload(configuration=args) - pcoll = pbegin.apply( - ExternalTransform( - self._urn, - payload.SerializeToString(), - self.expansion_service)) + def expand(self, pcoll): + pcoll = super(ReadFromPubSub, self).expand(pcoll) if self.with_attributes: pcoll = pcoll | Map(pubsub.PubsubMessage._from_proto_str) pcoll.element_type = pubsub.PubsubMessage @@ -111,13 +103,25 @@ def expand(self, pbegin): return pcoll -class WriteToPubSub(ptransform.PTransform): +WriteToPubsubSchema = typing.NamedTuple( + 'WriteToPubsubSchema', + [ + ('topic', unicode), + ('id_label', typing.Optional[unicode]), + # this is not implemented yet on the Java side: + # ('with_attributes', bool), + ('timestamp_attribute', typing.Optional[unicode]), + ] +) + + +class WriteToPubSub(beam.PTransform): """An external ``PTransform`` for writing messages to Cloud Pub/Sub.""" - _urn = 'beam:external:java:pubsub:write:v1' + URN = 'beam:external:java:pubsub:write:v1' def __init__(self, topic, with_attributes=False, id_label=None, - timestamp_attribute=None, expansion_service='localhost:8097'): + timestamp_attribute=None, expansion_service=None): """Initializes ``WriteToPubSub``. Args: @@ -132,52 +136,25 @@ def __init__(self, topic, with_attributes=False, id_label=None, timestamp_attribute: 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. """ - super(WriteToPubSub, self).__init__() - self.topic = topic + super(WriteToPubSub, self).__init__( + self.URN, + NamedTupleBasedPayloadBuilder( + WriteToPubsubSchema( + topic=topic, + id_label=id_label, + # with_attributes=with_attributes, + timestamp_attribute=timestamp_attribute, + ) + ), + expansion_service + ) self.with_attributes = with_attributes - self.id_label = id_label - self.timestamp_attribute = timestamp_attribute - self.expansion_service = expansion_service def expand(self, pvalue): - if self.with_attributes: - pcoll = pvalue | 'ToProtobuf' >> Map(pubsub.WriteToPubSub.to_proto_str) + pcoll = pvalue | 'ToProto' >> Map(pubsub.WriteToPubSub.to_proto_str) else: - pcoll = pvalue | 'ToProtobuf' >> Map( + pcoll = pvalue | 'ToProto' >> Map( lambda x: pubsub.PubsubMessage(x, {})._to_proto_str()) pcoll.element_type = bytes - - args = { - 'topic': _encode_str(self.topic), - } - - if self.id_label is not None: - args['id_label'] = _encode_str(self.id_label) - - if self.timestamp_attribute is not None: - args['timestamp_attribute'] = _encode_str(self.timestamp_attribute) - - payload = ExternalConfigurationPayload(configuration=args) - return pcoll.apply( - ExternalTransform( - self._urn, - payload.SerializeToString(), - self.expansion_service)) - - -def _encode_str(str_obj): - encoded_str = str_obj.encode('utf-8') - coder = LengthPrefixCoder(BytesCoder()) - coder_urns = ['beam:coder:bytes:v1'] - return ConfigValue( - coder_urn=coder_urns, - payload=coder.encode(encoded_str)) - - -def _encode_bool(bool_obj): - coder = VarIntCoder() - coder_urns = ['beam:coder:varint:v1'] - return ConfigValue( - coder_urn=coder_urns, - payload=coder.encode(int(bool_obj))) + return super(WriteToPubSub, self).expand(pcoll) diff --git a/sdks/python/apache_beam/transforms/external_test.py b/sdks/python/apache_beam/transforms/external_test.py index ba315f98dfb50..fe269770c376a 100644 --- a/sdks/python/apache_beam/transforms/external_test.py +++ b/sdks/python/apache_beam/transforms/external_test.py @@ -34,6 +34,7 @@ import apache_beam as beam from apache_beam import Pipeline +from apache_beam.coders import BooleanCoder from apache_beam.coders import FloatCoder from apache_beam.coders import IterableCoder from apache_beam.coders import StrUtf8Coder @@ -66,6 +67,7 @@ def get_payload(args): class PayloadBase(object): values = { 'integer_example': 1, + 'boolean': True, 'string_example': u'thing', 'list_of_strings': [u'foo', u'bar'], 'optional_kv': (u'key', 1.1), @@ -74,6 +76,7 @@ class PayloadBase(object): bytes_values = { 'integer_example': 1, + 'boolean': True, 'string_example': 'thing', 'list_of_strings': ['foo', 'bar'], 'optional_kv': ('key', 1.1), @@ -85,6 +88,10 @@ class PayloadBase(object): coder_urn=['beam:coder:varint:v1'], payload=VarIntCoder() .get_impl().encode_nested(values['integer_example'])), + 'boolean': ConfigValue( + coder_urn=['beam:coder:bool:v1'], + payload=BooleanCoder() + .get_impl().encode_nested(values['boolean'])), 'string_example': ConfigValue( coder_urn=['beam:coder:string_utf8:v1'], payload=StrUtf8Coder() @@ -151,6 +158,7 @@ def get_payload_from_typing_hints(self, values): 'TestSchema', [ ('integer_example', int), + ('boolean', bool), ('string_example', unicode), ('list_of_strings', typing.List[unicode]), ('optional_kv', typing.Optional[typing.Tuple[unicode, float]]), @@ -188,6 +196,10 @@ def test_implicit_payload_builder_with_bytes(self): coder_urn=['beam:coder:varint:v1'], payload=VarIntCoder() .get_impl().encode_nested(values['integer_example'])), + 'boolean': ConfigValue( + coder_urn=['beam:coder:bool:v1'], + payload=BooleanCoder() + .get_impl().encode_nested(values['boolean'])), 'string_example': ConfigValue( coder_urn=['beam:coder:bytes:v1'], payload=StrUtf8Coder() diff --git a/sdks/python/apache_beam/transforms/external_test_py3.py b/sdks/python/apache_beam/transforms/external_test_py3.py index 88fa870f17b58..980ad8027e60d 100644 --- a/sdks/python/apache_beam/transforms/external_test_py3.py +++ b/sdks/python/apache_beam/transforms/external_test_py3.py @@ -43,6 +43,7 @@ class AnnotatedTransform(beam.ExternalTransform): def __init__(self, integer_example: int, + boolean: bool, string_example: str, list_of_strings: typing.List[str], optional_kv: typing.Optional[typing.Tuple[str, float]] = None, @@ -53,6 +54,7 @@ def __init__(self, AnnotationBasedPayloadBuilder( self, integer_example=integer_example, + boolean=boolean, string_example=string_example, list_of_strings=list_of_strings, optional_kv=optional_kv, @@ -69,6 +71,7 @@ class AnnotatedTransform(beam.ExternalTransform): def __init__(self, integer_example: int, + boolean: bool, string_example: str, list_of_strings: typehints.List[str], optional_kv: typehints.Optional[typehints.KV[str, float]] = None, @@ -79,6 +82,7 @@ def __init__(self, AnnotationBasedPayloadBuilder( self, integer_example=integer_example, + boolean=boolean, string_example=string_example, list_of_strings=list_of_strings, optional_kv=optional_kv, diff --git a/sdks/python/apache_beam/transforms/external_test_py37.py b/sdks/python/apache_beam/transforms/external_test_py37.py index ad1ff72f0cfe6..1a3cc2dfa7ee0 100644 --- a/sdks/python/apache_beam/transforms/external_test_py37.py +++ b/sdks/python/apache_beam/transforms/external_test_py37.py @@ -44,6 +44,7 @@ class DataclassTransform(beam.ExternalTransform): URN = 'beam:external:fakeurn:v1' integer_example: int + boolean: bool string_example: str list_of_strings: typing.List[str] optional_kv: typing.Optional[typing.Tuple[str, float]] = None @@ -59,6 +60,7 @@ class DataclassTransform(beam.ExternalTransform): URN = 'beam:external:fakeurn:v1' integer_example: int + boolean: bool string_example: str list_of_strings: typehints.List[str] optional_kv: typehints.Optional[typehints.KV[str, float]] = None From 6dff94d8de9021e3c221da09482ed5d606ec4b52 Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Tue, 1 Oct 2019 14:43:47 -0700 Subject: [PATCH 03/57] Adjust the way that the new API is used to handle the encoded data Additional transforms must be added to convert to and from PubsubMessage proto objects --- .../apache_beam/io/external/gcp/pubsub.py | 64 +++++++++---------- 1 file changed, 31 insertions(+), 33 deletions(-) diff --git a/sdks/python/apache_beam/io/external/gcp/pubsub.py b/sdks/python/apache_beam/io/external/gcp/pubsub.py index 964a646b623e5..5d0683b243b8c 100644 --- a/sdks/python/apache_beam/io/external/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/external/gcp/pubsub.py @@ -21,6 +21,7 @@ from past.builtins import unicode +import apache_beam as beam from apache_beam.io.gcp import pubsub from apache_beam.transforms import Map from apache_beam.transforms.external import ExternalTransform @@ -38,7 +39,7 @@ ) -class ReadFromPubSub(ExternalTransform): +class ReadFromPubSub(beam.PTransform): """An external ``PTransform`` for reading from Cloud Pub/Sub.""" URN = 'beam:external:java:pubsub:read:v1' @@ -78,25 +79,22 @@ def __init__(self, topic=None, subscription=None, id_label=None, timestamp is optional, and digits beyond the first three (i.e., time units smaller than milliseconds) may be ignored. """ - super(ReadFromPubSub, self).__init__( - self.URN, - NamedTupleBasedPayloadBuilder( - ReadFromPubsubSchema( - topic=topic, - subscription=subscription, - id_label=id_label, - with_attributes=with_attributes, - timestamp_attribute=timestamp_attribute, - ) - ), - expansion_service - ) - self.with_attributes = with_attributes - - def expand(self, pcoll): - pcoll = super(ReadFromPubSub, self).expand(pcoll) - if self.with_attributes: - pcoll = pcoll | Map(pubsub.PubsubMessage._from_proto_str) + self.params = ReadFromPubsubSchema( + topic=topic, + subscription=subscription, + id_label=id_label, + with_attributes=with_attributes, + timestamp_attribute=timestamp_attribute) + self.expansion_service = expansion_service + + def expand(self, pbegin): + pcoll = pbegin.apply( + ExternalTransform( + self.URN, NamedTupleBasedPayloadBuilder(self.params), + self.expansion_service)) + + if self.params.with_attributes: + pcoll = pcoll | 'FromProto' >> Map(pubsub.PubsubMessage._from_proto_str) pcoll.element_type = pubsub.PubsubMessage else: pcoll.element_type = bytes @@ -136,18 +134,12 @@ def __init__(self, topic, with_attributes=False, id_label=None, timestamp_attribute: 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. """ - super(WriteToPubSub, self).__init__( - self.URN, - NamedTupleBasedPayloadBuilder( - WriteToPubsubSchema( - topic=topic, - id_label=id_label, - # with_attributes=with_attributes, - timestamp_attribute=timestamp_attribute, - ) - ), - expansion_service - ) + self.params = WriteToPubsubSchema( + topic=topic, + id_label=id_label, + # with_attributes=with_attributes, + timestamp_attribute=timestamp_attribute) + self.expansion_service = expansion_service self.with_attributes = with_attributes def expand(self, pvalue): @@ -157,4 +149,10 @@ def expand(self, pvalue): pcoll = pvalue | 'ToProto' >> Map( lambda x: pubsub.PubsubMessage(x, {})._to_proto_str()) pcoll.element_type = bytes - return super(WriteToPubSub, self).expand(pcoll) + + return pcoll.apply( + ExternalTransform( + self.URN, + NamedTupleBasedPayloadBuilder(self.params), + self.expansion_service) + ) From 65e6d7ba725748b8b2039ecd23a67567ec80a307 Mon Sep 17 00:00:00 2001 From: Leonardo Alves Miguel Date: Fri, 11 Oct 2019 09:53:11 -0300 Subject: [PATCH 04/57] [BEAM-7926] Update dependencies in Java Katas Updates: Gradle: 4.8 -> 5.0 (and fix deprecated build-in task overwritten declaration) org.apache.beam:beam-sdks-java-core: 2.13.0 -> 2.16.0 org.apache.beam:beam-runners-direct-java: 2.13.0 -> 2.16.0 com.google.guava:guava: 27.1-jre -> 28.1-jre joda-time:joda-time: 2.10.3 -> 2.10.4 org.slf4j:slf4j-api: 1.7.26 -> 1.7.28 org.apache.logging.log4j:log4j-slf4j-impl: 2.11.2 -> 2.12.1 org.assertj:assertj-core: 3.12.2 -> 3.13.2 org.hamcrest:hamcrest-core: 1.3 -> 2.1 org.hamcrest:hamcrest-library: 1.3 -> 2.1 --- learning/katas/java/build.gradle | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/learning/katas/java/build.gradle b/learning/katas/java/build.gradle index 4c962ec2f4985..23970726d6410 100644 --- a/learning/katas/java/build.gradle +++ b/learning/katas/java/build.gradle @@ -18,14 +18,14 @@ buildscript { ext { - beamVersion = '2.13.0' - guavaVersion = '27.1-jre' - jodaTimeVersion = '2.10.3' - slf4jVersion = '1.7.26' - log4jSlf4jImpl = '2.11.2' - - assertjVersion = '3.12.2' - hamcrestVersion = '1.3' + beamVersion = '2.16.0' + guavaVersion = '28.1-jre' + jodaTimeVersion = '2.10.4' + slf4jVersion = '1.7.28' + log4jSlf4jImpl = '2.12.1' + + assertjVersion = '3.13.2' + hamcrestVersion = '2.1' junitVersion = '4.12' } @@ -113,6 +113,6 @@ configure(subprojects.findAll { it.name != 'util' }) { } } -task wrapper(type: Wrapper) { - gradleVersion = '4.8' +wrapper { + gradleVersion = '5.0' } From 64263a5328b2d3ab7ce3ac4b0016b5e30c152f98 Mon Sep 17 00:00:00 2001 From: Jeff Klukas Date: Mon, 14 Oct 2019 09:54:25 -0400 Subject: [PATCH 05/57] [BEAM-8393] Fix Java BigQueryIO clustering support for multiple partitions Support for writing to clustered tables in BigQuery was added in 2.15, which involved adding a new optional clustering field to TableDestination. Clustering support is working for most cases, but fails with errors about incompatible partitioning specifications for any data that is handled by the MultiplePartitions branch of BigQueryIO logic. This PR fixes the bug by passing through the clustering specification in MatchTableDynamicDestinations. --- .../beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index b58d18d547121..a484a428c3c63 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -385,7 +385,8 @@ public TableDestination getTable(DestinationT destination) { return new TableDestination( wrappedDestination.getTableSpec(), existingTable.getDescription(), - existingTable.getTimePartitioning()); + existingTable.getTimePartitioning(), + existingTable.getClustering()); } } From ba15657ee86d3acc2cb4597446e428e36dc9d030 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 15 Oct 2019 17:43:57 +0200 Subject: [PATCH 06/57] Add DoFn lifecycle sequence diagram to website --- website/src/documentation/programming-guide.md | 8 ++++++++ website/src/images/dofn-sequence-diagram.png | Bin 0 -> 126448 bytes 2 files changed, 8 insertions(+) create mode 100644 website/src/images/dofn-sequence-diagram.png diff --git a/website/src/documentation/programming-guide.md b/website/src/documentation/programming-guide.md index 0c8a2c19b95fb..59a322b19bfb3 100644 --- a/website/src/documentation/programming-guide.md +++ b/website/src/documentation/programming-guide.md @@ -802,6 +802,10 @@ words = ... > **Note:** You can use Java 8 lambda functions with several other Beam > transforms, including `Filter`, `FlatMapElements`, and `Partition`. +##### 4.2.1.4. DoFn lifecycle {#dofn} +![This is a sequence diagram that shows the lifecycle of the DoFn]( + {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) + #### 4.2.2. GroupByKey {#groupbykey} `GroupByKey` is a Beam transform for processing collections of key/value pairs. @@ -3099,3 +3103,7 @@ public class MyMetricsDoFn extends DoFn { } } ``` +## 10. Focus on DoFn {#dofn} +![This is a sequence diagram that shows the lifecycle of the DoFn]( + {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) + diff --git a/website/src/images/dofn-sequence-diagram.png b/website/src/images/dofn-sequence-diagram.png new file mode 100644 index 0000000000000000000000000000000000000000..cf6570d00c730e3360b2149d8d0b85bec0ebd7b0 GIT binary patch literal 126448 zcmd42bx>SQ_%%of1b252?hxGF-Q5WqAhdwr)-97!-Ip=APCy{Q@S8HH9~O&K@*V|pWHC`B5{P951!=G{Hij=G$jk|Ew)e6tR? zyk3|AtJ3K_nC^4nE!kMKiD11Zagv2GNxI}Y_UF@Ge#3oA230E4?+-*n>fM_4CNa;$ zFB$UB;Qnn95+Awe=G>fkN?wT5%Z$pVvG?>m{lU(chyFI0szcuVUF)dpcLi`&JM^7(QEt7d&BWMjA8 zzK$s0IapZdnufaBu`zDAQ)mR9tiRCv-8&Q0@MVXIzaXD+=KzrmM2K|9yNp0wZemT; z1`q^sua5@Q-&J1bHgAoB;nY8$XW&+?Go_R+l<<|mujqaX)iP}DAV`m(yGJ}~Up<|C zA7mSgW4)oX_3k8Zkkma~!GoMghqQHNhbG?Zr5``zVI1ARYv1C84c`@g+F(h)B;j|$ z%M&~YT0Z{p$5A+`$*ybL){z704JuD%+$`S~Uf9KT%A|`PNa;ahMJAFQNpk3Y6UKoG z;%jvr-Wht+yQMb9XndkmNC|W~&z~DtZSOmDz1u(ibqT3jtEPlB%69S2bwbK-PEE7n z4Tqd)c4C2(s7evuu|mk(sF3J_=CV}2>;gx%*iBUE8|A?Z_|42Tj_7i+Gsb^8aUFjj zY(rX%^t7M@don{zJg(JJ;ep~~G&<^6zW7BGivb;+GhatM>boSKF?j4A%QBW7GAcC* zw6{K#`DHh2m3(qgD-uZRp;i047FQm}eYOdAi5`Q(R zZ2RgnB#O9+SBXmFW{O03#)K>M)1Q@H3bo*VCop&qodaVHByu%%;Oa&lT5mS;KiGo51fy!U;1}cenX?bFh>dh_3b(fFg(H%C`AlgxDbD|L zp~IK5Xbz31|H3(P)SCYZ8@=rem3rsOIynMmFon^c!HqiNY&@0++pS2Z>qWP@A3=qK z{hiu3kdP6KkiyI>ig{x04l&XCfxr0e^6VT~mgE!KSkft$NY zyOl!+R=Dyo$D0Ul;`AE~h4n}yp^)Jq`EGA^5#?`#O6WdA6L`gjV0lj@^WCa0#xbGl z!7GsVl7FhDETvwR{+^FEKug#^a>A#&R8nVkf6!*fzSUZfbpJ(U(gLqL8rmfD_pFet zLcFoe{0#{z^8!wQ2^td#l1j;!b|!iHGuu(2h8+WWJ{Rmw19=7`v9u30>n?U<<}f0E zduivNNCZSwu{t-~vT4V7`er+BUe+abJ|?V~nR^B#gQii2c(6ZDUGLi?T=N)b@QN@8 z$S&ZPL=qVbveEtk?AMEBEXJ<$B}uB;DRKL7h{qX!;$B1t1kyCc5Sm9;k$b-Td8}6Y zbw;eRmMxZIBDXG@t{Ag-=_22@1l0hEeRNqH#`4sTW+g27)OvJtgB{sP6>c$bQShsF$AKco+Opd4F^(A zZw7>z&Fs^Y8IDqfM2krb(-sIgClng2=0AKB3AYOMi&uldfDxJ&DNYedTA+kM2}W&h zc)G2+WvI{Yy&dq}k<%(u3CSeX69qQ={%nQr<2aVZ1j3=A z0HdBd@wO)!lnkqhMscY$`nq*5zTxu#^xn?6d)u9{w(p4Dg)VD9JN>j^8-ANx^$+g> zrLSM8un#61cEnq>zkiS@ET)N>v=S&{hGxf=@ZGuiCMj56-PSOWYp%<@V4y$NKiL4m zMbq-J)RNCle@^aJPPF9O#n*=J3AK3D#(H{rg5!rV=H7JX1KOA-Del-F0cmI~57*NK z$(RCy*r%opl@9_55uZ#Hbc?Zbv060)_Id;PA-2Smg6y+lv$3H+vx=a(kuQJsP!q*B zJ^@?IUQTk?di|6%HqF65jH;$$vm<)J?$HXXIZ7x`4Py*ghqRd}9qi}i6XW5rAR z*{73Mdn9ui#guxJV{|Wk2>qe3?RbOlG5%f&!c8>1f!lu4j0;w|X4BRI?+qo~o~^m8>J1m|GOReY3^5#l5aJo0(man#H@#L5-?j z5qUj&xu&qs-fUI$I8D@Mwq}@{iENpXAh~mzYDC_=r?O+ z4ev3s=lZYzmbG2|OVmH~ben*oCqMDasy0cjA-px}&|dPc00+b+U0T88F%=}@j`HWb zf;c>zH#=&krJi3Eh{qfOFX3s-!fLqq26`Y!(0aA^|&7N>1_qZ!mCqA{9ic-Z6sU!Dyd#??;F4vU@ZZnVWfaryTVf9Cfbw0*P2pzBGP{z*iyzP&%{@FZbK4!^GJ5McHpD}V?|x*_HzKVz8Wv1?Lb)|w ze>Bu(4cY!_{A=s1-b`hTb&d68JZH4C@)zu8Pr>6J8FbVkX=TX$p1u}UzuDnokAfi8 z1Xef(k%`ExIWc#$*^N1A`Um_awz9vmR0F8w_&7_cbi4jNSSs8N$CcNK0`nS0=TQ1Y zcD>aQH^7ceS`6nxZ?Hx*qdGxrl|`0t910Rr7Ry5Y?4Gc49in#db#p^Y`EZ8TFba~; z`c!aH>LNmkV05Em+Wl9=s0tZO)=H3z#Ju62UwQmS6nf6V9Bh1Df$dmt-DwV*$B7fF0Ta5U)7Y7 zOGgX=9**LWg~6)E%qy(A=vk;>y6Wa(rdUtFzqFbFb(Hgq8d+iWK&Soj_oGnCtf#Eg z2@UzJQGC+%fyO$t^$&}UwzR)O7EQxj4tAxLgj&3g`o6eV(DXmQ)f=U;;;x<|sG;tP z?faWRQt=-z-(d1_XL|lI?^fgHn?|`xhPf<(r-(~bXe~B~Y~Q_rrKi9e3u5!PCXcVX zkzI!RY-WH{K}v9|NT`q}v#a2#GM^C}_KlSMhjxeMcjR(}R_6gaG6J`SPZqtEI z-Nx(^QBi#~8GU_`MtnsgWpfx71T0!61XjPF`s1wfX0g+!+Vb&FE8ILk!l2wdhfr^u zf@i~EN(L#5l4#gYTT`Pk(`D?|u#&UV{fT0WQS=sxKHtlO@{J5KKT~iW zI3ZE{8pNy`KN*(@=))SovYk*q2y*qt-?S%J9o3+k=gWRy!14a*D*?9f@-@L!T^mhC z?dRlIC;f!kB~@W;7+7D6`E?vU)X^cw&thwHy2a6KmrGtg!?@satgiX)1)N1*7`5?j zXBiPx;_H93uW7;B+vVW-zDYLw_LC~tIw6af$W&r>nb};TJ?_^kG6t_5?}eep%G&zy z*fSnzd~mwL-Dz$*{UQr0Z4Qprg>1c{qA^kTc-kW1GyIM(IEKw%!L_9=UNZ~9`2J_BERUhHDE%H_**6?fot95!|9P{>*wU!g_#TWNL`RxxlfC zl}y%K@M86%yQl?!=ThU+i)9$Cf(dSnze+da1>!tSwKawj#KFH{@O}}@FCbg;LgvqJ zN@R7u@S;z$=+u%bYC5yk3&}!X1w+4nbfu)&&66K(9|@6$E)xxqw824C_HEjn#OUs=%xWut|0#zypkaN3~!xE*=NJXVFwUZlA8`=8MYKF5LA4{o(RK0z$PU{m22vK|LwjJ(mS8 z)T&E5>h#BH%82l}1I1Oa-doPHYH;T9MKYA^QkQ~w(B<7>7PFu}I^Sv7d(XklY*i(x zJ%ii(nP6jO?r>WU>u{8*1!Iy#%MRxn-352Y{ST>WlPc59T=oy>TC-3Bb4u_#1ut2q zO?Iv|9N_t@GZ+@7%AGp1R@CLR7n^7E_@aoitMUtT5oISom_$!4Q0u=BWd%*;pClzU zZArU!s<&;A$%>qbhRl6cG^Q72oqm^*K%7PzI zaM{*B9NA7kpfai5)u_=SSoGaW7Mmcr8x_fAe(*rBUjX}wJs$ikeL8JP)aR!(n@HBw z6;iq!Yy&Lw$r^lmYErDFq&O+7S3^Op6f6Z%2`q|k>{Sbeyj}B04*>9~L z&Nfu99eSEtFbkx-`%)*XD_X3hf272&-iljDo$l!ZlzWrSX6X3Wt2U4J14Gr;(@xFu%@*CS zxWd0=x@hdyZzJC)!J2N`Vp3A7ZEUU;x)YM0(yfl7*>_N~=u%^{h;)n%`i5*j85?a1 z?5&4I;A0z5&=E$BbCaQ}tO`sQQBZvSG>-}K5Re?kZWb21x$$^6eWE>g>cOs%5%+V% zmP>Vy{-Ged^dtI^bSp6~J;r^7&#!5gGqW*ee&S67hSQPsNcahet{bpCR~5 z)%QC0FWLQ8tcA7R$?i&GnRV3cV_T2r?n9oUpM9p(?lKusH3RQdb_U2M5IyO-Ir{Ng zIzS`OxP)Xkxa+6r$DdwFjHS5a^Gb>h0Z|5p@sCY4Vu|b;7SPiRgnV%%<({Gow2?Kl zqn9;y#$HFnnel#W*2zY#1GU{cW4%3h-&@p=bhf*t-N_7&A|%jP2x$@#{V)5= zR!i;Mm$7tn)L6g^DB%*p3E0$~LZXk;Ae<373LTS-k`mp`XuW0dN|I+kfQ+l9`lcZd zbv6{LewCsM^kp+EV(-}^fiKM%zdupo&Jub27aFU>igr&JN)7VTNzRIxvv$r zKfVhOQlXm?G46j-5B+Ve?!S!NuvnA_hFpoXeTR?l?%)OK`AQhw=62bVcR^b}N~qyUifR>2Jse_BM<^QBc)~&pOi0d@4>*SS zf0rbdbI*&FOk}u=<~7N!g)9w%2Xn4LcucoEdC#@c+z@E3P9P5!DT%@_(88vs0AWl3!p>Y;ede=gGq@Bsk-SOes|VYoqOmL2rzby zXscKvux^}PI+{PiJ)oIg`sTmnAGNd;PH-^4WE}9QKYGS-(MIfw{RP|bcG36u;7}9g z@@U$f$MQv{6WQJU{e1uyOf1;zYZlsH(_y3IU`YoL2O(4ld>f^zwy|$jC~)=g&}JEd zP7It}@b&dckQg;M1VJ=Tg!mmQ@M1y@L#IZ4=HC$`B}dMMjvPW$iJIx@305W(7Rw5x z0PgV*lk*cMGw#d&LHYVb2AN>4krT<5-w-jeut*JS2t1B(vC#)cOp<@&RUv?*<2Im3 zLGhr8Flx?N z)!dk);YVofpb+%j$yL?2A;YL3sMtuQ^SR_-7aT_X`c5*3OI84)PB>6GFd^{YOvHJ} zIb&f|aoe0yz`Vo(ukVQcx0&Q%5u_31kzXGLeu|)=hUH?bJDT-$nb6DglaYmmg(=Ac zt3(W8)cw_g$=Q0(?o=t(dT*4ql6uWeE%3R(6p!b%4X@iN*m;avlW|XYFz~k)Bk){h z`7l2ajn5T8sgRW;|DAiksyr6EvW`+0iGVvurBHtD>R=`eh3o^1#Ykj01}&-i5D{g{ zfnE%eKyQuN;8>voky>A^#i**U@74YD6VC@k{9VfD5blKr+la4w<4;y-BrY%II*m-e z&yQda{Knm3+4EBo7ip{(v5W@%YW(CIM8?FL%rK&Yz=oqR?c%dMT3BMoetCeQg!<>+3Zh{ zV5fj{=iuO&EZ4zGV=-eXb=S(bhLkhp_>9#|#P0D`=)n3oSZ2+U@3; zz~yK@xKJTG&w8dJ44N43~zWL-K=oG=6b@v@QvbuhTt<1#r-9l$>GEHWTj2xplG*%i@sy-^Lm0TqJIVAqiHZ)6j>>a*HDOPedkLsu z>{gSCSNtBAlJyyEpOqY?k(Uw z3OKkAHz$2(8+~fYk{tn%YI;xiSAleqW{b)(An4*BU)DM~T@JDJTiyH}w+3*6BmIk+ z2nBq2o1G6r$fe@3wcrM+RSI$}|K=gbk?GQ?l>`l@Fm6nhYS1|403K1&=EuTzIUw_X zrb4ePnt*5W7L8morx%OI#YP}I>Q7LJY2ad$6Vz}9hsMn}-GOv=TIHY8!NB9RC?A>c z9`9I7G%72M9d1vJlFmq(d0XEWb*^YAbp&EgBszARTS*OtuWtPXFFRM_{u zJO6pl%Vs;TI?SvO*ld~vNEQSmHWW4}OcZQX?49amvv~hP@!cQ_6E&OZJAT`Fw#SQ6 zfiP4G*v%PVshN~W{~#DN+~D4)hnuO+TIYl55(`2?LQpb&1}cJwun1;2;9eTe{p=>a z@QE~PuuYCzu<>KDVuhFGz^;~5)<9C&FSq{svKFBIC0sTRa=T2cb~2C8VkGnS_KXbh z%HS+M&l1R*mX?D*^-G_eK+xVqT9o&^`a84_svqI(xjkKo%msx@&hiSuSsgWwTqv?g~LrF%}{B z3&SbaX%LcJcn5>fc20_I>ad(E0!862iX8SQF1Ya;R@M^qfXWd^$HIY#&zZB2zKjYN z2gc$Or_Ms@adRB?Ln6lbWVvm3vN*w*@#LH4-fZ=cb&8GKqsEm<-c%-|fOye(TT&f? z{3v`bM~K^I$sak;2u|yC%CZd_FcMAhI`X1bVV3*jKO^q(GkM)nfe=+UiNBQ@B2Ome z7Om>Xk4rrxeTQ0w`0hn`|2k@>H3H> zC)em?+sWQlgyu>i47c5N5ZBff&G__2`%u5x1c?dDR17gfWUy_3prNqHhkjXv3}|*J zp?^guhXi>yH^a@vr`;@J?l3484@hDl za5c`VK$}WVTk8rfm4(rAlW7Y% zVkV`4UtlrStue$TA=`+Bf8W)FUo)Fr{$ZY#jl4x@1e+NetjVAu6g#N5{pA^MFF}`- zKOCJpPc&Q-8U^MY3HFF0!mz(N)`|MjfHa&OPjERQ*g4~qZ7iu6teVa4Sgthmx12FF z+;>S>)ui5-Ys4vGct&Jb){PrLwR$;Xh<7%g9Uw`=jE^EQ6Kf`yNtW#~8_DF71_M!? zd_zwt$HQXOSDhmXX%KUhG8gHOr%=;!SAdWiOs1c5L5iawWfp}6Q7|L!I=?KEi-@p@ zoAO&SqEOU`^=EP(z#x2U>n;_Gz$~%q@5N@bjK!@qO$R-MlwqK7A`$V6ws~CL_K1L! z8&ZHj2lpQ0HYpZqacovBNk94inOZuYU0-~s$6G<>v3P5KHE`2{)Y|UT;Yvg#8nwrf zA*bW7F6#C0$UJuL5v5|8ih+qaj?5%sDd_-|&HrLmhZMPJELMbuQLi~4GbaL*UO5fx z?qWv+zAltp8b>R25L^Q@XR1VvwP{OqWM#I&7B5S{w_+caniTGZ#cUuI9d}+t$jnHu z)y*NPo;XG57w8k1mgr!oHjA{VC1~B{kKenJOc3%Nh8|{?V?QQFu}r>yVuD!Z8Zjsk z3aNzODmcVOBaWUZB#?umMF>%m9U5WphtWG)xCc}U>7pYHQY};KZ=m-_6rI9BVJ|}n z)MQVI*Ldt!`9XoE?vd8fIz{qZ8U-@>17vdUuU+hCk8KZ32trg+s(vBQr&a`iXJ(mr!2020AA_yoX>Cgd z-D+OV>|1KtPGtc|a{xxewLH{2v8af}ve`-`Blm3oSk4A-8bhBkn^^i%vuMnoS9$w9 zqZ(6vquWO-*{?}hMEmDt6`kj__`euN_J9D#z>V)GUSKu#r`AGr6wD{7CCMq|2X(sL zoo|6snSXH0m;f$dRqC%>!?e`uUaQS1BOdZtW6Q}QGO`-mV+G^|PpTr!3{!zfJPvs? zI#3#v{mO4O9Tsp@TvC2hoiLl2FiiUNdo?{4Hoz+wZ)$o(8!A8)<6=Rf>Xu>1W9CMZ z)Bc$psJgi%IK-X5fLzjh2={rRs$Qr@Oq5hh8b};E5URE2!?Gj1PCz+S4ZDofls*() z3>%9bLQf;AsbVr^lEe(GnqfGlIF6q(S|v~n6;svn{GTisj$eiKUIwk4?k?w*n^|VAWDkg3w%^PAKvOytpwFrKHNAa-JXPvTbmc` zjJEsOkSzzQ1*n9)H}IlQ(V`PJQ=bbibSF}Lu;&4V(TrO_M{*GtOT@qTd)yBz6W!(C zxS@Iyd2J7OV8J0S)}V_8jp{l%_c~n<@wa zmUIyUiu)rzvK2CJybt4!aw5Hs`k8jSPg^`}h!Zk7t8ll!B>dblfa+ObMe!v=g6aIW z@L}hYNp#?c<{Z}rnBs$(U5|A)Ow7$n!))|oo5?eTGjd5HWbY{S8sNjqVt-2}2b!{i za-(ZBF`7sm#Ye_skLa{>N$4;cbc=dP93`H0SY!ie*-YsidxPp|k3h*-C~Wa!BpaxI zNpvXbDA@8x6DSqbg(;QtME6(a*TWzs1p;~3Mpp!7sy-L%eYS^*vZdF}S#PKL&)8CN)=leElY;`l%#Adm{Gmjp zpS*tibfk8|{53hz)17Y?{+_GU`ULiCdIb13n|S<1Kwpnsa!#Ktj@#g@3LE#D8UhAF z4d7HkhD@65Z>WC?I+cRW)Pd9Y*{dFu@*WQ%#mJu$PM!-+$VVu}S90JQjH^%((?5QV zI6}SucdWsQ*%>9AZ`UUTtyXFGxdGCJPn%MhjvoW*e4YeoR2x3+npsL_Z~F9S#+IY2 zm8dES3)bLo+8~t!wq>Z95;6^u#EqjORV9ZLM*Ns`y`|y8H6hYV>nPNKWyz{+svB=h zY_`cHCDsGKg&uJoB7uUygbkzPL7)eWr z%TyhA?TAyfJ*9dpHRlW_#Tjbnsg%(b&O!VVC6h5H38$m6yj01#*R~#WK#p);Lxsay*meR`=e*Ov!dr zAE|7!Ea{caSQS`=Qg7F464WsY!+mN>Qsj!GIf$4wy}#Ygw8z;z;eoXh#Nw?)I(M7* z=WF!yE!kaaD8yI_IlUmredqGAUo|EOBW3CDiB9Zq85HW9eNf(yw6apOA&%k%ds-mo zo*kA8=1Z2yt6Y`sDG>p+80Ue#LejWS%URYo4kb6PQD28ba-H0nE&1-cA1>*Gyi!ew zMmH(bPg7X8Hcz}Vp@q4HuD%5x&71R&Z}1rvS`OSCCTi@^P}5ciB52J|DUK@Mh$IcY zb#;fiX7eBNn*NxBfqRn0`v*uXr#j5-3b@*HO&?g2Dx#_Wnyk;;xlVqs+b^`s(&xg9 z;Z?}HhvpCv{vq1u%T%h3Sf}rf{%zc>^Pv!lxlI}~L0BZH@BCVA%%exCTP{m`igLkF zcB-~c4)T)!QnMe4Yei%Pe6Q(*ABYSUV31J%R27bw7?xJaRV-)-%9ahR^o{a25=U;x z3^3LXL24XANBvMx)71#|1FC|IJ~ zj6)92t0_Z8&5K1xdaFQf+^$padLDA+^S#8_mS5hOg4AOlqqs?3%ZV&bKorWIs(g;W z#gD3G;)-n$bmtpj`xbk1m*yvYfK0{`obD>oAboXqb_I8c`+!A*l>Q+UySh_tXm)Ol z0uVBqALqFLg`nUmpm{(fe78Y}qx=F&hLezxsFxFdn2a@4w?=DJvYZvg=7L6rZiINW z=W25DVniG!^iO%wgF==fIMpf}HUMef_caNH6lQuoT*4qSZ%L7$#nLD!>E)7ydf5?9 zBHoPPi`V@+Q)<|u!gnNiXFx=xiqEu1nHyK`_&(`73QvEGgxw&8qu9Xy6d>X(%NW%0 z+7VtV@~CEJT9B9>lM3){i8hS3dY&tO|DjZ+rLkH5a6$H`%a&=*p%hR}s$k7O3BMJ6 za6oDi2BRq@|0nH0^MMzPj6t{QSDT?iw*X&t+S$yqzPaNB@Rn2LRq^qIun7l>nOtx} zU^VzVSOi)@bF??ZlL!Mkl4wllA$$%0OVRdJ*D(nie|n zdL6T`8IB@7%*-fsCMa88n|c`kAC|2Z@oEVtN3ps zR~0qz{3i@~#9mlnon>HYa&Hp^LBE5ILhcxlewB>;AcP};aV_I4TGo<)3%4lz2f2Uy zAeh_1^-nw_H&g+hx1q_zkHmmhwA6gZlJPbHGzu^QNkCFk{J)$I5Mdd16&&tWb-Dx5sLy71VO8-{{sQ! z$l9;tM({2mR00>B)|<2TZ-#?K82L97AnXD5Fj86DG%OSJ6YheIor8Gu<}hG^&?seP ziElDn-RroR?hSbQfTe#`L(`wzaI^x#A9Q7$N&A}>07ruQ|FI)gulEMdv^0KV|A-g@AsjC_#OZLh zc9aoo6X5*jzp;Y<_Z|C_4nf6SCY50uOabn4eT~g)iJbZUdzW(KEY_O|P^(}8KQA>j zCwtpJ5-Gn0S>bqw1H?fH)0Qw@1_p*iTq7>UL%jc;L0*Sv`)_zsHgX;lP%H^>J2lfG z+`rrX)LuOmk`?bwOof(u^;D@8=TI~7koSBPB7w?ebQ0m-WN-ck^ZKn&lJlD%YXWf) zqUKOr%_J)L5QHin#e|OS68z5({?NQm8;}!rU1XuB;dk>Cxdu|L)gRoDUJ<;*z%wIufMt}J~Q5bVfFcSIdZ92 zp=#iOj3OOXg-d@O@izZBa3`Hy;#J(gIMDnKtaK`_wu4DnP~jC2#er4h{kNU3SwZxy z#_?^ptzYBqgINO#CQK?SF0fO~#)oKcE2DT_S^UQXKo)t8Jq++^RNO+1kzTUVgd+Y< zdevSWX^RRdKzor1Vwrrm_3{ni&JjZc!xj=1g_v$@>ay?yIz;pgcC$YF$ z{Pb0eg^e&%0CDAn|CjGEmGMXE@MbiP62}GKK&`=G`Cw&V)?!pUikU2*KFagz`52#lAkyr7PGeS|n(8A9G)Zr~@`{DQ5C zdrB6METT2FH!AIkE!DnosD-1EA0{{q`*yP=A9d7q7h7e`l=DGZL>bSYBT`1=&zGaWw+-pls6?9OFM%R-&kE4n1c6qH$j+y2OLgoA zbc2}-?fAWXZp5#Os}Ix#HkeE;$?y0w7%KhlliGzsv-c53cu{iKnakh^J+gvCE>o#i zTzSVa7~Z!zXz$k=R*dCLR-rpg0~!ucXpoa%bL>ycD#kYt?jQwl98s1)Z|jHPzgBxf z6tvTqHh!v25yY-X)2Zqt@1J#DanF>|cS+7pW=ClyijZ?GU9bfGB+<}I%F=&5{Y~K_y-Sc9eo^(f8DOl zlgyYK^@S2&tTVMutr@9V%hykKUDuO5-1UItqqqB9xWn^7Af+EBquuYM#{nHO0Qwz3 zwf}w|r10{1&~7x~5x)^G3zAYQ(-ZKzJ4Mvd?oK zL}={w%~{S@lm4|EbpznvOjiSJ!;lamOY-4FR?t(klUGc&K90th?fwzwHno+(7k11T zhVQ)t@>73pZ`x-Crnvq%4B%IFW4uyqBbfEF|5{IUucb`puc5(PbOne3QV%Z&V;7(j zDA}J|sioLBRn^vN;NYHHH;D{s?mNdJVsS7C{m{xt8vEtsRe6@BLTkR>pX4MCW3)RN zEUkOw;(XV|6TmeHcv$*v7J% z$fH{=>$TmS{9T&maE>i82ofJy3|6Z8JVTV1=Suddp9kwWZMhvmqXDUqsTW|xj)N&D^v`oL-!RI@ur=6V@_M4- zcpg9{1SgdSvGvA!QwD8k>9lN@$1?{h4a=(|2@YbYKbGcI!gf-pM;E8@3QZj~*VC0* zAX}H)5Y*KG$YCXa{#Me6{r?99()bet^^{^O#rn4YjE z_II5#1%sWoF-opd^TU^+lH~X08iZ;(O3MP?VS8#d*uf#DA3zNV*_8&!0%{)h>|*mN z!C#l8N@c~z6USoVqsVb;eta3sDv4f@a9RHoNn+Xd7*;U`QZLS0CrL2{Waslci`-lk zlK42Kd^3g3k~!fevW+tC$*KIYF~r@eHXY6BikbndCeM98hrw@?DxQxlAWW1aGpp8i z8n(;(tL-&{jbya`MVJxZIQS@S6U@>JzD6IcuD{d;oN^{ZIN+BnmgNpG1oi zciAL0EU{TW!AF5`u+U@*M6fpFZE?*davZ&7Zpea=jmW85v6*Gq8m!dAvcdFE=|$6x!ECd7x5qK1k}4 ziD{A(j1cpjx}%(z@eB#k2VU)_BiQ?VZ3FC@7p8qhSyp9gABGj@xCVJRa~CQWW(9^R z{y4ynj{OG0j|bc)``@OnY5>^jkw$1);jM!0Lp%y>iDKSEnZv7rljyCD1ER|LN@>eX z&rV|{3eRWQ#A9w%0_^lsbEhlQIEgGuCGVgNT$wKonINO=|0ltvz}rmMXh+WL4G%Q8 zYl&=ySKzw4-b>(K5XsVy2e@%b84Z#j3b;Lz#2QTvu;Vv%rc}A3&p9HjaC&D zxZ)^FKq@ApmZnrnI9m^qN@+j?vh!d|h0ao2nUAZd3bKz05m1`-+v~0d?#MdQC`tPp z5RM`rc4A1h4>^H_fRAHPmyX!NSZ2D!{Rq0>2@xin6#4TXy?rDFFtY&!XyHHhnGIkv zlzTp7BuU~!G@zh7FdHMieI*OLU4{P-z~2Dg+vyF><);V`J`~uoIr2C3D)x$A(J5*F zL-McZFmi>y+lIIT7i)Y0hXVg@`@fi@;4ezx`C}2#yC8t4M=AP^e0-aY4gkzud7N4} z{~i#ueC2-#`!=ktuZucCF@5=;JPn8;8Mr94h(c^4#u6wA!eMF!#Jyj07lovhDRO-m zIvT5+yca2R%Usn+-^R^F5A&lx9>U;(15&}3@0^MfIMZgqi#H@fWvR5TZC`lgW`mo0@t zrQV<4p9BAX>FqUyi%O6w?ATe*K>s|oaOj2u>+|sV7c{3HEdFIXAz3Ci_rAqnctZo` z+~*&R_pg!#CZ&@I{f|O1Mfj;((pf6wi|O1571Xd#w$0qwbyf$X{dGL^akJBkCcEfe zZ@J;6G^KX<|Ip^K%DvT4F{RrIDs7qW;+Ixc#bMKHD}35|nOe%2{ZvNV zs?RT!xLd85yW*BVq;^~A&|i6OF*6ccdE(kFovM0uw9nv*pUH<`oscg$M@SZ1EbDxj zHAcj7ZSN1T4ax8AzXrg$s@+Of9L=nY9Q5a#@de5A=~Z#JJYqt;(V#)Ob79?(vIoxT;ENX*75bZy+Rh>!W zoK$94Pc@MNWvO62i_Xa|j2Ahi)8wIu&~4Kl)+LUN%=bq$NT;zpo?R#OYj+?VVMDfw#>E2ARtQXo>Ly5SLO zf9q?8{mX#F@UP*~)nPRAR#=9}0MOotDcc?4TbVas)wI7o95_!R)r)V{!B;N);j!tM zZT+V#6reHtOU5a>ud4pnDN-L7S)O2^TKeC0iQMbKFx*K|$tTw_h&;&=g`vQ@PO(yRBE@L`|n>!!kn^acFuCdZ7U7{PpW|v9{$-=?mrd^l%K# z2&-C1@9_JDD*JunI(-L-IAj>O^kl_3JT2Pk3l;ewfRBw>R$CI`8+Ug%^UEb##s$}2 zH#aO0>?~p7va&6vKfWvMV2xcWMj0u0QZcdm{UxT~#WnIcm{W?tcK|q$;dxJLrlzMX z-m4Elhx~p6`~wdT;b<$l)BEKkVi*LzRIl8pQIU{O~a>ay(PrjxMY

Y=98u$m^mr~1tJ+q0zPpm25Vy2APPrepQhyC{k^1GhlAKq?b*Sfz*(%= z#Gw#Lm+RZpb(jK;k;v!5WvbljUus;JT#uS_dlsV-))XcJ$T7#yR z_VXh8*pqT|XDYJBkQ-tcvW0%OQ_^z9R{HQbPm5PKcu#4Xb$I(c1^hD}Db5hiNysz3 z>L=-r0=+5h8##i=m`VexT;^JM=A+e@zAodn>J5>S}9kU9E#9Slj z*@|l4HK5~K#OIUdAywH9sQw*CT|EEj)$z)YmTD7K1y}?hhd1GB7LcP5GT>>U{vCTn ziST=2W!vVU3xM_R1KKEfP!@=4S|8$gK>c(ar0^U})aVq}bzBfwk(~bH*vkDo{tZa3m0ai#b>{EbQVt+c5r9PD2;$q>`LUcT9=AkN z!cZ#0la8{bXj!FbbnM!yXFOJ>3UknoWxyeDoA~B7omZ_}zap2KhaJWs6v(IPhFbI34 zrUHPNppuf7=zKB*zZa3H5+Hdy_1b4UCk270D~v~*ngZ+&t;woE@kXjeEwp(b7NP$! z6+pKG*Z7}#v3kE42tFm*vd1SG?aq`363(($Fb0RDWeZ^b2kZ=9@t;QPtj-(!0E528 z!gWM0q|&?ZR|M(qu*E@stjLqdvO zD0n)TJA|$}UMO@7^q)NmfbN?L4z;68$jqPx)!)yt^8+Z8eGU6(pIloVHB0eP7l1}B zzpq=U*su(l^KSv4%v|Q&cJ9Xf^A(jz|02&qc;%Sdr1zo%?cP!AM=$DK3*2>Rs6PkG zMn0tJRQe4(OT}^r4g>+>r!c|2m9(WSO_H>kDfwW$gj7Aj509MbS&FbUrq6yjxP)ZQ zYYTly;&bMPWitl-Oj`35sDS?VA-;TVxv^>BBkb;3Cog)ULE)QDu?7g!;%+g-4{yZC z&yR3{rrk;+3Il3*(S=%=6m655u!b}0I~%05D?th3?vPq-lLt%iGUL8u(QX@SCO9`P?5kej2dQ%gubtyKX zapmMQF7|%k+uHUOl7KQzYxT4BU};=*Sc5a#k$Z zZ5I8ls!(`Ui^3fSqjOQXWJDWMzAiZHL`gNc;iqz`@*RnG%P9th*i4OW&>y9->dzh5 zepck+<@D9%Goq05kKfltpUWcIDLuqqQfZ?UCKLHKK(%5KiAvO|*$^W6sBAh|m}O3? zBl;idLzTd|mlG`E6_bOF0{C1}x!7Oh9wjZ@^GP%=i0OX!4=$1W<=uh)GD$)cOf&}g z_2oUwBqg1=WI{l{-9fdn{k%IZ5YS&1?;*o5c@WPhZ6j0$&2ugqN(H=C6fI8&6W;LL z|DZbn0O$tVqK4@JgR+2rqegD83~HGS^!&ecE(MtY@fPAeuKj;>?X_drsmeKv^A@hF z$k3L9QeXDh_{fptfICI=EuG33_q$ z>iD@KSqN`4YQDBwe@fP5g^2wv3`l zS67$n|6uE_!>a7Q?Qcbt?vn135+tO%k?xcdkdp2c>28$n?oI`1q`SLAy5U`0{ho8) z^E=O9&vgmxJJx-#Imh^nx!Ei@1_+YEz;TpUc-y{(Tn1D$(8h}V%(ol(b5S%vZs0$j zR=jBctOUtVh&KLGQ&r~gheHg~NwQy#kTd=62={U@E(#F<|8rq)#6M_3t|(n9!JqRL zk${{Q%RT`saWXi%;o;m|?mtN{o2zr5;VNpLT} zR?lGnyGcrrA7nQiRQTcr&U1y|pWrFvUqqata{ao-_HZg%L1SCiN^X!abAzwWWrT=my8dsK5yn!8-KWvla)j?-EwL9SFSHem_> z*Vd2UvE()DE>)jjF`d#KpXp7dVBAL1Sc?6W-8DszY1p9eoJBuxZuPm%x?{h;qE=TVxJ_`4vCq4b4JuWk&auFtkx zZwKYF9xtxRTW_yOJ`PL^Qen)0siVY67d&M3{&#j$f6p%V^!|ZRuX&N&c050DAN5_G zR7oJPt}N3xgtbf|(2nTbnU7((FBNl=)*yN@gGzUA&e$^WtAeX{Ei0ugIn4*0lv3ar zO_YSltru#g+C6|1@t86lKxpWi^*)A?Wcqa2dAsh5^ClXIt{qEl#I`>u&hx(*_6{;$wHRR) zA55TvmS1r7ENSHZ%?qGKjJ~x!U@FIu&K3ZB^(WvAHoN>LwwydLW_*Q`T-ay+=y|)%$bM!GS2i^SG2z_rub$4l;*bvs z{s*>x%)lCACQR1;K&qN)#4jpVmJD`2?~9j{&rx{g+DTnyVvo3qcHk3^$yrMeH6uDo z$ye46!qnknGEdUYBof)lvxmODsObpUl3TuAfN3zV69P_R`Ua)8F%Jr3>URJtp)wh} zj^nl=M(yuUTn!bcetvn3K`xgv`IgIzfXlk+W~?4+4PH+V=pA~Ua+?yncbyZyeS>@~ zhXiw<7cg!Ct%ox$9^_leE52IC|7y3j)Kc5pU2(IgxuU+>fzw+;LMtECf<#ESG|1{> zSw!fM@mM$bKytqV2ujL`@5fu z<6ELkr9iaVuFWdpNxwZH%>2<}jWI16?h9T58TM18PX~+KOv0~daX{&B_#$C(OLDHZ zU?{8L;Tr~uQt~5jBR=rgd{bHoD^<(_W2AF47rrVz{a^~H_lzYcIby{b;RVM9)g=tq zrH))b{{oxa$KgGXt*b}(d(~p!G_7A)V2P_<@jC*-= zmm$>%bYE?_DElwc9e-WW62&$-`lR50s)Ogv&iP4!6(=tdi+_rtD_w*9Ls`YEU+%_v zyW6$5dKTMLLY6L)DP=D=Cz$+H`;kdK3zt4wrQkDKsQblBBjya@ff? z>c2im2u0wD;v`;b7FnD5Hc?h!w}&Ni-PTo5&tRs4?N!R1^)oIoLhW(k&dhuyU@PL; zhU@8f_Wf^MuqA2)f)pa{zPXgjFMKS{Bc0Xo-l|xVMvAtU=d(EbAMR=9yRJ34-H=?! z*@=aS-Y%n(Q%#t_0CHJMifGnEO=p*o1j4uv7f;l;(ux%c1d9L z4wj|_%mp}QF0M~}EZ$Y?RN$QuQd<`r4dqnGcX3*CtBka)I+>OeS{poW>EeBG4;^v^6HK_;%mVmVM3zplN#UceU#!5FH4a&dj}9O%2qi{8;ok$*+Z8pCghg0b_{A3VCTRS|uH;NDQ#BZpm9$WJOV|m> z5{XpgUWmpNI&`Egi#F*upRY{k#Sr%Nz~LmB+Bhgu*;D{-ImmMM4tz3K zMx5m|DI~gXez_UkCzLphaxj_7-ZeIVy?(!eksJSrl+%c$qM=9zjg+7Yp{-SMsmZL@ zzv{fMW4LA@CMSHptsB*S$*M~8{ZiwSSuTj!t(sVIyIz`$ekpLt zp=LeflbaR9)HgI8zXO;dXYpq$)&8$@=QKW7Z zP~zgBTmk^+_-@j;CsV}DY{>O>jR3w52@#i7vv)o1X#QM|UbHA#5)O<>gm-ogekss| z9(vwOCXQcym4I%st$as9Ley>o5y9#%V*8Lsbp7?tmUun96P!wKaTDgp9M5eg)XXj` zyI(oCxQ3H4xY63I=$U3SsS-7q*C; z++r<9)aNd&KYEj$w-?fKge)xxhWm!%8F$PBkhXqd<3(pEn5;?xq_^PD()rc#IF3z! zfW@mT1d2epOE(EE)y?EZr@>a*&-d=FcS9+(luF-0;Cq47co&J=&8?;+9Hha$9Uwkn z0*BT=x#%erU=Jn-R5WlMs7n+X7Pz*3bk=Twf*)JU-?|DHd(^It7ZW|bS-)SLj(4l^KLxRlH{_b(Fp-i++r=g=ASOdpYm(T zg1``;UYo$h#uQPVat+-+vJ5f$dw;R_+LYwN*gYyb9}0@&N`$#cb%R;?Q+i2Gjn|e+ zhV7O-_ygc}!NMsaOiR8crM>gNGVRCp3lm&^=_f|v#_b>$Xh9hLrRC?|YAz+!7yslo z8~rtdnwD-7D2l2MW!`pO1Cj)zg#DyxVoS9&);|lyQ6}1lG(?>0ELdno=6h!Ts#juhJ;Go&(dPr9PJCDZXP{0o@u(qlzCW6$BU2luto3rUt zVhUl&Be%EZ4Aia$D)@4}!^5|OQI^WHM;LaSQ4PkvF$?-u3-ka2OqT=nan(OyX&z%3 z9*J}7y;+mguElfCmm@9Kc98InsrEvH?7hJ}VCm1tVKwagHJ(y(A)RyxIQ@9-&bQfv zi)VRTAdV{40yr_j?so~}P)-KDvD^g|f%N|0n5L1y1eob&RsuV0Wso!?L(lEc26+8vjD+ff^g&9%(KejJ=QC3S zSi-m%_@zGZ_g%L;SwQw{+>l{NH{y=4ATMAs?Lr|UOdkK?<^P6tbe*X*QS!paC*ATurxh6#0=X@0NwZk#KDnokuUri3DdKZ?qdK(ain&T_=PmGOaVsNBr@b;a5s^+eV+hdQ+*c^ZU)BhC zOd3hYxIY-}MZJ!7d1aG;NZH#X5lTA@t9TbERJghEZKkmU{0YdD2d9MCE|X zuQx=segL02%Q$oje;i7nd>o1p>WF65z=2By>#f6Gm0t2sTaGFQcjeU=L}1};bq@uc z2pE^$+Rg&wH_C<8{}bGpmxF2_hg!Ufjp1NI_CMM%h@b7!XQW5Whgt9gkOL#f0`Qff*t2C6a7XqH+Pagizm!_gVIr0nsEYhU(~ zCOmEV+iHp(#!)Br_%aE8McgeH<%9(pmLB%liG{t@)48d@e;RS^=N=_nV7mB0h|2>G z|4`&46woRuAK@};NPH_4A8c@Tz4t|@vRy8bfFMx?C6Z5hH-3)0M-P4Nc?u9O#p8_v zE*3fCK_Z|Cch(!>n8cxxM>A`Jy;WM+8DS1kP=$|dvHcn)xQ?NRaJ@#+>OJ#}K#_IL z3|v%_mGRI$TczF#*{;b7D_Ojc4k|Alim=l)3i)dB8QxU4*C`-FyKh)cU#v2?4>_cWNyEkVxHbu6(-tVtv#Dk6Upi7K8 zd$Hd($atV=37JaZyDo8J;_06~{+wYzzk=jI+MYzz{fk*3fw;5S^fs0o+9?f%PFC>;>fO0no{a1ti1ePZGe8B6TXOYivOjP^*!5Nbu zTWgHB-sLkJWVtWd2$qI$K(xy5WuRldQt@q0gZ32%zvMjc{CD7rz)MJa#ot%Iv$SUh zqf+MwNL5oNOqeaS3PZKYdJC;h z;a0x*`w#L|;4}K3zCZmV4gP>ggTLXvuLR&ha2??x|9`lS=y$-{i3j!dUYty;Dkbo! z+HP&@_Gq;c8vK+ta^LmNx#@8g^7p>j!6zN@CD>3 zTP!8>N#R=Fo}9(TrI_c7PdxNtjoRGAwI1uDmqUZACoOM8NP%h(?tq|47x!Ptq5b7k zY1l9a4?OaB534;(<>c!&5lbHKsB9nhU$ZMZcJup0>eBSDKAhgx1)?#nFm-sj9zb2Z z!w>a;!~V5cT5B-+;2ZZ7%UcM&3p?q5J(Mut8L zDyInvg5>_*{NdH3QpE}Eaq08>$&7<(e#_VHtsbI0-S)fked-Ze9S- zDUkQL;sFX7x;61aXRhGE@0Du|hjL>8TgX>C5LX@X*SNm`4m$J^Q`+&(4?^G+i>1Kz zqj9)=1Qik<03QSBWyxnmg=+hdes`xm`@Qd<_wn=+J`P8B-l$%foDVQ0cbm9=AEv)yd6tPedFIJNePda+u^Qmu1G)|7wiU1 zoBL0w1DuK;!8!_lURVuZ= z)PL+pJwEUB(dhVOwkO2~=J_3l3jJ!lUNH(hvhIAf{4hffFd;Tzhx-Dgsz^PjR}Tsc zMrJgn4SB@y+N|wPZ>_x1O=rDy25O6g9?V~n{cJ+!F=9)##(?<$R}uw({n(d9Vz_89 z5&a)@JX#QF#zlhJyZ>aI*dYOX5fOST1w4T3l78@|m_C!k2bQ`NTp0ds-TOyYlWO&9 z6l7)t1D!M~38i#sc-Gx@6McH$4iZ+XIH@l@4s!sK$7%MufPGW+LAdNQ%JIbBfAA8C zWKufW|DpBJgsA`DP?zhpg*%ULZWC zM)R}_GvbhX zot_czYN$CKF`fKyb#r5IwC>ZRoXe%S$+#M@2Jm;Z-a(>`N4ya61X;s>3RcbZ5Xe4z z-ews@avr4}qSJVl48QKwn$u{hU*v3s0bo!eB9f9#OYPi(y6jx5M9o?VRn`s!?zzJS zyp0H`gqIVu(kong^CPd=?dF0&D{i>vt?KN!hpx%MoxRzy&SEyAzxKSxM9li(<6Oec zKguY!I-0uQ*-AHeGo-+|>C!yhIowPEz+^!2#zpvwtw!c2)FIa9$4L*egXob0+Yn-r z@X#Xeud(F|fpzw0UFtoYsdwM$mS|pGZd>iYoKS)bV_+nVeEm)gkbcrF)VSEx*m7bq z4RgR)?5e6GG27JMO%0+X9C5uyo9B1J{0G?YI$S?@{ma)&F*Mgqfl$b&KwRk-dzagf zam-{0&b_mM`C#h%OT&{zr|5VqpB*MVubsDkxmSSo=6Jo2QyHHf2g1QE1ktIq*dBc2 zdI=gD61890>hqO|uEd_1x(SF8|B##-0p1!5Doy&uJQkM8v-}89i51F(f>d$VMSae0 zDrHGOxmO2rk77o6G>_fKEtT^i9;vU))bMOaRz__QxJ$!DM$Po6>}nuDGH8yYYvpN0 z90RdQKI6)2C^)sl8xl~Ak!mJ8d#++D4{rXxWMVyYHs>x;D^tbH=gfM;bmrHEy>%gR z;^v5HF{{Exo!Lj~ipYQZ4*DlRpoa~YcvUvP4`f`OZDPH;=he99Epn9|Cu)KoveKDM zX#?643p`eVwL%Pa3(&rLSP(b1 zsRqnU_ONM#5&vr5&ASD1h{zG)Y?$b5uWpVbp_G94{%Xc-_w|?JPZApn~wMd97r3}c0 zyhIHKm`N@!H_?$SScn&jI?3jj8cHO=BaNdWw-ol1$@e^h!oc z7nWowz%g*U4V{O|Hz>3=ZUi?iki8xu07`9{>-k5ZQ`PUng{kP6lS?;finTuVNT*C? z8RE8XDqM_jpsp;jC{x(Ap3-cXaBpzBi9Ec+(lF_vIQH`Qzpz$1Ci)+ek7Lf!%&Un! zFon%pA35|-R#hkfykfUMnG``v5MXjrEK>^`{?d+%Q*au6HPQZ;Pbovk1L&pVoG?85 z&WY0AY}HGUYz8s@9(HUEXS~@Fe89W}8by`Rf{jCEix^PXB5+6tlw0`p=4!a-RAqvd z^bKLwNsUW?G6P&|2}P%P>$BV Mj02{Uc;vYN5TFwUMYUEN#Pmq>E8lV{=Vb2P= zXi{$Yy8VLLW)CB3rOs1N`O(*pU=Tgf60Eb5+&K z1tQRwT)Vm=1`=B$Z-Cu8s2j_F0TG(%L;$OBNs|cfBEG`-Y1a?0yv?zQ@*xU2Iz_>& zPcts%Urqsm0q009i5vtK6-+doz3vg~{6!TkuXxtj(F%IT?|~(J5yd;PxClEWGX~Uv_F8=%+3fUmMJZ}pR-I=khuX!kC5+qYH0Ih`?UtREodP@UL zA-tg+usKGA1e{?hLD`xCo%-uvt~K2IF)x`QnGOnCu8m8dM>`;`HYup8{On+9(Vr! zBZ3br{TyX(ymZ98gJw~OD3W4lG6S_|s_p7th_XvahR~8tAUZFVk$$s{0D?#m5kkb-N@y6)XQ!sy zV#8s*d|ZY_YH>Ao8w9pcoN2V3=-G`{x`MbNXYZ5f&vK$YaC*JY$McOsguiL!IB9`h zopxa?FCli&CUkT?8(8_G0q&|kJL~>S2`Bl}8-`59p>z^}!#Hi=y6sU}FrYOW99G*aGl*a&a6CLdW8iapu-Z?lX^)ybce{nv(r;l<<;BY+= zrwyhdy>{*DkHx?D0s1r8lU}DEWB=`vKZ%p1cXud$^lN%u*?|oqp9B+&LHi9spX?z= zgX~oMcL>6J9Y~6N=V^|=!`k=Bc1V>`j)8FT&uvKrr{xP5UiZ(Oru)O*kZXU5N~s|M zw;G|{NagQ2We9^iXJ22X^7nS^q5m0eSd5QQi_H%Cys)?BIDdb#P2jtx(Artw42UYA zPl(sw3t9haJ72HdarWGU2h%uSIFCWH6GdWnz@X8o8~twpN4`$x{QqRB9KeRdXe#cjn1SZ1DHSa`(`JM zyn`=feHMuQ7{4TCrY*Jnh@RYl(3(HrMNti$(N%LB>>LGwB+al$AovYZ>elSYN|`_G z`fcZV%8_ci8IUH0wA{wZV39QP`9YZu;=Zc39s9@E{~>e2&4?cQJpMbn{=p8>N-!v) ztc)r97sy?{2&rN~_*0()mIM@ueD#5rCww4gWSz#p+7yA3*%f-<-<)j?bU|TI60~XK z(7_TyZNh>f0mDH^T1bT8dC2HUDfzcZBf)md)3g%w?(^)*|LVPOGB=iQ@41l&@%)RL z=2iA+D>p7xD(gd(;{^2GhAwIzDFPegk1WF8w&O8JTJOJ&9^!T1Z z-uc(}_@J~74!)-*edmMjY41@`58Ri z)4TI%KzxLRfucafrN@9O9T5qXYzl_!LI6U&?e<_KR9RZhlL}CF310b?)s)X}k6fMf z46;M&Pep;;su&~y4G>)lnQv%+idIDgrxhG9acmr~1-2rFYjtc+4fMlW&38xst7?Bk zFK`byyG4!TLKlT$ktK>A{84etR)nKyD2&P8U^abeVr`w(fjhTpj-oZzu zbPatnr!iNQk!A~&4UP|{z?XgsliResC(hJ4YxY}anp=de)_Qr~@mzAv95j7Ge_Uj( z)+ckP#APhJ;tx#19aA_>)MskArL*L{bOdK@Dr!7RBAMN=)+%MPpu-TcyWVIM+8z^kh z8;+M(;hMc}o&X?6Jmt7Qc6s!h(=7b?O;x>$3#R3QmN9NGCAu762TR>K-`4N{Qa#Q! z7(jF$o^9`9jAQU8I<0(V)f?dc`b}Lvs%VIDptvrBji@|=;5kxmQwOOS{lZX@E_ZG# zHL#UOId|j$Y(;W<^Xv0tBuGMHTAt!mM zmq!^OKS`z*_8~HHfv;J|17;Gh3R;q7(;FnkfxVAZ5>F%mBrEzWhJnIj(eK)joO(%M zdCD+EN}NS|fBWynBm=Y(qmVxX1>VncWix3K0dw!i4p!9&#QItKU+4|?fpH_svkBX|MM0EWi#$qPsCpaoCMPAjyhF8puq9G9BPope5F`0#EKFD$qXC7lk;ccg3$ z3|xPo>JKxFWr^!^2#$>*tHG;Gb+hK`T_0l4W28~3+A#+6@my%IjLQQn z-yoB0EvMwr+2$TKZX%((a`y9k~r*OYIZ`}GWMSoRrC_u|y= zA7sdI9+&6)sIFmWy`SKc;X`*38lqhx#>Vqy3~?&AQgbZ)v7{t5wG;Q|_xS-`Vz|~o zX!;dl9=qTE(~G7wSWp<5;gCWq{hODbBsZ@<@Klo$bW5 zV3ycmN7i@pa_#yse!btC;mcZrrrZ;so%qc1Qj*`xyR} zCh1AWUgg4kcNy^#;D|g!xgMXobzKOy!n4U2b-#ZIOFkqfUg{)a1#|2IZ>c?!pfhUS z?3vcfw54}m-ZpTpIyu+FXP}{B_i>LXA9>@6It;K6hOgx5uI#3QF}^}ni_cRz;2r0f zR-HuNdIkg>TQRZKf$?u2g7GngMth#wmO_HgKCN{s<=GPP3t4XM_IK!*%vRYlSfpEO zU=(LqOvxw78ow6@%}bn>F!2a2E#fvz_4lC;J!%zHdR|69)Z^PEO2j^% zljSj~JoWLw*pJN9R>}Dl#UOtc*6!$pc;p|AG=PeGtu=Y5Xws9tuWB`KZLO#O#f9=M zk!KIZaAvM(k{`k?E>y!-GHio3B^`}1DQ~vaH%2ioOf*~PpQCC7m?=H|_@fMCAB2axLU)s1ulE>c55;3}xYw;-WDxe5 zZ1`@mKrc1)DKKVI@prQKtU#JJsJ%@sN3RWtK4;gC8@H9&~ z;S1vCp;7dV^sD`i_u(fD3#3qGnj=X^{ULgwp=s=x4BEI;Ueie+P34FTV2xp zN@v|UC3I+LL|sNuIYyxkXJ5MWYzR?!mk$e;VysGEvEd4}d5Asg2qr@*ELMk9%2OS% zC>4sI!G?WzK=^%p?InXQe_ClB5^Px<`-NqwH1%`D5OD!2o}Mm$_pP{7lDk4RCl%+& z6nA9DzFy4d)3zm8oM2~?B-*8ipC<(}#i*pGn~O;HS``T?2DBN1;aexQSj;THbr9(? z7Mx~BsHqDrLSzYa1dt1Wl7vs~HZuAV4v8#mk*~ghh1g+JQUML8B|fXEiHUu-R)Du@ zgrJ3tL-%W_hVmIDB`fSRGH6(F3h6F1I<`i{WeFlORw=fRS=9`q3wup!3Qc)2=+wcC zE;a|v(Wuw=+)z7J8$!fGW3MvJ@dE5W)ZIggCrfHi-i<~2qGQU)GXWVA&0Xq z_r2wCnz`c7ia56Y*l4hRyyv&;tF+RKZp6?%7ky;yNCCG>eR>kZMHBnmR2z|8kOsc=W=NHTHeFq|!eCc0H6BeNnq9PlhL zuoL)vy2*;mWQsUWHqhh zQwxG8Fpxc1NfLfrVF7>S+s6jKv}j?{r+_7T-G`c^|>v;$sOkiFx~fDPk*Xo{0X6lIOX*mu8SE8W)(pvIA%;4m`I=#%MTeCsWJSO!^w}}PnsB}}8AbG! zd^JK&nuONRGI?Mf5%6*lba(OnN@^zC(zW6tW6i#aM#sa967&=J)=TGHm*%)#wN6(| z!3wWhR8&0iDfy$D`(u{_D&|iV9XT^G`1`(@$cZ)XKKRJ zk}SSRk=eBhg;r1reI{eI#bn5GAKY<~iJ~GL{V-U_63gx5DmTxrAz?*4JJPerB}~{3pFU$M#e%&?YR$1W=<_6g6p;DgW9u7z*K&8ikmwYAcz8hKp*OEYoPm#9rit3SZ5cKg*Df^{ z43o4Q z=7=IK`s5NWhiIvOvQgff2yoUeyw-1Gi&UbanNJNiuO;A3a|)s*BbS>gZ0>ja)FodU zBN(oG7K?F28lOB%ZW$YYdU`58e#9T&c41@l-XsRTWYAju$zYxysqySt<(fQRz!{e) z5)yow-b?}4R8~3{Qj^%Z$fB)^%12Y0brO$gBHBA$vC#n-y8Z~!Ck!&|JVN>9kML6SEZ}<;7^}ma`T3+VJ6kNE8 zevOAx3a0AQYW*tJ7t{zFw(9V;e>R2PQD_Zaqh`FARWaw+cQi!h0Y!`qK~aez=KwsTP;K5)#-j=!_Sne&LW!CHyJY0^Gl)^8i^o z@|+tKEa=r)iw5fij`=z7Yz*L1Bm~T`v9z3o@1L%*<|fO(^T|t)C4aYBY`V+(EN@U5NCUH zm9g4c^saZc-^U59(5mP; z1D3C2sbK18+_B!16kGHG1)7pB27_)bsXKYD;Tn%`4UCet?UchGD)IA4va8MT0`~o2 zsmUltjRyR%$+t4{C6y*q8jonD)|X|dYiY??X&5pZPU)!zz_vpszkZe@B;d^WjPr0b zGkq6`BZ5&?+?EG^rCB%hMq?t-ien<*3n%*x+~ug(C|AS_}=2@e^~Vbv(SO*kRbEj}qu zxD2Vg-Z9YB{|opiMCm1e7Em|849@N3`BUr&po@J6n)j1fN1>4?%5px6V1PO?!d%Hy z8H?c^lyjKStrRT}R<5s(d3H``b1sq^?ux`~!ORZ~P_d4R>6Ofd8nc~6B$#X>3bOZ( z+CsZ$xN8o@*^8W_5%S|vZ&xS^%r)=tpfmInUFCMGu)$6_Ri2pkcc**;jTkkYF7GPE zoxvKZ7^bnzJ2h)Vr#48_QLZA!jFR7^s^?vIv9o>`n<<7UDE&Je5QyXz_*^LZ|LZTv+ZkkBEOWn?tAUJTXDX1*rG-^Zz zQ1NETC`&P5VIfbX4|IsPAUYnLD!$O`f8B{hAS|p9Iq65yp0hgee{0rgn3rG&fV}v> z^NWLDf7E_1^yEK3ybSqaA3BLZ9BOD9Y&I5IC%o_PqN1Sip2=4wW(trfkP3u1neL*! z#6L^%O3$Lt$&)e-naZ;3j?9vvWGG$hGA^#4KRuoic6Yn{DyK9uUfm+AL!1mX zHuxeW&&7PTGTW8NcfaYwsgsV&jD zN7ZrlCH^u`-LpAQ=0WqG2BeVRha(I8d>jBYZTHp@w85h;9jZI4rl5UobHFsY{O}on zn<6QRjfFf(;pAnOR>>K#%#zm5>ckS1f*2lb{rsKlhI3R!^x&b-Hl*KV*u1&&7vRH# z+BjAAZjB}Rj`Qb2&XNf*5gPmo55g{)@d8zKMI`8{quLA@EGIH-b%dx{2dy=I1IzVZ z@9=7`-Bq_Rx>8SFx~)FI>Deb&!9516U#YEib-wrXybr}JP$CyCWVLC~vA*2?Iy~In zEV&tt^h>^?ByRCjrq2V|d1Hb+#YOHae(zkOSVu8RJ8l$+RU{c{r&)6fk(b_^kyo9q zQ(2ZyowLCU8UFTh``DZ1SH9Ve?e*@@$<#c@b@aM9%XywbYG{6=?5qE+DX|5tMpS&T z&u0*ofbTsCB+{liy;w<+NM&IgprRLD`HF;E?g$1Wxj}HS&_Ds7S2Y^Bm<402KMJ0`Z8aj(i=p&V zs7`H$!nIw0N+`JVrk7}_6uO-6B(|WvdqOM%F~df-G%*;&ek!J~`_zQ{D?+Ai=J|^0 zPU1^~x26VH_V%`V6w_aF{aPR}RJ#CUt=_&iq#5d`5rrS${t)pI+au2P)y#LyA;z@M zI;S6|%pZw@C(N)ZmyFza+Ry`wpNC|=pJ=r3vTyezQzTXEJt;PExHPZCr+vWTNw6_e z{8^^weEXY#(=ZCiq*xvD%$V>vV>DmfD|6aXu0sSe_FY!Uqk=b|7 zFkIrtmCl&Xn8aW-s!yRq>{ZVUWAEwa1Cn3B!_V%mh~L~i1Sy*?WOj=}eJ*3it(Z}t zfOsb&_8{iJ-*DYPrj@irLX3x;?AIiIxw=;MIPCt@VxE5|pbo{F)Fn7}yV4#8 zqlQVuY5Zte{F-)hO>*KQ&Tsd|?c33fm=vX+7cI{THow3uS%{U>hP%IkV=!GPQXro5 z*CYcaLw&9F8~*O=SK~i@c7%W5460DPk?LX@-{bJV%^I_I&wIO!z+WeN-Ov}^2Fu6O zfU;EZ`S{-BwPNlsZ|tniOo5HrN^``OOsj*8^Jp?wuvM;4!hE?mH5{_FrF*~@(Ib~= z-M+aSyY(YseejFC(ouum9!m&U6Ux%*xOmCF2#55IL)qYTbp|1ElD)GAXUhJITJu?! z9Och`XpubdJ{E`;A>o4yBifv-C$c9L>kWM$I}c0QAM016gHnAseG{SX@9*E9t0Twb zbwc1KyDt{5&H3OQ!N?RSYg$}WI7A_q<2YI0YW;@W=GldI93^Yu^6&Eer&W^uV3ggk zB(k(-tUwzf&UMZVP+9nRMFfLsHU*J$=IccW{+A*siD0`vWnUKZbpa;R;M-xn%lB_7 zPi=qFI2foT%$Dog1?qKOhVF!Mpjf3}ZYrf?$YQh8w%d_)>&}`IfhuNm9NiTy*AVX) z6=TPbq>e~hC(rQVbP0Ga-pv0}DZR6l`f93`8)ReVN11yorn&NYUtBM2lm!jHZE%0( zYRMHuAS!M6nEl|MOxJ5;^zf3(8tfZSZNJWQD}U~Iz<2g!LlvA;%bE-JARu+IUwO;t z$Zr0_;M0*j)};AeRFw#4@?8DPp)%<7F&-xE#Z8}Yo{QxtzlGI%P>So!{O;ni0#VJH z96sSgO)_}@`r09pL`8TJy4SHT6+=2Ut^fE}zFZ@;M(NiySox7|sVGbkxNbGyz%itRm6i!OcIwMf5D+Y6!FQrdt zVZ-Kg=z15+7#_0Ms*1MKmj?SKQuv<5e~lAmj`^TKF|jcwqa69pZ1-rmc!dXt3?&eC zdi6cB_~T=_Q!ztpt4mNR-36ITFs8bhLX2hFCaFmi#Skrrn&*d6lOZI2HSz3U$w*-@ z-l-0L8&#j5RcR+#YLbZKdzKf3imTFM$4~#?S}GOHKP@$?=Dm%@p-FII%1wHp4aUr^ zq;-f^tbkM9eUCzMXfV;nri;N2W{qPKsfliUUf~ypemQk|YbBx%+1B@3(d=Zbee6d) z=0gD{C>4F9FTtV{{GC!%In;LM2`au`Ml8RsqB+6J`@UZ>;ZCZ>&%W;KPto90!QgQ> zdWRe9&Smjw%4wGhnMUr9I&4V)ep9?_F<_3l6KwC~aOrwB>n!w~T9Zx7ma$X7r4(YOunZ;w{J*(X@L>@l;nk5(=l$zw{%hYMRm zYrD?j=KIPQhe~`B!UyV5pX?w3cMLooi^lX4QGECN6QnKK=j{&j_Ue~aV#EQ9cu!lL zZuu-2ENxbIkDav=*x(@g+;9_avRs=148y7#&4-D>@C@;^<=s+;kKo0KhR4cy&>A%+ z8YU57?#5oZ#$b6C&B(#Xxiy23;0vTucLY&_W99Oa>T#2v{k)F-(vN+{GB3Bn1nOfUi8Gxyejfy+%4Q9KS}Hep8Eb zqLS|eHfTG&Zfv=q5fMi~9dPTOBXcT{YBrP&Ga?u2c9edCfrp8dxoz0}+lpr9duZ=# zGji%$p!AYZ2h2u2H^gmYc{gV$Tt@ZMlPj?;uLs&%90Bp9se^m|nr_NqHQ2_{Vcqo0 zHn`jOv#WB@?PJn;oC^W7K=e__+ItdfITA$pJC3lZ!u{)w)?_0A{iX};!Om8KoJ0&Q z!dgsQWd+NqKyXCzK{OBMh$qN>j%zbhm#FaK7&gfq) zh7NDjkX3WnQAg(Or|oqOCrX%ydSI2?ze~> z>^hH@+VqVu`8|uRDDIkc&vv9A!Eg5Qs_XR z0f`W``O^i|zT4a7s|oP+3Cz!s54i(3?9@njc))inZhl{Hh`G}1k556nBYXKJ1glGZ ziaC+Zj2(~7I#Ajt;rQa0R;V3eNMqPu*EfSq3_N9&&jeCn6i+lpt=6+omp!TK7k1chr?6xqBe=Ih(P~h}be)9BBg+(8$_5re|_I zyR>iGfs3+=a@O(-e5>L500$xaXMUyNfQCK1jgVHMmwjdYYs@i(2)1OHBQHjA^lWc> zNxdGo8O6WJXN!a}K9iraiiOV*TnI8I`zK$% zrK$791;GRdI@=w+IO!t4A&%mCuQ2k_^5@GL{RHfm*dkm(S);F^Nt~m8c${(@nwsgr zz=60Am!e^3xmWwfOW0U*>a-~W!I|lpq5t(zLVY<@>fx<<=b@ER`wKPJ^nLKrk)EN; zT<}Yq3#7T2WWF8TiaAeG|CmxZV$L~A7hPTcp$o}(HyWClu@>uKP$#H!{SKz>+ZBn@ z2n1Z&S+?jxsd40NlunOFo~CdGmSxePn8kd{qiUjXakb8N^9UPYFb+S=aX!1x-=0D<<;n35ShP{y)~F(}QkrV_OO1dP4p<6;4hDN0u=>`RcPC-gQ8WCyUHG2P^=bYy~?{)rp zugl9n7?{2H+N-{wT5{Yo`;CJlISMZ9OYf@{WP)6kZ6j71i!LDht&54yq`@@&2CKy^ zq2~6wMvAt_hdl{G#*w)k#Y69tlCs?}3Xh}O8a<-(=Gey6n}&ZHxNP%GJri7EBay#* z_iT9UwPl){)~8;r{OG3I(6E>Gx*A}sY0y4>rhKKK}hMk zT>K_ayGBkM$MHw3-`Ddz-*+e6whn{7N|MtL?E}qLGaP3&xUPwq~0%C(}M9bC=q{#v+U_($llNO3sKfiuk-ws9#e*E*Dn>7R!tPj`k`J(VdWvW}g#FX%tSJ=4R z)Ao4Sqz~YD>LQ&N?N6?6>Z0c>x?EW6PQ8|gqeaQ*ro%m?_LdrBHPl7T&I;FyryFQ} zJT4CBTD(jr_MCibHsQT2^Y^qxJqF%7Ego~O+Bu4xM0I7C+;8qYlxTJ3nRZ2jD1SnV zwJ@E*+rl-1K6T&b7NY@a)ab-66Ewzb^9)xfRbn@*9A-%9TbHpu5mjDp>e$r2L_j%q zi#AIbF8;#%=K}ta`;(9~T{%qQHG7uwZ^o+G?05k6cf^qiYJJM;;Yx$~m(~DGmnP8G ztasY`ub+V@@XtG|cXE0v>>K=pdWX_-$eLL0GOpUJ^0_U} zIHKb8GRcPBDO$zq?4y7TBcNbZHj*8MQT$7lttu=ogJ$CuW_IJp)jmjOyGUm8yed%$h6uiG*Z_X!rv%;y95)8*F86nY< zZQ8|*e0z({k*&r%+*Uny6J@4`j0o*|J8{>#$~E=l>GH)K%hF^>Gg&~UiD;$Wu(M^_ z1Nk==18J${>HJRjB>YZ+%j4y@pG6~^3%PfbKN?1jt*ho~q&98NwA>ESB#+7WX^N8Y z-SP+NH4+{a!?!aYSr1!GlsQCAl;p`Yop|mC@4qr_@NPQXpj+|}n5pE@46Lxcx<@VP zgT~RaaAViu?-VSNbUV2SG>6JmT85zM!_yzVg-E3VNdR52fn6uGvXS9SJI z{!$nLraSEQ8SXc*r)gsqgkIcT!eJrJwcbOo)KE%g~T)|fe(Ai6`>pd(E^;sUQ_oZ zBO$FW@^z+-UOMP-fe}G@EekG#v}w8f@K2-3JMQv0GO|r31PrPg!t&!7# z?tkhg0|_v5JAs$Og8!ZCqir&(O9&iq;HRK6>X zooF*I>y_rWo$!A+jp;~J8>}Z^E;P9lkCSOyjlGDOr`CIaPd;G8Ej1DxecsVAduw9p zNd2=`W0Or&vB0^i+VsDeitH*LxtL9b%B!)2o2(q)JUqy2%A~u>+LT$gFpqr!gS7h2 zl#;yVTKBujL&hN=XRGf2I{CHzbVsr_w!F>x@cv7#Y|?U($s(c^H)yU4N+6-kaKHt@ zjX>&Gxj%Y{Zq%N4C-yY9Huo`?!R=2mdloCtC=sJ=hW^yJB7^U=yF-p$eAwhvU|O4} ztyu1AYRCb|z}q*b-FNjKnT5JuAcu|6-)e9mj4tCe_Z#)sfw<^&0v-~?f>3zsU667t zvm*|CBA&e8heLaVj2rr?GhLN4%X`rhZHp2*o^!u3y#FI(kyv zVRoq3P`x|xSSem4`l!i$e0p&3!6aim6ds?;#zwm5Vk&F~; zkZ7cXX@$2MSF^8Ug669nv^Ut^3HLCPKA9$b1sy9HiJYL@iU?yBdBmdLs|LP`HX@C0 zG7OYp$+nhfiZXf4)>azcxir3cYXgl955z-{)ofP((HSCcys*wvHAuGDNQRQNV8K%LM>y|y$xM_+ir5->e;Sc+XzzP0E?IgbQG*&Ncu061EuUTj3U?m=NC4`@Q|7i%q0F4 ziB}>g*^5P-Bcw*G2t!U(>#;E$BdUmU8GZCepT~EHB{c+WP!NfFS8)JsSiuV)#->2If z<>XWTLO=N5N7@Nj@j6K(gh$gS^jF3q$0hL7jL=O(AbwLafNZ8CbH0WyUxlnrL?_Ht z+jHUVOX3KQkmCFpiOZ&!S-lK$$&gIbf<)&R9Ln*e_bYJ@Zr?J;{bdvBMjwy)4PJ*B0|X&v+A)qG#@cXUL&{*Xcc15i|MqTTW(q+B*z zhKSLG9m=vk&#As0YT4@di*lE;F6g(k!(vHJ6ZxIrG@Y1(=`l}aS4|aQn`l;}iaUfk zHN+}I2oaoIc?KzVT30m|Qc))JAPwFU$>(BSO|ZaIDs6vp-3k26Dw$s_ z#f6&Yc51Lot}2E{gV`yTfQn2!DKlyayd}P~O!nu%g&3=a!0+ zyM?`}$ehm2nk`n=tdM9=&w!OAB)wM&Y+;Uc``~C+b!{*q zlB4;AVkR^)N2@Z?Nw(L;kDta+7omcoW++6uHCCjXW?LlU$g7Y9@3u<}(%h8^&)q17+{@HS67)AwOBGGM%LiWR;fz+Z zG9|@;HEW^*l=_4%k^T1#ibBmVjn{Tgv|hZW&Mp#klp_~OhQ7(f_&zB|NQBP>^Gq>Q z#}%K(O|{3lWKYf9J`j28kwy&fy?G(o@Q%@WV<`HNw|BYY>mBk(&(SY(dQ?e)mDCjZ z^kH5c)Dl}7)*W=#CyF4%=h-e$^$i)=w3IO? z__I>b6J)~`%rTV=WEmfg$o4-3kNTH?J(X!Ez=}TY)H(h@aSK#PfylT~u(zY9h5TVy za~=Tgs8(zuJzq^Ny;J5N&X@o~ejoglTW@IFfAKGm2LXoN26AmMyux_FM?j2~5v{nu z_U~0NqfxB4!PV=`W_rG3?E1^kP8JpxXmN63^nZTinhwyz5E)r9{6E4!lteY`F8G?T zyEZFX4Ja)~Lr&qU_i+h2P2|HS6m$HR8+BXd3o60yf8rCi$s|;q^i2HK21&Kk6XwWt zip;>&v)OrDeXmJw%N)Kki+e&^o-hT55EnbEVEJPA!S%2Z!6q4NPX6T4^h487Mf~fa z&pz*jevwY6gV~~CkZlijAsqZ$hhqqnjhdIbR8+@b529qg=!4rswccAkG?Hi1_D5?2 zDSXy%hd1soK5)e=>F_AlG!-#{;EH5@|NM^2r_8}9EfJZ+>(kZM70>V@q`{SR13U>_ z=q@-A26X{EZ~+DgZV>P5M`nMT{P{Ns0YX}?H<4X;C3u2h{~qS6=ZDkl~iNIJMo8Qsz*QmV}bvM(-9mtUy4 zXGM89EFLzty)f>}RT;V;7l|x3o>x$l z-uY<|cLn5RT~i zJz}dt|9f+*!#;kkyl2UgY)SrceztwC?9LI{mvR|chNKIv>NMA~Et)2VdGBee<4>bE zb-#w3t~-RuDJTvdG7a53!Uh=Q9=*VPCi5bxKGaz(Va%;!_dvi1yBU4yYpdgL*HynT zZUig_R94Ru#oq7F2>9M-sH+O!M%7bY@>0#N5ScE=QqwO+e|j4^3)-kA>nMK-!{P&+ z@v;?{DKPGC(vUMw7I%orCw5AykrOpo#CRxd3L(wTG*0&Eh(AIJ2P)K*>L$C_?3Jh= zatZ0^o+Z{QR$_RDT7rp#BkrwuvzZ4;&>f0KSh~C`nb!nOcWrA{fk{yhCr!&O;?zt3xd6Qm5HOFrk+V1`4(euO)2n0PhKqCQ$(}lPS zt+4oNo>?N7t%6KN<}#vYuK`AS6A`d%twkHDY5hC;^a!*RVMvT7D1Y~-U~VB}#vD|C z!-_sJo_hgy44&6(!gKsKBW_%a=xy2ze90q*F}tDlc~XeW6d^mPPJsq$hV7{GvO0F( z#Hhx5d3tco_%bT{4Z92L>Q-J+Wdu?wn?Za|&?Z*>LIB}d*-4F2tGh z%jurZpZSfP8f~ROmVgq|>e#&V&-B&aP8i3?=PGX#=5hD7V1;K&D=4+wSoqFycVk-? zy$^qVetPV-6Zxv82?$VS*jb-)naKCY(s?@mW;i14k8Ip?5hjIe3R677=->)!j8Pc5 z79TCJvYG{u8rXQ5%Q_7zEA?v=gHomFHR&CDDbhnQO#OEu_(W^K>MyoEabuG{y+Ao&{E+Vk_o zS8hN>q7X4ra(F!|-a<_I!%m7pGeXarh#GMSID2YC9zx@HyW!P~1t*2}tV@#3@=ctx z@jL#kyFFxx%C1IH?CIiIO5B8>h$20(+ekPt?Q3_+e!p#~b0 z`KnS*?)Lb*d+?AbLeLAbCh&swQA9kN9GEh?vyi7Ww3s8e?6q@ER44Y<;dhGgzMZ}e z-*L1WFeNEvEtd+R)@FT``HY`R&`0bRlPuwCFHFFPwHI64|Hv5UE-!Ez?BUCF##eOkWGhU=r zh0b}kxxtV2g7rl9#`x*!>#YqJDzeqxpmYEWkTU@7)oZfg4YTtnKrx<;+DSa~)cp#SIt`5?)xkLQkI63!13 z7(Xc(bg;>*=<(FQEYc=K5FVW|f^0=JXY&LKW?8TChsDzlIExFPVp@oIw5r~)XBkUG zI_w9Inoo$4d>H4%?5$Z=7$fc z!4t*?+B3XRV_du0M0IxmF_cf?SgJ&!)al|tgGn=$@+EO{mAVq+EkECzklVZ=E9<}R z0_6}If75JUn*0*E%eyYF(-YF2Y_WeDRT8_V1-Z=Dout-Htl_=0{4P#j>37B~1|zD5 zL|5jrq6d{&gfEtWoX7cH8G6+|5V+(yi`v$G1%joDGLbKHVMYb_JH5j&oJlx>fLG`A zdKvd9u{2_e_+8}#f;aB%lZ28ItuLyImfHu|jPhW_ zjk~oCUtzF@0ws$kDK?^Vc0WCwWhC{s-6?K*qd+JFBIOi7p0`DRR{Acp>YH3GIavNs z8jVE4&f$*AKl9pIJdac0c^kThE&awG?!Txrj77;xcr6s_WS9?rOfg^kue%Gto=cjX z2s20d76~2vRIaLXdlo()>b~+<;f1dr+62{{z3*T18~v13EsR7FCGTHR0aIF}(0zpk zmdF6q6bt2hTZQ+l1fCz2my!-fb&er=x2oa}R`l&MSt+yKm&EUuFD|D4)msSD1oQ`N zJVirVXOZpEEXA&>vLH6jUnnI#6n7UgtXzFooqqfK@#eU&0Mb9RcP(r9eFKdy6WL=!LIBEX`7H$*v%uJDtv z`tYb%-p+|TH1Eb=h{vCkXMK2tka8@MxQQ%2o|~o(9oUS_jbqPY-^Hj>oypq4fpDXi z19+f^Q`h6Y>XPRYocSo48;CD_41_e#pTWxf#xZXXyzOvZFb9beQ8ZIc?s){~yLWw9 z=6Rnb0kYMSj&B2G`Q*$WwE&4Q)(bb+4k4Twm!v)%{rZ0dZ1ET^(_CkG%kV=_1SxoM z0mO~8Suoc?FdPMQGkCwfFj4s*s-8-|tVHBNwkWowL`%@w#AC?W5g};4p!L8~H6o>S zYCj5>^}h40=vgeT=`m=pY;GK@X5;%AWc8!-eQcR?yT%HS<_9K5pW5LeET0g^r4>j5 z*JsVQ>9Qo+2Q~OTuamxbe>)zepub?EvGe%Rxdu-Pr6Y!H+f(KhPSPhjSPtkfo2sFbyO`$-GRPSIDtD!(r!(oUN8olt z4L_-nuX&S|j9InUwxp+gEM9EPev54IA0rdIYJT>;B92bqVH|SCp-31r_d|Z%4X&K} zl16GvV(jMg%GXKS4^8BG*8zvk3g~5q0exgKyb^H4Z2mem0ukM^eNrB~xoZj~pps#H zm=E-wr|;&}(_Z{Luj2yNS9v_D=OaoUdn$#R*JF^}d&1}*c>6MdrU3gz41j89yes?c z?xNV{=hJnGdG_eX<(<<89J!Ou03^*rgxkjbJSs_ZGW7b^=bwtqCrZ^&-00j7ZlrZ<>u;p7yW^iVAS6R^#2DQ zfA!N+;70|{MQu@sV6irt*5&0bSry|kYW!hGa~au$ub?zrGXa7H;d4ucUuxveYmdf2 zv?f)nq0PX8LUE`;h5sFk?CZLjEZD23Jg$ySg*^fe z!L|RDyCotl19)qJha8pfs}FDHVV>lj{=rq^N?5q|JL0`vjIByFkZvH-4e$M~^z`sm zJWQRj1G`Mq)PR7J=Vgv2Gn2k5Ng!%dkA*5d5hl!kYun5l=Ds9qWfAEQ7I{!1c zfJ_HgxeQDM46!8fGp2TgK&NgNN?GQlEf*WLI=DP-5DFD^n7f%uuL~bmM)S9oNJ;6(T3gfFV}apfI3lvttEWS zjuwN~EcY{z57bFwAf&s|3MAOSEAX`W`^19&@Son|07TP5B5E|Ltjv5hl^(Ud^sQ=% z5KfyE{G~RZn1mgLfdodVn9*v>%2%icG9sOi9#m7P{t|&KBI+Lhi+9wZs?uUb6&l3y zY#YB_eMYd=jQ~u{;=ZWnnkQl9bHM&a#Q?PiU!uX2AoEKFxu8iE(=se6d*2b3VtZcn zg7i9F6V=+gPijiTBn*e4ilbv7%?|^}2_F+tM%*pg>n_utVtYA;(>oOd&d!aN)wsxV zsw1xueIJ6kt4Ei{A2t3gAN}@Kp*8JTM?T>2_UBHxXkwv!V4Kq}x-hXQ*&54vhTY+R zO+E{W)^HI|nx+FbMcW9iQe1mLll12_p(pal#c*%^1kAY_2y88ox2xj~1|+wUt<)%N zn94rA>r78C{~tv)a;u>6!wn&^K8i*ay>B8M6DS6q_;X=S>;^?r>(&Jz)4R52c#~j| zt=~YadtXy{)?yxNQ=i#;_kxsvvC^uc)xY{JlK#_{kO%K{337r>MwaR__ctO6d3u<{ zs7Z2i^DUnaZ-in1wX%idBD5}qj{I7W$aaX)I~FLXo-z!{ zTRoN=!XqLr61(aC@akhYQtMbcp1x>-eQXtTJ~$9bZ#ZOHl6=%C$>WbR9l{_ZYuEy7 z(uhIEbzS`YPy}?fCEXEr-+?8+-y}a#1s)=BQntH9>SR2oAeFv}2PTRd`){USG_qT3 z==m)8K58;w-en~IW4Q|_{xeZH09#H^WvL!xkh|D!UFJq-ENwD>qrh#lEpVI(>$4DL&rkoNLU6`87Et7)(WtYRBAlL|YiT^onV^a& z%W*}UdsH-m0XhR^dM>XB*WB>$eQL8{Q--BRKZrafC?-}YLBwEMnH$o5?!M*gyF-~4 z`R9PpE!}WsfXbmw4F`WciOdj?7Cp0|N+hbWO~fVEO}tmxm`x!vH2a00zrWGw>m>Zw zj&P28)P88DC$JGoyFCVS#qKq-0xR^7sa>nUJDwt#W-FQ!W859jqDZFx?S&dCdYX*G z+zP8AZ0-It+VmX*#8QlE)LB`)Kor(&)t~-xcy&dk?{&)aM)tg*CKbpAd9|g{6}-lW zwd=-~a8A4(zHP0KL-cv^K}y29H-t;&0%1Uvc<$tR0!h)(@BGCDu!G`|;LgQ-m!dgt zy`bE$QMDg+RfHMEb@{>rkqx_Tk&F%>7km$X&XZ{bBWfth5QqX3K4WlC{VQLmBBt2@ z@7jE#^+*r;xi|!KmWEWEnQFRsk$A&Yn4z`|<+3XyB)zf@0y}Z*KVU9ZI>|H&V#Y!T z;4Y|4b42@>Pmg|3><4ma2x9>U-AbUwc#e7O1nl?CTg;5Wth|*EwyD#U)C(X}0 zViRO?6Ca)3eqmJUCOARItsyVJrnFCK9SuA)}$p4oJ?T%ge9T65r-J=p@7f6lVT$PtxN4eyH!=nF%)@wzVjb}b0VKvr~ z+0+Q>V0O)i*@Ng*TOZ!bn;OkDtEC1qTHRDK@KuqrhZh0q7%oMGOs*@TN`ZEk^U;>L zPQFGLYmejH@Wr(Ld>#X_u^iDTJFDD`FP2}&>-;8W*OkaZ2yRIWyG}Xe_89wQKPYCX zNZd8inMipH67)&nn3Fz;aVN;d7q7dyjzb}wq6O<(E%)q?|87ObT~zeXnuG$}%#R}f zqXo$2FLu8>AY;v=7mx($1RS!H8B%iA8||!`x%;#aW$L2iXqO6|7Dbrl2;Df%>4lr{ z{lUKD;t1$vWcj?%N%;9(qX`KqYd)wevh-SwPYWFihr$;Gi)N!XJLKYEO^N}TzrWK6 zH_4&?klu_w=K&&~JD(Js`Gc}au|`fzD;dB-O(uxrgbb889})tn9UUzsQcA|KS~<&; znXk@tEivd6kWrgv8K)VtGJA#wf=mpbQ}rzq6d0MNgdvR>Xv(;^>;VZVuyL9;0?_z| zZRG@(o@s*IB_R|iAr!#!>hIU0-BR(;)Mo4gS;rQN=0^+ayvSeFrL3dID~GbewP`H+^kH$-m}iDIyu*acj~oq~~)DEqLfn z1P^&QO7cf<29!3a&oJ}t9zFikT!76?{PW#5#PE>4iU~_ifMGGG<#PS`^tlv-$wG#a zJ^!8x_?9Vp!ZGf&-_OR&z((#*I(yIAnDs^V!qiThhz*tgtr4>_9o2$EpM3sEmSL&BPUGrZG^n_*yhy;p z?>Y0IMcesTee!#fzqtI!{?bIq%fb21iQBWMInx(2fSU^Tb!z%r~`BG`N)C3!3Pf6Z(@IOR2? zzCi4IC<;X@8Mm5v9nq9*D#A^(zQv!zPmM1*Ayib6Et%-t7fcQ&sHq?K$gt6gFu;Y8 znQ1`U`+vGAtSlt?=b3BCt(v!nHM8M>mwXJPzt+!$&Pd!AzC5tO@T0 z1s{O^@9$C1Mr)4H&LDm;8lo54;|IptrcS#zUDW+hOn{|~oU;aK^K{^ZOL^|^t@=tzLy zs)GoR2%=wxh|!z@n_pL)Z2$Z>9;eXsQCM$JHJzOUpunGh-u$|v0(qwq@20uh|JzgM zH3O{sD@OwO`2&fhmgyedAWtq89U$6^F`97`1>@UR;l7=;M_cAytKW|ffe zhc7KJZ}N>0r~bUB<1X!4zH1V@%VUUG+1%TUyg|G4G6A4r+ebpOzf!x4Cx$7ff7`#@aA?o^(BR=C8N-zd&)SYTP)upM)m*1a#1TzTY6U%dK*vdNDz?DyY>y;Uff9`3P%7# zl8_6zXqnrt0>kk<*{NEc73Z@O#cI#s+ zmKOrYb~qPTg7GXbUTO%(M}UziBi_m=b_^*viw3jZErxYt9-9&@A}t%)K?lb@A%h% z)J@N4URn=pFkYxaD|$LC2Mp7*Tgj6if_LI#{^DMv`w2w1uU!x@ttMqFjaVbU-kEWeP)nbD)(KiMrF=u=CJPJR2a_iMBE z5dGmRr!HVbvZM^p{=lfX6!$7$o$^ZB^P#nN;66h5T%Ag-4-bWs)xs4<=c1Pv5%)h% zpE6p7JJXMCoF)C^nzqqn@kL$I0qn>aIM?ogd z0t*izSVT58KJ>-XZ46WL+vI=Jkk|B&mI{ROUB4E{z5Vax&#jLRXfvwUf+6sA*7xl zlPBZ+>$tII;tP}V0=7%_5Di&is)+m2A9nFo(ko!@uv82WCJ&1X(ulDvIy3o$X%q#` zrWVvK7Cu6O8vW>in))I2=;4)yHVHt~gEGGCy}F41q-NRb@HaOoWFY##vzOo`C1b_4 zMV6(XJIu4*9bIReN-(Gwa#B;sG_XTO7UrHxr?bBTUNho9co7pvJ0aF$%suio55)q1 z=eZXj>Ngt*r4PO((h+O>cN!)kwOTY)U>#WtJl1Wj*%5%h~g99`b9e|uJ?FA6fB`0*&kU$o8> zs(1+$yfD8A30e&5H-OnNN|4tD85X64C2WFafk-yJmWKfTv7&l9^N7$yJyiXU{7kWT zD~|gu5@Iq)KiFL0no%m#fIh(Fh8uZ?!fL?MFHJ9ksI-S!k%oH3C~M5T!u@Brt__99 z=={NB_L{aDJ!_Z%zI*3HJbwq1V}o1O@+u~4MQ8!q$c}dc zZJKK>k--h#yDj=O$wf;}RfzWWfuf9TW+_dIgr2(*1|q%%#{RMTRkt=@#Jzh*G{8P8 z0}z}^2~*6fg>Y+Z^cn@QNi&dTeyElYVPcbELSSV9{)u`BFdT}O(@VpC7`N9&;HcEX<#C#I%t z?K;&}YOpxndaH~Fm^(4=5z?Z)APY}-YC;R=V0r8Izpw_qjLg^5(Igf{m=^l&_JYq?=uAzd1?5--P=ujna8-_OL+LqVF*QWw7l;pGqX}5!{iZo)q>C$ zWyPKGb8@h51BX$^NDWy3kNM_1irY+@NfCGR71}flLVBA=)nrh>g*86_E~GH3A5vFV z4wQib)X+Uj`AK1pn|3rZkO)hmDHlc*5rKG30~pdkAn$(EI|)*u$`G4T!;y8>xO@MC zF0aCx5{Xs#q4R}l%s?0a9J+Vz$Mvv6xkrMXrg!k`9mzJ|<(dKjsFwiv)jeu=O6sBh zk8K`;fkvr4pHmH*?w<}qjzpVm3y(8x^M0+ZQ({G5uwGg8B~2mez3DF)tKd+7;2ZA1 zcoHu)}o547Qfw~;EKJE(j*v@G|5dn%Ankm(>-SXqw2zeyn(C1w0?Aun0&%+=q0nzI67`vOk6cd;@?U zJdg7?*wz?X)2IzZC>Z}2JY$nsrCce2fmqQRxlq3Q6)vE=(6}T?Na-}0P9o?A51?Ko z(XQ@lq*4*!mc3T#y&q4cV;=D1F?Gfk9PpwP^B{(8biwOhcAU&FH4yw`N&Vj7xF>k% zGdJ?LdsI>Yl!;B92^yur%-)V+Tasht>Q&3Y*I)UY*1Tg)w|C(T=~;1?>~sFOO@2|e zQuwVNc2+Y61~t!7c3zARQ@08nkyw0u_5VT&Myv|)3VA6IQX*fV?j~l_3rJRq#;cxU z^gUU)^cqRY^1)UHvX2ICyDgU8l;iknei1YIR|}g(83fv=K&Cuz@nE~dr!#70jlpf? z-K_^*=I_zoNJ2r8E-91$3j@@9l=oXq*g&d$=L5{Y$ur7#Q16+ECb38DU)`V-J%I-Z zAR+YF$z5WH;C?kL`J4rcL{+|jr-JqX)Le0k-B1msk3lH_iJ1SCl5_tbr6iC@(pyIU zS6!bg_ouE;|No<|2jZ}R#0Q|>_$%WKC;GEr1Ts=2szb9&6H;0*C9tTEtzXLhoyWk# zX+oH4_zhp%*@~$@bLIH=FVWt}p(2S~i^^8}VqL7FDf!2Yi*k$F6zIWs_={7GzWD=F z0LdpOcvf!x$;j}3vv>kg9inIXD2Ew1Zz6?FK9l|pPRsA^@~)r>i8se3**JB#qWIDvg_s;#SM>fc^F7_F_CTghZmr<^D?B*Uc=xa7E!duKYj zAJ3@hD$bvI@=kwkVk=+X3Pc`-pst!CbIlrqy1EHm@9%AQFoN4fFNZQbr-np3EVV9w zA4=yI5Qri6Tl2)D62KWR(u^IyI&WfUVnVkbFHEA*po3!$5D*d~NEfj~{YJ9H4UEel z#Du?LJN>my#$!3y4c9OVBWsd;{9bQP2Kpy*$pRIW*vW?ro=~j?QgRG$U)swbjdbXZ zqIjHN@Ge9o`-%Oo~isF4`~fG`Hd80yo5 z<=cu?iBzI{SG6*PR*d-P7YL@85gIw9^X<)S~V?ahYWCSbQ5BBiSg^b%}1#OQX#T_uVNcn$)*`2r5Sa%l5g%fe!skNJ}T3&4NzI4cz@U3|DRm4*8>nD%CLiKd= z4V%gL%NM`*FAc!}a=gCuVtLJ?meT$c2N!q!h68HOH~R+U;F#an8 z*;tY0h)BLy7z-GMuu#xhYr2$|o?^+!E`>!Ny2i9;yQlKup4N+6gLtQ#>nm!Lo&9}$ zMMcGhR$r>&;o%fstC-$1ke=HuZ{%*|ikH8sMNgoLnp#!caW0+b4(Y^S^V!+kSK3V8 z*EcYz>e7s17EW2KXnFlBVgS<_7`I@zUVD8s{v;r3vX(OXS{aW*c=Jaqt5qEJsX?s` z13i5JjpdJ3&{r1PT3W=*>nAk9`#oIk!;c26;F`3ww2vm3{AS*;D#m4)Hy$ecrLhX{ zIsX)K{PvG%xtv9__wN{wkrGqr9TAVcsN@HRJWewHPpe)IMRW$=ebEy^3Y)88zxTkP z+uv&N9oGn6UFY(X6f${kt-;97R2E?_qlWevkIDCsXDfdlA6w`s%`Ge}SR86^{^*US zQbQ-iD^Pv+y%pA~p`no?=s+uaF>Z9L|dUyeKmFURM+^z z3|Qy3CP3sb31Z~1t>3szTT!s| z9J(ELt?ccgdw4usU})gn>)4G!0wq&oqHWk3}C=!Um8O*a1;}032VPSX%H$4tjck3XLg9Ep-Kl%*Zj)A|XEN!15 z;$*kJ9zZgWBE4b zY|xNPj!?DJkG^H<-!0co?^eFUat+0!q2JAr^f!?==pSwNa^YvFtql!iw1jV>PW$CVomRRFl_(mQBBYpp zn=ErCc%(l6yfMdcwK-p3P9(Gz_p^Ww>K8_zbEd=vn_00`MaG3uG=?rUd+A0q7@iF7 zCx@w5`A1MI{5E4}O?da__MP|6D-MlCzqh9~XUn5s5IW0=A+M*-xkhO^4Z8U#f}ot? zT-^#KhN`wi46?l`0xV(e#fthFuE+>g(FF)wG$bsOz_L_0E zv8uG}YX;(nSd;Jc?mXK{-n@IJ#^TEJC@EsF!!D;jGqio4TT*b}+UI~uk^fR<`3Q|uq7bfW zW6qU+GQls(@1P0~IPN$?<|@{V1Q+DXY>rHi(XnV`z^az2Xu|^@tuStJLx;EVw~9`; z4wk#_L*@27#$p6!lWz=Z-(ruo`TO%qOZ|B-Ot1_gCz)^zsDbiv5Ov)S=_jW10RFXVc*Ku5C93cW0(U58 z`0iVg+-T)9V^Sx%-XKmprTlp6F@N>2v0UC7+kQ$b<(AR1@|QztCD6B+%q3jVFG`6K z5PfLc0XV3rA~ql&{&{(8%X$fmlN5@cp@M|O(^eC{gVIZqgHgs5s^8YBTVk5tNMI_;6i|5*%_$ zV1oq<3{fkz=rK6?tQ1H8CW0l-m);_fmDna+Aa3BvLW5IM4uYoKq(#RYzu(K?IMAo) z#z;q>DRIfU)@Ts4j+`uPLuKTepaM4abGfko*&@>cnyfL9V$rCL7o zZE4}Tw5`M{s-zMhhJLKuuW{sCANd9faMYQpg!$jEYeD&H`VDeMpsD)zSI z&njh338`M3`-&Y)#BI}SW{IgNCGtUOQD;_s8w7J(L)CAYH@j6(bNhn9qc{CigGplO z zog1aT4~eZ)b0VawUi9HHyMBi2u&mflm&t>-qSu|Fb}r9N=(Oq$<+H%QYq+aw6PjNQ zM#qT*HxZu&(Svj)m5Av4p0G@41lUpEplv6J$lt=d>J#-|pdSByXhXte;zy=W{EIck21jk3zdb@#~vHLlWaF8N5xy zikWsSs%ODIX7&M0$&~qr(k1%~lX)`PllWAA+hlW=3j^4yA)GwiJpNig2LEbaAKn6? zdiw-A9n3$>6LTCG62?e$l@|^llPr-nv~$YKrgRSmXD%fYI^LHKTKzAXNC{MmY7pL)9v7Y^=iRSTYsnYt7iiDK`MEbiVb|7SF%B#LxXYl8 zNk@jI0n8fV%6y z>Sc#U?yQ?xltM(XN2j+^vGsD!6 z8X-(jSL=(ZhRIl5QCYMOxd3+74s?OINs*+S4&3i(Tk3*h#HZ{4c**S-zD{A|VBq4pS<%yfD1mklyw8C<%yZajAssRI z8)I-u_>B6V*8DjZROy=`8VO&9iyrt)qYZfpvsDd{d@8mSMcpS6#oUlQ|0!Ev!E zV@{~&S^pmVc8lng7Hz4TLIfjvpMmCbgIk92l%g5~b^DGsXeZc|$(;(I!fW+fFVOW! z?EQNh+5f}dTX9p1DIwAzrF5q# zAxJlO9gOGsy>Z|B2i!5v&@m4C?6vk^)<*?CsIJa-I$UIqG%c1+uhr{RlSi~xr9Z;d6Kq& zZMR1d}|2xrRNko=E8mW1Aakk@x*#GDm z2<%rAbyS)54omBPLWfDCD0AU+`a$T5X%GczBA)q){S$M|ecnxmKTQsSAX(pkev=d_ zlaRyoHay^#d5svN5QT^H?gj=O5X^kX+nB$S0fOqXQ-6S99+Hycb58fBZ(t&t!-L3| zjK_vr@7;5x2Z?@CK6aJte_II;(Hs@i;^GHbvHrIi73L*gql0nNOZ7GoF*bAquid~* zv?su%|5KYIZ{!PfQYl-$Ot4WG5B~L{AUfLT3m({Y@3Ch2A_nZQjH@V3&WM`#^TmO^ zeZB#l08uWlAQ+$uTQE|Y-RpP%(n;%6Kwg--YcUaTX(ro%VM;@&1_yv~W z90;y2Sxi4=1A~f*h8+x@5!Ixl_t^!wa?@Qlzf!D$TGiqDZcW^piFmiAp$8sKdG~9|$ST3_f%VpDVRb{B6+h7v|~}QAfO!>H|ZFbdZVE)P8~K>FJ^Uc09_UAzAOC zJf-{7$|5*8hCQ-4BDVsW!LYinrXh=Xa#AGu!RY20`QiDz4D!{#jh6Z18qW%{GMh_~&`6p4J(M44Lge^*$vMdE=K;clW)2j;~?Ca(knW%jwn= zx&ig^ACrnK$pPNB25;7=+?j`-J)Wp1plI?s_6t_dp7M-r%1Db1-?zi1h3N*!%&8~RDsa4W zdV7yjDP?c}1xgf1A!ale=UX6b?LU(@NeizlDxl#x5cNl$=<~brD|hDaVY&fk(WD>W zzke^%vwQe{Ms9Er#+l3}7=DZywz$7^8XUWb#+h18puEJu&Znf&Jy4Q{A>3RZOvpo1 zbp8NM)2S3hz-@-P`Oz`WIP!AD6$Q>Loy?zH^nuNNXhZvO(MgD8Y+rADStRW{c1wW-xtC7`E z0Mm39Q6w92lmfqvq4D?OJV*paN@9^y^NE*KxJ0))l`)+hYSOk)gPj~YzhaqzNQp_I z@{?gPg80m+!8%ts|A8V&0mE|T%RH%-mist&^m&zk zQJa-dL8A1IFDZ%J-`4nY?rIpBH^{*(cleF3`cYA+W0pT|* zz3=St?%^!ZO8Sl;t}V<3zZR%AYBYPjzx!q_z4?;Y&r#UDg;v+aqe7%e68PL(gT8+ylFD3BKgHUK7omx3hw?M!yx#&xm@F@h-}&KvR@f_veh!aeBG7s#Q|{=I_pJmo)c5 zr1-3|Qq_m~H15?>%xe;nD-=a37hZ1bH*2t-mX|#x`Jx`w-x2@XvahWD85v;`OhaOD zTBzavY~zt;h!FjQTT<_!@oHpJOR`(emSdu`Zkz1yohUtA&Y`kZUnN_$b+*rbU|La( zq6-aB(s(Sxna12%Ual||bR-?A!`jO>XM%xHY6JvjnFN-kKC6Iq07EYngR_|tg z&iKwOw$$NUI?m&I?J*0#A3MbVI3lzdCWa^4*- zxb5hV9V}tkv--zf`i+$*9*=V$6>!;*%nG=|$NNzROu)AxP%RvWTW{lH?P9VFqW<|6Rp?e zCyO1&%Yi37X%yM-?!@HFtj``D9dOm$x{wW_z|C%F-746 zehkOk-|_4)Qr@lV+8R;LRY}zs!V5CP;>pFdCYvRRdD5XQ7IJ35ut;#K*dm?I1As2# zo;~u2hf^9b`OME*uV4O zl5!k;*3H6UW2I3(#1bPVPC}?WQbzB?mqFoC zl!yS5swWDKeg!YL6P8wuVgt}BR zStV|155J}Jn)E;io~u>8TT){!n@jH7tM6_A4*KV_va}j)DpoiywD`z^ z%%ddU@2|2~s>wayoFo7?ju*AHwatgLp~+S%=}$q1#B-#y*FiAV ziDV0h5{i@Vm`8EKjfbKEOhdzCU_YcLvXIdDi9tC9tyX+G?z?hU zjpt|gEO@#<*Cm{v(8s^3Y_EJr8MPf6=;G_H-Nt7%| zOt#^d58N`Zg!pfAe$QGjQ8FHe?lb>bKpD)E|E1aN~5>F z^x89_VRX_(Nm*skVJs%q>I|&}XpbaIatc2&NJER7%*6}@-h0Z+hnnO*7VD>XXmNn< z&!5T0ZvO;!K#7RhYj+bUxyjhK3CO=~|^6myTAoC%PKP^fMA-^OD5T7u^8{Sk@zTyZtE)qT-*~vHHo&4weQZqihWd zL%f!??A7sGa;@(Sr%}8}cI1)Qs;+t&XgJ&i!NKlN7um<%Rku%iKGiC{AtYdl;vwwm zC-HRosIu^Z!J50v0$mYi`eEN(^GZ~y72#Yr4KAu;S-NxxmcQDAfM7X}+ZXoB$Q#5H zn5OO#8imw-gj5yU<+JuRh%tmynbkUaC$FdjKQt=gPUJ$z#a%N$sgsxL>W^QTT`pO! zMl(H85Wm!&Z0ae>pgE6wOUGxTsM=`(+sg@S(o!8-%x1m@GDvRmmgO5i*2oH zSFcA^82xM?8*3q@ub@MzP^5b-xs z1q7Q9@HCUq37|1F#UIpB2bA2Atw=soR=&(RlNyX3ptlxRp*!PGs4mV@xoPs^{mHhg zOter+=o!0(G)jb7*kN%a_3|J}L^Hr*hU%ZF^~mIWyEoE=w#viGjZW&UNTGBdBd(Ej z7{2ANaYr#3CE|TPwkU&HHDgJ|)1BRBTVvVv(4sPlB!}c4If-E9(*)#zMh^+FDYGk< zBhj9$;YeoV%YhkNp_4*0l`S_^s?&YR-5K|T<`UVMW<2T4)cOLU6ljYJFTOB++2#3E zmZT#|(olCuB*2eZsWVuyLZ#!^v6x^wv^(gHiwx!3(edT|R&yx?QlObI!086`BsSG) zf^t|r-Bz>9xNMG6_*SFx6R^`)G8SLT?g_?FTWfq-t&QAh^L;8l zX#5>F#V=5B>{Tu`EXzNKL^e5?JS;h0Y8G{q8kZF84+0mLF%$mwY*7jkHF;}6K81WCP6 z9U048z^p&~HL?oZ_fgHuuS^*XDOL4(C35D>ns{LEc>j*`(g#BFWG-k#Lv1mBG@$4U zF-5D@og1e>j><3{Y3GHx=CZ7r6X6lfn#p@mkW_UlC+ogR72ObtrMqk49`Qu2SlDxT z{bC&yr3P!$rvew2<&pmLognAO*7piR)t!D4-d#nf0P`6R3ffwin8AfcEcixXK1|QnO z1-xsaFTT1GsHarPg$5ppcZBM)t-J9kyT5@*XG1V?k2V@R_HZe6f*d7=d4H-0L^MF1 z6=wVj1Y90)Bo>#t;KM(w>uFY1=0omO1qEItb2Zjfs|bROBtA5!PlJUMYDxprx%JyX zYI`sa!pp8t?0y|@u!q01E8m%D2$YOdVt|E9>~@(ogPB=5D@cO8TFXgI5+|erpVgSo zkpguY*voJpvj7XqpDk>Qw2zzQIO!4poh zOZdhs(`aw3dN~Rq6B2-$_l4b}FD}C5F2+i^E{iePvr>8)8>$h_!zI%uof`L4*>wAn z%;Vq{8a&F0R#AuJhi|D0nwq?S)EV)APOAp(z_g4>LZ#m*vKJsboT6fK|0B6Dx3arm zuqQ`LVjqs=aCC^{a27iV3Osz3ETWf`oO$9pp=5%#gxYYDuK04KgpT~F+ zSF;K1)0{g*3VQ6}IpW?X2F+O8@PcY|@*gd__E!>!hcrdq!{K2uRAwyq*@#62iuof_T_sGSwVS zO4Sq_16|h#512o~6ndUB$BPFbpL5+gb??e+gsvF5^>+7~%gHd!SO-jIjOaI$>7juG z%oDF{sGLmfTQb+)5Q(1+UV9>XY{z8IVm~HJ<0_a6HD3A|yMoR~OYxHufmgEpS?8~z zqfh~fp{0|Xo$Gyk1}-st?!pu5xA`Qe<)@eV!Rxz+C1z5?BPhl~^!rW`zLuSV%So<{f&L0Y+{iHwje&CZ3@hQ@5SRu5R9-T$3xdBByMda>Q1har z1ZDhO=_LxlAj0zU%6yi8F~3K&4rEG*0Jxj4IWakx(jWw@sN}vHPy%s_X38HZiuyl6 zh{+=mwQ_xm6mDx3W3jjJq3tmPZyK+&BPQ*toZH4QtMLl0H)fc?YK!WQ67ebN{?gjW z<*d@m!h$c@BxM6d13^tp)>vcUveUiGmX4w%2Dk`JIx-xaN~mC9wA=u@@+mM(F{u~D z6^RQaHb;^3W$8bqVAB^6@V+27X!epkJ=nbGxTdDrs^AWk5#3;*n5{l11pn+|Ak;-_ zBwlxrQ)Apzr3L6#AF9zert9%mdJ{WAozx~}HpVMsI85bO^xomS?tWFj48@^I4=bRC z`FZ>q-0aw4^=4Dvdz;h*e%_Xo6zaepdbfogsv^s&VF{FN5hB54RPnj#BD#esQ?#nW}i ztlD``Z^ovrM>k35IwQ;xf~!aco}H5@?Kn~ecmQ7<8fxrS1m!S<>oEP^89N3hse(%i z;|Qjv2@0kJ&yN63&ImXug*<|DWMg3lO&+hft&>kQBLSH<-$~`&27Zi7BZjh{1R1^x$rmbX3`iQig~xj!}oym8et*0m@%Ezl7_S`)Fr{eKGbfVeY=; zzvR7&iOk=+U2q!vZE)_M*$Y)aUzH;GCf>byn@X$7?bK?-)VBV^WVY~}{;ZU{KT<^uTU@QTSWoLMzhe6P`NpJf1%m zj9i+$7OfPIMPl}M@4p$SV0G`k(L=l&yHXv$WB3Hy_9p4&K!0HR@_xb2b>GNEU%FD5O73+`!CB<+ZV@FoiINnG@a?*7lpHiuo!~(PMaU=pRGpU z)!~j~)guKAN|nywpq0n|8UX@D0{vGs9L=m=q&0J-`cQQ!KUL<`v?osaDXaZbhq^l) zewz_+Ys~M&*W`f_Ud|lgT#8CrMtX?l#Sr!(4sVtGmQB1vRunOB^>fm+={m?<6xH_P zg|)~2zGSKWL)`oJH{me^{9fz6>!&=lUdiDFJdT29BcH58r|T+QZ%2M2i70PM$E8(v zP&;+R0*ahj%|yD!K(n`A`=}9xJf+wEz#N?2lU9X@IU*&f=E-H8@6paNo-w_AM328Y z_2UK!vux3(DUM~vX<=t%FM}p(?S8P_!XJ!YexCj$lx`kXY3)Mf;j#-|fJsOk3*kj` z7}%Hm(4Q+w-Re9Xe^7$wktQg1)HS43cBn=s!_z>%?A zrsLvu7&ziYsdtQrI7s`wP5Sw@TRd)!d(j*Bo|o6SU|Cd^IK%4g9YR&lZ3r0_4BjUH zn$nwMCZhIINGe$@cVwOA#nfb)eqJ!?f~S42inh6E5i-4UUMEukCKXa`)Mm?%pPuul zKWRS(Vn=I(K~fSOp}s z=J>{U!Q!)WEMZ~tSNCoa)v^RVI@#sg@Ij8?FK!}Fy>|-%IrV3~beT{)ls0-LLxeOL2j=ojv(@HuK&0tz!4l zr4Pd`M|=I!#|=9lzH5!;7beZMN=Jrn(!QJjIr0p$go~=pJgR#Bb}uFI&}sFID@%P2 z2TUm95=KoCKGzdbbkjcox>Bi7g`VD!({@%D9Sw{qX#h^xAMcpPhr(b%8}2lxYQfR- zbCjfjd~FXv$Fp1Qr=WZEO39klZSR{heR42_2G$-)!9Q%i(i7i$yt}H-56G4l04>BY zt20-}6l}yUY;Kme#WjLFo8W{nQfa>=wf{Qjqa8Np7**H?5<2gFB%7OSsk8PvqDy-_ zNg(7O?_>#IM1eh?@zi#IF!_Bf0e&oSP%)yF`WUzKeeksH*$Rbx288JL&AhmW#r^Ri zKWhy5Wm#yM4^phu2cw4xHjkdoAgAhcQ}iX0cLa^Gu@_6Ra;DXi{rqCq5z-!O4>UKR z=Os)yFj01uQf)cY4xdLCPCUR_E>q>DLnwr1&%+Dx-vm&%EQlO#0AJVIp3(R*>G z*W7Fcp^6DOKBhJA`+BtcnM!MKWb&b#%152^6Bl0WD`z?aFngid=j|qDv#$?r=_XG=YzS z?SA0v0!cQ||FN=zI5Z}gaiU=3Nd=P#3FQ)O9Bvad@ITltdrd@eA}_V0`yyV8rinmQ z!aF#_-tL5*^Yom0F}L5R&wrBnUar_&$SXAl%!YO!=(eqNT!8i@P&r|sAW?`*W z`8}Xl#_L_^etAPPE9f1HdY>Dn+ty`Lh%;jz^q3k7^jZZH!7eDU@hgYm#TsbNby z)Vw0m)7z2FsF@?yB^yUX)qmLja(17LPWUuR6O1v+%{ z1I@PU;8ZwoXo-y(^6AxB286VKIy`{nnMjrZtF=zZ$(PY$w|~iUW=5BX{gmw|Lh>UL zoa#u50bw|@@wNEObUR`a=&~fh;6?kW^cAkBLL=uLM!W-UuHB5)1?fEez+?S(rFW;0 zoG#T*R`@-~j*O*ZetQ!YMV1sM%sLp_cgBL0`4w&THN+&#vlZ25%-rEe#|vNFsU(7T zsP5zX(vjxb_3Fz3*{ku|O|Gr7CcFhPhH0RM+y^h~F)oj(n<`0&6<;MBMwe`;{hm2E zmV;u4=SKh#Mzk7Vwy{BxvMM!{DVE%dyKW_YlMbcC07q8!i&zLM3EGJ`3J$Ko4$oSS z5aB6IBGOO(`V_U%$p+J4PS}a=B+f$K%FdG>r;xm%e(f*WNS_JH1IBNARrIDicrCKj z1922vy;cY&epxH=S$F153ul6`w+k9}RK?NJkJ6+JFtFW*DVxbd?yqX>Z>9*~7zlYv zL{g>vGkSio!oTs5EkmyCqweTu2UE6kgN71;jcrZ2_NA?};MUR};toIic;GWYclX{oDdLgYjK%-j!m}$zAQ^?!JmfNVY;R(To);%%< z58+~bQrY~Ym8dZ@04m+WAa#cYvRW+fx>Kor<3T6s9cH(D1`f!<`1iSQa;~8y-Ey9G zlMmsQ+6>O9Eb-)eKI^JFl1;=3%r8VQJ9ah>xCr`SF_^CF)#5Rujqw>Y-zg@Fc{EmK z>DquKrd08iCpM$9>e*cZUhmIz!5-%)HAONY@}e~yMdv8GJ4z1 z&YvxVa+NYA^MXsg)A_>X&vlBK;^%bKm9j!idwN0@xMB()Va7&0Zu~sQva~+3^JRX45E7^{Yv#9?iUx1Y`sT&D+`tBIIvQ-|d7d-Bmq2C} z0T|1(2Y}v*kBj}IXBGX@vu*-CtL8R5nAqN?6~?LKA(H!`i)OXEt>z$rz{%aGvDjgR~SH?;3%TX4VC@>0uHYj8cgrh3Qp1fiyxkM)O923`EDm6!FN;QDHg zZ!si3!CZV=GYN=2h8d2eL~%B_j6~p8epCU6OJFGV1V@gbF}>_lDCf&n)~th^=qJq^ zHx_mY`Mj)*p`1AlIV;PDCv*CAqYv!XyA-UxO_9$W%-o-nP(=uBSOsw;d~^@Y2abf` z)jK(It*;*6rtLW-iMs3Wt#TUR5`bZ=(FpwW9=^uCk>aSDjv*~8ILVdA2+irU0I!fR zyEWu7i=eUU;2<;4PJZq+n0HtK43|Tjd00K4%|e6JmF%sm`)l3VO49wji`u9lh!u1b zk5y&a7RG52p%+T0y<6k}v^q`qDqyJ`|73fCb&JrB6!`}O{M(CLm>f4S^GR$t1O82k z)WQ&R;0#I8jR=Tx_7jf~lW(p^R9%6gkC!_@LU^_3QZur}Bg^*IY;5rgLsmmx#kizR z>B2>xG^BkKg$yf`2qg$CK;p4C+z}FVbvnYq>=Bv0#R?t8*VbYprqP=66L|07frZ1} zJBP`hY*f?jFGmUDUd`jOrpht$Ho>7@LI?N;Lf+i#K^wj_x1k*O61qb4vfVQmeQxJ{ zLyPsj6dqE1MT5qnSy0*&1~(e+4|A`@ynz|QEt8z>1HEgKjuB$qrB=ou=R}zYPyk^B5IuX!2IQ0p{>(e^2JXJ-n?qE@N*KF{xLwmMP zBzAZmy9t-mL@@>^(Zycdx;mj&4bVSCgDw zeWZ{mD|I(NSVNn3p4Bgqi^ks%wNv1a7U6RTaU@Eey*UE7-sr@0HTA|DVj9h+i_-H0i9kUcoXs7&pBX|I(V68D1}3fz`8V{MbH z)_Y~uLL?r^6!{G^x+^Z1&Gf86rjKLK4t(lB`6c+*G{E^OI_W8l` zUAmArUnK*9-k)BCmE!-#a+O9okX(-LWuq|NiG+$_KsMMTKnZAV&!TyH+hmgnfm|Mt zbD27K^gi@e%-Bv~6h2A&6t-GQ-Wv9Vo%&}j6WHb6C;`32@R4=q?P}?RzUV#`Q!4!4 z=LCh)vu3auBBB}=Ra}ayV~dtN7|yp@hs9O25YF3&yTtT9B)O?MmzPbas{>0I46kxa zf@iJ-rbl?doWIYVax1rp)-uk)+4f_g9918iX6ajhO0KtNpi-Xi@mK)T`rjr zLw}SI!_|MRDp5SGJXq&-4l?(!m8(1q$C85EZTx1@0KSBL#i!5%J|z|JUa12rvTgc> zzpH-x;L}sCH<;S5Tev4hkfl&k1;xZ|HUMpR(&6DAaBGw9CMo&lUjedA1R*>0JW^Ct zT)ofZKs?`=zB@sHOsG5vLF%&l*n)C)R^i&EXz}J#R_Q74I5D>(@&Q7GXsu#I!sTFs zj|*dIz5E0I_@G!p!7zG-1lywjmr8J6+c7? z!qL#jVe?jgfoHQjf7m6UZVbv$1DCZr&(oS`POtp}pPVhG69abXo|jQEsC>WZ2a+3@ zmlE*$iK)r^tuI+r6;hv+4Bh-oKTV;$a#&2I4YXsj6hxeL5PyA_A5j?>%R8Ez#t`cYXs0}>j)a%=5J8bcTg0u6rLpgi_ddJwxI^| zb_B>tHT30gm($H75s_to7zg%;!e)>z;yE06HYNQ5y zxHvzJrw8Djf^kg$Ei|d^2c!U@l6LpmMBGP21j4%=NH8uhdj0diOdp_wD`I`Fr(P za+8$iJG|zKzC=`T9*(@Tu{5`^y1Hr*Gk-Y?e%CU`H+2`NR&UY9g|0-s9VhOA71~r!x)t0+ zj2=`e^Yi8g=bv#Z(uHT2)J_hbD6m2ElTJ>?0vx--6Y(Me3>9s*!f!o~@~#=nQG*u& z&dq;p7xPjmh-2vor0eYPLF)W9bvwaR6jw%yBM;(bPjtfsTH_dt)_*tf0AGA-!?C zs*VaWR)f)vG5Hyg>m!cp&R+o5?g29Fmdx{xXd3ZUx#8_Ck{5n~OF_^Zn4PCSgb1U$ z&p8X zz*Qoer{8isggH2>$a-Jkl?4_cWOB3QW9#8Tp4Xp+>5u zH(a--iNyau%RD+fQI-Dnq+I|p|MR3^!uj}Q{%7^B*3=gaVe5B>qdUTh1BRYkd2=2% z<*v1g8{!RW-*E+vRNNKOcN|uEE+RKZNN>0{mZ>d(4b=9*T&GN0F}ng=dPoqFLe}KA ztLW$p5IUUZNfIgGioBvn3d>~-E`L*-L6NunYUaO^vapL+e4eKyq8TUPmP8g+_#iAQ zPWn6mfT^@ydKAk9DuWP*oRX^+_KOi=a3GuziaI`))&TZgW^ImQg?)61FF)gJT~pH%EMF$SxTnFiHRKTCSGmWlxzSFYm6Vgkm&G?t3ih zK$VdScU>yoxn>d$ZP}PwVrJ}Qtf-0w_9ho-+=jp{&BKz;-c#r3rA`K>>8ivkRUv+M zM|PpcUC|zoc4gQ3=K&EoL(#Eb!=*p;4XTNR<)wyG-cc^lO1Fdk<@V7UAE03WAdyWZ z1$W*LOEqwk{QS0~eI(%R2M~eG9g**sxwdDHrelxE)3}fgJUj_bL$Bvi+Z65jq*H;Y zaDibu3E~ny9+>6rT-IA>T?A#9SLopAK@Cc21$Rnq1_fA+6hcw$0W)_Gr#3; zmq;s{FwHdQ485p-j=Ju3;3r+T-KAezxzl+r9Cat<$7{jRf}hZ=gQJz6sC186h{b;dZETEC&pXurGACoysG@)*ee0SK4`^F9#JFsaBbN9})Dj!1N zUum=N!YdX&X3q{g2IJ5t_st>qMMj)Irk0#t?9$FA0U~)z4o4vX;TUp2bd=_M>;;&Q zS%qyJQ_*VypiF&%on-Iz1?p&>t|vG~2S0OsPUIB65y^mY2=i#`(&$VYzgGhn!?zj2 z=p+i4&ZpoR=IZrFnU3*k^b6b*q@%r>VBr6n)V76@ZCK%5D6U3dUal<&KRX8R{sHkD zi)FDT*Y~)>)@G?%0wjl*0)YUbCAmq;k(8t@$p65mF%X;(MO9dU`Vl1HIAv(#|FKIo zpUYv?D$<+v+*f^OEweSDS1wu$pm~#J{9y(5P{Sve2`+XQaq^<#Dg;=4@V%|Feh(BsE)l{A|pa$z?FaK`qoX7A>Q_iWcI}{%WqE4#X4GZNg%3VURAHyt^=A zUjcQ>)9m*9fkrjIl%n}cwVXQn(hKJvlwS3KaX?!pH=5Y~vw_f-zEJCLZF#?PoJCB$ z{X%tnXcd!?lR1z>PbTA z0|y8=K>7eeV*wx?@R}nZsBt7)YOvJo)b?>J+NZ)4Dy~jeWxRGUZDfA`=m|QPd30}2 zMJ(cx(hJQOUa{&HU9sy#YMd`1*QL_eBbtC<6nXcR55uk#jYkgCI6x%jdYML6G&hJx zDDFWSxFq4^$;WrV(bC_g@V;RyTOaQIr*iFHKss_c+%Zri<3`&8k)r^spltr}CUtm_ zcJp2Dtbnm%>BB*IKbA`XX4s90&}^kJeFVRpnPJ44))wj5m_n`bS&eDJwE!CPm4`Vm zfTc0x=Ro`$q_1!`HreKvm0#?>G7!+u>oE%V9FS%+9GuWr=;GxJm}#WPq#uS~-k_Ro zkXRADPi;I0 zJc`v(wvXq}I@S0!oy%31G|M84H%}H23hVwn>r0bfOWyv`J*!~nf%L4F@CXReJ&w0s z$!*#@hOQv(B){q1#mne!yHxZj^tA5iMmIcqKiDJUSxi~545LU4LSzx13%l5SU@hLF zeSGDck|{Ol;g{8`n#ufGOahh9q)HGVfO`UGUHI<=e?fr0z!Rahw3Q#B46AEsj5>#Jd@awgKRwd_ z!pG4P0|^X0XVkGaSR270h0%GqZDnhO#n)|HG7=C_#}ISew&Z=nl$Y+Y=oT9;n~7<|k7cf+RN#?8tVhl2Mt*|!zGF`XIanhI!h{QxZb`_b zzyVq>zQwHGYg$}BplYfO@%2usi>AhXk8ZYSY*R?+Fo_*nb<38Fllw(l8t)WBWbP(X zok|VXd3s>4J<`qXdFoa2;FAuztZ<~R58cF0;nu;UZ^dWvm3vWU=QU>`P<`1qMC|*r zAdqo61V;*3ui;u`ipD__sKe>+W;rP(lzLRlfwW6A#0I8SHa1*NK!U(`Ugs#}0E81r zn%wK}ax-w1%GxInxQYk5240@*j~u>%qA$NXexjg8!l5du(_Q@q{zHKtf10nJieK7M zI;GlNtsWuG$Yh6`dU2lJm5IrJ)g73K{lm~SKY7gqR|;y{{~|0UF{|ZMpVr~CCnm+8 zaT1-&xXH~w-nCDdmg9{T?|2a`0!3%|n8^SkiMYQ&b6&`A8nN_wm<5gJlEK#yhNC1Z z{s|Hy1GafB%iGi?34T&IA5GK|`$3!ylxYDmag+$t&EiY|Q{K$Yu>px#rmY4-5h$h1 z&5#CmIldVDo3b=9=<$z6?_j;UiLPoeIIddy$+C0xP^e(g77F$YA?D7u8EbceSTM)+{UTXU{K23n|bbO zy~QBV=!(wKJwjE(E4vr$1V#~pz|;!0b_kZ)TRu6q!&e$8CO(D@Y(|Q@Wnfzm`hBd#AP=&m>8$!$!L$* z^08cwuA#GN%3=>uNqKoaUts}xp-}4sC5}9R4i)MbEkjwaVX^@!Mu3f3e9N|c0m`Y% z!UcP**>lJfpVYmB$El4MtI~@;S{`t-SgnNs4ASs17!audzLhSayKQ}wv}&vsxUGx`U2~9$3y=W8`8tin6;7HSgQSpL zPPbp0^c9D0z8B_w>^U~jrImA$yMBKCc|9Nu|Asr`f$DdOi2O!)L>qPlT9gcg8~k^r`{GafPl9w3^z`QXid zXv4v3!Xmr&B}x5R9Q?L%(p}1gMYHW$LoI&vItJgZkRKJb`S_e0^ec4^;N;$!cK1sA z_<$b3Au4{Lxm^~EEl;?id?&7|$IED!qdF-+C+n=zfjjxr9OhiqR%M2)EL~rkfg~l$_S1 zFUvE|;bq3;d-1m(fTc3$Kr|ZtVaOKf-qGZ((~S=(lcO zy&AFs;_(qf!sxA;Ge=+1TOcznTz*t+E6d$1^4N6++WDsdte^D~;a%e~_SC0JU6cr6 zR>_?Z96>}YYv}2^@=9SnY8(9MFTYkef4&A-J#3>^={;a%{f69*!4BpKh-}oiy7PCT zXE@3aH8#%iI89*WJdUVbPA2}J+wb-kdQyNe09L%DIFU#TrsWy9T4hxeB~B1$%L}6S zj0tjuFpivt;52?IC&`GRun|X7#el#wRN`|&_-(^vik(w?v)o!ap4)Tg4RW&Ys-9q~ zd}Z^6ZlV><5oX()$qx?duio){l|$d{Xu{c_;|&B5uasqy%qU>2x(4Ko=X=3m2mN2P z70Y-rgYxQlTL4L>&oyh}%>s9>!g#HuZIJd`(^`WVkT>yUmlrGdx%d**utM;o+0|TW zL#^m;&-Lmkm_UzDhP-19CQ?KXe504<_l$VoI|;!gqkMI%bryzSKK)j)ADZNRQSX7e z?#`nw7BnPLw^qRyxJi{JsFFJr2dGl{4(@&1(%IPR7c3x)M#V=umEPu*7#5wd!t^CYTUwx zl^nw_WkpHZ&pHcuL}er%{zh4=Te5Qqx}9MR0l&3++n{?Vy0&j75Ok+nfNXtwClie{ zjCOSI%$5q0q(++SCZpCKsXQ~Rqtq7ow*8LS0o>JTH4q#jW^cwaiDR8PKALsd5!SCF zWxI{DGaB(qR)yNS@+NPi6@%BL%WPf+Y{}`ItxsJKHrdTkWgHgld{UcwwZLo;a3c4r z&kAGEN@=}8{jMi&KY+Jz@;9j*cDt&Y)4s%w~ z1{v*l#wyp+QhPHHzQ=8qi>U-=yZqoScYgrdjR>D}-~S5#Qk z8I3k7D&#J|6)KmTT?*aopyy##+6i6+DFq=ceO9Bu#y{dvB_~zr{;Gu0J|VwLI`Y5zO_>-F?f?Gp#g0QVVD&{CN!!Z&=+1G0woKuP5FP<7oq!if)SVKNP!7t zcSt;_Qh;3Tn}3W!G>RWg37;VvZAbr}ZOnhQC*15waeMo!a(?$jUBs>bUoDXVL5Kyu z*HH}zGeleD=$O(y{YI345dqb*d7~iJFOVWy>uN$s4Mf;nL|s)6%USamcA@NLN$ndHD9EIhd{2P}6#0VJ#KN*B5{5tzn{p;+1Ned>w zY~R0q6Nfh_ErpVgh?sDUMhevbL)^P`h?2qldi|dqG{CJVDjIGom`Gu>B|!xT4WI?NIr3o%m?J0p~{( zRs%Eus9_&<6Pl-LaiU+&Hkmn+PpQ_=UWz_V6toTs{1L%PHbrn%g60#y-rDKgb*f|Eys}n1iL?$ zcKu7>eAo0n2wq2P$wn0IDlI^>F4EX;>>XOXSEyjZA3ikfl@~3Yz{ht7-El-|8??cP z&ncF`qk}L!`F_B)YaOfUq1$Rr(F}&(VO`F&*M8sgtzn(;qf0<_*vLPIi8(V@_uh$- z0*V)BOM8%NBnco zYz$E%uI=|OBAV$!uwic%IE7z#R-_UGEb(R;Gxbupz9=owyyf-)u*vl!bv%fod0)h< zJd$Y~BwKnfF$nO&r7c3TT!-_x2Kk@BJKp@#SW_Xra~-T1Cyg=lqnVGJpWcl0QvzlL|T-`%kzMJ>D; zCy_*%jl=DMqUz3(nXJoz`M{<*kg0O1aKJg@#|yg{Sf)vSrpDAyz<56iV~|)NY_#)e zad|#ze;w~(KDQ(sWXe*jJ_QQVQIv$!yN z{jUHM!EEWmG#a0Wfrl7sR{O&MK*1U>`HZ@p?%|p&#=J3m1~l{CluD=Pvs$UzD-kmH zjA?f*2j^APs>5O!Mob7YLDy5p|or?%VM z7b6%}R6{p7o?`!kw8e+Gn++WgPkN|+eld|~EL>Ojt`>(m)Z98jKfDmg6H<)3$7lkC z^wZ^C!YOGdb=}{F3{lFjRC;`25$8HIjxC=p+PD7+EsgM z*Mmyp$*y^M1um&mq2K%O<|hXCz0JtGFL5Lv?{^q(i!D|3!4 zPU%_#+7mQOoBK;>Kp}>Dk>4}WGbnyHDtbo6v2>P4Z!t~ZojkC{o8V?cHRTUNFSOEH zX)PDcKsyN^iF~=?K0H46FFPMlgv4h#u<*>fu51?hwPn=O`yB}B*@j}atV1MXj*wRL z#A;6Wy;L(rO~*d)!@q29r0h~D6jESK3m+|3@Wk7sudyPVy^_W2Ah^o^bVjmV~i0U@*AVSueQXy)~0SBfM}?kGFlgbEDgVP^}4SQ&TtII`Pd}YA$`M z^RQ?x6^v8ng$7_AVn4ZCL9Qr_f7|@)Lvi=b@WDY8%u*Fx?d8WYlGkvd5gS$X@Ge2= zA5`B%HQO5ROONofjJcD8;0#a9-V2Gfj@Ds(YJB`7v&JiEx%T^ScLF_2Dj^S%wqV>! z(_X@}onE%1z3+IRb7cL2>k16SF0FgEij~ay)bC*;>jD7&ne^UqI&IBTWUR7-5LO>o zqCbLtvft|n#HQVChI zWow%jcyhkj7$GYj$XD+P0QRRswASGWc}@Jfz!VEK8X(G&h%Y|O|mpW zRNIksXiI)S`zKYp90ql%-ShJw#x}$w5ZlqqXZv6JFwv+!cEAjPi=($^BJ+yxP5@l` z3*&*jc?|TbbfS+wm`hK5@|)@G`IMW0r{hSlE9K2ZweXFOHm|CLDX8dj!U1IfQn{>*V5i6+r1>mlhb*xp_DqQaK#GiMGZHZ z$4Ng=mR3dAc%gn_MkWUn##|+8i);Z5K!nse^S0CQ^T>jG_WVfUG&<2f5rIkp=GX_W zmiL=XiI(*9QdCO^mwV5Te(vzf*q7>CoLL)QSlFie?L9Zu@^P9>H4wfJH|`j{o(jKA z@F3-Nl}gdyn6)_&G-Y61ZRODe=}K^Gi=uLgOFY@m87NTpSr7*~=C&Q)&ZJCAsyX0# zqGI1kl*6E}If<>vmJ&{fc?1}wQuD09EG3{pi`ZbBe7a86jpzC@6Zfn1Qg{GR&SmRV zJncwez`IK-5LBd7VFHX>`J5KeM@kGLfrN)jB^$Of-PnjLBo##zEaG#PzrP_2h%|DY zj*B4U5G||+XE6o%=?jBET`34?ksf7q1Rb_D+2#xTe%3A4KX#tKK1%)AA1(B9 z_-~UF2~8x)*fHU|hh?cj`dJ6XZ6`ps&IUNrC6F%?^t!;MLeGsLxqmj=bMM-wy z@SzV>07fA{JUWnd$NSg#KG}THJuMOVK@6|1rRMnXA1RQ5kU%8cp)dmrCF;;8`GX~j z&g5v%^W(WE+#Ctt4`QhAQ^;0%uhjyX8e?PbaZ%v3UZQl2#C3iz?Hy0`2JMwE$0f-t zgr^#0U(L<6tgV6Hj*M0vuD7Tk*FyGo-OFIm^d=jW=GIJgHpvYrO3EcP4WhuN1LkdE z*Ldcg1rbqfarc0iN0afkid&S@Zu_(!iBva7W8wb&>W7hahZ6;<9uS(Rqd%g^LdnNS zNpj={J|Gf0C7wUD?uE0vV}&L@5K2z1IUa7Y#U3c3SmNVfw{|YVnotp|~=tQ^()cp?RhK=liNzmYcsK}fi7}oz|un9&;FwJ76 zxz;$1AtpInyrEA_T5Z7n=m~k2G1nF#l9R4@eflL)Y_NUYL(()XOe{Ydnv*IM)CMe# zKY|_tHWO6}V?cwe*Ah*t=R=R!24t!1o(Gl--6{7aLhrT$u9`YgXD0Lo^Q7q^P?5SB zJ6Tc!2fo(cHaOm+06D0FOv(hBM5rnfaOZhP*9|-cg5@g@#DnES%;(PG5|2xY0zx41 zP>aI^S7NxA!&Q##$;sJaWQ49|*}=!R5~LIDaF2cDui}*E z@cY|D{d4r1gda&O9;lyav6xia^jJrR$fHJ9@1 zKt7udewEMToqnc$eQbYf%A{BMeS0?;Z+GGBdBQ4pn{uD_k(x$ga<(lY<1(BUbDQWJ zi|WA}MpenXa5tW{>V(A5>$`fTZ{&&99*XFe@a1YwMO`pr&peArknADq9(q1LT`pkX zB025;z>Bkli=1JaARmVi2$_s0QXurZ>UhSldMq+<;yO=T4 z=oPDsUR`#wgHma-fTKIMLu5D)m}0$xkI`9L&V(y9u$1? z_iQ@DKdNI&)70jjfZ0n+pY7cFX+1XMK}ya5wCH`R_Er;)vf|p=7q=;!4LS8N{U=r= z7r?q7fllKvWY;~MHc4tVDHaDxlo)FVRTURkeT;k?C5hmDoLwxzs-0}OXd33gWEt1S z^{Ghe*jsZ8i>IbbJzHMs?b2>bznmM+&wGA(^?EgKr)IzB)`dkBJ6_0c5@=sJI~Pj2 zythkPlm_C*>DWqoTAHbWjFWb*C3Dr0=j~Ya1m9D3-!Uz}m3%b#GcJK$Z}hI-yzYQc zlAHZr&vwpm{EKaLqJizJQ@q`?VL1hAJk7~tQS~R<>96%oZ46#8-*>pH*oSVXpEgzZ zE$b-g#f95ZZ$@VJYjijBFBc!aNy+Gw%-XCAP;crH$U&)dteckN>WkW5nG;i)%(!5JQ+tvKTZ|e zUQQ|5>PG8XM87R`5_*wibJ*;M?abKcb3u zAA3OjA$^qsVZMEwRg|_RtvXQY$a^!LM-NsjGM!gKv(;CpAzTa2vJoiy5u+ z>tM|4#OxaT6B29mE6s0!W0r?eXfB#eXPEqu`J9u-7;{CYqw%(;Fge3smEZhFyA!u2 zvWiT~&jWkPPn=IKa1@f*kH)&0EcJ4EubupZQsvY`R!<)Jm4?WtDnN+7b1sX|woO}4 zShyFx2zn@2>2H2pwl9`wutIZt4CJuES{hozR zH2=Xnh)Ck;x+hYSB%NMGaw2ruc2uJ>{$XrWaQw_U%WH1&=AlfIl5I+J`}ONMADG%8~QqWC;^r)$E%Y zgJsU;g4#W+pN8u)7*PxLazSk^KtmUYl=RZJILp{NaQ-3vGp%e+tC)-Z&28&ZDI1b0 z1HUKG_TXYi=4WOVjVz(jDQs56SMGKuW~(ok(6PF18_Cp30JT2X>tgHBV#hQ(B?#~= zl;o6!y)N>>1tAU9^M&dWmnlUBcQ2Ruh+NjU$XH$b@zzPyqZHC8ShCNZbA-D%s76Cu z4T!C*H0qAkf;w6u(B=_Wx)n(t%b<{A>wp|gUBv^IiQC^}vhcYU3@L>K?1W0=OsY1< zxqb4xC7~}c2#xW%zIm2b9eOo-JMhAO@*2EYeDD9#8sbFqKsxDIO12{YfL0m4bStM+1BT=}*wXgW^=78d=M#pli7r+wVs{+{apETeL&E zbSmY2Z($G2*z_ds3se>@SG&@J<(Ff;k9)Yx|-c6DA72n=1Jo61*`NB8N9?8vRvPOg% zKg!%sjw|VC)~auytU|@&q?O3qZO(;nVl|naKj2r+e+le)$o#n2VhAFStUb=>$ca}| zv>{wU@9SI>s)9?=u{b}+p)j$?$pp3~2as|H8Gie=ctH~-mTjrJ)DdaiI+(eCWF7XP zz5RnQzrE~yzNj&6y0CBlI@gT@5vCLAlNPOPY7fnPw?j=X4QB|UQT zliPJ=!j`1s9v9{^N{xMy8q<@YuA}0Dg<-ANkB@l7aof@PCMu0KJWi&> zByhi!+GcZNaXOlMI7$uD*}}d=pLtNQsJ%j~ZahPj5mnUtVwDYKY2&Jua2Xi|y*k7_ zaWz>mSHCxV?pD^M{4>gpv_hceoF6{WJ7o>|Upt6+j ztOPXXQo3Cr$8sYa^^m=zT-B^iMMf5zDtdjnTkVb<7K@BkWhP6b{y1V%^3 zww2{PwIS+Pqb^$Trp1s&7Uyqs5+}snrtY#3Omm_drO+FVu3f$iRCpin$by&*Gya}T zws(9L{K`d(K0!G6F~?>2DEc#eM3ltB{a}dDAbQBsqysG zwouOikEQ)B^`>k~sJ@ncy2yulNtlezfwr3-MtCH3U`X-$%Z)Zuoj^MV+jU)>_)JJV zxqXIkmPy9NU0d5y!3x=D`odq79(X*L9F(nx4TrN^GFELkC1&nAev&e7&_b&Z)s4On znXZmrEyPr^VAjV?^o-Il?i%4MceQsxWkJ3En+xF5r8V1$jx{eDq1MOV@LLvvj2jjd zh&m~U*n`uT=J16}Y~4IGjsK-$1!9|F$=b%Jy zWNrOKNt8{ge0qe~tjqc1ohA0Pb4(}ds%kmj%AER4B46hSG<$i_}s{) zVHOJb3(;`N+(U-md{FLrbOB?6$1gK1@FLUge~L^vR$$=b;F3U7?PJf;v%Ckd1^uJ| z1uKa^w%Y(QfbzIhe*c3#F$|&*7tAQQ&rv`=wWYq`l_>M-S(M#5p(@8F0j}tCD-y$_ z(C3jFUA?xX_HQM&WqhRDGv1Rz-NPIzu`Jg4JDDtN^LW|XtHNKnSc@_+I4=#_GMN*1 z#dg-qS?r!1VNEIo-;fBv{0XIjBAe22=BGJ#o&3t}3D6Xf7#t?Kb?E?3A>O)c@DFg&fx0=UbAG}`V7pifCsI3t| z>&v>-J3KDu_^U3qBIVHBvvz;5P^NFCi%QJ{LzFMv^q9X(VjcouzS6g#7({VhFNBEU z-&&Fa7c?h;pm{C7Y*J`^=W%%PV9P)xl(*DywlK>%mfc+74HeVK-7AE=>Y}gbZ$)5f z)31%hqm=rrb`fM2+T!q?Kbr29dhD;!;^xRiez>HkhDLCqgHG~Cjr3(7p6KnL#K|_w zpFAR9FrOMUq}vNuwSdu1JBA$_VicAgS>%1ubPF?5E>cjsuy2A1HTaMQ13%{erv?Xa z9!-oGb*&gO-qEh2UuKHbZ}8nktnML`xL9}=TdL{2ks^xV&Xx`>(-fuDdhe+TH~{} z;C(LSfo+`Jb_D*_>Lc2& z9$0EG=fZc&UO|~ZFo1EU>vn(g_yXhPakPw_?`581UM;df6Fh#Fj1sj{ zW@75{nR(@`*TCEkSHPqtIiMx| zuQ$rVY($Ea{>P7&`-&d`X?JjGNYAIe-B;)CMH{i`Q6!7jS^A?{HO{kl=!O~n$*T-n zF|Vb0CSL7N@-OmMo~-+YE#hG}YW;|omwfzAb-S$Qq;XSwvXn)a_3k~&mRkUU&=@AQ zHSGAk4Cmgb>~0UZa)-OIa zx`+oZ6o80Y^(#7~Wd~1MFWHpzrEDPtX9x8WQ6enh%r>sjgmmpl^s?WKJzgtVRWKi5$51)qkG@63HSy~C75WN`c|SgUen2}+on zN$K;xBVBs^4pTaj!t%z~_eg@aRn>k?H2S|9Nx6N`C1TzVSKiqhwGG}dZ~XQyMT?`X zjsC>=+_f-Z`(bHjV-o$boy352kBwf`#Iho_MAO{Eu?F&Zi>@gAhT>Q*AI*Eu24t~c zg~j_ljTXCK1@d^7X%ywZ0Wx` zR+LcUpips@kd3|gfRxLq6fVYoX&ce<5kCvEeVITsxw~D0EuB_p?YJdMO4|3n%tS9wyAiD8lJ@k>aJ;~LHftkk9C zX;iNpRuA9+paJ5M!>xEU4bV9P+LK)X-#j?@R9xarmij^;B?ftEG+-m1d|#Z%?y{xe zXJ?iHvvts%7ca`fR_;dJxC5bL>j%vkyZ!Vl7!tWUzZKekX|1x|F|-U=FVL#t*SWS zyT^dRZF7B8+h4ys7cScAM*5AM_4gS`2yg7sTtlUQCo2=eN;r1IqN)kIMa?y{iAUZX zkq2D{cKwf{WBBnGhn5zJ(p0<_GKkW4wO*fj><&F~-$l2>iqc}sF!g+4V#)qIO>s$~ z^}0vF;5xhiAW9~l^-&4wL<}|4Qo|z-zpIn1L~xn#b6jqMFu40jpVpE@ms72kOvi{@ zXC%g@e+VfucTGIgRx6`rjCB2@?KcXpH%D%NJhri`b7LktdaGWs{@T{{48)0nz2B#eytg zTX@7*TQb4tK)J7PQhuqbBygSnZg$UhBuA~J^kQ;4Avt&}LR|MyK z2ckhVMO~ACCky4+bi%C^*1An(Qj_zBM4L~?-3U@9US-eHU&0n|{=HzCHn4qmu|i`* zCSK$b<9T#w&TdCCU8KOMM3M=H1j?|6Z`DAy|p>n(| zE$)LpBaZVb@bx}J25yR9&1v=Hv|#JAQfdAk44fi}>LAXS3|!9;FE5p8Qg?d`Cu&Q z+yZ2Oy2*&a-!}@FR*=sZ-z&fx2-|@FZ8z-wi$8QDDqtEDg&wpXlFMLOgCI}~XYvFn zsutv#LcSH!f}|#>N2%kE7?Ih|MO?U4v>YG#@eHLL$}C~!FSk1IJyyc z_x%0po*grTe?Iji^s8yQ&o4kQ+kz9!CdrI$tQuzSAn;Co6k5vNx}!Am$43~>L?VN$ z%RPRp%M;-0^2cA&=oS7@5EL zG2k*v9dAwUE_D-Yrp{pN3Vz-Pykd{gR^G>X(Iy;J5woV+K7|de+{M0 ze*Xdvyeujl0w}fH#?O_Z0zA~aBrD3(Y=(9I)z!SrF?dkZ&IATV9+|O^#z??xBouC$ zJ5-={_mg!g@Hy}b0v`nP1zUY%zHM2;Jq8h^!jEp9-#9H;px5k%EOPL=T6mO$Sim1Q zU$dAfiS0?WzyGb4$otsWSLbX~e%C$;%sRNBFZZwjXiwb^+HWg=veGCFL(zi}@jESi z1xApEKP~&SjX}rkmBG9ijJr-RD?!!()>^DrUAcX-I}a>;3$0!Zfrgow7Z(??3~F?r zzPqIUe9B-cLk`;R1~fK`ZcWyPf(F{pQrx$vhObzzlXyVK>M+okcV@Uy>tyByb|p3g zs7Md{Mbs2%%#pB$5ElMK4~a2iCA=m zQgdmR=!XNx)=bd%6?hn@8$cymVgND5_;77FU%M@(-U#Da2* zNhCTIFoO*&EiH}Ua^Ly|ji-_2O9rci>H$K`^CQRk{j6F8#eZK20qnm%*3o(L58|zZ zOA&^TSaFL4b!Wnl{(?~K&aE;8p}r4T(}qg zJRS5qibkYpdeCs~V{uyjb7DhuV!`_K9PO4L5?EGvp&GX|ZxLEMHbf!Zx!T^gy#6$K zSIqU+6Idcb^FFw4!z>XISmf{Xpm||5?`ORbCaeOa+U74IAvbACcI0mk&A8C!RpOlB z+hqGmfk>e9+E)g|cp@ql9N_v~DWEt~61#$Nsu+1GiX#$#Y1faDX>~wFD+f$;1BfgDtmDZv+eYi1CAtj_ON>b7zrOG5?1Dz;~ z@);xT7;SGjOyN!3Nd#Qb$$btWj}sY#Z6~VunJwQKiFoYX0``*w z+$`zBUQcP|Wr!ltaj0lo!tv0;acRW?`>H!IsE@KXykAh#K>l)Yx@Cw3Akr;`E&3ea=B|GnIOd`FSX1GPnp zWil5P<+69c(MY@2tvLNv(W&Ffj&T&TF+A$i2T>p4bOuOY{H^B(exa!FU6kO@4EGCN zWTh{k2n|E#%z7>BJz8SGVf6LZqymhJi6&wm*cmYj6=|1=-@nI=FDnOuKw*fdhS^MT z?hB9Y_)5$5ug}gMujsMCN&HSc+ zw@ckR12f9K!eM^!zPNa}S;pnO03C$?jBQn3tZR81&KMVlWo*JQqrK%sg&M}L4>H#n*_}SI{RXa?aJ6c{WOQ*smYV8 z+)QKd-_|zhG&teNShp=+lit(*6Bx^U4TDcxKhp@0%0G!$7$ayUJG067j~$qi zIz0Wg23#xJj_>@%_s71+wZipJj2`Agy{agTP|BM3sr>`_>oqZosmv8JTp`k}` z;jW$KO9py7{~p#GCdUx`V8a;38u+FxJ*5izrw*kr2K)gysMIqZ71*HGC=eAG;&oc# z4*wj7X#XY@tj`6`SXj&Yo#TIA+y=LsedAkJ3EG?fne|wIG;|_5Q&I>|O~`-v+K8Lk z|7Av-%_=i8$UBo*h&N2K!1kzt5Ax>cEOKhQPmAtX&yr zgf)0dzx|aEZ+kd&-k$sy%1eiV5$@IlAGeeLF)EP1}oB9E!{riU%p3S0t(>!z=C5nA^z6t&hGbUhZ z=-)}5gkM-UKQImWV%dKF&(aQnV^Kb$f9)<>u)AuIo!oq zhfn^lu+l$Y4F^D9RB5EYmoe~Y1G+V=*G6~M=mF{sia~>K9sfvxg&jB?+@${=Iu5)? zDSN{*!@nmIxJz07fux{yDZDI!`@?uGC{XIn&aTnTn&SW)iWRkI>(9`$;ajZjSO|VO z{AE8ScB6gy#~O=f1{Xdff&B7m0FmQ~klj2|Ryr_stzXAT{~o%SIy^sclAia^LIfuf zLSP7J%;|sbk=2gimy(FKbco=I1{?+p|IBWzf75N)*9s;7v}M-Pa6@#<*@b;|!gOq` zEri-7mA;RWVfgUDeiei7uyWTI_n^&F-5TgM^DMyu9fSA&%FYlt^a?mj5eBBL3ca-IuFS4_Vg%AqE?Z10Ejgn z3~CE%wt`5i6!go;(*(BGyW5S|L~|Dr_t~OBjOt1Aga{-Oa6G{?7VquRkOuNJ{AWiu zO3_)6CzA3|HY28)*+u>)*XMC|z%bVmKCekeWf^Al5ws^T+H|NkqBe>ANr;0pqp7%& z3LexUd@H1Xt`G))DT#3XW&sPr7cjlgkpdz_DpU89m3D*oXT{|C*2uiL+h8~%kCT|6 zMc)Y{6BZy~(G3FKIlckgU9y%6^RAEzi(V^a7zpMa7djw!K(iBJuOoHJc|bvjcSB*+ zDtW0^2Bt7+|%*T1|l{jx%2o>O2oI$h~(E04hot1Y_n>H3J~2GXvgz z-s)z(iT;Q678m{Xi|e{_i9Y`gc9HsDvC9)=gj0R865x5h{xLI-$5vNC;RYOn-~`VJ zLMh<77KDM%m>*W=va0HRy7(Hzny5tFRzcr_F-^gn81IJlQDp~II&O}0f6DVZ-V*2s zta1>^F?xi>QqFw`(B!}a!dq_kJv;9H)zuYaEb{SbgU>nF8L+Lvf)K!CRXBkdM8}MM z9iL?qzSC!TXN$MYHiv3hh7c zZsl|Y2>`$h!uW3i=t!!tgP(QJ9~stp%zgxpF@#iNGLTs@x>el%zAOo}sGgmpqqe*Z zUknJrf;|JT-7m3-dkkE-?_}brWL~qTNx8|ybznmD7S2I<75J!UyZ{oqH3OE z6e0>nei-28Q)P=eWTb3h*kNJ4iZy-A|A5mB-EHgFvnsWgJK@=}VR!h^$`r0_t<*fi z2~Bb7VX-)I)O@wH5fW!*3$%W{DZJR=9pKP;AWEw%i#V$Yc2NyA)-fuefe5Pd++20P zQ=aC7@ce3gHYEco9Q|;L1z|!SEeJAjQr=hj_5do8En+oNRM6fYfiFk>fM#3*8B47N z3c$JJ<(X^Y)gq-K6;Cp+eYAKm#w^%IRpkofxPr`)Ij_UmIsSr7O4Rf;hZ3raYeNO5 z`_9tTTwh#P1M%n;>*i^6yxuu4_wYF{y*wq!c>*XvN^$CV;Ty60Sl{gi`Q;KK?hQ`k zKK}-~j)UqK)9soRa51@ief7NtRv3>=u;2$T`(60uv5Ue&8Q?$#nyUeD%PCDJbM_)f zKqjd9>~P(2rH|74pkG>mVYDgT z7!9MufD}zD4RD5iN(rSRqY;p>i4%yQGcFYR3om^G=hlyYdR5+)Wl&UU4#8>$Joch- z&O;XQDC$6|Y*@I>xB`?q6_qQ6GC<}goJf=X-|t+Q0Ag+eI)ojrY5}ct#77!-c6Xb0 z=i6No*{DL9wHemMIsy};Suq%&G4a|@4ao$THeTtX;65AV6qkTuKm_Ei;Y?qx(YBBJ z%tL0O=x7GDZo|P+=2%#=`>_vk!;NBB&>+$PG?iew&?%+s-{(%`iUa8l%AdWo-K{E6 z2?9C%U5oL`XO_QeIJ|v)G!7t5Eh^A=Agj`obiaQkONfXRlrH7*il~C{y z=FgM=i46Zj4KOFfa>9IcVmKA$>_KY;z8XZrjSNC*{M8@fi^F?nc%y{!mG{NA3XA(2 zV-ug}5dlxnfGMh$M3WWMtoY34_qFc<@UQY~)~%Iz4CEk{rHFYdtFgR>z(Yz zvD*1QSAbqn?gJLnH-Q~NI89&WDxmq5?g*}=m-6hv={k}d9jOavAZdo?}GW=yuwC|*m++*HCL4z5Ue zP!&jv6GEI+Oq@f_t5wz-SA7qFxSBr&j4)i;}2f|*HNy+P4 zk~Nzur<3w`Z1(jHh#m+B*^)H>1yhl6cWyzO&5EeMp^pCd@Q}gs&C8?xeF?DaUKlVZ z{I@7^1_~j9c4I37Yv%tWLJW|hvwWl2uKM48z# z*1tn-Y(@So;(xvD)y=M`hU3`(+|@lz#9b$yuU)yN&f(C7z@}k zFSmi_;BPqoKbQsa{~2caFa7`kZNkkvEz^xCs5l{;E6%ATVd$aJ|5V-=8flU79Z3U+ZpvO!(E%;Q#ayn9=_A zXhNztXI_V*k&z*}u;h~`Pa=roevTb&xpUq{L_{Q^n_0v}J_*sE{3}N>m0Lv+vMh14 zPeGN|^-lK8;}h0-hhNcUz_Mnhu5i(nQY9TS=z zV%i~8o>YHM3epy}iGd;KK4O?RK+F-WEE0>>WQf$)DI~)n*B>D_Q$m3(kEavfmzi&j zlLzyE0EPz2wYR>!QUEXjm$2{#2I#?I0F$rVRTBI=Io%AZL-DHKg-JI$wb3>L+c$xy&G26v;UGfk%kOdDnS0~F(+q!`6d%Zx zIscbT8Tcq-Pd5@7FiJ9=o9we0J{+GYL*Ez7T`0u}$je;426=4f>)+_}6 zwkQWg(vYX1s*XU0eRIgY25k{%ofbBB3=ogVb~U{tiB-k(cc7cTg3X^4wr*R7xXzFcr-d(HQUG6L{1H?c1vN4!AWP8?~3b z66{Za(4#{J9_+FJry;mf+xBm>+}w8)zp=5)!V)8D@UcRg-H|?GlIiUD?zOS(sC}R3 zDWuMG_fAD}QtM^ca-T`R$O(;t-7!t?WyOPS^Nxs?L1i4bLG_ zj#pb8ZP&Oi@i88fohz_-$2q`6?#kGZZ#Hzr<)CWfd=pa7`5u)w__`i=-yi z9;d!aJ2v31)LUQf$Wk2D>APZDc<>nQ_?lOq+kEcYA%Nd0_a^L%Pi-UtkuhiA?gg zLv?890*W`)D-tc_41BBYet`!Uec$?K!Fa(J47f5%keSJM%m$7(`jJ``67KO{MjP7| zKDKPUVy?@UbPpT(7X3-3QpY0aD&HvVkhFMNIq$AIhQ~gtVSjeh^%jdT?`d_h2@RBb zrrSDnOwhmSy@p+?P*68$9>o-MX@|{{P82xvYE*%R{-7dj zEeMSt{1d`GZXvH3c+7JBuhSV3jBv)1{Z_yX6<6|)vFK+{{6a!`S*c~eVDB6TY3#@* z|FGGFQ=I(#6ro~YiCg57Ni@!t+QJVbcaKl5!ZMigAG?zH+dNTk4au5x@a)0}%Zz2c zUM4!+>HVQHa`;%IIr@9!+)5`uRVjr;R{X=uX0|Z0!HHiP@;K6oJwa90ij;P9#|Qg| zjc(AFKg(3g4W^CG!wG~@L%fqnBvag)wpN=DY%ju8A9O>Aq&0u^`%c$+h9ff!tnt&x z#G!I#HF%npVhHjokwde&9;jTten6CH{2(C@gGtQNCknebMLWZ$;T+aBY1y_xjtA5I?%= zi;@`dE1k3K9J)Uo`dD9n?WHez7?YsRFADukMh)qr7rEL{zyG~rvF(%Tx#RlAZEyWz zN|Tm|7r0`c`wLM3aV+2V34gitwcTF3n|#q*Y&e(i zgZ}C6VNl}7RynEw%_F-gz%OOEzDK^))McSG6&_o*_?kHHuIYY$y5-ff$fNm!Cw>vV zx-QZqxQT}q_pf4#Hy8#Nd4=305(MVxXDN?F>l{%y zOj_GsBjv5~aY$O$y=EXzy1ZcFBw$4e_w~G55$SXzjy^aQ#V*SLi$u{*KQP5tdUx&g z(XU$XyY`0l-?Su_DU3T^JbkTEU56ZrS)}ePl#UW*3|)-e(z@Tv0mtHlW6DE2B#o$_ zZVCo~MEvjE=vQTAoA;?5u-($0L+;C#a-q+DX!C`OQWF|^fc1qP`uZ{puh09Fo{?(z zMgC8(A}PxgTZ8v_-MP;0Uoj2ejAXu%vX^<-U@i<1s(QHgOEtHHXi;}Bx2sRK2+>fy{{500PgZnJDR+k!IduGx6?sN=9kX7n+ zcM)84U$ix?*jTzV_z<7#ifjTq>iUFu+x$lee!ZS}OI$>K+ZzG!en>;OC0 zd1c{fugSowcl?P!LC5~H{SrV3q|`jC(k!FatV3Sfe||QT-hFl4O&q$3=LemCt2e|W zecBP^4%mB^?9aFjMZ7}3f5OaeiydS9xRESE5|vvb{o|)&*+E{e!0J`Dgx}$I*Rqm| z#Sqo6dJj~*#?I2^x3yMCHW_^AF?I%Obev5Dj@ef^!;B`~mg=}DYt@o-l@DIvbHQf# zJ{xn~Wlb-E5COf@&xOOr;8zYdhlsZ4zzE|_e?GFRpFW>9l$x(GVVeVyTFzo%{+0Rn zV96<6-awY=n8{lvA|uh|8B<=#9*3ezj>_7CX4^C^gm@k5>>Ytq`S$;ZD4!cCAw@e+cBbjK!X3z60 zChH?}#HCYdk~{Nty5{+Cl=3nUd)s(zG$U(d=af;Zj8Ez-NG0gzIc@jvq@NadNw;nlmPe2gB~vd8}b?*X$kuf%k%~ z6W})&A^)2^y8b=o;x~D;h>uj46Uo=FRI1x}-{G|31$)jX6=v>k_h7*(u)gkgKx{@hu0efE9zwKyY&R@hQxGk0 z7K$$yRpPZ}vOJW?gFCdJ;(H+!;SOr@>KyPBYx z{r$JU+M_ejX(x^~u4O%C0r9t}O-UJQDm5R!Qg<0L64w!mvT2>Yx%SE-q58ntcze22 zy7N?6@8i{IkANaGN|a~=ODLc{4+`Mk-jfS}t<ztKuY!-uRPYi#&FaM2UZ>cHr z5#15{ssojfPr5zdQy}4BcPy1wskL${i2J_Bh*rHLN1aa7y(ev81ugtu$CJE7DOPO^ za?aCQCyyANlYjAMi&unv9JJ`|NO{tmd%ib9*S=4z#-2f-^C@pr;%E$>+IG=cws?uY zZ`izYA4R>?r(2HSq9}ks>09EX%XZEMEE~68>h$q794TE0`}&u(8#DHji;8u%qT$f1 zsLDLNP}#f5Y}Ip^u~BRU{o4 zLX}3wnl+Amony98q}={tQ*c=cjR_z({3PGcaC$kA*m_s*@m-x%)kT<+du#SsT_EFg zG2ch|q_5VjG9{p>NGg@sfcJ@9&WjlZ%lLTm1Kv|TU#2|~;N1US1;47To&`F;7AKr9 zVDzDicE07H`sMo1T|6tAxCB7$r$;Ln3cK>O!zfW%-{X9c zn|!OSaYUMjsaLWwh#Ic?-VYc$_0n=R8eLYoU2;}ym@Bx(qQ9_0%C(a5k+}@}!?Lcj z#pPlp`x%g?<~x`Gd5hlJa=BBk^AtM&ZK~K)M0KI~q|JD8XERV3{;LMtWK|Wo4%Rk_ zEBZzU%TT_|F`5{Lqj#yNsJ?hh`V@%dy&j0d;4)az7&C)Va138n_sAsizdqfs5F9ZE zEd(tEEJ8s_a9l}bn43RGCZaEr{^}&(NtpSv;DIj7^V*P*kGi}yuSyDx87R=7=+zOp z+XfAN#!S9!bkue?Xrx(TePmBf6?hy8TfH#cl}K3UkyrCJqnxnhFws42Q-OkqWD)~2 zA0j(lo^YQ21R)M}A9XZgu>4`AiJDezprx{X7`aTS|L}Ihw37}smrazH1xl2ohRDDH zl@4}r>mjl@-sG$&)7U$B`u@TZo)-FDko>D^_W}W(kIAy*Tg-34sLOpcS^C%^Un(Z4`RU8hHmj$W)JSpPCkWjgSjA~Ax8zfe>1`Jl z)VItjs)}aiLIs)#2F}oI(S^+mAZ<~pTE}5t9U=T&IHHAGZnQA;MN;McwON!I2R?+w z<`8JF9TIxO5B-eWbDAvqY>^NR8%MaQK485G1Q(YzSJY3&D_jKV2r9!!Zk}3T$aPvu zzUZ?=#UFXY*ZPAG2E6qN=No^#00C$rJI&lc>*WMcARbb=DUe2h2x346U#~9cxIm#x z+3(w9A+HPN(dF2Kq#0x^Jzq!(E|!bQ)FxAgexzcLY-o1jM z#|1;pQ-N1`dAj6b(<2vRO^BPD0&(^;2Fkx%1Qr1W_z5^bYqsa%)Ql^W|7l}`5|gr% zm9ShUJLmpA`w2aIwRlt<;ivLBKN88u9&7fa@Dv`T6s=<9dY-OVjaUXZcUi1QQv{lWTgxg;j5xCAoB*y7yS3$L&Yp~S5Os%Nz1eq%DD z$dGf`N^E#(7!fDsp=`3yPXk3m5uXp@@?#|uaXj`a_`I6fE4O6h;x>&%Kd578RG)oc zQ(q9WIb;)JQNvsilRgQ1LbEs5(fRpHN!W&y>PbrU$c=iEily`FgxI8Q*AaAfji?n1h3f<>4Ciq^{dqf4;|1cR*f(C78u zYo)c{^hF*bs_N#jDeIrJzju1`_4~KNstYPt8dAyoGbGjel2Mth17te+LrFl>W)I<_ zF^6qn!>=(copoBEx$Nxd!pt86hEod+*-K-4g%yDzMeXHBueTF{zCB(_J$HY`JzG7s zfp6zlC|qHI{)&o8MTh|mG$S|)ig;cXv2}0geE^0KbKGkX*7WWc*Po|VG#uPyF-81t>dTNB zk{os+a)HsfT`N$MOINalz*opmx_^?G91rX4*IX)=aX<1B2p#{heC`4H_>vI4l&ASq zn*Xh$l|fxKQ4#AY7X6f};Wy|+wA!tayIxu4x9`U;i(W4Dgi>zULlcCg(sgH*eCIrb` z$A4UQm2c$iTMNl5(i_)91g3IRsMBG_Av~Jl8j|nEVerH+HBXg+&EgUgp>I4c; z3cKu_tPNT-sHwvB(bInD#+X1Z8!PG0a5}#W(hpalT>f*v-LlU26j3ty;+C9Z7Khcd z7`iVgw&89*KSCQl_6Pw^_*T8|PN|rEj6RBx{8YQp>VOW$Xsmad%5+vW zHa~1RX6v0z^{h+xNf9>3Z4Nek^@qR-pW0{+1b&Mz?-9r?}~MDo4$#`#qrEX|C^D z-dP%EGPt8kx2{Qx?8UM!8x-`LczbdP-^l!mLj z!9-PBsD3f)p?K@}t(^;;d;^_?r9QBmf?4D^)dKkg=_C`$5OG?Sv(P78oUk)JApR>M z8AICU6M48lk2C4%vKw;d9#AIOzS%ER6=?*4k0NMjbi8D`cR<_Ip$Wg>K8K1 zaX<6uz8bG4?8388FRs(p!yeF+0o4^^2F><-%o2W|UH3C#w{HKtHP4p_i4-O>{n$3N zF+Ue_ecaQNFixoCr{fltC;iIg$M z50Yy#ToKpxdNOZ?*gp)^#m! z9F>=$`Wm;?C0~s)Zl!@MqKND(>xtY(E$o%d#=_%AK4AilY|y}a&q%H=%d^knXM$T} ztI8;#kGCBQ{{sVm^L|h%j86UOC*&0iwC)|Yu}ZXkd$K}fJ-Cz6XSGTR@&>`(?>w!> z1Mkn zR%`K6?1gOM?%voV`4R&h?vNj#b*;Msd9;+;DD~5g8tpSRH3i9&Y>8_l5QKmJ&pJ=n z7kTfR-Wd&)jugOMf*}wX%1}$6>Yki**%u*W zJwQ6O;?^>r885rkKJg*N7jM(JM32g~9lCv7d6r>K42UxiA7!hfR_c4>0MO1ekM$jG zZhcQQYQTjEV$xEI?-g}B5qtD%KsLBiDaE~Yza6`5pqiWUG`hK|G54)xCh$gCtttUg zWUE4>>6?)mXYlAXQLyxutshHpJ}?IrWB>A2ot7dC?Cp2(RL0KVsSJ>)wZR}V>^C*~ zt=~n`hgb$yfSZF)?uXAIAlJv#zY_)(x`cb{@NBt11YU^>yq{f%1^t6h)mY=#c7Nm3 zIXd~k`%0Iyq`c|A)nEJsqaq&u%w+jj`aoSjc5m%;&`1;;Lx>Al0uc`pm5HT{gHm$? zR_TC!{D{mC#lN`#u7d~&P2+l%I`<1b0l$8s`!cD&p~m%ya7{@+3M7oEbdvPJpHCyB zNBrL~{q@$b%s{1z=e2?OT$(3U;sXadDmk+B#eN`#dGW{@8Hfm9a3nfqF7XrI$f|n~z3s`I0pQpF6Z$qLU=U@)E4N z3tKGscgd!|gzOIQXI_q1Fv8wa!L_pY@T?XjfUdH;0H4Y*VT}tXH0cLyskf^z?9d`l zKqi@z$w{jq=I3o(YcnMvJ0(2|2x9>8BeW@GGnP*wcg8RD?wK;x6Ek`5{<20+j`u;@ z(ZzA^A7yWC%l&u>EH||BXvGDlT~4PWcya-L`8mfNDD}lV<86D_NSV)Y#^M(&aFf-& z7QRvg#PoT5V+qSb1zgx zR-B8di+Kv}B)1FLPrf>|tyIDH44AlCv5ksMK1{3y*n)X?n?6qxyNc^%EpMOk{2EzK zvJnT~1H$EK8ffyvPZzoy7N>KoGJe(C4jOh{P3znn5+r~tRy*Cvv*OVACx*q228QEm zbtRZy_t>i8^LGt%`mx5{@pJnS>1KTio*;}JNOfa$a6JFg>8e3ys;Rc$oo09?zBMnS zwY~}J+dftGF!zXBG@uT+Qs|K&#JGs0MbFI12tBJusTU|5g zf@XL$-EZ&*iIbEmTh$Fk4y8{sq~==j{GxJWUEXz}xzE$c35tSDBmOGb8|(T59NgT; zrJ)f|oRdnUVtbB3m7{p0>>ihmW74$JNW8`mOz#MLyHtT7E2KP?zMX`~>{o8^!q^+) zLa^niz0qoOGiNCtYTgtSB>>=r7~6%B3i7}iUI9pGAs=E!OP^3nfglw8H@utfq71T2 zniZTYb(>!z2=+#v>|HcYJ3Y{oI{sF8zxV$l?Ja<++}gfzMO0EsKsM4PQc8ETX%Li@ zIP?Ytq#Gm!5h(#_wscE3s30NT(%sz+-`XhWJnwm*Z@&M`e}-X3x$k?eb**b%@mp6Y z4LioCOXmJY0MtYpUQ7VU-vaNBnV~Siw$?qE@hN4xLs4*D4`eRCfof07LV?|(fv&(= zUu_Pt)B^Vi|J7TU{NnAu6|Sv_HfVL)xdrm}SA0w;Rr1eH#uIiuq&_|-;lx=yRs!2?N3h6y)Z%TUx9F<>!reC1!hL6-mPuWo+MLItbMW>W_qnZRk zHHh%4Fiu4re0OS5gT86kr8RIJww3ib^j4a4Z2N;G{AeFRoWQkYL?2ZLPJPN z{zMsuGQH#+tFlLT+^p;1<2p}Z)L@P0wMW~TQU(cCpcv~kh`_@Rn*paRfPe`S#i;SQ zhgjKUwkh~%CaB!XU*5CLu`5xauRblm1W!X2G!mnA0kxIh)FedtJ#myL_FnMZ=v$Y( z!Vb)KqI5pv6h-I3tx`aB?+<1^>C5+sz)jA#K>MHHSQf%^$X$ zU`5d^K_DN*VZVyX4NpB(Ek^O!6>=}p`Dihr4W`5|*gt7rvVj6eKyKFGPNu`wJ{PUmy&?5) zX56=Jzu%$*;2wfVmykXVc2Qp4wq5HNz_zx<+o#J@R(b|n619<@xI%Q-4L+_uMIu6v z?4KOm(dp>%7$<5Bz`y%X!n~qkZNK=)E(k8>$xj2(!@B$7f&$~y zbN(~|1Oioc?ok|&x%JxAOyqMdhc{msqwx0E@$N1+);f(&wCI+XR%(LES1%7#m$@{$ zx?Ze-o%xh5uJcfXM~!giqqYL|_itv=5pQk;t3NuOwc@2HEp2P7Ky zzLoU2)A1fEL{4Y7lZ6k+;_o@%#d;<5n?CsHTewh+5)OWcrwD$uBO9g{AGz;JqdB*$ zDx?u8Toln7z9Db77!6RumG5N$2Yx0ZT~};h2L~+$4O>3zzuSwDgmJLHggzhJMJxk~> z>FJbsM8&?J(R3kVkwGY3`qD#bBfUTfu>71pT!r7MA%^(V8xsWDJ-7udFlOy4Uyy$# zLdguGvHi`5@~0X03bMo~g1ln~yIbP8SU!*nyo+Yh z{R}clf0dZx0V?|Cs1;-g z8_bPKPyYB=MP?KaV0@Z33+;g*#Fawq&T1Ze!jnm=>=D2$b$FB zN2(`zR%Kaq$-itgF%%>d zOV$gy9^|5tQThh_Ds))a93K{UVAH5|DfNh=ktK-dwib1Kk@XCYR8w19>H!o)Pr21( z0KY_enS~W&#Q?X%Rg}&hx~VnlUsD-)SslfW^|pi2e*!0@K=j1_2%JFm3Jt#G)2X2g+IU~;fu7!HF+~yxR4afR(3J862q}F#+$8%6 zJhb6Kr_~Zn@cNCrxzD9vOaf$s-B(QHj3H+3!CoLK|Dup2N=`$ z198ajNKV8@dyx&jHbGLP!FZMZJ#=EgtwM(XD+J?r9M;wE4Al2zLZsl~29SYi$Hgk? ziwXK)88{r@1bHkU;z_RROFRJX4azNkQklPVY6eibsa^q}=K=?exVpGHtC2R$OiKUg}0|MK?RQ2jcZ``;O}9e1jTFL)^tR$)YZ>tUawHZ44TMD z{^Owi4i7_c)$bqS3Op2rUo9YI3z2s=OjvG#>Bz-BPJSCRBW(M|6=XX#Jwi=7c8qo@HM`;JdX{7>*51lc=1%9q;kQvhQ7K!E1|ef#2RLmw$oQY*wi_<%wL zgk95W*)0&Bg(<*?n1a_cgz1e2XIZ>R;7l`AwlG{9Y(d2dG<^$GNq?WlGm`Jtsl-8mzSkQX&m~0#c_4!V(~I zyhZJON|K2utmr>S%{rC+KBeGyBJ?Xsmmu6l2OZ>6XB<;!)&kOv`;>sv0wBE&r2&)S z1(y^WM9JK3`_=C=1~{%Ak*f2lV$wR3>CjbARFIB}{p9eb$Hux79qF@T3Bl&~V80&F z4PKou1H|1HhEO-!-`gssNDh+0Bqf5+w?qZG>0#omiUFSh`NMf4Gp@3NKn~lTyGYk; z`*+5gM+OLx{QD5k0JRz5)U@_@qop^|-CwCF!Ns%nqGzvYLURrIn!ntwxmoj>N0sY-DG)0uGx%s1zc-gZ0&zjGN1XRJ z>uqr(LBTz90cpUhxrfLeH)_~kU%He^C09Q4s-NpA;p4svA~f>xAp;x%2c(BP6ORJ#LqT)@(b0?>Ryq{!4Xsk4#af@v z+fjm_P#IqJqx$O^V1JC22eA@JdUQ|RPwdz&$JB{IHWw*KYYJpH?e(qI<0^T*H6_xY ztAS_I8UHN+hwAm-l9F5^KTmtqM+UWf#9WsB4m$N7!5|YTZgdP(o2WZKc)1>N761O- z=Zz9FCQo#+LB05l8%J2`vs+{1d{|tA7w=wamr=wzYQ@Km+?WXLBexI!#@$b!^^Vlk z>@54UpHZbJj6R=n!wai=L%c+Ndu!7-_VF|7)6W757%guN(|E~Er=b_o0|ql1l_|*po~H7y+tpu&}W34dryXKu|xzTnwaYG`eoqzJ>LC=hEWNM8W3F zu^=M>5q?G?lm}DAMzEa6#VN3FTt6^k^GwJ-Bg){ ziNn>=ETvOcoOsq?U~X+>&1HCbeB++I3s%f%{u{;g@*!xOjR*93qe2)a{ZIRW5VD6F zfyAr+pH5q8b-+}hup~)`$E9zBt^fq%XULcu@ z%JpCk0=Hi5ByrpA;K~GM!^Op^)mXJ77VtA0wx%0^j(v+uBl8z1nJ{t;tgN0|SH~Ks zZKN1hqwgU9=>;_r=c@-F9`bmD;vC7!X)>^h8XiRU7_jzCKl+?8HA{Pf>IBMqGl(cHFavK&>MkxbWH$!@|Fl9n4a{!l;}&B2Ng)L7>YsR0#HC-3@B?3Q8A|lU?Q#K zjmCkD9B%8GzXq~ZTqZ!lf#zm1&-XxqO;>N$HGujeIKZ?jF(0~@r&-$T(Rgoq2W05a z?98`qZ5Q4fy%*Q)%nqaj9rly#o4yD=&L5@^v*D+)$t?vXC^=&FV}yRZVW~ z{Zul9u{LDaCY!^27h^#`N&e{}k*g4Xa(0C2;tvU-xY>YL;eJ|{9?kE1AZNjm7BAK4 zCD$&|{z$XrQQgIfIX<1?XJ#c5kRjNx+?Qc*E`lRcW-*%nxe+9>6&f_(JKG=9eIVe% z4h;N^2nQUhkur-Qgk!OoT%}ec6ok?eZXeY z*$669gb8_iFlrQucX5FdFppCMg1<)tQ9{0g12%vH2V$-4>3e_P<3$EPcNIh}`r`eX z(E6of_pe~}R@VC2FX(Lb67l_&p}8+4b2fD*>b+BhOrQrz7jJ*j7tj8vZQ~xH0%i~B zLv1zbW@uS16XJY6mzbZneX@KjM_U0j+1@>U@M%cWYVc2O%S({={S6K(UW5xv0o0iv zz%;U5)u1v6q?95ILm4DHdV>-#nJpCF-r|l@&Syu3P@Vu`N*of_Lc*J(?*fp46d4Mv zlABsjE7huaEkbuhnJ?uGa!nJ3Jo7&X1>BjKn0V|jH79nmx6C2hh$E7s*Wf>5x!7qX zmJX^-1X+$(nNkOWYA*&8HO~9h)Y9R9U0$3+@bw=bfPf>~9zt9{Tq-CA=)xj)J9)U+ zgn=zE00%B!f&ci0RNQ)RT2P`INZlN`Gi@wm8u;;62%QAD;ibHpf(AJ}VEVex;bT|( z4))Ywb~4cZMSzq_sa1e?)45pNYYQV?!}SuP9)so~dlu>rYD@n!Zh?}yQ6}}2A73MX zj9VL=W6zPShYVdfTDNm$}ctcpRzDt>vf)qu4PPUc^2QA{TZ^CvjC?x0T;s`w+24m$B`Eot}NO!sR3=r zwu_@VH*G{k8rskW?w!@w?%oLbta-V`f=aShORoMEiJ)*O=4(mr{8qsWLq5HjE{-m@AK-0 z#dvZraUhdFyL;33PQxvvj&NZEKmTZFbuFh#@s5U5P4#NRdbu@tPxxnfU5XoCQ4%-cJw-F} zRxdGbwjpf}PhB1^M8vql=EC7Upg9=%-5M)@HnjNl*P|BYroX=E>7{ha!3#$%tNntZ zr~9jfQK{;NPFScWu7;yx4Mwh}0c@nq)oVrMWbyNQMEYUV7}pxV!3Cc#r^zyo63qyTj1f+)=P!>JfJoYv&-Yv-}7Id z?dpY~w_!xj3NX}Qq}9>T#aQuJ|J=Jn{0iGJ&`+Vz*FPHW6!Ah9A04y!fY|V?3C|Zg zCP-0o7wHF`us?J=rW)!CWgpAJM*@`Y!xM(F3mBMz`I6tbi6VRhN!h7EiK0xZVuPvm zvTfc;mS!a{>Rcbg7s^cKXN!1-I!u@i7jj|TKS^l|o<}T|{Dv1dt}sj-5C1>E47(%R zKsh@T#4-r%0I}F|-QAX(S3@9PnSqkii_VUxEDWOgG-r442TA&G9jaLlx~;A~2XUN` z+ux)p5YOpLk$)4VVq&d9P3FShi#-Mj{}k-{-0d*j>j}4N*x*)nOQr2yYzM*8&^;cv zd=Y`&N2c;aOfG=J3mGjqma@T0cx$?x`#lLC0UW164;t}jwb%hk-AC4Y8Yc$r7M7nD z)GeRePb}+4mYAi;!Jkh1D4kqf;6=uAh*=P-%V-}|iR3M6ir4eI$E3gaRLhF#kS#ObH8gVUvZSW@@5=Bgz%?lYdxTJ;qaLiUkRh2+z{*yB~`wV|K zjQD!V)|xi@dRyn`bf%Re1(_ulbnedUA7TxLgG;AqZ{_$xr8OAbxt*MuT*rdU>S2oK zXLAyg0mdG9}LmHbXP-nVI}d@aKPoh8Tb^d8!}Ep1bmqE_ZcGyh8( z_KS--qVw$&6no+$7U%ox>^k| zMYman^Q{)_(iL`3{K1_pyncfv$&=kYSC08Ot^xa`xQ0!^S{M~m$#G$}FAeO1!4bb?-J`*4|6z)hAq{x8OL7A*>cYfW^hg7epK4+&g1A80fS> zU9JI;{ASA={Wp)k5x$SC;F56?KHSbZcXIp8-b(sHm$~`nVB*k8hoK%1?W?ND&R?m|Y>x&FWu@lIJE@py!aja19jxe6>$hpH&n-iD z=$k8;M2QPB}PRES5gxg_J!u~5Oljrf}V)X3CX=>B*X#^zr<2{Q0NhoLRb#9 zQ48i2L+%P|v|HefnR?ZlW>KXbGnUxk(?y7p*J z+ROy=G70i-x>WKq73*}0rsTU6p<=R#9Iy2X#_p_-Y8p?$?ANNGI`QU-02971$n1|` zNCN~zn!4-`S`AuIi zo(z}*%pXaV3bN6U21ZF<21CRqKK+j`A+hkZ;xCCHH{ zWRVrtrWC%D{|bl1k5vbUBgqUwi~C78L{pC1vv97p0D(05yJNFfdm0&%PmA|!ak}10 zV}b6C%Q&g$VE5Z7y90<-eJn$X>{2SS%5$fBUtV*-j5zr_lcYNW1spTSPZVm1dkK7@ zVwu*M2vTLDu+R7pBokAMQe{g$XSrrp9H6?yAHI2*EeV^u5%?S-&>Q`M{^f)xUuh1+ zf3RDDWgj&Y!f`Gi^oSw`y99db3FKP(zY$hKA!mQ*U8xjF&JOVl3A$QU3|Ljm*Q|Tj ztBR7j0Y1<|#XdturHKNFH>Hb&f(gBKJrnFl2<5Nzgex1@7n-tV{S+~4N-(|qMBD?{ zThOfx2tGsR>gzw~lC1N=B;_e^r z5|*U_4D30F1KAG2Ve$p%WQzrudMP*mEEsI8A0WY_PsA#yqA5y86%W84gu}oUL^LkE# ztV!z?sj4YG>2aEOak9#C>~#?)N!p|O(n1BZsfpMscQMgbA&9vub0OB8`5ajJ(`T=m zmot*`f$+}!{F!oPihz|AfjyrgWBD&T@lKeW5Z+Vg?#fg?X5+G$;J7#LIB5!sa86GS z-;;fZC3N=8JipS|$FIqs*zy5#1F-rB3Uq~Px4xr`d{)oAKLZx*O=%>Yd<>S}C=`WY zz5oJ*RtH>n2kaIU=a*&m`W{#hL!?aB^9+R(Q0lexD-wE(1_3Aw>h`EQH&>2I@OGDv#|>v&qZ`A$fBF*joEeytbh)i@$fJLNdDoFuIGR^I&&r{rK22Y zQKr#CUAXqkFhX9my7$rp80{Re?&MJEojye&Kizhbj zg*G$8a~Suife$8UH-O{RXe5eIg8{a&(n1x|uj9I&_R_MF#?XO(zyG;NS#I6Q3S1B! zqr4FK>9N50&p{22P!!aNv#iPjz5*-eL+)n+E_*1VE#LiaAz{NNuGpXDANI$FyhjMu zZ^)b@Fsfi8HYh+w(2dmZsYAH|neaNxw)B>_!Dg9(0$9NW5R)YLG6nwa#%K9-$c_`v zCM`8m=$Iyk2pnxX{<#8=gvCe+>LRaLgE!Uqpo*#nO^Xjeu!c};C0~uEo zA$f%<{+7HzVm<(p1|s*B2vPtxTE+hxz~hLc)U=%EBlRv(Sw_{Zf8aM1y(83hD|{qM zRm{dU=~jF&?x>q!Kd7-Y0RhfZD=mBPkSi9Z$+I(1J4OVSdf2yB6mz@~$Fi2C#3n`s zSVpX2m;e+r@_1x3?MHZTm?DMZ$!YTuIm8e{}-zenTR`#3$P@ z2?Eq%Vl4hG;ZD(p0`qae8zqC9=6*C36T)@?<{APXlutZ3cP8L$Y-AL(Fp3La9L(6#5PQQyb5if+xq%p5kVow65#_)4^^b0mpaq(cb?eTHx81NC?95lL1<_5;+rbzD_y{e6(3#!8 zrMT69k>b~1`LFu4Lki}kT|HF!FWEnS>-~Uvr?6=^0VLc#n;>*pcmN2S;dQVSE zj6WI_810f^1=XSLgXI5AfE^tkM$g(yA_{v_@yAre4qyNZPm5~Tsv4mze{0)N7+fi? z1!#oKzn|g*PmNJg|HJ6yVWP7@{HlNgRSmu2ukI2-Ff6Z)fEk26GSqT5%KXpR)H1L+ z%KtvL4_KAqZ)0HurP1wqH$Y+kR>y#AG6Rj68uicvTSE4)(+gI`#!( z=hF^@fyN7=@Sdp_8-%%XMZc$1p>4%jqEsWv6zrep71t5_xkH6*3!jeT1&|J*awo!e zM?6cOh!C>xWsP(Gq23mY7cxq)oq;V5@f!Tw)ZdThgGa4{+w0dw|5iJ~lzz~vo+S5! z7$EhYuNuQqFz;|(&*FbA_!+SmvJtHRm4}7?E%>)5K>Ylk!|xAc01+7mzp`^;W4fgr z>7Ks=hS$OsFM_EyMgkd{A0`6EX1!8*!fg1E82uDb=)95WZ@a2796S}Jmo@kZxFi2q zaMVt%sCo!H>^FU`{0q;Lf0q303H+wa_3ne?J5Y|2s8=R%7Gm%rcd295goPchuhGyQ8K<#@taOoE7gp>_S-WC$}a z9GH3G4+Tc<{b6-sZ(uFOa;k#}@_E^1#Byg4tY81BoAUaLu{3~t`~30YJtc1RB#4ik zAuKFHJk4u=M!1+7n*qKZmMTjVIVfD~l{&8PeW@ueK%E!;(iIz0X+3l03wFE(X6JEZ zc;dAUiEWNlhzBW`$pF ze<O{R)(htycvD!Kicg(D(W?J#=i6I9|J#jZ;ew&;UkCy&64JyL z>`i1xl$5!o*{DnNYC1I@?n;k{_DYA}3+u1kaqRrH^ehHGz4oEwv8RW;QB(^rN^pSx zlL5pi0YqS(?MR}we^?V6mXEAY$8MR^UebcRVMIAggIUo2S?Y4bM!J@| zST!BL5)h?2kHRxCce=zxe~(G+PO(fYo%fjSscM*AP^vq_q|VGdOOmV^i|No$cYbvn zW%v8rBdQnqh$E3H0_>mAKLi5*AwdC7mC{F2!gAb$?nlg4W&lkh#})wd*n66M`JhDs5|JE9i@k4V;^M&r7njLhR$JxwqpeMgf~e&4l9XI zO?bT8UOP`IMunvwe}$Q7Ah4F8C%vfGl5a+l{Hf?HmUOesy^D)2yqA!Q2>@J*0i8OC#;_QjKUV!gWD~iR0~=S66GN z16=BXv3&EIg#yU|mll?!)G*d9kG9Subf70opgk?6{psX5n1=W_SWrk;I&*WD6A zXo9fBqyz!8?HEX5UCRe4WUoe);Eiz39Pt&5VAF7iNO0oWXgX+2qf#4a5?ES^Aj9B> zBL86wK>KLu;s#r~w`?MRJocOjkfLOV+zgRL*hJI_JsGB#(sRwj)FI#~HBE+p$e%O6 zSs0wELX+^=)KR)k)5=~}tfpi+y%v@~MmxAo>hu0#f8TgJ}pR13{T9~BG^$N zA%Y0WR>6yMX&5H;i}cfSTin`OQ^Iv3Uoe}(WtN{hdTfis=9+_wuVxM)F{g^-hhj@xT07!C?UlrE}5I9-@3<_f@JnIB#*8lxe;4RYz-)@*@ z-A9CeCig&?M?t7ef8*>L0N!G^P{)DDw?+3NMrsTJ^YM5Jm$;q}yX=)TVEEtCAX5W( zOKc5zV0i#{ks`LLDNJ&JuKd;*3=qXnK|2l zA_oucc^#Wp8s3eH*=*MP2uJ-1JkR;a>RWbBem0iikuU=5u_*fZ6pJj$<0LCe(~RDT zVG{rugIO7g5b-8(TOmet>kdXY2=e;A59;C$f{j(FLdt0fn^5Ovdb1 z`sqKew}-qUf-Ec+PVvs}L@=2jjynzk)G3Rl?ub%uExt|&+(GzDn`;9ud~M?E6kgLx z8|NG7-28#0VUBGVfGHsOLCZX^RP|G%bb=9EOeHoumRTzNj`X-{bS$z~wscP776o?=YT;VqF5h!E!MGApE3HK*E6!_mctu zOsQ_xo#qp0!YJfpSZNsBoB^OH=Gh4-M2`88j*XM0xoJ`3MZZeo>=SUsc~{!VbheHg z%G~Vz!+i-v@^Baf&eRTNPn^00yeg9IjsB>cE--%WAEmw}W=Cn}Qy}BQ8S!#Xy=sHsdWYyiU>7?Udj+5^8t9j-ARtx4J z+8~KdF_VeZEx6w6u<+;O9N*T0=a=Sq5JAVn9|Y)l0FtX=UG07IYF@{Ee?U*!jAzPf z%e}&V)T-XS1*4UJ4BB>^MS^=N*@xZjXe&GSj^f_y)N-1) zm+KAZcB(=1q6;=3OS+{Lr)uj1qYsWIqi{OOrA#upXB$3$#d{s(9ihN0ZIbA0{AhAf zqm1#(S0onzFXcSo<0PP2&{cEZs4tddr0$JhdC5Ds6Q!;n7&AA9<={Sl82?<4jHbv1QnPE zOfwP)e(qN6Rul@`Xb?D)ibXMX{(S8pQLw$q-(*8Hbf)Jmtc60oFiiJ-u_4pIuRDMj zsUHg=xyg`lCcfa>KJR)kFk2`)=4B z@f!UW%gxp?^9DRE7TqoJzF~r8qlX-q36>5E;Sm7Ql{AUBhPJ$Vo$ebQzgE=7JmB&v zPpd5uVB_T@9vI-}6*J~e6lToZVRf6eirEjmXIl+MMBx)-{gV?cHCx|pprU0vHChcI z9)Wi3t_irJ?j5FIP8zxoiMe!mDIn6t&;QM_dTb*0t&u;C58arl^xQ8eHZ{eORR)%E z&*)X#K3Rn#_hh#ZY+}8$XynT(s?3Nj$=9OWcpR&wbrR5PEt+SvXGktj&M(Sz` z598?s*PJF!>K8|lq1iO4<-++Rp53{HIIDX?wzO~Age>dN4?^W88N5UaKAJ`u6tI*a}P? z*4Z`c!`4fH^OuSdyz`UE|Nnfd?>~b%Wi{U8Tf?q{I}G|oiyf`gO<;YC3v>y&rKp}j zGKH$+4uGl%xb+hJt^fJM-u-^9rmzj;GSoDhfMQ<$Zyqy&baPxDPQw9o={g(X&U2LP zjA%uXi;R8%h__NTrI_jvI2<-f3jHx-#74jxBMIa?@h2B7OqTiAoZiLy@%Yn49pkc! zT8W^`08S>GEF*ReYBt?zr|%c3FJoJCHTgR=_Zt^ou!0d61ttmhs*jLa%IGO==^tt?_EcVVTs>YGEa~N%9rs^~qdt z$^_Oy;Z!bq2v&&I&}`kQ%Vya0j+TEkDGy%5Ow%y!#Qpx$%uj{G02aR}Vi$vL(Q%m~ zha`{ia=c0Ez~jW0_r{424~jO!vBjhqw%K{TxgrzqH3#pL2hwdZmb z-tFb_6=x|Qcx5K2sMO%oyNs+gFB;y?ZUNQ*_SMZzvgtIoC?Xxpu+BGIbEBu~PL;l^ zKp;xUxzNj%?O4`qH+}J?1dg}v>MN|ai1sv@7>h&hEoIywG%^pWKq8I5)D4feXwxP> z?$@1epg*R+aJh~J!2M>$3=U^t!8R@p`YjW=waUfB(u2ODMk^X?*93X}Fn!e7c8a#r zSRfP)HW;aj?Ks1*LUhX+YE5%aXL#M5Y+r14=}8;%MCmBZtzn`Q`rh%hiGY%WUX3_{ zWKfdlzR-T0WDkS`c#IveHNlSvXv5>^T+|G(^soNliDcJ0Gm zk*}542EK6e17x)GwLU|O55UlDJ^5(x4Rkjbv4$WFO?AWoDw%Or68?Q(YX>LICqVm2 zGV~I5q<==kebis981Woorna7{D&GG0kcC|Pa$#W`tfEv&Q?|_4uIdl!?q4QJpG!G_ z4FA0UR952Oufe(YN5t=dmy-W^NeLUaRIJ!D3IR;ibE=sl`fpNf%8&Cc9$b|`;|rd|9*W{UV!kwO*M42Sn=uT3v+G628=uY z`?WUtS`dG3Z*17V_W0kc1kg2rlpY&Lm=!P})D<7Ni~k?$$_B80AAYCbZx{<~sx{pe za?ZI8kaM8hIK4q6#_@AL#CDvafOr_NEM0CM;%hAC2)AXv?V&HUWLpd--j+0SWpqd) zLWH>N54u}eQm_1ZAkyCm^8-v~hA6sS^`CnK5SjoYUY45{C^^9u-ZzBnfAy#q;!#M9 z=O~!LHSjEs7HpRybh^g24+6jkJ4A8*_Zu;?uY;qURxv(wv^D+;4qs zc{(gM5H4)U6Iq4-2(Ccw?9L>D1_gsKW*iX3)qf})DTWh|aN5bskk3|R$v}fY5wk{4 z8Kz>r{c%rc3-i5MBsh*c1`L6mc#Wa5Q^CVrGkA*Y?mk9AAXrkYHxW0GkbfY+4dj*m%KM*~a}URbZAS!bmX@qNz~yhc}*X*tj3+g2%ss z%_O^}2@FVKa!25gj%jeDu&uf#n?beH7{FGdQ$Mp+4(Nw@`H-h;=WkvFbq5H({_VCR zkc&A5AV~RgjO~owv0ij!=Gff^mn>e$W^wl^IW(t)1N5R-t1l_pKVv~esZJhrBO!LZ z5`f}!o!LYQ8k4tF*j65EgL^bha<5mvr-t!#_0QC9{=cU-l!~3zgLGH?eU;X4x_!a^ zf6=XvJ^mJHeP3tP+J8%m#~|vwKDsuTfbUoaCa&6tx!wM_gtbryHOAKy`iH5 zyQvng0qZ)y?ELeUQWFl(p^aDr_zxTAf_ywsG;4WmU_n_Uu57wwaDBvlJ94PkS88 zx{`VO%B;}FcGybgJTuSGDxPlrIokQjF}DTBQFcwfc}wf2(4MEJ>8Jq2#?+_acq)u< zud!0nd>{d@`oRw$tHPk`mF$gnru`ZIe^`6Wf;oT|+EM?wQs1E<+ZyDvIx5 zkR|N1CCDzG@mREQXK6SvBestjXF6UKPpZ9G5lB8i<3uZGlwh|&Ye59}KSKS3CHy>M zR~qLS7P4J#JvmywTRNYm=G%HcS$~$C!Td1wod#R!C`U;D@Vh3UWPPCzYa;Im9{b72 zv}F(2b}8|hC!B{2=riQ^z|gYZ*`z6Fk+XkrTI(a1raeE1X!=m~oCQ-zvRY^R4S}xF zI&>&P_GaQ2`BPVK-gW_9=8T+f(b@gWYOQ%T6WF?YNNbiy(nq2U-TEVL^Y$Wq*hRe? zuWFZ$N328b&bRp0{XhvkNY+Q@QE+{9+^^ zRAJF2zSL|~HE*QETp7N-kf632$EDkUk8ncCBDeocZn1YMNywu>WHyyTv|)&x!zK`I z@qYLpZ=8{O)BmGiHw=X%|LR5=|Bs;}i62Ls%N21xC*(1Je_mx?5SKJa$&jn5viiO? zg8iU}uk@$qfvTwngMfVxDhREVoI@SJ&B-IHdwSSqivuAOa8A;xI(QY?6`Q88%aqEb zP|I&;4Zl#cs#0gsvQ1x5f30NU(yyWE(6_#~v%hG* zmdQsQnYkKEY+h{8siLvUd9vM5zPF?yX*M$HnWYj}w&MDoW~J!l&dG9$<9dS*^DeV4;v@`r*Xb3uy`G4}x4JAN1}ggk%d7$(2Vt7}rI;l@ z4^Ck_oApaY9Z~(3YC^d*LnCuZE_cW>ExKw%uo7I}R9SrSt8+b#B$qqEsFIS+E7mBL zGB$>B6E_XXo$S*&_twi?x|eo(G^_Fnp#;w!JUa|73+IU{wIptgI9;M5jxNw~*HqMY z^EQ@G$}bwsD_y?qRMm)csJcA5`PgDWlfj}ogOk+BrL;=WbNi`)DOKc*!ZnGD&{`LB2Y|0@yXiL>;)|+Yo?m9$FV#X zG7qet)175CcM2*e?nQl!%4{a6_Qi;cZLXEv%$A7SOasm7((64*3n~e9W%#t5Ttkhc z*h))>hMe43zVzFS%)h%UI=(#H*-(ZaL)_`G6?VKiWLYa`HRNVPV7erBVl%y~8AI-2f>4Mhi=-}ec6 zc~40)o^t8tQjuQfYA(F6DAaMOt^3*DeY6vqv3FD{nNyPcLZEC+rC^d%?(WzwG}N{9 zM{VC3dRJ3v*>1eRZa4j<%5nR$rs1*5{?MVW+Nqpt&CtW=@U<5*UApej(08P#8?FTx zr^{6&u4=&%+W8f_DgLCNw>N9kUpQBSgPix|9zX3WpG_+$JslPW0cl?MynSvV$xHt+ zRyk=dwSEFTT{Q6OLJ$C5zn#ii+#IAbs)%!wu8>~5MY4H<}U6X@vTO?|IQ9cD| zM36+7pyTmh>+WsEW_|D!C@CMg+>+Yl`m^zh(<(jIl9|Ek)rRJcdoy#%r<0g?LoW{| zxaHocE>)+12(exw$a3e!!2lhkZgz#&quuIcF6Qq3N=G!TfH>*JssqX1RO_i9>ZF@? zmHC-#fmzNswbj{-jiFC3nFo4T4|-1SCgpNLwbD>0JyRkti_7yTjhNPLj-{ANI~O2_ zM*=RkjT|EPjYwkvG$*I%Es&KYX!;q72|611mL)9YRyB&wrr%t>8(7%u;JV`T)(Q)E zcIxhAPDIf0KA*F?y5>$>2H8>vUqMe)hBgsxsV#HKF923XcwTA@Sx(BjP8@wCP&f7? zr*5i0&334OiI;L48NXM*QJH9u#V;aovYrd1sZ{w!ZQ(uVIk8|{#(X}t)3Fn+Pg8ND z)rXT>k#N0x7JQv>=W4s0rp0wv|J4S`&A~U*A!ZBx^i@U$!6ms>mRhG?Bc^{oq zgR(9WPI!-fhrC^f1H)xl ztsb(R5|x}>>#p3?{Ozz9i3lFSm8yl=3^J6Y6oH3}rhRtjo#vUBe&ws1N%aK~I#RaH zF;}tBx1C>XXorj9Ud27__yhQ<$K}TSkk%{Ona_w ztbT7~TMd&jZR&3Y9Qd4jzK1QOTPuhWZhs(@uLIX}-fm^XGF{Z8YAC(9b8*J8*!Ps> z;z3Jch^f9|;1 zbQ<5qbdAc|UC&nph1yP8%uy!ckOB!%+*IiADR<=i+H zh%9XSV<`C2%hHe&$tS8djd>5!$i&FnOyq`~%{wwu5|3%+wu-EYthWY52yzP-L8`@e zTJ{J)shu9GQ?I$q-oNp;XSS1r==f9_LhoE#W(kJO2K;j2kK=xS8@|r$TU`Qb6*uIpa)C1OSo|1D@WNum5yg|C zcV@NPQ3o}KpFmxEXe@IujcvF|sZO+Eo}2^M=9a7z8{QuvgaY)I{NF$bFl~*CE@#r3 zIZoa;h(TP-8t{H*u<%INd+tkrs&L6_G^>92kQE(7M>@?np*Pr%#y_peK0+VV5x%T- zeQzum6CcsN)wp(gs9S*h7$1Lrg?rO`Wb;F`C~f0QN9~Lq4K|gh=Bh2U+}nA9dfpr} z-4+~Mfn8#?8f@%!XWVXag0S{Yi1jarl^(~8P}J#$$y`@@?_sS{d#rWG<~E3R3n%3D zcuJMgX7Ju*wQgVJ%l+<~hjs8kArHbR%2b}Ji+0Zpz9f3~Fz+e7dPxzlw6rGL8oDs?#7*3|#lFc30r;|f*3Y~%yR5u4C%fkX8 zOwrjIjs?{97}vslxZLGNwBqDYnaMbJT(`Q}ct-;+1({{_W$oAAHWLf0!{GAPmmnOZ zwN015sUEW16`lu7GEt!O;b89{8GcUx4DwTwr*>@Gk)ZPYPWR93@P2HUssA%5ofE>2}pZ3l?9P0P&`z;blL=h??YnEcN zuR~%6lO-{>WZyy}OURyt>{&DRJu(tmvV^it)=1WD2{T!<&3(<(_wTVJeQKYFh z{ygeaUm!2B2YWWVeuOJh1Nq0Njz!@j^tNx=b zQ{NqQ`+OeMW&BDo6wN9$DVzQYWK(S(b9&}(qbnE9wSPPus`Z?U-31FxnTi2fJk{Dm z+)sMXy^kmbb7|qt^wQYo80)Z8zAiN*GhY@e3_@&{4bmv-s~-j_TUzaHuDg8odB-q3 z-{XbG{w&nl{Ro}HuP?83S1xdJn)-&V45Zw`d#%5HhSFq7NNVcXb;o9TH(f=f_}Ul4 zHEI(4xFe;E?+w~#LRJhmhJ_&`ukUmDiyR!-U1n3?#2M^j8y0T9x^N5Sj8Csce}>uV zyx*D+LmW@e%MpRp;LNv*A>D>0mvi>HR?p0W#lRkgAf-=1l7Wjx06%~Pb?9?Vg>V2*P5plUR@xdOpQx8L+0 z*co!@FA1pgn5wdc$MiU0&%E))q#=Sjia+cH6?Rd?P%QV3$8M_D@18MNE_7RLN~V%lSPkr^r(A9wFAAF8UZr`NSR|y_ zM1LBw#D7qlX@%uv&HPN7n6&2Xd1kR6)o!2BJ#-D*4{O{PC5Fv5%}ie&T5g)K;M!B( zSwFlLR7A0SWSo=246JNh>$eK`c9sfl{ZTm>Y}V3l$lczRTgRzc`~^RIzb|#pu6Nj| zWAqUT9@8GGDGb7GSt_tDp*Uqt!tVd zRuxjfl4lyXOgw#DVrTyxq+jyNp<%lVF1X;ZkZhT&CO3Z^&xu$OOsA${;jdp`-%1_q z(N!!t*8a3*udMWN!fCCZGjoURX2+XP>w0N^zfgrR};o7uiT&HZ++=SplNSn z1=v^d+cFLh2BFnY&R`h%BLO{$ktF5`x=*oKL?cpr}D zZJM$3b(AkE+l5U7nlz~RGeQtD@?&*t(ZiirBSm+|Bs{?;wvp^Z%qSS{ABWM@n)gf? z7OFfyn!Q2ECw$W$#~^{`{_5z`Zk?+0>5M>GV|;hV(dE zug7f>W+#EI5GpT;9{8;$MTh+mp!8>bjkrfV%qdAk=0;7Qt zQ0LB~G>UEwR#4HCxA1BYO|abT6q*`4ms!OyUnE%QcxGLBXl8Z6zZu(-IA57k)Z<%^ ztS|14M@>8p2Pk)+^-CuAs<1KSlzqm1hR423of@pt&db=>(Z3Rqf;E~}L+Y=!BYmWT zvh}*;`2;S-jEzo&-|~(CBxJ26*WRl4&-RR<%}!P!Y4{BAm7P(?9MEd^IkU`A(cRZ+VX;8OKPF z!<_%=fOFt~=V5cJW(K~6($n=IfA^3(nPitl0+;$`q5Ekq?J)Lzd5QKvM4SC`6;gR$ z8gXB+pi?@!~!DtZ?$|QcLN1^xiQh-3VDRz(p$Yx?_r=|SqklL4 z5Pwm)D8ic~z=PgVyxs&*Y6e0vF?*9nE~)Q$Wc`{SCK7~VT&7knaG&@e%bou%9zl4y z9S3dBBFPLCDYVKV2IiCJID4~VMiWyi*2}$Ibqb7>!ZVTkL3!ggA&hu=qcoVb#z$Lg z-o$87Q7c(DOKU)A2$Gs1#WU2&1QiT`sD4M&69q!VTfGqckPmsyn9A2RN%=g~RRr0j$sU2o^=S zVv>X(sN_iYI6HTKzjuimF%Xsl4C1UKbvcusMW+cO<{vYimMn11kFYbU>?_5HyDJCD zI+|$PKun*~GOJT0WDo^A?L$DW7t`LmPqjZZi5dT3ka7~%Wt!JPrWh)BpQks4vqz>} zi&;`6I}v6x5lv_GB}b<{4hCutH~6%My*yzC393T6Dsnkm)ivt*Tk-uberZ}|on(H$ z%XWj{UEwz)|4wXeD?*3w<{0k|iPZqrzW(xX2IdnOsP@OJ84mgJyaJy-YPYW`zZDkH z<}(tzl+9w9vx?mAkqm12{3KR45sA7!99LkXAG(wh*E21m?OEl`k&pqR0QU<;Y@a*o z@5DUFl#3LVrbvm)n#?a=5aOv`jIk+(uZCo8TV?w{$6is_+Ws)TRj8Oanx~RKhCbG5 zGUfHMp%`A_lZakIA5)*hcHeWIf0BRT0ONwHl7dd`#wD#`OJq=8{?3B{#Q==;Wl2FI zxI+N!se=!Zx}!5u9?l4CMc>prCaI4KKqj~%NieV60&b1U|iP)tL3|98cMr`gX^z9bl96A821~ET%k3i#t z+GR>Vlwt@$zg5=~4I*6;VEUBkizu9@2h8}!BwoM(nK5OMU|p$;zx&jAxhOWcL(L{w z3z?CnG*}_fuf^$FKOJ~H-1yv(li+v66=I3fej+1Eo5&{^6+`PGMGof+nGK`?Mrw!~ zN+8nN8@qD(0+nUa;=fBJY&qUV)3+S?R~LX8Am1uCXSmkShem4tz-Penb))I!;yVqb+8ZT?r zB@6{GK<+g*s}diOWVrPEY5iu@(yOS#O|*J;IPo4y8I5v#0ebm2$6vCp)_&1eZ@M}rRw?B39)nUm`-o;#%XcK zS?eX!6>BF}T)C$U4GLUyb>{*>A}Z^b6aF^wGgc!xsz+)Oe)G#O z*8Vn(xSb`eva{gX%TaWZ5@RLr&E#r!PHz<`Ci=it3)KV$7Z%(5s+1;X7vkf{eqpsd zU>g-KB_Zgs)nl$g0-ubFVYV{$CW!^{DbjjrJ*pyntFX7iu6$8$DY_?>KD--nHGcvi zm6g_rI>Np1oB%!LEUryOiil6N)}hc$XwNZ_XUfd<^`1vEvO%qzDp}Nrtn{R+Q zU%3|jU}jT>(=}PEgjI5fgOedmt7H(2*lX|zWLI|wrgTT|HCicv??_AD&sf2f#Gdjv z*FI)o>i}V;uq70IU{WZK15h7mCwDcBGKBU+7kJ>nPca)?z8dIz@+?0}mJ428t)5>8 zQ#HG!Rh{jCblvgl;_Br?TNTo|3Ph22s?4vgXIJ00EErf3*}#1EL2_Y`T>LyB0oXQ_mb%K7ACTebK7^0tBI-Nd*aBapg{5@m2!u%wGY z)7*pIu=!DaURR;G>e$%1=L4J{-S(FAeaG(o9GgG=T_2f|`5k#jOGF7cajkhj0<4{P zF4k-R6ZThS3Y2Ii)I`VAQP8dS3v`PE1xocfSsYe6bJe2RNo@J8efMg6N+A67L$01d zo(b`R)CvM()oKDXi411L2{xg0^V>R$vsO!}iB1*73tTr{j07xe!7{rs?d-PB-OETj z`6+gYt!TvRw7<_D&vC*qj-eZObc>nHb|T#kI~T?vGrSoMUvyaD%mA$ zw{m)7vcC`V!Z}DLbSzuTlr=olAmRHstifrcC{`m;A62j2V;SctVPl|=bp!w2f)cXg zXUt#RNph5s@B)F}QGn@)ZM4h~1H(7NwN^&71R6j+!@k$7B?E%WPnrTNCiMpnBCb&< zDIiBtnrd@L)n8&hW^+nHUQBaP+ps}1FZ8`3Cw7xCSeBikAq54*ickuYxovt4wwrTv zu4&yOsl4RJlaa6vA1-Jn?I~UY4=3LaaA@lxTgK|{9b0Pqd2Pw=!UCQnIq0IJGKg{+U|FxR9wp8one@iHqhrw9|Q4b0Vkw&n+-q2jElY`)e~}CpaKT zC+s|tD{CR}QGaZ#2-%#LQ0emKI*%n2GocQ6ydhu;5zauIG9GOvce0T(J~8Pw%Tq#6Xd(gM}X%Dah=`<^6545(MR9GH2kQA zllaJN7a(W$i8^ApZC?M2txOr%Vm!9c1HNO#OB^_K1b77T(6({$lcPQbI`I-VJ@PJu ze(d^`?MP~KP+cVlHVY&!oR)y*=miJu+xFaG1AjQ=XWw||FRL{|$jqEIBBp*~1AwM6 zBjEIZ9B%&kR=9MbW^))6v$VStz1#1#pM3favdv^8p6J-a;K_+Gd=`Yt1nNoH+^EkO7!Fe9CA-%o5*}e*O>vykDxCJ)i2- zNXaK4P%J4G2b%{@H2r`DD@ri-RJ~M;js||hH*cf?5GmPn{fC4>6+#*pm_gIq426jOzd!S^(Ht4=Bz~Cm=YgTMN}4YrqDk4*+W@z;bAxle~|) z*DjC?TxJT)S|eQ77Xc#Ew^%t=0vbRx76&dFxXu8mKY-m37#~Ib#8$3k`wVrluV~fh zj2Rf zY-tMX%G}~xfEC7Cr`TAasx5F>C75D(WGt>k%5Kd$Rn^ZU@Lw9u`|>oI0Ta`vp$BYe z{PUbdRZ5^u7F%1O+a%Sta@7(ufMtt8J+L6=aj9J!=#^Y3%4GG~ywLX+1Eo5m39!GT zbD+Bh08di*^XENKDL*nyu)C?Z>`Xe{1F&K@p-uGwmKa0> zQ?4VDpilEWSLaR&lYs{Vzo-zvE1vdNz!x&rQJMh1ofSG}Fu?k$glf_iX=EU3KDz?d z+_T27ok7$u0DgulW9ob1%M)^y4{`O~60>J_mwoY6F~0!jCWr1HIvIZoSXLQ7JI}yW z*EQBBmi^z~`)20jjn@hJdgJbsMk{%6>&=l6Rv+=Urxk<7Qq2UWfaZ=lm@5^U^&aY5 z>wSJ+d8ALITbh2-VtsD|SX7lLo{+d0JLSU<47etS;X<{L=4StnjAr|FeVSG z0j_#6?SOANJu6g_uoMA^KifEEvB+`h&=sXI`i6iLl01DWY2%&XUQFj(LH1&;Germ~ zeCTMy{<lQKbq8 z{^`Qz8k?n?So1OV_lp(gOv2_W6>T0FdlMn7-|O+a%l0QLc(%p^Xd0KLhn9P?;cfS^ z6Jqv~mu`Q$#@NXRiT^T95y$`5Qe1BBr%X@A*Q{u*`7dRXvu%f=Tw&X0qJYq9v2S-i zi^om1cE1i$Mw0(R!i6Fa!uhRSHk~};p?3|EnyME{SiZG@v|i-B@l|YRq_N#g$NtSk z@Wj|&>XERQYv<2^fbpGsc?7xDw!Wg~4$fh4Ei8(b*Jn%*PF#tCr<7Atw-ars#B*9> zf`YflJD4KF0o8d)XBz-{b09(oHKZ0H2KKuFdcYE980tf?{rk(Blnq5zqncgLCVjBt z9X87q3}33+4Z6b`iR~tbjt&f#E=q21Gt*Xx#!Afna6cMTdO!tn`{mmuNr%z~a#N3MlG+V1yB$r9 zjf*9TRTC~F8iVz~3;Bo*!F|6N_?%7rT&)KVS-2livq&?YSABBHuXBv+ZXwNWY+4}M zvJZ{veeePlbgW(W&bwC=Qu`866Lz(b4{+6#4>1fnOPL86z5APQfMr(GXPnmYFaaT) zYUBmHly|Dx)KhvZPrZv0$xW|XwLT8<#NQh7N&A2XHpNxV63LiS!!lYNUFK?3vG1Om z`FiEd3y_?k_{%*+{{zyxSe|WIWxa{p=Y0@F5f3e6>BWxi?f6YYc(KI+S20!+O! z8{45LWH-kzYp(B6cVS8~Sf&24cT&mkLbAhxu7heVR*~|{H68C_wt|mQBQTwtrA;Ra zKUz~9Vvjr44og9e%br%Vj;)BU+g6!9TjrRTS&}J>nkb@wF=fF%^P`sQp`JAmr)M{v z$OGO-ba$tWOD%!a=yZD7-U0aVAsY20fzMo_MVR@vhc}cXL3|eQWBoU81^r-eFX`9= z2nSv);`ioge|yvU{1JCF1Lf%1D=q;TGzJ?uwyl+AGUYMmy%E$YkUD?+!*~Zp*kSnb zWe!sm0Q=rI0N$QeSRy5E!h&gcYpvY_FooZYg(sdKM`Q@_o(Hygr5l0eLc$^t=?|-x zaljPs!L!qM!xA*yO^!N#pR=b&vDgXow!!`yDYHyfNAHcbdrqqmBEo=|RUdwz{Fd^L?RZHI6% zzj^jBe!9Tma>JCH6OuKvfg8_3a|7uX{yw?dGc;5md?UdE>g9Q=rVtt$_=B@@D{I0u zk=XoGcJk(GRNA=5tJzxGn&u|BObuCM;z?uW@#9Ak$W-qM3&v6Hcl7nE!qf<3 z@45sI_o3jR&@rx0_~b z^)62HGOA4HB~XLKxgvUjg&NTOOLUKp3CU7U2NGg86Q7X2t&K`<>^KWDZh2UcETh zjz}?`P`k6DW|GF2$V!dfWfM0E;5h0RR91 literal 0 HcmV?d00001 From 5add5609cd13a7c4c32235cc30454c153ee912f1 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 15 Oct 2019 17:44:24 +0200 Subject: [PATCH 07/57] Add Source lifecycle sequence diagram to website --- .../documentation/io/developing-io-overview.md | 4 ++++ website/src/documentation/programming-guide.md | 3 +++ website/src/images/source-sequence-diagram.png | Bin 0 -> 73657 bytes 3 files changed, 7 insertions(+) create mode 100644 website/src/images/source-sequence-diagram.png diff --git a/website/src/documentation/io/developing-io-overview.md b/website/src/documentation/io/developing-io-overview.md index eabb8d9919ec8..d0a6ddb1fe014 100644 --- a/website/src/documentation/io/developing-io-overview.md +++ b/website/src/documentation/io/developing-io-overview.md @@ -101,6 +101,10 @@ For example, if you'd like to read from a new file format that contains many records per file, or if you'd like to read from a key-value store that supports read operations in sorted key order. +### Source lifecycle {#source} + ![This is a sequence diagram that shows the lifecycle of the Source]( + {{ "/images/source-sequence-diagram.png" | prepend: site.baseurl }}) + ### Using ParDo and GroupByKey For data stores or file types where the data can be read in parallel, you can diff --git a/website/src/documentation/programming-guide.md b/website/src/documentation/programming-guide.md index 59a322b19bfb3..8229c35e4b85d 100644 --- a/website/src/documentation/programming-guide.md +++ b/website/src/documentation/programming-guide.md @@ -3107,3 +3107,6 @@ public class MyMetricsDoFn extends DoFn { ![This is a sequence diagram that shows the lifecycle of the DoFn]( {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) +## 11. Focus on Source {#source} + ![This is a sequence diagram that shows the lifecycle of the Source]( + {{ "/images/source-sequence-diagram.png" | prepend: site.baseurl }}) \ No newline at end of file diff --git a/website/src/images/source-sequence-diagram.png b/website/src/images/source-sequence-diagram.png new file mode 100644 index 0000000000000000000000000000000000000000..d544eea3d851e5ca1075edcb6d66c3e89a91a981 GIT binary patch literal 73657 zcmeFYWn7fq+cpY_0s?}RBHhvo($d{1-O}A9Go&D$BFz9wcjwU3Nas+aGz=j%L$fdM z`+0uv|9!vg{c-R8d%nRn>sr@Z>pYM1JlC;~6{(>nkN^0^V>C20d_@JB_h@JjcG1u< z60jcuPu6WjdeG2_&Ouswo=zX--OXHWRb4Et9MRC2^adKs1@Ue&npnb_gtmXi;i-=O zMqnLgO-J{CfYmA&eEIb=JniZDeOfaYiJf$Z_M~BFHP}* zm{aD(BLZzIH1aQZrqtL`lY63U@INPI)}!mzV0Uw@CiW2jvCkOv zH{uLA#3j&&s$AK<7daYug;Mp6%)u^SO1Gkb5YPw~othA#y+5t8yk2 zXIe(E)rFyR6Z=x9_xVcQ%ac!^NX8d=e_j*tgS^S~)c&A`%D#Zw7zV!6RT(Ck?~9z? z)mZq|vKD_P+4OfG<Yaw3KkeZy^>hoeO2R(ov2rN=hq7-rhcDs^8_)rfN{4Q1Mes z8kOFAPKLOidhSE_b0nL)*=d$9oP#`YDZ=S1zn|+-u=$H9(^nidPc)_TXt5N#LU3hD z;~rbce8L~d@1)gG=kJx=o~)iyK9R9vBpuzZ2`2*;45$|?K$e&JBxmhQp@c9M>-y<= zH-X-TOtwhM1My{0xHn78wdL8jW{*tPZ+Y(gTkxcL!O=c#SQVLaboh2V%{jvt+hBM9 zFCE?wOSUJ^ko|%2VR5_3yu_-z>r{pIY+aHISzz@duA-hh04f38IiU%y%_Vfkl^<5my?Op_5nZp=dV~a;RG3yY#M=4g^jx_yK*4SW-$UQ zz2I&FrCN1V_cn_eTI*}yqmdV6;%b6c-aPr_8f_Km4!EyCvBiY2-#;F6Tr|HD0AX|u z<-}yYcA(ItYn#zZEhSOz-^oBvcpu?;+jD`YF3ue-Db-eLX1b2g)A~5$uv}?-x&653 zQpkeGDbTmp_5`XM_mE%gYV}(w9qMt@Pe{*?$7M~$&}?YJ>s7G!TTW}XEnjnvHXSSZ zVuy8K*Me5?Yq5|Z^~)6uTuUZNLT0Zd1LJb@3m&p0veo5;?@Ke%nPO?NGp`)ft)#^! z6oz}B7+_H8Gsn0^icPxDfS*uuz#Pu zoMHuMShff2(qup$CO8C2_gblM5Gc!UKd=jmj~}fl@?(OIH;uWKABZ^HYlP!+7JQWb z{PIk;zi$&9s`5DKa0hdnYcK3jW3AIc>Z;34z1h|Xet*&39X18FFdoE=Jd{Ug+UfdC zX+6=xxU=%Jpw2q9Wpu%m`g?L3@4hNA*5`ql6Q6F7hj72x?Q(xf#CRKbj~K{KUMl9J zz>Tv`lg4*`QkiA{_EJ)fp$9vseIY+?xSUU}w9mCj4n^Hfy5n%YF`V98yq$yyx%JK; z2pomuZlnk|Y-Rqq?vctS@XfGU>fBTmZVN9Qcpw=8jeQssp|;k7A0p0U1l=OCJTW3(B1!%7$mnt50f|!Ca|O|HtBtn({ypAT6AxD{9KI907#S(hQL{Qa>Z0q?m*{yOCl>_f=Xw6(g@?((*uZdgqaE2K-iJh$ zCzf14IdRop_o0g9+j`$_PDq`&^tR1kvVH{(!9SYKD7&Z~uf@lH7wam;J;qhlkUsKD zIgg=mptpSS9eWH94YxLx;%2@{=!rJR=`hxdXH$;Si4I|{#4bIqsd!<$r0u zfO6?&gG|ZmFln(J?%q)D5BjeOsL}9*6&F%a1eHQ7lm+khr1XuEL#2MV38Ckr!HOrpaT~ zEH#?8LWgxfTGC*goiG_tOA1rUkBz7Pfh9)e&+gaDV_>tWIwRWaGKtCpQK)7_wO!|;Z2POo4SK;Gc z(J4$$Z{hNF>iBp{_ZaE?zO?T0$4-A_vJTUf@~ODw5S7`JizXzS>Y{xB4gtEb{%L6i ze{4NLP6QLLgj25Zk#YAvjvA3X!JD6sR;t6SxU^eAa=)a#<5yOl=Cw31Ufb2GQT2DG{{v%Y6T}%2f5i#Ef*Zft!jahihnomOA9D zjD9C;=!JINonFkw@8z-u&9Jvx)J$zrkXALKgLPWW70Zx@blWp5&nQe$@Ht#EGV4jx z_!Af7T07{fP!}hU^(bD#kC*6g{tp7)H({ z5+5uVFDQgAo9?2=Q!z56EaE3S6ucgbAn;fXrERnd{5gAs8fz`!`Y=HTD#x?=>|xRv z{nPFKr^5T~m%gpSN9GqXWm)9chlP1)3O3@9rBk8bn`MP*4n?{G-i@Jgy>@4x%T#;v z`dCY>t-L~)c)zjmT0XqKlm5y$V7p|$zU7!h@m|owJcf_C8--BYK>GI z-c0Jrc<$^?`;C0y9O9kk=haC^hd^OpXmGw3t~!%xK9f^++>%=eF8aX%+p_I5(Uq<# zd@zH@kaM3aUG(uioU=w1pA)OU4R%M%xA}%li6G+#th_VP1WmeBUQ{&AKVHp)od&sX z_jXn4CR#9x)e7AsriRv%B2+M1BJS6u{CoM-rOj6YLK39x0qGXhT&2wVM`BM%bTZh; z!u%e?I(Fg*=!UpKte1ST!GtePKUVEZXB#Aac*OKjOH1P6ZM}TSPl&sEOo<>$+n%&q zf`1(%)ihH@vo2?MImoT-k+yc!!T~c8hCY#B-48BKPh7OY?U6E>CoNE&Ei^e*nJ}Gl ziX^fRlVD!8x3a>z7vvo+|Cwre;AnZ_E7_sM6Z)0)1=pXhlm&^An?|umc*%jPMiXu-tJvMuPdXx{m@=?>>W5gSsm7*)+@Sy0g(`6K!q!Cv{N%%ra;qyYG zFji*S;=-hPj5|K;81O0;ecVgayC)RfQe2YRM%Fboif3#cLekw4Za%g4)ow6$cZaI4%sCnRJG!pV~51 z$}yj;t26wr48SzTeP~N()|PFs^GTOqKaG^qyM(XL@&kN4#U(ziRWG+x)1!}wryKZ^R*E@kB?mJJ(My^}Y`iZh21n4P-NxJ8AYd(vlG;0W_3S6{wZJ$?&Om}vampSBx#)4ts>d{P6XWNf(3zl@qYE5<-d;#dEoN+nX#!<`WDxu56O1!x~<%i zXY?!hQxXIF?f0i4pW1AiGz%Yq8-fyx3fx;SN`I6$VY=0t1+Er`e=FT`ASmgr4X7-i z<4Sc)5!1L{SDAh_C)TZ8Yd;SokLtHBY;)ijBAGQqfqH_2``%V!#S*Z51ad<^(* z8mxgt+a1PRM8dzSscmaE+YfWDH-r0*#^fW2)AK@AKaF z`|&Xz)w)`h94wn`9+TqL?YiyifkrSptrLDN*>*;q+!8P7gg+K<>w5TjwRUL!H>_vo zg@Lu%jI)1eFz>UGpM&VZIoW+L;V)f5itNX5``A{RK+a&<@svLZ#;1EHUW%=z_H8ML zzQfU2^d7FSIw@kqhj`C2@i{{TL$K=xR41nFG`_vlR|h;<;zex9VPgex*%= zwmFkIvF+m})w#*h6aDHM{GC}a1k|UTW^#1bS`$90x3EK&tD31Km9K2I zUg8w*Vsq*H*njNEz2Ikm`UmaJo7A}40D;tp@?8?mN7`*Am7o!Bqk-e;lOOdj>C>u5 z9}(QVkkTVEvedJ+j$Udsnjkg@cyS9ptxwYn4gAfU{^Ju5UDw)?D9tUC)z) z$$rf6XAb-MMfh;w@4;I5^1r)GL`J=X7;zs{{#KJqv7`snd&Vex1ICp_a4y9bmQ6cXsia5gkuen@lD61 zt*mJ(&T47Df*tOz2L5(=M*nBXz#qTTbYlSH10LHspSzljsqwF9_A$(KZ3#0xtquRy z{T-U0pKKd;X)$lGIAl!Up`p>DDauG{eKgsJ;52JbUk{k7zJHfkLjPTgoFP3^S(Xl8 ziH9+tF`wuuDJu!S4BJ!E7mUg2!x~XV3fT1X}_PgD^-U>gmgOwO^fXuB%8Wv{%D4b3aHz z5o011F7OUthd~&jf*VGI!+c{CJV2yRp8V%;Qci;zey2qemG$b!80d7>&*(*Iv%B#A z=}qBfa658w1jN0XHX-(74i=OYEs}K9aqS!mTF44-S7f41ygLu0nsYvLP;7q!v0;Ao z&4S_j{51X`LHbDzpTk@b76xI2BwC`H?Z1OlB9h3@RLp0O4j30j(avb{43mG=#fb2% z+PG;*QmE4Vo=d!Q6)RVKy{?FuVG2@EH-_`@{2hv&Dq7-Qj>3E?ztz;!J-b`44}bKr zX{S&s3NkWRhA!<^@|13vQvT!3!DV4VX!N3(YCSUlOkq$2UFPaurgevI=9F5xUcne6 z;!JdX$u^M-mv9f_Jz0Y{wuq^J6JNEc!Ir55-J+~1%tD-3jf z5nT2t2k81jtVimfuK!4&C3Zj+0`W(}aM}YrZBDmF>r4YK`?y**h?*|dc>2o_kqGcnyA4g?P0j7) zc8*TD-fNMqG>dYxA7MIe7lF{ze&K+-OB*l}W^*uS=;GynbA_lHGwB={gNPi&i5w1Y z+$-eF8+pW1`EDcza3|Iyf&L3b&u27Yi>sZ1_bZhYBjeFzoV9ziWzf^4-TVCDTnp^{ z*XS}JB)u`6#k`%;m&!!JHlGMFZzX($CTE{flPWF^)l@QHu zGQG`Xp>lU1;auf+TuAtOJRG(=U6dJW1uR5`QHwhg1rJ}1ZZG;~uEXh%%2H9igS`(^0ty{!@{;Eq= zYe%8*e!V8qgBC5g?Wsim0NE~mq{g`6TQ_fe;NK(iz%53=@4)>mMto_Q?i@Zjyf$$a zrk$%+qYUAgt(bSOn#wvOXd2!^GE99w;Pq< zKiya&$4fO9^4GvBsfZ{aF4X)`N@laFFlcaJU+nOQis)KQb~2>_9@J{cd<%TPkwEOC z%j(o&rug?&>n(Cg)m(jDN1SM3FVA{3qwzyG#x>BW+@u{uo`m&-Att80@;0^{_e{=eq6^NP;WH$oRszc6^tc(@8Gw054eU8Jly+W zvE})C5E|x5Dga%Z#5j#{_{R&oG0EF>SX0gT3)`iiwTJ|Lh%Nx+tEM z%V^X;3+E+(=QVcEc zN(aion`79WF)^L*)7|3>ePU6AMY=d)`pv_PHxnB}u>z|*9az9xCvqA!PsA$TrH*eX zNBXF~V3ph81@aheAP%EJexP7L@onQ4uEUg%%+KKR+cjSB?p&kyZA<~#_+^c7)FBH( zh(`1}t>W6w0Y^4GA@_D` z;}IS8UpI)Q&nbEkY$OmjE$oVuo1`W@6sa4TsRj|YBH}cV7Ij;bNmbQie)B$q3L>N% zQl9P^$%Vs9MN~D+zbDU|tD#;tL| zbvywfS~E_!a;)z=(miT0w~Es-knM@IzM(&J&ByauT|-!{)ruVuZak7GiB@W4K;GL9 zwG#ZwGKdMw(1qgrodUF{iP;S^CbQ^?JnmM#$$axI847|M3DQ}n*&o+%SnR!acva7e z9dGK!#XR(H1{?Sy^YG4mw;|hOvo;xT0OW_T^UY%hg|>uNKDiWxcG_$fs_NiF{hZ&E&q3seCTHVCT*q>teHy$jd{jn6=JR3#X zpi&E>_QaI2#D%Z6NyV+dKH&mkdLD$-HT9pJt1#TY`ZAH4R-sv>7Dq1%S{ID$^B5Bv zuuAokVScLhiNow=P4Ul886c8ZnY1ta5?u^?E2->i__kerA)NCia!(xXS@lO5+gQ{q z=$5n%k1}M|vEo*!BI)Uk`Q>ENd-qzlvk^*AKDXRjZ=AIqX?i1iafdMD?Ljr;P9U7C z$*>!i?N`JNb~4pkM4s)B=^_nouP(`ucity+&z-@7YYiFXEw4WENUY=~O5JgQDC=c9#-%KwLudM%L z8#=BW8C;jxqryz*S|CnYy~ybxo?KVmq#^Eq6rB4sf^1i$(WxOFW#l66clGUHvF68c zj@Go*cie;ac>l{)2P87jCw+`n`%uBb9i_njZj12xPr6y-;fNJ-WLW)R$KwS|>^gaD zF#o>IF>!P+Fv;o8rk#|nwux{5vRYg4z-WZ4O#5gKYkx@j9);HV8`a7-6SVX1TkT6!JL~eag;9RjcS)Cxzne^f=F;VB>F18~V-V!wAlRvI zITfO4RhDgz4<=|(XO3<3kzcX7rq-I<^G3L6@9~ek5;6|-a6i>Ik!ih>D&Cg+I4 zQAux1Zj&6Q{DzgHLbaEkd}rSqnz8^sY}qb%-yA-SKJUThZsxI}A~NK){^@G2k?vtd z%Yc|@C^W|45J;qMcQdNj{8|tIq-$eqeTRtqD+IUhlWd4WYMQQMt5O|MNA1<-4CgE7 zv;QbP@lGB^Ovrqa@vB!z9gH1IVHYVszc!I=ZReUdo-7QUfSs1LKQi6^T#fz}>9U4@b7D?Z~i z)*$z1l&&~6zctR;WS#>2LoMC&ShLYIy!M)C7t}S2sdjcjxLa~HJ=EMIL2Hm-<#%gk z_;!=kf!96k+R6VkQKVc6f?IKBMtX1@t=2d)(B_Oq?Vm-wCC;J$Ar&v(7B`2+?nvCN z)9d+tV=`NyW#P&B>r{wUk?|ZX&IOdt33mF@et&_6I6L{`C=2rRy=kq|SqLpd7>45< z8SI{raux+-JA>nhw^-glms7?*&TWYb4OV6%?U)nqv@&@st_r9Q@}sLfKLwtQR+OLq z+|;T56R2$uZ`5If{rxh8gKSd`p&LUwd=;~K+y;HTh@}%)lGE?rcXI*z6o;M9 zV*D2t7zxmrj+;L0OqrrND+UC{SL-O)GP^Z-lyA8rSeF5d9$9VrxiI(B0BC*GqSe z6s|z%C^ipguCw}#?-5#!$j4Jdj)w)4OUfJ#{86Xt1wz zsnDIb(U&blGAE$TQ=`=?BcOab$z94fK-IX&nUip#-52a!3cD4352CIS2i!S6$0|OJ zeJpu5z86|a7m&!e%VwAARLVFgEG7dgz4PR$d_@QZ6F7Iaize8_Y#Ze}6DQc7Ldy1j z2H$L#eH>P+pHzPSr{oZBc)#+20Ae`ie7{3r_{Odth%y`wqJA0I93cN4Bxhz8;cA(L zWU3o@|Hh(xdjR1^y6D+vLbm6wq&~Kkn>*Z{rh7W!SXvfRCWs z*(JasXFO|XEc4XMVE@nF_D7;!?DJPL{QS(HB)sYU z{lbO4N3J`Y4+D*FTU7TJdz`ZxrV3Fli%%;3K0#LRm{G9#Q`4rVuoXL9@lIe9r#=rP zAis6l{4#Z(y_UqyQ!NIPV%Iz1sS%s>Zt=7X)$_w@- zL732+3dBq1HF&QZ6F=9gL@N`AH7+<&8DJL7 zKAx%|zB%QPwFBq2|556c$#0M+61MpIoXdSQEL(Su|L!)GMS$$3-h{U!{IG;GmX^>H z0~UF^?QyP5+0)oFh0XRycJa(P` z{xr4S?|M_`I0mHScOSAG;u#%ox-k0l^LTu1PD+5~GbGKrSaighN7a!91GnI6B`BaH zS*SMZwhk!|E!S*cRq%EMp#P`Vvlkcy+x!pX0u7(=&^|pMnbT{$zqMq!IKF zgkDx0IRWq!Mt3{(@>w9}RJ%ROVx;utOjiGd` zZ7QAN)v!zxx*(JEqr~OISXz7RPpXeUd3vs zsIg^Lt#zlfox@YXb{nvJndw{GPfIa-r@E%q4=t$c<21>v4U#H7v_2U&yi>kLjl&%e zbC>pSL}(i5`;;z_H1sP1Qo`7;g4-g_NIgvIFD)KH;W(VhzL!X(^QXgQio^OMgB-I4 zDAEJ5kVgPa|Hw}Uw3a?);fg~$C3+=iwY%tRw7I%Dob180I0-EP@AETd(EaDs?~U)F z?Nd!jx2<~~O#x8H**)iHK9R7&!^0(hr&{{H2I(t8k^K!`izrVgOo8>WIONKARE6!( zln!bu7`eKByHa_K4;F39RHy<4u&2t-Kbc6(bFU~oT-Lhci=g+&dAFzse_^7 zM$u-!5qe#I-MRqpyxHm3UO(aTR*GIR-QMj>kMQK4u_Y$M05W8{w-3Bw+cXcEHZ04>!c?T;{G9y9wgAgxoEj@wrkzDJPT5WA6TrI>V}5a#0(d>@#{y=In6zl<*c-{f{<^A{CnsJ3 zH4s17%^aja&=^=-_au>8EXS@qt((A8ScG;_S~QRy46PT4J_d_SvIJHAy~4C|Z=A9v*llc2<8oPq~j9|(?Ze5HL_ ztX-DUem-X?$%A7^n?M3k(fJ<=H2)!+|DqCr*}pXUDiD@V$70C-fysYA{o5copHL#7 z3R2)(r|W+v^+Fn;*cb9J&(QjAoqV3T1}^294fQ1{1RePi{Vued7y8?Q-3t;$ zN^j3Lx+#mGEjs4Nrwa3bhjkK+MxX(c-nvU6=pJA~VxxDQf-%8LrX-2kQ-}D=zlhkT zmW)5Pa2mJGJl@yt!ml4(l`UUXoZXK@Z?7;fhzh3gYkPD|NX*ST76MF?Pz59M;|J4r z_XhmdYsqVK*t15D0X258o<3p@nhEh;CGV#6q||W>jDA2=6#7MizT;Gq>f)2nGra+w zd4u0QpLtc@{_1*AK1<43VKEblD#uIgu|Ict+L_oHX8v~zz25@VS>p*C;R$ix5Z z0lx?pdLMw-c{BCG5tY?s2L@?ZnWw{)(DgzwWIs!l;m~^~e|+!lF^EoX5bFBte6&a4 zsZyM`BuhfXk)OuE%V00{_yFW0RvnLuQG;KySL?no^z=?lr|2CdG%Z^eR1U>nr!UU-h1{W4W{*xgGsmqr61#gkJ_d8A(ruD9poMj+Uoyw~Tx|SeBa2 z@>b>%V%MINHaf)`bPixI!&JNZ4P3{*PA8_{K`j3>zI^<23cr za5hR0Q<)D?1!g&|)(=^|^8l#fqXn@(Bo2?*cw<3^0uAq)-8Hhz<)9^f+#o$_}XKnRj`+Mh``3$4wUM8FEcE0gL#0vZ|*~<^A+;;XB zG&DBTtJxW>N!09NBuzyS-;4%ZaO7>rp^pga;?(gBH_teE@uXk`qT{zan(C+lmqczk zxGauApv#viv6=D5WNfyOCAe*JT#4m0vua%c!HPxEj3ATsKG^$?+BQuqu@aUl|LCpx zh?t@CxC%v%;#V8e-SJr8u`qGLp2BwC*@scq*viRVyS?Z71KCcTOQw5w;Dhl_gC8&O z4??tY9e;>ZEMo>;rFitmQ94A6n>b1j;5FFffac^x-wZQ%2u}{{zC6O^BzWENXYx4^ z3cIKQNMLze;m_T*%`KXOTOa|*-Spn8aq&CHXR|E}%cwc3+(u5X3qEsxu8STGFOb0xGC(FUWs~$`KNo~gVOg+{isO#=#U7g}$N8;My z)^x5`k{#^1|ElbLLzJ;WXF&sEVJxGeZNkL=Qk&xS$57>5QE}wM%*zuMitDv$Lb7T^!_tW<=oo0r`+Go26060_q=-%T@ zf1Bdq;+3(tCDPg=V_OzhVdVIxrMPCc!esmqd|@I|hScYZyFwkhRB>kjhr0t&bi2KM zH2&r&F~6!Z4GP{HJCUM8p}EUB3df$L@x^%f?XSZYf)!M?YPQIbc5Q?G%p&K3?edKu z!rM#W`nE|ttD4)u^N7-duNc0&zpvD+pqnlOgxuu@{+qhVJ-!l~_QL}^EENCwixcf{|;F)eiyC`A$FvE7LS>7|5a6ysIRqvRX;? zgDzy#pxW9+QB)|`b<{7qes0Odf3i~@Ced+MrQ=n;4Kr+i*TcwaAu;)GM9)q&8}pK) zs>7LqImY#Yw%p#=$FqP}(JbJuXUC^2&fhxqhW_?8KPNsYq6u)PH+J&e0U0(qdl4)= ztJPBG`fn{jmd7{$v(0dFrejWu#@eOBj!`v{buH5mEuEJ=^6O~C3cWsJ#Xpi1Ivc%M z*iF8t@wy>WoOG^&rVdd%QzP77ftpf??so)>I7L~%?nIm6AAnQDv=vtbe&J@Z?Bi$N zq!Ng^$~sWdoDXSLlXvSZ1m2+up5OX?7r&6xdH*|9?;oo_OxCC0SCUQS>h^aTzux~m zpY20R!a)7PZr=OAR{DshF;hV|%rJxf`v8~2)ZC&GtiKeZHM{u6ImzuPYWZHvm#m#x zR0q#^p*GzZjs#^MuX<7}*yK|ANNH!jwhEgLSP@xWV@?{Y@TqjV@<}RVoOVMwp_=U- zv~_%-Th?XX=GxH05x$oG0QxoB&MSpxg?QjdME}XHUa_B~yr(aCE|5K`@=F$082o4~ z_O7YphS=19%!60?V&%gRUP4 zq<;&UYIz*St6*b-ma6eA1T{&&itn&h+q_>(zbc1(+@5^4J$L8j7bs01o6r2MNk`D(LK$a>wPvD!#=bx-181Cv&F zn9}wBz<9YuSzM)`P;Fw`@VCxO_gLaxq4Q_AB4mtek&R5GvR9f(Id2Y{c-(uiQ)!cb zrCf}v<>!1{P(<$Cr0X13yFWQNcXFxjf5zANgY9e6Qst4E$Fs2^&->|?NqW(u=U;+! zF?2FvqPJ_$r!j*LbMWoo)R(w)Is!gf*Nwt!tkMlU;Mrws@7ymPS?>KRa|LS{b5FJw zc5hYtZb6@Z%cPx;_*O80-?UWzu&OnkKs3m9rTgc-X;S6S%b?X;WhLd^7`{}(()aLkNCp8KfmWM4a56wQ|DOF8Dl-q?&z3+f|(qh4|T= zv5^cFTpkhe#)49X!2RyuI{6AuGkMIS+RwJ43rlzQWv+x~#~LQb#S@=ua#GaSk5oCi zXsf>+Cq@E>KCJ$!8@ew^`MwY&{nvW6#+X+JkjKOKPKN*ZT@)u{>C{INPFF$j{X%?f$#Yci*;qU zS0hz(8|8(_Ef$xB4knaW*R!$vg*pXplkd%ZRkO>?KhqKtKc|WLhMgb69RwziusHq? z!}yQt!kf>1hQH3+X)TRza9_D5dL6hXWu9|41hTT^vy5hMly=l}M^UTwt5ezVnr7G$ z4~X5v)mk!}n#0MHn}DoqK(Np@CwoX|n9ku;e6^Z+)M905AYIzY?r;p0J*{85z>naN zcURlVuTTP28@A6pF#-U%P@4ke>(GGmeyjI9v^R}Sg=R-&P?c=;ybIE3as7nC9WL!G zT}3bZcq#drsg(-!`dVy>{Nzf2Cg~U|1`?)UhySrkQW| z^;H?D$1TE>*GF9s*m6id$y8}%4Fd@Ebk7xdrL$YI;ohG?&G?0ZQ}HpKd-7K3nvNb`KIcYAFaUt4ve*OA9*VEWn1#B9y-uzpGF z6wky;<4dm2>h_Urn)gUT0d^EJt7+xhYP`%R8=Npn5S6OZN*ziMg}N-(!x$HjcHm2D zLboI`ptSFVhIGK*jxk+=o;p|Swtz{;L8r=%z;aPS9`SoG+jL&PYM9F@s1)zw+vXE5 zdV}p9$;*EB)@IeLGgKZMjTPelV)~HWecYL#X)#8g9K&ihUT`nsClm_zj%UrQ;n*?t~@5? zvTTPU8{@aR>%SBzFFN_VH2e;cUx=d;*&li)XIRD6?sz<)yzG+9kC@6hNB?oPy+D(G z<{ItU*TVcsWYsg)P>NJJAHm5W3WO{~!0o77dBCSfP7QlG+sw%gh^t#4v0wZ-(EI}? zHCOet;}|=;6JS?m(?&h$L1~Y&pBef_4B>K)eg3-#e2Yn5>A@-H3M^+1GPwCi`Iog!_Qj~I@|bepmaC<9AEjDRMQ8p{=Eo*?)T2(1!9IDfk0 z6bM{Dni~*d|ADAu%)B`wO3dCuPZ<9ppFe<32{dO^3;sK~4;XK&8KxdkW~ZfK0y9fO z;~Ry_lf=pweJO&D+`FW%xDn0GAG%5rTHSh5wpzV4RZ}(wBspAS%9{R z%t5wFJRe%eWr9AQk`YFDx-8p1{Fk_k$OM9fvsM=N->8oEopmT^mpU;!=s7^yq0t-v zL+znqx&zWQ7BPUx$pxr9tUlshGpysS(eeMAezH|S80G-MhZf<9{_n?Vlhp8lF3o5G zB0ePb{eaxsXeRg5-PPvF$NvnOmQVnlXV)!n4baKb4q+0iDn3#{SxylXJ_sh-y_)>x zMTqlu*AUQ!=FoOV3z$t8uTkmg#SIw{~!?+ z@Bt2U<_|;Oe?0r=&tMW-!k39^LLVN>kVyagWCi#L!^?jro_izsGFyQ_BtpO?;K^47 z8w)BIwS3Zl2vE#Sz$_oGu@JNgYM2XxT#&Z*?E$4W-&6GeKii{Co*$jM0QVn<#->C+ zbP4vrc_c~eg!5;0uiopHnp_a@NN}!9YZKkXf0_3`XyJqxz_CinZ+W@@a*h9G3|0wR z4SI-s7fHm+uzz2GJ_UfH0XU8%oy4V^#@j?{Z$g*V8_Ji7WBtMatIPiH82EuPbR!Ao zBxxbo)ow<@zTolrJU}~k}E)?w7(uHGx=S%0*pH3|Ime| zen1Qx92X?A!}6D;O$I1*O&Bx=ALBQDknQ8t&K_q@0B0fp;4JwOW(J>wH6Ze-2DJ71 zkwi@Uh^o%U$vpWbh|lV7?_~<7p(CI&o@;b6%oKF{&hWn3)hrW`_Rf~+j(hJfm|2aa zGng=_1@!q$H1wan&G+U~Dc%K*l zR~?9y!V|_LT6JHOcuF-YLqDtsmJ}!GP)|y>=8J&b=YPBBKh4ysUn@wC*ye%5uNYkC?4AUfLSZIl8Nm z#xT}D09wrZ-vPnRpf8Ga#fIpLXt7MUYK(D;sgywUOvA9ny+A}g-8T|!wVOv^H8yJs z2nEa4bHpkGsfH(%vy__=%_qa`PB$0FT^sS_qp z?+U{oCZ$@eBENzjcoXklT>~0s=Nx~e{<0Xkh);}a_3W{$>1+d4s~Uxj>IBPpD9 z-62?pz%>LTeJKA&cD?Fc;6}_Cu%0$-tn(S3$xz)=ts$$vWMf_Ur2O$(UxjQmS!HZ! z=hf7?*v%j5rZHn;Yy%Y6WEDPK>qS+}4FHq_{no+oSc$H~q2OzLjo-5U6zXpPc<7ft-_mmi$oHsg)M$?V$-4-&H< zb@vd`5fWiU6eb9=$^K+qZ4)9&JdCBN`0E}LwD{N?fYkB-S>Gm!*_nFs`9+e|uVk=8 zLxG}baU;@#P zQKCTkA5Z`l7H~D9eC;idy$s0yBIn(IAq9!~_dZhY7~fwGL4uY9v#o=L3eany%o#DK zzMyRm*zF=k`EA!mDM0_HS=c?qY{n?dv`XH`<^5e6*S|~i6^+98^NbrEUnXEl!T;?Q zLO|mmb&5^&*XUS(jgA8teXltoL$%Gz#m=O^r9x(lm#OhA)ToA=XPKn44MQTHBDQLW9|ARtG;Kr|saM?p|Q z$(aTuND#?Lkeo9}6om!>Ns=U}t&&j$n+z%vB?w9mN{|eaGfZvJd(OT0`@fm_|Cw3; zti|$pqN(hRDfxi)n<4+^ zBM!3wa-_uWh;cpiS5qnhgM*!-c<00qz>q`@bfT4cxEBE$LhRHGG%PB{2ls~nEz#Iy zcR(;ZVTTCjzy${;LKMd*fMi}1&vxvzW*g?EL2Zg)HRIoQ0xf6uaPIJAF9B>FSGI4IqcqClcBSDdhFSi#5o0GI77 z7697|Vb(g5-!Vf%0QNBWufs<`{~m!wM-@F_Lp~6y`8~N9=X#0&kOMIaw(QZL1g7CQ z1U^GcCmuL#>G(YjT;f(P7vO-Nc%*OuFaO)a34NL2&3a{yaj0VR^#7E3;e0tIlyJY# z7%Y7IpV;_+dkbI3yz;@n=yw>O!NdJS-X3Y*vtBz|!cy-XT7K;Y3KwPbRX@ znQ02j)R6{|1}%FP5+dO;G)O_0*<-Y1WlZfg)IgE`eK+9oRw5va8I^W_zib z*LOXKxS~%7u+jF;a@#1aQRhqL)In%lR0|awul@Q+qBmUSd(d?W8Asozvqn$Q4WE-PrA`i&x#ei( z-ujZ}J&If#{r7s2RES2{y@<-8;!F0;JZor?dX?(8HN~pOn6&a#I;1a# zMadTM7*CFG&+U-j)ovq)k0B&j;1+rTW!gbYTq$f(bD772J?z)pRACQYXnCA%WJ@n% zkbW?G$HIM-1CV@6nrMHo`fFp!gfHD!6L<`u2htak8~5f5*q)y)Davz~_W<1KF~`#0 zK8L#m5WlR7>awNrn-a{et9;aiCBxxE;evI{eBZ}L=6#y~PLd0uTbY8wO4S&Ew>xyu zB4FMY5(#WLPgz6N#>jDPd!8@-^DB>7Fpg#nyZma37P?)khUf(?2jiH z&Gwi^6$3zTOJ#a0;xHuQGxZ|j385G?Wg!^^BHrjlFeLsCWIzT+O&Ixva`CkuQ~M9b z(BktBAbTG`w%aGE1Y9SopwZG^>NZGG1x)Ks%s{4fLAR&@eee7g>kj%Q(wmC3tf6%-?qOF z9eeWEd93JyaZuO5!N$389QmEbk5Pd;ta|!WN|NQ*#%NUe^#YlqNJ#32(3a?xocyvT z=+!lWe)96|>q*}MS@%Of_Gt}F6rK1GtI8A0w|82c?Vx?v_=Qq}dy<8DBTECp$@6AN z$JS|fwVKpRV84lsyO3DFX^dqnUv9lNK|Ry)BD~nPx5Ez>B|Kj;`lLr&&K{KV_KcG9 z(7c-~b^n-s5Q8qJ{(O{7DJzv`VZQXcd1z=`gSOHI){dF(0*81mZTF;YXien&n5WZ? zQTnb8=}^QvDAwOSDPi|%!v#nO6a$L`{~1Ee8Qi8QjplSox|0w-r4PbqycB+1)*c-0 zR8-f~+)~93mJSvX@s02IPAuk-uihgO>oI>|YTm&zqV<5b@}<6%E~U?|a=Q5XfXf7cJh}+7>RRRj=o?@? zxq@|R6YWM?#w>hP^fe7?{>%LdEGb^_Kk9oKf;Y)Q@YXs;*1l!mvLsvhtqL zO#u-DTeK zwK;9bk&>6vOZnb-ruStowyT7Ti7r(EX)w-%c2RqBDgU!g{b7J&Lvf^j$}20Z>)cTObu4lG$0#bn)^~ROKf6r0 zr1js~>aYzJa}4;d4^DKU<>{ggKC@V}%p+~e*e}WWjrw3G`9V)*K=EZ27Vg&yM|UxKh}CijRT6@i3keJf~aT7|d;W zPT0fe6h_u=)6AGslC@F^ZBJ`ozCiq2N`ZLX1_W5ojeF1X;M~G2xZtznBoTvuuzZ`zK8%kkIia<+v0qQY= z_HXDbkxaS)ZcN`z*^u}jcn@NwU&l|~+7v7h^Z$_wppM{y@#Xce+(_YYUlZh9)FXy zi-YNFi5oz{*uqPywvX8mguZ<3&GW<^QjDE5x1#rkMvu%^-q@{-2)p@v8}Z3{V4j5? zU;?LAMCT|xSJY|yKIpH|m6A{=^=#5}aJ(sgZsnt1^Xk3(tOAoGf=?7!etBlE(-?qz zT zSOmIJ2v(CPc=bzOw>-Jl2@cOf}PKyU>BpVIv(vBI|+6SDkMba#IR(m(@WQePaGcB|jU z4+1I}j^Q@_wXqj7GvwTSLBO&-U;9((Je%(Po5j{zY+-_SeP$4V-vAQpgH~zt<^*03MsGl__7tCx@ebF$ zT}!t)UOh1Zt9;icw*47BWAZ7ibd6JRp(`&1?Dcc}C!~RpRXN!VSmU=yNGBHCFt%yW zKxW6-%nwf+C9Ocu!3|?is zo@ku>gdIq^Z<#kfYSm^6NidFruN}5yq|aplYt5PXU?m9F>hPbe70I=08c#S09!$5{ zT%NGtUVbDYKArhopLGWUh@dYo0-&1`&vK~O7HdBievC2#O4lYwT1o1>fbC@A0^pQF zu$=PGrG;J7Sh&gg3TUk^Ckg~o-vSzNH$rH=GqY4_$iI=VH#N!AC6hL^wnCGeR&s%u zs?Zd(UXf)qgqM~)|86sBGh6aGfUE8l$Fdyhlhxmc|DZeqYu5hHYliG8FqU$}9-r9^ z-0z$I;bE9$k#IZTu(FJzjl3K22mqQuR9d~Wq_n-gP;KDP>7gy4_hKdbVdd5rj0E7h z0dFU*XQ(2j*IcF$mAYxkAbhu&rHDnv4akkoSR&6~pA3dSp@wOwp*ZXEg8^ z-}&{?VDoN;d{HW=Rn+YK9>kaue((4;o74(o^9?9|H1(eyx!>#sPE;Lycx+I-${~Yw znh=Fack9x5sV(t|sg=o84)ZE#z0>2wLd9}qfBW1YqSvh#G6;{9GAp{H<{#=|HUNIH zV%utI;S_K6P9YaXf@M@ikg`q3DA|;5PX3jjeW(3?eShEeT%M$RJxY9Ul79da4wr7* z-oh6Y-#9%AG_zv9oPk>?hMq{fS_L+e9^t{(6WuULm$Q;A@q^LcX(st6_+$Oi=X-XsIv~96 z$Tmo~TN*BN<>U?aGrgFZma=Imf;Vrg}YgS2A0yI{dRz~NhVJa{4bcXwLOW9{`* z+7+JUoG=V&ny~ns_#@bLg{-+13Ei1=SFAN(3iJi5P}R!9hfowBYGXJ{V#c_|H1%0eBXCbJ)Ya^S}3-neQUV%A2tmo>wmIM z66bQq*%I`f(KJrAWC9kgPf9)s!72DR{FKBwYO*OZ(~;9NV2%HVWx~7dcr%)|;1=7z zXY>f~!L$X-H~HOXZszM>i>>@Wy{J8-=;MEL(IGt=p@hFaeD{>~zWHXaQ&qfF91bZL z=_p}1S?J1Z&tt-e#(Ejw*;{q%7i)c=k+|>K_(%wy6 zoB#<@!ZGF!^I@2;MMc6c`RkOx;XjMuFC{DRYP}08rsW%4D{x$~A9Nx4oO+c})I;vc z?>x1E0X*R3Fd$WILOCG?4?LuJ3v)>Nfz(LSJT+P$g$1Ib2cbM5 zxcPs!xu8>63;aQJjws-PG8&i9d^fA^ZeseBCrD9(cxGfBEu#<5;{9NdDE0BYd-|qQ zDZ$n{kn6|?2OxbmU58AN0%wF0Pid|7x9ss?Zx#6p#!8*Hce--4d2`^u>T@q$2r*Q+ zv}%8@R#WHnV^QnYg>l-+VV8q_WQZWNh01e&)s{6vYs|r@saByGZl9$_HanO-Lz%SV zx4YE~%mQ^hM3nx%Vw=Hz$VrC~_wNHUqXu+S5s)EGB5EcZcB)av5n7GhyA3>g#~Nb? zhgF5=E>rQD1d9T_!Q*3CXr@pOGnKfVV1NY7WG*xf=Xg(V<1A-%$CmP-ONj-m~FyLxEh`>O?)av&w%eKp#bIrSxXEs z`)T-@u1RQ|b%}l4K-7BMBifG88UO(cwRyp1{6FAQZqM87mABwiX8Ajn5_sI6>Gv*h zYGwBT#Wmme8Bpv|5?Wh_%GH~!LF7GbX`aspy=U^~7oe8r&gcUyA|EgZxG)fGoE4AJ zPRL{aNwmUniPQ79f0Ha9U?zx72Sb;a)tJeHV}Ze>-*H(O&Ot^?ioyfGL!QyW!YKM4 zvGR**WlFwyO)p81LhMfvNc1s9LAOoZ;*3Ht2 z!XfgFFY+HBoHQ`fHuQS2y2#1^op(HBWQ#Lp2M5$psNsTpE~H? zZ^zTj^Wzuzw;qr}So&_QG%|O%2=T5~HW4yBx&`i}b_pX07T| zigYr0yPOjciTvnkAWZ~gQQgVEfZ9WV@uf+9mzFCkr#xo5F36j}sFv?JafTuMW8_~s zwWGA=^NS~&N>o#Yz7+%D(H?MP?ufrOTxcGS49vR;1Y|4?%pt6WAo6Ot8)a@q=hE%*jp(kVt?!V`d?_clgJus@QCiH1z0M7zwJnQS_W zU+Pm;TZOpgaYRvfzETym<9xNW1}s1~GQ$aLr5{2}!;kyqvQ>-~MTyt)&r)Y^W&FwW z)>KLu|E;N4_P*r;uZ1Tw?&(t*7%>{y^dN(!%Euq;U7z6KDmaE?C*=Xhx!eXJO6r$8 z>OVd_g@**^?B?MbhNBlEWzY8`vPh=%T5zH{P@Vq@W9Bl{gdVd04-2UWfXH+w2Iq$%KD~8wu9K0-0iW+dWk?^D2v2~CA8|q zrad}`B7&$qlp%R!d$~>A4pM0%Txqu7H&6ifZP@sQ7|tWoNMA_QDGoQuM`|5SOE%$s zwrJ?2FC>=b>(U2&3f#*YGkE_sX5ajiz{bcxT)oQ-iaLq?U}=ty=byZ?)L+v& z_Y1!m-oTO3(YQ{Xz_U|upn{v9FtzRqr~H10KXP^PC#A;@umOG)vV1Y33RFTYZ}}`g z$bHf%W`vjrx`8F!+lFLwY{8M9cEj+OaLs~5h97~e|1=cnKbNln+Uf_UerNd8AMikW z5eiuQ#BeRP*!khc->7C&2_nkQc*{4Sp*D!r=4r*bT36s3A~FQ+zvSWZcEnp5E0_7p zcOGJuF*?}brKl^~z@9J3fmtlog}+@^3GgudYZ)B@EqPRrq z@)u~CfM-Y4b7AQ?>oprXpM)-HGK%ttPOf9#K7F!JpK*xAP~32R^5)nfSWPWLIz_UG znyVtuqk-@RN=L4;T#i%yUE>L-?>Q(#s z+ME7#`3>~c!C}Q{R=5(wn*dC$gFQNteWyKl_9eBdB2hH$g)T0w?6LG?ho6Cie-|Rc zCjyaVMEQygY~?f*61+4Yxc8C%1)KuvKfv&pDjdW^l!hV(Lv=3cnAs;pqVmaq9;ZiW zIM?|22XrZ%YN!X&l*6H3XZar>tnqmwG+a&UVh7P7Q2g$X-@r(E^5Aql%10c+y2wA@ z`$*wHGN4A9F(7)%{#Y?cP4=PU|KD6fa1ahv{r&$%0QtWTLH|Fv!vBLP7wJ2IL|`Wh zM+g5I;Bi7u+@?WD7QVg-hlIii(EooV6gWwT%8S2n1tO2%Pm!`?{`Zjo|MxGb@j)`8 z|6zPM*ttt!h$Nt({jb5};FnapeTxE@hdX1y5K7 z>B`>$j{i@BWlMebnBe_B&|vwm|8*0iI@!}urMHo`PnN6$^r~I;?%LaIt(?9q-TJ05 z*ilw$>jlMckKhgtnt=zrO$IOpe}Bx0TvT`aZ%%mJCaxdrfCa9qZ?^ZX-PX#~8dP5N zU^p29rJ-DN$;h`8>8buZLAwa9E2L5HM-d1nh8;L~JKjn&JMp|9!a5k0L%`qF_>a`{ z6!<$r@O>Z=Af1w2y*}OZNwmG1w^JQ+Jf*hO5A#wbRZ(uxpZd$8|NX7`!G{+!II1@& zBiWmAUzqDt4Y}Rtz^x*6X>o-=soah@a&>UN|Mlv+nBzvkg3dv$`NZxWXVdz~V$cz8 zL=j%%y74R+%HSPM3B2i#_JEsl0O4z4tg6^GhTZY5Mj%pi0h+Ht+4rZrwUk(Q8UYj; zs%U_}74YBN30_IcR3WE4i!Ma_(LW+af*%3zB{{-iq=D#(-|^UQ14nj!xaMTHk}T4u z-2MF*Dr;w%7lv8979>+yX1BCM+N0=4hrr?bnZo=!#|mGnWz_R*K_Dx6h+1ad%J0LTH+B5h^t z&5Macw@hTk+X0h({SwA^Ves#gj7HEb)CgY#;d=`hG6MlbSNIO8*A~x%)Zwi(@wHlH zOf52e+7@PrvM#cJ*#dEJ|LqW(J5b-QMD&?Ji;s6Tz<}v-0kZ~DgfF!ls^~?CZ7vWk zh(hU18{al~So-fSMD+?nq&l`pbpa&QX zw8)4cq90fY$l-J&UtLAS&Pa_@$iIPCRb(-EIW`hR6yOaGBbV0Vs2xG(Q<{Z^Bh}(Wy$a7aDFG!= zK^{mH+Z>}n^6Y^gq{=@r3X{0|wEjXBA$)3NJ=}ims|7+fY5G(o__q#O%pxbX;{X@# zsro>lIOBVk=yMB;KzQ{ku%(W5y;ZUm1eq8SC7Kq^UHMbF*OW*atxZ%~PwVsf=!tX2 zfIOazT;HkX@$=wp%sb0kSD+Bl; zbxw7Z7A3c!Q6OF!*o#yqVxej%v$}tgJzIKi0-n|g6X3|ZPp%$6@UrTMpja86om0Ns zl$RG>xlB5K7JfW2S`Yn-h=R2Tkq{JAEQM{C%e*a&Y6{?V*=`kHFf%i01NlYlq%{!W z&4!%E^B-p8_kf-w{tK4>i{PMZEJYy9!-F|}=Ht^!jNIz^Jnyq}H(`2k0vTV^yMzmR zT_EqCENmrjh-R8eb|u<`+d(F@50))0j@I|rqA3i1YKO3rDJJmD)uj|`Zb4tf6?4}p zq3O&)n8X@>@hg4+Ct1&j&rkwC&-{e`f0oks#oEV7#)5Rv3`uP;I@%5ZrCb^WDMwfn&fuH8*3K;#J zA0{bf5PvjG7*VP@fX{5C;%$8Z_&y3}e15ZD$e^GWLefO1A?El~JPuyQ-S-4^uFrIW z__rQ?*AfufTHxf{Tj^AVN#TYrI+o_SZkd<$f(ooUU*>ZgBzd5jiNxY<f>su6S&ohx$i#T zJtj5wh^|&Eo+#Sc$@iu6JfdmYl)D?3dh;)-E@}S+N3A=CAv@1z3m-Grl24U7Z6U?- zwS7dX@{L01a@DOhs}o_uo9tVm+L)YKqeV)qf~z-etqUbbH6F?mn|BtwO7J~HGe_Jw zjgQn9?!vyIqbMw(Wo}ay$&1px$_%0b7)CL5h=#hN9nC|SW}@Q1#NE>LdflZZzn6ca zJUG6v&zhnD#su80dq~+@KCNM5ID`F`;LhFM=aBbTf~EG9uPdv;vm53=nuh9qEO#sX zUb{uWcLX2>J8RRrhfXZpWs$^0x6xNCO~qbG&tT)7T6K@+3tU65K*h4ec?qfS&THMt z%RUmpWf+~Gjmy^_B@-O4xD&6SEk0Q0*5Bfab|o6J$uGTw1tv=zxEcc=nx=sDsm^-y zeW)%BTx~2MpAI{_sAYaHpQc$1ZSyI$a7d_#S*n?p2Sj4FUgil3+IQvpa*`$DnUg)+ zOHK>xknm{|Rmj{0igENL45X6iqX1rQP8tV?l#mxyTv!P#AEN^x~u9jl{R#ZBbq%$MNwlM8i@AqWC16N>>=7 zd@i&SAC9Hi4Cm3f`(4bzgdu!XviY6F(_^(3)xarp+ClM!{zO z+D%mXp(Udk;7|J?HnwdbAZMF8i^T)8@lFvWW4?HA&3ammmQR+n`yI)qR--Iueatws z_Dw(+(;Sr@G}(_Wa7i*zGmVJb+TQPdNp?1?>3~OBv2?e6+{rRqE&Rh za&ck5lL*(lipS+Sl`<*(Sxv!;+_LT-+7EQhdzyx(D|^WU`*uZm84ZGzS*D?G?^2#> z)5&j_|I)^~^U^9DMpG5^Y}pI4Eb79(_{Ej3%jo`TN;b%Zc(b)oj>DiiFJk{ z@zQzyXV}o?2X~IP!k-nMJdXaaCnw_U#-6f?YjE@xsL<0c#EOdz^%q7oRHPn!j-&pMtg! z8=`=weV_?r8Q;j1)jgqMvpe@HqOXWmKc%XXI6^rVhqE)vJpF3sDI?nox^XUA!LK6 z@nK3Z_kn~E6k}f9OIV*h2StmCbah5i##W~Co5!LGBYP{EM9!xw7HaKL2Q$6OwI!YX zc=RCeXh;Pnp!CuRd=ZUt8@zvka(>EH3qHIum7Gyj*oo)zG15vEPBwV5gPOGY&Ohm? zLs#fMajB3DbOVB67z~#L7ab!HNfVGDJ9N14W+sNOj0P!ptOo5|e~O$}5MeZ!2`HWy z;w^u=U9a++?`lFYx%mRW0AX+n&Vm97cK?I1D43Dqu@|T||M4-&g8`SE9+&)= z2;SYJ$x&!|^vZ3o-Ot#D8Y!qnedWsh)pxPN-5QD<_SLWA?TzIyrj0m543YaqM{IT% z&-2QYI8aP^EDXg%oEQF7bK|B+6h5lPg~%~rr1Z^G3+C@L__yZgXGCju=O!+-iRKNioB`pd`hje? z>6vW!*PkTNWT}-d8%1jXGUKO11-nb*N~_D0U!H&S>0eD+DRCOjUJr;me?>9hsG3L6 ze&7)Rd`0`=s=Og=gvVV|Y4ILQ06Bg4#i1o9Na0k456wbqH(i5kcb0}cq{6H@H}hi7 z@tI_Lw3}M^U4M7kVMyP6(P`3O`0_rZBjx*f=OzgGUr_oD8JGKTV0%*ESM=04C7 z$Ei`!hA}E}WRR)0V+zZDNO|c_=1Iau6(2{XbTQv{;fc01(VSUJf^5MW+J$S~LofEW zR&Cr)4oyX_cP+#AzvjUuR?DRl_Ey0gQVF1{_wos5s`5piNZ>dG2jh&Zx-jHE*>#yv zv`q?@8OL5MN=z%l4&}MyTW%5(0T`83T~KCm!I!0I2AvL~!CTr`998@b6*dVXasK(414;`C_WdZH#<7etDCA_!0Zq)eX+S@|(u?P_ES?G)?BQCjPBA71m^ATvP z!~1bP0Fvmy?eB9NCoa>uDuub`^~#wHYRi^A9~{})Tvo@RqBBzGqQBLh-vxDw!v5Ya zxJGXHZew0^;$B0uaE6yququ$Tg22oI$_z0z55oTU1?u;>DXoFMcHqP|)!hmI@BIlsk``>;$Y^Pkbt_4xPQd-4@R z9<`;5`#(~J5^@|EJ%^E_aW%f#KH2f2SOTI)0axIBnY5IAu_6EU7v8C9v%%F_f{z3Rn|9Fo9}^R7h!!7IZ#T^!Gw?gn&qO}FR9H<~2OIqu zdV3WrJQwk28W;xbWQF+GfyYtrmdKrX+t`pSoxya8n2x^CDZ`a^orEJ6GV4|2l%4n5 zaqm5*zjA0sN4lGRPQ1|@Vp-n?UlF9K<7rmuL|;A(Ra0X|QAp_Acje3j_iDrlNT(7> z+0`-vthFH{3Nr7w@fV&7>3$;q`#^M9;1L4O>Y^hQWDUQb+ot{7oAkMlB2mtL?t0F= zrONfH?mRI==#ujs&0@pIGkvNgS3>zXBagYhE1IqiIHnhb6|3GD>h}*CP<$k$mJ|)k3zPY0~t%hT2kp3O}p;fu?G5w;c?^ z%B&6gE(-D`yXeFwyD_HwM|}MH?=Agop?d#u0m|Hkkl`CM`ZmRJSX;gS@iBzO`fFxW zEo9~9tJB{k{WjFa&Gi&&QV_S;^2NL#Rnv#h-jPQm&gI!=k+I3{6_3K6p1C!FB>P)W zP##5k5^8?H;snLmfY>mgK=la(FD<$h%iSJQ}tZV2ezTmtdTsv zv~FhPH3QX(prBBiC+QA3W7Rg=_mvydT$*Q`{OW^pF|o8t-D_%uO$AOPUE z?YUSiNmVP-mBDoUVR1J*S-k})0pfx9$?8}K&Za_pKA-oJywx=UN9o73GaYZzXjoQB zo8kOwQGLBlA$%Rjcr)ncgM6(=U26E% z-;81U7!~7U>lqT`yAf3AdtI$(ex5dF*gL!D3k?NnPcF$!!j@KN-434e)B{rK3MuVu3n-_%deFhp>gS zq=)}kr5MmAu^jyhCD``p3@QNs;D<5Q_xIoMWxf`E6=U0(D?InOTlGTm`0aEYSs@=d zRXE5`3F1I`jw-sq#dw%%oIMhrVi<(+!GjK85mCWVT|vL)kqh3qcJOiu8CW38Gp3sD zSRA4rrkaS+9>D=+P@4XcdmZA-XCTz&OaKBEmHb zS3V&uJf|4ux_rVa@NlvCL1f=??$f|s*Ju!_VSKF3D(ZQW?TKL=6p_}2n0W9L=DLRb zO^GxEg_TLszB?2qv^!gJW54{W&c47KmK zYgQA0%i|y_Th<)ik#Q0i-e?f?pLRPT-1WI6aWp(=Pu@VlftsWA9R$GcbUaYUl%HaY zxGzyNxp{w&=G-}y`%`!%7VCeW(^(3~PdJ|%9)ugbyapRa!iYn5B#>Bg&ibiJ^t@!? zn!5JGG^0~fb%9{AcWN*FHrW~uy!>++=XxER7}Bo8GvvTa4wXDJ_)RL6aZugheN5|l z(Q5XNh>wcV_Mby7es5HO2EIO@1YhBgIBAh2FdudN6~?;%yKk|TeaLlRNVe=nSrAL= z6>q;A7^;wX#=%6gwm*0Dim=2az{Kn$M~EjsSjG#&U}h72mam&>u1aoyDGAxE>^wsW z5_8Fx@iC=I{yxhpWQV(;LA+>&SS{IWHsn1Q{<_zy8yWCrIh=9uDh77pq36-xNAap` z&FLejn5a~))y4uU=a!|vOeX!*K1*#2^H#2nH_Ua^Uh?1H{6m?CluuioJ^rkUe|q?_ zZBcmYOOywnS~An`bEJJj?V+t@oT|M(H=VW(^?W-C?-Tlc!#$1c6N%^N*WXqCNEKIY z>Z6Uo3`V_@S?C`BD6TY8l*FRa5fxL!adZlI{AImg zu{!Gp-j$VN;~goXp-hadxl*ZA^`%0(MHWh6E90pBHMeX)C__hj{0QCi=XLtA2UYy4qpI(+qUu zoHEh#gec*O5-E&uCtK8obXM)Cxp-aE$H~9?L=*aEG}h*SUN2u2pgzran{4{j)ch1x z(Uv$KY^MA+miN*N)cW}3+qpwGYR~XJ3^!~@t|q0saC6(;c{l1K`8iU0j^$G~C}&qI z^&0LQ2`I#@=)4#jTjhW2?#Izp_=-ANTt&#Pnq75aDDJ{NVf6lPKl$9f9`&c)=M<94 zL#>yyl$Ff)XKyUc&r!v{$rUhrSF8M3J(4Gtk<=Wwz^J*hx()RN!@=U8+U0Ef>4ev> zHY4@IIggmPiua5uXPj6xP)V)##&_mdmnOGPH}1-6aCBwADr-t7DHfDd@`U{Bla9j9>8lLYvTBQ`9WI>xi8_zn)j&yQ=t1CY;s0{ABD2 z7S1M{eTzanPwi5#=}ei(^IFF)Te59h`)AF27|);EwEM=5rW6b`<1*4vvA>1k-+M=} z(<5k_h+pBG)3>8jtq{X*cjt6e`B~NGer?|IAFIu~%7BzIb_N$N`pg{&T*9sj#?i{v+U zCs*zePNQPpIZ|@0uM(okFO=m@>&nNNHSnpwd;BG!ldu3CkPd%`_mqgpYVDP42+ zk|f_i&$<$k=dz-5{@0Y9;fcP*?jgISX3SntHF|^YMJ(r$R|>PWjb`QveKGHhIlj>u ziuy2KvL1`y(&%aqNpk63T^R4IFLZO1FLjy@mD#URZSub)*KvCDa%j|rf_lM%zh*SD zj>pB>eJpE8UG_&|8lsu&EmkqL+LiG+ri#ob*RUJ?*}V*FRz5}&t)b^E5U-_IwPvL$ zUvs)m!iYdI8q({4Iktm??syFsPWp;K>w+#D+s(G2<+c%M{c)p%V|;X4Ii5Wp6>X++&qUF9BCA)*KJ^+;PT99u;*_?gnTb9AEGfB7 z>lQv1KP>SSrgeX|9PMno>B@qs*4<8lBYGj0%OZ9~wwWV(wasBWF4OAm{5l}vyHlDm zSa&XtL8px5bBSAC#8Z1SC%x{@By(HexLKv^qL2RiZE&Xf9@p7;l<)3NlD=q}Jb!#P z`NREX`E_&e)uGV zn!TQc0zEk^gE9*7*u;TKN5wbU1O8#N=R1|s#J9AwPo`bqO%sVLJ*k9BUF2eRE}ATS ztwRa77QDf-qb9Is$0Ddg>#K?pFZETv(|v*a!sFR^{4D+k^c9!!sMX_`&1a(1v_7v$ zwB9T!#c94Hp5vygRm!|I6V=}OmaQY!tbyU}%;{)W7mv)wl#}3Q2CG%L#4qYBetczh znqq(3;}b3K^*cNKRsrB(-^xUdv)OoGR*$9mi>fH$is<>KWrB6g`=SW)D|H0#cSOs5 zw{?OUfza8=S*Q<)2P(wlbb(ovQ`PjuNt`~6=j zH!fAL^mqtouFy#3*skA>5!z?t+&)hBaP%{L``pKH1)R`I|Hy^d6lSZgf$3zY7h*AwmDY1UFTc1~9vAun_x3xBr3}(b zjk=^mRo)B~%@&V3QVFs?U&7CFm%s?76aZRxY=F5s+j;?wwdu{5?!eeqP z@a2sSjA+b^sy55bir}TK`_(d0t8`S?jp=P7ngY!2#1n$8_!XYArZP)5tw{RaRv!uf zvCJZyA2jVzUcY{$dz)oTZEpxGJ-zcEpPx_+PzO%WSg;#C)W(iPKK!4l~-XwH{Z-@LCxtl$rN1J7L7n32n&<5g>^G7>s~5?7xSix53^}_;A92S{gHIuiDgBh);;N6o zY2s^C@}?=rK#>wD)%D0CUBC8x6hZhUJ?-oh*NCsLJmlJpUJ+W7_(b*%-+y74A#4aK~e`9|xoDGJ!rQ?Z=pkNZX+D=p1##)*ro&K7K` zm(nu*MWNj{Yytva(Tn^0$1V7$l26JbK23Cbr!t4IWQ1~((Ib zPR=Yh{$=7&inubtCvSAnuKIdz?bla(M|imuqF)*?n>M!LCw$1(!WaI3$-^?tzn-g#Q5z&tJXU*y4^P2Bewnp``5ry3tcW0 z@8A$@;RieP&VT&Y20`h@x@`IEqSWd45wY??tXGCAkE>}NxsWgqmSB|oBzx?&=;Pu7 z)@z(yH{wrRtvMQh=GM#Gys?qyq}3{PoNq?hj`6Qg_m1v!o+f9}qEHpTn=BW0d4`jv ze_dlD!*^}x$Y@tiRCd)1@!7?~(^E{xS=83M^2Lk36O{V=z?eL!#+`HED-yF?P_mWE zItA&8Q)1QVGZ{1|Wb|)*F2JD)d$C^~ao}BR}&A;B*!) zSzrqui^f(9o|;OozQtSZQFpPJwY-%iCl2Ls*(IF0`LT9tVMgyOYdl<*?bbUTXR+D8 z=u%M8U$=fn-=|Q-rWa(bwvzUylRSA@_%@~5fp<{@a|4>&=uOL?svdW1S(z?<@iUPn z=5llW`_}imt`{>!{qAq=8dclJr#`PH;*0h3>)y&(?PgyH37CDe5cyn&$KJZUN`TS+whl#`T+4B6bn;wB35j{5!=CgaRNB%8v60FZ#4U8Z-$L3n%y-$H#IU9joPUL&_Dggr zv}AuxZkZ=(y?Ru5|4JtVdga<`xFP^(X5TkU?@b;?wAZ}+nfb|Dy0la8X2bi^=Py>L zO`BTfOOGr4U`_p2=3?gYRx2}00AFuO%RK9fH(FPCce3tu;O@MSedcH3;8hD>)`s!` z>{4g2U(8!)(fB~afwCCg*r?pCGKb`BP#ZvN+N4K;)3*|M$F2^uMnvrdwcapi_MR8GI9HsaIB zPZ!B`4*sc|>gVk6niwuLHH#wklElnxe|>k`faRA#fbN5!s>pHX z%q^5&rsnJFPwiVp&DD3QJ$8^SVu<|Pox|ziY`B^ zlCwI_;XI~Yu4_8DTY)pfD`xQo`BH`^_ofROT>Grp1t$hm_xJ zw=Q};yxLxt-vF{%R>A-t0J&s|I&SfpguvBE{lak;nIXNJ4nH7qt4#Np_u(1 z*%SgWplRaBs~m23OGZ)P6axceIE|dLezjO$=`~Zuj2iaM&pQ}asKw%)_Kwg>8M~UQR&Kr`|iEhL;RWd^tWl6sYev=JJczbJu^p;V#VaPW^2>Q zuX28HAusE9nz6!HSCUrX@d{V*MDS0|m$2Ez;K-KQY$Jbob)5f01L|Y+D6{rNzp(qt zd%1>xs(z5AKswc+I9hBi0tAf|D^P1`WM9DX-k|wn7dpAEfsuE`_w+x^;|rLO+&TIr zmxOekpY=ED4%_#XNBN`M z{`o}04Z4%!s-o*Er>4@VsJsu&*7(LDtMr=ve3>%PX?O7C@In~}TW~YH61X>)+#dcH zAbOF!NA8~GRWq}X<0_r>!1ZX8Ufl9~idql5=;O)+yp^2q*XLAcRa$RgLMSYU9vl#u zH6FkDArH&K7jS#6q!-C0kVTG_-exdf+2Nmle!KJ))<7Dje3F^_91>oPPXf4(vdalD zRz$l3Ra?VEbdHEV%r)vniuN`0gZCKz#q==ZDT;*Ys^OnAxOVtK_f~)(LeVoH+)V0j z-5~zq$_Dpz-iyhWoqi@mhr9sckPa!O$9Pf}NTD|(AR6ZxiUg~+F&lCC7K`^&=T)Lx z4gXNkJgkS0@@)Bagc3HCf=yl&oiZf1B(Z<+Ww|*s%G?>uCMt9ddP5{ zi^WQ9Y}b#<$h7G>d>mT%=uOu@v$9qVvz0zQg(}L+&ve4>*Fj@T$C{>{uOHnezON5u zKrSM3=8NM5ynA#=g8R_+ksxBk!7>4Op}UOFh|Y;t_sB;u$sC%LLo81wmd-IYAOHvY{lccUB?DkcjDoUA?L*4W7mNm@6xW=`=*wHT=9C$C zVVV^{F{W=8ADpkJiReVqiSvNU@Oblh1uOocZP9xorvTxEf5{-+5)gHnz#(25p=ADk zIS^_^N>HU8s|h@!xN?TqP~W@(3cy@Ym57Im>hZ#K(wn3xs3j9W?)?VRJ|VXlP}!o^ zW_!(HbVLmF^U)hU!o3eQL;EbBBKtvIDBxUJ7^;i~foU@os=t=PXx6wajaQ1$aut88 z&J85U8tGQz!xdf{y69#z40v zd7tlUlB;}4-;6f2k`pyyNX}nIL76*e=VRr6+1XhqqwAU?Z}aw*!C93vjPw8HWsXjTIg=WZfKbu8*W4KX;BDnd|}5b?Kd*ow%z5Rleoq z)1==)Q5pv|*kJY!hGz`K428B)P>tsN(FdA!7Ir=kocAaA#G#{3tBRWICbyR2Nh&A{ z6>)Ve43@KjB3!GxJ4aiUzHfP~5&oU}M$j1(a*C6rnWUK4Cnd$*XK%51cjp^WeHJaE ztfCR|R6|!o0Z!qK$)~T?pi_WHDDD#%v79b~6^putu z4Y(>tv=erk%M4@@){Hm=t@T28t}{+5je$LKgU%p;TcNX*=4S0!+g?T(FQ-Q7X*NW`qIl}7WYKk@tJAxNC5}%G_MzO!eE;~_#~|muWF@vrL+1DE!-w9z` zIWbqJ;Dl@Vr3<+!61n+2;j;=nvVAlo%9uYm#0!r)Bk{1_H_retj5Z(Ptw95_a1!cO zlpT05CK*Sk4b(pDKQyrR2Jsy7b<`hDuuSv?&I;8E*JeSL_9@zx@NTG%SqQ@tgEND~ z4U?0%Lr+|Up{`Nc++qTbZ?)%Pi8E@`kw>Vi-k%M1+YQ`4NqzNVPoD*$hFL=~x%ZPO z?#ryl_n=G6ZS(c*kDC2ga0%1@5BA<~J7p4?YlvG3-X$2IeQ|S(sE(uW@q#Hz1 zz)dQtu#rybR#8eqq*Lkc&U0;X=FFVm_rxFPk9V#2tmT?DGs|Z`_jBiUU!SOHQ|Dt{m0h%@x%M)`*#gfW zY7`sg&^GQr9wtv)B5-g;;Q9pTR511{tA*w4!NwUStpk#|49l;Bj{^-Akg1d815cOu zqh0{Z;30?)V)e7STn;T(z;GzkK-mY6YcQh&xp3s)R6Ij%=uPCbB%EYE=07LrOU_RWvpm;=ui$lYv#vD%RJP4qc`D2PnIDdBr-k;7Gy>`=fT>Rm@5D2Na$gqfd zc_5VH{G{;#&~t+5YmzUO#|4q<1&`RcTdHa2X$!Wn{ph%_Kj#&%;Ongvo7JzhmoZ5$ zq`FE+K}Wdr)!$HKB2;1WC-!pa0)n>m(%i0Y7zJN9zim{C7+Q)u<$#MkTD{|Y`8N+@ zsNyT$)$C&mzfIJt5a2fWOpn3MdjOJ&Xr&A)Z|Q(5{?^P@l5PmLMk4PV%@0|so`F)u zn3gD$b*&vsC8Sa09k*j24%uBGljS0 z&1oFCPT6{{AWY;Af@%;ola6pHewpR#(`x6z6`0L}0_UWV$;YvOBIn^{(N2^b-O59y z)eZp*EM8GuavnX%Dg5*Vym;Akalz^5?M`=mI22Gw`cEMYf*OM2rdq>L9!HJFupt6R zW{*Z!lRyRcpu3Z_*UVRh=9pvso$slUFXWeCVpM=v0qgm2x3|?43_tcqMhbn%i*g7I z2=)jn`jHm|oMA?5E~Q))Xo_06{jnUoUE=I9P?cr7hv35FFi+6aPji7hYz}{zM<#+k zYmt(?@t3zv!?ZaF^&>$_owaRda>5)SwM|9#OrGBRQp*ft=_~t&#!A?tNM57j&bB!J za_Ulq)dDqlh?pZaG5!;gtg8j*tCp@>SomZ6oM(Lfz*{+B^@YeTS>&~Hk;8gEa1(uC zIHJcWRD9rHT;=veXK5>YW2OVBR}JLHMm%0iqG!bY@yNQ1Ftyv8$|vgMJmhr@#O_f| zqX_{!@+hBz@jq{(2Qo#`r3#J=;j=B|)%dmz4Vh1Ezas*k zU?I5|95lJZB;jj`m?Yu2XDESmSO9xmJNqvy)DFiae~OcPf5xKeQkl-Hp&tHqp3RxA zH`V9@kL+H_$P!yWUa}2Y+F^f2xIk4E`!L1Z@-PNA1aKK*(BSq8SJrLe-|OJqpLIaN z_*0IKQn}zg#lp}Q`D;GB5GMcg=r{Q4+gIw2_p|9CFx6RpS`z1X)G9_|G9?(_V;cTRR*A_eKf!W@&CgE`I&}g z0h~2M=pT+H0*lMI&gke?6fCnbQ#2k05-fKUI;4ylgvMo5=2_40|XbzS`U7~xF{d+ClZjjJ;;>RaIw@;Ao- zzLl4?%kqmWDje*<^{RT{KeZJJC=y57*T>M~&YZv;Pznok+Lc6D35QW#?@E8r%ouKc zY*hQ=Z&u4MX&TumW+DOtQ3O@kH<#H+fVaf``}1ucASZe(_AgBG{vUnLl^}a<+q_4> zvyl_PikHC96ZKw^`Ja9~P;m6FYH){~Uh^!0yVI7M_V2FDm6n=zhqy5U4r68QZ|K3X zjyQik@Cq&2$3GBH33e3m2IA9upL?`7V;AM6nX63yCsW}I7Q0^clh=G@mcrP~dc(B+ zcZdpjVK$hqp=s57z{!Ul^iS4?j{)pzj{;C(*aO(plCF}itcAe%7XlIH@9~%RK4k~h z6T$be5i-gN$UwKc|D#*sHWeZrej3)v55fT>EKOq<1x950B4jrWyfb@&jr6H8gpvl1 z9W?sq_y0wX!68Lh-C2Ef1wOYrw!Wa+Gs~7>4ioz#-ZL;6`1~@AlwZ1jR`W$I#g!|w zZ@I<2@bfQ253<7R~S%QJbe*6-ySlC_IR@R$2 zFqxOJ{zvMBE>I_`6#1G@sS~2a+)XL*D9k_RTosDdq4pnH9p=F5=wo4WIc0Usz6b^S z#12xMbuq@icUBW-8--f1y^*Bg^FKtT1iGJjkm%4&Pnc|mZi%5YxNh+c$3c!O;Ww${ zL-2nhb^N>M-?dUUn0Pd*xaaHuw#{UJy@? zwR*`$qu>90G^$qdPwvR4$@S0XUacRzR;074zfAV8&)IEs6J~PTTfMe$*_QxNBT9np zdQ<8`>{K>aV*`gumyMHVhrqCPUKQ9kVFglMbh+p1F_`(A+v4=XJ~c zJQE-#u*gTe%PvYM5jI|sPn-Be4N0vjD1$Lt9h1UVyZr>Qd```YBMD(*nJqn&=q2!N zH?yhzvoxvf9T@brF$8c$F+$%Cldu& zXR)MP1SKAvb@N!cZ1*cxA%fmREtAKk=37ppK4ke=0;aaSF z>OwpBDfysinxM(t9@UHA$k-@L6Cw}P*_*y;ok8xbF|Djo-KTq6;P~=-qAb4^A8C{> zr6%vY{ehEX-GTo65@XiAH#!GP-hu}^sOGA5kzP+9wA~qh>T)ojCRpvMIa=d!9YE%2 z=jQvZ&$CIzChZP}SBd~K+`DPy_$BXqj^#&px3yg#i^YSqb;|J^<%uxWg1c@ENU`9K zqwA#4Ip=;>7h67T)<{VqvFN64*hu}lOEc@K9=3>Z7`qeEq>|d)*b^t6fPcSfYYZP( ze7J-dyMe=2HB-o{!mQ&z(@^%P`dYD_TJvnjT87l~>1B(bsFn`hY*tQ5sdhe8F1@fo z=hS{e3$rMKBDp7$NPjDaJUy)G{jt7GyhGh}?fqnYK+geHzgE_9LQ5u+NuMTw_e$u; z+19zX*68pv#@=y=q-P7sxB`O5;16P0^ds5H>jrge?(N2@(XknNr!7WbjpG{Y`1wdKN%jc*+cQLNM~$Up(F?a&rIB?m z8kURE_EVF{wL^GV?her zs_%8g03tjD1eLA830|tKHt%C^j^oNk%D?G~jF%XL6M}*swV{qK@98c~Hq7&N84QMJ zVQr^V*GAq*R*Up|db<@F{07(a*$?nI5r(qctEC&Gne0c%y{e2uywL!fZpS~-3jiS zvF+yGkCf3*euQkoFUyP$<|6GIePo9fj|2nkPqzF8YaJ9WmoQ(~&*fvkHNp7m4@KqS z-p;~I$K?EXp@_@}(L2&ZrDDWWTDZmA8yf*TCXO|9s@7ds0YR-!7^#&FV~b~3m|0O< z>KE{MqCB_R`z~?nxifBb_;$JI0}e>-mvFg#8C=;banrGO+1o@sAiJqRHCD|E@NDFE zykhH?$i){|lAloTo(bBt6stm^PpTCsB!#3MQ7*w}t7f(dM;NStmB%>;1QP_~hNIlL zy{56)%4MRJ)H_*1RbMwFX2Or7&7PaKhA`oqSbZi^H9b7XfVb6{Gk6Bs%xGG$ck}L#@@3)Y`%d+ieuIZKTI3`viM=AozqM@o_HH+`j^D%O$&)9L~QAb8^dR1i8`bL9im)(G8 z5Iw?JZz4i9=OKRcZ&&@MIP0ANJEz_I0JJ(4KAb5Ft33dLq@2~ERVIGOw5fp+E=Lw2xwGNc=bP3 zfD1!2Op5xg&*0yC!#XYH|tp#op^glt)=-wNH;y2-;hY-j!fJ4 zkRH9C_ALSL50tR)>wQ0*Na%&}f0S9QPhOS4>)l&|htVs~=o zT{@|tEY4rFiPl59mJ<6@VpVB?uKSjx&;QgkQ=R;a`SM|8;y`*+qi`yc+r#&bPVe?J znb(YwBxYSQL`Zq+O)}2TrWU4-B&DO}nV=(vpIg_P)e6;^={zbAOPsR1lZ)=^et0TM zzIXG!B1?C985Pdg;Cf)>0Gl;;bb!d5Qw1XDcA=qBbLhvI!x|0y* z_w4ZWzqkMjl_r_3EM07u*!j&bVQh2GHDz_Yz9lwVKv9GJY?*KP6Ljt+LzOKnJQ zXefh+N*3c;!>=z{AV+~q)HT=v_6|{C;wKArG^W56c^$0BTA?Q5B>0O&nsy|~fF|g{ zwg|XZ6uE3XcA(yk0x#h=;HOBhl%xY!MLMu4i$rDMYLEBLn`7AGAKnuek*Uwcg3z|G zBxG@6?b7WBF1N6r?Iop7wzZP31Z_S(?-k>hdAsEgdFSC{Wr5M^)mYgvg;#JAsB1pP zc|rm3foj_F1yc*P@ymT`ly~lXl1Z=4(Asq7B`vX3Aay7$D=f_fGanuoa8u1Go4foZ5R2Cw8| zxoYjq6B+f%`(&ETK12m;BC^?vQj&+uFCruAJcpDe_Cq5B()J5n=Ho6Un#q>haZLh6 z(|6>V$&8lW#3X`7<#1Wo8^uD@Vjm%06s(aJx#{$Bxb86B0OOv3B&hfp;DmK*5xe$Qb(-&Q;{I8qGSJp1K=gP2Abvr0+}SO9gW zxhuJ9;H+OY#VKDPn|+C|CK5|GGEay+p^~AYM+h4>;GmR#IyJ5j0gSr_aRI6E06drP z$BA`|a}G*|sNJLbFa@H#2McX>lTGgi2MUd(7)l|xk=lLeTX*A+tm%jj>^5aO zU4hLt8SRYX8OUlx!@}G~hWIQ^|+fub&=Xbck%_2@c`}YB_(?~V1QTpMYO8HY#7w4sjsO8O?mINcBi4C70-De&CP@`Oc)v&XDs76=g?9zNc zwPL+P1!0wcA_`=bTN+ok;yVrOQrV?W&N(Mt|2=EYjpZ0VryF5Kq$J1C5kyEQgUuyM z5$TUl%?_X*0)VG8B{Cdcwe9S7S~tIXHmHHmB(Q1{JEN0qFj^|jA$*QbEvbp5WM{Ajr<3moFw zzFQqyXSG|ObXaQW;6_nI&u8I{x2p*CoYir*BrlqEV9HU`+$<3y&m=O);czrzq%1`| z>S=kYZl6ONZl&X3ps;_!NKTMF*Yiy7i?E1vmL?<`%i09J2CuThzvgE$)p4ogyC^l`n` z2J`pSY=ynWhiBgk_2XQ)xBWqw_<_up67jxFd&^c-@}N>ouVvvrs)c+Xzp3XpTyJ zq{G-VF_Tv+??b0Z0?A?|+JHuRlQ|=Mq_irSIZ-g@U)uI0tO@U&YLnh z+{21cS?R5J&mPnb2^f{%Nh(ob%dg+Jv$$${Feq9h-N*R)m8F#aSN_ds0(~-(d;)k* z6;f~ZkL0{$f4xs+Ft503#>U_KCDvk0>E=^kXZNV8g2C>K3+o7%4z+jh4RdBZY&#xT zc?LX04lRh-3Lnl2A*fXA;PzT5rmssfck$7<3_E0MOy7nKfAAvm*GV!uT=tYlAxr?DLR>?lxG)4}*VVx;$Cj-^!dAz1%UEss>8q`@eO?}Ho-6`&z- zuxTBtKX%%jRmoII^#>eC2n4Uc5wx^z07umfb-jLiYW#@dT{}68(KMbc;KZKccMND| z7N-GJhP`Vk^ZMqyo==iqGW8~?qwxFCm%(SotSWk;oC_p;>vGXPKOXQc0iEhhmDsff z9KxIqQ6ti4IN3~P))@;0m&vL;#dvcURkd-y@XCNRMDcm|d%ON<$^kR==qfRTM;~$^ zQk1DS&ZM82xiHx@-rQw1(X8y`RF-H_A2+Jccp<91`M&H|qUdhLk^TpGzFh-=)8z`W zOTMvRR1H^Md|fz^Wx?;b@YN5SJMX}x;j&vK~x>q3N|obkAzu zOJO!YC@ON_wEbm3W?b{6@fZUbRQIdu#E_|*mG0pPGgIdybbXb(KF?CV9E!@8uH|Nv=(Zp$zAZ5BwAPnP#35l;5+cAoEp}+ZbfJLVt0QV$S8LChb?mM@d}1hFcA;060G{IfC=XY=k#2fVUm)&7IjAueKy*4#5QJ zOGk&&5@$HqWK*Y+0>A7KLIzc3fM}hUrrlP_;%vYKkQZ3ZYfU?-N@Uvhw-&~XPJVvG zEZL?^+jqVs4~N{tW&wPnRR@gU>ARA1PDgG?mHn{wnad$wCBX-qkytmR;PL{wzo}2A zd;yUA%UQg6k>!sxO_3N!2Kif5D^ICugJY62p$DLw#bB|_(r_4_fcS@{Lbn~OXrV@} zRVd{p2NaBM7iONu_W6Qmb6pG$LM`7Ok(+-j40G>%d)JIo(9a$WQH%khns+ts0I|4d zK*#AAA4Eqd$5VXx*-Fa%?Oz0KTZ>;ovMG6 zY+i{&R~9~|SST&*C4XaCJwak#M*E)jQvevrGD5k~T{uVC z+ZPKJ8J@CNdtf8-&u`JG=Tj~D*|S?|k02DJM1Q?t_{)%21? zrL8g>=(fiNJr8%y)XjsRO&Dz8{YJffR%w4{%STq9Qm)(sNhcSEyHaHj-D69y)0FgD zNtAS*%`acn_WFu1otNh#`=56PUOAoLp|5+_^Wc?9iry(rKdw#c9+L@W_fv;XSmGk` zNM~-C^IT;8>k*AVkMyf;ETbNgg-7mRn@IHh^T_fVoe;>#S)VD|T2aXV(P{o9Lv6&6 zZYRr4Cq+Fa={>cabeV8vyYV#t?-fouhhqms8xydLkjT8q8vKL6~~*z$ee988i_pYS!KWYh@k(xJe+5S8$9ZgQdQPy2W;%P+q; zNfgpU-j1X?PV{x6KZOmYP8Y#+^>h!kV^kLAy7sk&H%pwqQ_GEQ6A~!z4;zI_TN;;B z#i;iT0gMa7sl2yv=1x=+yiUW{TYuPjeR1vD)B+^p@{cJ2uwigdkLSq#Y>$-*?I(jOuC^0j*O;Di1y6v%@5 z@#)<)Gb8w$S8sFjf9|3iQOgl=-&Lj(+PAK`fLc1JKY`59B5@Lz(O76vAfU{Iw4PQm z0H;014R(GZ-50V2HvVtY@BO8SH}FNnW@h}krV0<25$ysGBkBlMzh_JM&T`dn2-hv2 zXl~lJ6XOq(@tO%5c{uq$P*_onk2p5opDv#2d4FdA{t=gdV58oEp8cgI zpIZLk%FW3eHUD>Fa#Wzw+@+|BjK`<`S4&9d7BOK`!~ouA^`CX#{|AtsGV%ld?+F6E z$?(beDHBUqTR`|f(L>l#{;w+RS7iRzI{n}H4c$G2RW11+=D;5o>Zy$X_kWPgvCm<` zKR!xdgC0QrSrx(fMpwWCpW@7l9#i!{a|?L<_u%}$p8xO85q}&Ke)r`6_j{g}=*dWB z3Dj02RZWw%q*Ar0uCyM-6?eG5$bB}}^)I+_2 zN$@trxqlw82$qp5r`SiI{a?(vN^x*bq#jc}BAme3PgogBnD2qRF(8x5D6%uun~i|dDl*_}8pZ1cJTox*kwlRL!9(soTj*uB zIE&G0P4L}eLPtlBpp!UP;k1%eM98E_QQ@-jo`hLh6veOG;!lB+EzJ<_p#YN>;V?g3 z;ufG6cGvBHP7>OEFM@Jw0(vMa_m+7uak4Z(4dqh6|5jr!AQiIOoF*+KTPJX*iGi<` zMWv*`@1;eAEz+ukijkU#N%02C0Sh>!O`l$zXB)OW%T_{5uZ7^2jRu;wj zP!_^$Ca_0RPku4Y!N_sbb`5yW8Mqi!y}&54O4QKoxep(QIJLyk5r;gz_&S8snYvY_ zS8qscA~^JgqkoC|`${!oz2e>7BBwG)9yphEq}Q&PzpC3OM8h+36iAS#>LmDS)!9 zSL>eHewX{IrlIk-D*L&VqFLrC_WSQ&Qh>SE(MwDWA@Gfo?Hu0#I^E_m z4?I;silH$=j3S0#LH91%gX{=|QH9}x7FBvzHu-*gW4%hE-{3wC7 ziY)}3Y}Cb7?6PjjarkeTW;&B%LWX#y8dcZ^WLI|dKELwu&p_Q*P-NmR$3P35ZJ_fD zXfO&v)HpA2-;3r)*v1w%BPPIom*)1HCp#BEQ#J5iEXJjyICC9^zSLEP;b^##S< z_MnOr3fu&D*jBh|~)-l<0KXYIHfNX$bX!}L68Yj^79Qz~({XgrPvk_89$&8C;UK=D+F=q>> z5@KbLjlyY>^f+8=ygQYuV?O6Aq8w)v2QiA`wUC1IuVex`yE!MBsikmt;CkpAWFJ+z zGFB(TJdxvch0IJ4lg4pPFoas9_AUQ6HAtTUyrQb;g{QNN=brQmEOM{e`aw}L_Lwy z(%Ut8tcYc%zZB3(lY;D|N2Jn^8;v6_9Jy(`t16( zIozw?dzT)e`5nBgrlPxyc^R*zAcad-0We{tdna>BCb3c&?e8^X&W(T`|gM*e(Mb( z!kjIVLGJz8*U*swL3{6bJxSCE-<{dx*Z&k|*x1GGfEBM4s&Z8ei1W9kr1byry2XS0cisILZgt)ppzA$rsO*OsKW(r)PD!(`K3TB z%S1ZPvjiTc42UKwqa_+Y51{xz*Asg_;&H3<`2~tAhmS6l#Y}-4TUeXlra8@0ZgfL^ z^XQ=Tk-6`1O|L3P*EPFbhV2WYI>l`v%3GaK?eJk&V@VAAK4uzOI>fl+BzZU@WaMA( z<}5fl_4Ojb$je3*ZpVhhCFbfSOxWUlu!I8*+otJEI)Q}Wq~E09c(`3C6=2HbQB z!4zmPQg^Z7Oc%NLAs~x$3FoP1jL^Yrde8mM%&c<_2-1Ym0+yd&!EVtjRd zrX%06vnL<=@T&PNIE}9+@xTf7aNr@{9p$bt%;)$xUV--g>`=t@5Y$B<^GYZW`?XUq z`rwS+#6_zbM8hV>rIkwooy`GnqaLF!@!USwE85&>JWETY+P(0vbCbtsauE zw3MV^_y+s*2S9?%7cF$V^mv}z*;zRHtFD1l;8~L{gijAQmM>+L5+HK?AOj4upy%;A zc8|UVvX~4!YGEDK_wr=M;Gy^)tl1)fPcM6Z%~LmOIJKJKje|DA$~u=`58c?>5?Q;n ztY@A{I!9R2OSB~MI0K5|1_y25=B8=D!tK+loW+lNoInRRo%`h|=OR&FkSXCy^$WXR zM^Oj=Zn6T>2wN)(?Owg10+-|D*9CKBMx5^^pkd%q9t~S4pdy?4$xJI=aa(WUeRQ7D zeVQEYMppr2y+W6ZqdwUl7jKQCEEL7+%=3g0%QG|Si$*oH+!sFaiyOB=J+PeK5~B^; z>f5Arm&t42-Jwb;f-&PX@4dnG$a`nSpS}wwYy#i2_ah~UeH$18hQSxXSUKh|GfHAK zT7znz%bv5SOkZn2;xHPQqekKFoe2`uD17>YqWXP80E3kG)Ao05l!e^t>8TNd1HNqL zn2q7o)WuZ-p5cpkdOHSci5jTRG)fM zRbbj0qk!uu)jC4kq+*pDonho5;8y{nR1N$YS@4faB}{c%#K?Ys@e1Ct?VBu)ZP1=6hu>`cK^IsbkNMNoG6O;D~b7Aw%C|9J*N z!5W%Cs!=S|TmjMK-&Sm^su-noBPxaMf013Qy-~#L|0O}0(%QEPM)@Y}-Of(~3_eKL zW8(OkB;(=R#Ix@bpy=Az@&A5>B;81o#>`(lT|#-9oC9O=f}!e~a6(8-xYz%?`9CvI zwuzcW#t4+xohn&OAIEG-s9tawik5AI=~J*A`OfLZ)Up*0p2As_-n3TCIY&BKl_6db z$WwclK~e)ZO8Db<%?IF2X;D$syyBe6m9AIE@pDMV6QYJR&h;9G@j+nACe<_s+usR& zb9Mv${i!LOPXVz3qtN1_GXrEf7I3e4F#WnfLc0`s7qPsr(V60b%6qtB|KnB23&ERh5e4sv&_TKfc<Y9lS3>2E(z3=e>>tivq0}5|7JGRy%yEDJ?B0ljs>DBE0kxY_doARda^XYBbxq4 zyHpVE9-MT%%EF#*Fw9P9|DM@8pv8gv%jARee4@7JNKx5Pn^!$3F3p!c=r`#wRZ@0ORs_L3nt$8Gh>inOK5bKw)%9Z?w2sh5MqjD%z zbpIF#Q5c%v=@l!cdJoRwu-7~`=#UB7Ao%A5`z`N;yc7;8#xT=0otqnsj9?EK@4a&p z3+bBwa=-prh{YTy$bCm|?#e;@UcvwSD}VA1|NrTi%e3ABU*9XlBcaOQ<+ayVTG0^v ztA$b=F`}=~*RRm8yv$B74L)jfH~yPE&qL%4@>O@JYRX>m`~%%c^p2wp=930ap!65Y zXUsC=?qZ>PGVO`K7J83Zw4N3NGr&JGoY(cB&qIQ)Q#JwRJq&l~Yfv~O0^EW#1webM z02<#LP^=Lpc$PuXy%?@?Qo_K*WB^FqZev;@h*F@m_>X^co=hxzqE4kf3p3nJ^-%O5 zU`edQt;@s-5X>$pL+C(F0gl!vC2V>5mpe0-(7LtKz#<`f151oU#UjVUoM|DL^MQXG zbFwp{JsPfYbJ|<{&|wT|k#(peBLg6r$tOSDE+|sejJOs+Gz`w$`TF`$)|CZ9q6nxT zBKKG703xKIOV?vi4%`scGoj?pU<%EGxVUEBp$DHjh2oonfST1diPzRJh?FDg@Mveu zZhKLa1!SW|kwR{cq-Al_)j==-qa+HYFR*fyyvJmE!{0?~)gM0=*7|F2@eaY2PvBJn zSQxJL-&3VxVOwefFXHYFUYgF^#!WQ86Ak*nUg=6#$`_UcCvc)(lqwi?_gv%;1D|JU z2-l*dsc%1isp|%~Z!Fm6VvU@C^Q<3Kz-60DL+T^p*tl1~?et3^fe6^N;V40$ol_dD zCWS5ozm2~K6f`1SKA|~VAb1_EY6U2o0i@NcnW_wHJ3FL;B+4-tKk8+xv5^V6+P{)+ z{4yA4kK^pNjUuqL45%!JVV+EPBxHdV8wSm5Jo#Cv}j zlPR3|LVCg!o;&iybww2ekt&WjKH6{gM#sck;vX8ryRiQg7f`7y_S|o<1QWeM=<;+Q z@Utl(MyynXedF`vb^j<{dF&5F zAN7S{@VaINpnc8$>9;47P9NUAqB)+lJoYdKaVtuEB__nYR{xrN9pWD7{HJ%~lqQQP z`8JA)mP;LAU_DrzIVh;u`c({Rbf3>{8ZxNpdEZEy6Wg2xCt4S z1b){iTD8Jx_hceVTp20uuY83gsv-8)3(9Ev-y6v|0T6HJ79a`CQf*Nzk+#_=D8^fg zxZu*9janVKKmkN>AOSs z;i_qSvUUz}(Kc)(fm+y|87|74|AOpbj)mGhjRW}P>7UOj#;mSi$J+cfD4jvRMGeZ6 zWrL@Pyr5^sC9A@-jROC0QfZ7|w~Ag1CKuOC>UPu-xbWKmjY z#h}GCcl*7Ya5%;v`_V%fKI{iv=|1|li5wJ86AwHZ= z7$4?+{YTtWe;aibCbNBuf&otm|GaQZ2t)Fd7DspCfm<^1u_GCR-=YJVo2A^vB0_>h}qeNt^bI~JKY zo>~SD_;GJnjBnf8_lA9#shW4=EIt??@I!BjVjY-TYdbmC8&8vF148@#lW_E4N8SY` z@sd7G5f|ZPsmHb0{a!jlUaKAxFQ3>wKqe2DYY6A{@(y$L{9A}4$s<_mUO6o2Af9z0 z(cC`k*NoG#Ad9)%od`#fBu>FT1hJZ99>c)(UIaKz4GA6T)@$=sd4x{p=y3G&h2fYh zWC#q6QLK&hwXZoN+T|UgcHI(m%UrasH_p;+(tm4o8Xiq z*L~#{z1ik#$07TfL+bSnk$Vb?}5;+-W^lb(; zU;2nLH95YUrp5OCc8cJ~MRVKy(yP(NY3hzP-5MF z^Oo^WvHX6(8@r~jp1BP7n>!{4t+6M;gEQ@3ej%%H;6y7^IK4zWMwiXk>9?f<8^iZb zK0Z>)5HM;bJr))-OJxwRoaX&fneTYNH2(KwlBgd*FD%iLvWA;{jwEW6INF zo{@h3RODQQsmRf4qVZmF9KC$JCga0z^febnlzG3#DzZ?kTeF%}UHn4l{N~5o*F^nb zcJxZyH0(s*Z3dn3A)@5h1||j(mI^7B3VrZvTth4@gaz6y0xV)yR#{bbLWx@jTDB^E zXufaCain`C=mXKk&qPU(zIuBjnE9^nVTXE^Q)#-XYo)bPhyB>{j&OLmywi*OE14C~ z>iE;=2g`>zK8*fQ<0ageZ;f+2@>9w=RB0UC#J0Ab(jF5U#?`@{TVV2h%peGIclZ zvIQo0GX{LV?yoyF!=C$nX0iOWrZd*1+Y>Ho*)l`r5|!dxJyC_l`rcj%m9-VFX@&gA^r0!7Zpj<3hX;Qw&&&$^vvgTP# zHMVYdbP&zm_TRphmQq%+BXXY1Y$4cmtk8%&dFHa*sN1|%)uZ*9Qp0Ip?ah|^y&jVm z0vX0E@$Nf4YNdvk1oAtcYtreS$Ddy*GG#0melQcslU%l(|BY+UZ2+l09)>)hF&&Pz ze^@6Jm8~p=l&X7#XYiqS@_Eke$c<*5u4h^Dq`3L?|M5ErbF(_HF*ZXswJjbdvOwqf} zruk{_%I7vd@%`uyrIO&r#Kn2R0c5c5(N~NEtM57oj^8O3b{pcCpW%LY@uG~FIku8m zFiFpYPS-3PXGl0Ii7)-IjbHuRa!{UfxHauF&%A5M!Ql>Noz~iOxq);e{o<9&9VK!n zob4XRYw;a})0gCOWmep-aqgK+@N08973uEB7M?7*rC(xEy>mUnbN``-MbDy)kKD=qpu za-h-nOnWFSSET>Ll2S4Hd9QD@U2fI;_yWad7u2N?4oB5?z2XOZ2P;JvV-JouzHm<^ zEf@;>v1wHv43+5^I%w-yFzNQQ4Lvvq$)=;>g4Q;|Hx@$$y`_g6*%ar|i6)F(OAF{E zf;DpATjqX#&QVpnx#w)cSbdQES=vF!@hhQY+fazb^fiy%(KE4+2hwhwH6eU*-ZyqR zPkLXwhQmI2Zg9QhlBS4MFj8a$S+vxDET>b)t))@E5qZ)hfs|^z0>aP>9mESFCx;&Z z3uW8SU_n2v#>*^<;Yl5MzLO*licZnIDo`(&`&lUCWdDq1mZ@|lLJa96-Wt+d^9MU-OMlA8j4`1=&k<;frkZ znh#Gn@(6}jXf&&C_iUDZNL*eHvR{f1ZP6wt{wvYIJc3*<^(TugD_L=RKy%8#*8sCxQ@D4YBS^kqG~{;$ss+{Eyc`$6FCZGF02qw{JG zVv|Qk4RwF`4^K8^ZpVoXOftjjWE7bF*`-Nn!I2`g-ts}G#i2$m(Q>Baq;%l(S&fCe z9k2C<-ZQ(-q&EuO;0$67tqbL?WaS}Ws-Z8W9J zVdlU#(yPt_p)fo4C6BzUA?-N4dflRG_xpE8osf``dyOe7;X@7+B8AZ(82djV!mEy^ z2IFqdg)LiIA6D>O2$n&bdmW@8l=>g0jy*e%7F13!cg*rWT6#ys-R$=wLc=0bs$aph z9cOzhfvRD4a_#3%5{Z_cTgN3XSojtPo8GamQ zS}DqZ{PKfcbk-p5QeFW={!x#WyQzzOdX-eX2)~3b;XKFtnmIkG^dBSS`VR5hPjEbx zTmk2Ooxu^b;Bw8syk@D9+bU+l-YNRcMnht$#}8Mh`9q}7NRC=0cj^A~IYqfL%EJUo zhZeE{B_|P^`m7|!m=cQJ*V@uS3=w&jXHqMdd5I>+oP=zYDuMo~1zfn0Y zf=2U-3B3-#;in(02jy3&EhQzILt2}7z;RY!D0)9Ha?ZimwUZ;QN{OPghm9BO*FEIH z4iOYYLj%i_Ng7kLo2zrYRPlNGm9LL z>l=LyJd?Qj5u$ii4zgmfTdqpguL!Yd zxp>vw=l_9AAhRQa2~*)8pY8Jmzr5Ph9~TQ9hlCNmZU^~f>iiGK{d2QSb0mEUW9M;p@q7MFSEP4 zl;P*g{$op_4&W8_w6u5d^H$JOQ^^abb;8l;68WGhSpLv)H}hoW#Ng*gprsuzuM(U# zQ=I=9^d1f#s=%dAbc{dzd=*sxPA21`A$WojKh&M!UhM=>t4t z?nY3YT>IV7I%wPmb)7~`P&JI-Z3{jAcIDu3qhl$EDQUIGx_Zmqfk?F+urkM9P16A& z6--#t87=I2(6nnP<56G`Fq5FRsqZbM!Qbs8xS=}uv2M_5>u~O%AN}%JZ)REoYoPF_ z6mWqr0=^LM`=T|HU@0V1%X??>eK3B?f#lp89#?idyzzptOQ<2K#W;}@87^zLAyz5&^W~|jn$2n z^yKS{fD-2Q^&e1XJ)Xxp>A(q$`79x|*85HjTYbCVD7*v;d5;D~0=Z?hc7OhQq;#xC zk13H0K;E{5$8ziy^xDZx2{yi{a?%pRP_%3DSQm7J2hbb36+kC053D}+f@+@wP_p?I zgRfQfnk4xxL%-|lRgRY0HLfKKQrj7|q%z^%Ne%PA0;sK^H3h)^c|$iiiYiFWd$|HP z4^?qm&t)n)r$}ATB`q521Sdh+9%-051p$H*IOy^w{nY2lRZj9kWywG zJat&xr1U=iQf~DUNG1Yq1t>VHg1cM8BypX(MC_a>jUO7_$ceq^>D>-#C7_uc0`X(KQ8m#R(Jr*W&UY6`OJ(58 zIOsIyVS3@}odTXw=g)cNzUqYWpa29=%0F zS?d4>-wI!N_bo&%ZMfE!0-fW2;)V0UK`rVz8sEVta<1HOhg^%J_ybf(XN|8Lbz2cK zAFZwsf+g9W4&vu$A%t33&DnK8snv#*)`+2B-!zncHib`jYS{jCL^P`(hIN&C37r&? zB-(%H3_6ZD1M2w8b??5&f(A>zb{aMY{oM>apPz)Q;*|}Cz-F@q9)n`{JgZvHYhB>@ zIkxl7Xh4NT9TFkI61VlqW>k4=lLN&qSRt051Mr5 z@BXV+yTmM-^4YgrRvjDQ_^0Ei-^Y@kFpf0^gq9B@ zS^4WkZ0|S_0;cU0Wd!e-ODK^GF8)ZfR$I<^}{5xpM{g1 zy{Mz=-j~vp9`nAu63FXR<(rh%k~Xe-UJJ4>p`Pb3x&7qb7bAuwSj9HyiHcZMz2eA& zTg=xQ;_L|UN!%r{4eJ9a-%uTg;-NxUW8biG6sB>RQ z^K^F?eT(Z%ck~%wq*#i|gU}Fq|=mDLHfJz2GH0C;8UhxX7#oH;; z>MgFYE!$mM)ibM5Wbga+M7APH#LH8SUf1oNP+__wxTG5Q={o6hBS^EQ)wN3LHiBOU z=z(;j3{vyPX&nKBLr{pGw0S3GTejKjV5uxo8uqKf7!5T)eP903Y?xu&z*DK5E#-Bd zTMkZ;du_fZitrkEw;~9pa@IL?y2GS5G+uGFE5~olD?JlC&*L;~8<3r{*IE4Z{?3Z@ z(wAqe`qg;PA8-s}veOAkk2a{`U?rc9>qn+fIOif^(-Ex;zk zT;?%z$Y)Ery~j2ZRo5iCgx21>4z{xY$`a}(Y7MS;&0H8H&)c-aHAw50>o@nR+No(s zYbZmTBpxw*V%QAhpJyf|Q@dK` z#OeZUz(j*^o)O(6*@SFeT`il}oF>8AH+d&ti7oc&dpTei>Ute*S;-R>Xx-=(cBVv2 zUz<0!ljs$)CC{v>7I)Q4uQC@Y&r}v?ygk}r zP5rSm2AiG*;bEuiyOrpUHvM<$h2}YBBaZO`^@<^Ah<#!KsPmg%m7P?CPrrYyBtTjU ziM{V1xJJ-Kfbm`2Y4_8k;&n3L&_|4csQDoM&meA6h6d&i!k&=;;^o(W;$``vx$C>A zZt@6RyL^@HFD}4;4XdQj`E-_LG?mYk?{N2AgKQOdjyE2d_HgK4pL!ORNxF@pdAqe(s*L1pDrW_r0){@QmJRD z1xwqaZSVI99g-X6<1OVaU3pRgjrlux^by1rcdMVWcS7|J0lp7-r$=b){GLfvLz$5k zzq=^>9}l3PMw`s`7=>U|U8o1f#RnT*l!QxD_B;2SOe7rwA&6d%g2EJoy(RkGc^e{h ze_cC%XQa?D!bDMV6zpBl2Vz|iS|6nQhzvW`-=31bOqA}02ute1k#%e(MIsrp?-j-xS&HI0FZ6rv`}zGo&->5&*ZcNgpU>r*>-sL& z_c)I8IM0&-1$2Ne$Oun1YeIVmCar(-w@ONNpGH1@YgPXNOv7lxi<%_)Y&*r;H>oD! zW_bB*2wXc2zwkeSH=8VNf3(?HJ@pVRG8laS0le=>RxBkLe(`P3K4$>or(j@Y`2L-K zgKm3lkSCZ{{@Yy@m3Ts5K`B&4UNkC&5UHQ!;{eY%?Lmdkzbq9ITc>rR|8EV1A|LC=*KZ(fnFFv= za^ctC8QSTuJJUWw+Wln!X!(Fz%Xt%;ZI2YD@+sfy^jqp1ed|Y2!*)T_T6+jOZtV`p zzQcizG@2AXS3pX3Z$BpmR)3@LI{mjGsF);cGaw@6ZRDfa5F_d+SG>k@X@c#BVn3aF zjF|Orh0X{1gtXsKVEw}T8?e==KzrHfgCH-W>(zFiq2ht>Z zZEeBp-~a(}1F&>XbiK&9!;S0yBm>NB3MyDP9g9-fHv<7$Zi*E#P=um6A zk1TRF^8ymiUa1~W^60*@_=dy$xUodp7ZwiF*v?PsF0kQB2*2f5e|+|+g z)n~d4Ys>!|T*}2ob{*YSek=AsO+Tc7Rk|!cV40fX0Zb%;ExSi>8mU;gs{phP=*q;cUVdkh4+&pj@eoHFPU;?QenAXjL~hTzfVl4-h*L;@VjEM+paxBi^tGx0oaKZRAYBWl`4lYO1X}lE-`u( zki~V!7&Qx9qZmkW*QH1tI-20v`b6{e@7k#8TznYV7-WOI`@KeGxX&Los`js3DN$Jg zH@`i6q6-ktPWG6x1$G(Q8DQqCyq%-r(jR7&k6wv9vD%r>P{|^hT5I}Q)V!*~q4W8x z3aMirl)f@|AinWRG%Yd0DQE6DA0C*S*{POlIrq7!JgYE+4=oI+lG!cVyI7mPDGSA?s#w@WyW0lkvtU!g=i~ zQgXE9XM-1J1_plb7g1*)3P41d2GKmXwT|m5R}D!7GM4gqfi;U@o^Nq8B5gwUYm8oR z%{!nwT)V~GdVOXpr{$d6uD|Z{9N=9G|hGDpE_y+Jq0Wjgv0Zk zg+=mQceuggXoNJT>x9t{Ku|3~iPC#pDl4$9!S{XjJ|^l5F zoJP|sRP~}I-DUQ3OaACl&>`N7mrowHzW*!Z^(ep7fjdVes)#D&VwBC! zK=if>4L_fAyM{oq|9%BN7+3wh{0({v;@v#28pEpyGAkaniYkddVK8F6>IX>dgz|Yn zVVF?2^au%~=e(k}`*x@kT^X$KIz=I-H4D#QGrAJz_c>NbYLI+dmcF(26_uE-B)H0P z9d*+pck6qXsW|^&7wN}@xryyj8ONSIm^D?QLhdM8tOoPez*LLD4*yaD;<(X>?nn>K zL!)XAP`3(G+@A#2R*6r=8(t1t>{Ml;u20qL1Mr@4k%dOuu+p<_hkQTBYo!nZz7SP= ztNcy7t9U<<%3UX})6gBhNs}h5C!fsu*|O^E!0yobU*L5HWOAL53VIi-a<_3VZlqf} zcT0_%QPN_vU1X_llKY`9I1WtDlXde2epUGOn;lgsF?_COuU4zxCX!?3(=G6p{n!GA znk?t(;X{d?7N^bp-5gG8C^k|dPlmNc+K2O-6&V_@OvF;ebaD+JsT~@pSOn8O?`AYF z{rJj@B?r>Ox(2^|?gV&3u z=S?{LuIC+(=y$O-1x3xrDXGa7H5;%s3={b7+dr~ckxM*!(f;79+$RQYZn%6Aqj z%SgD34{}F4W`R8ChDCy}a)6}L)O_(48=`XD;cfh#d8NbQ@_6rx?`l<5)8vVQGoEp~ z-FP3V1e`RLf5^smuKOkp-7w!voG`a-TUZ>;cQG@UwXBdo>l8uamIq#L*};ke5mf-y z8?F0pAJuUuG9IV?j0Zw1HX{XA47>@t?Z>WM*#lRAI=wk!|G2`kz=q<1tyC{UAj`@q zDf*0yqjZbc;dkWWe7SSQx0Vv$eHYV#;l_sZPi;^ex>b$9?QBoryLnHY?(okPlIw-# z#i$+-7le%==<4xLvalDPgMiIaGP+?X{X>CriFqK!67mS+BO zq(Xx?yB+M;zLPJI6MnEE*=rXty!1BKRLb}(bfz~jy)n(fqYrk z@oD^mK37bZVd{vL=Z-V^_JZl}eKiEFRlE;VTcy{d2qXboZfy!BXV(r?kib30B^bVd zenjm}l=I9@c4-ivF3RzCe`~I|~W~@oryE03KOjp@|9&<8aG}&_`l;q>cSJ$a`U2hL|Uv6bxaxTSYpLPZis=V3xMASE}KMSBg5!Tpi7jQqrtb-lFf}$Z>!p zaa!ycv_Z)4dSP9@3R3pfvW3UexzN0%?T%XBw%9&6DLwKm?pS&<30{Ak0)h3 z(%Z|jKFa)Yin8)d*f`!wpu}a&4~{#hxr1l}jeJIG&9g0V@Wh-GfM5z3UH3tXuZBz< zK|&`jSVMAmC4Urq%Uym7ocY(FQ-T#^W4P^rM~}l9lAXlL;Hw+1F;@nvDm5Z@vs357 zY8SyOYwER-Rlj2FSjnEB3O2{`!e0aaD@~ttEc;gC3Er0nAZ`^u6>w@}0Jkdga&d?5 zq}i#dM1%Ut4+>RYywMwP+1_Xl7Z(NXeKUnDeoEan zU^onmNtw>;LDCtcGREBJAdv z@aF1Djhha&SyRrq|G|C0ow9wuMWAkzJ0}^_S<>qgA=hGh-IPi-d5RCL4*I#I+7Ld^ z>@d+Sv9IH6Kycv(p+!EUfXjCYJZy8qC*BaJd{vk_X87zU7^HhszC%YHV<0X^+Fsl| zGI`qXtNGB!JG~kW;coH!gY_#d{Df>ax)ei^7~uxQdpZ~&>L62hAWTw zq7M1evoX;!&WZ3oKbj~CT90?BkyeP|s(_C=FC}`4b~$8y2n!Dc613#v4)fBJb_J7kpCDKc>_|sdgYFga> zD&BLQtw+}P5J{MpJ9OhYX$9aceRx-+pbLgLWdk z9#~UtROk$7 z35rama5OtK#0X!F)Pu6AlR)~5pfcrUJVoFk95WXoUh4E&7)TXDQ9XK{>$;%4gd<>L zmsAd6-C8rh1^x2ja8Z-rW|iJu@EJF?EQFY|6p-5e^7E!}rk!3Xu3DLv13I*XX@?eX z@lA=+%|$weHadxiE?$S$R|l&_ij*1IQ@%W&^+LrP zp4uLAeTy$Qab}$_dIoOx4~#Njm=d9c1i6j48y>>=tU=4xgS_dM7P)0Y_4`7`)QT z9%9Po)Ws_T_Hcl-?qVo?W{ni#$N{L_g`q*tmxPZ;%f0BpzW9b(Dr+l zD0M?aHXopzT?ZNZqQxKOg#O&l<2%@Jjp&?S2bC~iFTUKe`@y0+kK4zS;(fotqfiXB z6-M`*gFl&qpzu3Bis~IsE1dWIa}eTGvXKxvM^0Q0xy=*GsU{KP{H&9lPZ@|*ye)M8 ze2POj(oz)(wi8B=396+8mbo==Q6K+F7r}e)9bGBbs$gknz$A?25!F9XTTSpGfvihm zUuZ9~?>ZcJxC#P;*>0YAN+g79mf~|EkEI75YI-rzr$eIp*D$3Kx~vT^hL-?h5TwmM zaVq>;Rui1$`0NZY$W4RK&Q)%mnh7fFy+(i{JH-pguR{ab>EP+BBG zYwkxxfU22*05rf7XAjd%Px3ZP?y^hTY&0QET9pJ?uj%gsHy_?l6! zF}*lsTCKxu0n(W|t5+8!vW)Vqp53l39) z%lFe6pTP~tm=GC+pOmO9o zCtEN$g{y5KaUhre3D2>t$HkT#yzC64VPrpzF*G-)NybsPrF82fUI z{8OPYQ`xkaZvL zT*rhIz=Xxno`)^?U19tXjP0yu;C~0kA{@{DdB9CJoDC%giUZdCA;QKdOm|Jo+qYBC zfMj<(d;HDl*^Mz$bOB$SmUw`7h$5MVH!3AJ#ma=K{tQ^~Zf`z3{{ygtyi&Zvh+=Z0 z1I5exnDH2`OK>Zo9GGa3bN;2;1NtBumQX>;>o;Z6D${DTZXO8_ zPLq9n@2ZgM>h7Fhzm_H_6qe3q7kJRECj*yA9CZ5K2MMVC`#E?_RlI$BGePx3USiV@ zrzF56PGOfxg$O*=kWoWGQkTbK&kUnO18N?~mrb!!%H*97yn%Z)^M?GovCK z4W3DYJ_zAw7fyS3zwwP|&V%()sss8sGGKq~c?SY*b|k$=IUM>frm)0E`TY452u16Q za?^cx7qG08_|C#tjU_*NL;iggG95=%l*s;XS8Tip0~TQCaa~Ttyh#&u8vF8yQ8E%rH?;Gr}pX&){^=@R_$D zZpxzS&8Xpjn3tEJT5kPd6(sfGCpvZI?_>4H!>LNs!gf4vYxf2C83t7LGR{CtS%5E^o zS$p<-Wo+X({DOvN7mu3C$+M6(z;9Q-BO5nYf1n+*X^}RIQwYq})50BlL_px99N;6D zLOB3#psJYN#{si0gvc8bcSlT(3`MW(?4fOHF_iM3=*)i%@hP(6+pMZucPQYx2w=1a z2}h)>#}*)w5b@GMlqggeOS#ko{bDPji*tK*KtRV^73u z?dUnE$HUr!4=S8(4p?mkW5ri{*NzFBv5${rmT!NtE^@79Rz-|#J8IX21t>+<7rza@ z+B(d{3ECj?^867r0RvI^6n_BIPN<)nC+^!I&adRoEGQ#l@ErheK9tq%0l9Yy5`jw~ zk+5uy#p|m81w0=e2Wg)LRD%w0P-Lrb2R@P0f>4o(r))7%({arJN!};L(>uyoyao9Sv76UB@!uTq2_o+&I*XNs_g?6=S z>pSDl9YOPg_d;(}u$A=}J0wOV_{7nX)51Y`^Ea^6X_<%0w4IQCovfH`kdt8Q@=ic- zHggW-JoiuR#0CHR=m${!DaRKOX>!B-Vf zylFppzbGIM8v}dE1|csuM{c_GHUHw#t3?Nba-=w=ui67hEI>5Nc8IL)zixON_?B}y z*jJ~t0+#jAcKN=b3ij>ck9mV%4E+qi&w^d#>~m-oTk7uurcDuvw@9!Hte523EUtVB z<6`@}=HV48%K*hqtLJ7Q1U30KyYZH2UIk^VCA-5K#9L2C2wU(62(2i^Gt7k@-ilZl zJ0ZB)wt#q7?ZDB*E_S{xrs&a-#jS8s>%C-n088^Y zDvXKB65A~Q{dS*gGtT4V$?frJh3f40c@-v~zb;TASFJ$cZxWnk9)4P>3Vkq3qq?L6 zVe>$Yx}jaRi7^?98x3LZV^e(#B#5g}wX_@X{DpPbkz2Tjx|4|2FOYEyX}4@UKD!Fg zuB^G};loq`5s0jNV70ND3195>*^Fyepg)JFlG(7y0vrA>P0u*()t-UX%ZM>aLnCev zS`_af@K!)pz$Eti3H#1NabXXafpR5Jyf-fV_GBEar>E1*r*4|FgsGdyeT(J2FH|v- zV3$D@U9l zDr`(kv2&)x!c&<+2hZb3pl|ei@JHWIn6Qzsk#4Alm;5|V;>}4VGglQ1H}ON$-p#Xz z4vr#z#S9HNSqzccur0%0ywz_qxHJqjTgeh22&iC3zeUxFY_Lx?Ix zV7apL3rk+OloWgzFB5YKqX@xNdA_JBYWuYQIr=X*98eF$kKWqvE`|yXD=9hublFV3 z^z8z;E6Mn65`Jbn=B6z75P!N9QC#wwP#)G+-~1^xg@AOKH4Qe7tNpdtI%x>K-7>Fs z7F-W;P1Y=_i0#1CXR1iX_vxERXq;W+ho$vtl&mWaEzIpO^J&660^Z-RLmD+O+hyqr z#M4xcGqO|u+siF7#i;tVfq*r?yEtk;DrDT2ZxqyRmzHVZa-~wAQqZ!NiFHHSFFv?j zH`VdYa}*o4Y4!xM(?rHE1Ha|dr<6G)n0v=CLaCpf`d#~)x+f3R(|jR3AC=jj6#Awa z!isb&--cc%^Hl!iC+12Dy*}ys7_5=GJwSlEZ~e7zlA?sSqD~;HcYiUJ}0}9Q+2ZLex znZ_mLY7ia6&f&3_aL1=1Fzn%XSw>=yPQnV<@iJ55C%75 z%(-=)8quI;fd*|r8O31+esz-QaCVzhm>~YKdv`fU(1)o)*DxHuM1V2&$uC%5#!zvz zmqs8wmRn3Rv%ZWjv1GR?gf+)N`}}z0bIns>mt31CC7}=5?C_ee@#M66mRR+r`#)O! zZ9sUU2vo5N>$e%kp;`EcmGR`adpN}m!(Rm3R$xri^lrnPC$oDQ46tOYUoqrAcL}OH zbX>oM)`xQ)?LLfWr8Y2X@BM8au!a3jNBU1}N;l_24dP`ngG`f5o7jO^|C!mlmLI`+ zIW+pjI{K% zXLxlmra1NeYWpW>ay}}5ktd2XXZiK_i#AerzBVb|1GgBbavQQ||SEls)PaS`lGs|ZSQ zk=FU`(2PO-g8FR@6K>~2hxBFkL_affIc3PIoQ{mZA`-Jvu@WWoe^t}Bp9}t*z@ZO*s}lTV6DWOer}0{?}D4xjGO0-NgyMa zY-wXiuZs6oXuHeksh3byaIlcWbd+|CSC$Kvv6VHDec%C(MwDQxHzxZF>h|XMO{7Y}V z95WPt&3c?&VPJkqJLJ8Zy(?buo~{qYneAT~eVxMw@v><1ujbcryE0$G-(^oXeCBbL zI(%}!ImZ4zp~J2|=Gljv8(CEw)3iZ7g|t!X8iw;md48?j*V1i5Qtq2QOy2&`S;gD_ z3?_ Date: Wed, 16 Oct 2019 11:37:58 +0200 Subject: [PATCH 08/57] Fix menus and width for lifecycle images --- website/src/_includes/head.html | 3 +++ website/src/documentation/programming-guide.md | 9 +-------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/website/src/_includes/head.html b/website/src/_includes/head.html index 2fd0083dabe6b..aba87c49c0e42 100644 --- a/website/src/_includes/head.html +++ b/website/src/_includes/head.html @@ -19,6 +19,9 @@ + diff --git a/website/src/documentation/programming-guide.md b/website/src/documentation/programming-guide.md index 8229c35e4b85d..6ac9b22825b9d 100644 --- a/website/src/documentation/programming-guide.md +++ b/website/src/documentation/programming-guide.md @@ -3102,11 +3102,4 @@ public class MyMetricsDoFn extends DoFn { context.output(context.element()); } } -``` -## 10. Focus on DoFn {#dofn} -![This is a sequence diagram that shows the lifecycle of the DoFn]( - {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) - -## 11. Focus on Source {#source} - ![This is a sequence diagram that shows the lifecycle of the Source]( - {{ "/images/source-sequence-diagram.png" | prepend: site.baseurl }}) \ No newline at end of file +``` \ No newline at end of file From fccd4e2bbb9c7c7e0c0ef8e48ccb26f90d83314a Mon Sep 17 00:00:00 2001 From: Valentyn Tymofieiev Date: Thu, 17 Oct 2019 08:48:36 -0700 Subject: [PATCH 09/57] Add ValidatesRunner test for Impulse transform --- sdks/python/apache_beam/transforms/ptransform_test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index cf640b81fe9f2..ad201d1a77210 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -194,6 +194,13 @@ def test_do_with_multiple_outputs_maintains_unique_name(self): assert_that(r2.m, equal_to([3, 4, 5]), label='r2') pipeline.run() + @attr('ValidatesRunner') + def test_impulse(self): + pipeline = TestPipeline() + result = pipeline | beam.Impulse() | beam.Map(lambda _: 0) + assert_that(result, equal_to([0])) + pipeline.run() + # TODO(BEAM-3544): Disable this test in streaming temporarily. # Remove sickbay-streaming tag after it's resolved. @attr('ValidatesRunner', 'sickbay-streaming') From 250ab6f3befb1f3a09147365475c06649e542b5f Mon Sep 17 00:00:00 2001 From: Valentyn Tymofieiev Date: Thu, 17 Oct 2019 09:20:55 -0700 Subject: [PATCH 10/57] Use base64 string for representing impulse payload in legacy DF runner codepath. --- .../apache_beam/runners/dataflow/dataflow_runner.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 9b2cf684bc757..dbc234a056f3c 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -23,6 +23,7 @@ from __future__ import absolute_import from __future__ import division +import base64 import json import logging import sys @@ -629,8 +630,16 @@ def run_Impulse(self, transform_node, options): coders.BytesCoder(), coders.coders.GlobalWindowCoder()).get_impl().encode_nested( window.GlobalWindows.windowed_value(b'')) + + from apache_beam.runners.dataflow.internal import apiclient + if apiclient._use_fnapi(options): + encoded_impulse_as_str = self.byte_array_to_json_string( + encoded_impulse_element) + else: + encoded_impulse_as_str = base64.b64encode( + encoded_impulse_element).decode('ascii') step.add_property(PropertyNames.IMPULSE_ELEMENT, - self.byte_array_to_json_string(encoded_impulse_element)) + encoded_impulse_as_str) step.encoding = self._get_encoded_output_coder(transform_node) step.add_property( From 63cabe7849fc3b046e5fcfe8ab66b9bc58534dd5 Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Thu, 17 Oct 2019 18:13:13 -0700 Subject: [PATCH 11/57] Must use Boolean arg to make use of the BooleanCoder in Java --- .../org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 6 ++++-- .../beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java | 12 ++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 5fddeb1459208..3212b3b0bbf41 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -813,8 +813,10 @@ public void setTimestampAttribute(@Nullable String timestampAttribute) { this.timestampAttribute = timestampAttribute; } - public void setWithAttributes(Long needsAttributes) { - this.needsAttributes = needsAttributes >= 1; + public void setWithAttributes(Boolean needsAttributes) { + // we must use Boolean instead of boolean because the external payload system + // inspects the native type of each coder urn, and BooleanCoder wants Boolean. + this.needsAttributes = needsAttributes; } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java index ea43ec0522efb..50f75289122c4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java @@ -30,8 +30,8 @@ import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.core.construction.expansion.ExpansionService; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -52,7 +52,7 @@ public class PubsubIOExternalTest { public void testConstructPubsubRead() throws Exception { String topic = "projects/project-1234/topics/topic_name"; String idAttribute = "id_foo"; - Long needsAttributes = 1L; + Boolean needsAttributes = true; ExternalTransforms.ExternalConfigurationPayload payload = ExternalTransforms.ExternalConfigurationPayload.newBuilder() @@ -71,8 +71,8 @@ public void testConstructPubsubRead() throws Exception { .putConfiguration( "with_attributes", ExternalTransforms.ConfigValue.newBuilder() - .addCoderUrn("beam:coder:string_utf8:v1") - .setPayload(ByteString.copyFrom(encodeVarLong(needsAttributes))) + .addCoderUrn("beam:coder:bool:v1") + .setPayload(ByteString.copyFrom(encodeBoolean(needsAttributes))) .build()) .build(); @@ -210,9 +210,9 @@ private static byte[] encodeString(String str) throws IOException { return baos.toByteArray(); } - private static byte[] encodeVarLong(Long value) throws IOException { + private static byte[] encodeBoolean(Boolean value) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - VarLongCoder.of().encode(value, baos); + BooleanCoder.of().encode(value, baos); return baos.toByteArray(); } From b1e3fd1da059cfeb42cb8a08468f8841e6ec5b32 Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Sun, 20 Oct 2019 13:25:00 -0700 Subject: [PATCH 12/57] Fix docs --- sdks/python/apache_beam/io/external/gcp/pubsub.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/external/gcp/pubsub.py b/sdks/python/apache_beam/io/external/gcp/pubsub.py index 5d0683b243b8c..7d66819fdb89d 100644 --- a/sdks/python/apache_beam/io/external/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/external/gcp/pubsub.py @@ -64,7 +64,8 @@ def __init__(self, topic=None, subscription=None, id_label=None, that no duplicate data will be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be strictly best effort. with_attributes: - True - output elements will be :class:`~PubsubMessage` objects. + True - output elements will be + :class:`~apache_beam.io.gcp.pubsub.PubsubMessage` objects. False - output elements will be of type ``bytes`` (message data only). timestamp_attribute: Message value to use as element timestamp. If None, @@ -125,7 +126,8 @@ def __init__(self, topic, with_attributes=False, id_label=None, Args: topic: Cloud Pub/Sub topic in the form "/topics//". with_attributes: - True - input elements will be :class:`~PubsubMessage` objects. + True - input elements will be + :class:`~apache_beam.io.gcp.pubsub.PubsubMessage` objects. False - input elements will be of type ``bytes`` (message data only). id_label: If set, will set an attribute for each Cloud Pub/Sub message From 90f5dff51b9858d9c9eba9ff4819b0013afa218f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Oct 2019 19:23:23 -0700 Subject: [PATCH 13/57] [BEAM-8456] Add pipeline option to have Data Catalog truncate sub-millisecond precision. --- .../datacatalog/DataCatalogPipelineOptions.java | 8 ++++++++ .../datacatalog/DataCatalogTableProvider.java | 16 +++++++++++----- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java index 47fffcf6baa4a..6818691a32f67 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java @@ -32,4 +32,12 @@ public interface DataCatalogPipelineOptions extends PipelineOptions { String getDataCatalogEndpoint(); void setDataCatalogEndpoint(String dataCatalogEndpoint); + + /** Whether to truncate timestamps in tables described by Data Catalog. */ + @Description("Truncate sub-millisecond precision timestamps in tables described by Data Catalog") + @Validation.Required + @Default.Boolean(false) + Boolean getTruncateTimestamps(); + + void setTruncateTimestamps(Boolean newValue); } diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java index 321cd0e8928bc..04561113bd62c 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java @@ -45,20 +45,26 @@ /** Uses DataCatalog to get the source type and schema for a table. */ public class DataCatalogTableProvider extends FullNameTableProvider { - private Map delegateProviders; - private Map tableCache; - private DataCatalogBlockingStub dataCatalog; + private final Map delegateProviders; + private final DataCatalogBlockingStub dataCatalog; + private final boolean truncateTimestamps; + + private final Map tableCache; private DataCatalogTableProvider( - Map delegateProviders, DataCatalogBlockingStub dataCatalog) { + Map delegateProviders, + DataCatalogBlockingStub dataCatalog, + boolean truncateTimestamps) { this.tableCache = new HashMap<>(); this.delegateProviders = ImmutableMap.copyOf(delegateProviders); this.dataCatalog = dataCatalog; + this.truncateTimestamps = truncateTimestamps; } public static DataCatalogTableProvider create(DataCatalogPipelineOptions options) { - return new DataCatalogTableProvider(getSupportedProviders(), createDataCatalogClient(options)); + return new DataCatalogTableProvider( + getSupportedProviders(), createDataCatalogClient(options), options.getTruncateTimestamps()); } private static DataCatalogBlockingStub createDataCatalogClient( From b2329dc314e71339cfea8287f0c8e1453cf252a2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Oct 2019 19:33:13 -0700 Subject: [PATCH 14/57] Inline Data Catalog TableUtils to its only use --- .../datacatalog/DataCatalogTableProvider.java | 43 +++++++++++- .../meta/provider/datacatalog/TableUtils.java | 68 ------------------- 2 files changed, 42 insertions(+), 69 deletions(-) delete mode 100644 sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableUtils.java diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java index 04561113bd62c..ee79b861ab8c3 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java @@ -21,11 +21,13 @@ import com.google.cloud.datacatalog.DataCatalogGrpc; import com.google.cloud.datacatalog.DataCatalogGrpc.DataCatalogBlockingStub; +import com.google.cloud.datacatalog.Entry; import com.google.cloud.datacatalog.LookupEntryRequest; import io.grpc.ManagedChannelBuilder; import io.grpc.Status; import io.grpc.StatusRuntimeException; import io.grpc.auth.MoreCallCredentials; +import java.net.URI; import java.util.HashMap; import java.util.Map; import java.util.stream.Stream; @@ -39,12 +41,18 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.pubsub.PubsubJsonTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap; /** Uses DataCatalog to get the source type and schema for a table. */ public class DataCatalogTableProvider extends FullNameTableProvider { + private static final Map TABLE_FACTORIES = + ImmutableMap.builder() + .put("bigquery.googleapis.com", BigQueryUtils::tableBuilder) + .put("pubsub.googleapis.com", PubsubUtils::tableBuilder) + .build(); private final Map delegateProviders; private final DataCatalogBlockingStub dataCatalog; private final boolean truncateTimestamps; @@ -81,6 +89,35 @@ private static Map getSupportedProviders() { .collect(toMap(TableProvider::getTableType, p -> p)); } + static Table toBeamTable(String tableName, Entry entry) { + if (entry.getSchema().getColumnsCount() == 0) { + throw new UnsupportedOperationException( + "Entry doesn't have a schema. Please attach a schema to '" + + tableName + + "' in Data Catalog: " + + entry.toString()); + } + Schema schema = SchemaUtils.fromDataCatalog(entry.getSchema()); + + String service = URI.create(entry.getLinkedResource()).getAuthority().toLowerCase(); + + Table.Builder table = null; + if (TABLE_FACTORIES.containsKey(service)) { + table = TABLE_FACTORIES.get(service).tableBuilder(entry); + } + + if (GcsUtils.isGcs(entry)) { + table = GcsUtils.tableBuilder(entry); + } + + if (table != null) { + return table.schema(schema).name(tableName).build(); + } + + throw new UnsupportedOperationException( + "Unsupported SQL source kind: " + entry.getLinkedResource()); + } + @Override public String getTableType() { return "google.cloud.datacatalog"; @@ -132,7 +169,7 @@ public Map getTables() { private Table loadTableFromDC(String tableName) { try { - return TableUtils.toBeamTable( + return toBeamTable( tableName, dataCatalog.lookupEntry( LookupEntryRequest.newBuilder().setSqlResource(tableName).build())); @@ -148,4 +185,8 @@ private Table loadTableFromDC(String tableName) { public BeamSqlTable buildBeamSqlTable(Table table) { return delegateProviders.get(table.getType()).buildBeamSqlTable(table); } + + interface TableFactory { + Table.Builder tableBuilder(Entry entry); + } } diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableUtils.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableUtils.java deleted file mode 100644 index 6c0b62ec89397..0000000000000 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableUtils.java +++ /dev/null @@ -1,68 +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.sql.meta.provider.datacatalog; - -import com.google.cloud.datacatalog.Entry; -import java.net.URI; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap; - -/** Common utilities to create Beam SQL tables from Data Catalog schemas. */ -class TableUtils { - - interface TableFactory { - Table.Builder tableBuilder(Entry entry); - } - - private static final Map TABLE_FACTORIES = - ImmutableMap.builder() - .put("bigquery.googleapis.com", BigQueryUtils::tableBuilder) - .put("pubsub.googleapis.com", PubsubUtils::tableBuilder) - .build(); - - static Table toBeamTable(String tableName, Entry entry) { - if (entry.getSchema().getColumnsCount() == 0) { - throw new UnsupportedOperationException( - "Entry doesn't have a schema. Please attach a schema to '" - + tableName - + "' in Data Catalog: " - + entry.toString()); - } - Schema schema = SchemaUtils.fromDataCatalog(entry.getSchema()); - - String service = URI.create(entry.getLinkedResource()).getAuthority().toLowerCase(); - - Table.Builder table = null; - if (TABLE_FACTORIES.containsKey(service)) { - table = TABLE_FACTORIES.get(service).tableBuilder(entry); - } - - if (GcsUtils.isGcs(entry)) { - table = GcsUtils.tableBuilder(entry); - } - - if (table != null) { - return table.schema(schema).name(tableName).build(); - } - - throw new UnsupportedOperationException( - "Unsupported SQL source kind: " + entry.getLinkedResource()); - } -} From 25054b82292d2a9a2c539964d4a87e89e4dfdaad Mon Sep 17 00:00:00 2001 From: Alex Van Boxel Date: Sun, 28 Apr 2019 13:16:23 +0200 Subject: [PATCH 15/57] [BEAM-5967] Add handling of DynamicMessage in ProtoCoder The ProtoCoder was unable to handle DynamicMessage as it was unable to get a message specific parser. The Coder is expanded to handle DynamicMessage as a special case. It stores the complete descriptor set when (de)serializing. Design decision: Although DynamicMessage could in theory have a different schema per message in a single stream this doesn't make sense. The common use-case for using DynamicMessages is when the schema is not known at compile type, but is known at pipeline build time (example, like pulled from a schema registry). With this restriction we only need to store the schema (or descriptor) once when the pipeline is serialized and send to the workers. --- .../beam/sdk/testing/CoderProperties.java | 10 + .../sdk/extensions/protobuf/ProtoCoder.java | 90 ++++++- .../sdk/extensions/protobuf/ProtoDomain.java | 246 ++++++++++++++++++ .../protobuf/IsDynamicMessageEqual.java | 69 +++++ .../extensions/protobuf/ProtoCoderTest.java | 25 ++ 5 files changed, 435 insertions(+), 5 deletions(-) create mode 100644 sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java create mode 100644 sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/IsDynamicMessageEqual.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index f10e95b9b0a70..1c43fc6e1e644 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -104,6 +104,16 @@ public static void coderDecodeEncodeEqualInContext( assertThat(decodeEncode(coder, context, value), equalTo(value)); } + /** + * Verifies that for the given {@code Coder}, {@code Coder.Context}, and value of type {@code + * T}, encoding followed by decoding yields an equal value of type {@code T}. + */ + public static void coderDecodeEncodeInContext( + Coder coder, Coder.Context context, T value, org.hamcrest.Matcher matcher) + throws Exception { + assertThat(decodeEncode(coder, context, value), matcher); + } + /** * Verifies that for the given {@code Coder>}, and value of type {@code * Collection}, encoding followed by decoding yields an equal value of type {@code diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java index e2a919afb10df..faca3881f1fcc 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java @@ -19,11 +19,15 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; import com.google.protobuf.Parser; import java.io.IOException; import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -107,6 +111,8 @@ */ public class ProtoCoder extends CustomCoder { + public static final long serialVersionUID = -5043999806040629525L; + /** Returns a {@link ProtoCoder} for the given Protocol Buffers {@link Message}. */ public static ProtoCoder of(Class protoMessageClass) { return new ProtoCoder<>(protoMessageClass, ImmutableSet.of()); @@ -122,6 +128,36 @@ public static ProtoCoder of(TypeDescriptor protoMessag return of(protoMessageClass); } + /** + * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given + * {@link Descriptors.Descriptor}. + */ + public static ProtoCoder of(Descriptors.Descriptor protoMessageDescriptor) { + return new ProtoCoder<>( + ProtoDomain.buildFrom(protoMessageDescriptor), + protoMessageDescriptor.getFullName(), + ImmutableSet.of()); + } + + /** + * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given + * {@link Descriptors.Descriptor}. The message descriptor should be part of the provided {@link + * ProtoDomain}, this will ensure object equality within messages from the same domain. + */ + public static ProtoCoder of( + ProtoDomain domain, Descriptors.Descriptor protoMessageDescriptor) { + return new ProtoCoder<>(domain, protoMessageDescriptor.getFullName(), ImmutableSet.of()); + } + + /** + * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given + * message name in a {@link ProtoDomain}. The message descriptor should be part of the provided * + * {@link ProtoDomain}, this will ensure object equality within messages from the same domain. + */ + public static ProtoCoder of(ProtoDomain domain, String messageName) { + return new ProtoCoder<>(domain, messageName, ImmutableSet.of()); + } + /** * Returns a {@link ProtoCoder} like this one, but with the extensions from the given classes * registered. @@ -269,21 +305,65 @@ public ExtensionRegistry getExtensionRegistry() { private transient ExtensionRegistry memoizedExtensionRegistry; private transient Parser memoizedParser; + // Descriptor used by DynamicMessage. + private transient ProtoDomain domain; + private transient String messageName; + /** Private constructor. */ private ProtoCoder(Class protoMessageClass, Set> extensionHostClasses) { this.protoMessageClass = protoMessageClass; this.extensionHostClasses = extensionHostClasses; + this.domain = null; + this.messageName = null; + } + + private ProtoCoder(ProtoDomain domain, String messageName, Set> extensionHostClasses) { + @SuppressWarnings("unchecked") + Class protoMessageClass = (Class) DynamicMessage.class; + this.protoMessageClass = protoMessageClass; + this.extensionHostClasses = extensionHostClasses; + this.domain = domain; + this.messageName = messageName; + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.defaultWriteObject(); + if (DynamicMessage.class.equals(this.protoMessageClass)) { + if (this.domain == null) { + throw new RuntimeException("DynamicMessages require provider a proto domain to the coder."); + } + oos.writeObject(domain); + oos.writeObject(messageName); + } + } + + private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { + ois.defaultReadObject(); + if (DynamicMessage.class.equals(this.protoMessageClass)) { + this.domain = (ProtoDomain) ois.readObject(); + this.messageName = (String) ois.readObject(); + } } /** Get the memoized {@link Parser}, possibly initializing it lazily. */ private Parser getParser() { if (memoizedParser == null) { try { - @SuppressWarnings("unchecked") - T protoMessageInstance = (T) protoMessageClass.getMethod("getDefaultInstance").invoke(null); - @SuppressWarnings("unchecked") - Parser tParser = (Parser) protoMessageInstance.getParserForType(); - memoizedParser = tParser; + if (DynamicMessage.class.equals(protoMessageClass)) { + @SuppressWarnings("unchecked") + T protoMessageInstance = + (T) DynamicMessage.newBuilder(domain.getDescriptor(messageName)).build(); + @SuppressWarnings("unchecked") + Parser tParser = (Parser) protoMessageInstance.getParserForType(); + memoizedParser = tParser; + } else { + @SuppressWarnings("unchecked") + T protoMessageInstance = + (T) protoMessageClass.getMethod("getDefaultInstance").invoke(null); + @SuppressWarnings("unchecked") + Parser tParser = (Parser) protoMessageInstance.getParserForType(); + memoizedParser = tParser; + } } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { throw new IllegalArgumentException(e); } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java new file mode 100644 index 0000000000000..d1aca66cd2061 --- /dev/null +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 com.google.protobuf.DescriptorProtos; +import com.google.protobuf.Descriptors; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * ProtoDomain is a container class for Protobuf descriptors. By using a domain for all descriptors + * that are related to each other the FileDescriptorSet needs to be serialized only once in the + * graph. + * + *

Using a domain also grantees that all Descriptors have object equality, just like statically + * compiled Proto classes Descriptors. A lot of Java code isn't used to the new DynamicMessages an + * assume always Object equality. Because of this the domain class is immutable. + * + *

ProtoDomains aren't assumed to be used on with normal Message descriptors, only with + * DynamicMessage descriptors. + */ +public final class ProtoDomain implements Serializable { + public static final long serialVersionUID = 1L; + private transient DescriptorProtos.FileDescriptorSet fileDescriptorSet; + private transient int hashCode; + + private transient Map fileDescriptorMap; + private transient Map descriptorMap; + + private transient Map fileOptionMap; + private transient Map messageOptionMap; + private transient Map fieldOptionMap; + + ProtoDomain() { + this(DescriptorProtos.FileDescriptorSet.newBuilder().build()); + } + + private ProtoDomain(DescriptorProtos.FileDescriptorSet fileDescriptorSet) { + this.fileDescriptorSet = fileDescriptorSet; + hashCode = java.util.Arrays.hashCode(this.fileDescriptorSet.toByteArray()); + crosswire(); + } + + private static Map extractProtoMap( + DescriptorProtos.FileDescriptorSet fileDescriptorSet) { + HashMap map = new HashMap<>(); + fileDescriptorSet.getFileList().forEach(fdp -> map.put(fdp.getName(), fdp)); + return map; + } + + private static Descriptors.FileDescriptor convertToFileDescriptorMap( + String name, + Map inMap, + Map outMap) { + if (outMap.containsKey(name)) { + return outMap.get(name); + } + DescriptorProtos.FileDescriptorProto fileDescriptorProto = inMap.get(name); + if (fileDescriptorProto == null) { + if ("google/protobuf/descriptor.proto".equals(name)) { + outMap.put( + "google/protobuf/descriptor.proto", + DescriptorProtos.FieldOptions.getDescriptor().getFile()); + return DescriptorProtos.FieldOptions.getDescriptor().getFile(); + } + return null; + } else { + List dependencies = new ArrayList<>(); + if (fileDescriptorProto.getDependencyCount() > 0) { + fileDescriptorProto + .getDependencyList() + .forEach( + dependencyName -> { + Descriptors.FileDescriptor fileDescriptor = + convertToFileDescriptorMap(dependencyName, inMap, outMap); + if (fileDescriptor != null) { + dependencies.add(fileDescriptor); + } + }); + } + try { + Descriptors.FileDescriptor fileDescriptor = + Descriptors.FileDescriptor.buildFrom( + fileDescriptorProto, dependencies.toArray(new Descriptors.FileDescriptor[0])); + outMap.put(name, fileDescriptor); + return fileDescriptor; + } catch (Descriptors.DescriptorValidationException e) { + throw new RuntimeException(e); + } + } + } + + private static void visitFileDescriptorTree(Map map, Descriptors.FileDescriptor fileDescriptor) { + if (!map.containsKey(fileDescriptor.getName())) { + map.put(fileDescriptor.getName(), fileDescriptor); + List dependencies = fileDescriptor.getDependencies(); + dependencies.forEach(fd -> visitFileDescriptorTree(map, fd)); + } + } + + public static ProtoDomain buildFrom(Descriptors.Descriptor descriptor) { + return buildFrom(descriptor.getFile()); + } + + public static ProtoDomain buildFrom(DescriptorProtos.FileDescriptorSet fileDescriptorSet) { + return new ProtoDomain(fileDescriptorSet); + } + + public static ProtoDomain buildFrom(Descriptors.FileDescriptor fileDescriptor) { + HashMap fileDescriptorMap = new HashMap<>(); + visitFileDescriptorTree(fileDescriptorMap, fileDescriptor); + DescriptorProtos.FileDescriptorSet.Builder builder = + DescriptorProtos.FileDescriptorSet.newBuilder(); + fileDescriptorMap.values().forEach(fd -> builder.addFile(fd.toProto())); + return new ProtoDomain(builder.build()); + } + + public static ProtoDomain buildFrom(InputStream inputStream) throws IOException { + return buildFrom(DescriptorProtos.FileDescriptorSet.parseFrom(inputStream)); + } + + private void crosswire() { + HashMap map = new HashMap<>(); + fileDescriptorSet.getFileList().forEach(fdp -> map.put(fdp.getName(), fdp)); + + Map outMap = new HashMap<>(); + map.forEach((fileName, proto) -> convertToFileDescriptorMap(fileName, map, outMap)); + fileDescriptorMap = outMap; + + indexOptionsByNumber(fileDescriptorMap.values()); + indexDescriptorByName(); + } + + private void indexDescriptorByName() { + descriptorMap = new HashMap<>(); + fileDescriptorMap + .values() + .forEach( + fileDescriptor -> { + fileDescriptor + .getMessageTypes() + .forEach( + descriptor -> { + descriptorMap.put(descriptor.getFullName(), descriptor); + }); + }); + } + + private void indexOptionsByNumber(Collection fileDescriptors) { + fieldOptionMap = new HashMap<>(); + fileOptionMap = new HashMap<>(); + messageOptionMap = new HashMap<>(); + fileDescriptors.forEach( + (fileDescriptor) -> { + fileDescriptor + .getExtensions() + .forEach( + extension -> { + switch (extension.toProto().getExtendee()) { + case ".google.protobuf.FileOptions": + fileOptionMap.put(extension.getNumber(), extension); + break; + case ".google.protobuf.MessageOptions": + messageOptionMap.put(extension.getNumber(), extension); + break; + case ".google.protobuf.FieldOptions": + fieldOptionMap.put(extension.getNumber(), extension); + break; + default: + break; + } + }); + }); + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + byte[] buffer = fileDescriptorSet.toByteArray(); + oos.writeInt(buffer.length); + oos.write(buffer); + } + + private void readObject(ObjectInputStream ois) throws IOException { + byte[] buffer = new byte[ois.readInt()]; + ois.readFully(buffer); + fileDescriptorSet = DescriptorProtos.FileDescriptorSet.parseFrom(buffer); + hashCode = java.util.Arrays.hashCode(buffer); + crosswire(); + } + + public Descriptors.FileDescriptor getFileDescriptor(String name) { + return fileDescriptorMap.get(name); + } + + public Descriptors.Descriptor getDescriptor(String fullName) { + return descriptorMap.get(fullName); + } + + public Descriptors.FieldDescriptor getFieldOptionById(int id) { + return fieldOptionMap.get(id); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ProtoDomain that = (ProtoDomain) o; + return hashCode == that.hashCode; + } + + @Override + public int hashCode() { + return Objects.hash(hashCode); + } + + public boolean contains(Descriptors.Descriptor descriptor) { + return getDescriptor(descriptor.getFullName()) != null; + } +} diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/IsDynamicMessageEqual.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/IsDynamicMessageEqual.java new file mode 100644 index 0000000000000..7b079631576f0 --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/IsDynamicMessageEqual.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.protobuf; + +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.Message; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; + +/** + * Is the DynamicMessage equal to another message. This special matcher exist because the + * DynamicMessage is protobuf does a object equality in it's equals operator. + * + *

Follow https://github.com/protocolbuffers/protobuf/issues/6100 for tracking the issue. If this + * is resolved we can remove this code. + */ +public class IsDynamicMessageEqual extends BaseMatcher { + private final DynamicMessage expectedValue; + + public IsDynamicMessageEqual(DynamicMessage equalArg) { + expectedValue = equalArg; + } + + public static IsDynamicMessageEqual equalTo(DynamicMessage operand) { + return new IsDynamicMessageEqual(operand); + } + + @Override + public boolean matches(Object actualValue) { + + if (actualValue == null) { + return expectedValue == null; + } + + if (!(actualValue instanceof Message)) { + return false; + } + final Message actualMessage = (Message) actualValue; + + if (!actualMessage.toByteString().equals(expectedValue.toByteString())) { + return false; + } + + return actualMessage + .getDescriptorForType() + .getFullName() + .equals(expectedValue.getDescriptorForType().getFullName()); + } + + @Override + public void describeTo(Description description) { + description.appendValue(expectedValue); + } +} diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java index 04ed9a67dd1a8..25c62517e5dca 100644 --- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java @@ -17,9 +17,12 @@ */ package org.apache.beam.sdk.extensions.protobuf; +import static org.apache.beam.sdk.testing.CoderProperties.ALL_CONTEXTS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.protobuf.DynamicMessage; +import java.io.ObjectStreamClass; import java.util.Collections; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -167,4 +170,26 @@ public void testNonDeterministicProperty() throws CoderException { Coder coder = ProtoCoder.of(MessageWithMap.class); assertNotEquals(CoderUtils.encodeToBase64(coder, msg2), CoderUtils.encodeToBase64(coder, msg1)); } + + @Test + public void testDynamicMessage() throws Exception { + DynamicMessage message = + DynamicMessage.newBuilder(MessageA.getDescriptor()) + .setField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") + .build(); + Coder coder = ProtoCoder.of(message.getDescriptorForType()); + + // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) + for (Coder.Context context : ALL_CONTEXTS) { + CoderProperties.coderDecodeEncodeInContext( + coder, context, message, IsDynamicMessageEqual.equalTo(message)); + } + } + + @Test + public void testSerialVersionID() { + long serialVersionID = ObjectStreamClass.lookup(ProtoCoder.class).getSerialVersionUID(); + assertEquals(-5043999806040629525L, serialVersionID); + } } From ba4a121254c47fcebdd8810aad139e881a250e31 Mon Sep 17 00:00:00 2001 From: Alex Van Boxel Date: Mon, 14 Oct 2019 22:27:38 +0200 Subject: [PATCH 16/57] [BEAM-5967] Add Nullable annotations and corrected JavaDoc after review --- .../main/java/org/apache/beam/sdk/testing/CoderProperties.java | 3 ++- .../org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index 1c43fc6e1e644..e89cacd879063 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -106,7 +106,8 @@ public static void coderDecodeEncodeEqualInContext( /** * Verifies that for the given {@code Coder}, {@code Coder.Context}, and value of type {@code - * T}, encoding followed by decoding yields an equal value of type {@code T}. + * T}, encoding followed by decoding yields a value of type {@code T} and tests that the matcher + * succeeds on the values. */ public static void coderDecodeEncodeInContext( Coder coder, Coder.Context context, T value, org.hamcrest.Matcher matcher) diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java index d1aca66cd2061..e9a5d48ed35b6 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import javax.annotation.Nullable; /** * ProtoDomain is a container class for Protobuf descriptors. By using a domain for all descriptors @@ -72,6 +73,7 @@ private static Map extractProtoMap return map; } + @Nullable private static Descriptors.FileDescriptor convertToFileDescriptorMap( String name, Map inMap, From cb774734f24ac143b4cbde1ae6d0d925c8b35499 Mon Sep 17 00:00:00 2001 From: Alex Van Boxel Date: Fri, 18 Oct 2019 00:16:35 +0200 Subject: [PATCH 17/57] Backported ProtoDomainTest for schema aware PR --- .../extensions/protobuf/ProtoCoderTest.java | 30 ++++++++++ .../extensions/protobuf/ProtoDomainTest.java | 55 +++++++++++++++++++ 2 files changed, 85 insertions(+) create mode 100644 sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomainTest.java diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java index 25c62517e5dca..0a493931c9eaa 100644 --- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java @@ -187,6 +187,36 @@ public void testDynamicMessage() throws Exception { } } + @Test + public void testDynamicNestedRepeatedMessage() throws Exception { + DynamicMessage message = + DynamicMessage.newBuilder(MessageA.getDescriptor()) + .setField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") + .addRepeatedField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), + DynamicMessage.newBuilder(MessageB.getDescriptor()) + .setField( + MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), + true) + .build()) + .addRepeatedField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), + DynamicMessage.newBuilder(MessageB.getDescriptor()) + .setField( + MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), + false) + .build()) + .build(); + Coder coder = ProtoCoder.of(message.getDescriptorForType()); + + // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) + for (Coder.Context context : ALL_CONTEXTS) { + CoderProperties.coderDecodeEncodeInContext( + coder, context, message, IsDynamicMessageEqual.equalTo(message)); + } + } + @Test public void testSerialVersionID() { long serialVersionID = ObjectStreamClass.lookup(ProtoCoder.class).getSerialVersionUID(); diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomainTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomainTest.java new file mode 100644 index 0000000000000..5ff909bdf7ab2 --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomainTest.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.sdk.extensions.protobuf; + +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link ProtoDomain}. */ +@RunWith(JUnit4.class) +public class ProtoDomainTest { + + @Test + public void testNamespaceEqual() { + ProtoDomain domainFromInt32 = ProtoDomain.buildFrom(Int32Value.getDescriptor()); + ProtoDomain domainFromInt64 = ProtoDomain.buildFrom(Int64Value.getDescriptor()); + Assert.assertTrue(domainFromInt64.equals(domainFromInt32)); + } + + @Test + public void testContainsDescriptor() { + ProtoDomain domainFromInt32 = ProtoDomain.buildFrom(Int32Value.getDescriptor()); + Assert.assertTrue(domainFromInt32.contains(Int32Value.getDescriptor())); + } + + @Test + public void testContainsOtherDescriptorSameFile() { + ProtoDomain domain = ProtoDomain.buildFrom(Int32Value.getDescriptor()); + Assert.assertTrue(domain.contains(Int64Value.getDescriptor())); + } + + @Test + public void testBuildForFile() { + ProtoDomain domain = ProtoDomain.buildFrom(Int32Value.getDescriptor().getFile()); + Assert.assertNotNull(domain.getFileDescriptor("google/protobuf/wrappers.proto")); + } +} From 64829e18a40bab6f76db4aef3a06d2a32254533e Mon Sep 17 00:00:00 2001 From: Alex Van Boxel Date: Mon, 21 Oct 2019 22:48:15 +0200 Subject: [PATCH 18/57] Split the ProtoCoder into ProtoCoder and DynamicProtoCoder. --- .../protobuf/DynamicProtoCoder.java | 205 ++++++++++++++++++ .../sdk/extensions/protobuf/ProtoCoder.java | 119 ++-------- .../protobuf/DynamicProtoCoderTest.java | 92 ++++++++ .../extensions/protobuf/ProtoCoderTest.java | 48 ---- 4 files changed, 320 insertions(+), 144 deletions(-) create mode 100644 sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoder.java create mode 100644 sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoder.java new file mode 100644 index 0000000000000..96ca0faba7b46 --- /dev/null +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoder.java @@ -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. + */ +package org.apache.beam.sdk.extensions.protobuf; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; + +/** + * A {@link Coder} using Google Protocol Buffers binary format. {@link DynamicProtoCoder} supports + * both Protocol Buffers syntax versions 2 and 3. + * + *

To learn more about Protocol Buffers, visit: https://developers.google.com/protocol-buffers + * + *

{@link DynamicProtoCoder} is not registered in the global {@link CoderRegistry} as the + * descriptor is required to create the coder. + */ +public class DynamicProtoCoder extends ProtoCoder { + + public static final long serialVersionUID = 1L; + + /** + * Returns a {@link DynamicProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the + * given {@link Descriptors.Descriptor}. + */ + public static DynamicProtoCoder of(Descriptors.Descriptor protoMessageDescriptor) { + return new DynamicProtoCoder( + ProtoDomain.buildFrom(protoMessageDescriptor), + protoMessageDescriptor.getFullName(), + ImmutableSet.of()); + } + + /** + * Returns a {@link DynamicProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the + * given {@link Descriptors.Descriptor}. The message descriptor should be part of the provided + * {@link ProtoDomain}, this will ensure object equality within messages from the same domain. + */ + public static DynamicProtoCoder of( + ProtoDomain domain, Descriptors.Descriptor protoMessageDescriptor) { + return new DynamicProtoCoder(domain, protoMessageDescriptor.getFullName(), ImmutableSet.of()); + } + + /** + * Returns a {@link DynamicProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the + * given message name in a {@link ProtoDomain}. The message descriptor should be part of the + * provided * {@link ProtoDomain}, this will ensure object equality within messages from the same + * domain. + */ + public static DynamicProtoCoder of(ProtoDomain domain, String messageName) { + return new DynamicProtoCoder(domain, messageName, ImmutableSet.of()); + } + + /** + * Returns a {@link DynamicProtoCoder} like this one, but with the extensions from the given + * classes registered. + * + *

Each of the extension host classes must be an class automatically generated by the Protocol + * Buffers compiler, {@code protoc}, that contains messages. + * + *

Does not modify this object. + */ + @Override + public DynamicProtoCoder withExtensionsFrom(Iterable> moreExtensionHosts) { + validateExtensions(moreExtensionHosts); + return new DynamicProtoCoder( + this.domain, + this.messageName, + new ImmutableSet.Builder>() + .addAll(extensionHostClasses) + .addAll(moreExtensionHosts) + .build()); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + DynamicProtoCoder otherCoder = (DynamicProtoCoder) other; + return protoMessageClass.equals(otherCoder.protoMessageClass) + && Sets.newHashSet(extensionHostClasses) + .equals(Sets.newHashSet(otherCoder.extensionHostClasses)) + && domain.equals(otherCoder.domain) + && messageName.equals(otherCoder.messageName); + } + + @Override + public int hashCode() { + return Objects.hash(protoMessageClass, extensionHostClasses, domain, messageName); + } + + //////////////////////////////////////////////////////////////////////////////////// + // Private implementation details below. + + // Constants used to serialize and deserialize + private static final String PROTO_MESSAGE_CLASS = "dynamic_proto_message_class"; + private static final String PROTO_EXTENSION_HOSTS = "dynamic_proto_extension_hosts"; + + // Descriptor used by DynamicMessage. + private transient ProtoDomain domain; + private transient String messageName; + + private DynamicProtoCoder( + ProtoDomain domain, String messageName, Set> extensionHostClasses) { + super(DynamicMessage.class, extensionHostClasses); + this.domain = domain; + this.messageName = messageName; + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.defaultWriteObject(); + oos.writeObject(domain); + oos.writeObject(messageName); + } + + private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { + ois.defaultReadObject(); + this.domain = (ProtoDomain) ois.readObject(); + this.messageName = (String) ois.readObject(); + } + + /** Get the memoized {@link Parser}, possibly initializing it lazily. */ + @Override + protected Parser getParser() { + if (memoizedParser == null) { + DynamicMessage protoMessageInstance = + DynamicMessage.newBuilder(domain.getDescriptor(messageName)).build(); + memoizedParser = protoMessageInstance.getParserForType(); + } + return memoizedParser; + } + + /** + * Returns a {@link CoderProvider} which uses the {@link DynamicProtoCoder} for {@link Message + * proto messages}. + * + *

This method is invoked reflectively from {@link DefaultCoder}. + */ + public static CoderProvider getCoderProvider() { + return new ProtoCoderProvider(); + } + + static final TypeDescriptor MESSAGE_TYPE = new TypeDescriptor() {}; + + /** A {@link CoderProvider} for {@link Message proto messages}. */ + private static class ProtoCoderProvider extends CoderProvider { + + @Override + public Coder coderFor( + TypeDescriptor typeDescriptor, List> componentCoders) + throws CannotProvideCoderException { + if (!typeDescriptor.isSubtypeOf(MESSAGE_TYPE)) { + throw new CannotProvideCoderException( + String.format( + "Cannot provide %s because %s is not a subclass of %s", + DynamicProtoCoder.class.getSimpleName(), typeDescriptor, Message.class.getName())); + } + + @SuppressWarnings("unchecked") + TypeDescriptor messageType = + (TypeDescriptor) typeDescriptor; + try { + @SuppressWarnings("unchecked") + Coder coder = (Coder) DynamicProtoCoder.of(messageType); + return coder; + } catch (IllegalArgumentException e) { + throw new CannotProvideCoderException(e); + } + } + } +} diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java index faca3881f1fcc..0b2d717da2885 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java @@ -19,15 +19,12 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; import com.google.protobuf.Parser; import java.io.IOException; import java.io.InputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -36,8 +33,6 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -129,45 +124,11 @@ public static ProtoCoder of(TypeDescriptor protoMessag } /** - * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given - * {@link Descriptors.Descriptor}. - */ - public static ProtoCoder of(Descriptors.Descriptor protoMessageDescriptor) { - return new ProtoCoder<>( - ProtoDomain.buildFrom(protoMessageDescriptor), - protoMessageDescriptor.getFullName(), - ImmutableSet.of()); - } - - /** - * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given - * {@link Descriptors.Descriptor}. The message descriptor should be part of the provided {@link - * ProtoDomain}, this will ensure object equality within messages from the same domain. - */ - public static ProtoCoder of( - ProtoDomain domain, Descriptors.Descriptor protoMessageDescriptor) { - return new ProtoCoder<>(domain, protoMessageDescriptor.getFullName(), ImmutableSet.of()); - } - - /** - * Returns a {@link ProtoCoder} for the Protocol Buffers {@link DynamicMessage} for the given - * message name in a {@link ProtoDomain}. The message descriptor should be part of the provided * - * {@link ProtoDomain}, this will ensure object equality within messages from the same domain. - */ - public static ProtoCoder of(ProtoDomain domain, String messageName) { - return new ProtoCoder<>(domain, messageName, ImmutableSet.of()); - } - - /** - * Returns a {@link ProtoCoder} like this one, but with the extensions from the given classes - * registered. + * Validate that all extensionHosts are able to be registered. * - *

Each of the extension host classes must be an class automatically generated by the Protocol - * Buffers compiler, {@code protoc}, that contains messages. - * - *

Does not modify this object. + * @param moreExtensionHosts */ - public ProtoCoder withExtensionsFrom(Iterable> moreExtensionHosts) { + void validateExtensions(Iterable> moreExtensionHosts) { for (Class extensionHost : moreExtensionHosts) { // Attempt to access the required method, to make sure it's present. try { @@ -182,7 +143,19 @@ public ProtoCoder withExtensionsFrom(Iterable> moreExtensionHosts) { e); } } + } + /** + * Returns a {@link ProtoCoder} like this one, but with the extensions from the given classes + * registered. + * + *

Each of the extension host classes must be an class automatically generated by the Protocol + * Buffers compiler, {@code protoc}, that contains messages. + * + *

Does not modify this object. + */ + public ProtoCoder withExtensionsFrom(Iterable> moreExtensionHosts) { + validateExtensions(moreExtensionHosts); return new ProtoCoder<>( protoMessageClass, new ImmutableSet.Builder>() @@ -236,7 +209,7 @@ public boolean equals(Object other) { if (this == other) { return true; } - if (!(other instanceof ProtoCoder)) { + if (other == null || getClass() != other.getClass()) { return false; } ProtoCoder otherCoder = (ProtoCoder) other; @@ -289,13 +262,13 @@ public ExtensionRegistry getExtensionRegistry() { // Private implementation details below. /** The {@link Message} type to be coded. */ - private final Class protoMessageClass; + final Class protoMessageClass; /** * All extension host classes included in this {@link ProtoCoder}. The extensions from these * classes will be included in the {@link ExtensionRegistry} used during encoding and decoding. */ - private final Set> extensionHostClasses; + final Set> extensionHostClasses; // Constants used to serialize and deserialize private static final String PROTO_MESSAGE_CLASS = "proto_message_class"; @@ -303,59 +276,21 @@ public ExtensionRegistry getExtensionRegistry() { // Transient fields that are lazy initialized and then memoized. private transient ExtensionRegistry memoizedExtensionRegistry; - private transient Parser memoizedParser; - - // Descriptor used by DynamicMessage. - private transient ProtoDomain domain; - private transient String messageName; + transient Parser memoizedParser; /** Private constructor. */ - private ProtoCoder(Class protoMessageClass, Set> extensionHostClasses) { + protected ProtoCoder(Class protoMessageClass, Set> extensionHostClasses) { this.protoMessageClass = protoMessageClass; this.extensionHostClasses = extensionHostClasses; - this.domain = null; - this.messageName = null; - } - - private ProtoCoder(ProtoDomain domain, String messageName, Set> extensionHostClasses) { - @SuppressWarnings("unchecked") - Class protoMessageClass = (Class) DynamicMessage.class; - this.protoMessageClass = protoMessageClass; - this.extensionHostClasses = extensionHostClasses; - this.domain = domain; - this.messageName = messageName; - } - - private void writeObject(ObjectOutputStream oos) throws IOException { - oos.defaultWriteObject(); - if (DynamicMessage.class.equals(this.protoMessageClass)) { - if (this.domain == null) { - throw new RuntimeException("DynamicMessages require provider a proto domain to the coder."); - } - oos.writeObject(domain); - oos.writeObject(messageName); - } - } - - private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { - ois.defaultReadObject(); - if (DynamicMessage.class.equals(this.protoMessageClass)) { - this.domain = (ProtoDomain) ois.readObject(); - this.messageName = (String) ois.readObject(); - } } /** Get the memoized {@link Parser}, possibly initializing it lazily. */ - private Parser getParser() { + protected Parser getParser() { if (memoizedParser == null) { try { if (DynamicMessage.class.equals(protoMessageClass)) { - @SuppressWarnings("unchecked") - T protoMessageInstance = - (T) DynamicMessage.newBuilder(domain.getDescriptor(messageName)).build(); - @SuppressWarnings("unchecked") - Parser tParser = (Parser) protoMessageInstance.getParserForType(); - memoizedParser = tParser; + throw new IllegalArgumentException( + "DynamicMessage is not supported by the ProtoCoder, use the DynamicProtoCoder."); } else { @SuppressWarnings("unchecked") T protoMessageInstance = @@ -409,12 +344,4 @@ public Coder coderFor( } } } - - private SortedSet getSortedExtensionClasses() { - SortedSet ret = new TreeSet<>(); - for (Class clazz : extensionHostClasses) { - ret.add(clazz.getName()); - } - return ret; - } } diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java new file mode 100644 index 0000000000000..10395832b36ad --- /dev/null +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.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.sdk.extensions.protobuf; + +import static org.apache.beam.sdk.testing.CoderProperties.ALL_CONTEXTS; +import static org.junit.Assert.assertEquals; + +import com.google.protobuf.DynamicMessage; +import java.io.ObjectStreamClass; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.protobuf.Proto2CoderTestMessages.MessageA; +import org.apache.beam.sdk.extensions.protobuf.Proto2CoderTestMessages.MessageB; +import org.apache.beam.sdk.testing.CoderProperties; +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 ProtoCoder}. */ +@RunWith(JUnit4.class) +public class DynamicProtoCoderTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testDynamicMessage() throws Exception { + DynamicMessage message = + DynamicMessage.newBuilder(MessageA.getDescriptor()) + .setField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") + .build(); + Coder coder = DynamicProtoCoder.of(message.getDescriptorForType()); + + // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) + for (Coder.Context context : ALL_CONTEXTS) { + CoderProperties.coderDecodeEncodeInContext( + coder, context, message, IsDynamicMessageEqual.equalTo(message)); + } + } + + @Test + public void testDynamicNestedRepeatedMessage() throws Exception { + DynamicMessage message = + DynamicMessage.newBuilder(MessageA.getDescriptor()) + .setField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") + .addRepeatedField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), + DynamicMessage.newBuilder(MessageB.getDescriptor()) + .setField( + MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), + true) + .build()) + .addRepeatedField( + MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), + DynamicMessage.newBuilder(MessageB.getDescriptor()) + .setField( + MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), + false) + .build()) + .build(); + Coder coder = DynamicProtoCoder.of(message.getDescriptorForType()); + + // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) + for (Coder.Context context : ALL_CONTEXTS) { + CoderProperties.coderDecodeEncodeInContext( + coder, context, message, IsDynamicMessageEqual.equalTo(message)); + } + } + + @Test + public void testSerialVersionID() { + long serialVersionID = ObjectStreamClass.lookup(DynamicProtoCoder.class).getSerialVersionUID(); + assertEquals(1L, serialVersionID); + } +} diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java index 0a493931c9eaa..38aa92bfc2236 100644 --- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java @@ -17,11 +17,9 @@ */ package org.apache.beam.sdk.extensions.protobuf; -import static org.apache.beam.sdk.testing.CoderProperties.ALL_CONTEXTS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import com.google.protobuf.DynamicMessage; import java.io.ObjectStreamClass; import java.util.Collections; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -171,52 +169,6 @@ public void testNonDeterministicProperty() throws CoderException { assertNotEquals(CoderUtils.encodeToBase64(coder, msg2), CoderUtils.encodeToBase64(coder, msg1)); } - @Test - public void testDynamicMessage() throws Exception { - DynamicMessage message = - DynamicMessage.newBuilder(MessageA.getDescriptor()) - .setField( - MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") - .build(); - Coder coder = ProtoCoder.of(message.getDescriptorForType()); - - // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) - for (Coder.Context context : ALL_CONTEXTS) { - CoderProperties.coderDecodeEncodeInContext( - coder, context, message, IsDynamicMessageEqual.equalTo(message)); - } - } - - @Test - public void testDynamicNestedRepeatedMessage() throws Exception { - DynamicMessage message = - DynamicMessage.newBuilder(MessageA.getDescriptor()) - .setField( - MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD1_FIELD_NUMBER), "foo") - .addRepeatedField( - MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), - DynamicMessage.newBuilder(MessageB.getDescriptor()) - .setField( - MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), - true) - .build()) - .addRepeatedField( - MessageA.getDescriptor().findFieldByNumber(MessageA.FIELD2_FIELD_NUMBER), - DynamicMessage.newBuilder(MessageB.getDescriptor()) - .setField( - MessageB.getDescriptor().findFieldByNumber(MessageB.FIELD1_FIELD_NUMBER), - false) - .build()) - .build(); - Coder coder = ProtoCoder.of(message.getDescriptorForType()); - - // Special code to check the DynamicMessage equality (@see IsDynamicMessageEqual) - for (Coder.Context context : ALL_CONTEXTS) { - CoderProperties.coderDecodeEncodeInContext( - coder, context, message, IsDynamicMessageEqual.equalTo(message)); - } - } - @Test public void testSerialVersionID() { long serialVersionID = ObjectStreamClass.lookup(ProtoCoder.class).getSerialVersionUID(); From 4fa1501b377e70df0971ec4a4b8f585fd8f34a6c Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Tue, 22 Oct 2019 13:14:20 -0700 Subject: [PATCH 19/57] [BEAM-7389] Show code snippet outputs as stdout (#9790) [BEAM-7389] Show code snippet outputs as stdout (#9790) --- .../transforms/elementwise/filter_test.py | 38 ++--- .../transforms/elementwise/flatmap_test.py | 42 ++--- .../transforms/elementwise/keys_test.py | 26 ++- .../transforms/elementwise/kvswap_test.py | 26 ++- .../transforms/elementwise/map_test.py | 44 +++-- .../snippets/transforms/elementwise/pardo.py | 1 - .../transforms/elementwise/pardo_test.py | 41 ++--- .../transforms/elementwise/partition.py | 49 ++---- .../transforms/elementwise/partition_test.py | 66 +++++--- .../transforms/elementwise/regex_test.py | 156 ++++++++---------- .../transforms/elementwise/tostring_test.py | 79 +++------ .../transforms/elementwise/values_test.py | 26 ++- .../elementwise/withtimestamps_test.py | 68 ++++---- .../apache_beam/examples/snippets/util.py | 36 ++-- .../examples/snippets/util_test.py | 56 ++++--- 15 files changed, 348 insertions(+), 406 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py index d989e43caf565..724b1b9c75f52 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py @@ -23,40 +23,34 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import filter def check_perennials(actual): - # [START perennials] - perennials = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'}, - ] - # [END perennials] - assert_that(actual, equal_to(perennials)) + expected = '''[START perennials] +{'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +{'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +{'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'} +[END perennials]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_valid_plants(actual): - # [START valid_plants] - valid_plants = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'}, - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'}, - ] - # [END valid_plants] - assert_that(actual, equal_to(valid_plants)) + expected = '''[START valid_plants] +{'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +{'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'} +{'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +{'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'} +[END valid_plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.filter.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.filter.print', str) class FilterTest(unittest.TestCase): def test_filter_function(self): filter.filter_function(check_perennials) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py index 718dceef78247..5c326e93bd08d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py @@ -23,42 +23,36 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import flatmap def check_plants(actual): - # [START plants] - plants = [ - '🍓Strawberry', - '🥕Carrot', - '🍆Eggplant', - '🍅Tomato', - '🥔Potato', - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +🍓Strawberry +🥕Carrot +🍆Eggplant +🍅Tomato +🥔Potato +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_valid_plants(actual): - # [START valid_plants] - valid_plants = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'}, - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'}, - ] - # [END valid_plants] - assert_that(actual, equal_to(valid_plants)) + expected = '''[START valid_plants] +{'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +{'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'} +{'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +{'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'} +[END valid_plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.flatmap.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.flatmap.print', str) class FlatMapTest(unittest.TestCase): def test_flatmap_simple(self): flatmap.flatmap_simple(check_plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py index 780c5e43ec281..e4a843b2b2470 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py @@ -23,30 +23,26 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import keys def check_icons(actual): - # [START icons] - icons = [ - '🍓', - '🥕', - '🍆', - '🍅', - '🥔', - ] - # [END icons] - assert_that(actual, equal_to(icons)) + expected = '''[START icons] +🍓 +🥕 +🍆 +🍅 +🥔 +[END icons]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.keys.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.keys.print', str) class KeysTest(unittest.TestCase): def test_keys(self): keys.keys(check_icons) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py index ea7698bf32968..83f211dad3952 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py @@ -23,30 +23,26 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import kvswap def check_plants(actual): - # [START plants] - plants = [ - ('Strawberry', '🍓'), - ('Carrot', '🥕'), - ('Eggplant', '🍆'), - ('Tomato', '🍅'), - ('Potato', '🥔'), - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +('Strawberry', '🍓') +('Carrot', '🥕') +('Eggplant', '🍆') +('Tomato', '🍅') +('Potato', '🥔') +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.kvswap.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.kvswap.print', str) class KvSwapTest(unittest.TestCase): def test_kvswap(self): kvswap.kvswap(check_plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py index 41861761f6190..eb7767587b14c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py @@ -23,43 +23,37 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import map def check_plants(actual): - # [START plants] - plants = [ - '🍓Strawberry', - '🥕Carrot', - '🍆Eggplant', - '🍅Tomato', - '🥔Potato', - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +🍓Strawberry +🥕Carrot +🍆Eggplant +🍅Tomato +🥔Potato +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_plant_details(actual): - # [START plant_details] - plant_details = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'}, - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'}, - {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'}, - ] - # [END plant_details] - assert_that(actual, equal_to(plant_details)) + expected = '''[START plant_details] +{'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +{'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'} +{'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +{'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'} +{'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'} +[END plant_details]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.map.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.map.print', str) class MapTest(unittest.TestCase): def test_map_simple(self): map.map_simple(check_plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py index 971e9f0638b8e..4ecd74d30250c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py @@ -18,7 +18,6 @@ from __future__ import absolute_import from __future__ import print_function -from __future__ import unicode_literals def pardo_dofn(test=None): diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py index 8507e01b6e957..cbf49032f00cd 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py @@ -18,38 +18,42 @@ from __future__ import absolute_import from __future__ import print_function -from __future__ import unicode_literals -import io import platform import sys import unittest import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from . import pardo +# TODO: Remove this after Python 2 deprecation. +# https://issues.apache.org/jira/browse/BEAM-8124 +if sys.version_info[0] == 2: + from io import BytesIO as StringIO +else: + from io import StringIO + def check_plants(actual): - # [START plants] - plants = [ - '🍓Strawberry', - '🥕Carrot', - '🍆Eggplant', - '🍅Tomato', - '🥔Potato', - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +🍓Strawberry +🥕Carrot +🍆Eggplant +🍅Tomato +🥔Potato +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_dofn_params(actual): # pylint: disable=line-too-long - dofn_params = '\n'.join('''[START dofn_params] + expected = '\n'.join('''[START dofn_params] # timestamp type(timestamp) -> timestamp.micros -> 1584675660000000 @@ -63,7 +67,7 @@ def check_dofn_params(actual): window.max_timestamp() -> Timestamp(1584675689.999999) (2020-03-20 03:41:29.999999) [END dofn_params]'''.splitlines()[1:-1]) # pylint: enable=line-too-long - assert_that(actual, equal_to([dofn_params])) + assert_that(actual, equal_to([expected])) def check_dofn_methods(actual): @@ -83,23 +87,22 @@ def check_dofn_methods(actual): @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.pardo.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.pardo.print', str) class ParDoTest(unittest.TestCase): def test_pardo_dofn(self): pardo.pardo_dofn(check_plants) # TODO: Remove this after Python 2 deprecation. # https://issues.apache.org/jira/browse/BEAM-8124 - @unittest.skipIf(sys.version_info[0] < 3 and platform.system() == 'Windows', + @unittest.skipIf(sys.version_info[0] == 2 and platform.system() == 'Windows', 'Python 2 on Windows uses `long` rather than `int`') def test_pardo_dofn_params(self): pardo.pardo_dofn_params(check_dofn_params) @mock.patch('apache_beam.Pipeline', TestPipeline) -@mock.patch('sys.stdout', new_callable=io.StringIO) +@mock.patch('sys.stdout', new_callable=StringIO) class ParDoStdoutTest(unittest.TestCase): def test_pardo_dofn_methods(self, mock_stdout): expected = pardo.pardo_dofn_methods(check_dofn_methods) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py index 6f839d4a0ae43..56336074bc365 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py @@ -21,6 +21,7 @@ def partition_function(test=None): + # pylint: disable=line-too-long, expression-not-assigned # [START partition_function] import apache_beam as beam @@ -41,24 +42,18 @@ def by_duration(plant, num_partitions): ]) | 'Partition' >> beam.Partition(by_duration, len(durations)) ) - _ = ( - annuals - | 'Annuals' >> beam.Map(lambda x: print('annual: ' + str(x))) - ) - _ = ( - biennials - | 'Biennials' >> beam.Map(lambda x: print('biennial: ' + str(x))) - ) - _ = ( - perennials - | 'Perennials' >> beam.Map(lambda x: print('perennial: ' + str(x))) - ) + + annuals | 'Annuals' >> beam.Map(lambda x: print('annual: {}'.format(x))) + biennials | 'Biennials' >> beam.Map(lambda x: print('biennial: {}'.format(x))) + perennials | 'Perennials' >> beam.Map(lambda x: print('perennial: {}'.format(x))) # [END partition_function] + # pylint: enable=line-too-long, expression-not-assigned if test: test(annuals, biennials, perennials) def partition_lambda(test=None): + # pylint: disable=line-too-long, expression-not-assigned # [START partition_lambda] import apache_beam as beam @@ -79,24 +74,18 @@ def partition_lambda(test=None): len(durations), ) ) - _ = ( - annuals - | 'Annuals' >> beam.Map(lambda x: print('annual: ' + str(x))) - ) - _ = ( - biennials - | 'Biennials' >> beam.Map(lambda x: print('biennial: ' + str(x))) - ) - _ = ( - perennials - | 'Perennials' >> beam.Map(lambda x: print('perennial: ' + str(x))) - ) + + annuals | 'Annuals' >> beam.Map(lambda x: print('annual: {}'.format(x))) + biennials | 'Biennials' >> beam.Map(lambda x: print('biennial: {}'.format(x))) + perennials | 'Perennials' >> beam.Map(lambda x: print('perennial: {}'.format(x))) # [END partition_lambda] + # pylint: enable=line-too-long, expression-not-assigned if test: test(annuals, biennials, perennials) def partition_multiple_arguments(test=None): + # pylint: disable=expression-not-assigned # [START partition_multiple_arguments] import apache_beam as beam import json @@ -123,14 +112,10 @@ def split_dataset(plant, num_partitions, ratio): ]) | 'Partition' >> beam.Partition(split_dataset, 2, ratio=[8, 2]) ) - _ = ( - train_dataset - | 'Train' >> beam.Map(lambda x: print('train: ' + str(x))) - ) - _ = ( - test_dataset - | 'Test' >> beam.Map(lambda x: print('test: ' + str(x))) - ) + + train_dataset | 'Train' >> beam.Map(lambda x: print('train: {}'.format(x))) + test_dataset | 'Test' >> beam.Map(lambda x: print('test: {}'.format(x))) # [END partition_multiple_arguments] + # pylint: enable=expression-not-assigned if test: test(train_dataset, test_dataset) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py index 0b8ae3d107cea..45842039446fc 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py @@ -23,52 +23,70 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import partition def check_partitions(actual1, actual2, actual3): - # [START partitions] + expected = '''[START partitions] +perennial: {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +biennial: {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'} +perennial: {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +annual: {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'} +perennial: {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'} +[END partitions]'''.splitlines()[1:-1] + annuals = [ - {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'}, + line.split(':', 1)[1].strip() + for line in expected + if line.split(':', 1)[0] == 'annual' ] biennials = [ - {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'}, + line.split(':', 1)[1].strip() + for line in expected + if line.split(':', 1)[0] == 'biennial' ] perennials = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'}, + line.split(':', 1)[1].strip() + for line in expected + if line.split(':', 1)[0] == 'perennial' ] - # [END partitions] - assert_that(actual1, equal_to(annuals), label='assert annuals') - assert_that(actual2, equal_to(biennials), label='assert biennials') - assert_that(actual3, equal_to(perennials), label='assert perennials') + + assert_matches_stdout(actual1, annuals, 'annuals') + assert_matches_stdout(actual2, biennials, 'biennials') + assert_matches_stdout(actual3, perennials, 'perennials') def check_split_datasets(actual1, actual2): - # [START train_test] + expected = '''[START train_test] +train: {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'} +train: {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'} +test: {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'} +test: {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'} +train: {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'} +[END train_test]'''.splitlines()[1:-1] + train_dataset = [ - {'icon': '🍓', 'name': 'Strawberry', 'duration': 'perennial'}, - {'icon': '🥕', 'name': 'Carrot', 'duration': 'biennial'}, - {'icon': '🥔', 'name': 'Potato', 'duration': 'perennial'}, + line.split(':', 1)[1].strip() + for line in expected + if line.split(':', 1)[0] == 'train' ] test_dataset = [ - {'icon': '🍆', 'name': 'Eggplant', 'duration': 'perennial'}, - {'icon': '🍅', 'name': 'Tomato', 'duration': 'annual'}, + line.split(':', 1)[1].strip() + for line in expected + if line.split(':', 1)[0] == 'test' ] - # [END train_test] - assert_that(actual1, equal_to(train_dataset), label='assert train') - assert_that(actual2, equal_to(test_dataset), label='assert test') + + assert_matches_stdout(actual1, train_dataset, 'train_dataset') + assert_matches_stdout(actual2, test_dataset, 'test_dataset') @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.partition.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.partition.print', + lambda elem: elem) class PartitionTest(unittest.TestCase): def test_partition_function(self): partition.partition_function(check_partitions) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py index 83123124512cc..9df9f62495143 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py @@ -23,123 +23,105 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import regex def check_matches(actual): - # [START plants_matches] - plants_matches = [ - '🍓, Strawberry, perennial', - '🥕, Carrot, biennial', - '🍆, Eggplant, perennial', - '🍅, Tomato, annual', - '🥔, Potato, perennial', - ] - # [END plants_matches] - assert_that(actual, equal_to(plants_matches)) + expected = '''[START plants_matches] +🍓, Strawberry, perennial +🥕, Carrot, biennial +🍆, Eggplant, perennial +🍅, Tomato, annual +🥔, Potato, perennial +[END plants_matches]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_all_matches(actual): - # [START plants_all_matches] - plants_all_matches = [ - ['🍓, Strawberry, perennial', '🍓', 'Strawberry', 'perennial'], - ['🥕, Carrot, biennial', '🥕', 'Carrot', 'biennial'], - ['🍆, Eggplant, perennial', '🍆', 'Eggplant', 'perennial'], - ['🍅, Tomato, annual', '🍅', 'Tomato', 'annual'], - ['🥔, Potato, perennial', '🥔', 'Potato', 'perennial'], - ] - # [END plants_all_matches] - assert_that(actual, equal_to(plants_all_matches)) + expected = '''[START plants_all_matches] +['🍓, Strawberry, perennial', '🍓', 'Strawberry', 'perennial'] +['🥕, Carrot, biennial', '🥕', 'Carrot', 'biennial'] +['🍆, Eggplant, perennial', '🍆', 'Eggplant', 'perennial'] +['🍅, Tomato, annual', '🍅', 'Tomato', 'annual'] +['🥔, Potato, perennial', '🥔', 'Potato', 'perennial'] +[END plants_all_matches]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_matches_kv(actual): - # [START plants_matches_kv] - plants_matches_kv = [ - ('🍓', '🍓, Strawberry, perennial'), - ('🥕', '🥕, Carrot, biennial'), - ('🍆', '🍆, Eggplant, perennial'), - ('🍅', '🍅, Tomato, annual'), - ('🥔', '🥔, Potato, perennial'), - ] - # [END plants_matches_kv] - assert_that(actual, equal_to(plants_matches_kv)) + expected = '''[START plants_matches_kv] +('🍓', '🍓, Strawberry, perennial') +('🥕', '🥕, Carrot, biennial') +('🍆', '🍆, Eggplant, perennial') +('🍅', '🍅, Tomato, annual') +('🥔', '🥔, Potato, perennial') +[END plants_matches_kv]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_find_all(actual): - # [START plants_find_all] - plants_find_all = [ - ['🍓, Strawberry, perennial'], - ['🥕, Carrot, biennial'], - ['🍆, Eggplant, perennial', '🍌, Banana, perennial'], - ['🍅, Tomato, annual', '🍉, Watermelon, annual'], - ['🥔, Potato, perennial'], - ] - # [END plants_find_all] - assert_that(actual, equal_to(plants_find_all)) + expected = '''[START plants_find_all] +['🍓, Strawberry, perennial'] +['🥕, Carrot, biennial'] +['🍆, Eggplant, perennial', '🍌, Banana, perennial'] +['🍅, Tomato, annual', '🍉, Watermelon, annual'] +['🥔, Potato, perennial'] +[END plants_find_all]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_find_kv(actual): - # [START plants_find_kv] - plants_find_all = [ - ('🍓', '🍓, Strawberry, perennial'), - ('🥕', '🥕, Carrot, biennial'), - ('🍆', '🍆, Eggplant, perennial'), - ('🍌', '🍌, Banana, perennial'), - ('🍅', '🍅, Tomato, annual'), - ('🍉', '🍉, Watermelon, annual'), - ('🥔', '🥔, Potato, perennial'), - ] - # [END plants_find_kv] - assert_that(actual, equal_to(plants_find_all)) + expected = '''[START plants_find_kv] +('🍓', '🍓, Strawberry, perennial') +('🥕', '🥕, Carrot, biennial') +('🍆', '🍆, Eggplant, perennial') +('🍌', '🍌, Banana, perennial') +('🍅', '🍅, Tomato, annual') +('🍉', '🍉, Watermelon, annual') +('🥔', '🥔, Potato, perennial') +[END plants_find_kv]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_replace_all(actual): - # [START plants_replace_all] - plants_replace_all = [ - '🍓,Strawberry,perennial', - '🥕,Carrot,biennial', - '🍆,Eggplant,perennial', - '🍅,Tomato,annual', - '🥔,Potato,perennial', - ] - # [END plants_replace_all] - assert_that(actual, equal_to(plants_replace_all)) + expected = '''[START plants_replace_all] +🍓,Strawberry,perennial +🥕,Carrot,biennial +🍆,Eggplant,perennial +🍅,Tomato,annual +🥔,Potato,perennial +[END plants_replace_all]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_replace_first(actual): - # [START plants_replace_first] - plants_replace_first = [ - '🍓: Strawberry, perennial', - '🥕: Carrot, biennial', - '🍆: Eggplant, perennial', - '🍅: Tomato, annual', - '🥔: Potato, perennial', - ] - # [END plants_replace_first] - assert_that(actual, equal_to(plants_replace_first)) + expected = '''[START plants_replace_first] +🍓: Strawberry, perennial +🥕: Carrot, biennial +🍆: Eggplant, perennial +🍅: Tomato, annual +🥔: Potato, perennial +[END plants_replace_first]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_split(actual): - # [START plants_split] - plants_split = [ - ['🍓', 'Strawberry', 'perennial'], - ['🥕', 'Carrot', 'biennial'], - ['🍆', 'Eggplant', 'perennial'], - ['🍅', 'Tomato', 'annual'], - ['🥔', 'Potato', 'perennial'], - ] - # [END plants_split] - assert_that(actual, equal_to(plants_split)) + expected = '''[START plants_split] +['🍓', 'Strawberry', 'perennial'] +['🥕', 'Carrot', 'biennial'] +['🍆', 'Eggplant', 'perennial'] +['🍅', 'Tomato', 'annual'] +['🥔', 'Potato', 'perennial'] +[END plants_split]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.regex.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.regex.print', str) class RegexTest(unittest.TestCase): def test_matches(self): regex.regex_matches(check_matches) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py index b253ea1a71967..04939a7cef71d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py @@ -19,77 +19,52 @@ from __future__ import absolute_import from __future__ import print_function -import sys import unittest import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import tostring def check_plants(actual): - # [START plants] - plants = [ - '🍓,Strawberry', - '🥕,Carrot', - '🍆,Eggplant', - '🍅,Tomato', - '🥔,Potato', - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +🍓,Strawberry +🥕,Carrot +🍆,Eggplant +🍅,Tomato +🥔,Potato +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_plant_lists(actual): - # [START plant_lists] - plant_lists = [ - "['🍓', 'Strawberry', 'perennial']", - "['🥕', 'Carrot', 'biennial']", - "['🍆', 'Eggplant', 'perennial']", - "['🍅', 'Tomato', 'annual']", - "['🥔', 'Potato', 'perennial']", - ] - # [END plant_lists] - - # Some unicode characters become escaped with double backslashes. - import apache_beam as beam - - def normalize_escaping(elem): - # In Python 2 all utf-8 characters are escaped with double backslashes. - # TODO: Remove this after Python 2 deprecation. - # https://issues.apache.org/jira/browse/BEAM-8124 - if sys.version_info.major == 2: - return elem.decode('string-escape') - - # In Python 3.5 some utf-8 characters are escaped with double backslashes. - if '\\' in elem: - return bytes(elem, 'utf-8').decode('unicode-escape') - return elem - actual = actual | beam.Map(normalize_escaping) - assert_that(actual, equal_to(plant_lists)) + expected = '''[START plant_lists] +['🍓', 'Strawberry', 'perennial'] +['🥕', 'Carrot', 'biennial'] +['🍆', 'Eggplant', 'perennial'] +['🍅', 'Tomato', 'annual'] +['🥔', 'Potato', 'perennial'] +[END plant_lists]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_plants_csv(actual): - # [START plants_csv] - plants_csv = [ - '🍓,Strawberry,perennial', - '🥕,Carrot,biennial', - '🍆,Eggplant,perennial', - '🍅,Tomato,annual', - '🥔,Potato,perennial', - ] - # [END plants_csv] - assert_that(actual, equal_to(plants_csv)) + expected = '''[START plants_csv] +🍓,Strawberry,perennial +🥕,Carrot,biennial +🍆,Eggplant,perennial +🍅,Tomato,annual +🥔,Potato,perennial +[END plants_csv]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.tostring.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.tostring.print', str) class ToStringTest(unittest.TestCase): def test_tostring_kvs(self): tostring.tostring_kvs(check_plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py index 06abef6bfc4f7..7a3b8f36f15eb 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py @@ -23,30 +23,26 @@ import mock +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import values def check_plants(actual): - # [START plants] - plants = [ - 'Strawberry', - 'Carrot', - 'Eggplant', - 'Tomato', - 'Potato', - ] - # [END plants] - assert_that(actual, equal_to(plants)) + expected = '''[START plants] +Strawberry +Carrot +Eggplant +Tomato +Potato +[END plants]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.values.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.values.print', str) class ValuesTest(unittest.TestCase): def test_values(self): values.values(check_plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py index 53fa7e202a6a5..ad8c31b8a7043 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py @@ -23,63 +23,55 @@ import mock +import apache_beam as beam +from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from . import withtimestamps def check_plant_timestamps(actual): - # [START plant_timestamps] - plant_timestamps = [ - '2020-04-01 00:00:00 - Strawberry', - '2020-06-01 00:00:00 - Carrot', - '2020-03-01 00:00:00 - Artichoke', - '2020-05-01 00:00:00 - Tomato', - '2020-09-01 00:00:00 - Potato', - ] - # [END plant_timestamps] - assert_that(actual, equal_to(plant_timestamps)) + expected = '''[START plant_timestamps] +2020-04-01 00:00:00 - Strawberry +2020-06-01 00:00:00 - Carrot +2020-03-01 00:00:00 - Artichoke +2020-05-01 00:00:00 - Tomato +2020-09-01 00:00:00 - Potato +[END plant_timestamps]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_plant_events(actual): - # [START plant_events] - plant_events = [ - '1 - Strawberry', - '4 - Carrot', - '2 - Artichoke', - '3 - Tomato', - '5 - Potato', - ] - # [END plant_events] - assert_that(actual, equal_to(plant_events)) + expected = '''[START plant_events] +1 - Strawberry +4 - Carrot +2 - Artichoke +3 - Tomato +5 - Potato +[END plant_events]'''.splitlines()[1:-1] + assert_matches_stdout(actual, expected) def check_plant_processing_times(actual): - import apache_beam as beam - - # [START plant_processing_times] - plant_processing_times = [ - '2020-03-20 20:12:42.145594 - Strawberry', - '2020-03-20 20:12:42.145827 - Carrot', - '2020-03-20 20:12:42.145962 - Artichoke', - '2020-03-20 20:12:42.146093 - Tomato', - '2020-03-20 20:12:42.146216 - Potato', - ] - # [END plant_processing_times] + expected = '''[START plant_processing_times] +2020-03-20 20:12:42.145594 - Strawberry +2020-03-20 20:12:42.145827 - Carrot +2020-03-20 20:12:42.145962 - Artichoke +2020-03-20 20:12:42.146093 - Tomato +2020-03-20 20:12:42.146216 - Potato +[END plant_processing_times]'''.splitlines()[1:-1] # Since `time.time()` will always give something different, we'll # simply strip the timestamp information before testing the results. actual = actual | beam.Map(lambda row: row.split('-')[-1].strip()) - expected = [row.split('-')[-1].strip() for row in plant_processing_times] - assert_that(actual, equal_to(expected)) + expected = [row.split('-')[-1].strip() for row in expected] + assert_matches_stdout(actual, expected) @mock.patch('apache_beam.Pipeline', TestPipeline) -# pylint: disable=line-too-long -@mock.patch('apache_beam.examples.snippets.transforms.elementwise.withtimestamps.print', lambda elem: elem) -# pylint: enable=line-too-long +@mock.patch( + 'apache_beam.examples.snippets.transforms.elementwise.withtimestamps.print', + str) class WithTimestampsTest(unittest.TestCase): def test_event_time(self): withtimestamps.withtimestamps_event_time(check_plant_timestamps) diff --git a/sdks/python/apache_beam/examples/snippets/util.py b/sdks/python/apache_beam/examples/snippets/util.py index 6e6e9e0ac2809..ac17c5200886c 100644 --- a/sdks/python/apache_beam/examples/snippets/util.py +++ b/sdks/python/apache_beam/examples/snippets/util.py @@ -17,28 +17,32 @@ from __future__ import absolute_import -import argparse +import ast import shlex import subprocess as sp +import apache_beam as beam +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to -def parse_example(argv=None): - """Parse the command line arguments and return it as a string function call. - Examples: - python path/to/snippets.py function_name - python path/to/snippets.py function_name arg1 - python path/to/snippets.py function_name arg1 arg2 ... argN +def assert_matches_stdout(actual, expected_stdout, label=''): + """Asserts a PCollection of strings matches the expected stdout elements. + + Args: + actual (beam.PCollection): A PCollection. + expected (List[str]): A list of stdout elements, one line per element. + label (str): [optional] Label to make transform names unique. """ - parser = argparse.ArgumentParser() - parser.add_argument('example', help='Name of the example to run.') - parser.add_argument('args', nargs=argparse.REMAINDER, - help='Arguments for example.') - args = parser.parse_args(argv) - - # Return the example as a string representing the Python function call. - example_args = ', '.join([repr(arg) for arg in args.args]) - return '{}({})'.format(args.example, example_args) + def stdout_to_python_object(elem_str): + try: + return ast.literal_eval(elem_str) + except (SyntaxError, ValueError): + return elem_str + + actual = actual | label >> beam.Map(stdout_to_python_object) + expected = list(map(stdout_to_python_object, expected_stdout)) + assert_that(actual, equal_to(expected), 'assert ' + label) def run_shell_commands(commands, **kwargs): diff --git a/sdks/python/apache_beam/examples/snippets/util_test.py b/sdks/python/apache_beam/examples/snippets/util_test.py index a23e9168e9421..8ac20130435c0 100644 --- a/sdks/python/apache_beam/examples/snippets/util_test.py +++ b/sdks/python/apache_beam/examples/snippets/util_test.py @@ -1,3 +1,4 @@ +# coding=utf-8 # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -21,30 +22,43 @@ from mock import patch -from apache_beam.examples.snippets.util import * +import apache_beam as beam +from apache_beam.examples.snippets import util +from apache_beam.testing.test_pipeline import TestPipeline class UtilTest(unittest.TestCase): - def test_parse_example_empty(self): - # python path/to/snippets.py - argv = [] - with self.assertRaises(SystemExit): - self.assertEqual(parse_example(argv), 'example()') - - def test_parse_example_no_arguments(self): - # python path/to/snippets.py example - argv = ['example'] - self.assertEqual(parse_example(argv), 'example()') - - def test_parse_example_one_argument(self): - # python path/to/snippets.py example A - argv = ['example', 'A'] - self.assertEqual(parse_example(argv), "example('A')") + def test_assert_matches_stdout_object(self): + expected = [ + "{'a': '🍓', 'b': True}", + "{'a': '🥕', 'b': 42}", + "{'a': '🍆', 'b': '\"hello\"'}", + "{'a': '🍅', 'b': [1, 2, 3]}", + "{'b': 'B', 'a': '🥔'}", + ] + with TestPipeline() as pipeline: + actual = ( + pipeline + | beam.Create([ + {'a': '🍓', 'b': True}, + {'a': '🥕', 'b': 42}, + {'a': '🍆', 'b': '"hello"'}, + {'a': '🍅', 'b': [1, 2, 3]}, + {'a': '🥔', 'b': 'B'}, + ]) + | beam.Map(str) + ) + util.assert_matches_stdout(actual, expected) - def test_parse_example_multiple_arguments(self): - # python path/to/snippets.py example A B "C's" - argv = ['example', 'A', 'B', "C's"] - self.assertEqual(parse_example(argv), "example('A', 'B', \"C's\")") + def test_assert_matches_stdout_string(self): + expected = ['🍓', '🥕', '🍆', '🍅', '🥔'] + with TestPipeline() as pipeline: + actual = ( + pipeline + | beam.Create(['🍓', '🥕', '🍆', '🍅', '🥔']) + | beam.Map(str) + ) + util.assert_matches_stdout(actual, expected) @patch('subprocess.call', lambda cmd: None) def test_run_shell_commands(self): @@ -54,7 +68,7 @@ def test_run_shell_commands(self): ' !echo {variable} ', ' echo "quoted arguments work" # trailing comment ', ] - actual = list(run_shell_commands(commands, variable='hello world')) + actual = list(util.run_shell_commands(commands, variable='hello world')) expected = [ ['echo', 'this', 'is', 'a', 'shell', 'command'], ['echo', 'hello', 'world'], From 52c45a7957b0161b855d264f7add1ad004240fe8 Mon Sep 17 00:00:00 2001 From: pabloem Date: Tue, 22 Oct 2019 14:52:01 -0700 Subject: [PATCH 20/57] Fixing BQ write transform --- sdks/python/apache_beam/io/gcp/bigquery_file_loads.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py index e74a39b515790..cb285ea02e133 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py @@ -527,6 +527,8 @@ def _check_job_states(self, job_references): return WaitForBQJobs.FAILED elif job.status.state == 'DONE': continue + else: + return WaitForBQJobs.WAITING return WaitForBQJobs.ALL_DONE From 662356923fafc8fb4537592be40218e0ee9d7d44 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Wed, 23 Oct 2019 09:26:57 +0200 Subject: [PATCH 21/57] Add introductory content to the lifecycle sequence diagrams --- website/src/documentation/io/developing-io-overview.md | 6 ++++++ website/src/documentation/programming-guide.md | 7 +++++++ 2 files changed, 13 insertions(+) diff --git a/website/src/documentation/io/developing-io-overview.md b/website/src/documentation/io/developing-io-overview.md index d0a6ddb1fe014..56f275544b3a5 100644 --- a/website/src/documentation/io/developing-io-overview.md +++ b/website/src/documentation/io/developing-io-overview.md @@ -102,6 +102,12 @@ records per file, or if you'd like to read from a key-value store that supports read operations in sorted key order. ### Source lifecycle {#source} +Here is a sequence diagram that shows the lifecycle of the Source during + the execution of the Read transform of an IO. The comments give useful + information to IO developers such as the constraints that + apply to the objects or particular cases such as streaming mode. + + ![This is a sequence diagram that shows the lifecycle of the Source]( {{ "/images/source-sequence-diagram.png" | prepend: site.baseurl }}) diff --git a/website/src/documentation/programming-guide.md b/website/src/documentation/programming-guide.md index 6ac9b22825b9d..afb11635188a7 100644 --- a/website/src/documentation/programming-guide.md +++ b/website/src/documentation/programming-guide.md @@ -803,6 +803,13 @@ words = ... > transforms, including `Filter`, `FlatMapElements`, and `Partition`. ##### 4.2.1.4. DoFn lifecycle {#dofn} +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 instanciation use cases. + + ![This is a sequence diagram that shows the lifecycle of the DoFn]( {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) From a7c8766eac10ffb623208894057217b2c4ad46d6 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Tue, 30 Jul 2019 11:18:20 +0200 Subject: [PATCH 22/57] [BEAM-7520] Add ValidatesRunner test for timer ordering --- .../apache/beam/sdk/transforms/ParDoTest.java | 129 ++++++++++++++++++ 1 file changed, 129 insertions(+) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index ee7c7842cb891..bddd2fe9805d5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -46,11 +46,16 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.function.IntFunction; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -110,6 +115,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; @@ -3488,6 +3494,129 @@ public void onTimer(OutputReceiver r) { pipeline.run(); } + /** A test makes sure that an event time timers are correctly ordered. */ + @Test(timeout = 20000) + @Category({ + ValidatesRunner.class, + UsesTimersInParDo.class, + UsesTestStream.class, + UsesStatefulParDo.class + }) + public void testEventTimeTimerOrdering() throws Exception { + + final String timerIdBagAppend = "append"; + final String timerIdGc = "gc"; + final String bag = "bag"; + final String minTimestamp = "minTs"; + + final int numTestElements = 100; + final Instant now = new Instant(1500000000000L); + final Instant gcTimerStamp = now.plus(numTestElements + 1); + + DoFn, String> fn = + new DoFn, String>() { + + @TimerId(timerIdBagAppend) + private final TimerSpec appendSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @TimerId(timerIdGc) + private final TimerSpec gcSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @StateId(bag) + private final StateSpec>> bagStateSpec = + StateSpecs.bag(); + + @StateId(minTimestamp) + private final StateSpec> minTimestampSpec = StateSpecs.value(); + + @ProcessElement + public void processElement( + ProcessContext context, + @TimerId(timerIdBagAppend) Timer bagTimer, + @TimerId(timerIdGc) Timer gcTimer, + @StateId(bag) BagState> bagState, + @StateId(minTimestamp) ValueState minStampState) { + + Instant currentMinStamp = + MoreObjects.firstNonNull(minStampState.read(), BoundedWindow.TIMESTAMP_MAX_VALUE); + if (currentMinStamp.equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { + gcTimer.set(gcTimerStamp); + } + if (currentMinStamp.isAfter(context.timestamp())) { + minStampState.write(context.timestamp()); + bagTimer.set(context.timestamp()); + } + bagState.add(TimestampedValue.of(context.element().getValue(), context.timestamp())); + } + + @OnTimer(timerIdBagAppend) + public void onTimer( + OnTimerContext context, + @TimerId(timerIdBagAppend) Timer timer, + @StateId(bag) BagState> bagState) { + + List> flush = new ArrayList<>(); + Instant flushTime = context.timestamp(); + for (TimestampedValue val : bagState.read()) { + if (!val.getTimestamp().isAfter(flushTime)) { + flush.add(val); + } + } + flush.sort(Comparator.comparing(TimestampedValue::getTimestamp)); + context.output( + Joiner.on(":").join(flush.stream().map(TimestampedValue::getValue).iterator())); + Instant newMinStamp = flushTime.plus(1); + if (flush.size() < numTestElements) { + timer.set(newMinStamp); + } + } + + @OnTimer(timerIdGc) + public void onTimer( + OnTimerContext context, @StateId(bag) BagState> bagState) { + + String output = + Joiner.on(":") + .join( + StreamSupport.stream(bagState.read().spliterator(), false) + .map(TimestampedValue::getValue) + .iterator()) + + ":cleanup"; + context.output(output); + bagState.clear(); + } + }; + + TestStream.Builder> builder = + TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) + .advanceWatermarkTo(new Instant(0)); + + for (int i = 0; i < numTestElements; i++) { + builder = builder.addElements(TimestampedValue.of(KV.of("dummy", "" + i), now.plus(i))); + builder = builder.advanceWatermarkTo(now.plus(i / 10 * 10)); + } + + TestStream> stream = builder.advanceWatermarkToInfinity(); + + PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); + List expected = + IntStream.rangeClosed(0, numTestElements) + .mapToObj(expandFn(numTestElements)) + .collect(Collectors.toList()); + PAssert.that(output).containsInAnyOrder(expected); + pipeline.run(); + } + + private IntFunction expandFn(int numTestElements) { + return i -> + Joiner.on(":") + .join( + IntStream.rangeClosed(0, Math.min(numTestElements - 1, i)) + .mapToObj(String::valueOf) + .iterator()) + + (i == numTestElements ? ":cleanup" : ""); + } + @Test @Category({ ValidatesRunner.class, From c29d06b2d4bcba8bdc4d6872135e0f1a81e4b087 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Tue, 30 Jul 2019 11:18:47 +0200 Subject: [PATCH 23/57] [BEAM-7520] Synchronize watermarks with bundle processing and add timer pushback --- .../runners/direct/DirectTimerInternals.java | 12 + .../runners/direct/EvaluationContext.java | 9 +- .../beam/runners/direct/QuiescenceDriver.java | 6 +- .../direct/StatefulParDoEvaluatorFactory.java | 26 +- .../direct/WatermarkCallbackExecutor.java | 23 +- .../beam/runners/direct/WatermarkManager.java | 199 +++++++++++++-- .../beam/runners/direct/DirectRunnerTest.java | 230 +++++++++++++++++- .../StatefulParDoEvaluatorFactoryTest.java | 8 + .../runners/direct/WatermarkManagerTest.java | 39 ++- .../apache/beam/sdk/transforms/ParDoTest.java | 182 ++++++++++++-- 10 files changed, 686 insertions(+), 48 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index 63b5008165c20..8f3ab48d312b4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import java.util.stream.StreamSupport; import javax.annotation.Nullable; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.TimerInternals; @@ -80,6 +81,12 @@ public TimerUpdate getTimerUpdate() { return timerUpdateBuilder.build(); } + public boolean containsUpdateForTimeBefore(Instant time) { + TimerUpdate update = timerUpdateBuilder.build(); + return hasTimeBefore(update.getSetTimers(), time) + || hasTimeBefore(update.getDeletedTimers(), time); + } + @Override public Instant currentProcessingTime() { return processingTimeClock.now(); @@ -101,4 +108,9 @@ public Instant currentInputWatermarkTime() { public Instant currentOutputWatermarkTime() { return watermarks.getOutputWatermark(); } + + private boolean hasTimeBefore(Iterable timers, Instant time) { + return StreamSupport.stream(timers.spliterator(), false) + .anyMatch(td -> td.getTimestamp().isBefore(time)); + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index c5ebfaf97835d..22e0a8a4adde4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -140,6 +140,7 @@ public void initialize( * null} if the transform that produced the result is a root transform * @param completedTimers the timers that were delivered to produce the {@code completedBundle}, * or an empty iterable if no timers were delivered + * @param pushedBackTimers timers that have been pushed back during processing * @param result the result of evaluating the input bundle * @return the committed bundles contained within the handled {@code result} */ @@ -226,7 +227,11 @@ private void fireAllAvailableCallbacks() { private void fireAvailableCallbacks(AppliedPTransform producingTransform) { TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform); Instant outputWatermark = watermarks.getOutputWatermark(); - callbackExecutor.fireForWatermark(producingTransform, outputWatermark); + try { + callbackExecutor.fireForWatermark(producingTransform, outputWatermark); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } } /** Create a {@link UncommittedBundle} for use by a source. */ @@ -369,7 +374,7 @@ void forceRefresh() { *

This is a destructive operation. Timers will only appear in the result of this method once * for each time they are set. */ - public Collection>> extractFiredTimers() { + Collection>> extractFiredTimers() { forceRefresh(); return watermarkManager.extractFiredTimers(); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java index 08029977dd6d2..ca0ad612b8fc1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java @@ -249,6 +249,7 @@ private enum ExecutorState { * Exception)}. */ private class TimerIterableCompletionCallback implements CompletionCallback { + private final Iterable timers; TimerIterableCompletionCallback(Iterable timers) { @@ -258,8 +259,9 @@ private class TimerIterableCompletionCallback implements CompletionCallback { @Override public final CommittedResult handleResult( CommittedBundle inputBundle, TransformResult result) { - CommittedResult> committedResult = - evaluationContext.handleResult(inputBundle, timers, result); + + final CommittedResult> committedResult; + committedResult = evaluationContext.handleResult(inputBundle, timers, result); for (CommittedBundle outputBundle : committedResult.getOutputs()) { pendingWork.offer( WorkUpdate.fromBundle( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 366ca05455fb7..e1080e5a8f7b8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -20,10 +20,14 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; +import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.PriorityQueue; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.StateNamespace; @@ -34,6 +38,7 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; +import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -56,6 +61,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LoadingCache; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.joda.time.Instant; /** A {@link TransformEvaluatorFactory} for stateful {@link ParDo}. */ final class StatefulParDoEvaluatorFactory implements TransformEvaluatorFactory { @@ -232,10 +238,13 @@ private static class StatefulParDoEvaluator implements TransformEvaluator>> { private final DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator; + private final List pushedBackTimers = new ArrayList<>(); + private final DirectTimerInternals timerInternals; public StatefulParDoEvaluator( DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator) { this.delegateEvaluator = delegateEvaluator; + this.timerInternals = delegateEvaluator.getParDoEvaluator().getStepContext().timerInternals(); } @Override @@ -245,7 +254,12 @@ public void processElement(WindowedValue>> gbkRes delegateEvaluator.processElement(windowedValue); } - for (TimerData timer : gbkResult.getValue().timersIterable()) { + Instant currentInputWatermark = timerInternals.currentInputWatermarkTime(); + PriorityQueue toBeFiredTimers = + new PriorityQueue<>(Comparator.comparing(TimerData::getTimestamp)); + gbkResult.getValue().timersIterable().forEach(toBeFiredTimers::add); + while (!toBeFiredTimers.isEmpty()) { + TimerData timer = toBeFiredTimers.poll(); checkState( timer.getNamespace() instanceof WindowNamespace, "Expected Timer %s to be in a %s, but got %s", @@ -255,17 +269,23 @@ public void processElement(WindowedValue>> gbkRes WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); BoundedWindow timerWindow = windowNamespace.getWindow(); delegateEvaluator.onTimer(timer, timerWindow); + if (timerInternals.containsUpdateForTimeBefore(currentInputWatermark)) { + break; + } } + pushedBackTimers.addAll(toBeFiredTimers); } @Override public TransformResult>> finishBundle() throws Exception { TransformResult> delegateResult = delegateEvaluator.finishBundle(); - + TimerUpdate timerUpdate = + delegateResult.getTimerUpdate().withPushedBackTimers(pushedBackTimers); + pushedBackTimers.clear(); StepTransformResult.Builder>> regroupedResult = StepTransformResult.>>withHold( delegateResult.getTransform(), delegateResult.getWatermarkHold()) - .withTimerUpdate(delegateResult.getTimerUpdate()) + .withTimerUpdate(timerUpdate) .withState(delegateResult.getState()) .withMetricUpdates(delegateResult.getLogicalMetricUpdates()) .addOutput(Lists.newArrayList(delegateResult.getOutputBundles())); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java index 7f6800e554ba5..1ca90dbf03bef 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java @@ -19,9 +19,12 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; import java.util.PriorityQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import javax.annotation.Nonnull; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -116,14 +119,30 @@ public void callOnWindowExpiration( * Schedule all pending callbacks that must have produced output by the time of the provided * watermark. */ - public void fireForWatermark(AppliedPTransform step, Instant watermark) { + public void fireForWatermark(AppliedPTransform step, Instant watermark) + throws InterruptedException { PriorityQueue callbackQueue = callbacks.get(step); if (callbackQueue == null) { return; } synchronized (callbackQueue) { + List toFire = new ArrayList<>(); while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) { - executor.execute(callbackQueue.poll().getCallback()); + toFire.add(callbackQueue.poll().getCallback()); + } + if (!toFire.isEmpty()) { + CountDownLatch latch = new CountDownLatch(toFire.size()); + toFire.forEach( + r -> + executor.execute( + () -> { + try { + r.run(); + } finally { + latch.countDown(); + } + })); + latch.await(); } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 82dc0ae4416d4..d9e7ac213adbf 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -36,11 +36,17 @@ import java.util.Objects; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Consumer; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -63,7 +69,9 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashBasedTable; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Ordering; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.SortedMultiset; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Table; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.TreeMultiset; @@ -237,13 +245,18 @@ static class AppliedPTransformInputWatermark implements Watermark { // This per-key sorted set allows quick retrieval of timers that should fire for a key private final Map, NavigableSet> objectTimers; - private AtomicReference currentWatermark; + private final AtomicReference currentWatermark; + + private final Consumer timerUpdateNotification; public AppliedPTransformInputWatermark( - String name, Collection inputWatermarks) { - this.name = name; + String name, + Collection inputWatermarks, + Consumer timerUpdateNotification) { + this.name = name; this.inputWatermarks = inputWatermarks; + // The ordering must order elements by timestamp, and must not compare two distinct elements // as equal. This is built on the assumption that any element added as a pending element will // be consumed without modifications. @@ -255,7 +268,8 @@ public AppliedPTransformInputWatermark( this.pendingTimers = TreeMultiset.create(); this.objectTimers = new HashMap<>(); this.existingTimers = new HashMap<>(); - currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); + this.currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); + this.timerUpdateNotification = timerUpdateNotification; } @Override @@ -333,12 +347,15 @@ synchronized void updateTimers(TimerUpdate update) { if (existingTimer == null) { pendingTimers.add(timer); keyTimers.add(timer); - } else if (!existingTimer.equals(timer)) { + } else { + // reinitialize the timer even if identical, + // because it might be removed from objectTimers + // by timer push back pendingTimers.remove(existingTimer); keyTimers.remove(existingTimer); pendingTimers.add(timer); keyTimers.add(timer); - } // else the timer is already set identically, so noop + } existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer); } @@ -364,6 +381,13 @@ synchronized void updateTimers(TimerUpdate update) { pendingTimers.remove(timer); } } + + if (!update.isEmpty()) { + // notify of TimerData update + Iterables.concat( + update.getCompletedTimers(), update.getDeletedTimers(), update.getSetTimers()) + .forEach(timerUpdateNotification); + } } @VisibleForTesting @@ -487,8 +511,13 @@ private static class SynchronizedProcessingTimeInputWatermark implements Waterma private AtomicReference earliestHold; + private final Consumer timerUpdateNotification; + public SynchronizedProcessingTimeInputWatermark( - String name, Collection inputWms) { + String name, + Collection inputWms, + Consumer timerUpdateNotification) { + this.name = name; this.inputWms = inputWms; this.pendingBundles = new HashSet<>(); @@ -500,7 +529,8 @@ public SynchronizedProcessingTimeInputWatermark( for (Watermark wm : inputWms) { initialHold = INSTANT_ORDERING.min(initialHold, wm.get()); } - earliestHold = new AtomicReference<>(initialHold); + this.earliestHold = new AtomicReference<>(initialHold); + this.timerUpdateNotification = timerUpdateNotification; } @Override @@ -619,6 +649,11 @@ private synchronized void updateTimers(TimerUpdate update) { for (TimerData completedTimer : update.completedTimers) { pendingTimers.remove(completedTimer); } + + // notify of TimerData update + Iterables.concat( + update.getCompletedTimers(), update.getDeletedTimers(), update.getSetTimers()) + .forEach(timerUpdateNotification); } private synchronized Map, List> extractFiredDomainTimers( @@ -829,6 +864,14 @@ private static Map, List> extractFiredTimers( @GuardedBy("refreshLock") private final Set pendingRefreshes; + /** + * A set of executables with currently extracted timers, that are to be processed. Note that, due + * to consistency, we can have only single extracted set of timers that are being processed by + * bundle processor at a time. + */ + private final Map> transformsWithAlreadyExtractedTimers = + new ConcurrentHashMap<>(); + /** * Creates a new {@link WatermarkManager}. All watermarks within the newly created {@link * WatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the minimum watermark, @@ -881,13 +924,18 @@ private TransformWatermarks getTransformWatermark(ExecutableT executable) { if (wms == null) { List inputCollectionWatermarks = getInputWatermarks(executable); AppliedPTransformInputWatermark inputWatermark = - new AppliedPTransformInputWatermark(name + ".in", inputCollectionWatermarks); + new AppliedPTransformInputWatermark( + name + ".in", + inputCollectionWatermarks, + timerUpdateConsumer(transformsWithAlreadyExtractedTimers, executable)); AppliedPTransformOutputWatermark outputWatermark = new AppliedPTransformOutputWatermark(name + ".out", inputWatermark); SynchronizedProcessingTimeInputWatermark inputProcessingWatermark = new SynchronizedProcessingTimeInputWatermark( - name + ".inProcessing", getInputProcessingWatermarks(executable)); + name + ".inProcessing", + getInputProcessingWatermarks(executable), + timerUpdateConsumer(transformsWithAlreadyExtractedTimers, executable)); SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark = new SynchronizedProcessingTimeOutputWatermark( name + ".outProcessing", inputProcessingWatermark); @@ -904,6 +952,25 @@ private TransformWatermarks getTransformWatermark(ExecutableT executable) { return wms; } + private static Consumer timerUpdateConsumer( + Map> transformsWithAlreadyExtractedTimers, ExecutableT executable) { + + return update -> { + String timerIdWithNs = TimerUpdate.getTimerIdWithNamespace(update); + transformsWithAlreadyExtractedTimers.compute( + executable, + (k, v) -> { + if (v != null) { + v.remove(timerIdWithNs); + if (v.isEmpty()) { + v = null; + } + } + return v; + }); + }; + } + private Collection getInputProcessingWatermarks(ExecutableT executable) { ImmutableList.Builder inputWmsBuilder = ImmutableList.builder(); Collection inputs = graph.getPerElementInputs(executable); @@ -1122,7 +1189,7 @@ private Set refreshAllOf(Set toRefresh) { return newRefreshes; } - private Set refreshWatermarks(ExecutableT toRefresh) { + private Set refreshWatermarks(final ExecutableT toRefresh) { TransformWatermarks myWatermarks = transformToWatermarks.get(toRefresh); WatermarkUpdate updateResult = myWatermarks.refresh(); if (updateResult.isAdvanced()) { @@ -1145,9 +1212,28 @@ public Collection> extractFiredTimers() { try { for (Map.Entry watermarksEntry : transformToWatermarks.entrySet()) { - Collection> firedTimers = - watermarksEntry.getValue().extractFiredTimers(); - allTimers.addAll(firedTimers); + ExecutableT transform = watermarksEntry.getKey(); + if (!transformsWithAlreadyExtractedTimers.containsKey(transform)) { + TransformWatermarks watermarks = watermarksEntry.getValue(); + Collection> firedTimers = watermarks.extractFiredTimers(); + if (!firedTimers.isEmpty()) { + List newTimers = + firedTimers.stream() + .flatMap(f -> f.getTimers().stream()) + .collect(Collectors.toList()); + transformsWithAlreadyExtractedTimers.compute( + transform, + (k, v) -> { + if (v == null) { + v = new HashSet<>(); + } + final Set toUpdate = v; + newTimers.forEach(td -> toUpdate.add(TimerUpdate.getTimerIdWithNamespace(td))); + return v; + }); + allTimers.addAll(firedTimers); + } + } } return allTimers; } finally { @@ -1264,6 +1350,8 @@ public class TransformWatermarks { private Instant latestSynchronizedInputWm; private Instant latestSynchronizedOutputWm; + private final ReadWriteLock transformWatermarkLock = new ReentrantReadWriteLock(); + private TransformWatermarks( ExecutableT executable, AppliedPTransformInputWatermark inputWatermark, @@ -1318,6 +1406,10 @@ public synchronized Instant getSynchronizedProcessingOutputTime() { return latestSynchronizedOutputWm; } + private ReadWriteLock getWatermarkLock() { + return transformWatermarkLock; + } + private WatermarkUpdate refresh() { inputWatermark.refresh(); synchronizedProcessingInputWatermark.refresh(); @@ -1397,19 +1489,24 @@ public String toString() { * *

setTimers and deletedTimers are collections of {@link TimerData} that have been added to the * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as - * the input to the executed step. + * the input to the executed step. pushedBackTimers are timers that were in completedTimers at the + * input, but were pushed back due to processing constraints. */ public static class TimerUpdate { private final StructuralKey key; private final Iterable completedTimers; - private final Iterable setTimers; private final Iterable deletedTimers; + private final Iterable pushedBackTimers; /** Returns a TimerUpdate for a null key with no timers. */ public static TimerUpdate empty() { return new TimerUpdate( - null, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); + null, + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); } /** @@ -1479,19 +1576,31 @@ public TimerUpdate build() { key, ImmutableList.copyOf(completedTimers), ImmutableList.copyOf(setTimers), - ImmutableList.copyOf(deletedTimers)); + ImmutableList.copyOf(deletedTimers), + Collections.emptyList()); } } + private static Map indexTimerData(Iterable timerData) { + return StreamSupport.stream(timerData.spliterator(), false) + .collect(Collectors.toMap(TimerUpdate::getTimerIdWithNamespace, e -> e, (a, b) -> b)); + } + + private static String getTimerIdWithNamespace(TimerData td) { + return td.getNamespace() + td.getTimerId(); + } + private TimerUpdate( StructuralKey key, Iterable completedTimers, Iterable setTimers, - Iterable deletedTimers) { + Iterable deletedTimers, + Iterable pushedBackTimers) { this.key = key; this.completedTimers = completedTimers; this.setTimers = setTimers; this.deletedTimers = deletedTimers; + this.pushedBackTimers = pushedBackTimers; } @VisibleForTesting @@ -1514,11 +1623,45 @@ public Iterable getDeletedTimers() { return deletedTimers; } + Iterable getPushedBackTimers() { + return pushedBackTimers; + } + + boolean isEmpty() { + return Iterables.isEmpty(completedTimers) + && Iterables.isEmpty(setTimers) + && Iterables.isEmpty(deletedTimers) + && Iterables.isEmpty(pushedBackTimers); + } + /** * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers. + * Note that if any of the completed timers is in pushedBackTimers, then it is set instead. The + * pushedBackTimers are cleared afterwards. */ public TimerUpdate withCompletedTimers(Iterable completedTimers) { - return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers); + List timersToComplete = new ArrayList<>(); + Set pushedBack = Sets.newHashSet(pushedBackTimers); + Map newSetTimers = indexTimerData(setTimers); + for (TimerData td : completedTimers) { + String timerIdWithNs = getTimerIdWithNamespace(td); + if (!pushedBack.contains(td)) { + timersToComplete.add(td); + } else if (!newSetTimers.containsKey(timerIdWithNs)) { + newSetTimers.put(timerIdWithNs, td); + } + } + return new TimerUpdate( + key, timersToComplete, newSetTimers.values(), deletedTimers, Collections.emptyList()); + } + + /** + * Returns a {@link TimerUpdate} that is like this one, but with the pushedBackTimersare removed + * set by provided pushedBackTimers. + */ + public TimerUpdate withPushedBackTimers(Iterable pushedBackTimers) { + return new TimerUpdate( + key, completedTimers, setTimers, deletedTimers, Lists.newArrayList(pushedBackTimers)); } @Override @@ -1537,6 +1680,17 @@ public boolean equals(Object other) { && Objects.equals(this.setTimers, that.setTimers) && Objects.equals(this.deletedTimers, that.deletedTimers); } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("key", key) + .add("setTimers", setTimers) + .add("completedTimers", completedTimers) + .add("deletedTimers", deletedTimers) + .add("pushedBackTimers", pushedBackTimers) + .toString(); + } } /** @@ -1580,7 +1734,10 @@ public Collection getTimers() { @Override public String toString() { - return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString(); + return MoreObjects.toStringHelper(FiredTimers.class) + .add("key", key) + .add("timers", timers) + .toString(); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 9cc5a87a29227..b58cab91a7317 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -34,14 +34,18 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -51,12 +55,14 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -67,19 +73,30 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; 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.Sum; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; @@ -93,9 +110,13 @@ public class DirectRunnerTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); private Pipeline getPipeline() { + return getPipeline(true); + } + + private Pipeline getPipeline(boolean blockOnRun) { PipelineOptions opts = PipelineOptionsFactory.create(); opts.setRunner(DirectRunner.class); - + opts.as(DirectOptions.class).setBlockOnRun(blockOnRun); return Pipeline.create(opts); } @@ -616,6 +637,60 @@ public void processElement(ProcessContext c) { p.run(); } + /** + * Test running of {@link Pipeline} which has two {@link POutput POutputs} and finishing the first + * one triggers data being fed into the second one. + */ + @Test(timeout = 10000) + public void testTwoPOutputsInPipelineWithCascade() throws InterruptedException { + + StaticQueue start = StaticQueue.of("start", VarIntCoder.of()); + StaticQueue messages = StaticQueue.of("messages", VarIntCoder.of()); + + Pipeline pipeline = getPipeline(false); + pipeline.begin().apply("outputStartSignal", outputStartTo(start)); + PCollection result = + pipeline + .apply("processMessages", messages.read()) + .apply( + Window.into(new GlobalWindows()) + .triggering(AfterWatermark.pastEndOfWindow()) + .discardingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(Sum.integersGlobally()); + + // the result should be 6, after the data will have been written + PAssert.that(result).containsInAnyOrder(6); + + PipelineResult run = pipeline.run(); + + // wait until a message has been written to the start queue + while (start.take() == null) {} + + // and publish messages + messages.add(1).add(2).add(3).terminate(); + + run.waitUntilFinish(); + } + + private PTransform outputStartTo(StaticQueue queue) { + return new PTransform() { + @Override + public PDone expand(PBegin input) { + input + .apply(Create.of(1)) + .apply( + MapElements.into(TypeDescriptors.voids()) + .via( + in -> { + queue.add(in); + return null; + })); + return PDone.in(input.getPipeline()); + } + }; + } + /** * Options for testing if {@link DirectRunner} drops {@link PipelineOptions} marked with {@link * JsonIgnore} fields. @@ -684,4 +759,157 @@ public Coder getOutputCoder() { return underlying.getOutputCoder(); } } + + private static class StaticQueue implements Serializable { + + static class StaticQueueSource extends UnboundedSource> { + + static class Checkpoint implements CheckpointMark, Serializable { + + final T read; + + Checkpoint(T read) { + this.read = read; + } + + @Override + public void finalizeCheckpoint() throws IOException { + // nop + } + } + + final StaticQueue queue; + + StaticQueueSource(StaticQueue queue) { + this.queue = queue; + } + + @Override + public List>> split( + int desiredNumSplits, PipelineOptions options) throws Exception { + return Arrays.asList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions po, Checkpoint cmt) { + return new UnboundedReader() { + + T read = cmt == null ? null : cmt.read; + boolean finished = false; + + @Override + public boolean start() throws IOException { + return advance(); + } + + @Override + public boolean advance() throws IOException { + try { + Optional taken = queue.take(); + if (taken.isPresent()) { + read = taken.get(); + return true; + } + finished = true; + return false; + } catch (InterruptedException ex) { + throw new IOException(ex); + } + } + + @Override + public Instant getWatermark() { + if (finished) { + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } + return BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + @Override + public CheckpointMark getCheckpointMark() { + return new Checkpoint(read); + } + + @Override + public UnboundedSource getCurrentSource() { + return StaticQueueSource.this; + } + + @Override + public T getCurrent() throws NoSuchElementException { + return read; + } + + @Override + public Instant getCurrentTimestamp() { + return getWatermark(); + } + + @Override + public void close() throws IOException { + // nop + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public Coder> getCheckpointMarkCoder() { + return (Coder) SerializableCoder.of(Checkpoint.class); + } + + @Override + public Coder getOutputCoder() { + return queue.coder; + } + } + + static final Map> QUEUES = new ConcurrentHashMap<>(); + + static StaticQueue of(String name, Coder coder) { + return new StaticQueue<>(name, coder); + } + + private final String name; + private final Coder coder; + private final transient BlockingQueue> queue = new ArrayBlockingQueue<>(10); + + StaticQueue(String name, Coder coder) { + this.name = name; + this.coder = coder; + Preconditions.checkState( + QUEUES.put(name, this) == null, "Queue " + name + " already exists."); + } + + StaticQueue add(T elem) { + queue.add(Optional.of(elem)); + return this; + } + + @Nullable + Optional take() throws InterruptedException { + return queue.take(); + } + + PTransform> read() { + return new PTransform>() { + @Override + public PCollection expand(PBegin input) { + return input.apply("readFrom:" + name, Read.from(asSource())); + } + }; + } + + UnboundedSource asSource() { + return new StaticQueueSource<>(this); + } + + void terminate() { + queue.add(Optional.empty()); + } + + private Object readResolve() { + return QUEUES.get(name); + } + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 171d9dd1fb0e7..04d03e8363b22 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -42,6 +42,7 @@ import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; +import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -96,6 +97,11 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable { private final transient PipelineOptions options = PipelineOptionsFactory.create(); private final transient StateInternals stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(KEY, null); + private final transient DirectTimerInternals timerInternals = + DirectTimerInternals.create( + MockClock.fromInstant(Instant.now()), + Mockito.mock(TransformWatermarks.class), + TimerUpdate.builder(StructuralKey.of(KEY, StringUtf8Coder.of()))); private static final BundleFactory BUNDLE_FACTORY = ImmutableListBundleFactory.create(); @@ -103,10 +109,12 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable { public transient TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @SuppressWarnings("unchecked") @Before public void setup() { MockitoAnnotations.initMocks(this); when((StateInternals) mockStepContext.stateInternals()).thenReturn(stateInternals); + when(mockStepContext.timerInternals()).thenReturn(timerInternals); when(mockEvaluationContext.createSideInputReader(anyList())) .thenReturn( SideInputContainer.create(mockEvaluationContext, Collections.emptyList()) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 54a5ff6753347..5e9cfc211b69f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; import java.io.Serializable; @@ -1250,6 +1251,19 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { Collections.emptyList(), new Instant(50_000L)); manager.refreshAll(); + assertTrue(manager.extractFiredTimers().isEmpty()); + + // confirm processing of the firstExtracted timers + manager.updateWatermarks( + null, + TimerUpdate.builder(key).withCompletedTimers(firstFired.getTimers()).build(), + graph.getProducer(filtered), + null, + Collections.emptyList(), + new Instant(1000L)); + + manager.refreshAll(); + Collection>> secondFiredTimers = manager.extractFiredTimers(); assertThat(secondFiredTimers, not(emptyIterable())); @@ -1314,6 +1328,18 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { Collections.emptyList(), new Instant(50_000L)); manager.refreshAll(); + assertTrue(manager.extractFiredTimers().isEmpty()); + + manager.updateWatermarks( + null, + TimerUpdate.builder(key).withCompletedTimers(firstFired.getTimers()).build(), + graph.getProducer(filtered), + null, + Collections.emptyList(), + new Instant(1000L)); + + manager.refreshAll(); + Collection>> secondFiredTimers = manager.extractFiredTimers(); assertThat(secondFiredTimers, not(emptyIterable())); @@ -1381,6 +1407,16 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { Collections.emptyList(), new Instant(50_000L)); manager.refreshAll(); + assertTrue(manager.extractFiredTimers().isEmpty()); + + manager.updateWatermarks( + null, + TimerUpdate.builder(key).withCompletedTimers(firstFired.getTimers()).build(), + graph.getProducer(filtered), + null, + Collections.emptyList(), + new Instant(1000L)); + Collection>> secondFiredTimers = manager.extractFiredTimers(); assertThat(secondFiredTimers, not(emptyIterable())); @@ -1497,7 +1533,8 @@ public void inputWatermarkDuplicates() { Watermark mockWatermark = Mockito.mock(Watermark.class); AppliedPTransformInputWatermark underTest = - new AppliedPTransformInputWatermark("underTest", ImmutableList.of(mockWatermark)); + new AppliedPTransformInputWatermark( + "underTest", ImmutableList.of(mockWatermark), update -> {}); // Refresh when(mockWatermark.get()).thenReturn(new Instant(0)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index bddd2fe9805d5..65032841da803 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -55,6 +55,7 @@ import java.util.function.IntFunction; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.LongStream; import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; @@ -63,6 +64,7 @@ import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptions; @@ -108,9 +110,11 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; @@ -3495,7 +3499,7 @@ public void onTimer(OutputReceiver r) { } /** A test makes sure that an event time timers are correctly ordered. */ - @Test(timeout = 20000) + @Test @Category({ ValidatesRunner.class, UsesTimersInParDo.class, @@ -3503,14 +3507,47 @@ public void onTimer(OutputReceiver r) { UsesStatefulParDo.class }) public void testEventTimeTimerOrdering() throws Exception { + final int numTestElements = 100; + final Instant now = new Instant(1500000000000L); + TestStream.Builder> builder = + TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) + .advanceWatermarkTo(new Instant(0)); + + for (int i = 0; i < numTestElements; i++) { + builder = builder.addElements(TimestampedValue.of(KV.of("dummy", "" + i), now.plus(i))); + builder = builder.advanceWatermarkTo(now.plus(i / 10 * 10)); + } + + testEventTimeTimerOrderingWithInputPTransform( + now, numTestElements, builder.advanceWatermarkToInfinity()); + } + + /** A test makes sure that an event time timers are correctly ordered using Create transform. */ + @Test + @Category({ValidatesRunner.class, UsesTimersInParDo.class, UsesStatefulParDo.class}) + public void testEventTimeTimerOrderingWithCreate() throws Exception { + final int numTestElements = 100; + final Instant now = new Instant(1500000000000L); + + List>> elements = new ArrayList<>(); + for (int i = 0; i < numTestElements; i++) { + elements.add(TimestampedValue.of(KV.of("dummy", "" + i), now.plus(i))); + } + + testEventTimeTimerOrderingWithInputPTransform( + now, numTestElements, Create.timestamped(elements)); + } + + private void testEventTimeTimerOrderingWithInputPTransform( + Instant now, + int numTestElements, + PTransform>> transform) + throws Exception { final String timerIdBagAppend = "append"; final String timerIdGc = "gc"; final String bag = "bag"; final String minTimestamp = "minTs"; - - final int numTestElements = 100; - final Instant now = new Instant(1500000000000L); final Instant gcTimerStamp = now.plus(numTestElements + 1); DoFn, String> fn = @@ -3579,6 +3616,7 @@ public void onTimer( Joiner.on(":") .join( StreamSupport.stream(bagState.read().spliterator(), false) + .sorted(Comparator.comparing(TimestampedValue::getTimestamp)) .map(TimestampedValue::getValue) .iterator()) + ":cleanup"; @@ -3587,18 +3625,7 @@ public void onTimer( } }; - TestStream.Builder> builder = - TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .advanceWatermarkTo(new Instant(0)); - - for (int i = 0; i < numTestElements; i++) { - builder = builder.addElements(TimestampedValue.of(KV.of("dummy", "" + i), now.plus(i))); - builder = builder.advanceWatermarkTo(now.plus(i / 10 * 10)); - } - - TestStream> stream = builder.advanceWatermarkToInfinity(); - - PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); + PCollection output = pipeline.apply(transform).apply(ParDo.of(fn)); List expected = IntStream.rangeClosed(0, numTestElements) .mapToObj(expandFn(numTestElements)) @@ -3667,6 +3694,129 @@ public void duplicateTimerSetting() { pipeline.run().waitUntilFinish(); } + + @Test + @Category({ValidatesRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) + public void testTwoTimersSettingEachOther() { + Instant now = new Instant(1500000000000L); + Instant end = now.plus(100); + TestStream> input = + TestStream.create(KvCoder.of(VoidCoder.of(), VoidCoder.of())) + .addElements(KV.of(null, null)) + .advanceWatermarkToInfinity(); + pipeline.apply(TwoTimerTest.of(now, end, input)); + pipeline.run(); + } + + @Test + @Category({ValidatesRunner.class, UsesTimersInParDo.class}) + public void testTwoTimersSettingEachOtherWithCreateAsInput() { + Instant now = new Instant(1500000000000L); + Instant end = now.plus(100); + pipeline.apply(TwoTimerTest.of(now, end, Create.of(KV.of(null, null)))); + pipeline.run(); + } + + private static class TwoTimerTest extends PTransform { + + private static PTransform of( + Instant start, Instant end, PTransform>> input) { + return new TwoTimerTest(start, end, input); + } + + private final Instant start; + private final Instant end; + private final transient PTransform>> inputPTransform; + + public TwoTimerTest( + Instant start, Instant end, PTransform>> input) { + this.start = start; + this.end = end; + this.inputPTransform = input; + } + + @Override + public PDone expand(PBegin input) { + + final String timerName1 = "t1"; + final String timerName2 = "t2"; + final String countStateName = "count"; + PCollection result = + input + .apply(inputPTransform) + .apply( + ParDo.of( + new DoFn, String>() { + + @TimerId(timerName1) + final TimerSpec timerSpec1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @TimerId(timerName2) + final TimerSpec timerSpec2 = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @StateId(countStateName) + final StateSpec> countStateSpec = StateSpecs.value(); + + @ProcessElement + public void processElement( + ProcessContext context, + @TimerId(timerName1) Timer t1, + @TimerId(timerName2) Timer t2, + @StateId(countStateName) ValueState state) { + + state.write(0); + t1.set(start); + // set the t2 timer after end, so that we test that + // timers are correctly ordered in this case + t2.set(end.plus(1)); + } + + @OnTimer(timerName1) + public void onTimer1( + OnTimerContext context, + @TimerId(timerName2) Timer t2, + @StateId(countStateName) ValueState state) { + + Integer current = state.read(); + t2.set(context.timestamp()); + + context.output( + "t1:" + + current + + ":" + + context.timestamp().minus(start.getMillis()).getMillis()); + } + + @OnTimer(timerName2) + public void onTimer2( + OnTimerContext context, + @TimerId(timerName1) Timer t1, + @StateId(countStateName) ValueState state) { + Integer current = state.read(); + if (context.timestamp().isBefore(end)) { + state.write(current + 1); + t1.set(context.timestamp().plus(1)); + } else { + state.write(-1); + } + context.output( + "t2:" + + current + + ":" + + context.timestamp().minus(start.getMillis()).getMillis()); + } + })); + + List expected = + LongStream.rangeClosed(0, 100) + .mapToObj(e -> (Long) e) + .flatMap(e -> Arrays.asList("t1:" + e + ":" + e, "t2:" + e + ":" + e).stream()) + .collect(Collectors.toList()); + PAssert.that(result).containsInAnyOrder(expected); + + return PDone.in(input.getPipeline()); + } + } } /** Tests validating Timer coder inference behaviors. */ From 0049037837e968c6649edcf246fe375d4c47890a Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Fri, 18 Oct 2019 10:55:37 +0200 Subject: [PATCH 24/57] [BEAM-7520] add UsesStrictTimerOrdering category, ignore test for samza and portable flink --- .../flink/job-server/flink_job_server.gradle | 1 + runners/samza/build.gradle | 1 + .../sdk/testing/UsesStrictTimerOrdering.java | 24 +++++++++++++++++++ .../apache/beam/sdk/transforms/ParDoTest.java | 20 ++++++++++++---- 4 files changed, 42 insertions(+), 4 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStrictTimerOrdering.java diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index ee856e791ffee..d6664ccc158a5 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -149,6 +149,7 @@ def portableValidatesRunnerTask(String name, Boolean streaming) { excludeCategories 'org.apache.beam.sdk.testing.UsesSetState' if (streaming) { excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime' + excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering' } else { excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream' } diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index 209db64646f9d..ae6f48fc9ad9f 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -87,6 +87,7 @@ task validatesRunner(type: Test) { excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream' excludeCategories 'org.apache.beam.sdk.testing.UsesMetricsPusher' excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle' + excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering' } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStrictTimerOrdering.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStrictTimerOrdering.java new file mode 100644 index 0000000000000..ad9fda1b3e4b0 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStrictTimerOrdering.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +/** + * Category for tests that enforce strict event-time ordering of fired timers, even in situations + * where multiple tests mutually set one another and watermark hops arbitrarily far to the future. + */ +public @interface UsesStrictTimerOrdering {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 65032841da803..db5733550d519 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -91,6 +91,7 @@ import org.apache.beam.sdk.testing.UsesSideInputs; import org.apache.beam.sdk.testing.UsesSideInputsWithDifferentCoders; import org.apache.beam.sdk.testing.UsesStatefulParDo; +import org.apache.beam.sdk.testing.UsesStrictTimerOrdering; import org.apache.beam.sdk.testing.UsesTestStream; import org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime; import org.apache.beam.sdk.testing.UsesTimersInParDo; @@ -3504,7 +3505,8 @@ public void onTimer(OutputReceiver r) { ValidatesRunner.class, UsesTimersInParDo.class, UsesTestStream.class, - UsesStatefulParDo.class + UsesStatefulParDo.class, + UsesStrictTimerOrdering.class }) public void testEventTimeTimerOrdering() throws Exception { final int numTestElements = 100; @@ -3524,7 +3526,12 @@ public void testEventTimeTimerOrdering() throws Exception { /** A test makes sure that an event time timers are correctly ordered using Create transform. */ @Test - @Category({ValidatesRunner.class, UsesTimersInParDo.class, UsesStatefulParDo.class}) + @Category({ + ValidatesRunner.class, + UsesTimersInParDo.class, + UsesStatefulParDo.class, + UsesStrictTimerOrdering.class + }) public void testEventTimeTimerOrderingWithCreate() throws Exception { final int numTestElements = 100; final Instant now = new Instant(1500000000000L); @@ -3696,7 +3703,12 @@ public void duplicateTimerSetting() { } @Test - @Category({ValidatesRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) + @Category({ + ValidatesRunner.class, + UsesTimersInParDo.class, + UsesTestStream.class, + UsesStrictTimerOrdering.class + }) public void testTwoTimersSettingEachOther() { Instant now = new Instant(1500000000000L); Instant end = now.plus(100); @@ -3709,7 +3721,7 @@ public void testTwoTimersSettingEachOther() { } @Test - @Category({ValidatesRunner.class, UsesTimersInParDo.class}) + @Category({ValidatesRunner.class, UsesTimersInParDo.class, UsesStrictTimerOrdering.class}) public void testTwoTimersSettingEachOtherWithCreateAsInput() { Instant now = new Instant(1500000000000L); Instant end = now.plus(100); From 7392db01c2f64a235784e8a6dc9b2c56572bea6a Mon Sep 17 00:00:00 2001 From: David Moravek Date: Wed, 23 Oct 2019 17:32:23 +0200 Subject: [PATCH 25/57] [BEAM-7730] Add ability to override resources for a specific flink runner version. --- runners/flink/1.7/build.gradle | 2 ++ runners/flink/1.8/build.gradle | 2 ++ runners/flink/1.9/build.gradle | 2 ++ runners/flink/flink_runner.gradle | 14 ++++++++++++++ 4 files changed, 20 insertions(+) diff --git a/runners/flink/1.7/build.gradle b/runners/flink/1.7/build.gradle index 79d5d5c0f6e3b..13c22e5b89a9b 100644 --- a/runners/flink/1.7/build.gradle +++ b/runners/flink/1.7/build.gradle @@ -25,6 +25,8 @@ project.ext { // Version specific code overrides. main_source_overrides = ['./src/main/java'] test_source_overrides = ['./src/test/java'] + main_resources_overrides = [] + test_resources_overrides = [] archives_base_name = 'beam-runners-flink-1.7' } diff --git a/runners/flink/1.8/build.gradle b/runners/flink/1.8/build.gradle index 25106c8e90313..2a05f8cef871e 100644 --- a/runners/flink/1.8/build.gradle +++ b/runners/flink/1.8/build.gradle @@ -25,6 +25,8 @@ project.ext { // Version specific code overrides. main_source_overrides = ["${basePath}/1.7/src/main/java", './src/main/java'] test_source_overrides = ["${basePath}/1.7/src/test/java", './src/test/java'] + main_resources_overrides = [] + test_resources_overrides = [] archives_base_name = 'beam-runners-flink-1.8' } diff --git a/runners/flink/1.9/build.gradle b/runners/flink/1.9/build.gradle index 196919fc4c7f8..3396f0b72149c 100644 --- a/runners/flink/1.9/build.gradle +++ b/runners/flink/1.9/build.gradle @@ -24,6 +24,8 @@ project.ext { // Version specific code overrides. main_source_overrides = ["${basePath}/1.7/src/main/java", "${basePath}/1.8/src/main/java", './src/main/java'] test_source_overrides = ["${basePath}/1.7/src/test/java", "${basePath}/1.8/src/test/java", './src/test/java'] + main_resources_overrides = [] + test_resources_overrides = [] archives_base_name = 'beam-runners-flink-1.9' } diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index 368f4425d4c3a..1ffcaf6c54ead 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -53,6 +53,13 @@ def copySourceOverrides = tasks.register('copySourceOverrides', Copy) { } compileJava.dependsOn copySourceOverrides +def copyResourcesOverrides = tasks.register('copyResourcesOverrides', Copy) { + it.from main_resources_overrides + it.into "${sourceOverridesBase}/main/resources" + it.duplicatesStrategy DuplicatesStrategy.INCLUDE +} +compileJava.dependsOn copyResourcesOverrides + def copyTestSourceOverrides = tasks.register('copyTestSourceOverrides', Copy) { it.from test_source_overrides it.into "${sourceOverridesBase}/test/java" @@ -60,6 +67,13 @@ def copyTestSourceOverrides = tasks.register('copyTestSourceOverrides', Copy) { } compileTestJava.dependsOn copyTestSourceOverrides +def copyTestResourcesOverrides = tasks.register('copyTestResourcesOverrides', Copy) { + it.from test_resources_overrides + it.into "${sourceOverridesBase}/test/resources" + it.duplicatesStrategy DuplicatesStrategy.INCLUDE +} +compileJava.dependsOn copyTestResourcesOverrides + /* * We have to explicitly set all directories here to make sure each * version of Flink has the correct overrides set. From 1a8391da9222ab8d0493b0007bd60bdbeeb5e275 Mon Sep 17 00:00:00 2001 From: Ning Kang Date: Wed, 23 Oct 2019 13:24:55 -0700 Subject: [PATCH 26/57] Merge pull request #9854 from [BEAM-8457] Label Dataflow jobs from Notebook * [BEAM-8457] Label Dataflow jobs from Notebook 1. Changed the pipeline.run() API to allow a runner and an option parameter so that a pipeline initially bundled w/ an interactive runner can be directly run by other runners from notebook. 2. Implicitly added the necessary source information through user labels when the user does p.run(runner=DataflowRunner(), options=options) or DataflowRunner().run_pipeline(p, options). 3. User '--labels' doesn't support character '.' or '"'. When applying version related label, replace '.' w/ '_'. Avoid enclosing any label with '"'. * Avoid changing from_runner_api() and Pipeline() signatures. --- sdks/python/apache_beam/pipeline.py | 48 +++++++++++++++---- .../runners/dataflow/dataflow_runner.py | 10 ++++ .../runners/interactive/interactive_runner.py | 2 +- 3 files changed, 51 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index a776d3065a14e..5574a825e467f 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -171,6 +171,10 @@ def __init__(self, runner=None, options=None, argv=None): # If a transform is applied and the full label is already in the set # then the transform will have to be cloned with a new label. self.applied_labels = set() + # A boolean value indicating whether the pipeline is created in an + # interactive environment such as interactive notebooks. Initialized as + # None. The value is set ad hoc when `pipeline.run()` is invoked. + self.interactive = None @property @deprecated(since='First stable release', @@ -395,28 +399,56 @@ def replace_all(self, replacements): for override in replacements: self._check_replacement(override) - def run(self, test_runner_api=True): - """Runs the pipeline. Returns whatever our runner returns after running.""" - + def run(self, test_runner_api=True, runner=None, options=None, + interactive=None): + """Runs the pipeline. Returns whatever our runner returns after running. + + If another runner instance and options are provided, that runner will + execute the pipeline with the given options. If either of them is not set, + a ValueError is raised. The usage is similar to directly invoking + `runner.run_pipeline(pipeline, options)`. + Additionally, an interactive field can be set to override the pipeline's + self.interactive field to mark current pipeline as being initiated from an + interactive environment. + """ + from apache_beam.runners.interactive import interactive_runner + if interactive: + self.interactive = interactive + elif isinstance(self.runner, interactive_runner.InteractiveRunner): + self.interactive = True + else: + self.interactive = False + runner_in_use = self.runner + options_in_use = self._options + if runner and options: + runner_in_use = runner + options_in_use = options + elif not runner and options: + raise ValueError('Parameter runner is not given when parameter options ' + 'is given.') + elif not options and runner: + raise ValueError('Parameter options is not given when parameter runner ' + 'is given.') # When possible, invoke a round trip through the runner API. if test_runner_api and self._verify_runner_api_compatible(): return Pipeline.from_runner_api( self.to_runner_api(use_fake_coders=True), - self.runner, - self._options).run(False) + runner_in_use, + options_in_use).run(test_runner_api=False, + interactive=self.interactive) - if self._options.view_as(TypeOptions).runtime_type_check: + if options_in_use.view_as(TypeOptions).runtime_type_check: from apache_beam.typehints import typecheck self.visit(typecheck.TypeCheckVisitor()) - if self._options.view_as(SetupOptions).save_main_session: + if options_in_use.view_as(SetupOptions).save_main_session: # If this option is chosen, verify we can pickle the main session early. tmpdir = tempfile.mkdtemp() try: pickler.dump_session(os.path.join(tmpdir, 'main_session.pickle')) finally: shutil.rmtree(tmpdir) - return self.runner.run_pipeline(self, self._options) + return runner_in_use.run_pipeline(self, options_in_use) def __enter__(self): return self diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 4928550143d69..f57be74687c94 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -364,6 +364,16 @@ def visit_transform(self, transform_node): def run_pipeline(self, pipeline, options): """Remotely executes entire pipeline or parts reachable from node.""" + # Label goog-dataflow-notebook if pipeline is initiated from interactive + # runner. + if pipeline.interactive: + notebook_version = ('goog-dataflow-notebook=' + + beam.version.__version__.replace('.', '_')) + if options.view_as(GoogleCloudOptions).labels: + options.view_as(GoogleCloudOptions).labels.append(notebook_version) + else: + options.view_as(GoogleCloudOptions).labels = [notebook_version] + # Import here to avoid adding the dependency for local running scenarios. try: # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner.py b/sdks/python/apache_beam/runners/interactive/interactive_runner.py index 94c0de725d848..56a3c180fd7c2 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_runner.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_runner.py @@ -146,7 +146,7 @@ def run_pipeline(self, pipeline, options): cache_manager=self._cache_manager, pipeline_graph_renderer=self._renderer) display.start_periodic_update() - result = pipeline_to_execute.run() + result = pipeline_to_execute.run(interactive=True) result.wait_until_finish() display.stop_periodic_update() From a116aed095af66ce3a4a811610daa91ec1d759f9 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Wed, 23 Oct 2019 11:03:37 -0700 Subject: [PATCH 27/57] [Go SDK] Fix Logging call depths. --- .../pkg/beam/core/runtime/harness/logging.go | 2 +- sdks/go/pkg/beam/log/log.go | 36 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging.go b/sdks/go/pkg/beam/core/runtime/harness/logging.go index 63afe7c96569e..690589adc4cf1 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/logging.go +++ b/sdks/go/pkg/beam/core/runtime/harness/logging.go @@ -61,7 +61,7 @@ func (l *logger) Log(ctx context.Context, sev log.Severity, calldepth int, msg s Severity: convertSeverity(sev), Message: msg, } - if _, file, line, ok := runtime.Caller(calldepth); ok { + if _, file, line, ok := runtime.Caller(calldepth + 1); ok { entry.LogLocation = fmt.Sprintf("%v:%v", file, line) } if id, ok := tryGetInstID(ctx); ok { diff --git a/sdks/go/pkg/beam/log/log.go b/sdks/go/pkg/beam/log/log.go index 70cd199c07c57..0bf0740a55b92 100644 --- a/sdks/go/pkg/beam/log/log.go +++ b/sdks/go/pkg/beam/log/log.go @@ -68,80 +68,80 @@ func Output(ctx context.Context, sev Severity, calldepth int, msg string) { // Debug writes the fmt.Sprint-formatted arguments to the global logger with // debug severity. func Debug(ctx context.Context, v ...interface{}) { - Output(ctx, SevDebug, 2, fmt.Sprint(v...)) + Output(ctx, SevDebug, 1, fmt.Sprint(v...)) } // Debugf writes the fmt.Sprintf-formatted arguments to the global logger with // debug severity. func Debugf(ctx context.Context, format string, v ...interface{}) { - Output(ctx, SevDebug, 2, fmt.Sprintf(format, v...)) + Output(ctx, SevDebug, 1, fmt.Sprintf(format, v...)) } // Debugln writes the fmt.Sprintln-formatted arguments to the global logger with // debug severity. func Debugln(ctx context.Context, v ...interface{}) { - Output(ctx, SevDebug, 2, fmt.Sprintln(v...)) + Output(ctx, SevDebug, 1, fmt.Sprintln(v...)) } // Info writes the fmt.Sprint-formatted arguments to the global logger with // info severity. func Info(ctx context.Context, v ...interface{}) { - Output(ctx, SevInfo, 2, fmt.Sprint(v...)) + Output(ctx, SevInfo, 1, fmt.Sprint(v...)) } // Infof writes the fmt.Sprintf-formatted arguments to the global logger with // info severity. func Infof(ctx context.Context, format string, v ...interface{}) { - Output(ctx, SevInfo, 2, fmt.Sprintf(format, v...)) + Output(ctx, SevInfo, 1, fmt.Sprintf(format, v...)) } // Infoln writes the fmt.Sprintln-formatted arguments to the global logger with // info severity. func Infoln(ctx context.Context, v ...interface{}) { - Output(ctx, SevInfo, 2, fmt.Sprintln(v...)) + Output(ctx, SevInfo, 1, fmt.Sprintln(v...)) } // Warn writes the fmt.Sprint-formatted arguments to the global logger with // warn severity. func Warn(ctx context.Context, v ...interface{}) { - Output(ctx, SevWarn, 2, fmt.Sprint(v...)) + Output(ctx, SevWarn, 1, fmt.Sprint(v...)) } // Warnf writes the fmt.Sprintf-formatted arguments to the global logger with // warn severity. func Warnf(ctx context.Context, format string, v ...interface{}) { - Output(ctx, SevWarn, 2, fmt.Sprintf(format, v...)) + Output(ctx, SevWarn, 1, fmt.Sprintf(format, v...)) } // Warnln writes the fmt.Sprintln-formatted arguments to the global logger with // warn severity. func Warnln(ctx context.Context, v ...interface{}) { - Output(ctx, SevWarn, 2, fmt.Sprintln(v...)) + Output(ctx, SevWarn, 1, fmt.Sprintln(v...)) } // Error writes the fmt.Sprint-formatted arguments to the global logger with // error severity. func Error(ctx context.Context, v ...interface{}) { - Output(ctx, SevError, 2, fmt.Sprint(v...)) + Output(ctx, SevError, 1, fmt.Sprint(v...)) } // Errorf writes the fmt.Sprintf-formatted arguments to the global logger with // error severity. func Errorf(ctx context.Context, format string, v ...interface{}) { - Output(ctx, SevError, 2, fmt.Sprintf(format, v...)) + Output(ctx, SevError, 1, fmt.Sprintf(format, v...)) } // Errorln writes the fmt.Sprintln-formatted arguments to the global logger with // error severity. func Errorln(ctx context.Context, v ...interface{}) { - Output(ctx, SevError, 2, fmt.Sprintln(v...)) + Output(ctx, SevError, 1, fmt.Sprintln(v...)) } // Fatal writes the fmt.Sprint-formatted arguments to the global logger with // fatal severity. It then panics. func Fatal(ctx context.Context, v ...interface{}) { msg := fmt.Sprint(v...) - Output(ctx, SevFatal, 2, msg) + Output(ctx, SevFatal, 1, msg) panic(msg) } @@ -149,7 +149,7 @@ func Fatal(ctx context.Context, v ...interface{}) { // fatal severity. It then panics. func Fatalf(ctx context.Context, format string, v ...interface{}) { msg := fmt.Sprintf(format, v...) - Output(ctx, SevFatal, 2, msg) + Output(ctx, SevFatal, 1, msg) panic(msg) } @@ -157,27 +157,27 @@ func Fatalf(ctx context.Context, format string, v ...interface{}) { // fatal severity. It then panics. func Fatalln(ctx context.Context, v ...interface{}) { msg := fmt.Sprintln(v...) - Output(ctx, SevFatal, 2, msg) + Output(ctx, SevFatal, 1, msg) panic(msg) } // Exit writes the fmt.Sprint-formatted arguments to the global logger with // fatal severity. It then exits. func Exit(ctx context.Context, v ...interface{}) { - Output(ctx, SevFatal, 2, fmt.Sprint(v...)) + Output(ctx, SevFatal, 1, fmt.Sprint(v...)) os.Exit(1) } // Exitf writes the fmt.Sprintf-formatted arguments to the global logger with // fatal severity. It then exits. func Exitf(ctx context.Context, format string, v ...interface{}) { - Output(ctx, SevFatal, 2, fmt.Sprintf(format, v...)) + Output(ctx, SevFatal, 1, fmt.Sprintf(format, v...)) os.Exit(1) } // Exitln writes the fmt.Sprintln-formatted arguments to the global logger with // fatal severity. It then exits. func Exitln(ctx context.Context, v ...interface{}) { - Output(ctx, SevFatal, 2, fmt.Sprintln(v...)) + Output(ctx, SevFatal, 1, fmt.Sprintln(v...)) os.Exit(1) } From c211d368a57f393a26f0a0ab3ae10c4b15a0066f Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Wed, 23 Oct 2019 11:09:30 -0700 Subject: [PATCH 28/57] [Go SDK] Track failed instructions in harness. --- .../pkg/beam/core/runtime/harness/datamgr.go | 18 ++-- .../pkg/beam/core/runtime/harness/harness.go | 94 +++++++++++-------- .../pkg/beam/core/runtime/harness/logging.go | 2 +- .../pkg/beam/core/runtime/harness/statemgr.go | 12 +-- 4 files changed, 72 insertions(+), 54 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 453cf9fbb549e..b8c136a2d13de 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -36,7 +36,7 @@ const ( // The indirection makes it easier to control access. type ScopedDataManager struct { mgr *DataChannelManager - instID string + instID instructionID // TODO(herohde) 7/20/2018: capture and force close open reads/writes. However, // we would need the underlying Close to be idempotent or a separate method. @@ -45,10 +45,11 @@ type ScopedDataManager struct { } // NewScopedDataManager returns a ScopedDataManager for the given instruction. -func NewScopedDataManager(mgr *DataChannelManager, instID string) *ScopedDataManager { +func NewScopedDataManager(mgr *DataChannelManager, instID instructionID) *ScopedDataManager { return &ScopedDataManager{mgr: mgr, instID: instID} } +// OpenRead opens an io.ReadCloser on the given stream. func (s *ScopedDataManager) OpenRead(ctx context.Context, id exec.StreamID) (io.ReadCloser, error) { ch, err := s.open(ctx, id.Port) if err != nil { @@ -57,6 +58,7 @@ func (s *ScopedDataManager) OpenRead(ctx context.Context, id exec.StreamID) (io. return ch.OpenRead(ctx, id.PtransformID, s.instID), nil } +// OpenWrite opens an io.WriteCloser on the given stream. func (s *ScopedDataManager) OpenWrite(ctx context.Context, id exec.StreamID) (io.WriteCloser, error) { ch, err := s.open(ctx, id.Port) if err != nil { @@ -119,7 +121,7 @@ func (m *DataChannelManager) Open(ctx context.Context, port exec.Port) (*DataCha // clientID identifies a client of a connected channel. type clientID struct { ptransformID string - instID string + instID instructionID } // This is a reduced version of the full gRPC interface to help with testing. @@ -169,11 +171,11 @@ func makeDataChannel(ctx context.Context, id string, client dataClient) *DataCha return ret } -func (c *DataChannel) OpenRead(ctx context.Context, ptransformID string, instID string) io.ReadCloser { +func (c *DataChannel) OpenRead(ctx context.Context, ptransformID string, instID instructionID) io.ReadCloser { return c.makeReader(ctx, clientID{ptransformID: ptransformID, instID: instID}) } -func (c *DataChannel) OpenWrite(ctx context.Context, ptransformID string, instID string) io.WriteCloser { +func (c *DataChannel) OpenWrite(ctx context.Context, ptransformID string, instID instructionID) io.WriteCloser { return c.makeWriter(ctx, clientID{ptransformID: ptransformID, instID: instID}) } @@ -198,7 +200,7 @@ func (c *DataChannel) read(ctx context.Context) { // to reduce lock contention. for _, elm := range msg.GetData() { - id := clientID{ptransformID: elm.TransformId, instID: elm.GetInstructionId()} + id := clientID{ptransformID: elm.TransformId, instID: instructionID(elm.GetInstructionId())} // log.Printf("Chan read (%v): %v\n", sid, elm.GetData()) @@ -333,7 +335,7 @@ func (w *dataWriter) Close() error { msg := &pb.Elements{ Data: []*pb.Elements_Data{ { - InstructionId: w.id.instID, + InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, // Empty data == sentinel }, @@ -357,7 +359,7 @@ func (w *dataWriter) Flush() error { msg := &pb.Elements{ Data: []*pb.Elements_Data{ { - InstructionId: w.id.instID, + InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, Data: w.buf, }, diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index dcc7922098bf7..28bdf4c4cb0a0 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -82,8 +82,9 @@ func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error { }() ctrl := &control{ - plans: make(map[string]*exec.Plan), - active: make(map[string]*exec.Plan), + plans: make(map[bundleDescriptorID]*exec.Plan), + active: make(map[instructionID]*exec.Plan), + failed: make(map[instructionID]error), data: &DataChannelManager{}, state: &StateChannelManager{}, } @@ -132,12 +133,17 @@ func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error { } } +type bundleDescriptorID string +type instructionID string + type control struct { // plans that are candidates for execution. - plans map[string]*exec.Plan // protected by mu + plans map[bundleDescriptorID]*exec.Plan // protected by mu // plans that are actively being executed. // a plan can only be in one of these maps at any time. - active map[string]*exec.Plan // protected by mu + active map[instructionID]*exec.Plan // protected by mu + // plans that have failed during execution + failed map[instructionID]error // protected by mu mu sync.Mutex data *DataChannelManager @@ -145,8 +151,8 @@ type control struct { } func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRequest) *fnpb.InstructionResponse { - id := req.GetInstructionId() - ctx = setInstID(ctx, id) + instID := instructionID(req.GetInstructionId()) + ctx = setInstID(ctx, instID) switch { case req.GetRegister() != nil: @@ -155,19 +161,19 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe for _, desc := range msg.GetProcessBundleDescriptor() { p, err := exec.UnmarshalPlan(desc) if err != nil { - return fail(id, "Invalid bundle desc: %v", err) + return fail(ctx, instID, "Invalid bundle desc: %v", err) } - pid := desc.GetId() - log.Debugf(ctx, "Plan %v: %v", pid, p) + bdID := bundleDescriptorID(desc.GetId()) + log.Debugf(ctx, "Plan %v: %v", bdID, p) c.mu.Lock() - c.plans[pid] = p + c.plans[bdID] = p c.mu.Unlock() } return &fnpb.InstructionResponse{ - InstructionId: id, + InstructionId: string(instID), Response: &fnpb.InstructionResponse_Register{ Register: &fnpb.RegisterResponse{}, }, @@ -178,40 +184,43 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe // NOTE: the harness sends a 0-length process bundle request to sources (changed?) - log.Debugf(ctx, "PB: %v", msg) - - ref := msg.GetProcessBundleDescriptorId() + bdID := bundleDescriptorID(msg.GetProcessBundleDescriptorId()) + log.Debugf(ctx, "PB [%v]: %v", instID, msg) c.mu.Lock() - plan, ok := c.plans[ref] + plan, ok := c.plans[bdID] // Make the plan active, and remove it from candidates // since a plan can't be run concurrently. - c.active[id] = plan - delete(c.plans, ref) + c.active[instID] = plan + delete(c.plans, bdID) c.mu.Unlock() if !ok { - return fail(id, "execution plan for %v not found", ref) + return fail(ctx, instID, "execution plan for %v not found", bdID) } - data := NewScopedDataManager(c.data, id) - state := NewScopedStateReader(c.state, id) - err := plan.Execute(ctx, id, exec.DataContext{Data: data, State: state}) + data := NewScopedDataManager(c.data, instID) + state := NewScopedStateReader(c.state, instID) + err := plan.Execute(ctx, string(instID), exec.DataContext{Data: data, State: state}) data.Close() state.Close() m := plan.Metrics() // Move the plan back to the candidate state c.mu.Lock() - c.plans[plan.ID()] = plan - delete(c.active, id) + // Mark the instruction as failed. + if err != nil { + c.failed[instID] = err + } + c.plans[bdID] = plan + delete(c.active, instID) c.mu.Unlock() if err != nil { - return fail(id, "execute failed: %v", err) + return fail(ctx, instID, "process bundle failed for instruction %v using plan %v : %v", instID, bdID, err) } return &fnpb.InstructionResponse{ - InstructionId: id, + InstructionId: string(instID), Response: &fnpb.InstructionResponse_ProcessBundle{ ProcessBundle: &fnpb.ProcessBundleResponse{ Metrics: m, @@ -222,20 +231,22 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe case req.GetProcessBundleProgress() != nil: msg := req.GetProcessBundleProgress() - // log.Debugf(ctx, "PB Progress: %v", msg) - - ref := msg.GetInstructionId() + ref := instructionID(msg.GetInstructionId()) c.mu.Lock() plan, ok := c.active[ref] + err := c.failed[ref] c.mu.Unlock() + if err != nil { + return fail(ctx, instID, "failed to return progress: instruction %v failed: %v", ref, err) + } if !ok { - return fail(id, "execution plan for %v not found", ref) + return fail(ctx, instID, "failed to return progress: instruction %v not active", ref) } m := plan.Metrics() return &fnpb.InstructionResponse{ - InstructionId: id, + InstructionId: string(instID), Response: &fnpb.InstructionResponse_ProcessBundleProgress{ ProcessBundleProgress: &fnpb.ProcessBundleProgressResponse{ Metrics: m, @@ -247,27 +258,31 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe msg := req.GetProcessBundleSplit() log.Debugf(ctx, "PB Split: %v", msg) - ref := msg.GetInstructionId() + ref := instructionID(msg.GetInstructionId()) c.mu.Lock() plan, ok := c.active[ref] + err := c.failed[ref] c.mu.Unlock() + if err != nil { + return fail(ctx, instID, "failed to split: instruction %v failed: %v", ref, err) + } if !ok { - return fail(id, "execution plan for %v not found", ref) + return fail(ctx, instID, "failed to split: execution plan for %v not active", ref) } // Get the desired splits for the root FnAPI read operation. ds := msg.GetDesiredSplits()[plan.SourcePTransformID()] if ds == nil { - return fail(id, "failed to split: desired splits for root was empty.") + return fail(ctx, instID, "failed to split: desired splits for root of %v was empty.", ref) } - split, err := plan.Split(exec.SplitPoints{ds.GetAllowedSplitPoints(), ds.GetFractionOfRemainder()}) + split, err := plan.Split(exec.SplitPoints{Splits: ds.GetAllowedSplitPoints(), Frac: ds.GetFractionOfRemainder()}) if err != nil { - return fail(id, "unable to split: %v", err) + return fail(ctx, instID, "unable to split %v: %v", ref, err) } return &fnpb.InstructionResponse{ - InstructionId: id, + InstructionId: string(instID), Response: &fnpb.InstructionResponse_ProcessBundleSplit{ ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{ ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{ @@ -281,15 +296,16 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe } default: - return fail(id, "Unexpected request: %v", req) + return fail(ctx, instID, "Unexpected request: %v", req) } } -func fail(id, format string, args ...interface{}) *fnpb.InstructionResponse { +func fail(ctx context.Context, id instructionID, format string, args ...interface{}) *fnpb.InstructionResponse { + log.Output(ctx, log.SevError, 1, fmt.Sprintf(format, args...)) dummy := &fnpb.InstructionResponse_Register{Register: &fnpb.RegisterResponse{}} return &fnpb.InstructionResponse{ - InstructionId: id, + InstructionId: string(id), Error: fmt.Sprintf(format, args...), Response: dummy, } diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging.go b/sdks/go/pkg/beam/core/runtime/harness/logging.go index 690589adc4cf1..7abc5462758a1 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/logging.go +++ b/sdks/go/pkg/beam/core/runtime/harness/logging.go @@ -37,7 +37,7 @@ type contextKey string const instKey contextKey = "beam:inst" -func setInstID(ctx context.Context, id string) context.Context { +func setInstID(ctx context.Context, id instructionID) context.Context { return context.WithValue(ctx, instKey, id) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go index ff20ff9e9db5d..966988834e3b0 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go @@ -34,7 +34,7 @@ import ( // for side input use. The indirection makes it easier to control access. type ScopedStateReader struct { mgr *StateChannelManager - instID string + instID instructionID opened []io.Closer // track open readers to force close all closed bool @@ -42,7 +42,7 @@ type ScopedStateReader struct { } // NewScopedStateReader returns a ScopedStateReader for the given instruction. -func NewScopedStateReader(mgr *StateChannelManager, instID string) *ScopedStateReader { +func NewScopedStateReader(mgr *StateChannelManager, instID instructionID) *ScopedStateReader { return &ScopedStateReader{mgr: mgr, instID: instID} } @@ -103,7 +103,7 @@ func (s *ScopedStateReader) Close() error { } type stateKeyReader struct { - instID string + instID instructionID key *pb.StateKey token []byte @@ -115,7 +115,7 @@ type stateKeyReader struct { mu sync.Mutex } -func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, instID string, k, w []byte) *stateKeyReader { +func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, instID instructionID, k, w []byte) *stateKeyReader { key := &pb.StateKey{ Type: &pb.StateKey_MultimapSideInput_{ MultimapSideInput: &pb.StateKey_MultimapSideInput{ @@ -133,7 +133,7 @@ func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, } } -func newRunnerReader(ch *StateChannel, instID string, k []byte) *stateKeyReader { +func newRunnerReader(ch *StateChannel, instID instructionID, k []byte) *stateKeyReader { key := &pb.StateKey{ Type: &pb.StateKey_Runner_{ Runner: &pb.StateKey_Runner{ @@ -166,7 +166,7 @@ func (r *stateKeyReader) Read(buf []byte) (int, error) { req := &pb.StateRequest{ // Id: set by channel - InstructionId: r.instID, + InstructionId: string(r.instID), StateKey: r.key, Request: &pb.StateRequest_Get{ Get: &pb.StateGetRequest{ From 8e574a177794aa04345cf8cd0c2ce1717ee40ec6 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Wed, 23 Oct 2019 11:19:59 -0700 Subject: [PATCH 29/57] [Go SDK] Fix log on failed data channel read. --- sdks/go/pkg/beam/core/runtime/harness/datamgr.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index b8c136a2d13de..61ee841cf2498 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -79,6 +79,7 @@ func (s *ScopedDataManager) open(ctx context.Context, port exec.Port) (*DataChan return local.Open(ctx, port) // don't hold lock over potentially slow operation } +// Close prevents new IO for this instruction. func (s *ScopedDataManager) Close() error { s.mu.Lock() s.closed = true @@ -171,10 +172,12 @@ func makeDataChannel(ctx context.Context, id string, client dataClient) *DataCha return ret } +// OpenRead returns an io.ReadCloser of the data elements for the given instruction and ptransform. func (c *DataChannel) OpenRead(ctx context.Context, ptransformID string, instID instructionID) io.ReadCloser { return c.makeReader(ctx, clientID{ptransformID: ptransformID, instID: instID}) } +// OpenWrite returns an io.WriteCloser of the data elements for the given instruction and ptransform. func (c *DataChannel) OpenWrite(ctx context.Context, ptransformID string, instID instructionID) io.WriteCloser { return c.makeWriter(ctx, clientID{ptransformID: ptransformID, instID: instID}) } @@ -189,7 +192,7 @@ func (c *DataChannel) read(ctx context.Context) { log.Warnf(ctx, "DataChannel.read %v closed", c.id) return } - log.Errorf(ctx, "DataChannel.read %v bad", c.id) + log.Errorf(ctx, "DataChannel.read %v bad: %v", c.id, err) return } From 06fa1cc2a92a6f4f0e05f162ee2eb78b072920d9 Mon Sep 17 00:00:00 2001 From: Udi Meiri Date: Wed, 23 Oct 2019 17:49:31 -0700 Subject: [PATCH 30/57] [BEAM-7981] Fix double iterable stripping (#9708) ... in CallableWrapperDoFn Relax strip_iterable exception in wrapper to warning. Existing uses ParDo/Map(fn) might be already decorating the fn with @with_output_types(t), where t is not iterable. The original code accepted that, so I prefer not to break existing pipelines despite possible bugs. Also updated some docstrings. - Change strip_iterable to not modify self. - Change to raise exception if DoFn.process has PEP 484 type hints and its output type is not iterable. --- sdks/python/apache_beam/transforms/core.py | 29 +++++++----------- .../apache_beam/typehints/decorators.py | 10 +++++-- .../typehints/typed_pipeline_test_py3.py | 30 ++++++++++++------- .../python/apache_beam/typehints/typehints.py | 3 +- 4 files changed, 40 insertions(+), 32 deletions(-) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 306de6aa5938e..8733fee1cf460 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -583,10 +583,9 @@ def default_type_hints(self): fn_type_hints = typehints.decorators.IOTypeHints.from_callable(self.process) if fn_type_hints is not None: try: - fn_type_hints.strip_iterable() + return fn_type_hints.strip_iterable() except ValueError as e: raise ValueError('Return value not iterable: %s: %s' % (self, e)) - return fn_type_hints # TODO(sourabhbajaj): Do we want to remove the responsibility of these from # the DoFn or maybe the runner @@ -676,22 +675,14 @@ def __repr__(self): def default_type_hints(self): fn_type_hints = typehints.decorators.IOTypeHints.from_callable(self._fn) - if fn_type_hints is not None: - try: - fn_type_hints.strip_iterable() - except ValueError as e: - raise ValueError('Return value not iterable: %s: %s' % (self._fn, e)) type_hints = get_type_hints(self._fn).with_defaults(fn_type_hints) - # If the fn was a DoFn annotated with a type-hint that hinted a return - # type compatible with Iterable[Any], then we strip off the outer - # container type due to the 'flatten' portion of FlatMap. - # TODO(robertwb): Should we require an iterable specification for FlatMap? - if type_hints.output_types: - args, kwargs = type_hints.output_types - if len(args) == 1 and is_consistent_with( - args[0], typehints.Iterable[typehints.Any]): - type_hints = type_hints.copy() - type_hints.set_output_types(element_type(args[0]), **kwargs) + # The fn's output type should be iterable. Strip off the outer + # container type due to the 'flatten' portion of FlatMap/ParDo. + try: + type_hints = type_hints.strip_iterable() + except ValueError as e: + # TODO(BEAM-8466): Raise exception here if using stricter type checking. + logging.warning('%s: %s', self.display_data()['fn'].value, e) return type_hints def infer_output_type(self, input_type): @@ -1099,8 +1090,8 @@ class ParDo(PTransformWithSideInputs): Args: pcoll (~apache_beam.pvalue.PCollection): a :class:`~apache_beam.pvalue.PCollection` to be processed. - fn (DoFn): a :class:`DoFn` object to be applied to each element - of **pcoll** argument. + fn (`typing.Union[DoFn, typing.Callable]`): a :class:`DoFn` object to be + applied to each element of **pcoll** argument, or a Callable. *args: positional arguments passed to the :class:`DoFn` object. **kwargs: keyword arguments passed to the :class:`DoFn` object. diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py index 218e3b1ef0197..aab45d947eacd 100644 --- a/sdks/python/apache_beam/typehints/decorators.py +++ b/sdks/python/apache_beam/typehints/decorators.py @@ -294,16 +294,22 @@ def strip_iterable(self): """Removes outer Iterable (or equivalent) from output type. Only affects instances with simple output types, otherwise is a no-op. + Does not modify self. Example: Generator[Tuple(int, int)] becomes Tuple(int, int) + Returns: + A possible copy of this instance with a possibly different output type. + Raises: ValueError if output type is simple and not iterable. """ if not self.has_simple_output_type(): - return + return self yielded_type = typehints.get_yielded_type(self.output_types[0][0]) - self.output_types = ((yielded_type,), {}) + res = self.copy() + res.output_types = ((yielded_type,), {}) + return res def copy(self): return IOTypeHints(self.input_types, self.output_types) diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py index 640ca2b89aeb3..2c23dcb4d1ae1 100644 --- a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py +++ b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py @@ -76,8 +76,8 @@ def test_typed_callable_instance(self): # Type hints applied to ParDo instance take precedence over callable # decorators and annotations. @typehints.with_input_types(typehints.Tuple[int, int]) - @typehints.with_output_types(int) - def do_fn(element: typehints.Tuple[int, int]) -> typehints.Generator[int]: + @typehints.with_output_types(typehints.Generator[int]) + def do_fn(element: typehints.Tuple[int, int]) -> typehints.Generator[str]: yield str(element) pardo = beam.ParDo(do_fn).with_input_types(int).with_output_types(str) @@ -92,10 +92,8 @@ def do_fn(element: typehints.Tuple[int, int]) -> typehints.Generator[int]: r'requires.*int.*got.*str'): _ = [1, 2, 3] | (pardo | 'again' >> pardo) - @unittest.skip('BEAM-7981: Iterable in output type should not be removed.') def test_typed_callable_iterable_output(self): - # TODO(BEAM-7981): Both Iterables get stripped in - # CallableWrapperDoFn.default_type_hints, but only one should. + # Only the outer Iterable should be stripped. def do_fn(element: int) -> typehints.Iterable[typehints.Iterable[str]]: return [[str(element)] * 2] @@ -111,10 +109,11 @@ def process(self, element: int) -> str: _ = [1, 2, 3] | beam.ParDo(MyDoFn()) def test_typed_callable_not_iterable(self): - def do_fn(element: typehints.Tuple[int, int]) -> int: - return element[0] - with self.assertRaisesRegex(ValueError, r'int.*is not iterable'): + def do_fn(element: int) -> int: + return [element] # Return a list to not fail the pipeline. + with self.assertLogs() as cm: _ = [1, 2, 3] | beam.ParDo(do_fn) + self.assertRegex(''.join(cm.output), r'int.*is not iterable') def test_typed_dofn_kwonly(self): class MyDoFn(beam.DoFn): @@ -163,7 +162,7 @@ class MyDoFn(beam.DoFn): def process(self, element: int) -> str: return str(element) - with self.assertRaisesRegex(ValueError, r'Return value not iterable'): + with self.assertRaisesRegex(ValueError, r'str.*is not iterable'): _ = beam.ParDo(MyDoFn()).get_type_hints() def test_pardo_wrapper(self): @@ -174,12 +173,23 @@ def do_fn(element: int) -> typehints.Iterable[str]: self.assertEqual(th.input_types, ((int,), {})) self.assertEqual(th.output_types, ((str,), {})) + def test_pardo_wrapper_tuple(self): + # Test case for callables that return key-value pairs for GBK. The outer + # Iterable should be stripped but the inner Tuple left intact. + def do_fn(element: int) -> typehints.Iterable[typehints.Tuple[str, int]]: + return [(str(element), element)] + + th = beam.ParDo(do_fn).get_type_hints() + self.assertEqual(th.input_types, ((int,), {})) + self.assertEqual(th.output_types, ((typehints.Tuple[str, int],), {})) + def test_pardo_wrapper_not_iterable(self): def do_fn(element: int) -> str: return str(element) - with self.assertRaisesRegex(ValueError, r'Return value not iterable'): + with self.assertLogs() as cm: _ = beam.ParDo(do_fn).get_type_hints() + self.assertRegex(''.join(cm.output), r'do_fn.* not iterable') def test_flat_map_wrapper(self): def map_fn(element: int) -> typehints.Iterable[int]: diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 4a9c739f43ad1..6062e6f00d32e 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -1171,7 +1171,8 @@ def is_consistent_with(sub, base): def get_yielded_type(type_hint): """Obtains the type of elements yielded by an iterable. - Note that "iterable" here means: can be iterated over in a for loop. + Note that "iterable" here means: can be iterated over in a for loop, excluding + strings. Args: type_hint: (TypeConstraint) The iterable in question. Must be normalize()-d. From 4725b54fe457ee44c733a83b0d34adb50ff193ae Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Thu, 24 Oct 2019 16:34:45 +0200 Subject: [PATCH 31/57] [BEAM-7520] ignore UsesStrictTimerOrdering for both batch and streaming flink --- runners/flink/job-server/flink_job_server.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index d6664ccc158a5..560f195708838 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -147,9 +147,9 @@ def portableValidatesRunnerTask(String name, Boolean streaming) { excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle' excludeCategories 'org.apache.beam.sdk.testing.UsesMapState' excludeCategories 'org.apache.beam.sdk.testing.UsesSetState' + excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering' if (streaming) { excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime' - excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering' } else { excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream' } From 7b3ce81c0e5af627bf7bdfd3849a0be297d0de28 Mon Sep 17 00:00:00 2001 From: Chad Dombrova Date: Thu, 24 Oct 2019 08:37:59 -0700 Subject: [PATCH 32/57] Mark kafka and pubsub external transforms as experimental --- sdks/python/apache_beam/io/external/gcp/pubsub.py | 12 ++++++++++-- sdks/python/apache_beam/io/external/kafka.py | 6 ++++-- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/io/external/gcp/pubsub.py b/sdks/python/apache_beam/io/external/gcp/pubsub.py index 7d66819fdb89d..f0988ed759628 100644 --- a/sdks/python/apache_beam/io/external/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/external/gcp/pubsub.py @@ -40,7 +40,11 @@ class ReadFromPubSub(beam.PTransform): - """An external ``PTransform`` for reading from Cloud Pub/Sub.""" + """An external ``PTransform`` for reading from Cloud Pub/Sub. + + Experimental; no backwards compatibility guarantees. It requires special + preparation of the Java SDK. See BEAM-7870. + """ URN = 'beam:external:java:pubsub:read:v1' @@ -115,7 +119,11 @@ def expand(self, pbegin): class WriteToPubSub(beam.PTransform): - """An external ``PTransform`` for writing messages to Cloud Pub/Sub.""" + """An external ``PTransform`` for writing messages to Cloud Pub/Sub. + + Experimental; no backwards compatibility guarantees. It requires special + preparation of the Java SDK. See BEAM-7870. + """ URN = 'beam:external:java:pubsub:write:v1' diff --git a/sdks/python/apache_beam/io/external/kafka.py b/sdks/python/apache_beam/io/external/kafka.py index f824515b1aa4b..04d91a79a26d8 100644 --- a/sdks/python/apache_beam/io/external/kafka.py +++ b/sdks/python/apache_beam/io/external/kafka.py @@ -64,7 +64,8 @@ class ReadFromKafka(ExternalTransform): Note: Runners need to support translating Read operations in order to use this source. At the moment only the Flink Runner supports this. - Experimental; no backwards compatibility guarantees. + Experimental; no backwards compatibility guarantees. It requires special + preparation of the Java SDK. See BEAM-7870. """ # Returns the key/value data as raw byte arrays @@ -128,7 +129,8 @@ class WriteToKafka(ExternalTransform): If no Kafka Serializer for key/value is provided, then key/value are assumed to be byte arrays. - Experimental; no backwards compatibility guarantees. + Experimental; no backwards compatibility guarantees. It requires special + preparation of the Java SDK. See BEAM-7870. """ # Default serializer which passes raw bytes to Kafka From 814222826b7f2ce5962226ddbd04245aada7c154 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Thu, 24 Oct 2019 10:11:24 -0700 Subject: [PATCH 33/57] Moving to 2.18.0-SNAPSHOT on master branch. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- gradle.properties | 4 ++-- sdks/python/apache_beam/version.py | 2 +- 3 files changed, 4 insertions(+), 4 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 b824535955ee9..3f7a4aa139c0a 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -300,7 +300,7 @@ class BeamModulePlugin implements Plugin { // Automatically use the official release version if we are performing a release // otherwise append '-SNAPSHOT' - project.version = '2.17.0' + project.version = '2.18.0' if (!isRelease(project)) { project.version += '-SNAPSHOT' } diff --git a/gradle.properties b/gradle.properties index 412f5a1838f7e..6eb4913b1d69b 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,5 +23,5 @@ offlineRepositoryRoot=offline-repository signing.gnupg.executable=gpg signing.gnupg.useLegacyGpg=true -version=2.17.0-SNAPSHOT -python_sdk_version=2.17.0.dev +version=2.18.0-SNAPSHOT +python_sdk_version=2.18.0.dev diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index 13651147d9017..f32561a8f8ec5 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -18,4 +18,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.17.0.dev' +__version__ = '2.18.0.dev' From 7f93150fe9bc9e8a5777196e43d90864072a94e7 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Thu, 24 Oct 2019 10:51:46 -0700 Subject: [PATCH 34/57] Revert "Moving to 2.18.0-SNAPSHOT on master branch." This reverts commit 814222826b7f2ce5962226ddbd04245aada7c154. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- gradle.properties | 4 ++-- sdks/python/apache_beam/version.py | 2 +- 3 files changed, 4 insertions(+), 4 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 3f7a4aa139c0a..b824535955ee9 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -300,7 +300,7 @@ class BeamModulePlugin implements Plugin { // Automatically use the official release version if we are performing a release // otherwise append '-SNAPSHOT' - project.version = '2.18.0' + project.version = '2.17.0' if (!isRelease(project)) { project.version += '-SNAPSHOT' } diff --git a/gradle.properties b/gradle.properties index 6eb4913b1d69b..412f5a1838f7e 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,5 +23,5 @@ offlineRepositoryRoot=offline-repository signing.gnupg.executable=gpg signing.gnupg.useLegacyGpg=true -version=2.18.0-SNAPSHOT -python_sdk_version=2.18.0.dev +version=2.17.0-SNAPSHOT +python_sdk_version=2.17.0.dev diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index f32561a8f8ec5..13651147d9017 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -18,4 +18,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.18.0.dev' +__version__ = '2.17.0.dev' From f21f41724f1d9bb07ffffb49644499240f7e9bfe Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Thu, 24 Oct 2019 10:57:28 -0700 Subject: [PATCH 35/57] Moving to 2.18.0-SNAPSHOT on master branch. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- gradle.properties | 4 ++-- sdks/python/apache_beam/version.py | 2 +- 3 files changed, 4 insertions(+), 4 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 b824535955ee9..3f7a4aa139c0a 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -300,7 +300,7 @@ class BeamModulePlugin implements Plugin { // Automatically use the official release version if we are performing a release // otherwise append '-SNAPSHOT' - project.version = '2.17.0' + project.version = '2.18.0' if (!isRelease(project)) { project.version += '-SNAPSHOT' } diff --git a/gradle.properties b/gradle.properties index 412f5a1838f7e..6eb4913b1d69b 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,5 +23,5 @@ offlineRepositoryRoot=offline-repository signing.gnupg.executable=gpg signing.gnupg.useLegacyGpg=true -version=2.17.0-SNAPSHOT -python_sdk_version=2.17.0.dev +version=2.18.0-SNAPSHOT +python_sdk_version=2.18.0.dev diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index 13651147d9017..f32561a8f8ec5 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -18,4 +18,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.17.0.dev' +__version__ = '2.18.0.dev' From a9cace344dfabca332d8294378ee2f37bc312478 Mon Sep 17 00:00:00 2001 From: Hai Lu Date: Thu, 24 Oct 2019 15:05:53 -0700 Subject: [PATCH 36/57] [BEAM-8341]: basic bundling support for portable runner (#9777) --- .../runners/samza/SamzaPipelineOptions.java | 12 ++ .../samza/SamzaPipelineOptionsValidator.java | 41 ++++- .../beam/runners/samza/SamzaRunner.java | 7 +- .../beam/runners/samza/runtime/DoFnOp.java | 141 +++++++++++++++--- .../runners/samza/runtime/GroupByKeyOp.java | 6 +- .../apache/beam/runners/samza/runtime/Op.java | 2 +- .../beam/runners/samza/runtime/OpAdapter.java | 2 +- .../samza/runtime/SamzaDoFnRunners.java | 74 ++++++--- .../ParDoBoundMultiTranslator.java | 2 + .../translation/SamzaPipelineTranslator.java | 4 + 10 files changed, 243 insertions(+), 48 deletions(-) diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java index ed4437f19e719..3ff64e3afa700 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java @@ -105,4 +105,16 @@ public interface SamzaPipelineOptions extends PipelineOptions { List getMetricsReporters(); void setMetricsReporters(List reporters); + + @Description("The maximum number of elements in a bundle.") + @Default.Long(1) + long getMaxBundleSize(); + + void setMaxBundleSize(long maxBundleSize); + + @Description("The maximum time to wait before finalising a bundle (in milliseconds).") + @Default.Long(1000) + long getMaxBundleTimeMs(); + + void setMaxBundleTimeMs(long maxBundleTimeMs); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java index 24ed33054a9cd..f965e5a566f6b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java @@ -18,17 +18,44 @@ package org.apache.beam.runners.samza; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsValidator; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.config.TaskConfig; /** Validates that the {@link SamzaPipelineOptions} conforms to all the criteria. */ public class SamzaPipelineOptionsValidator { - public static SamzaPipelineOptions validate(PipelineOptions opts) { - final SamzaPipelineOptions samzaOptions = - PipelineOptionsValidator.validate(SamzaPipelineOptions.class, opts); + public static void validate(SamzaPipelineOptions opts) { + checkArgument(opts.getMaxSourceParallelism() >= 1); + validateBundlingRelatedOptions(opts); + } + + /* + * Perform some bundling related validation for pipeline option . + */ + private static void validateBundlingRelatedOptions(SamzaPipelineOptions pipelineOptions) { + if (pipelineOptions.getMaxBundleSize() > 1) { + // TODO: remove this check and implement bundling for side input, timer, etc in DoFnOp.java + checkState( + isPortable(pipelineOptions), + "Bundling is not supported in non portable mode. Please disable by setting maxBundleSize to 1."); + + String taskConcurrencyConfig = TaskConfig.MAX_CONCURRENCY(); + Map configs = + pipelineOptions.getConfigOverride() == null + ? new HashMap<>() + : pipelineOptions.getConfigOverride(); + long taskConcurrency = Long.parseLong(configs.getOrDefault(taskConcurrencyConfig, "1")); + checkState( + taskConcurrency == 1, + "Bundling is not supported if " + + taskConcurrencyConfig + + " is greater than 1. Please disable bundling by setting maxBundleSize to 1. Or disable task concurrency."); + } + } - checkArgument(samzaOptions.getMaxSourceParallelism() >= 1); - return samzaOptions; + private static boolean isPortable(SamzaPipelineOptions options) { + return options instanceof SamzaPortablePipelineOptions; } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java index 0eb50c4b6e9a3..4a9462652411b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.values.PValue; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators; import org.apache.samza.application.StreamApplication; @@ -56,7 +57,8 @@ public class SamzaRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(SamzaRunner.class); public static SamzaRunner fromOptions(PipelineOptions opts) { - final SamzaPipelineOptions samzaOptions = SamzaPipelineOptionsValidator.validate(opts); + final SamzaPipelineOptions samzaOptions = + PipelineOptionsValidator.validate(SamzaPipelineOptions.class, opts); return new SamzaRunner(samzaOptions); } @@ -133,6 +135,9 @@ public SamzaPipelineResult run(Pipeline pipeline) { pipeline, new TranslationContext(appDescriptor, idMap, options)); }; + // perform a final round of validation for the pipeline options now that all configs are + // generated + SamzaPipelineOptionsValidator.validate(options); ApplicationRunner runner = runSamzaApp(app, config); return new SamzaPipelineResult(app, runner, executionContext, listener, config); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index 86e2ee68d7a55..795b8c0fd80da 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.Map; import java.util.ServiceLoader; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -32,14 +34,15 @@ import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; +import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.samza.SamzaExecutionContext; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.join.RawUnionValue; @@ -57,6 +60,7 @@ import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +68,7 @@ /** Samza operator for {@link DoFn}. */ public class DoFnOp implements Op { private static final Logger LOG = LoggerFactory.getLogger(DoFnOp.class); + private static final long MIN_BUNDLE_CHECK_TIME_MS = 10L; private final TupleTag mainOutputTag; private final DoFn doFn; @@ -77,8 +82,11 @@ public class DoFnOp implements Op { private final String transformFullName; private final String transformId; private final Coder inputCoder; + private final Coder> windowedValueCoder; private final HashMap, Coder> outputCoders; private final PCollection.IsBounded isBounded; + private final String bundleCheckTimerId; + private final String bundleStateId; // portable api related private final boolean isPortable; @@ -90,6 +98,7 @@ public class DoFnOp implements Op { private transient PushbackSideInputDoFnRunner pushbackFnRunner; private transient SideInputHandler sideInputHandler; private transient DoFnInvoker doFnInvoker; + private transient SamzaPipelineOptions samzaPipelineOptions; // This is derivable from pushbackValues which is persisted to a store. // TODO: eagerly initialize the hold in init @@ -100,9 +109,16 @@ public class DoFnOp implements Op { // TODO: add this to checkpointable state private transient Instant inputWatermark; + private transient Instant bundleWatermarkHold; private transient Instant sideInputWatermark; private transient List> pushbackValues; private transient StageBundleFactory stageBundleFactory; + private transient long maxBundleSize; + private transient long maxBundleTimeMs; + private transient AtomicLong currentBundleElementCount; + private transient AtomicLong bundleStartTime; + private transient AtomicBoolean isBundleStarted; + private transient Scheduler> bundleTimerScheduler; private DoFnSchemaInformation doFnSchemaInformation; private Map> sideInputMapping; @@ -111,6 +127,7 @@ public DoFnOp( DoFn doFn, Coder keyCoder, Coder inputCoder, + Coder> windowedValueCoder, Map, Coder> outputCoders, Collection> sideInputs, List> sideOutputTags, @@ -130,6 +147,7 @@ public DoFnOp( this.sideInputs = sideInputs; this.sideOutputTags = sideOutputTags; this.inputCoder = inputCoder; + this.windowedValueCoder = windowedValueCoder; this.outputCoders = new HashMap<>(outputCoders); this.windowingStrategy = windowingStrategy; this.idToViewMap = new HashMap<>(idToViewMap); @@ -141,6 +159,8 @@ public DoFnOp( this.isPortable = isPortable; this.stagePayload = stagePayload; this.idToTupleTagMap = new HashMap<>(idToTupleTagMap); + this.bundleCheckTimerId = "_samza_bundle_check_" + transformId; + this.bundleStateId = "_samza_bundle_" + transformId; this.doFnSchemaInformation = doFnSchemaInformation; this.sideInputMapping = sideInputMapping; } @@ -154,17 +174,26 @@ public void open( this.inputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; this.sideInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; this.pushbackWatermarkHold = BoundedWindow.TIMESTAMP_MAX_VALUE; + this.currentBundleElementCount = new AtomicLong(0L); + this.bundleStartTime = new AtomicLong(Long.MAX_VALUE); + this.isBundleStarted = new AtomicBoolean(false); + this.bundleWatermarkHold = null; final DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); - final SamzaPipelineOptions pipelineOptions = - Base64Serializer.deserializeUnchecked( - config.get("beamPipelineOptions"), SerializablePipelineOptions.class) - .get() - .as(SamzaPipelineOptions.class); + final SamzaExecutionContext samzaExecutionContext = + (SamzaExecutionContext) context.getApplicationContainerContext(); + this.samzaPipelineOptions = samzaExecutionContext.getPipelineOptions(); + this.maxBundleSize = samzaPipelineOptions.getMaxBundleSize(); + this.maxBundleTimeMs = samzaPipelineOptions.getMaxBundleTimeMs(); + this.bundleTimerScheduler = timerRegistry; + + if (this.maxBundleSize > 1) { + scheduleNextBundleCheck(); + } final SamzaStoreStateInternals.Factory nonKeyedStateInternalsFactory = SamzaStoreStateInternals.createStateInternalFactory( - transformId, null, context.getTaskContext(), pipelineOptions, signature); + transformId, null, context.getTaskContext(), samzaPipelineOptions, signature); this.timerInternalsFactory = SamzaTimerInternalsFactory.createTimerInternalFactory( @@ -174,18 +203,23 @@ public void open( nonKeyedStateInternalsFactory, windowingStrategy, isBounded, - pipelineOptions); + samzaPipelineOptions); this.sideInputHandler = new SideInputHandler(sideInputs, nonKeyedStateInternalsFactory.stateInternalsForKey(null)); if (isPortable) { - SamzaExecutionContext samzaExecutionContext = - (SamzaExecutionContext) context.getApplicationContainerContext(); - ExecutableStage executableStage = ExecutableStage.fromPayload(stagePayload); + // storing events within a bundle in states + final BagState> bundledEventsBagState = + nonKeyedStateInternalsFactory + .stateInternalsForKey(null) + .state(StateNamespaces.global(), StateTags.bag(bundleStateId, windowedValueCoder)); + final ExecutableStage executableStage = ExecutableStage.fromPayload(stagePayload); stageBundleFactory = samzaExecutionContext.getJobBundleFactory().forStage(executableStage); this.fnRunner = SamzaDoFnRunners.createPortable( + samzaPipelineOptions, + bundledEventsBagState, outputManagerFactory.create(emitter), stageBundleFactory, mainOutputTag, @@ -195,7 +229,7 @@ public void open( } else { this.fnRunner = SamzaDoFnRunners.create( - pipelineOptions, + samzaPipelineOptions, doFn, windowingStrategy, transformFullName, @@ -229,6 +263,25 @@ public void open( doFnInvoker.invokeSetup(); } + /* + * Schedule in processing time to check whether the current bundle should be closed. Note that + * we only approximately achieve max bundle time by checking as frequent as half of the max bundle + * time set by users. This would violate the max bundle time by up to half of it but should + * acceptable in most cases (and cheaper than scheduling a timer at the beginning of every bundle). + */ + private void scheduleNextBundleCheck() { + final Instant nextBundleCheckTime = + Instant.now().plus(Duration.millis(maxBundleTimeMs / 2 + MIN_BUNDLE_CHECK_TIME_MS)); + final TimerInternals.TimerData timerData = + TimerInternals.TimerData.of( + bundleCheckTimerId, + StateNamespaces.global(), + nextBundleCheckTime, + TimeDomain.PROCESSING_TIME); + bundleTimerScheduler.schedule( + new KeyedTimerData<>(new byte[0], null, timerData), nextBundleCheckTime.getMillis()); + } + private String getTimerStateId(DoFnSignature signature) { final StringBuilder builder = new StringBuilder("timer"); if (signature.usesTimers()) { @@ -237,9 +290,39 @@ private String getTimerStateId(DoFnSignature signature) { return builder.toString(); } + private void attemptStartBundle() { + if (isBundleStarted.compareAndSet(false, true)) { + currentBundleElementCount.set(0L); + bundleStartTime.set(System.currentTimeMillis()); + pushbackFnRunner.startBundle(); + } + } + + private void finishBundle(OpEmitter emitter) { + if (isBundleStarted.compareAndSet(true, false)) { + currentBundleElementCount.set(0L); + bundleStartTime.set(Long.MAX_VALUE); + pushbackFnRunner.finishBundle(); + if (bundleWatermarkHold != null) { + doProcessWatermark(bundleWatermarkHold, emitter); + } + bundleWatermarkHold = null; + } + } + + private void attemptFinishBundle(OpEmitter emitter) { + if (!isBundleStarted.get()) { + return; + } + if (currentBundleElementCount.get() >= maxBundleSize + || System.currentTimeMillis() - bundleStartTime.get() > maxBundleTimeMs) { + finishBundle(emitter); + } + } + @Override public void processElement(WindowedValue inputElement, OpEmitter emitter) { - pushbackFnRunner.startBundle(); + attemptStartBundle(); final Iterable> rejectedValues = pushbackFnRunner.processElementInReadyWindows(inputElement); @@ -250,11 +333,11 @@ public void processElement(WindowedValue inputElement, OpEmitter emit pushbackValues.add(rejectedValue); } - pushbackFnRunner.finishBundle(); + currentBundleElementCount.incrementAndGet(); + attemptFinishBundle(emitter); } - @Override - public void processWatermark(Instant watermark, OpEmitter emitter) { + private void doProcessWatermark(Instant watermark, OpEmitter emitter) { this.inputWatermark = watermark; if (sideInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MAX_VALUE)) { @@ -280,6 +363,20 @@ public void processWatermark(Instant watermark, OpEmitter emitter) { } } + @Override + public void processWatermark(Instant watermark, OpEmitter emitter) { + if (!isBundleStarted.get()) { + doProcessWatermark(watermark, emitter); + } else { + // if there is a bundle in progress, hold back the watermark until end of the bundle + this.bundleWatermarkHold = watermark; + if (watermark.isEqual(BoundedWindow.TIMESTAMP_MAX_VALUE)) { + // for batch mode, the max watermark should force the bundle to close + finishBundle(emitter); + } + } + } + @Override public void processSideInput( String id, WindowedValue> elements, OpEmitter emitter) { @@ -317,7 +414,14 @@ public void processSideInputWatermark(Instant watermark, OpEmitter emitter } @Override - public void processTimer(KeyedTimerData keyedTimerData) { + public void processTimer(KeyedTimerData keyedTimerData, OpEmitter emitter) { + // this is internal timer in processing time to check whether a bundle should be closed + if (bundleCheckTimerId.equals(keyedTimerData.getTimerData().getTimerId())) { + attemptFinishBundle(emitter); + scheduleNextBundleCheck(); + return; + } + pushbackFnRunner.startBundle(); fireTimer(keyedTimerData); pushbackFnRunner.finishBundle(); @@ -327,6 +431,7 @@ public void processTimer(KeyedTimerData keyedTimerData) { @Override public void close() { + bundleWatermarkHold = null; doFnInvoker.invokeTeardown(); try (AutoCloseable closer = stageBundleFactory) { // do nothing diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java index 387ca9a82124a..1b0ab61f57009 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java @@ -195,7 +195,7 @@ public void processElement( } @Override - public void processWatermark(Instant watermark, OpEmitter> ctx) { + public void processWatermark(Instant watermark, OpEmitter> emitter) { timerInternalsFactory.setInputWatermark(watermark); fnRunner.startBundle(); @@ -207,12 +207,12 @@ public void processWatermark(Instant watermark, OpEmitter> ctx) { if (timerInternalsFactory.getOutputWatermark() == null || timerInternalsFactory.getOutputWatermark().isBefore(watermark)) { timerInternalsFactory.setOutputWatermark(watermark); - ctx.emitWatermark(timerInternalsFactory.getOutputWatermark()); + emitter.emitWatermark(timerInternalsFactory.getOutputWatermark()); } } @Override - public void processTimer(KeyedTimerData keyedTimerData) { + public void processTimer(KeyedTimerData keyedTimerData, OpEmitter> emitter) { fnRunner.startBundle(); fireTimer(keyedTimerData.getKey(), keyedTimerData.getTimerData()); fnRunner.finishBundle(); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java index cbf5c46d37308..93e6a9c2f6133 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java @@ -58,7 +58,7 @@ default void processSideInput( default void processSideInputWatermark(Instant watermark, OpEmitter emitter) {} - default void processTimer(KeyedTimerData keyedTimerData) {}; + default void processTimer(KeyedTimerData keyedTimerData, OpEmitter emitter) {} default void close() {} } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java index 8b958db5cfcbc..e663a045bf2f0 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java @@ -128,7 +128,7 @@ public Collection> onCallback(KeyedTimerData keyedTimerData, assert outputList.isEmpty(); try { - op.processTimer(keyedTimerData); + op.processTimer(keyedTimerData, emitter); } catch (Exception e) { LOG.error("Op {} threw an exception during processing timer", this.getClass().getName(), e); throw UserCodeException.wrap(e); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java index 49b4a28cff7ea..3b1b938ec0ced 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java @@ -37,6 +37,7 @@ import org.apache.beam.runners.samza.metrics.DoFnRunnerWithMetrics; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; @@ -163,6 +164,8 @@ private static StatefulDoFnRunner.StateCleaner createStateClean /** Create DoFnRunner for portable runner. */ public static DoFnRunner createPortable( + SamzaPipelineOptions pipelineOptions, + BagState> bundledEventsBag, DoFnRunners.OutputManager outputManager, StageBundleFactory stageBundleFactory, TupleTag mainOutputTag, @@ -173,7 +176,7 @@ public static DoFnRunner createPortable( (SamzaExecutionContext) context.getApplicationContainerContext(); final DoFnRunner sdkHarnessDoFnRunner = new SdkHarnessDoFnRunner<>( - outputManager, stageBundleFactory, mainOutputTag, idToTupleTagMap); + outputManager, stageBundleFactory, mainOutputTag, idToTupleTagMap, bundledEventsBag); return DoFnRunnerWithMetrics.wrap( sdkHarnessDoFnRunner, executionContext.getMetricsContainer(), transformFullName); } @@ -184,23 +187,25 @@ private static class SdkHarnessDoFnRunner implements DoFnRunner mainOutputTag; private final Map> idToTupleTagMap; private final LinkedBlockingQueue> outputQueue = new LinkedBlockingQueue<>(); + private final BagState> bundledEventsBag; + private RemoteBundle remoteBundle; + private FnDataReceiver> inputReceiver; private SdkHarnessDoFnRunner( DoFnRunners.OutputManager outputManager, StageBundleFactory stageBundleFactory, TupleTag mainOutputTag, - Map> idToTupleTagMap) { + Map> idToTupleTagMap, + BagState> bundledEventsBag) { this.outputManager = outputManager; this.stageBundleFactory = stageBundleFactory; this.mainOutputTag = mainOutputTag; this.idToTupleTagMap = idToTupleTagMap; + this.bundledEventsBag = bundledEventsBag; } @Override - public void startBundle() {} - - @Override - public void processElement(WindowedValue elem) { + public void startBundle() { try { OutputReceiverFactory receiverFactory = new OutputReceiverFactory() { @@ -213,31 +218,66 @@ public FnDataReceiver create(String pCollectionId) { } }; - try (RemoteBundle bundle = + remoteBundle = stageBundleFactory.getBundle( receiverFactory, StateRequestHandler.unsupported(), - BundleProgressHandler.ignored())) { - Iterables.getOnlyElement(bundle.getInputReceivers().values()).accept(elem); - } + BundleProgressHandler.ignored()); - // RemoteBundle close blocks until all results are received - KV result; - while ((result = outputQueue.poll()) != null) { - outputManager.output( - idToTupleTagMap.get(result.getKey()), (WindowedValue) result.getValue()); - } + // TODO: side input support needs to implement to handle this properly + inputReceiver = Iterables.getOnlyElement(remoteBundle.getInputReceivers().values()); + bundledEventsBag + .read() + .forEach( + elem -> { + try { + inputReceiver.accept(elem); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); } catch (Exception e) { throw new RuntimeException(e); } } + @Override + public void processElement(WindowedValue elem) { + try { + bundledEventsBag.add(elem); + inputReceiver.accept(elem); + emitResults(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void emitResults() { + KV result; + while ((result = outputQueue.poll()) != null) { + outputManager.output( + idToTupleTagMap.get(result.getKey()), (WindowedValue) result.getValue()); + } + } + @Override public void onTimer( String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {} @Override - public void finishBundle() {} + public void finishBundle() { + try { + // RemoteBundle close blocks until all results are received + remoteBundle.close(); + emitResults(); + bundledEventsBag.clear(); + } catch (Exception e) { + throw new RuntimeException("Failed to finish remote bundle", e); + } finally { + remoteBundle = null; + inputReceiver = null; + } + } @Override public DoFn getFn() { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java index f00c34b14d6c2..6550ebf5af8a6 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java @@ -144,6 +144,7 @@ private static void doTranslate( transform.getFn(), keyCoder, (Coder) input.getCoder(), + null, outputCoders, transform.getSideInputs().values(), transform.getAdditionalOutputTags().getAll(), @@ -254,6 +255,7 @@ private static void doTranslatePortable( new NoOpDoFn<>(), null, // key coder not in use windowedInputCoder.getValueCoder(), // input coder not in use + windowedInputCoder, Collections.emptyMap(), // output coders not in use Collections.emptyList(), // sideInputs not in use until side input support new ArrayList<>(idToTupleTagMap.values()), // used by java runner only diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java index c30b181294d30..bfa2e10621c26 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.samza.translation; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.service.AutoService; import java.util.HashMap; @@ -53,6 +54,9 @@ private static Map> loadTranslators() { private SamzaPipelineTranslator() {} public static void translate(Pipeline pipeline, TranslationContext ctx) { + checkState( + ctx.getPipelineOptions().getMaxBundleSize() <= 1, + "bundling is not supported for non portable mode. Please disable bundling (by setting max bundle size to 1)."); final TransformVisitorFn translateFn = new TransformVisitorFn() { From 70190ab39e4d6197c31dd73b27c793e178b3c43a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 24 Oct 2019 16:16:33 -0700 Subject: [PATCH 37/57] [BEAM-8480] Explicitly set restriction coder for bounded reader wrapper SDF. --- sdks/python/apache_beam/io/iobase.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index 605c1bf0984dd..5b6673089fac6 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -1440,6 +1440,9 @@ def split(self, element, restriction): def restriction_size(self, element, restriction): return restriction.weight + def restriction_coder(self): + return coders.DillCoder() + def __init__(self, source): if not isinstance(source, BoundedSource): raise RuntimeError('SDFBoundedSourceWrapper can only wrap BoundedSource') From 1d59072b835283c48e5d047074e67e2db2911171 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Thu, 24 Oct 2019 17:25:22 -0700 Subject: [PATCH 38/57] Revert "Merge pull request #9854 from [BEAM-8457] Label Dataflow jobs from Notebook" This reverts commit 1a8391da9222ab8d0493b0007bd60bdbeeb5e275. --- sdks/python/apache_beam/pipeline.py | 48 ++++--------------- .../runners/dataflow/dataflow_runner.py | 10 ---- .../runners/interactive/interactive_runner.py | 2 +- 3 files changed, 9 insertions(+), 51 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 5574a825e467f..a776d3065a14e 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -171,10 +171,6 @@ def __init__(self, runner=None, options=None, argv=None): # If a transform is applied and the full label is already in the set # then the transform will have to be cloned with a new label. self.applied_labels = set() - # A boolean value indicating whether the pipeline is created in an - # interactive environment such as interactive notebooks. Initialized as - # None. The value is set ad hoc when `pipeline.run()` is invoked. - self.interactive = None @property @deprecated(since='First stable release', @@ -399,56 +395,28 @@ def replace_all(self, replacements): for override in replacements: self._check_replacement(override) - def run(self, test_runner_api=True, runner=None, options=None, - interactive=None): - """Runs the pipeline. Returns whatever our runner returns after running. - - If another runner instance and options are provided, that runner will - execute the pipeline with the given options. If either of them is not set, - a ValueError is raised. The usage is similar to directly invoking - `runner.run_pipeline(pipeline, options)`. - Additionally, an interactive field can be set to override the pipeline's - self.interactive field to mark current pipeline as being initiated from an - interactive environment. - """ - from apache_beam.runners.interactive import interactive_runner - if interactive: - self.interactive = interactive - elif isinstance(self.runner, interactive_runner.InteractiveRunner): - self.interactive = True - else: - self.interactive = False - runner_in_use = self.runner - options_in_use = self._options - if runner and options: - runner_in_use = runner - options_in_use = options - elif not runner and options: - raise ValueError('Parameter runner is not given when parameter options ' - 'is given.') - elif not options and runner: - raise ValueError('Parameter options is not given when parameter runner ' - 'is given.') + def run(self, test_runner_api=True): + """Runs the pipeline. Returns whatever our runner returns after running.""" + # When possible, invoke a round trip through the runner API. if test_runner_api and self._verify_runner_api_compatible(): return Pipeline.from_runner_api( self.to_runner_api(use_fake_coders=True), - runner_in_use, - options_in_use).run(test_runner_api=False, - interactive=self.interactive) + self.runner, + self._options).run(False) - if options_in_use.view_as(TypeOptions).runtime_type_check: + if self._options.view_as(TypeOptions).runtime_type_check: from apache_beam.typehints import typecheck self.visit(typecheck.TypeCheckVisitor()) - if options_in_use.view_as(SetupOptions).save_main_session: + if self._options.view_as(SetupOptions).save_main_session: # If this option is chosen, verify we can pickle the main session early. tmpdir = tempfile.mkdtemp() try: pickler.dump_session(os.path.join(tmpdir, 'main_session.pickle')) finally: shutil.rmtree(tmpdir) - return runner_in_use.run_pipeline(self, options_in_use) + return self.runner.run_pipeline(self, self._options) def __enter__(self): return self diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index f57be74687c94..4928550143d69 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -364,16 +364,6 @@ def visit_transform(self, transform_node): def run_pipeline(self, pipeline, options): """Remotely executes entire pipeline or parts reachable from node.""" - # Label goog-dataflow-notebook if pipeline is initiated from interactive - # runner. - if pipeline.interactive: - notebook_version = ('goog-dataflow-notebook=' + - beam.version.__version__.replace('.', '_')) - if options.view_as(GoogleCloudOptions).labels: - options.view_as(GoogleCloudOptions).labels.append(notebook_version) - else: - options.view_as(GoogleCloudOptions).labels = [notebook_version] - # Import here to avoid adding the dependency for local running scenarios. try: # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner.py b/sdks/python/apache_beam/runners/interactive/interactive_runner.py index 56a3c180fd7c2..94c0de725d848 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_runner.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_runner.py @@ -146,7 +146,7 @@ def run_pipeline(self, pipeline, options): cache_manager=self._cache_manager, pipeline_graph_renderer=self._renderer) display.start_periodic_update() - result = pipeline_to_execute.run(interactive=True) + result = pipeline_to_execute.run() result.wait_until_finish() display.stop_periodic_update() From 177c1348727a64e4e7555c914a492d199c8f57d0 Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Thu, 24 Oct 2019 17:43:09 -0700 Subject: [PATCH 39/57] [BEAM-7389] Add code snippet for CoGroupByKey (#9791) [BEAM-7389] Add code snippet for CoGroupByKey (#9791) --- .../transforms/aggregation/__init__.py | 18 ++++++ .../transforms/aggregation/cogroupbykey.py | 49 +++++++++++++++ .../aggregation/cogroupbykey_test.py | 59 +++++++++++++++++++ .../transforms/elementwise/partition_test.py | 10 ++-- .../apache_beam/examples/snippets/util.py | 10 +++- .../examples/snippets/util_test.py | 11 ++++ 6 files changed, 149 insertions(+), 8 deletions(-) create mode 100644 sdks/python/apache_beam/examples/snippets/transforms/aggregation/__init__.py create mode 100644 sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey.py create mode 100644 sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey_test.py diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/__init__.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/__init__.py new file mode 100644 index 0000000000000..6569e3fe5de4d --- /dev/null +++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/__init__.py @@ -0,0 +1,18 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import absolute_import diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey.py new file mode 100644 index 0000000000000..c507e03921407 --- /dev/null +++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey.py @@ -0,0 +1,49 @@ +# coding=utf-8 +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import absolute_import +from __future__ import print_function + + +def cogroupbykey(test=None): + # [START cogroupbykey] + import apache_beam as beam + + with beam.Pipeline() as pipeline: + icon_pairs = pipeline | 'Create icons' >> beam.Create([ + ('Apple', '🍎'), + ('Apple', '🍏'), + ('Eggplant', '🍆'), + ('Tomato', '🍅'), + ]) + + duration_pairs = pipeline | 'Create durations' >> beam.Create([ + ('Apple', 'perennial'), + ('Carrot', 'biennial'), + ('Tomato', 'perennial'), + ('Tomato', 'annual'), + ]) + + plants = ( + ({'icons': icon_pairs, 'durations': duration_pairs}) + | 'Merge' >> beam.CoGroupByKey() + | beam.Map(print) + ) + # [END cogroupbykey] + if test: + test(plants) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey_test.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey_test.py new file mode 100644 index 0000000000000..ff86628f787b6 --- /dev/null +++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/cogroupbykey_test.py @@ -0,0 +1,59 @@ +# coding=utf-8 +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import absolute_import +from __future__ import print_function + +import unittest + +import mock + +from apache_beam.examples.snippets.util import assert_matches_stdout +from apache_beam.testing.test_pipeline import TestPipeline + +from . import cogroupbykey + + +def check_plants(actual): + expected = '''[START plants] +('Apple', {'icons': ['🍎', '🍏'], 'durations': ['perennial']}) +('Carrot', {'icons': [], 'durations': ['biennial']}) +('Tomato', {'icons': ['🍅'], 'durations': ['perennial', 'annual']}) +('Eggplant', {'icons': ['🍆'], 'durations': []}) +[END plants]'''.splitlines()[1:-1] + + # Make it deterministic by sorting all sublists in each element. + def normalize_element(elem): + name, details = elem + details['icons'] = sorted(details['icons']) + details['durations'] = sorted(details['durations']) + return name, details + assert_matches_stdout(actual, expected, normalize_element) + + +@mock.patch('apache_beam.Pipeline', TestPipeline) +@mock.patch( + 'apache_beam.examples.snippets.transforms.aggregation.cogroupbykey.print', + str) +class CoGroupByKeyTest(unittest.TestCase): + def test_cogroupbykey(self): + cogroupbykey.cogroupbykey(check_plants) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py index 45842039446fc..4f98ab16eb67a 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py @@ -54,9 +54,9 @@ def check_partitions(actual1, actual2, actual3): if line.split(':', 1)[0] == 'perennial' ] - assert_matches_stdout(actual1, annuals, 'annuals') - assert_matches_stdout(actual2, biennials, 'biennials') - assert_matches_stdout(actual3, perennials, 'perennials') + assert_matches_stdout(actual1, annuals, label='annuals') + assert_matches_stdout(actual2, biennials, label='biennials') + assert_matches_stdout(actual3, perennials, label='perennials') def check_split_datasets(actual1, actual2): @@ -79,8 +79,8 @@ def check_split_datasets(actual1, actual2): if line.split(':', 1)[0] == 'test' ] - assert_matches_stdout(actual1, train_dataset, 'train_dataset') - assert_matches_stdout(actual2, test_dataset, 'test_dataset') + assert_matches_stdout(actual1, train_dataset, label='train_dataset') + assert_matches_stdout(actual2, test_dataset, label='test_dataset') @mock.patch('apache_beam.Pipeline', TestPipeline) diff --git a/sdks/python/apache_beam/examples/snippets/util.py b/sdks/python/apache_beam/examples/snippets/util.py index ac17c5200886c..60c2c7edc63b1 100644 --- a/sdks/python/apache_beam/examples/snippets/util.py +++ b/sdks/python/apache_beam/examples/snippets/util.py @@ -26,19 +26,23 @@ from apache_beam.testing.util import equal_to -def assert_matches_stdout(actual, expected_stdout, label=''): +def assert_matches_stdout( + actual, expected_stdout, normalize_fn=lambda elem: elem, label=''): """Asserts a PCollection of strings matches the expected stdout elements. Args: actual (beam.PCollection): A PCollection. expected (List[str]): A list of stdout elements, one line per element. + normalize_fn (Function[any]): A function to normalize elements before + comparing them. Can be used to sort lists before comparing. label (str): [optional] Label to make transform names unique. """ def stdout_to_python_object(elem_str): try: - return ast.literal_eval(elem_str) + elem = ast.literal_eval(elem_str) except (SyntaxError, ValueError): - return elem_str + elem = elem_str + return normalize_fn(elem) actual = actual | label >> beam.Map(stdout_to_python_object) expected = list(map(stdout_to_python_object, expected_stdout)) diff --git a/sdks/python/apache_beam/examples/snippets/util_test.py b/sdks/python/apache_beam/examples/snippets/util_test.py index 8ac20130435c0..fcf395562cce0 100644 --- a/sdks/python/apache_beam/examples/snippets/util_test.py +++ b/sdks/python/apache_beam/examples/snippets/util_test.py @@ -60,6 +60,17 @@ def test_assert_matches_stdout_string(self): ) util.assert_matches_stdout(actual, expected) + def test_assert_matches_stdout_sorted_keys(self): + expected = [{'list': [1, 2]}, {'list': [3, 4]}] + with TestPipeline() as pipeline: + actual = ( + pipeline + | beam.Create([{'list': [2, 1]}, {'list': [4, 3]}]) + | beam.Map(str) + ) + util.assert_matches_stdout( + actual, expected, lambda elem: {'sorted': sorted(elem['list'])}) + @patch('subprocess.call', lambda cmd: None) def test_run_shell_commands(self): commands = [ From 43ce5ca8e881225f204750bc39fe32f2d0495283 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 24 Oct 2019 17:09:32 -0700 Subject: [PATCH 40/57] [Go SDK] Fix post commits: runtime error in logger --- .../pkg/beam/core/runtime/harness/logging.go | 2 +- .../beam/core/runtime/harness/logging_test.go | 51 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) create mode 100644 sdks/go/pkg/beam/core/runtime/harness/logging_test.go diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging.go b/sdks/go/pkg/beam/core/runtime/harness/logging.go index 7abc5462758a1..2a1d0fae77065 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/logging.go +++ b/sdks/go/pkg/beam/core/runtime/harness/logging.go @@ -46,7 +46,7 @@ func tryGetInstID(ctx context.Context) (string, bool) { if id == nil { return "", false } - return id.(string), true + return string(id.(instructionID)), true } type logger struct { diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging_test.go b/sdks/go/pkg/beam/core/runtime/harness/logging_test.go new file mode 100644 index 0000000000000..606b3c71cd866 --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/harness/logging_test.go @@ -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 harness + +import ( + "context" + "strings" + "testing" + + "github.com/apache/beam/sdks/go/pkg/beam/log" + pb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1" +) + +func TestLogger(t *testing.T) { + ch := make(chan *pb.LogEntry, 1) + l := logger{out: ch} + + instID := "INST" + ctx := setInstID(context.Background(), instructionID(instID)) + msg := "expectedMessage" + l.Log(ctx, log.SevInfo, 0, msg) + + e := <-ch + + if got, want := e.GetInstructionId(), instID; got != want { + t.Errorf("incorrect InstructionID: got %v, want %v", got, want) + } + if got, want := e.GetMessage(), msg; got != want { + t.Errorf("incorrect Message: got %v, want %v", got, want) + } + // This check will fail if the imports change. + if got, want := e.GetLogLocation(), "logging_test.go:34"; !strings.HasSuffix(got, want) { + t.Errorf("incorrect LogLocation: got %v, want suffix %v", got, want) + } + if got, want := e.GetSeverity(), pb.LogEntry_Severity_INFO; got != want { + t.Errorf("incorrect Severity: got %v, want %v", got, want) + } +} From 05ee7f1045a0bf26ef7cebcbd0559089c671d624 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Wed, 23 Oct 2019 17:23:42 +0200 Subject: [PATCH 41/57] [BEAM-8462] Bump up Flink version inside gradle task name --- .test-infra/dataproc/flink_cluster.sh | 2 +- .test-infra/jenkins/Flink.groovy | 4 +++- .test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy | 2 +- .test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy | 2 +- .test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy | 2 +- .test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy | 2 +- 6 files changed, 8 insertions(+), 6 deletions(-) diff --git a/.test-infra/dataproc/flink_cluster.sh b/.test-infra/dataproc/flink_cluster.sh index 86d9b23fe8bd4..a24e34a44d39b 100755 --- a/.test-infra/dataproc/flink_cluster.sh +++ b/.test-infra/dataproc/flink_cluster.sh @@ -34,7 +34,7 @@ # HARNESS_IMAGES_TO_PULL='gcr.io//python:latest gcr.io//java:latest' \ # JOB_SERVER_IMAGE=gcr.io//job-server-flink:latest \ # ARTIFACTS_DIR=gs:// \ -# FLINK_DOWNLOAD_URL=http://archive.apache.org/dist/flink/flink-1.7.0/flink-1.7.0-bin-hadoop28-scala_2.12.tgz \ +# FLINK_DOWNLOAD_URL=https://archive.apache.org/dist/flink/flink-1.9.1/flink-1.9.1-bin-scala_2.11.tgz \ # FLINK_NUM_WORKERS=2 \ # FLINK_TASKMANAGER_SLOTS=1 \ # DETACHED_MODE=false \ diff --git a/.test-infra/jenkins/Flink.groovy b/.test-infra/jenkins/Flink.groovy index a986d648cb5be..0c0df64b89a69 100644 --- a/.test-infra/jenkins/Flink.groovy +++ b/.test-infra/jenkins/Flink.groovy @@ -17,7 +17,8 @@ */ class Flink { - private static final String flinkDownloadUrl = 'https://archive.apache.org/dist/flink/flink-1.7.0/flink-1.7.0-bin-hadoop28-scala_2.11.tgz' + private static final String flinkDownloadUrl = 'https://archive.apache.org/dist/flink/flink-1.9.1/flink-1.9.1-bin-scala_2.11.tgz' + private static final String hadoopDownloadUrl = 'https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-7.0/flink-shaded-hadoop-2-uber-2.8.3-7.0.jar' private static final String FLINK_DIR = '"$WORKSPACE/src/.test-infra/dataproc"' private static final String FLINK_SCRIPT = 'flink_cluster.sh' private def job @@ -53,6 +54,7 @@ class Flink { env("CLUSTER_NAME", clusterName) env("GCS_BUCKET", gcsBucket) env("FLINK_DOWNLOAD_URL", flinkDownloadUrl) + env("HADOOP_DOWNLOAD_URL", hadoopDownloadUrl) env("FLINK_NUM_WORKERS", workerCount) env("FLINK_TASKMANAGER_SLOTS", slotsPerTaskmanager) env("DETACHED_MODE", 'true') diff --git a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy index 2f847e785eece..884cbc87bb5f9 100644 --- a/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy +++ b/.test-infra/jenkins/job_LoadTests_Combine_Flink_Python.groovy @@ -105,7 +105,7 @@ def batchLoadTestJob = { scope, triggeringContext -> List testScenarios = scenarios(datasetName, pythonHarnessImageTag) publisher.publish(':sdks:python:container:py2:docker', 'python2.7_sdk') - publisher.publish(':runners:flink:1.7:job-server-container:docker', 'flink-job-server') + publisher.publish(':runners:flink:1.9:job-server-container:docker', 'flink-job-server') def flink = new Flink(scope, 'beam_LoadTests_Python_Combine_Flink_Batch') flink.setUp([pythonHarnessImageTag], numberOfWorkers, publisher.getFullImageName('flink-job-server')) diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy index 7f3bb2197ca45..d6b497f441eb5 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Flink_Python.groovy @@ -172,7 +172,7 @@ def loadTest = { scope, triggeringContext -> List testScenarios = scenarios(datasetName, pythonHarnessImageTag) publisher.publish(':sdks:python:container:py2:docker', 'python2.7_sdk') - publisher.publish(':runners:flink:1.7:job-server-container:docker', 'flink-job-server') + publisher.publish(':runners:flink:1.9:job-server-container:docker', 'flink-job-server') def flink = new Flink(scope, 'beam_LoadTests_Python_GBK_Flink_Batch') flink.setUp([pythonHarnessImageTag], numberOfWorkers, publisher.getFullImageName('flink-job-server')) diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy index 97c37dd390d7e..5841ae3445a40 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Flink_Python.groovy @@ -129,7 +129,7 @@ def loadTest = { scope, triggeringContext -> List testScenarios = scenarios(datasetName, pythonHarnessImageTag) publisher.publish(':sdks:python:container:py2:docker', 'python2.7_sdk') - publisher.publish(':runners:flink:1.7:job-server-container:docker', 'flink-job-server') + publisher.publish(':runners:flink:1.9:job-server-container:docker', 'flink-job-server') Flink flink = new Flink(scope, 'beam_LoadTests_Python_ParDo_Flink_Batch') flink.setUp([pythonHarnessImageTag], numberOfWorkers, publisher.getFullImageName('flink-job-server')) diff --git a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy b/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy index 026d19754a938..b6549c87b8fa4 100644 --- a/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy +++ b/.test-infra/jenkins/job_LoadTests_coGBK_Flink_Python.groovy @@ -157,7 +157,7 @@ def loadTest = { scope, triggeringContext -> List testScenarios = scenarios(datasetName, pythonHarnessImageTag) publisher.publish(':sdks:python:container:py2:docker', 'python2.7_sdk') - publisher.publish('runners:flink:1.7:job-server-container:docker', 'flink-job-server') + publisher.publish(':runners:flink:1.9:job-server-container:docker', 'flink-job-server') def flink = new Flink(scope, 'beam_LoadTests_Python_CoGBK_Flink_Batch') flink.setUp([pythonHarnessImageTag], numberOfWorkers, publisher.getFullImageName('flink-job-server')) From e456b7a303558e1e944428fe98a91e27f714e9c7 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Thu, 24 Oct 2019 12:49:04 +0200 Subject: [PATCH 42/57] [BEAM-8462] Put Hadoop jar into the FLINK_INSTALL_DIR/lib directory Flink binaries since version 1.8 no longer includes hadoop, which is a dependency. A recommended solution is to download Hadoop jar manually and put it into the FLINK_INSTALL_DIR/lib directory. See https://issues.apache.org/jira/browse/FLINK-11266 for more information. --- .test-infra/dataproc/flink_cluster.sh | 5 ++++- .test-infra/dataproc/init-actions/flink.sh | 9 ++++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/.test-infra/dataproc/flink_cluster.sh b/.test-infra/dataproc/flink_cluster.sh index a24e34a44d39b..e183a1f110daf 100755 --- a/.test-infra/dataproc/flink_cluster.sh +++ b/.test-infra/dataproc/flink_cluster.sh @@ -24,6 +24,7 @@ # JOB_SERVER_IMAGE: Url to job server docker image to pull on dataproc master (optional) # ARTIFACTS_DIR: Url to bucket where artifacts will be stored for staging (optional) # FLINK_DOWNLOAD_URL: Url to Flink .tar archive to be installed on the cluster +# HADOOP_DOWNLOAD_URL: Url to a pre-packaged Hadoop jar # FLINK_NUM_WORKERS: Number of Flink workers # FLINK_TASKMANAGER_SLOTS: Number of slots per Flink task manager # DETACHED_MODE: Detached mode: should the SSH tunnel run in detached mode? @@ -35,6 +36,7 @@ # JOB_SERVER_IMAGE=gcr.io//job-server-flink:latest \ # ARTIFACTS_DIR=gs:// \ # FLINK_DOWNLOAD_URL=https://archive.apache.org/dist/flink/flink-1.9.1/flink-1.9.1-bin-scala_2.11.tgz \ +# HADOOP_DOWNLOAD_URL=https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-7.0/flink-shaded-hadoop-2-uber-2.8.3-7.0.jar \ # FLINK_NUM_WORKERS=2 \ # FLINK_TASKMANAGER_SLOTS=1 \ # DETACHED_MODE=false \ @@ -118,7 +120,8 @@ function start_tunnel() { function create_cluster() { local metadata="flink-snapshot-url=${FLINK_DOWNLOAD_URL}," metadata+="flink-start-yarn-session=true," - metadata+="flink-taskmanager-slots=${FLINK_TASKMANAGER_SLOTS}" + metadata+="flink-taskmanager-slots=${FLINK_TASKMANAGER_SLOTS}," + metadata+="hadoop-jar-url=${HADOOP_DOWNLOAD_URL}" [[ -n "${HARNESS_IMAGES_TO_PULL:=}" ]] && metadata+=",beam-sdk-harness-images-to-pull=${HARNESS_IMAGES_TO_PULL}" [[ -n "${JOB_SERVER_IMAGE:=}" ]] && metadata+=",beam-job-server-image=${JOB_SERVER_IMAGE}" diff --git a/.test-infra/dataproc/init-actions/flink.sh b/.test-infra/dataproc/init-actions/flink.sh index 19598723a1c4e..7e06b7ef10232 100644 --- a/.test-infra/dataproc/init-actions/flink.sh +++ b/.test-infra/dataproc/init-actions/flink.sh @@ -56,6 +56,9 @@ readonly START_FLINK_YARN_SESSION_DEFAULT=true # Set this to install flink from a snapshot URL instead of apt readonly FLINK_SNAPSHOT_URL_METADATA_KEY='flink-snapshot-url' +# Set this to install pre-packaged Hadoop jar +readonly HADOOP_JAR_URL_METADATA_KEY='hadoop-jar-url' + # Set this to define how many task slots are there per flink task manager readonly FLINK_TASKMANAGER_SLOTS_METADATA_KEY='flink-taskmanager-slots' @@ -88,6 +91,7 @@ function install_apt_get() { function install_flink_snapshot() { local work_dir="$(mktemp -d)" local flink_url="$(/usr/share/google/get_metadata_value "attributes/${FLINK_SNAPSHOT_URL_METADATA_KEY}")" + local hadoop_url="$(/usr/share/google/get_metadata_value "attributes/${HADOOP_JAR_URL_METADATA_KEY}")" local flink_local="${work_dir}/flink.tgz" local flink_toplevel_pattern="${work_dir}/flink-*" @@ -103,6 +107,9 @@ function install_flink_snapshot() { popd # work_dir + if [[ ! -z "${hadoop_url}" ]]; then + cd "${FLINK_INSTALL_DIR}/lib"; curl -O "${hadoop_url}" + fi } function configure_flink() { @@ -205,4 +212,4 @@ function main() { fi } -main \ No newline at end of file +main From c4287a362f542b21548bd50c0ff2a9358541d82b Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Thu, 24 Oct 2019 15:39:36 +0200 Subject: [PATCH 43/57] [BEAM-8462] Add --region=global parameter to gcloud dataproc calls This solves the warning "Dataproc --region flag will become required in January 2020" --- .test-infra/dataproc/flink_cluster.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.test-infra/dataproc/flink_cluster.sh b/.test-infra/dataproc/flink_cluster.sh index e183a1f110daf..78a45c461469b 100755 --- a/.test-infra/dataproc/flink_cluster.sh +++ b/.test-infra/dataproc/flink_cluster.sh @@ -134,7 +134,7 @@ function create_cluster() { # Docker init action restarts yarn so we need to start yarn session after this restart happens. # This is why flink init action is invoked last. - gcloud dataproc clusters create $CLUSTER_NAME --num-workers=$num_dataproc_workers --initialization-actions $DOCKER_INIT,$BEAM_INIT,$FLINK_INIT --metadata "${metadata}", --image-version=$image_version --zone=$GCLOUD_ZONE --quiet + gcloud dataproc clusters create $CLUSTER_NAME --region=global --num-workers=$num_dataproc_workers --initialization-actions $DOCKER_INIT,$BEAM_INIT,$FLINK_INIT --metadata "${metadata}", --image-version=$image_version --zone=$GCLOUD_ZONE --quiet } # Runs init actions for Docker, Portability framework (Beam) and Flink cluster @@ -155,7 +155,7 @@ function restart() { # Deletes a Flink cluster. function delete() { - gcloud dataproc clusters delete $CLUSTER_NAME --quiet + gcloud dataproc clusters delete $CLUSTER_NAME --region=global --quiet } "$@" From a365c67baa5bacb9910d0a2e97bb0e879e1525bc Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Mon, 7 Oct 2019 18:04:05 -0700 Subject: [PATCH 44/57] Combine serializer and deserializer --- .../apache/beam/sdk/transforms/JsonToRow.java | 2 +- .../apache/beam/sdk/transforms/ToJson.java | 2 +- .../org/apache/beam/sdk/util/RowJson.java | 365 ++++++++++++++++++ .../beam/sdk/util/RowJsonDeserializer.java | 271 ------------- .../beam/sdk/util/RowJsonSerializer.java | 111 ------ .../apache/beam/sdk/util/RowJsonUtils.java | 11 +- .../beam/sdk/util/RowJsonValidation.java | 2 +- .../beam/sdk/util/RowJsonValueExtractors.java | 2 +- .../org/apache/beam/sdk/util/RowJsonTest.java | 14 +- .../provider/pubsub/PubsubMessageToRow.java | 4 +- 10 files changed, 385 insertions(+), 399 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonDeserializer.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonSerializer.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java index 33fb2a6710a47..ab0b7403676f2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.util.RowJsonDeserializer; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java index 28d6c464994ea..641c21a60ca3e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java @@ -24,7 +24,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.util.RowJsonSerializer; +import org.apache.beam.sdk.util.RowJson.RowJsonSerializer; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java new file mode 100644 index 0000000000000..e5d5e35b9402c --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java @@ -0,0 +1,365 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util; + +import static java.util.stream.Collectors.toList; +import static org.apache.beam.sdk.schemas.Schema.TypeName.BOOLEAN; +import static org.apache.beam.sdk.schemas.Schema.TypeName.BYTE; +import static org.apache.beam.sdk.schemas.Schema.TypeName.DECIMAL; +import static org.apache.beam.sdk.schemas.Schema.TypeName.DOUBLE; +import static org.apache.beam.sdk.schemas.Schema.TypeName.FLOAT; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT16; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT32; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT64; +import static org.apache.beam.sdk.schemas.Schema.TypeName.STRING; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.booleanValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.byteValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.decimalValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.doubleValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.floatValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.intValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.longValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.shortValueExtractor; +import static org.apache.beam.sdk.util.RowJsonValueExtractors.stringValueExtractor; +import static org.apache.beam.sdk.values.Row.toRow; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.node.JsonNodeType; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import javax.annotation.Nullable; +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.util.RowJsonValueExtractors.ValueExtractor; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; + +/** + * Jackson serializer and deserializer for {@link Row Rows}. + * + *

Supports converting between JSON primitive types and: + * + *

    + *
  • {@link Schema.TypeName#BYTE} + *
  • {@link Schema.TypeName#INT16} + *
  • {@link Schema.TypeName#INT32} + *
  • {@link Schema.TypeName#INT64} + *
  • {@link Schema.TypeName#FLOAT} + *
  • {@link Schema.TypeName#DOUBLE} + *
  • {@link Schema.TypeName#BOOLEAN} + *
  • {@link Schema.TypeName#STRING} + *
+ */ +public class RowJson { + /** Jackson deserializer for parsing JSON into {@link Row Rows}. */ + public static class RowJsonDeserializer extends StdDeserializer { + + private static final boolean SEQUENTIAL = false; + + private static final ImmutableMap> JSON_VALUE_GETTERS = + ImmutableMap.>builder() + .put(BYTE, byteValueExtractor()) + .put(INT16, shortValueExtractor()) + .put(INT32, intValueExtractor()) + .put(INT64, longValueExtractor()) + .put(FLOAT, floatValueExtractor()) + .put(DOUBLE, doubleValueExtractor()) + .put(BOOLEAN, booleanValueExtractor()) + .put(STRING, stringValueExtractor()) + .put(DECIMAL, decimalValueExtractor()) + .build(); + + private final Schema schema; + + /** Creates a deserializer for a {@link Row} {@link Schema}. */ + public static RowJsonDeserializer forSchema(Schema schema) { + schema.getFields().forEach(RowJsonValidation::verifyFieldTypeSupported); + return new RowJsonDeserializer(schema); + } + + private RowJsonDeserializer(Schema schema) { + super(Row.class); + this.schema = schema; + } + + @Override + public Row deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + + // Parse and convert the root object to Row as if it's a nested field with name 'root' + return (Row) + extractJsonNodeValue( + FieldValue.of("root", FieldType.row(schema), jsonParser.readValueAsTree())); + } + + private static Object extractJsonNodeValue(FieldValue fieldValue) { + if (!fieldValue.isJsonValuePresent()) { + throw new UnsupportedRowJsonException( + "Field '" + fieldValue.name() + "' is not present in the JSON object"); + } + + if (fieldValue.isJsonNull()) { + return null; + } + + if (fieldValue.isRowType()) { + return jsonObjectToRow(fieldValue); + } + + if (fieldValue.isArrayType()) { + return jsonArrayToList(fieldValue); + } + + return extractJsonPrimitiveValue(fieldValue); + } + + private static Row jsonObjectToRow(FieldValue rowFieldValue) { + if (!rowFieldValue.isJsonObject()) { + throw new UnsupportedRowJsonException( + "Expected JSON object for field '" + + rowFieldValue.name() + + "'. " + + "Unable to convert '" + + rowFieldValue.jsonValue().asText() + + "'" + + " to Beam Row, it is not a JSON object. Currently only JSON objects " + + "can be parsed to Beam Rows"); + } + + return rowFieldValue.rowSchema().getFields().stream() + .map( + schemaField -> + extractJsonNodeValue( + FieldValue.of( + schemaField.getName(), + schemaField.getType(), + rowFieldValue.jsonFieldValue(schemaField.getName())))) + .collect(toRow(rowFieldValue.rowSchema())); + } + + private static Object jsonArrayToList(FieldValue arrayFieldValue) { + if (!arrayFieldValue.isJsonArray()) { + throw new UnsupportedRowJsonException( + "Expected JSON array for field '" + + arrayFieldValue.name() + + "'. " + + "Instead got " + + arrayFieldValue.jsonNodeType().name()); + } + + return arrayFieldValue + .jsonArrayElements() + .map( + jsonArrayElement -> + extractJsonNodeValue( + FieldValue.of( + arrayFieldValue.name() + "[]", + arrayFieldValue.arrayElementType(), + jsonArrayElement))) + .collect(toList()); + } + + private static Object extractJsonPrimitiveValue(FieldValue fieldValue) { + try { + return JSON_VALUE_GETTERS.get(fieldValue.typeName()).extractValue(fieldValue.jsonValue()); + } catch (RuntimeException e) { + throw new UnsupportedRowJsonException( + "Unable to get value from field '" + + fieldValue.name() + + "'. " + + "Schema type '" + + fieldValue.typeName() + + "'. " + + "JSON node type " + + fieldValue.jsonNodeType().name(), + e); + } + } + + /** + * Helper class to keep track of schema field type, name, and actual json value for the field. + */ + @AutoValue + abstract static class FieldValue { + abstract String name(); + + abstract FieldType type(); + + abstract @Nullable JsonNode jsonValue(); + + TypeName typeName() { + return type().getTypeName(); + } + + boolean isJsonValuePresent() { + return jsonValue() != null; + } + + boolean isJsonNull() { + return jsonValue().isNull(); + } + + JsonNodeType jsonNodeType() { + return jsonValue().getNodeType(); + } + + boolean isJsonArray() { + return jsonValue().isArray(); + } + + Stream jsonArrayElements() { + return StreamSupport.stream(jsonValue().spliterator(), SEQUENTIAL); + } + + boolean isArrayType() { + return TypeName.ARRAY.equals(type().getTypeName()); + } + + FieldType arrayElementType() { + return type().getCollectionElementType(); + } + + boolean isJsonObject() { + return jsonValue().isObject(); + } + + JsonNode jsonFieldValue(String fieldName) { + return jsonValue().get(fieldName); + } + + boolean isRowType() { + return TypeName.ROW.equals(type().getTypeName()); + } + + Schema rowSchema() { + return type().getRowSchema(); + } + + static FieldValue of(String name, FieldType type, JsonNode jsonValue) { + return new AutoValue_RowJson_RowJsonDeserializer_FieldValue(name, type, jsonValue); + } + } + + /** Gets thrown when Row parsing fails for any reason. */ + public static class UnsupportedRowJsonException extends RuntimeException { + + UnsupportedRowJsonException(String message, Throwable reason) { + super(message, reason); + } + + UnsupportedRowJsonException(String message) { + super(message); + } + } + } + + /** Jackson serializer for converting {@link Row Rows} to JSON. */ + public static class RowJsonSerializer extends StdSerializer { + + private final Schema schema; + + /** Creates a serializer for a {@link Row} {@link Schema}. */ + public static RowJsonSerializer forSchema(Schema schema) { + schema.getFields().forEach(RowJsonValidation::verifyFieldTypeSupported); + return new RowJsonSerializer(schema); + } + + private RowJsonSerializer(Schema schema) { + super(Row.class); + this.schema = schema; + } + + @Override + public void serialize(Row value, JsonGenerator gen, SerializerProvider provider) + throws IOException { + writeRow(value, this.schema, gen); + } + + // TODO: ByteBuddy generate based on schema? + private void writeRow(Row row, Schema schema, JsonGenerator gen) throws IOException { + gen.writeStartObject(); + for (int i = 0; i < schema.getFieldCount(); ++i) { + Field field = schema.getField(i); + Object value = row.getValue(i); + gen.writeFieldName(field.getName()); + if (field.getType().getNullable() && value == null) { + gen.writeNull(); + continue; + } + writeValue(gen, field.getType(), value); + } + gen.writeEndObject(); + } + + private void writeValue(JsonGenerator gen, FieldType type, Object value) throws IOException { + switch (type.getTypeName()) { + case BOOLEAN: + gen.writeBoolean((boolean) value); + break; + case STRING: + gen.writeString((String) value); + break; + case BYTE: + gen.writeNumber((byte) value); + break; + case DOUBLE: + gen.writeNumber((double) value); + break; + case FLOAT: + gen.writeNumber((float) value); + break; + case INT16: + gen.writeNumber((short) value); + break; + case INT32: + gen.writeNumber((int) value); + break; + case INT64: + gen.writeNumber((long) value); + break; + case DECIMAL: + gen.writeNumber((BigDecimal) value); + break; + case ARRAY: + gen.writeStartArray(); + for (Object element : (List) value) { + writeValue(gen, type.getCollectionElementType(), element); + } + gen.writeEndArray(); + break; + case ROW: + writeRow((Row) value, type.getRowSchema(), gen); + break; + default: + throw new IllegalArgumentException("Unsupported field type: " + type); + } + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonDeserializer.java deleted file mode 100644 index 192972670d7d4..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonDeserializer.java +++ /dev/null @@ -1,271 +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.util; - -import static java.util.stream.Collectors.toList; -import static org.apache.beam.sdk.schemas.Schema.TypeName.BOOLEAN; -import static org.apache.beam.sdk.schemas.Schema.TypeName.BYTE; -import static org.apache.beam.sdk.schemas.Schema.TypeName.DECIMAL; -import static org.apache.beam.sdk.schemas.Schema.TypeName.DOUBLE; -import static org.apache.beam.sdk.schemas.Schema.TypeName.FLOAT; -import static org.apache.beam.sdk.schemas.Schema.TypeName.INT16; -import static org.apache.beam.sdk.schemas.Schema.TypeName.INT32; -import static org.apache.beam.sdk.schemas.Schema.TypeName.INT64; -import static org.apache.beam.sdk.schemas.Schema.TypeName.STRING; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.booleanValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.byteValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.decimalValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.doubleValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.floatValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.intValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.longValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.shortValueExtractor; -import static org.apache.beam.sdk.util.RowJsonValueExtractors.stringValueExtractor; -import static org.apache.beam.sdk.values.Row.toRow; - -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.deser.std.StdDeserializer; -import com.fasterxml.jackson.databind.node.JsonNodeType; -import com.google.auto.value.AutoValue; -import java.io.IOException; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; -import javax.annotation.Nullable; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.util.RowJsonValueExtractors.ValueExtractor; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; - -/** - * Jackson deserializer for {@link Row Rows}. - * - *

Supports converting JSON primitive types to: - * - *

    - *
  • {@link Schema.TypeName#BYTE} - *
  • {@link Schema.TypeName#INT16} - *
  • {@link Schema.TypeName#INT32} - *
  • {@link Schema.TypeName#INT64} - *
  • {@link Schema.TypeName#FLOAT} - *
  • {@link Schema.TypeName#DOUBLE} - *
  • {@link Schema.TypeName#BOOLEAN} - *
  • {@link Schema.TypeName#STRING} - *
- */ -public class RowJsonDeserializer extends StdDeserializer { - - private static final boolean SEQUENTIAL = false; - - private static final ImmutableMap> JSON_VALUE_GETTERS = - ImmutableMap.>builder() - .put(BYTE, byteValueExtractor()) - .put(INT16, shortValueExtractor()) - .put(INT32, intValueExtractor()) - .put(INT64, longValueExtractor()) - .put(FLOAT, floatValueExtractor()) - .put(DOUBLE, doubleValueExtractor()) - .put(BOOLEAN, booleanValueExtractor()) - .put(STRING, stringValueExtractor()) - .put(DECIMAL, decimalValueExtractor()) - .build(); - - private final Schema schema; - - /** Creates a deserializer for a {@link Row} {@link Schema}. */ - public static RowJsonDeserializer forSchema(Schema schema) { - schema.getFields().forEach(RowJsonValidation::verifyFieldTypeSupported); - return new RowJsonDeserializer(schema); - } - - private RowJsonDeserializer(Schema schema) { - super(Row.class); - this.schema = schema; - } - - @Override - public Row deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) - throws IOException { - - // Parse and convert the root object to Row as if it's a nested field with name 'root' - return (Row) - extractJsonNodeValue( - FieldValue.of("root", FieldType.row(schema), jsonParser.readValueAsTree())); - } - - private static Object extractJsonNodeValue(FieldValue fieldValue) { - if (!fieldValue.isJsonValuePresent()) { - throw new UnsupportedRowJsonException( - "Field '" + fieldValue.name() + "' is not present in the JSON object"); - } - - if (fieldValue.isJsonNull()) { - return null; - } - - if (fieldValue.isRowType()) { - return jsonObjectToRow(fieldValue); - } - - if (fieldValue.isArrayType()) { - return jsonArrayToList(fieldValue); - } - - return extractJsonPrimitiveValue(fieldValue); - } - - private static Row jsonObjectToRow(FieldValue rowFieldValue) { - if (!rowFieldValue.isJsonObject()) { - throw new UnsupportedRowJsonException( - "Expected JSON object for field '" - + rowFieldValue.name() - + "'. " - + "Unable to convert '" - + rowFieldValue.jsonValue().asText() - + "'" - + " to Beam Row, it is not a JSON object. Currently only JSON objects " - + "can be parsed to Beam Rows"); - } - - return rowFieldValue.rowSchema().getFields().stream() - .map( - schemaField -> - extractJsonNodeValue( - FieldValue.of( - schemaField.getName(), - schemaField.getType(), - rowFieldValue.jsonFieldValue(schemaField.getName())))) - .collect(toRow(rowFieldValue.rowSchema())); - } - - private static Object jsonArrayToList(FieldValue arrayFieldValue) { - if (!arrayFieldValue.isJsonArray()) { - throw new UnsupportedRowJsonException( - "Expected JSON array for field '" - + arrayFieldValue.name() - + "'. " - + "Instead got " - + arrayFieldValue.jsonNodeType().name()); - } - - return arrayFieldValue - .jsonArrayElements() - .map( - jsonArrayElement -> - extractJsonNodeValue( - FieldValue.of( - arrayFieldValue.name() + "[]", - arrayFieldValue.arrayElementType(), - jsonArrayElement))) - .collect(toList()); - } - - private static Object extractJsonPrimitiveValue(FieldValue fieldValue) { - try { - return JSON_VALUE_GETTERS.get(fieldValue.typeName()).extractValue(fieldValue.jsonValue()); - } catch (RuntimeException e) { - throw new UnsupportedRowJsonException( - "Unable to get value from field '" - + fieldValue.name() - + "'. " - + "Schema type '" - + fieldValue.typeName() - + "'. " - + "JSON node type " - + fieldValue.jsonNodeType().name(), - e); - } - } - - /** Helper class to keep track of schema field type, name, and actual json value for the field. */ - @AutoValue - abstract static class FieldValue { - abstract String name(); - - abstract FieldType type(); - - abstract @Nullable JsonNode jsonValue(); - - TypeName typeName() { - return type().getTypeName(); - } - - boolean isJsonValuePresent() { - return jsonValue() != null; - } - - boolean isJsonNull() { - return jsonValue().isNull(); - } - - JsonNodeType jsonNodeType() { - return jsonValue().getNodeType(); - } - - boolean isJsonArray() { - return jsonValue().isArray(); - } - - Stream jsonArrayElements() { - return StreamSupport.stream(jsonValue().spliterator(), SEQUENTIAL); - } - - boolean isArrayType() { - return TypeName.ARRAY.equals(type().getTypeName()); - } - - FieldType arrayElementType() { - return type().getCollectionElementType(); - } - - boolean isJsonObject() { - return jsonValue().isObject(); - } - - JsonNode jsonFieldValue(String fieldName) { - return jsonValue().get(fieldName); - } - - boolean isRowType() { - return TypeName.ROW.equals(type().getTypeName()); - } - - Schema rowSchema() { - return type().getRowSchema(); - } - - static FieldValue of(String name, FieldType type, JsonNode jsonValue) { - return new AutoValue_RowJsonDeserializer_FieldValue(name, type, jsonValue); - } - } - - /** Gets thrown when Row parsing fails for any reason. */ - public static class UnsupportedRowJsonException extends RuntimeException { - - UnsupportedRowJsonException(String message, Throwable reason) { - super(message, reason); - } - - UnsupportedRowJsonException(String message) { - super(message); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonSerializer.java deleted file mode 100644 index 0cb16727d9072..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonSerializer.java +++ /dev/null @@ -1,111 +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.util; - -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.ser.std.StdSerializer; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.List; -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.values.Row; - -public class RowJsonSerializer extends StdSerializer { - - private final Schema schema; - - /** Creates a serializer for a {@link Row} {@link Schema}. */ - public static RowJsonSerializer forSchema(Schema schema) { - schema.getFields().forEach(RowJsonValidation::verifyFieldTypeSupported); - return new RowJsonSerializer(schema); - } - - private RowJsonSerializer(Schema schema) { - super(Row.class); - this.schema = schema; - } - - @Override - public void serialize(Row value, JsonGenerator gen, SerializerProvider provider) - throws IOException { - writeRow(value, this.schema, gen); - } - - // TODO: ByteBuddy generate based on schema? - private void writeRow(Row row, Schema schema, JsonGenerator gen) throws IOException { - gen.writeStartObject(); - for (int i = 0; i < schema.getFieldCount(); ++i) { - Field field = schema.getField(i); - Object value = row.getValue(i); - gen.writeFieldName(field.getName()); - if (field.getType().getNullable() && value == null) { - gen.writeNull(); - continue; - } - writeValue(gen, field.getType(), value); - } - gen.writeEndObject(); - } - - private void writeValue(JsonGenerator gen, FieldType type, Object value) throws IOException { - switch (type.getTypeName()) { - case BOOLEAN: - gen.writeBoolean((boolean) value); - break; - case STRING: - gen.writeString((String) value); - break; - case BYTE: - gen.writeNumber((byte) value); - break; - case DOUBLE: - gen.writeNumber((double) value); - break; - case FLOAT: - gen.writeNumber((float) value); - break; - case INT16: - gen.writeNumber((short) value); - break; - case INT32: - gen.writeNumber((int) value); - break; - case INT64: - gen.writeNumber((long) value); - break; - case DECIMAL: - gen.writeNumber((BigDecimal) value); - break; - case ARRAY: - gen.writeStartArray(); - for (Object element : (List) value) { - writeValue(gen, type.getCollectionElementType(), element); - } - gen.writeEndArray(); - break; - case ROW: - writeRow((Row) value, type.getRowSchema(), gen); - break; - default: - throw new IllegalArgumentException("Unsupported field type: " + type); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java index 598dc744f48a1..d0447f76374cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java @@ -24,14 +24,17 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import java.io.IOException; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.util.RowJsonDeserializer.UnsupportedRowJsonException; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException; import org.apache.beam.sdk.values.Row; -/** Utilities for working with {@link RowJsonSerializer} and {@link RowJsonDeserializer}. */ +/** + * Utilities for working with {@link RowJson.RowJsonSerializer} and {@link + * RowJson.RowJsonDeserializer}. + */ @Internal public class RowJsonUtils { - public static ObjectMapper newObjectMapperWith(RowJsonDeserializer deserializer) { + public static ObjectMapper newObjectMapperWith(RowJson.RowJsonDeserializer deserializer) { SimpleModule module = new SimpleModule("rowDeserializationModule"); module.addDeserializer(Row.class, deserializer); @@ -41,7 +44,7 @@ public static ObjectMapper newObjectMapperWith(RowJsonDeserializer deserializer) return objectMapper; } - public static ObjectMapper newObjectMapperWith(RowJsonSerializer serializer) { + public static ObjectMapper newObjectMapperWith(RowJson.RowJsonSerializer serializer) { SimpleModule module = new SimpleModule("rowSerializationModule"); module.addSerializer(Row.class, serializer); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java index 2ab7aec223e62..69cd1d3898237 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java @@ -60,7 +60,7 @@ static void verifyFieldTypeSupported(Schema.FieldType fieldType) { } if (!SUPPORTED_TYPES.contains(fieldTypeName)) { - throw new RowJsonDeserializer.UnsupportedRowJsonException( + throw new RowJson.RowJsonDeserializer.UnsupportedRowJsonException( fieldTypeName.name() + " is not supported when converting JSON objects to Rows. " + "Supported types are: " diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValueExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValueExtractors.java index 4db0823ed17f9..13bf854ef6f35 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValueExtractors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValueExtractors.java @@ -22,7 +22,7 @@ import java.math.BigDecimal; import java.util.function.Function; import java.util.function.Predicate; -import org.apache.beam.sdk.util.RowJsonDeserializer.UnsupportedRowJsonException; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException; /** * Contains utilities for extracting primitive values from JSON nodes. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java index f1f3fe9654415..3277bb06b0959 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java @@ -30,7 +30,7 @@ import java.util.Collection; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.util.RowJsonDeserializer.UnsupportedRowJsonException; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matcher; @@ -45,7 +45,7 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -/** Unit tests for {@link RowJsonDeserializer} and {@link RowJsonSerializer}. */ +/** Unit tests for {@link RowJson.RowJsonDeserializer} and {@link RowJson.RowJsonSerializer}. */ @RunWith(Enclosed.class) public class RowJsonTest { @RunWith(Parameterized.class) @@ -313,7 +313,7 @@ public void testDeserializerThrowsForUnsupportedType() throws Exception { thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); - RowJsonDeserializer.forSchema(schema); + RowJson.RowJsonDeserializer.forSchema(schema); } @Test @@ -323,7 +323,7 @@ public void testDeserializerThrowsForUnsupportedArrayElementType() throws Except thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); - RowJsonDeserializer.forSchema(schema); + RowJson.RowJsonDeserializer.forSchema(schema); } @Test @@ -336,7 +336,7 @@ public void testDeserializerThrowsForUnsupportedNestedFieldType() throws Excepti thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); - RowJsonDeserializer.forSchema(schema); + RowJson.RowJsonDeserializer.forSchema(schema); } @Test @@ -511,8 +511,8 @@ private static Matcher unsupportedWithMessage(Strin private static ObjectMapper newObjectMapperFor(Schema schema) { SimpleModule simpleModule = new SimpleModule("rowSerializationTesModule"); - simpleModule.addSerializer(Row.class, RowJsonSerializer.forSchema(schema)); - simpleModule.addDeserializer(Row.class, RowJsonDeserializer.forSchema(schema)); + simpleModule.addSerializer(Row.class, RowJson.RowJsonSerializer.forSchema(schema)); + simpleModule.addDeserializer(Row.class, RowJson.RowJsonDeserializer.forSchema(schema)); ObjectMapper objectMapper = new ObjectMapper(); objectMapper.registerModule(simpleModule); return objectMapper; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubMessageToRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubMessageToRow.java index 654e7225eecbe..3d9a712175669 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubMessageToRow.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubMessageToRow.java @@ -31,8 +31,8 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.TypeName; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.RowJsonDeserializer; -import org.apache.beam.sdk.util.RowJsonDeserializer.UnsupportedRowJsonException; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer; +import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException; import org.apache.beam.sdk.util.RowJsonUtils; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; From d335596bd4be5c7ea6dd2cb9ba38a974c1f5b555 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 9 Oct 2019 17:24:00 -0700 Subject: [PATCH 45/57] Pass along exception --- .../src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java index d0447f76374cc..a882625b977fa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java @@ -68,7 +68,7 @@ public static String rowToJson(ObjectMapper objectMapper, Row row) { try { return objectMapper.writeValueAsString(row); } catch (JsonProcessingException e) { - throw new IllegalArgumentException("Unable to serilize row: " + row); + throw new IllegalArgumentException("Unable to serilize row: " + row, e); } } } From 1239781b6f1423a991958d621c587ce409723eee Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 10 Oct 2019 11:39:50 -0700 Subject: [PATCH 46/57] expose of method --- .../main/java/org/apache/beam/sdk/transforms/ToJson.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java index 641c21a60ca3e..91cf7cb935613 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java @@ -41,8 +41,11 @@ public class ToJson extends PTransform, PCollection> { private transient volatile @Nullable ObjectMapper objectMapper; - static ToJson of() { - return new ToJson(); + private ToJson() { + } + + public static ToJson of() { + return new ToJson<>(); } @Override From 6f9e11adb0bba59a22b0a9a1f30ce9ebbbb7cb4a Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 10 Oct 2019 11:44:36 -0700 Subject: [PATCH 47/57] fixup! --- .../src/main/java/org/apache/beam/sdk/transforms/ToJson.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java index 91cf7cb935613..edeea36ace90a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java @@ -41,8 +41,7 @@ public class ToJson extends PTransform, PCollection> { private transient volatile @Nullable ObjectMapper objectMapper; - private ToJson() { - } + private ToJson() {} public static ToJson of() { return new ToJson<>(); From 0cf8ae8b953dff216991b2fdae365170cb448487 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 17 Oct 2019 13:21:32 -0700 Subject: [PATCH 48/57] Merge incorrectly split strings --- .../java/org/apache/beam/sdk/util/RowJson.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java index e5d5e35b9402c..49ded074969de 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java @@ -145,12 +145,9 @@ private static Row jsonObjectToRow(FieldValue rowFieldValue) { throw new UnsupportedRowJsonException( "Expected JSON object for field '" + rowFieldValue.name() - + "'. " - + "Unable to convert '" + + "'. Unable to convert '" + rowFieldValue.jsonValue().asText() - + "'" - + " to Beam Row, it is not a JSON object. Currently only JSON objects " - + "can be parsed to Beam Rows"); + + "' to Beam Row, it is not a JSON object. Currently only JSON objects can be parsed to Beam Rows"); } return rowFieldValue.rowSchema().getFields().stream() @@ -169,8 +166,7 @@ private static Object jsonArrayToList(FieldValue arrayFieldValue) { throw new UnsupportedRowJsonException( "Expected JSON array for field '" + arrayFieldValue.name() - + "'. " - + "Instead got " + + "'. Instead got " + arrayFieldValue.jsonNodeType().name()); } @@ -193,11 +189,9 @@ private static Object extractJsonPrimitiveValue(FieldValue fieldValue) { throw new UnsupportedRowJsonException( "Unable to get value from field '" + fieldValue.name() - + "'. " - + "Schema type '" + + "'. Schema type '" + fieldValue.typeName() - + "'. " - + "JSON node type " + + "'. JSON node type " + fieldValue.jsonNodeType().name(), e); } From b0d4dbfaa3d226e414189585429f17f7d5269bc7 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 24 Oct 2019 14:51:22 -0700 Subject: [PATCH 49/57] [Go SDK] Propagate data channel failures. --- .../pkg/beam/core/runtime/harness/datamgr.go | 50 ++++++- .../beam/core/runtime/harness/datamgr_test.go | 127 +++++++++++++++--- 2 files changed, 149 insertions(+), 28 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 61ee841cf2498..42c8d73633add 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -144,6 +144,9 @@ type DataChannel struct { readers map[clientID]*dataReader // TODO: early/late closed, bad instructions, finer locks, reconnect? + // readErr indicates a client.Recv error and is used to prevent new readers. + readErr error + mu sync.Mutex // guards both the readers and writers maps. } @@ -174,7 +177,12 @@ func makeDataChannel(ctx context.Context, id string, client dataClient) *DataCha // OpenRead returns an io.ReadCloser of the data elements for the given instruction and ptransform. func (c *DataChannel) OpenRead(ctx context.Context, ptransformID string, instID instructionID) io.ReadCloser { - return c.makeReader(ctx, clientID{ptransformID: ptransformID, instID: instID}) + cid := clientID{ptransformID: ptransformID, instID: instID} + if c.readErr != nil { + log.Errorf(ctx, "opening a reader %v on a closed channel", cid) + return &errReader{c.readErr} + } + return c.makeReader(ctx, cid) } // OpenWrite returns an io.WriteCloser of the data elements for the given instruction and ptransform. @@ -187,8 +195,21 @@ func (c *DataChannel) read(ctx context.Context) { for { msg, err := c.client.Recv() if err != nil { + // This connection is bad, so we should close and delete all extant streams. + c.mu.Lock() + c.readErr = err // prevent not yet opened readers from hanging. + for _, r := range c.readers { + log.Errorf(ctx, "DataChannel.read %v reader %v closing due to error on channel", c.id, r.id) + if !r.completed { + r.completed = true + r.err = err + close(r.buf) + } + delete(cache, r.id) + } + c.mu.Unlock() + if err == io.EOF { - // TODO(herohde) 10/12/2017: can this happen before shutdown? Reconnect? log.Warnf(ctx, "DataChannel.read %v closed", c.id) return } @@ -205,8 +226,6 @@ func (c *DataChannel) read(ctx context.Context) { for _, elm := range msg.GetData() { id := clientID{ptransformID: elm.TransformId, instID: instructionID(elm.GetInstructionId())} - // log.Printf("Chan read (%v): %v\n", sid, elm.GetData()) - var r *dataReader if local, ok := cache[id]; ok { r = local @@ -224,6 +243,7 @@ func (c *DataChannel) read(ctx context.Context) { } if len(elm.GetData()) == 0 { // Sentinel EOF segment for stream. Close buffer to signal EOF. + r.completed = true close(r.buf) // Clean up local bookkeeping. We'll never see another message @@ -242,11 +262,24 @@ func (c *DataChannel) read(ctx context.Context) { case r.buf <- elm.GetData(): case <-r.done: r.completed = true + close(r.buf) } } } } +type errReader struct { + err error +} + +func (r *errReader) Read(_ []byte) (int, error) { + return 0, r.err +} + +func (r *errReader) Close() error { + return r.err +} + func (c *DataChannel) makeReader(ctx context.Context, id clientID) *dataReader { c.mu.Lock() defer c.mu.Unlock() @@ -286,6 +319,7 @@ type dataReader struct { cur []byte channel *DataChannel completed bool + err error } func (r *dataReader) Close() error { @@ -298,7 +332,11 @@ func (r *dataReader) Read(buf []byte) (int, error) { if r.cur == nil { b, ok := <-r.buf if !ok { - return 0, io.EOF + log.Errorf(context.TODO(), "dataReader.Read %v channel closed: %v", r.id, r.err) + if r.err == nil { + return 0, io.EOF + } + return 0, r.err } r.cur = b } @@ -378,7 +416,7 @@ func (w *dataWriter) Write(p []byte) (n int, err error) { l := len(w.buf) // We can't fit this message into the buffer. We need to flush the buffer if err := w.Flush(); err != nil { - return 0, errors.Wrapf(err, "datamgr.go: error flushing buffer of length %d", l) + return 0, errors.Wrapf(err, "datamgr.go [%v]: error flushing buffer of length %d", w.id, l) } } diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index 1bbf22ea03614..de4a7ee54fff8 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -17,6 +17,7 @@ package harness import ( "context" + "fmt" "io" "io/ioutil" "log" @@ -25,10 +26,13 @@ import ( pb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1" ) +const extraData = 2 + type fakeClient struct { t *testing.T done chan bool calls int + err error } func (f *fakeClient) Recv() (*pb.Elements, error) { @@ -42,7 +46,8 @@ func (f *fakeClient) Recv() (*pb.Elements, error) { msg := pb.Elements{} - for i := 0; i < bufElements+1; i++ { + // Send extraData more than the number of elements buffered in the channel. + for i := 0; i < bufElements+extraData; i++ { msg.Data = append(msg.Data, &elemData) } @@ -51,16 +56,16 @@ func (f *fakeClient) Recv() (*pb.Elements, error) { // Subsequent calls return no data. switch f.calls { case 1: - return &msg, nil + return &msg, f.err case 2: - return &msg, nil + return &msg, f.err case 3: elemData.Data = []byte{} msg.Data = []*pb.Elements_Data{&elemData} // Broadcasting done here means that this code providing messages // has not been blocked by the bug blocking the dataReader // from getting more messages. - return &msg, nil + return &msg, f.err default: f.done <- true return nil, io.EOF @@ -74,24 +79,102 @@ func (f *fakeClient) Send(*pb.Elements) error { func TestDataChannelTerminateOnClose(t *testing.T) { // The logging of channels closed is quite noisy for this test log.SetOutput(ioutil.Discard) - done := make(chan bool, 1) - client := &fakeClient{t: t, done: done} - c := makeDataChannel(context.Background(), "id", client) - - r := c.OpenRead(context.Background(), "ptr", "inst_ref") - var read = make([]byte, 4) - - // We don't read up all the buffered data, but immediately close the reader. - // Previously, since nothing was consuming the incoming gRPC data, the whole - // data channel would get stuck, and the client.Recv() call was eventually - // no longer called. - _, err := r.Read(read) - if err != nil { - t.Errorf("Unexpected error from read: %v", err) + + tests := []struct { + name string + caseFn func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) + }{ + { + name: "onClose", + caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { + // We don't read up all the buffered data, but immediately close the reader. + // Previously, since nothing was consuming the incoming gRPC data, the whole + // data channel would get stuck, and the client.Recv() call was eventually + // no longer called. + r.Close() + + // If done is signaled, that means client.Recv() has been called to flush the + // channel, meaning consumer code isn't stuck. + <-client.done + i := 1 // For the earlier Read. + for { + read := make([]byte, 4) + if n, err := r.Read(read); err != io.EOF { + i++ + // There are bufElements+extraData available from the fakeClient, so drain the channel + // before marking a test failure. + if i > bufElements+extraData { + t.Errorf("Unexpected error from read: %v, read %d bytes, %v", err, n, read) + } + continue + } + break + } + // This Test can hang on failure. + }, + }, { + name: "onRecvError", + caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { + // Set the 3rd Recv call to have an error. + // The SDK starts reading in a goroutine immeadiately after open. + expectedError := fmt.Errorf("EXPECTED ERROR") + client.err = expectedError + t.Log("ERROR SET") + + i := 1 // For the earlier Read. + for { + read := make([]byte, 4) + i++ + if n, err := r.Read(read); err != expectedError { + // There are bufElements+extraData available from the fakeClient, so drain the channel + // before marking a test failure. + if i > bufElements+extraData { + t.Errorf("Unexpected error from read %d: %v, read %d bytes, %v", i, err, n, read) + } + continue + } + break + } + t.Logf("exited after %d Read calls", i) + + // Verify that new readers return errors on their reads after a client.Recv error. + if n, err := c.OpenRead(context.Background(), "ptr", "inst_ref").Read(make([]byte, 4)); err != expectedError { + t.Errorf("Unexpected error from read: got %v, want, %v read %d bytes.", err, expectedError, n) + } + }, + }, { + name: "onSentinel", + caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { + i := 1 // For the earlier Read. + for { + read := make([]byte, 4) + i++ + if n, err := r.Read(read); err == io.EOF { + break + } else if err != nil { + t.Errorf("Unexpected error from read %d: %v, read %d bytes, %v", i, err, n, read) + } + continue + } + t.Logf("exited after %d Read calls", i) + // This Test can hang on failure. + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + done := make(chan bool, 1) + client := &fakeClient{t: t, done: done} + c := makeDataChannel(context.Background(), "id", client) + + r := c.OpenRead(context.Background(), "ptr", "inst_ref") + + n, err := r.Read(make([]byte, 4)) + if err != nil { + t.Errorf("Unexpected error from read: %v, read %d bytes.", err, n) + } + test.caseFn(t, r, client, c) + }) } - r.Close() - // If done is signaled, that means client.Recv() has been called to flush the - // channel, meaning consumer code isn't stuck. - <-done } From e9c7f99ba76abf94909f3900a9b58926ba688e0f Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 24 Oct 2019 17:45:08 -0700 Subject: [PATCH 50/57] [Go SDK] Simplify datamgr tests. --- .../beam/core/runtime/harness/datamgr_test.go | 91 +++++++------------ 1 file changed, 31 insertions(+), 60 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index de4a7ee54fff8..b82785e172b44 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -76,16 +76,20 @@ func (f *fakeClient) Send(*pb.Elements) error { return nil } -func TestDataChannelTerminateOnClose(t *testing.T) { +func TestDataChannelTerminate(t *testing.T) { // The logging of channels closed is quite noisy for this test log.SetOutput(ioutil.Discard) + expectedError := fmt.Errorf("EXPECTED ERROR") + tests := []struct { - name string - caseFn func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) + name string + expectedError error + caseFn func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) }{ { - name: "onClose", + name: "onClose", + expectedError: io.EOF, caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { // We don't read up all the buffered data, but immediately close the reader. // Previously, since nothing was consuming the incoming gRPC data, the whole @@ -96,68 +100,20 @@ func TestDataChannelTerminateOnClose(t *testing.T) { // If done is signaled, that means client.Recv() has been called to flush the // channel, meaning consumer code isn't stuck. <-client.done - i := 1 // For the earlier Read. - for { - read := make([]byte, 4) - if n, err := r.Read(read); err != io.EOF { - i++ - // There are bufElements+extraData available from the fakeClient, so drain the channel - // before marking a test failure. - if i > bufElements+extraData { - t.Errorf("Unexpected error from read: %v, read %d bytes, %v", err, n, read) - } - continue - } - break - } - // This Test can hang on failure. }, }, { - name: "onRecvError", + name: "onSentinel", + expectedError: io.EOF, caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { - // Set the 3rd Recv call to have an error. - // The SDK starts reading in a goroutine immeadiately after open. - expectedError := fmt.Errorf("EXPECTED ERROR") - client.err = expectedError - t.Log("ERROR SET") - - i := 1 // For the earlier Read. - for { - read := make([]byte, 4) - i++ - if n, err := r.Read(read); err != expectedError { - // There are bufElements+extraData available from the fakeClient, so drain the channel - // before marking a test failure. - if i > bufElements+extraData { - t.Errorf("Unexpected error from read %d: %v, read %d bytes, %v", i, err, n, read) - } - continue - } - break - } - t.Logf("exited after %d Read calls", i) - - // Verify that new readers return errors on their reads after a client.Recv error. - if n, err := c.OpenRead(context.Background(), "ptr", "inst_ref").Read(make([]byte, 4)); err != expectedError { - t.Errorf("Unexpected error from read: got %v, want, %v read %d bytes.", err, expectedError, n) - } + // fakeClient eventually returns a sentinel element. }, }, { - name: "onSentinel", + name: "onRecvError", + expectedError: expectedError, caseFn: func(t *testing.T, r io.ReadCloser, client *fakeClient, c *DataChannel) { - i := 1 // For the earlier Read. - for { - read := make([]byte, 4) - i++ - if n, err := r.Read(read); err == io.EOF { - break - } else if err != nil { - t.Errorf("Unexpected error from read %d: %v, read %d bytes, %v", i, err, n, read) - } - continue - } - t.Logf("exited after %d Read calls", i) - // This Test can hang on failure. + // The SDK starts reading in a goroutine immeadiately after open. + // Set the 2nd Recv call to have an error. + client.err = expectedError }, }, } @@ -174,6 +130,21 @@ func TestDataChannelTerminateOnClose(t *testing.T) { t.Errorf("Unexpected error from read: %v, read %d bytes.", err, n) } test.caseFn(t, r, client, c) + // Drain the reader. + i := 1 // For the earlier Read. + for err == nil { + read := make([]byte, 4) + _, err = r.Read(read) + i++ + } + + if got, want := err, test.expectedError; got != want { + t.Errorf("Unexpected error from read %d: got %v, want %v", i, got, want) + } + // Verify that new readers return the same their reads after client.Recv is done. + if n, err := c.OpenRead(context.Background(), "ptr", "inst_ref").Read(make([]byte, 4)); err != test.expectedError { + t.Errorf("Unexpected error from read: got %v, want, %v read %d bytes.", err, test.expectedError, n) + } }) } From 01c355ba47bee4ee6174c578729554b58c052fb7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Oct 2019 20:46:58 -0700 Subject: [PATCH 51/57] Make BigQueryUtils, PubsubUtils, GcsUtils implementations of TableFactory; respect truncateTimestamps option --- ...ryUtils.java => BigQueryTableFactory.java} | 32 +++-- .../datacatalog/ChainedTableFactory.java | 50 ++++++++ .../DataCatalogPipelineOptions.java | 4 +- .../datacatalog/DataCatalogTableProvider.java | 114 ++++++++---------- .../{GcsUtils.java => GcsTableFactory.java} | 29 +++-- ...bsubUtils.java => PubsubTableFactory.java} | 26 ++-- .../provider/datacatalog/TableFactory.java | 37 ++++++ 7 files changed, 196 insertions(+), 96 deletions(-) rename sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/{BigQueryUtils.java => BigQueryTableFactory.java} (65%) create mode 100644 sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/ChainedTableFactory.java rename sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/{GcsUtils.java => GcsTableFactory.java} (77%) rename sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/{PubsubUtils.java => PubsubTableFactory.java} (71%) create mode 100644 sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableFactory.java diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryUtils.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryTableFactory.java similarity index 65% rename from sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryUtils.java rename to sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryTableFactory.java index c199ed072a59f..2e87aacc2443a 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryUtils.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/BigQueryTableFactory.java @@ -20,23 +20,39 @@ import com.alibaba.fastjson.JSONObject; import com.google.cloud.datacatalog.Entry; import java.net.URI; +import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.Table.Builder; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; -/** Utils to extract BQ-specific entry information. */ -class BigQueryUtils { +/** {@link TableFactory} that understands Data Catalog BigQuery entries. */ +class BigQueryTableFactory implements TableFactory { + private static final String BIGQUERY_API = "bigquery.googleapis.com"; private static final Pattern BQ_PATH_PATTERN = Pattern.compile( "/projects/(?[^/]+)/datasets/(?[^/]+)/tables/(?[^/]+)"); - static Table.Builder tableBuilder(Entry entry) { - return Table.builder() - .location(getLocation(entry)) - .properties(new JSONObject()) - .type("bigquery") - .comment(""); + private final boolean truncateTimestamps; + + public BigQueryTableFactory(boolean truncateTimestamps) { + this.truncateTimestamps = truncateTimestamps; + } + + @Override + public Optional tableBuilder(Entry entry) { + if (!URI.create(entry.getLinkedResource()).getAuthority().toLowerCase().equals(BIGQUERY_API)) { + return Optional.empty(); + } + + return Optional.of( + Table.builder() + .location(getLocation(entry)) + .properties(new JSONObject(ImmutableMap.of("truncateTimestamps", truncateTimestamps))) + .type("bigquery") + .comment("")); } private static String getLocation(Entry entry) { diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/ChainedTableFactory.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/ChainedTableFactory.java new file mode 100644 index 0000000000000..0aaf9945f640a --- /dev/null +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/ChainedTableFactory.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.sdk.extensions.sql.meta.provider.datacatalog; + +import com.google.cloud.datacatalog.Entry; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import org.apache.beam.sdk.extensions.sql.meta.Table; + +/** {@link TableFactory} that uses the first applicable sub-{@link TableFactory}. */ +class ChainedTableFactory implements TableFactory { + + private final List subTableFactories; + + public static ChainedTableFactory of(TableFactory... subTableFactories) { + return new ChainedTableFactory(Arrays.asList(subTableFactories)); + } + + private ChainedTableFactory(List subTableFactories) { + this.subTableFactories = subTableFactories; + } + + /** Creates a Beam SQL table description from a GCS fileset entry. */ + @Override + public Optional tableBuilder(Entry entry) { + for (TableFactory tableFactory : subTableFactories) { + Optional tableBuilder = tableFactory.tableBuilder(entry); + if (tableBuilder.isPresent()) { + return tableBuilder; + } + } + return Optional.empty(); + } +} diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java index 6818691a32f67..0b9d3b7f201b4 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogPipelineOptions.java @@ -37,7 +37,7 @@ public interface DataCatalogPipelineOptions extends PipelineOptions { @Description("Truncate sub-millisecond precision timestamps in tables described by Data Catalog") @Validation.Required @Default.Boolean(false) - Boolean getTruncateTimestamps(); + boolean getTruncateTimestamps(); - void setTruncateTimestamps(Boolean newValue); + void setTruncateTimestamps(boolean newValue); } diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java index ee79b861ab8c3..359b3c86982fe 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java @@ -27,9 +27,9 @@ import io.grpc.Status; import io.grpc.StatusRuntimeException; import io.grpc.auth.MoreCallCredentials; -import java.net.URI; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.stream.Stream; import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -43,79 +43,35 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; /** Uses DataCatalog to get the source type and schema for a table. */ public class DataCatalogTableProvider extends FullNameTableProvider { - private static final Map TABLE_FACTORIES = - ImmutableMap.builder() - .put("bigquery.googleapis.com", BigQueryUtils::tableBuilder) - .put("pubsub.googleapis.com", PubsubUtils::tableBuilder) - .build(); - private final Map delegateProviders; - private final DataCatalogBlockingStub dataCatalog; - private final boolean truncateTimestamps; + private static final TableFactory PUBSUB_TABLE_FACTORY = new PubsubTableFactory(); + private static final TableFactory GCS_TABLE_FACTORY = new GcsTableFactory(); + + private static final Map DELEGATE_PROVIDERS = + Stream.of(new PubsubJsonTableProvider(), new BigQueryTableProvider(), new TextTableProvider()) + .collect(toMap(TableProvider::getTableType, p -> p)); + private final DataCatalogBlockingStub dataCatalog; private final Map tableCache; + private final TableFactory tableFactory; private DataCatalogTableProvider( - Map delegateProviders, - DataCatalogBlockingStub dataCatalog, - boolean truncateTimestamps) { + DataCatalogBlockingStub dataCatalog, boolean truncateTimestamps) { this.tableCache = new HashMap<>(); - this.delegateProviders = ImmutableMap.copyOf(delegateProviders); this.dataCatalog = dataCatalog; - this.truncateTimestamps = truncateTimestamps; + this.tableFactory = + ChainedTableFactory.of( + PUBSUB_TABLE_FACTORY, GCS_TABLE_FACTORY, new BigQueryTableFactory(truncateTimestamps)); } public static DataCatalogTableProvider create(DataCatalogPipelineOptions options) { return new DataCatalogTableProvider( - getSupportedProviders(), createDataCatalogClient(options), options.getTruncateTimestamps()); - } - - private static DataCatalogBlockingStub createDataCatalogClient( - DataCatalogPipelineOptions options) { - return DataCatalogGrpc.newBlockingStub( - ManagedChannelBuilder.forTarget(options.getDataCatalogEndpoint()).build()) - .withCallCredentials( - MoreCallCredentials.from(options.as(GcpOptions.class).getGcpCredential())); - } - - private static Map getSupportedProviders() { - return Stream.of( - new PubsubJsonTableProvider(), new BigQueryTableProvider(), new TextTableProvider()) - .collect(toMap(TableProvider::getTableType, p -> p)); - } - - static Table toBeamTable(String tableName, Entry entry) { - if (entry.getSchema().getColumnsCount() == 0) { - throw new UnsupportedOperationException( - "Entry doesn't have a schema. Please attach a schema to '" - + tableName - + "' in Data Catalog: " - + entry.toString()); - } - Schema schema = SchemaUtils.fromDataCatalog(entry.getSchema()); - - String service = URI.create(entry.getLinkedResource()).getAuthority().toLowerCase(); - - Table.Builder table = null; - if (TABLE_FACTORIES.containsKey(service)) { - table = TABLE_FACTORIES.get(service).tableBuilder(entry); - } - - if (GcsUtils.isGcs(entry)) { - table = GcsUtils.tableBuilder(entry); - } - - if (table != null) { - return table.schema(schema).name(tableName).build(); - } - - throw new UnsupportedOperationException( - "Unsupported SQL source kind: " + entry.getLinkedResource()); + createDataCatalogClient(options), options.getTruncateTimestamps()); } @Override @@ -159,6 +115,11 @@ public Map getTables() { return loadTable(fullEscapedTableName); } + @Override + public BeamSqlTable buildBeamSqlTable(Table table) { + return DELEGATE_PROVIDERS.get(table.getType()).buildBeamSqlTable(table); + } + private @Nullable Table loadTable(String tableName) { if (!tableCache.containsKey(tableName)) { tableCache.put(tableName, loadTableFromDC(tableName)); @@ -169,7 +130,7 @@ public Map getTables() { private Table loadTableFromDC(String tableName) { try { - return toBeamTable( + return toCalciteTable( tableName, dataCatalog.lookupEntry( LookupEntryRequest.newBuilder().setSqlResource(tableName).build())); @@ -181,12 +142,35 @@ private Table loadTableFromDC(String tableName) { } } - @Override - public BeamSqlTable buildBeamSqlTable(Table table) { - return delegateProviders.get(table.getType()).buildBeamSqlTable(table); + private static DataCatalogBlockingStub createDataCatalogClient( + DataCatalogPipelineOptions options) { + return DataCatalogGrpc.newBlockingStub( + ManagedChannelBuilder.forTarget(options.getDataCatalogEndpoint()).build()) + .withCallCredentials( + MoreCallCredentials.from(options.as(GcpOptions.class).getGcpCredential())); } - interface TableFactory { - Table.Builder tableBuilder(Entry entry); + private Table toCalciteTable(String tableName, Entry entry) { + if (entry.getSchema().getColumnsCount() == 0) { + throw new UnsupportedOperationException( + "Entry doesn't have a schema. Please attach a schema to '" + + tableName + + "' in Data Catalog: " + + entry.toString()); + } + Schema schema = SchemaUtils.fromDataCatalog(entry.getSchema()); + + Optional tableBuilder = tableFactory.tableBuilder(entry); + if (!tableBuilder.isPresent()) { + throw new UnsupportedOperationException( + String.format( + "Unsupported Data Catalog entry: %s", + MoreObjects.toStringHelper(entry) + .add("linkedResource", entry.getLinkedResource()) + .add("hasGcsFilesetSpec", entry.hasGcsFilesetSpec()) + .toString())); + } + + return tableBuilder.get().schema(schema).name(tableName).build(); } } diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsUtils.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsTableFactory.java similarity index 77% rename from sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsUtils.java rename to sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsTableFactory.java index d354e9d8de530..02a4a30cfc2cd 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsUtils.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/GcsTableFactory.java @@ -21,18 +21,20 @@ import com.google.cloud.datacatalog.Entry; import com.google.cloud.datacatalog.GcsFilesetSpec; import java.util.List; +import java.util.Optional; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.Table.Builder; -/** Utils to handle GCS entries from Cloud Data Catalog. */ -class GcsUtils { - - /** Check if the entry represents a GCS fileset in Data Catalog. */ - static boolean isGcs(Entry entry) { - return entry.hasGcsFilesetSpec(); - } +/** {@link TableFactory} that understands Data Catalog GCS entries. */ +class GcsTableFactory implements TableFactory { /** Creates a Beam SQL table description from a GCS fileset entry. */ - static Table.Builder tableBuilder(Entry entry) { + @Override + public Optional tableBuilder(Entry entry) { + if (!entry.hasGcsFilesetSpec()) { + return Optional.empty(); + } + GcsFilesetSpec gcsFilesetSpec = entry.getGcsFilesetSpec(); List filePatterns = gcsFilesetSpec.getFilePatternsList(); @@ -50,10 +52,11 @@ static Table.Builder tableBuilder(Entry entry) { + "Only file patterns with 'gs://' schema are supported at the moment."); } - return Table.builder() - .type("text") - .location(filePattern) - .properties(new JSONObject()) - .comment(""); + return Optional.of( + Table.builder() + .type("text") + .location(filePattern) + .properties(new JSONObject()) + .comment("")); } } diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubUtils.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubTableFactory.java similarity index 71% rename from sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubUtils.java rename to sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubTableFactory.java index 856eec9d1ea41..5a8f6e53d38b1 100644 --- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubUtils.java +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/PubsubTableFactory.java @@ -20,22 +20,32 @@ import com.alibaba.fastjson.JSONObject; import com.google.cloud.datacatalog.Entry; import java.net.URI; +import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.Table.Builder; -/** Utils to extract Pubsub-specific entry information. */ -class PubsubUtils { +/** {@link TableFactory} that understands Data Catalog Pubsub entries. */ +class PubsubTableFactory implements TableFactory { + + private static final String PUBSUB_API = "pubsub.googleapis.com"; private static final Pattern PS_PATH_PATTERN = Pattern.compile("/projects/(?[^/]+)/topics/(?[^/]+)"); - static Table.Builder tableBuilder(Entry entry) { - return Table.builder() - .location(getLocation(entry)) - .properties(new JSONObject()) - .type("pubsub") - .comment(""); + @Override + public Optional tableBuilder(Entry entry) { + if (!URI.create(entry.getLinkedResource()).getAuthority().toLowerCase().equals(PUBSUB_API)) { + return Optional.empty(); + } + + return Optional.of( + Table.builder() + .location(getLocation(entry)) + .properties(new JSONObject()) + .type("pubsub") + .comment("")); } private static String getLocation(Entry entry) { diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableFactory.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableFactory.java new file mode 100644 index 0000000000000..a2a230a5805ab --- /dev/null +++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/TableFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.meta.provider.datacatalog; + +import com.google.cloud.datacatalog.Entry; +import java.util.Optional; +import org.apache.beam.sdk.extensions.sql.meta.Table; + +/** + * A {@link TableFactory} may be able to interpret a given Data Catalog {@link Entry} into + * Beam SQL {@link Table}. + */ +interface TableFactory { + + /** + * If this {@link TableFactory} instance can interpret the given {@link Entry}, then a Beam SQL + * {@link Table} is constructed, else returns {@link Optional#empty}. + * + *

The {@link Table} is returned as a builder for further customization by the caller. + */ + Optional tableBuilder(Entry entry); +} From 9c0a628b9dc519d24710ebdc551a8dff33b2ff37 Mon Sep 17 00:00:00 2001 From: Craig Chambers Date: Fri, 25 Oct 2019 12:32:26 -0700 Subject: [PATCH 52/57] In Go, base counter names on PTransform unique_name --- sdks/go/pkg/beam/core/runtime/exec/translate.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index e6caafc6c01cf..139db34e13cdc 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -17,7 +17,6 @@ package exec import ( "fmt" - "path" "strconv" "strings" @@ -382,9 +381,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { if err != nil { return nil, err } - // transform.UniqueName may be per-bundle, which isn't useful for metrics. - // Use the short name for the DoFn instead. - n.PID = path.Base(n.Fn.Name()) + n.PID = transform.GetUniqueName() input := unmarshalKeyedValues(transform.GetInputs()) for i := 1; i < len(input); i++ { @@ -414,9 +411,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { } cn.UsesKey = typex.IsKV(in[0].Type) - // transform.UniqueName may be per-bundle, which isn't useful for metrics. - // Use the short name for the DoFn instead. - cn.PID = path.Base(cn.Fn.Name()) + cn.PID = transform.GetUniqueName() switch urn { case urnPerKeyCombinePre: From 4b5f7e5eeb14493e820346aef01bb0e8fc784e0b Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 25 Oct 2019 16:37:22 -0700 Subject: [PATCH 53/57] Merge #9858 for [BEAM-8446] Adding a test checking the wait for BQ jobs (#9858) * Adding a test checking the wait for BQ jobs * Fix lint issue * Skip tests in py2 --- .../io/gcp/bigquery_file_loads_test.py | 85 +++++++++++++++++++ sdks/python/apache_beam/testing/util.py | 5 +- 2 files changed, 89 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py index 3a1c1eb520dc1..035be18d9d1a9 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py @@ -23,6 +23,7 @@ import logging import os import random +import sys import time import unittest @@ -424,6 +425,90 @@ def test_records_traverse_transform_with_mocks(self): assert_that(jobs, equal_to([job_reference]), label='CheckJobs') + @unittest.skipIf(sys.version_info[0] == 2, + 'Mock pickling problems in Py 2') + @mock.patch('time.sleep') + def test_wait_for_job_completion(self, sleep_mock): + job_references = [bigquery_api.JobReference(), + bigquery_api.JobReference()] + job_references[0].projectId = 'project1' + job_references[0].jobId = 'jobId1' + job_references[1].projectId = 'project1' + job_references[1].jobId = 'jobId2' + + job_1_waiting = mock.Mock() + job_1_waiting.status.state = 'RUNNING' + job_2_done = mock.Mock() + job_2_done.status.state = 'DONE' + job_2_done.status.errorResult = None + + job_1_done = mock.Mock() + job_1_done.status.state = 'DONE' + job_1_done.status.errorResult = None + + bq_client = mock.Mock() + bq_client.jobs.Get.side_effect = [ + job_1_waiting, + job_2_done, + job_1_done, + job_2_done] + + waiting_dofn = bqfl.WaitForBQJobs(bq_client) + + dest_list = [(i, job) for i, job in enumerate(job_references)] + + with TestPipeline('DirectRunner') as p: + references = beam.pvalue.AsList(p | 'job_ref' >> beam.Create(dest_list)) + outputs = (p + | beam.Create(['']) + | beam.ParDo(waiting_dofn, references)) + + assert_that(outputs, + equal_to(dest_list)) + + sleep_mock.assert_called_once() + + @unittest.skipIf(sys.version_info[0] == 2, + 'Mock pickling problems in Py 2') + @mock.patch('time.sleep') + def test_one_job_failed_after_waiting(self, sleep_mock): + job_references = [bigquery_api.JobReference(), + bigquery_api.JobReference()] + job_references[0].projectId = 'project1' + job_references[0].jobId = 'jobId1' + job_references[1].projectId = 'project1' + job_references[1].jobId = 'jobId2' + + job_1_waiting = mock.Mock() + job_1_waiting.status.state = 'RUNNING' + job_2_done = mock.Mock() + job_2_done.status.state = 'DONE' + job_2_done.status.errorResult = None + + job_1_error = mock.Mock() + job_1_error.status.state = 'DONE' + job_1_error.status.errorResult = 'Some problems happened' + + bq_client = mock.Mock() + bq_client.jobs.Get.side_effect = [ + job_1_waiting, + job_2_done, + job_1_error, + job_2_done] + + waiting_dofn = bqfl.WaitForBQJobs(bq_client) + + dest_list = [(i, job) for i, job in enumerate(job_references)] + + with self.assertRaises(Exception): + with TestPipeline('DirectRunner') as p: + references = beam.pvalue.AsList(p | 'job_ref' >> beam.Create(dest_list)) + _ = (p + | beam.Create(['']) + | beam.ParDo(waiting_dofn, references)) + + sleep_mock.assert_called_once() + def test_multiple_partition_files(self): destination = 'project1:dataset1.table1' diff --git a/sdks/python/apache_beam/testing/util.py b/sdks/python/apache_beam/testing/util.py index 6d77ee7dd7d73..b52e61bba7a93 100644 --- a/sdks/python/apache_beam/testing/util.py +++ b/sdks/python/apache_beam/testing/util.py @@ -209,7 +209,10 @@ def assert_that(actual, matcher, label='assert_that', Returns: Ignored. """ - assert isinstance(actual, pvalue.PCollection) + assert isinstance( + actual, + pvalue.PCollection), ('%s is not a supported type for Beam assert' + % type(actual)) class ReifyTimestampWindow(DoFn): def process(self, element, timestamp=DoFn.TimestampParam, From c3c5999e3a82d865810f799767c179e2c17d304b Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 25 Oct 2019 16:38:15 -0700 Subject: [PATCH 54/57] Improve user support story - and add community events link (#9875) * Point Support to Contact Us, and add Events link. * Encouraging the use of user@ and SO for questions. * Adding contact us --- website/src/_includes/footer.html | 3 ++- website/src/community/contact-us.md | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/website/src/_includes/footer.html b/website/src/_includes/footer.html index 8f4d983945ccc..2052fa922afe4 100644 --- a/website/src/_includes/footer.html +++ b/website/src/_includes/footer.html @@ -42,11 +42,12 @@ width="14" height="14" alt="External link.">

+ diff --git a/website/src/community/contact-us.md b/website/src/community/contact-us.md index de8a884fd44e3..9b79ad2030aec 100644 --- a/website/src/community/contact-us.md +++ b/website/src/community/contact-us.md @@ -25,7 +25,7 @@ limitations under the License. # Contact Us There are many ways to reach the Beam user and developer communities - use -whichever one seems best! +whichever one seems best. | How to contact us | When to use it | | ----------------- | ---------------| @@ -38,6 +38,8 @@ whichever one seems best! | [Slack](https://s.apache.org/beam-slack-channel) | Chat with users and developers in the ASF Slack. Note: Please [join the #beam channel](https://s.apache.org/beam-slack-channel) after you [created an account](https://s.apache.org/slack-invite). Please do not ask Beam questions in #general. | {:.table} +If you have questions about how to use Apache Beam, we recommend you try out the [user@](https://lists.apache.org/list.html?user@beam.apache.org) mailing list, and [StackOverflow](http://stackoverflow.com/questions/tagged/apache-beam). + [^1]: To subscribe or unsubscribe, a blank email is fine. If you wish to report a security vulnerability, please contact [security@apache.org](mailto:security@apache.org). Apache Beam follows the typical [Apache vulnerability handling process](https://apache.org/security/committers.html#vulnerability-handling). From 59e11f72e78e29da146cda83f199099c16fd3ebd Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 25 Oct 2019 16:44:55 -0700 Subject: [PATCH 55/57] Job timeout after 120 minutes instead of 100 --- .test-infra/jenkins/job_PostCommit_Python37.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/job_PostCommit_Python37.groovy b/.test-infra/jenkins/job_PostCommit_Python37.groovy index ea511cd7746b2..faa56833e0988 100644 --- a/.test-infra/jenkins/job_PostCommit_Python37.groovy +++ b/.test-infra/jenkins/job_PostCommit_Python37.groovy @@ -27,7 +27,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python37', 'Run Python 3.7 P previousNames('/beam_PostCommit_Python3_Verify/') // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) + commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) publishers { archiveJunit('**/nosetests*.xml') From 501df3a3a91889d83870d4c27e01d02cbb782218 Mon Sep 17 00:00:00 2001 From: Daniel Oliveira Date: Fri, 25 Oct 2019 16:53:49 -0700 Subject: [PATCH 56/57] [BEAM-8493] Add standard double coder to Go SDK. For upcoming features (in this case, SDF), we need to support the standard double coder from beam_runner_api.proto. This commit adds relevant support. --- sdks/go/pkg/beam/coder.go | 6 ++- sdks/go/pkg/beam/core/graph/coder/coder.go | 6 +++ sdks/go/pkg/beam/core/graph/coder/double.go | 41 +++++++++++++++++++ sdks/go/pkg/beam/core/runtime/exec/coder.go | 24 +++++++++++ .../pkg/beam/core/runtime/exec/coder_test.go | 3 ++ sdks/go/pkg/beam/core/runtime/graphx/coder.go | 7 ++++ .../beam/core/runtime/graphx/coder_test.go | 4 ++ .../pkg/beam/core/runtime/graphx/dataflow.go | 7 ++++ sdks/go/pkg/beam/create_test.go | 3 +- 9 files changed, 99 insertions(+), 2 deletions(-) create mode 100644 sdks/go/pkg/beam/core/graph/coder/double.go diff --git a/sdks/go/pkg/beam/coder.go b/sdks/go/pkg/beam/coder.go index bfdf1e75edcdc..dbca4364c822f 100644 --- a/sdks/go/pkg/beam/coder.go +++ b/sdks/go/pkg/beam/coder.go @@ -151,13 +151,17 @@ func inferCoder(t FullType) (*coder.Coder, error) { return nil, err } return &coder.Coder{Kind: coder.Custom, T: t, Custom: c}, nil - case reflectx.Float32, reflectx.Float64: + + case reflectx.Float32: c, err := coderx.NewFloat(t.Type()) if err != nil { return nil, err } return &coder.Coder{Kind: coder.Custom, T: t, Custom: c}, nil + case reflectx.Float64: + return &coder.Coder{Kind: coder.Double, T: t}, nil + case reflectx.String: c, err := coderx.NewString() if err != nil { diff --git a/sdks/go/pkg/beam/core/graph/coder/coder.go b/sdks/go/pkg/beam/core/graph/coder/coder.go index 8bc1a48c8e429..61ebdc6f3d379 100644 --- a/sdks/go/pkg/beam/core/graph/coder/coder.go +++ b/sdks/go/pkg/beam/core/graph/coder/coder.go @@ -161,6 +161,7 @@ const ( Bytes Kind = "bytes" // Implicitly length-prefixed as part of the encoding Bool Kind = "bool" VarInt Kind = "varint" + Double Kind = "double" WindowedValue Kind = "W" KV Kind = "KV" @@ -256,6 +257,11 @@ func NewVarInt() *Coder { return &Coder{Kind: VarInt, T: typex.New(reflectx.Int64)} } +// NewDouble returns a new double coder using the built-in scheme. +func NewDouble() *Coder { + return &Coder{Kind: Double, T: typex.New(reflectx.Float64)} +} + // IsW returns true iff the coder is for a WindowedValue. func IsW(c *Coder) bool { return c.Kind == WindowedValue diff --git a/sdks/go/pkg/beam/core/graph/coder/double.go b/sdks/go/pkg/beam/core/graph/coder/double.go new file mode 100644 index 0000000000000..bb47afe5f42fb --- /dev/null +++ b/sdks/go/pkg/beam/core/graph/coder/double.go @@ -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 coder + +import ( + "encoding/binary" + "io" + "math" + + "github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx" +) + +// EncodeDouble encodes a float64 in big endian format. +func EncodeDouble(value float64, w io.Writer) error { + var data [8]byte + binary.BigEndian.PutUint64(data[:], math.Float64bits(value)) + _, err := ioutilx.WriteUnsafe(w, data[:]) + return err +} + +// DecodeDouble decodes a float64 in big endian format. +func DecodeDouble(r io.Reader) (float64, error) { + var data [8]byte + if err := ioutilx.ReadNBufUnsafe(r, data[:]); err != nil { + return 0, err + } + return math.Float64frombits(binary.BigEndian.Uint64(data[:])), nil +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 9f891754c42dd..0ef72607fbd21 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -72,6 +72,9 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder { case coder.VarInt: return &varIntEncoder{} + case coder.Double: + return &doubleEncoder{} + case coder.Custom: return &customEncoder{ t: c.Custom.Type, @@ -102,6 +105,9 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder { case coder.VarInt: return &varIntDecoder{} + case coder.Double: + return &doubleDecoder{} + case coder.Custom: return &customDecoder{ t: c.Custom.Type, @@ -204,6 +210,24 @@ func (*varIntDecoder) Decode(r io.Reader) (*FullValue, error) { return &FullValue{Elm: n}, nil } +type doubleEncoder struct{} + +func (*doubleEncoder) Encode(val *FullValue, w io.Writer) error { + // Encoding: beam double (big-endian 64-bit IEEE 754 double) + return coder.EncodeDouble(val.Elm.(float64), w) +} + +type doubleDecoder struct{} + +func (*doubleDecoder) Decode(r io.Reader) (*FullValue, error) { + // Encoding: beam double (big-endian 64-bit IEEE 754 double) + f, err := coder.DecodeDouble(r) + if err != nil { + return nil, err + } + return &FullValue{Elm: f}, nil +} + type customEncoder struct { t reflect.Type enc Encoder diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder_test.go b/sdks/go/pkg/beam/core/runtime/exec/coder_test.go index 1aa7258f9f8f1..4f663fa5b58c9 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder_test.go @@ -41,6 +41,9 @@ func TestCoders(t *testing.T) { }, { coder: coder.NewVarInt(), val: &FullValue{Elm: int64(65)}, + }, { + coder: coder.NewDouble(), + val: &FullValue{Elm: float64(12.9)}, }, { coder: func() *coder.Coder { c, _ := coderx.NewString() diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder.go b/sdks/go/pkg/beam/core/runtime/graphx/coder.go index 8033f6af3da1b..8c1cec4447f20 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder.go @@ -34,6 +34,7 @@ const ( urnBytesCoder = "beam:coder:bytes:v1" urnBoolCoder = "beam:coder:bool:v1" urnVarIntCoder = "beam:coder:varint:v1" + urnDoubleCoder = "beam:coder:double:v1" urnLengthPrefixCoder = "beam:coder:length_prefix:v1" urnKVCoder = "beam:coder:kv:v1" urnIterableCoder = "beam:coder:iterable:v1" @@ -162,6 +163,9 @@ func (b *CoderUnmarshaller) makeCoder(c *pb.Coder) (*coder.Coder, error) { case urnVarIntCoder: return coder.NewVarInt(), nil + case urnDoubleCoder: + return coder.NewDouble(), nil + case urnKVCoder: if len(components) != 2 { return nil, errors.Errorf("could not unmarshal KV coder from %v, want exactly 2 components but have %d", c, len(components)) @@ -377,6 +381,9 @@ func (b *CoderMarshaller) Add(c *coder.Coder) string { case coder.VarInt: return b.internBuiltInCoder(urnVarIntCoder) + case coder.Double: + return b.internBuiltInCoder(urnDoubleCoder) + default: panic(fmt.Sprintf("Failed to marshal custom coder %v, unexpected coder kind: %v", c, c.Kind)) } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder_test.go b/sdks/go/pkg/beam/core/runtime/graphx/coder_test.go index 1d3f04b358087..2a99df61d0bed 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder_test.go @@ -53,6 +53,10 @@ func TestMarshalUnmarshalCoders(t *testing.T) { "varint", coder.NewVarInt(), }, + { + "double", + coder.NewDouble(), + }, { "foo", foo, diff --git a/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go b/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go index 8d7712a3af462..da6d9b520e777 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go @@ -42,6 +42,7 @@ const ( bytesType = "kind:bytes" boolType = "kind:bool" varIntType = "kind:varint" + doubleType = "kind:double" streamType = "kind:stream" pairType = "kind:pair" lengthPrefixType = "kind:length_prefix" @@ -154,6 +155,9 @@ func EncodeCoderRef(c *coder.Coder) (*CoderRef, error) { case coder.VarInt: return &CoderRef{Type: varIntType}, nil + case coder.Double: + return &CoderRef{Type: doubleType}, nil + default: return nil, errors.Errorf("bad coder kind: %v", c.Kind) } @@ -184,6 +188,9 @@ func DecodeCoderRef(c *CoderRef) (*coder.Coder, error) { case varIntType: return coder.NewVarInt(), nil + case doubleType: + return coder.NewDouble(), nil + case pairType: if len(c.Components) != 2 { return nil, errors.Errorf("bad pair: %+v", c) diff --git a/sdks/go/pkg/beam/create_test.go b/sdks/go/pkg/beam/create_test.go index c9dee7d082a1e..a9da52468ceb8 100644 --- a/sdks/go/pkg/beam/create_test.go +++ b/sdks/go/pkg/beam/create_test.go @@ -35,7 +35,8 @@ func TestCreate(t *testing.T) { }{ {[]interface{}{1, 2, 3}}, {[]interface{}{"1", "2", "3"}}, - {[]interface{}{0.1, 0.2, 0.3}}, + {[]interface{}{float32(0.1), float32(0.2), float32(0.3)}}, + {[]interface{}{float64(0.1), float64(0.2), float64(0.3)}}, {[]interface{}{uint(1), uint(2), uint(3)}}, {[]interface{}{false, true, true, false, true}}, {[]interface{}{wc{"a", 23}, wc{"b", 42}, wc{"c", 5}}}, From 6418fb6c4852ba4e07257c0b04c36d6fb9b28815 Mon Sep 17 00:00:00 2001 From: Ryan Skraba Date: Thu, 24 Oct 2019 16:23:52 +0200 Subject: [PATCH 57/57] Move sequence diagrams to PlantUML SVG. The plantuml source used to generate the diagrams is embedded in the SVG resources and can be modified and regenerated (by hand, at this time). Remove annotation on Reader. --- .../io/developing-io-overview.md | 6 +- .../src/documentation/programming-guide.md | 4 +- website/src/images/dofn-sequence-diagram.png | Bin 126448 -> 0 bytes website/src/images/dofn-sequence-diagram.svg | 94 ++++++++++++++++ .../src/images/source-sequence-diagram.png | Bin 73657 -> 0 bytes .../src/images/source-sequence-diagram.svg | 106 ++++++++++++++++++ 6 files changed, 205 insertions(+), 5 deletions(-) delete mode 100644 website/src/images/dofn-sequence-diagram.png create mode 100644 website/src/images/dofn-sequence-diagram.svg delete mode 100644 website/src/images/source-sequence-diagram.png create mode 100644 website/src/images/source-sequence-diagram.svg diff --git a/website/src/documentation/io/developing-io-overview.md b/website/src/documentation/io/developing-io-overview.md index 56f275544b3a5..b17a71065bf9e 100644 --- a/website/src/documentation/io/developing-io-overview.md +++ b/website/src/documentation/io/developing-io-overview.md @@ -107,9 +107,9 @@ Here is a sequence diagram that shows the lifecycle of the Source during information to IO developers such as the constraints that apply to the objects or particular cases such as streaming mode. - - ![This is a sequence diagram that shows the lifecycle of the Source]( - {{ "/images/source-sequence-diagram.png" | prepend: site.baseurl }}) + +![This is a sequence diagram that shows the lifecycle of the Source]( + {{ "/images/source-sequence-diagram.svg" | prepend: site.baseurl }}) ### Using ParDo and GroupByKey diff --git a/website/src/documentation/programming-guide.md b/website/src/documentation/programming-guide.md index afb11635188a7..d78b609ef1b87 100644 --- a/website/src/documentation/programming-guide.md +++ b/website/src/documentation/programming-guide.md @@ -809,9 +809,9 @@ Here is a sequence diagram that shows the lifecycle of the DoFn during apply to the objects or particular cases such as failover or instance reuse. They also give instanciation use cases. - + ![This is a sequence diagram that shows the lifecycle of the DoFn]( - {{ "/images/dofn-sequence-diagram.png" | prepend: site.baseurl }}) + {{ "/images/dofn-sequence-diagram.svg" | prepend: site.baseurl }}) #### 4.2.2. GroupByKey {#groupbykey} diff --git a/website/src/images/dofn-sequence-diagram.png b/website/src/images/dofn-sequence-diagram.png deleted file mode 100644 index cf6570d00c730e3360b2149d8d0b85bec0ebd7b0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 126448 zcmd42bx>SQ_%%of1b252?hxGF-Q5WqAhdwr)-97!-Ip=APCy{Q@S8HH9~O&K@*V|pWHC`B5{P951!=G{Hij=G$jk|Ew)e6tR? zyk3|AtJ3K_nC^4nE!kMKiD11Zagv2GNxI}Y_UF@Ge#3oA230E4?+-*n>fM_4CNa;$ zFB$UB;Qnn95+Awe=G>fkN?wT5%Z$pVvG?>m{lU(chyFI0szcuVUF)dpcLi`&JM^7(QEt7d&BWMjA8 zzK$s0IapZdnufaBu`zDAQ)mR9tiRCv-8&Q0@MVXIzaXD+=KzrmM2K|9yNp0wZemT; z1`q^sua5@Q-&J1bHgAoB;nY8$XW&+?Go_R+l<<|mujqaX)iP}DAV`m(yGJ}~Up<|C zA7mSgW4)oX_3k8Zkkma~!GoMghqQHNhbG?Zr5``zVI1ARYv1C84c`@g+F(h)B;j|$ z%M&~YT0Z{p$5A+`$*ybL){z704JuD%+$`S~Uf9KT%A|`PNa;ahMJAFQNpk3Y6UKoG z;%jvr-Wht+yQMb9XndkmNC|W~&z~DtZSOmDz1u(ibqT3jtEPlB%69S2bwbK-PEE7n z4Tqd)c4C2(s7evuu|mk(sF3J_=CV}2>;gx%*iBUE8|A?Z_|42Tj_7i+Gsb^8aUFjj zY(rX%^t7M@don{zJg(JJ;ep~~G&<^6zW7BGivb;+GhatM>boSKF?j4A%QBW7GAcC* zw6{K#`DHh2m3(qgD-uZRp;i047FQm}eYOdAi5`Q(R zZ2RgnB#O9+SBXmFW{O03#)K>M)1Q@H3bo*VCop&qodaVHByu%%;Oa&lT5mS;KiGo51fy!U;1}cenX?bFh>dh_3b(fFg(H%C`AlgxDbD|L zp~IK5Xbz31|H3(P)SCYZ8@=rem3rsOIynMmFon^c!HqiNY&@0++pS2Z>qWP@A3=qK z{hiu3kdP6KkiyI>ig{x04l&XCfxr0e^6VT~mgE!KSkft$NY zyOl!+R=Dyo$D0Ul;`AE~h4n}yp^)Jq`EGA^5#?`#O6WdA6L`gjV0lj@^WCa0#xbGl z!7GsVl7FhDETvwR{+^FEKug#^a>A#&R8nVkf6!*fzSUZfbpJ(U(gLqL8rmfD_pFet zLcFoe{0#{z^8!wQ2^td#l1j;!b|!iHGuu(2h8+WWJ{Rmw19=7`v9u30>n?U<<}f0E zduivNNCZSwu{t-~vT4V7`er+BUe+abJ|?V~nR^B#gQii2c(6ZDUGLi?T=N)b@QN@8 z$S&ZPL=qVbveEtk?AMEBEXJ<$B}uB;DRKL7h{qX!;$B1t1kyCc5Sm9;k$b-Td8}6Y zbw;eRmMxZIBDXG@t{Ag-=_22@1l0hEeRNqH#`4sTW+g27)OvJtgB{sP6>c$bQShsF$AKco+Opd4F^(A zZw7>z&Fs^Y8IDqfM2krb(-sIgClng2=0AKB3AYOMi&uldfDxJ&DNYedTA+kM2}W&h zc)G2+WvI{Yy&dq}k<%(u3CSeX69qQ={%nQr<2aVZ1j3=A z0HdBd@wO)!lnkqhMscY$`nq*5zTxu#^xn?6d)u9{w(p4Dg)VD9JN>j^8-ANx^$+g> zrLSM8un#61cEnq>zkiS@ET)N>v=S&{hGxf=@ZGuiCMj56-PSOWYp%<@V4y$NKiL4m zMbq-J)RNCle@^aJPPF9O#n*=J3AK3D#(H{rg5!rV=H7JX1KOA-Del-F0cmI~57*NK z$(RCy*r%opl@9_55uZ#Hbc?Zbv060)_Id;PA-2Smg6y+lv$3H+vx=a(kuQJsP!q*B zJ^@?IUQTk?di|6%HqF65jH;$$vm<)J?$HXXIZ7x`4Py*ghqRd}9qi}i6XW5rAR z*{73Mdn9ui#guxJV{|Wk2>qe3?RbOlG5%f&!c8>1f!lu4j0;w|X4BRI?+qo~o~^m8>J1m|GOReY3^5#l5aJo0(man#H@#L5-?j z5qUj&xu&qs-fUI$I8D@Mwq}@{iENpXAh~mzYDC_=r?O+ z4ev3s=lZYzmbG2|OVmH~ben*oCqMDasy0cjA-px}&|dPc00+b+U0T88F%=}@j`HWb zf;c>zH#=&krJi3Eh{qfOFX3s-!fLqq26`Y!(0aA^|&7N>1_qZ!mCqA{9ic-Z6sU!Dyd#??;F4vU@ZZnVWfaryTVf9Cfbw0*P2pzBGP{z*iyzP&%{@FZbK4!^GJ5McHpD}V?|x*_HzKVz8Wv1?Lb)|w ze>Bu(4cY!_{A=s1-b`hTb&d68JZH4C@)zu8Pr>6J8FbVkX=TX$p1u}UzuDnokAfi8 z1Xef(k%`ExIWc#$*^N1A`Um_awz9vmR0F8w_&7_cbi4jNSSs8N$CcNK0`nS0=TQ1Y zcD>aQH^7ceS`6nxZ?Hx*qdGxrl|`0t910Rr7Ry5Y?4Gc49in#db#p^Y`EZ8TFba~; z`c!aH>LNmkV05Em+Wl9=s0tZO)=H3z#Ju62UwQmS6nf6V9Bh1Df$dmt-DwV*$B7fF0Ta5U)7Y7 zOGgX=9**LWg~6)E%qy(A=vk;>y6Wa(rdUtFzqFbFb(Hgq8d+iWK&Soj_oGnCtf#Eg z2@UzJQGC+%fyO$t^$&}UwzR)O7EQxj4tAxLgj&3g`o6eV(DXmQ)f=U;;;x<|sG;tP z?faWRQt=-z-(d1_XL|lI?^fgHn?|`xhPf<(r-(~bXe~B~Y~Q_rrKi9e3u5!PCXcVX zkzI!RY-WH{K}v9|NT`q}v#a2#GM^C}_KlSMhjxeMcjR(}R_6gaG6J`SPZqtEI z-Nx(^QBi#~8GU_`MtnsgWpfx71T0!61XjPF`s1wfX0g+!+Vb&FE8ILk!l2wdhfr^u zf@i~EN(L#5l4#gYTT`Pk(`D?|u#&UV{fT0WQS=sxKHtlO@{J5KKT~iW zI3ZE{8pNy`KN*(@=))SovYk*q2y*qt-?S%J9o3+k=gWRy!14a*D*?9f@-@L!T^mhC z?dRlIC;f!kB~@W;7+7D6`E?vU)X^cw&thwHy2a6KmrGtg!?@satgiX)1)N1*7`5?j zXBiPx;_H93uW7;B+vVW-zDYLw_LC~tIw6af$W&r>nb};TJ?_^kG6t_5?}eep%G&zy z*fSnzd~mwL-Dz$*{UQr0Z4Qprg>1c{qA^kTc-kW1GyIM(IEKw%!L_9=UNZ~9`2J_BERUhHDE%H_**6?fot95!|9P{>*wU!g_#TWNL`RxxlfC zl}y%K@M86%yQl?!=ThU+i)9$Cf(dSnze+da1>!tSwKawj#KFH{@O}}@FCbg;LgvqJ zN@R7u@S;z$=+u%bYC5yk3&}!X1w+4nbfu)&&66K(9|@6$E)xxqw824C_HEjn#OUs=%xWut|0#zypkaN3~!xE*=NJXVFwUZlA8`=8MYKF5LA4{o(RK0z$PU{m22vK|LwjJ(mS8 z)T&E5>h#BH%82l}1I1Oa-doPHYH;T9MKYA^QkQ~w(B<7>7PFu}I^Sv7d(XklY*i(x zJ%ii(nP6jO?r>WU>u{8*1!Iy#%MRxn-352Y{ST>WlPc59T=oy>TC-3Bb4u_#1ut2q zO?Iv|9N_t@GZ+@7%AGp1R@CLR7n^7E_@aoitMUtT5oISom_$!4Q0u=BWd%*;pClzU zZArU!s<&;A$%>qbhRl6cG^Q72oqm^*K%7PzI zaM{*B9NA7kpfai5)u_=SSoGaW7Mmcr8x_fAe(*rBUjX}wJs$ikeL8JP)aR!(n@HBw z6;iq!Yy&Lw$r^lmYErDFq&O+7S3^Op6f6Z%2`q|k>{Sbeyj}B04*>9~L z&Nfu99eSEtFbkx-`%)*XD_X3hf272&-iljDo$l!ZlzWrSX6X3Wt2U4J14Gr;(@xFu%@*CS zxWd0=x@hdyZzJC)!J2N`Vp3A7ZEUU;x)YM0(yfl7*>_N~=u%^{h;)n%`i5*j85?a1 z?5&4I;A0z5&=E$BbCaQ}tO`sQQBZvSG>-}K5Re?kZWb21x$$^6eWE>g>cOs%5%+V% zmP>Vy{-Ged^dtI^bSp6~J;r^7&#!5gGqW*ee&S67hSQPsNcahet{bpCR~5 z)%QC0FWLQ8tcA7R$?i&GnRV3cV_T2r?n9oUpM9p(?lKusH3RQdb_U2M5IyO-Ir{Ng zIzS`OxP)Xkxa+6r$DdwFjHS5a^Gb>h0Z|5p@sCY4Vu|b;7SPiRgnV%%<({Gow2?Kl zqn9;y#$HFnnel#W*2zY#1GU{cW4%3h-&@p=bhf*t-N_7&A|%jP2x$@#{V)5= zR!i;Mm$7tn)L6g^DB%*p3E0$~LZXk;Ae<373LTS-k`mp`XuW0dN|I+kfQ+l9`lcZd zbv6{LewCsM^kp+EV(-}^fiKM%zdupo&Jub27aFU>igr&JN)7VTNzRIxvv$r zKfVhOQlXm?G46j-5B+Ve?!S!NuvnA_hFpoXeTR?l?%)OK`AQhw=62bVcR^b}N~qyUifR>2Jse_BM<^QBc)~&pOi0d@4>*SS zf0rbdbI*&FOk}u=<~7N!g)9w%2Xn4LcucoEdC#@c+z@E3P9P5!DT%@_(88vs0AWl3!p>Y;ede=gGq@Bsk-SOes|VYoqOmL2rzby zXscKvux^}PI+{PiJ)oIg`sTmnAGNd;PH-^4WE}9QKYGS-(MIfw{RP|bcG36u;7}9g z@@U$f$MQv{6WQJU{e1uyOf1;zYZlsH(_y3IU`YoL2O(4ld>f^zwy|$jC~)=g&}JEd zP7It}@b&dckQg;M1VJ=Tg!mmQ@M1y@L#IZ4=HC$`B}dMMjvPW$iJIx@305W(7Rw5x z0PgV*lk*cMGw#d&LHYVb2AN>4krT<5-w-jeut*JS2t1B(vC#)cOp<@&RUv?*<2Im3 zLGhr8Flx?N z)!dk);YVofpb+%j$yL?2A;YL3sMtuQ^SR_-7aT_X`c5*3OI84)PB>6GFd^{YOvHJ} zIb&f|aoe0yz`Vo(ukVQcx0&Q%5u_31kzXGLeu|)=hUH?bJDT-$nb6DglaYmmg(=Ac zt3(W8)cw_g$=Q0(?o=t(dT*4ql6uWeE%3R(6p!b%4X@iN*m;avlW|XYFz~k)Bk){h z`7l2ajn5T8sgRW;|DAiksyr6EvW`+0iGVvurBHtD>R=`eh3o^1#Ykj01}&-i5D{g{ zfnE%eKyQuN;8>voky>A^#i**U@74YD6VC@k{9VfD5blKr+la4w<4;y-BrY%II*m-e z&yQda{Knm3+4EBo7ip{(v5W@%YW(CIM8?FL%rK&Yz=oqR?c%dMT3BMoetCeQg!<>+3Zh{ zV5fj{=iuO&EZ4zGV=-eXb=S(bhLkhp_>9#|#P0D`=)n3oSZ2+U@3; zz~yK@xKJTG&w8dJ44N43~zWL-K=oG=6b@v@QvbuhTt<1#r-9l$>GEHWTj2xplG*%i@sy-^Lm0TqJIVAqiHZ)6j>>a*HDOPedkLsu z>{gSCSNtBAlJyyEpOqY?k(Uw z3OKkAHz$2(8+~fYk{tn%YI;xiSAleqW{b)(An4*BU)DM~T@JDJTiyH}w+3*6BmIk+ z2nBq2o1G6r$fe@3wcrM+RSI$}|K=gbk?GQ?l>`l@Fm6nhYS1|403K1&=EuTzIUw_X zrb4ePnt*5W7L8morx%OI#YP}I>Q7LJY2ad$6Vz}9hsMn}-GOv=TIHY8!NB9RC?A>c z9`9I7G%72M9d1vJlFmq(d0XEWb*^YAbp&EgBszARTS*OtuWtPXFFRM_{u zJO6pl%Vs;TI?SvO*ld~vNEQSmHWW4}OcZQX?49amvv~hP@!cQ_6E&OZJAT`Fw#SQ6 zfiP4G*v%PVshN~W{~#DN+~D4)hnuO+TIYl55(`2?LQpb&1}cJwun1;2;9eTe{p=>a z@QE~PuuYCzu<>KDVuhFGz^;~5)<9C&FSq{svKFBIC0sTRa=T2cb~2C8VkGnS_KXbh z%HS+M&l1R*mX?D*^-G_eK+xVqT9o&^`a84_svqI(xjkKo%msx@&hiSuSsgWwTqv?g~LrF%}{B z3&SbaX%LcJcn5>fc20_I>ad(E0!862iX8SQF1Ya;R@M^qfXWd^$HIY#&zZB2zKjYN z2gc$Or_Ms@adRB?Ln6lbWVvm3vN*w*@#LH4-fZ=cb&8GKqsEm<-c%-|fOye(TT&f? z{3v`bM~K^I$sak;2u|yC%CZd_FcMAhI`X1bVV3*jKO^q(GkM)nfe=+UiNBQ@B2Ome z7Om>Xk4rrxeTQ0w`0hn`|2k@>H3H> zC)em?+sWQlgyu>i47c5N5ZBff&G__2`%u5x1c?dDR17gfWUy_3prNqHhkjXv3}|*J zp?^guhXi>yH^a@vr`;@J?l3484@hDl za5c`VK$}WVTk8rfm4(rAlW7Y% zVkV`4UtlrStue$TA=`+Bf8W)FUo)Fr{$ZY#jl4x@1e+NetjVAu6g#N5{pA^MFF}`- zKOCJpPc&Q-8U^MY3HFF0!mz(N)`|MjfHa&OPjERQ*g4~qZ7iu6teVa4Sgthmx12FF z+;>S>)ui5-Ys4vGct&Jb){PrLwR$;Xh<7%g9Uw`=jE^EQ6Kf`yNtW#~8_DF71_M!? zd_zwt$HQXOSDhmXX%KUhG8gHOr%=;!SAdWiOs1c5L5iawWfp}6Q7|L!I=?KEi-@p@ zoAO&SqEOU`^=EP(z#x2U>n;_Gz$~%q@5N@bjK!@qO$R-MlwqK7A`$V6ws~CL_K1L! z8&ZHj2lpQ0HYpZqacovBNk94inOZuYU0-~s$6G<>v3P5KHE`2{)Y|UT;Yvg#8nwrf zA*bW7F6#C0$UJuL5v5|8ih+qaj?5%sDd_-|&HrLmhZMPJELMbuQLi~4GbaL*UO5fx z?qWv+zAltp8b>R25L^Q@XR1VvwP{OqWM#I&7B5S{w_+caniTGZ#cUuI9d}+t$jnHu z)y*NPo;XG57w8k1mgr!oHjA{VC1~B{kKenJOc3%Nh8|{?V?QQFu}r>yVuD!Z8Zjsk z3aNzODmcVOBaWUZB#?umMF>%m9U5WphtWG)xCc}U>7pYHQY};KZ=m-_6rI9BVJ|}n z)MQVI*Ldt!`9XoE?vd8fIz{qZ8U-@>17vdUuU+hCk8KZ32trg+s(vBQr&a`iXJ(mr!2020AA_yoX>Cgd z-D+OV>|1KtPGtc|a{xxewLH{2v8af}ve`-`Blm3oSk4A-8bhBkn^^i%vuMnoS9$w9 zqZ(6vquWO-*{?}hMEmDt6`kj__`euN_J9D#z>V)GUSKu#r`AGr6wD{7CCMq|2X(sL zoo|6snSXH0m;f$dRqC%>!?e`uUaQS1BOdZtW6Q}QGO`-mV+G^|PpTr!3{!zfJPvs? zI#3#v{mO4O9Tsp@TvC2hoiLl2FiiUNdo?{4Hoz+wZ)$o(8!A8)<6=Rf>Xu>1W9CMZ z)Bc$psJgi%IK-X5fLzjh2={rRs$Qr@Oq5hh8b};E5URE2!?Gj1PCz+S4ZDofls*() z3>%9bLQf;AsbVr^lEe(GnqfGlIF6q(S|v~n6;svn{GTisj$eiKUIwk4?k?w*n^|VAWDkg3w%^PAKvOytpwFrKHNAa-JXPvTbmc` zjJEsOkSzzQ1*n9)H}IlQ(V`PJQ=bbibSF}Lu;&4V(TrO_M{*GtOT@qTd)yBz6W!(C zxS@Iyd2J7OV8J0S)}V_8jp{l%_c~n<@wa zmUIyUiu)rzvK2CJybt4!aw5Hs`k8jSPg^`}h!Zk7t8ll!B>dblfa+ObMe!v=g6aIW z@L}hYNp#?c<{Z}rnBs$(U5|A)Ow7$n!))|oo5?eTGjd5HWbY{S8sNjqVt-2}2b!{i za-(ZBF`7sm#Ye_skLa{>N$4;cbc=dP93`H0SY!ie*-YsidxPp|k3h*-C~Wa!BpaxI zNpvXbDA@8x6DSqbg(;QtME6(a*TWzs1p;~3Mpp!7sy-L%eYS^*vZdF}S#PKL&)8CN)=leElY;`l%#Adm{Gmjp zpS*tibfk8|{53hz)17Y?{+_GU`ULiCdIb13n|S<1Kwpnsa!#Ktj@#g@3LE#D8UhAF z4d7HkhD@65Z>WC?I+cRW)Pd9Y*{dFu@*WQ%#mJu$PM!-+$VVu}S90JQjH^%((?5QV zI6}SucdWsQ*%>9AZ`UUTtyXFGxdGCJPn%MhjvoW*e4YeoR2x3+npsL_Z~F9S#+IY2 zm8dES3)bLo+8~t!wq>Z95;6^u#EqjORV9ZLM*Ns`y`|y8H6hYV>nPNKWyz{+svB=h zY_`cHCDsGKg&uJoB7uUygbkzPL7)eWr z%TyhA?TAyfJ*9dpHRlW_#Tjbnsg%(b&O!VVC6h5H38$m6yj01#*R~#WK#p);Lxsay*meR`=e*Ov!dr zAE|7!Ea{caSQS`=Qg7F464WsY!+mN>Qsj!GIf$4wy}#Ygw8z;z;eoXh#Nw?)I(M7* z=WF!yE!kaaD8yI_IlUmredqGAUo|EOBW3CDiB9Zq85HW9eNf(yw6apOA&%k%ds-mo zo*kA8=1Z2yt6Y`sDG>p+80Ue#LejWS%URYo4kb6PQD28ba-H0nE&1-cA1>*Gyi!ew zMmH(bPg7X8Hcz}Vp@q4HuD%5x&71R&Z}1rvS`OSCCTi@^P}5ciB52J|DUK@Mh$IcY zb#;fiX7eBNn*NxBfqRn0`v*uXr#j5-3b@*HO&?g2Dx#_Wnyk;;xlVqs+b^`s(&xg9 z;Z?}HhvpCv{vq1u%T%h3Sf}rf{%zc>^Pv!lxlI}~L0BZH@BCVA%%exCTP{m`igLkF zcB-~c4)T)!QnMe4Yei%Pe6Q(*ABYSUV31J%R27bw7?xJaRV-)-%9ahR^o{a25=U;x z3^3LXL24XANBvMx)71#|1FC|IJ~ zj6)92t0_Z8&5K1xdaFQf+^$padLDA+^S#8_mS5hOg4AOlqqs?3%ZV&bKorWIs(g;W z#gD3G;)-n$bmtpj`xbk1m*yvYfK0{`obD>oAboXqb_I8c`+!A*l>Q+UySh_tXm)Ol z0uVBqALqFLg`nUmpm{(fe78Y}qx=F&hLezxsFxFdn2a@4w?=DJvYZvg=7L6rZiINW z=W25DVniG!^iO%wgF==fIMpf}HUMef_caNH6lQuoT*4qSZ%L7$#nLD!>E)7ydf5?9 zBHoPPi`V@+Q)<|u!gnNiXFx=xiqEu1nHyK`_&(`73QvEGgxw&8qu9Xy6d>X(%NW%0 z+7VtV@~CEJT9B9>lM3){i8hS3dY&tO|DjZ+rLkH5a6$H`%a&=*p%hR}s$k7O3BMJ6 za6oDi2BRq@|0nH0^MMzPj6t{QSDT?iw*X&t+S$yqzPaNB@Rn2LRq^qIun7l>nOtx} zU^VzVSOi)@bF??ZlL!Mkl4wllA$$%0OVRdJ*D(nie|n zdL6T`8IB@7%*-fsCMa88n|c`kAC|2Z@oEVtN3ps zR~0qz{3i@~#9mlnon>HYa&Hp^LBE5ILhcxlewB>;AcP};aV_I4TGo<)3%4lz2f2Uy zAeh_1^-nw_H&g+hx1q_zkHmmhwA6gZlJPbHGzu^QNkCFk{J)$I5Mdd16&&tWb-Dx5sLy71VO8-{{sQ! z$l9;tM({2mR00>B)|<2TZ-#?K82L97AnXD5Fj86DG%OSJ6YheIor8Gu<}hG^&?seP ziElDn-RroR?hSbQfTe#`L(`wzaI^x#A9Q7$N&A}>07ruQ|FI)gulEMdv^0KV|A-g@AsjC_#OZLh zc9aoo6X5*jzp;Y<_Z|C_4nf6SCY50uOabn4eT~g)iJbZUdzW(KEY_O|P^(}8KQA>j zCwtpJ5-Gn0S>bqw1H?fH)0Qw@1_p*iTq7>UL%jc;L0*Sv`)_zsHgX;lP%H^>J2lfG z+`rrX)LuOmk`?bwOof(u^;D@8=TI~7koSBPB7w?ebQ0m-WN-ck^ZKn&lJlD%YXWf) zqUKOr%_J)L5QHin#e|OS68z5({?NQm8;}!rU1XuB;dk>Cxdu|L)gRoDUJ<;*z%wIufMt}J~Q5bVfFcSIdZ92 zp=#iOj3OOXg-d@O@izZBa3`Hy;#J(gIMDnKtaK`_wu4DnP~jC2#er4h{kNU3SwZxy z#_?^ptzYBqgINO#CQK?SF0fO~#)oKcE2DT_S^UQXKo)t8Jq++^RNO+1kzTUVgd+Y< zdevSWX^RRdKzor1Vwrrm_3{ni&JjZc!xj=1g_v$@>ay?yIz;pgcC$YF$ z{Pb0eg^e&%0CDAn|CjGEmGMXE@MbiP62}GKK&`=G`Cw&V)?!pUikU2*KFagz`52#lAkyr7PGeS|n(8A9G)Zr~@`{DQ5C zdrB6METT2FH!AIkE!DnosD-1EA0{{q`*yP=A9d7q7h7e`l=DGZL>bSYBT`1=&zGaWw+-pls6?9OFM%R-&kE4n1c6qH$j+y2OLgoA zbc2}-?fAWXZp5#Os}Ix#HkeE;$?y0w7%KhlliGzsv-c53cu{iKnakh^J+gvCE>o#i zTzSVa7~Z!zXz$k=R*dCLR-rpg0~!ucXpoa%bL>ycD#kYt?jQwl98s1)Z|jHPzgBxf z6tvTqHh!v25yY-X)2Zqt@1J#DanF>|cS+7pW=ClyijZ?GU9bfGB+<}I%F=&5{Y~K_y-Sc9eo^(f8DOl zlgyYK^@S2&tTVMutr@9V%hykKUDuO5-1UItqqqB9xWn^7Af+EBquuYM#{nHO0Qwz3 zwf}w|r10{1&~7x~5x)^G3zAYQ(-ZKzJ4Mvd?oK zL}={w%~{S@lm4|EbpznvOjiSJ!;lamOY-4FR?t(klUGc&K90th?fwzwHno+(7k11T zhVQ)t@>73pZ`x-Crnvq%4B%IFW4uyqBbfEF|5{IUucb`puc5(PbOne3QV%Z&V;7(j zDA}J|sioLBRn^vN;NYHHH;D{s?mNdJVsS7C{m{xt8vEtsRe6@BLTkR>pX4MCW3)RN zEUkOw;(XV|6TmeHcv$*v7J% z$fH{=>$TmS{9T&maE>i82ofJy3|6Z8JVTV1=Suddp9kwWZMhvmqXDUqsTW|xj)N&D^v`oL-!RI@ur=6V@_M4- zcpg9{1SgdSvGvA!QwD8k>9lN@$1?{h4a=(|2@YbYKbGcI!gf-pM;E8@3QZj~*VC0* zAX}H)5Y*KG$YCXa{#Me6{r?99()bet^^{^O#rn4YjE z_II5#1%sWoF-opd^TU^+lH~X08iZ;(O3MP?VS8#d*uf#DA3zNV*_8&!0%{)h>|*mN z!C#l8N@c~z6USoVqsVb;eta3sDv4f@a9RHoNn+Xd7*;U`QZLS0CrL2{Waslci`-lk zlK42Kd^3g3k~!fevW+tC$*KIYF~r@eHXY6BikbndCeM98hrw@?DxQxlAWW1aGpp8i z8n(;(tL-&{jbya`MVJxZIQS@S6U@>JzD6IcuD{d;oN^{ZIN+BnmgNpG1oi zciAL0EU{TW!AF5`u+U@*M6fpFZE?*davZ&7Zpea=jmW85v6*Gq8m!dAvcdFE=|$6x!ECd7x5qK1k}4 ziD{A(j1cpjx}%(z@eB#k2VU)_BiQ?VZ3FC@7p8qhSyp9gABGj@xCVJRa~CQWW(9^R z{y4ynj{OG0j|bc)``@OnY5>^jkw$1);jM!0Lp%y>iDKSEnZv7rljyCD1ER|LN@>eX z&rV|{3eRWQ#A9w%0_^lsbEhlQIEgGuCGVgNT$wKonINO=|0ltvz}rmMXh+WL4G%Q8 zYl&=ySKzw4-b>(K5XsVy2e@%b84Z#j3b;Lz#2QTvu;Vv%rc}A3&p9HjaC&D zxZ)^FKq@ApmZnrnI9m^qN@+j?vh!d|h0ao2nUAZd3bKz05m1`-+v~0d?#MdQC`tPp z5RM`rc4A1h4>^H_fRAHPmyX!NSZ2D!{Rq0>2@xin6#4TXy?rDFFtY&!XyHHhnGIkv zlzTp7BuU~!G@zh7FdHMieI*OLU4{P-z~2Dg+vyF><);V`J`~uoIr2C3D)x$A(J5*F zL-McZFmi>y+lIIT7i)Y0hXVg@`@fi@;4ezx`C}2#yC8t4M=AP^e0-aY4gkzud7N4} z{~i#ueC2-#`!=ktuZucCF@5=;JPn8;8Mr94h(c^4#u6wA!eMF!#Jyj07lovhDRO-m zIvT5+yca2R%Usn+-^R^F5A&lx9>U;(15&}3@0^MfIMZgqi#H@fWvR5TZC`lgW`mo0@t zrQV<4p9BAX>FqUyi%O6w?ATe*K>s|oaOj2u>+|sV7c{3HEdFIXAz3Ci_rAqnctZo` z+~*&R_pg!#CZ&@I{f|O1Mfj;((pf6wi|O1571Xd#w$0qwbyf$X{dGL^akJBkCcEfe zZ@J;6G^KX<|Ip^K%DvT4F{RrIDs7qW;+Ixc#bMKHD}35|nOe%2{ZvNV zs?RT!xLd85yW*BVq;^~A&|i6OF*6ccdE(kFovM0uw9nv*pUH<`oscg$M@SZ1EbDxj zHAcj7ZSN1T4ax8AzXrg$s@+Of9L=nY9Q5a#@de5A=~Z#JJYqt;(V#)Ob79?(vIoxT;ENX*75bZy+Rh>!W zoK$94Pc@MNWvO62i_Xa|j2Ahi)8wIu&~4Kl)+LUN%=bq$NT;zpo?R#OYj+?VVMDfw#>E2ARtQXo>Ly5SLO zf9q?8{mX#F@UP*~)nPRAR#=9}0MOotDcc?4TbVas)wI7o95_!R)r)V{!B;N);j!tM zZT+V#6reHtOU5a>ud4pnDN-L7S)O2^TKeC0iQMbKFx*K|$tTw_h&;&=g`vQ@PO(yRBE@L`|n>!!kn^acFuCdZ7U7{PpW|v9{$-=?mrd^l%K# z2&-C1@9_JDD*JunI(-L-IAj>O^kl_3JT2Pk3l;ewfRBw>R$CI`8+Ug%^UEb##s$}2 zH#aO0>?~p7va&6vKfWvMV2xcWMj0u0QZcdm{UxT~#WnIcm{W?tcK|q$;dxJLrlzMX z-m4Elhx~p6`~wdT;b<$l)BEKkVi*LzRIl8pQIU{O~a>ay(PrjxMY

Y=98u$m^mr~1tJ+q0zPpm25Vy2APPrepQhyC{k^1GhlAKq?b*Sfz*(%= z#Gw#Lm+RZpb(jK;k;v!5WvbljUus;JT#uS_dlsV-))XcJ$T7#yR z_VXh8*pqT|XDYJBkQ-tcvW0%OQ_^z9R{HQbPm5PKcu#4Xb$I(c1^hD}Db5hiNysz3 z>L=-r0=+5h8##i=m`VexT;^JM=A+e@zAodn>J5>S}9kU9E#9Slj z*@|l4HK5~K#OIUdAywH9sQw*CT|EEj)$z)YmTD7K1y}?hhd1GB7LcP5GT>>U{vCTn ziST=2W!vVU3xM_R1KKEfP!@=4S|8$gK>c(ar0^U})aVq}bzBfwk(~bH*vkDo{tZa3m0ai#b>{EbQVt+c5r9PD2;$q>`LUcT9=AkN z!cZ#0la8{bXj!FbbnM!yXFOJ>3UknoWxyeDoA~B7omZ_}zap2KhaJWs6v(IPhFbI34 zrUHPNppuf7=zKB*zZa3H5+Hdy_1b4UCk270D~v~*ngZ+&t;woE@kXjeEwp(b7NP$! z6+pKG*Z7}#v3kE42tFm*vd1SG?aq`363(($Fb0RDWeZ^b2kZ=9@t;QPtj-(!0E528 z!gWM0q|&?ZR|M(qu*E@stjLqdvO zD0n)TJA|$}UMO@7^q)NmfbN?L4z;68$jqPx)!)yt^8+Z8eGU6(pIloVHB0eP7l1}B zzpq=U*su(l^KSv4%v|Q&cJ9Xf^A(jz|02&qc;%Sdr1zo%?cP!AM=$DK3*2>Rs6PkG zMn0tJRQe4(OT}^r4g>+>r!c|2m9(WSO_H>kDfwW$gj7Aj509MbS&FbUrq6yjxP)ZQ zYYTly;&bMPWitl-Oj`35sDS?VA-;TVxv^>BBkb;3Cog)ULE)QDu?7g!;%+g-4{yZC z&yR3{rrk;+3Il3*(S=%=6m655u!b}0I~%05D?th3?vPq-lLt%iGUL8u(QX@SCO9`P?5kej2dQ%gubtyKX zapmMQF7|%k+uHUOl7KQzYxT4BU};=*Sc5a#k$Z zZ5I8ls!(`Ui^3fSqjOQXWJDWMzAiZHL`gNc;iqz`@*RnG%P9th*i4OW&>y9->dzh5 zepck+<@D9%Goq05kKfltpUWcIDLuqqQfZ?UCKLHKK(%5KiAvO|*$^W6sBAh|m}O3? zBl;idLzTd|mlG`E6_bOF0{C1}x!7Oh9wjZ@^GP%=i0OX!4=$1W<=uh)GD$)cOf&}g z_2oUwBqg1=WI{l{-9fdn{k%IZ5YS&1?;*o5c@WPhZ6j0$&2ugqN(H=C6fI8&6W;LL z|DZbn0O$tVqK4@JgR+2rqegD83~HGS^!&ecE(MtY@fPAeuKj;>?X_drsmeKv^A@hF z$k3L9QeXDh_{fptfICI=EuG33_q$ z>iD@KSqN`4YQDBwe@fP5g^2wv3`l zS67$n|6uE_!>a7Q?Qcbt?vn135+tO%k?xcdkdp2c>28$n?oI`1q`SLAy5U`0{ho8) z^E=O9&vgmxJJx-#Imh^nx!Ei@1_+YEz;TpUc-y{(Tn1D$(8h}V%(ol(b5S%vZs0$j zR=jBctOUtVh&KLGQ&r~gheHg~NwQy#kTd=62={U@E(#F<|8rq)#6M_3t|(n9!JqRL zk${{Q%RT`saWXi%;o;m|?mtN{o2zr5;VNpLT} zR?lGnyGcrrA7nQiRQTcr&U1y|pWrFvUqqata{ao-_HZg%L1SCiN^X!abAzwWWrT=my8dsK5yn!8-KWvla)j?-EwL9SFSHem_> z*Vd2UvE()DE>)jjF`d#KpXp7dVBAL1Sc?6W-8DszY1p9eoJBuxZuPm%x?{h;qE=TVxJ_`4vCq4b4JuWk&auFtkx zZwKYF9xtxRTW_yOJ`PL^Qen)0siVY67d&M3{&#j$f6p%V^!|ZRuX&N&c050DAN5_G zR7oJPt}N3xgtbf|(2nTbnU7((FBNl=)*yN@gGzUA&e$^WtAeX{Ei0ugIn4*0lv3ar zO_YSltru#g+C6|1@t86lKxpWi^*)A?Wcqa2dAsh5^ClXIt{qEl#I`>u&hx(*_6{;$wHRR) zA55TvmS1r7ENSHZ%?qGKjJ~x!U@FIu&K3ZB^(WvAHoN>LwwydLW_*Q`T-ay+=y|)%$bM!GS2i^SG2z_rub$4l;*bvs z{s*>x%)lCACQR1;K&qN)#4jpVmJD`2?~9j{&rx{g+DTnyVvo3qcHk3^$yrMeH6uDo z$ye46!qnknGEdUYBof)lvxmODsObpUl3TuAfN3zV69P_R`Ua)8F%Jr3>URJtp)wh} zj^nl=M(yuUTn!bcetvn3K`xgv`IgIzfXlk+W~?4+4PH+V=pA~Ua+?yncbyZyeS>@~ zhXiw<7cg!Ct%ox$9^_leE52IC|7y3j)Kc5pU2(IgxuU+>fzw+;LMtECf<#ESG|1{> zSw!fM@mM$bKytqV2ujL`@5fu z<6ELkr9iaVuFWdpNxwZH%>2<}jWI16?h9T58TM18PX~+KOv0~daX{&B_#$C(OLDHZ zU?{8L;Tr~uQt~5jBR=rgd{bHoD^<(_W2AF47rrVz{a^~H_lzYcIby{b;RVM9)g=tq zrH))b{{oxa$KgGXt*b}(d(~p!G_7A)V2P_<@jC*-= zmm$>%bYE?_DElwc9e-WW62&$-`lR50s)Ogv&iP4!6(=tdi+_rtD_w*9Ls`YEU+%_v zyW6$5dKTMLLY6L)DP=D=Cz$+H`;kdK3zt4wrQkDKsQblBBjya@ff? z>c2im2u0wD;v`;b7FnD5Hc?h!w}&Ni-PTo5&tRs4?N!R1^)oIoLhW(k&dhuyU@PL; zhU@8f_Wf^MuqA2)f)pa{zPXgjFMKS{Bc0Xo-l|xVMvAtU=d(EbAMR=9yRJ34-H=?! z*@=aS-Y%n(Q%#t_0CHJMifGnEO=p*o1j4uv7f;l;(ux%c1d9L z4wj|_%mp}QF0M~}EZ$Y?RN$QuQd<`r4dqnGcX3*CtBka)I+>OeS{poW>EeBG4;^v^6HK_;%mVmVM3zplN#UceU#!5FH4a&dj}9O%2qi{8;ok$*+Z8pCghg0b_{A3VCTRS|uH;NDQ#BZpm9$WJOV|m> z5{XpgUWmpNI&`Egi#F*upRY{k#Sr%Nz~LmB+Bhgu*;D{-ImmMM4tz3K zMx5m|DI~gXez_UkCzLphaxj_7-ZeIVy?(!eksJSrl+%c$qM=9zjg+7Yp{-SMsmZL@ zzv{fMW4LA@CMSHptsB*S$*M~8{ZiwSSuTj!t(sVIyIz`$ekpLt zp=LeflbaR9)HgI8zXO;dXYpq$)&8$@=QKW7Z zP~zgBTmk^+_-@j;CsV}DY{>O>jR3w52@#i7vv)o1X#QM|UbHA#5)O<>gm-ogekss| z9(vwOCXQcym4I%st$as9Ley>o5y9#%V*8Lsbp7?tmUun96P!wKaTDgp9M5eg)XXj` zyI(oCxQ3H4xY63I=$U3SsS-7q*C; z++r<9)aNd&KYEj$w-?fKge)xxhWm!%8F$PBkhXqd<3(pEn5;?xq_^PD()rc#IF3z! zfW@mT1d2epOE(EE)y?EZr@>a*&-d=FcS9+(luF-0;Cq47co&J=&8?;+9Hha$9Uwkn z0*BT=x#%erU=Jn-R5WlMs7n+X7Pz*3bk=Twf*)JU-?|DHd(^It7ZW|bS-)SLj(4l^KLxRlH{_b(Fp-i++r=g=ASOdpYm(T zg1``;UYo$h#uQPVat+-+vJ5f$dw;R_+LYwN*gYyb9}0@&N`$#cb%R;?Q+i2Gjn|e+ zhV7O-_ygc}!NMsaOiR8crM>gNGVRCp3lm&^=_f|v#_b>$Xh9hLrRC?|YAz+!7yslo z8~rtdnwD-7D2l2MW!`pO1Cj)zg#DyxVoS9&);|lyQ6}1lG(?>0ELdno=6h!Ts#juhJ;Go&(dPr9PJCDZXP{0o@u(qlzCW6$BU2luto3rUt zVhUl&Be%EZ4Aia$D)@4}!^5|OQI^WHM;LaSQ4PkvF$?-u3-ka2OqT=nan(OyX&z%3 z9*J}7y;+mguElfCmm@9Kc98InsrEvH?7hJ}VCm1tVKwagHJ(y(A)RyxIQ@9-&bQfv zi)VRTAdV{40yr_j?so~}P)-KDvD^g|f%N|0n5L1y1eob&RsuV0Wso!?L(lEc26+8vjD+ff^g&9%(KejJ=QC3S zSi-m%_@zGZ_g%L;SwQw{+>l{NH{y=4ATMAs?Lr|UOdkK?<^P6tbe*X*QS!paC*ATurxh6#0=X@0NwZk#KDnokuUri3DdKZ?qdK(ain&T_=PmGOaVsNBr@b;a5s^+eV+hdQ+*c^ZU)BhC zOd3hYxIY-}MZJ!7d1aG;NZH#X5lTA@t9TbERJghEZKkmU{0YdD2d9MCE|X zuQx=segL02%Q$oje;i7nd>o1p>WF65z=2By>#f6Gm0t2sTaGFQcjeU=L}1};bq@uc z2pE^$+Rg&wH_C<8{}bGpmxF2_hg!Ufjp1NI_CMM%h@b7!XQW5Whgt9gkOL#f0`Qff*t2C6a7XqH+Pagizm!_gVIr0nsEYhU(~ zCOmEV+iHp(#!)Br_%aE8McgeH<%9(pmLB%liG{t@)48d@e;RS^=N=_nV7mB0h|2>G z|4`&46woRuAK@};NPH_4A8c@Tz4t|@vRy8bfFMx?C6Z5hH-3)0M-P4Nc?u9O#p8_v zE*3fCK_Z|Cch(!>n8cxxM>A`Jy;WM+8DS1kP=$|dvHcn)xQ?NRaJ@#+>OJ#}K#_IL z3|v%_mGRI$TczF#*{;b7D_Ojc4k|Alim=l)3i)dB8QxU4*C`-FyKh)cU#v2?4>_cWNyEkVxHbu6(-tVtv#Dk6Upi7K8 zd$Hd($atV=37JaZyDo8J;_06~{+wYzzk=jI+MYzz{fk*3fw;5S^fs0o+9?f%PFC>;>fO0no{a1ti1ePZGe8B6TXOYivOjP^*!5Nbu zTWgHB-sLkJWVtWd2$qI$K(xy5WuRldQt@q0gZ32%zvMjc{CD7rz)MJa#ot%Iv$SUh zqf+MwNL5oNOqeaS3PZKYdJC;h z;a0x*`w#L|;4}K3zCZmV4gP>ggTLXvuLR&ha2??x|9`lS=y$-{i3j!dUYty;Dkbo! z+HP&@_Gq;c8vK+ta^LmNx#@8g^7p>j!6zN@CD>3 zTP!8>N#R=Fo}9(TrI_c7PdxNtjoRGAwI1uDmqUZACoOM8NP%h(?tq|47x!Ptq5b7k zY1l9a4?OaB534;(<>c!&5lbHKsB9nhU$ZMZcJup0>eBSDKAhgx1)?#nFm-sj9zb2Z z!w>a;!~V5cT5B-+;2ZZ7%UcM&3p?q5J(Mut8L zDyInvg5>_*{NdH3QpE}Eaq08>$&7<(e#_VHtsbI0-S)fked-Ze9S- zDUkQL;sFX7x;61aXRhGE@0Du|hjL>8TgX>C5LX@X*SNm`4m$J^Q`+&(4?^G+i>1Kz zqj9)=1Qik<03QSBWyxnmg=+hdes`xm`@Qd<_wn=+J`P8B-l$%foDVQ0cbm9=AEv)yd6tPedFIJNePda+u^Qmu1G)|7wiU1 zoBL0w1DuK;!8!_lURVuZ= z)PL+pJwEUB(dhVOwkO2~=J_3l3jJ!lUNH(hvhIAf{4hffFd;Tzhx-Dgsz^PjR}Tsc zMrJgn4SB@y+N|wPZ>_x1O=rDy25O6g9?V~n{cJ+!F=9)##(?<$R}uw({n(d9Vz_89 z5&a)@JX#QF#zlhJyZ>aI*dYOX5fOST1w4T3l78@|m_C!k2bQ`NTp0ds-TOyYlWO&9 z6l7)t1D!M~38i#sc-Gx@6McH$4iZ+XIH@l@4s!sK$7%MufPGW+LAdNQ%JIbBfAA8C zWKufW|DpBJgsA`DP?zhpg*%ULZWC zM)R}_GvbhX zot_czYN$CKF`fKyb#r5IwC>ZRoXe%S$+#M@2Jm;Z-a(>`N4ya61X;s>3RcbZ5Xe4z z-ews@avr4}qSJVl48QKwn$u{hU*v3s0bo!eB9f9#OYPi(y6jx5M9o?VRn`s!?zzJS zyp0H`gqIVu(kong^CPd=?dF0&D{i>vt?KN!hpx%MoxRzy&SEyAzxKSxM9li(<6Oec zKguY!I-0uQ*-AHeGo-+|>C!yhIowPEz+^!2#zpvwtw!c2)FIa9$4L*egXob0+Yn-r z@X#Xeud(F|fpzw0UFtoYsdwM$mS|pGZd>iYoKS)bV_+nVeEm)gkbcrF)VSEx*m7bq z4RgR)?5e6GG27JMO%0+X9C5uyo9B1J{0G?YI$S?@{ma)&F*Mgqfl$b&KwRk-dzagf zam-{0&b_mM`C#h%OT&{zr|5VqpB*MVubsDkxmSSo=6Jo2QyHHf2g1QE1ktIq*dBc2 zdI=gD61890>hqO|uEd_1x(SF8|B##-0p1!5Doy&uJQkM8v-}89i51F(f>d$VMSae0 zDrHGOxmO2rk77o6G>_fKEtT^i9;vU))bMOaRz__QxJ$!DM$Po6>}nuDGH8yYYvpN0 z90RdQKI6)2C^)sl8xl~Ak!mJ8d#++D4{rXxWMVyYHs>x;D^tbH=gfM;bmrHEy>%gR z;^v5HF{{Exo!Lj~ipYQZ4*DlRpoa~YcvUvP4`f`OZDPH;=he99Epn9|Cu)KoveKDM zX#?643p`eVwL%Pa3(&rLSP(b1 zsRqnU_ONM#5&vr5&ASD1h{zG)Y?$b5uWpVbp_G94{%Xc-_w|?JPZApn~wMd97r3}c0 zyhIHKm`N@!H_?$SScn&jI?3jj8cHO=BaNdWw-ol1$@e^h!oc z7nWowz%g*U4V{O|Hz>3=ZUi?iki8xu07`9{>-k5ZQ`PUng{kP6lS?;finTuVNT*C? z8RE8XDqM_jpsp;jC{x(Ap3-cXaBpzBi9Ec+(lF_vIQH`Qzpz$1Ci)+ek7Lf!%&Un! zFon%pA35|-R#hkfykfUMnG``v5MXjrEK>^`{?d+%Q*au6HPQZ;Pbovk1L&pVoG?85 z&WY0AY}HGUYz8s@9(HUEXS~@Fe89W}8by`Rf{jCEix^PXB5+6tlw0`p=4!a-RAqvd z^bKLwNsUW?G6P&|2}P%P>$BV Mj02{Uc;vYN5TFwUMYUEN#Pmq>E8lV{=Vb2P= zXi{$Yy8VLLW)CB3rOs1N`O(*pU=Tgf60Eb5+&K z1tQRwT)Vm=1`=B$Z-Cu8s2j_F0TG(%L;$OBNs|cfBEG`-Y1a?0yv?zQ@*xU2Iz_>& zPcts%Urqsm0q009i5vtK6-+doz3vg~{6!TkuXxtj(F%IT?|~(J5yd;PxClEWGX~Uv_F8=%+3fUmMJZ}pR-I=khuX!kC5+qYH0Ih`?UtREodP@UL zA-tg+usKGA1e{?hLD`xCo%-uvt~K2IF)x`QnGOnCu8m8dM>`;`HYup8{On+9(Vr! zBZ3br{TyX(ymZ98gJw~OD3W4lG6S_|s_p7th_XvahR~8tAUZFVk$$s{0D?#m5kkb-N@y6)XQ!sy zV#8s*d|ZY_YH>Ao8w9pcoN2V3=-G`{x`MbNXYZ5f&vK$YaC*JY$McOsguiL!IB9`h zopxa?FCli&CUkT?8(8_G0q&|kJL~>S2`Bl}8-`59p>z^}!#Hi=y6sU}FrYOW99G*aGl*a&a6CLdW8iapu-Z?lX^)ybce{nv(r;l<<;BY+= zrwyhdy>{*DkHx?D0s1r8lU}DEWB=`vKZ%p1cXud$^lN%u*?|oqp9B+&LHi9spX?z= zgX~oMcL>6J9Y~6N=V^|=!`k=Bc1V>`j)8FT&uvKrr{xP5UiZ(Oru)O*kZXU5N~s|M zw;G|{NagQ2We9^iXJ22X^7nS^q5m0eSd5QQi_H%Cys)?BIDdb#P2jtx(Artw42UYA zPl(sw3t9haJ72HdarWGU2h%uSIFCWH6GdWnz@X8o8~twpN4`$x{QqRB9KeRdXe#cjn1SZ1DHSa`(`JM zyn`=feHMuQ7{4TCrY*Jnh@RYl(3(HrMNti$(N%LB>>LGwB+al$AovYZ>elSYN|`_G z`fcZV%8_ci8IUH0wA{wZV39QP`9YZu;=Zc39s9@E{~>e2&4?cQJpMbn{=p8>N-!v) ztc)r97sy?{2&rN~_*0()mIM@ueD#5rCww4gWSz#p+7yA3*%f-<-<)j?bU|TI60~XK z(7_TyZNh>f0mDH^T1bT8dC2HUDfzcZBf)md)3g%w?(^)*|LVPOGB=iQ@41l&@%)RL z=2iA+D>p7xD(gd(;{^2GhAwIzDFPegk1WF8w&O8JTJOJ&9^!T1Z z-uc(}_@J~74!)-*edmMjY41@`58Ri z)4TI%KzxLRfucafrN@9O9T5qXYzl_!LI6U&?e<_KR9RZhlL}CF310b?)s)X}k6fMf z46;M&Pep;;su&~y4G>)lnQv%+idIDgrxhG9acmr~1-2rFYjtc+4fMlW&38xst7?Bk zFK`byyG4!TLKlT$ktK>A{84etR)nKyD2&P8U^abeVr`w(fjhTpj-oZzu zbPatnr!iNQk!A~&4UP|{z?XgsliResC(hJ4YxY}anp=de)_Qr~@mzAv95j7Ge_Uj( z)+ckP#APhJ;tx#19aA_>)MskArL*L{bOdK@Dr!7RBAMN=)+%MPpu-TcyWVIM+8z^kh z8;+M(;hMc}o&X?6Jmt7Qc6s!h(=7b?O;x>$3#R3QmN9NGCAu762TR>K-`4N{Qa#Q! z7(jF$o^9`9jAQU8I<0(V)f?dc`b}Lvs%VIDptvrBji@|=;5kxmQwOOS{lZX@E_ZG# zHL#UOId|j$Y(;W<^Xv0tBuGMHTAt!mM zmq!^OKS`z*_8~HHfv;J|17;Gh3R;q7(;FnkfxVAZ5>F%mBrEzWhJnIj(eK)joO(%M zdCD+EN}NS|fBWynBm=Y(qmVxX1>VncWix3K0dw!i4p!9&#QItKU+4|?fpH_svkBX|MM0EWi#$qPsCpaoCMPAjyhF8puq9G9BPope5F`0#EKFD$qXC7lk;ccg3$ z3|xPo>JKxFWr^!^2#$>*tHG;Gb+hK`T_0l4W28~3+A#+6@my%IjLQQn z-yoB0EvMwr+2$TKZX%((a`y9k~r*OYIZ`}GWMSoRrC_u|y= zA7sdI9+&6)sIFmWy`SKc;X`*38lqhx#>Vqy3~?&AQgbZ)v7{t5wG;Q|_xS-`Vz|~o zX!;dl9=qTE(~G7wSWp<5;gCWq{hODbBsZ@<@Klo$bW5 zV3ycmN7i@pa_#yse!btC;mcZrrrZ;so%qc1Qj*`xyR} zCh1AWUgg4kcNy^#;D|g!xgMXobzKOy!n4U2b-#ZIOFkqfUg{)a1#|2IZ>c?!pfhUS z?3vcfw54}m-ZpTpIyu+FXP}{B_i>LXA9>@6It;K6hOgx5uI#3QF}^}ni_cRz;2r0f zR-HuNdIkg>TQRZKf$?u2g7GngMth#wmO_HgKCN{s<=GPP3t4XM_IK!*%vRYlSfpEO zU=(LqOvxw78ow6@%}bn>F!2a2E#fvz_4lC;J!%zHdR|69)Z^PEO2j^% zljSj~JoWLw*pJN9R>}Dl#UOtc*6!$pc;p|AG=PeGtu=Y5Xws9tuWB`KZLO#O#f9=M zk!KIZaAvM(k{`k?E>y!-GHio3B^`}1DQ~vaH%2ioOf*~PpQCC7m?=H|_@fMCAB2axLU)s1ulE>c55;3}xYw;-WDxe5 zZ1`@mKrc1)DKKVI@prQKtU#JJsJ%@sN3RWtK4;gC8@H9&~ z;S1vCp;7dV^sD`i_u(fD3#3qGnj=X^{ULgwp=s=x4BEI;Ueie+P34FTV2xp zN@v|UC3I+LL|sNuIYyxkXJ5MWYzR?!mk$e;VysGEvEd4}d5Asg2qr@*ELMk9%2OS% zC>4sI!G?WzK=^%p?InXQe_ClB5^Px<`-NqwH1%`D5OD!2o}Mm$_pP{7lDk4RCl%+& z6nA9DzFy4d)3zm8oM2~?B-*8ipC<(}#i*pGn~O;HS``T?2DBN1;aexQSj;THbr9(? z7Mx~BsHqDrLSzYa1dt1Wl7vs~HZuAV4v8#mk*~ghh1g+JQUML8B|fXEiHUu-R)Du@ zgrJ3tL-%W_hVmIDB`fSRGH6(F3h6F1I<`i{WeFlORw=fRS=9`q3wup!3Qc)2=+wcC zE;a|v(Wuw=+)z7J8$!fGW3MvJ@dE5W)ZIggCrfHi-i<~2qGQU)GXWVA&0Xq z_r2wCnz`c7ia56Y*l4hRyyv&;tF+RKZp6?%7ky;yNCCG>eR>kZMHBnmR2z|8kOsc=W=NHTHeFq|!eCc0H6BeNnq9PlhL zuoL)vy2*;mWQsUWHqhh zQwxG8Fpxc1NfLfrVF7>S+s6jKv}j?{r+_7T-G`c^|>v;$sOkiFx~fDPk*Xo{0X6lIOX*mu8SE8W)(pvIA%;4m`I=#%MTeCsWJSO!^w}}PnsB}}8AbG! zd^JK&nuONRGI?Mf5%6*lba(OnN@^zC(zW6tW6i#aM#sa967&=J)=TGHm*%)#wN6(| z!3wWhR8&0iDfy$D`(u{_D&|iV9XT^G`1`(@$cZ)XKKRJ zk}SSRk=eBhg;r1reI{eI#bn5GAKY<~iJ~GL{V-U_63gx5DmTxrAz?*4JJPerB}~{3pFU$M#e%&?YR$1W=<_6g6p;DgW9u7z*K&8ikmwYAcz8hKp*OEYoPm#9rit3SZ5cKg*Df^{ z43o4Q z=7=IK`s5NWhiIvOvQgff2yoUeyw-1Gi&UbanNJNiuO;A3a|)s*BbS>gZ0>ja)FodU zBN(oG7K?F28lOB%ZW$YYdU`58e#9T&c41@l-XsRTWYAju$zYxysqySt<(fQRz!{e) z5)yow-b?}4R8~3{Qj^%Z$fB)^%12Y0brO$gBHBA$vC#n-y8Z~!Ck!&|JVN>9kML6SEZ}<;7^}ma`T3+VJ6kNE8 zevOAx3a0AQYW*tJ7t{zFw(9V;e>R2PQD_Zaqh`FARWaw+cQi!h0Y!`qK~aez=KwsTP;K5)#-j=!_Sne&LW!CHyJY0^Gl)^8i^o z@|+tKEa=r)iw5fij`=z7Yz*L1Bm~T`v9z3o@1L%*<|fO(^T|t)C4aYBY`V+(EN@U5NCUH zm9g4c^saZc-^U59(5mP; z1D3C2sbK18+_B!16kGHG1)7pB27_)bsXKYD;Tn%`4UCet?UchGD)IA4va8MT0`~o2 zsmUltjRyR%$+t4{C6y*q8jonD)|X|dYiY??X&5pZPU)!zz_vpszkZe@B;d^WjPr0b zGkq6`BZ5&?+?EG^rCB%hMq?t-ien<*3n%*x+~ug(C|AS_}=2@e^~Vbv(SO*kRbEj}qu zxD2Vg-Z9YB{|opiMCm1e7Em|849@N3`BUr&po@J6n)j1fN1>4?%5px6V1PO?!d%Hy z8H?c^lyjKStrRT}R<5s(d3H``b1sq^?ux`~!ORZ~P_d4R>6Ofd8nc~6B$#X>3bOZ( z+CsZ$xN8o@*^8W_5%S|vZ&xS^%r)=tpfmInUFCMGu)$6_Ri2pkcc**;jTkkYF7GPE zoxvKZ7^bnzJ2h)Vr#48_QLZA!jFR7^s^?vIv9o>`n<<7UDE&Je5QyXz_*^LZ|LZTv+ZkkBEOWn?tAUJTXDX1*rG-^Zz zQ1NETC`&P5VIfbX4|IsPAUYnLD!$O`f8B{hAS|p9Iq65yp0hgee{0rgn3rG&fV}v> z^NWLDf7E_1^yEK3ybSqaA3BLZ9BOD9Y&I5IC%o_PqN1Sip2=4wW(trfkP3u1neL*! z#6L^%O3$Lt$&)e-naZ;3j?9vvWGG$hGA^#4KRuoic6Yn{DyK9uUfm+AL!1mX zHuxeW&&7PTGTW8NcfaYwsgsV&jD zN7ZrlCH^u`-LpAQ=0WqG2BeVRha(I8d>jBYZTHp@w85h;9jZI4rl5UobHFsY{O}on zn<6QRjfFf(;pAnOR>>K#%#zm5>ckS1f*2lb{rsKlhI3R!^x&b-Hl*KV*u1&&7vRH# z+BjAAZjB}Rj`Qb2&XNf*5gPmo55g{)@d8zKMI`8{quLA@EGIH-b%dx{2dy=I1IzVZ z@9=7`-Bq_Rx>8SFx~)FI>Deb&!9516U#YEib-wrXybr}JP$CyCWVLC~vA*2?Iy~In zEV&tt^h>^?ByRCjrq2V|d1Hb+#YOHae(zkOSVu8RJ8l$+RU{c{r&)6fk(b_^kyo9q zQ(2ZyowLCU8UFTh``DZ1SH9Ve?e*@@$<#c@b@aM9%XywbYG{6=?5qE+DX|5tMpS&T z&u0*ofbTsCB+{liy;w<+NM&IgprRLD`HF;E?g$1Wxj}HS&_Ds7S2Y^Bm<402KMJ0`Z8aj(i=p&V zs7`H$!nIw0N+`JVrk7}_6uO-6B(|WvdqOM%F~df-G%*;&ek!J~`_zQ{D?+Ai=J|^0 zPU1^~x26VH_V%`V6w_aF{aPR}RJ#CUt=_&iq#5d`5rrS${t)pI+au2P)y#LyA;z@M zI;S6|%pZw@C(N)ZmyFza+Ry`wpNC|=pJ=r3vTyezQzTXEJt;PExHPZCr+vWTNw6_e z{8^^weEXY#(=ZCiq*xvD%$V>vV>DmfD|6aXu0sSe_FY!Uqk=b|7 zFkIrtmCl&Xn8aW-s!yRq>{ZVUWAEwa1Cn3B!_V%mh~L~i1Sy*?WOj=}eJ*3it(Z}t zfOsb&_8{iJ-*DYPrj@irLX3x;?AIiIxw=;MIPCt@VxE5|pbo{F)Fn7}yV4#8 zqlQVuY5Zte{F-)hO>*KQ&Tsd|?c33fm=vX+7cI{THow3uS%{U>hP%IkV=!GPQXro5 z*CYcaLw&9F8~*O=SK~i@c7%W5460DPk?LX@-{bJV%^I_I&wIO!z+WeN-Ov}^2Fu6O zfU;EZ`S{-BwPNlsZ|tniOo5HrN^``OOsj*8^Jp?wuvM;4!hE?mH5{_FrF*~@(Ib~= z-M+aSyY(YseejFC(ouum9!m&U6Ux%*xOmCF2#55IL)qYTbp|1ElD)GAXUhJITJu?! z9Och`XpubdJ{E`;A>o4yBifv-C$c9L>kWM$I}c0QAM016gHnAseG{SX@9*E9t0Twb zbwc1KyDt{5&H3OQ!N?RSYg$}WI7A_q<2YI0YW;@W=GldI93^Yu^6&Eer&W^uV3ggk zB(k(-tUwzf&UMZVP+9nRMFfLsHU*J$=IccW{+A*siD0`vWnUKZbpa;R;M-xn%lB_7 zPi=qFI2foT%$Dog1?qKOhVF!Mpjf3}ZYrf?$YQh8w%d_)>&}`IfhuNm9NiTy*AVX) z6=TPbq>e~hC(rQVbP0Ga-pv0}DZR6l`f93`8)ReVN11yorn&NYUtBM2lm!jHZE%0( zYRMHuAS!M6nEl|MOxJ5;^zf3(8tfZSZNJWQD}U~Iz<2g!LlvA;%bE-JARu+IUwO;t z$Zr0_;M0*j)};AeRFw#4@?8DPp)%<7F&-xE#Z8}Yo{QxtzlGI%P>So!{O;ni0#VJH z96sSgO)_}@`r09pL`8TJy4SHT6+=2Ut^fE}zFZ@;M(NiySox7|sVGbkxNbGyz%itRm6i!OcIwMf5D+Y6!FQrdt zVZ-Kg=z15+7#_0Ms*1MKmj?SKQuv<5e~lAmj`^TKF|jcwqa69pZ1-rmc!dXt3?&eC zdi6cB_~T=_Q!ztpt4mNR-36ITFs8bhLX2hFCaFmi#Skrrn&*d6lOZI2HSz3U$w*-@ z-l-0L8&#j5RcR+#YLbZKdzKf3imTFM$4~#?S}GOHKP@$?=Dm%@p-FII%1wHp4aUr^ zq;-f^tbkM9eUCzMXfV;nri;N2W{qPKsfliUUf~ypemQk|YbBx%+1B@3(d=Zbee6d) z=0gD{C>4F9FTtV{{GC!%In;LM2`au`Ml8RsqB+6J`@UZ>;ZCZ>&%W;KPto90!QgQ> zdWRe9&Smjw%4wGhnMUr9I&4V)ep9?_F<_3l6KwC~aOrwB>n!w~T9Zx7ma$X7r4(YOunZ;w{J*(X@L>@l;nk5(=l$zw{%hYMRm zYrD?j=KIPQhe~`B!UyV5pX?w3cMLooi^lX4QGECN6QnKK=j{&j_Ue~aV#EQ9cu!lL zZuu-2ENxbIkDav=*x(@g+;9_avRs=148y7#&4-D>@C@;^<=s+;kKo0KhR4cy&>A%+ z8YU57?#5oZ#$b6C&B(#Xxiy23;0vTucLY&_W99Oa>T#2v{k)F-(vN+{GB3Bn1nOfUi8Gxyejfy+%4Q9KS}Hep8Eb zqLS|eHfTG&Zfv=q5fMi~9dPTOBXcT{YBrP&Ga?u2c9edCfrp8dxoz0}+lpr9duZ=# zGji%$p!AYZ2h2u2H^gmYc{gV$Tt@ZMlPj?;uLs&%90Bp9se^m|nr_NqHQ2_{Vcqo0 zHn`jOv#WB@?PJn;oC^W7K=e__+ItdfITA$pJC3lZ!u{)w)?_0A{iX};!Om8KoJ0&Q z!dgsQWd+NqKyXCzK{OBMh$qN>j%zbhm#FaK7&gfq) zh7NDjkX3WnQAg(Or|oqOCrX%ydSI2?ze~> z>^hH@+VqVu`8|uRDDIkc&vv9A!Eg5Qs_XR z0f`W``O^i|zT4a7s|oP+3Cz!s54i(3?9@njc))inZhl{Hh`G}1k556nBYXKJ1glGZ ziaC+Zj2(~7I#Ajt;rQa0R;V3eNMqPu*EfSq3_N9&&jeCn6i+lpt=6+omp!TK7k1chr?6xqBe=Ih(P~h}be)9BBg+(8$_5re|_I zyR>iGfs3+=a@O(-e5>L500$xaXMUyNfQCK1jgVHMmwjdYYs@i(2)1OHBQHjA^lWc> zNxdGo8O6WJXN!a}K9iraiiOV*TnI8I`zK$% zrK$791;GRdI@=w+IO!t4A&%mCuQ2k_^5@GL{RHfm*dkm(S);F^Nt~m8c${(@nwsgr zz=60Am!e^3xmWwfOW0U*>a-~W!I|lpq5t(zLVY<@>fx<<=b@ER`wKPJ^nLKrk)EN; zT<}Yq3#7T2WWF8TiaAeG|CmxZV$L~A7hPTcp$o}(HyWClu@>uKP$#H!{SKz>+ZBn@ z2n1Z&S+?jxsd40NlunOFo~CdGmSxePn8kd{qiUjXakb8N^9UPYFb+S=aX!1x-=0D<<;n35ShP{y)~F(}QkrV_OO1dP4p<6;4hDN0u=>`RcPC-gQ8WCyUHG2P^=bYy~?{)rp zugl9n7?{2H+N-{wT5{Yo`;CJlISMZ9OYf@{WP)6kZ6j71i!LDht&54yq`@@&2CKy^ zq2~6wMvAt_hdl{G#*w)k#Y69tlCs?}3Xh}O8a<-(=Gey6n}&ZHxNP%GJri7EBay#* z_iT9UwPl){)~8;r{OG3I(6E>Gx*A}sY0y4>rhKKK}hMk zT>K_ayGBkM$MHw3-`Ddz-*+e6whn{7N|MtL?E}qLGaP3&xUPwq~0%C(}M9bC=q{#v+U_($llNO3sKfiuk-ws9#e*E*Dn>7R!tPj`k`J(VdWvW}g#FX%tSJ=4R z)Ao4Sqz~YD>LQ&N?N6?6>Z0c>x?EW6PQ8|gqeaQ*ro%m?_LdrBHPl7T&I;FyryFQ} zJT4CBTD(jr_MCibHsQT2^Y^qxJqF%7Ego~O+Bu4xM0I7C+;8qYlxTJ3nRZ2jD1SnV zwJ@E*+rl-1K6T&b7NY@a)ab-66Ewzb^9)xfRbn@*9A-%9TbHpu5mjDp>e$r2L_j%q zi#AIbF8;#%=K}ta`;(9~T{%qQHG7uwZ^o+G?05k6cf^qiYJJM;;Yx$~m(~DGmnP8G ztasY`ub+V@@XtG|cXE0v>>K=pdWX_-$eLL0GOpUJ^0_U} zIHKb8GRcPBDO$zq?4y7TBcNbZHj*8MQT$7lttu=ogJ$CuW_IJp)jmjOyGUm8yed%$h6uiG*Z_X!rv%;y95)8*F86nY< zZQ8|*e0z({k*&r%+*Uny6J@4`j0o*|J8{>#$~E=l>GH)K%hF^>Gg&~UiD;$Wu(M^_ z1Nk==18J${>HJRjB>YZ+%j4y@pG6~^3%PfbKN?1jt*ho~q&98NwA>ESB#+7WX^N8Y z-SP+NH4+{a!?!aYSr1!GlsQCAl;p`Yop|mC@4qr_@NPQXpj+|}n5pE@46Lxcx<@VP zgT~RaaAViu?-VSNbUV2SG>6JmT85zM!_yzVg-E3VNdR52fn6uGvXS9SJI z{!$nLraSEQ8SXc*r)gsqgkIcT!eJrJwcbOo)KE%g~T)|fe(Ai6`>pd(E^;sUQ_oZ zBO$FW@^z+-UOMP-fe}G@EekG#v}w8f@K2-3JMQv0GO|r31PrPg!t&!7# z?tkhg0|_v5JAs$Og8!ZCqir&(O9&iq;HRK6>X zooF*I>y_rWo$!A+jp;~J8>}Z^E;P9lkCSOyjlGDOr`CIaPd;G8Ej1DxecsVAduw9p zNd2=`W0Or&vB0^i+VsDeitH*LxtL9b%B!)2o2(q)JUqy2%A~u>+LT$gFpqr!gS7h2 zl#;yVTKBujL&hN=XRGf2I{CHzbVsr_w!F>x@cv7#Y|?U($s(c^H)yU4N+6-kaKHt@ zjX>&Gxj%Y{Zq%N4C-yY9Huo`?!R=2mdloCtC=sJ=hW^yJB7^U=yF-p$eAwhvU|O4} ztyu1AYRCb|z}q*b-FNjKnT5JuAcu|6-)e9mj4tCe_Z#)sfw<^&0v-~?f>3zsU667t zvm*|CBA&e8heLaVj2rr?GhLN4%X`rhZHp2*o^!u3y#FI(kyv zVRoq3P`x|xSSem4`l!i$e0p&3!6aim6ds?;#zwm5Vk&F~; zkZ7cXX@$2MSF^8Ug669nv^Ut^3HLCPKA9$b1sy9HiJYL@iU?yBdBmdLs|LP`HX@C0 zG7OYp$+nhfiZXf4)>azcxir3cYXgl955z-{)ofP((HSCcys*wvHAuGDNQRQNV8K%LM>y|y$xM_+ir5->e;Sc+XzzP0E?IgbQG*&Ncu061EuUTj3U?m=NC4`@Q|7i%q0F4 ziB}>g*^5P-Bcw*G2t!U(>#;E$BdUmU8GZCepT~EHB{c+WP!NfFS8)JsSiuV)#->2If z<>XWTLO=N5N7@Nj@j6K(gh$gS^jF3q$0hL7jL=O(AbwLafNZ8CbH0WyUxlnrL?_Ht z+jHUVOX3KQkmCFpiOZ&!S-lK$$&gIbf<)&R9Ln*e_bYJ@Zr?J;{bdvBMjwy)4PJ*B0|X&v+A)qG#@cXUL&{*Xcc15i|MqTTW(q+B*z zhKSLG9m=vk&#As0YT4@di*lE;F6g(k!(vHJ6ZxIrG@Y1(=`l}aS4|aQn`l;}iaUfk zHN+}I2oaoIc?KzVT30m|Qc))JAPwFU$>(BSO|ZaIDs6vp-3k26Dw$s_ z#f6&Yc51Lot}2E{gV`yTfQn2!DKlyayd}P~O!nu%g&3=a!0+ zyM?`}$ehm2nk`n=tdM9=&w!OAB)wM&Y+;Uc``~C+b!{*q zlB4;AVkR^)N2@Z?Nw(L;kDta+7omcoW++6uHCCjXW?LlU$g7Y9@3u<}(%h8^&)q17+{@HS67)AwOBGGM%LiWR;fz+Z zG9|@;HEW^*l=_4%k^T1#ibBmVjn{Tgv|hZW&Mp#klp_~OhQ7(f_&zB|NQBP>^Gq>Q z#}%K(O|{3lWKYf9J`j28kwy&fy?G(o@Q%@WV<`HNw|BYY>mBk(&(SY(dQ?e)mDCjZ z^kH5c)Dl}7)*W=#CyF4%=h-e$^$i)=w3IO? z__I>b6J)~`%rTV=WEmfg$o4-3kNTH?J(X!Ez=}TY)H(h@aSK#PfylT~u(zY9h5TVy za~=Tgs8(zuJzq^Ny;J5N&X@o~ejoglTW@IFfAKGm2LXoN26AmMyux_FM?j2~5v{nu z_U~0NqfxB4!PV=`W_rG3?E1^kP8JpxXmN63^nZTinhwyz5E)r9{6E4!lteY`F8G?T zyEZFX4Ja)~Lr&qU_i+h2P2|HS6m$HR8+BXd3o60yf8rCi$s|;q^i2HK21&Kk6XwWt zip;>&v)OrDeXmJw%N)Kki+e&^o-hT55EnbEVEJPA!S%2Z!6q4NPX6T4^h487Mf~fa z&pz*jevwY6gV~~CkZlijAsqZ$hhqqnjhdIbR8+@b529qg=!4rswccAkG?Hi1_D5?2 zDSXy%hd1soK5)e=>F_AlG!-#{;EH5@|NM^2r_8}9EfJZ+>(kZM70>V@q`{SR13U>_ z=q@-A26X{EZ~+DgZV>P5M`nMT{P{Ns0YX}?H<4X;C3u2h{~qS6=ZDkl~iNIJMo8Qsz*QmV}bvM(-9mtUy4 zXGM89EFLzty)f>}RT;V;7l|x3o>x$l z-uY<|cLn5RT~i zJz}dt|9f+*!#;kkyl2UgY)SrceztwC?9LI{mvR|chNKIv>NMA~Et)2VdGBee<4>bE zb-#w3t~-RuDJTvdG7a53!Uh=Q9=*VPCi5bxKGaz(Va%;!_dvi1yBU4yYpdgL*HynT zZUig_R94Ru#oq7F2>9M-sH+O!M%7bY@>0#N5ScE=QqwO+e|j4^3)-kA>nMK-!{P&+ z@v;?{DKPGC(vUMw7I%orCw5AykrOpo#CRxd3L(wTG*0&Eh(AIJ2P)K*>L$C_?3Jh= zatZ0^o+Z{QR$_RDT7rp#BkrwuvzZ4;&>f0KSh~C`nb!nOcWrA{fk{yhCr!&O;?zt3xd6Qm5HOFrk+V1`4(euO)2n0PhKqCQ$(}lPS zt+4oNo>?N7t%6KN<}#vYuK`AS6A`d%twkHDY5hC;^a!*RVMvT7D1Y~-U~VB}#vD|C z!-_sJo_hgy44&6(!gKsKBW_%a=xy2ze90q*F}tDlc~XeW6d^mPPJsq$hV7{GvO0F( z#Hhx5d3tco_%bT{4Z92L>Q-J+Wdu?wn?Za|&?Z*>LIB}d*-4F2tGh z%jurZpZSfP8f~ROmVgq|>e#&V&-B&aP8i3?=PGX#=5hD7V1;K&D=4+wSoqFycVk-? zy$^qVetPV-6Zxv82?$VS*jb-)naKCY(s?@mW;i14k8Ip?5hjIe3R677=->)!j8Pc5 z79TCJvYG{u8rXQ5%Q_7zEA?v=gHomFHR&CDDbhnQO#OEu_(W^K>MyoEabuG{y+Ao&{E+Vk_o zS8hN>q7X4ra(F!|-a<_I!%m7pGeXarh#GMSID2YC9zx@HyW!P~1t*2}tV@#3@=ctx z@jL#kyFFxx%C1IH?CIiIO5B8>h$20(+ekPt?Q3_+e!p#~b0 z`KnS*?)Lb*d+?AbLeLAbCh&swQA9kN9GEh?vyi7Ww3s8e?6q@ER44Y<;dhGgzMZ}e z-*L1WFeNEvEtd+R)@FT``HY`R&`0bRlPuwCFHFFPwHI64|Hv5UE-!Ez?BUCF##eOkWGhU=r zh0b}kxxtV2g7rl9#`x*!>#YqJDzeqxpmYEWkTU@7)oZfg4YTtnKrx<;+DSa~)cp#SIt`5?)xkLQkI63!13 z7(Xc(bg;>*=<(FQEYc=K5FVW|f^0=JXY&LKW?8TChsDzlIExFPVp@oIw5r~)XBkUG zI_w9Inoo$4d>H4%?5$Z=7$fc z!4t*?+B3XRV_du0M0IxmF_cf?SgJ&!)al|tgGn=$@+EO{mAVq+EkECzklVZ=E9<}R z0_6}If75JUn*0*E%eyYF(-YF2Y_WeDRT8_V1-Z=Dout-Htl_=0{4P#j>37B~1|zD5 zL|5jrq6d{&gfEtWoX7cH8G6+|5V+(yi`v$G1%joDGLbKHVMYb_JH5j&oJlx>fLG`A zdKvd9u{2_e_+8}#f;aB%lZ28ItuLyImfHu|jPhW_ zjk~oCUtzF@0ws$kDK?^Vc0WCwWhC{s-6?K*qd+JFBIOi7p0`DRR{Acp>YH3GIavNs z8jVE4&f$*AKl9pIJdac0c^kThE&awG?!Txrj77;xcr6s_WS9?rOfg^kue%Gto=cjX z2s20d76~2vRIaLXdlo()>b~+<;f1dr+62{{z3*T18~v13EsR7FCGTHR0aIF}(0zpk zmdF6q6bt2hTZQ+l1fCz2my!-fb&er=x2oa}R`l&MSt+yKm&EUuFD|D4)msSD1oQ`N zJVirVXOZpEEXA&>vLH6jUnnI#6n7UgtXzFooqqfK@#eU&0Mb9RcP(r9eFKdy6WL=!LIBEX`7H$*v%uJDtv z`tYb%-p+|TH1Eb=h{vCkXMK2tka8@MxQQ%2o|~o(9oUS_jbqPY-^Hj>oypq4fpDXi z19+f^Q`h6Y>XPRYocSo48;CD_41_e#pTWxf#xZXXyzOvZFb9beQ8ZIc?s){~yLWw9 z=6Rnb0kYMSj&B2G`Q*$WwE&4Q)(bb+4k4Twm!v)%{rZ0dZ1ET^(_CkG%kV=_1SxoM z0mO~8Suoc?FdPMQGkCwfFj4s*s-8-|tVHBNwkWowL`%@w#AC?W5g};4p!L8~H6o>S zYCj5>^}h40=vgeT=`m=pY;GK@X5;%AWc8!-eQcR?yT%HS<_9K5pW5LeET0g^r4>j5 z*JsVQ>9Qo+2Q~OTuamxbe>)zepub?EvGe%Rxdu-Pr6Y!H+f(KhPSPhjSPtkfo2sFbyO`$-GRPSIDtD!(r!(oUN8olt z4L_-nuX&S|j9InUwxp+gEM9EPev54IA0rdIYJT>;B92bqVH|SCp-31r_d|Z%4X&K} zl16GvV(jMg%GXKS4^8BG*8zvk3g~5q0exgKyb^H4Z2mem0ukM^eNrB~xoZj~pps#H zm=E-wr|;&}(_Z{Luj2yNS9v_D=OaoUdn$#R*JF^}d&1}*c>6MdrU3gz41j89yes?c z?xNV{=hJnGdG_eX<(<<89J!Ou03^*rgxkjbJSs_ZGW7b^=bwtqCrZ^&-00j7ZlrZ<>u;p7yW^iVAS6R^#2DQ zfA!N+;70|{MQu@sV6irt*5&0bSry|kYW!hGa~au$ub?zrGXa7H;d4ucUuxveYmdf2 zv?f)nq0PX8LUE`;h5sFk?CZLjEZD23Jg$ySg*^fe z!L|RDyCotl19)qJha8pfs}FDHVV>lj{=rq^N?5q|JL0`vjIByFkZvH-4e$M~^z`sm zJWQRj1G`Mq)PR7J=Vgv2Gn2k5Ng!%dkA*5d5hl!kYun5l=Ds9qWfAEQ7I{!1c zfJ_HgxeQDM46!8fGp2TgK&NgNN?GQlEf*WLI=DP-5DFD^n7f%uuL~bmM)S9oNJ;6(T3gfFV}apfI3lvttEWS zjuwN~EcY{z57bFwAf&s|3MAOSEAX`W`^19&@Son|07TP5B5E|Ltjv5hl^(Ud^sQ=% z5KfyE{G~RZn1mgLfdodVn9*v>%2%icG9sOi9#m7P{t|&KBI+Lhi+9wZs?uUb6&l3y zY#YB_eMYd=jQ~u{;=ZWnnkQl9bHM&a#Q?PiU!uX2AoEKFxu8iE(=se6d*2b3VtZcn zg7i9F6V=+gPijiTBn*e4ilbv7%?|^}2_F+tM%*pg>n_utVtYA;(>oOd&d!aN)wsxV zsw1xueIJ6kt4Ei{A2t3gAN}@Kp*8JTM?T>2_UBHxXkwv!V4Kq}x-hXQ*&54vhTY+R zO+E{W)^HI|nx+FbMcW9iQe1mLll12_p(pal#c*%^1kAY_2y88ox2xj~1|+wUt<)%N zn94rA>r78C{~tv)a;u>6!wn&^K8i*ay>B8M6DS6q_;X=S>;^?r>(&Jz)4R52c#~j| zt=~YadtXy{)?yxNQ=i#;_kxsvvC^uc)xY{JlK#_{kO%K{337r>MwaR__ctO6d3u<{ zs7Z2i^DUnaZ-in1wX%idBD5}qj{I7W$aaX)I~FLXo-z!{ zTRoN=!XqLr61(aC@akhYQtMbcp1x>-eQXtTJ~$9bZ#ZOHl6=%C$>WbR9l{_ZYuEy7 z(uhIEbzS`YPy}?fCEXEr-+?8+-y}a#1s)=BQntH9>SR2oAeFv}2PTRd`){USG_qT3 z==m)8K58;w-en~IW4Q|_{xeZH09#H^WvL!xkh|D!UFJq-ENwD>qrh#lEpVI(>$4DL&rkoNLU6`87Et7)(WtYRBAlL|YiT^onV^a& z%W*}UdsH-m0XhR^dM>XB*WB>$eQL8{Q--BRKZrafC?-}YLBwEMnH$o5?!M*gyF-~4 z`R9PpE!}WsfXbmw4F`WciOdj?7Cp0|N+hbWO~fVEO}tmxm`x!vH2a00zrWGw>m>Zw zj&P28)P88DC$JGoyFCVS#qKq-0xR^7sa>nUJDwt#W-FQ!W859jqDZFx?S&dCdYX*G z+zP8AZ0-It+VmX*#8QlE)LB`)Kor(&)t~-xcy&dk?{&)aM)tg*CKbpAd9|g{6}-lW zwd=-~a8A4(zHP0KL-cv^K}y29H-t;&0%1Uvc<$tR0!h)(@BGCDu!G`|;LgQ-m!dgt zy`bE$QMDg+RfHMEb@{>rkqx_Tk&F%>7km$X&XZ{bBWfth5QqX3K4WlC{VQLmBBt2@ z@7jE#^+*r;xi|!KmWEWEnQFRsk$A&Yn4z`|<+3XyB)zf@0y}Z*KVU9ZI>|H&V#Y!T z;4Y|4b42@>Pmg|3><4ma2x9>U-AbUwc#e7O1nl?CTg;5Wth|*EwyD#U)C(X}0 zViRO?6Ca)3eqmJUCOARItsyVJrnFCK9SuA)}$p4oJ?T%ge9T65r-J=p@7f6lVT$PtxN4eyH!=nF%)@wzVjb}b0VKvr~ z+0+Q>V0O)i*@Ng*TOZ!bn;OkDtEC1qTHRDK@KuqrhZh0q7%oMGOs*@TN`ZEk^U;>L zPQFGLYmejH@Wr(Ld>#X_u^iDTJFDD`FP2}&>-;8W*OkaZ2yRIWyG}Xe_89wQKPYCX zNZd8inMipH67)&nn3Fz;aVN;d7q7dyjzb}wq6O<(E%)q?|87ObT~zeXnuG$}%#R}f zqXo$2FLu8>AY;v=7mx($1RS!H8B%iA8||!`x%;#aW$L2iXqO6|7Dbrl2;Df%>4lr{ z{lUKD;t1$vWcj?%N%;9(qX`KqYd)wevh-SwPYWFihr$;Gi)N!XJLKYEO^N}TzrWK6 zH_4&?klu_w=K&&~JD(Js`Gc}au|`fzD;dB-O(uxrgbb889})tn9UUzsQcA|KS~<&; znXk@tEivd6kWrgv8K)VtGJA#wf=mpbQ}rzq6d0MNgdvR>Xv(;^>;VZVuyL9;0?_z| zZRG@(o@s*IB_R|iAr!#!>hIU0-BR(;)Mo4gS;rQN=0^+ayvSeFrL3dID~GbewP`H+^kH$-m}iDIyu*acj~oq~~)DEqLfn z1P^&QO7cf<29!3a&oJ}t9zFikT!76?{PW#5#PE>4iU~_ifMGGG<#PS`^tlv-$wG#a zJ^!8x_?9Vp!ZGf&-_OR&z((#*I(yIAnDs^V!qiThhz*tgtr4>_9o2$EpM3sEmSL&BPUGrZG^n_*yhy;p z?>Y0IMcesTee!#fzqtI!{?bIq%fb21iQBWMInx(2fSU^Tb!z%r~`BG`N)C3!3Pf6Z(@IOR2? zzCi4IC<;X@8Mm5v9nq9*D#A^(zQv!zPmM1*Ayib6Et%-t7fcQ&sHq?K$gt6gFu;Y8 znQ1`U`+vGAtSlt?=b3BCt(v!nHM8M>mwXJPzt+!$&Pd!AzC5tO@T0 z1s{O^@9$C1Mr)4H&LDm;8lo54;|IptrcS#zUDW+hOn{|~oU;aK^K{^ZOL^|^t@=tzLy zs)GoR2%=wxh|!z@n_pL)Z2$Z>9;eXsQCM$JHJzOUpunGh-u$|v0(qwq@20uh|JzgM zH3O{sD@OwO`2&fhmgyedAWtq89U$6^F`97`1>@UR;l7=;M_cAytKW|ffe zhc7KJZ}N>0r~bUB<1X!4zH1V@%VUUG+1%TUyg|G4G6A4r+ebpOzf!x4Cx$7ff7`#@aA?o^(BR=C8N-zd&)SYTP)upM)m*1a#1TzTY6U%dK*vdNDz?DyY>y;Uff9`3P%7# zl8_6zXqnrt0>kk<*{NEc73Z@O#cI#s+ zmKOrYb~qPTg7GXbUTO%(M}UziBi_m=b_^*viw3jZErxYt9-9&@A}t%)K?lb@A%h% z)J@N4URn=pFkYxaD|$LC2Mp7*Tgj6if_LI#{^DMv`w2w1uU!x@ttMqFjaVbU-kEWeP)nbD)(KiMrF=u=CJPJR2a_iMBE z5dGmRr!HVbvZM^p{=lfX6!$7$o$^ZB^P#nN;66h5T%Ag-4-bWs)xs4<=c1Pv5%)h% zpE6p7JJXMCoF)C^nzqqn@kL$I0qn>aIM?ogd z0t*izSVT58KJ>-XZ46WL+vI=Jkk|B&mI{ROUB4E{z5Vax&#jLRXfvwUf+6sA*7xl zlPBZ+>$tII;tP}V0=7%_5Di&is)+m2A9nFo(ko!@uv82WCJ&1X(ulDvIy3o$X%q#` zrWVvK7Cu6O8vW>in))I2=;4)yHVHt~gEGGCy}F41q-NRb@HaOoWFY##vzOo`C1b_4 zMV6(XJIu4*9bIReN-(Gwa#B;sG_XTO7UrHxr?bBTUNho9co7pvJ0aF$%suio55)q1 z=eZXj>Ngt*r4PO((h+O>cN!)kwOTY)U>#WtJl1Wj*%5%h~g99`b9e|uJ?FA6fB`0*&kU$o8> zs(1+$yfD8A30e&5H-OnNN|4tD85X64C2WFafk-yJmWKfTv7&l9^N7$yJyiXU{7kWT zD~|gu5@Iq)KiFL0no%m#fIh(Fh8uZ?!fL?MFHJ9ksI-S!k%oH3C~M5T!u@Brt__99 z=={NB_L{aDJ!_Z%zI*3HJbwq1V}o1O@+u~4MQ8!q$c}dc zZJKK>k--h#yDj=O$wf;}RfzWWfuf9TW+_dIgr2(*1|q%%#{RMTRkt=@#Jzh*G{8P8 z0}z}^2~*6fg>Y+Z^cn@QNi&dTeyElYVPcbELSSV9{)u`BFdT}O(@VpC7`N9&;HcEX<#C#I%t z?K;&}YOpxndaH~Fm^(4=5z?Z)APY}-YC;R=V0r8Izpw_qjLg^5(Igf{m=^l&_JYq?=uAzd1?5--P=ujna8-_OL+LqVF*QWw7l;pGqX}5!{iZo)q>C$ zWyPKGb8@h51BX$^NDWy3kNM_1irY+@NfCGR71}flLVBA=)nrh>g*86_E~GH3A5vFV z4wQib)X+Uj`AK1pn|3rZkO)hmDHlc*5rKG30~pdkAn$(EI|)*u$`G4T!;y8>xO@MC zF0aCx5{Xs#q4R}l%s?0a9J+Vz$Mvv6xkrMXrg!k`9mzJ|<(dKjsFwiv)jeu=O6sBh zk8K`;fkvr4pHmH*?w<}qjzpVm3y(8x^M0+ZQ({G5uwGg8B~2mez3DF)tKd+7;2ZA1 zcoHu)}o547Qfw~;EKJE(j*v@G|5dn%Ankm(>-SXqw2zeyn(C1w0?Aun0&%+=q0nzI67`vOk6cd;@?U zJdg7?*wz?X)2IzZC>Z}2JY$nsrCce2fmqQRxlq3Q6)vE=(6}T?Na-}0P9o?A51?Ko z(XQ@lq*4*!mc3T#y&q4cV;=D1F?Gfk9PpwP^B{(8biwOhcAU&FH4yw`N&Vj7xF>k% zGdJ?LdsI>Yl!;B92^yur%-)V+Tasht>Q&3Y*I)UY*1Tg)w|C(T=~;1?>~sFOO@2|e zQuwVNc2+Y61~t!7c3zARQ@08nkyw0u_5VT&Myv|)3VA6IQX*fV?j~l_3rJRq#;cxU z^gUU)^cqRY^1)UHvX2ICyDgU8l;iknei1YIR|}g(83fv=K&Cuz@nE~dr!#70jlpf? z-K_^*=I_zoNJ2r8E-91$3j@@9l=oXq*g&d$=L5{Y$ur7#Q16+ECb38DU)`V-J%I-Z zAR+YF$z5WH;C?kL`J4rcL{+|jr-JqX)Le0k-B1msk3lH_iJ1SCl5_tbr6iC@(pyIU zS6!bg_ouE;|No<|2jZ}R#0Q|>_$%WKC;GEr1Ts=2szb9&6H;0*C9tTEtzXLhoyWk# zX+oH4_zhp%*@~$@bLIH=FVWt}p(2S~i^^8}VqL7FDf!2Yi*k$F6zIWs_={7GzWD=F z0LdpOcvf!x$;j}3vv>kg9inIXD2Ew1Zz6?FK9l|pPRsA^@~)r>i8se3**JB#qWIDvg_s;#SM>fc^F7_F_CTghZmr<^D?B*Uc=xa7E!duKYj zAJ3@hD$bvI@=kwkVk=+X3Pc`-pst!CbIlrqy1EHm@9%AQFoN4fFNZQbr-np3EVV9w zA4=yI5Qri6Tl2)D62KWR(u^IyI&WfUVnVkbFHEA*po3!$5D*d~NEfj~{YJ9H4UEel z#Du?LJN>my#$!3y4c9OVBWsd;{9bQP2Kpy*$pRIW*vW?ro=~j?QgRG$U)swbjdbXZ zqIjHN@Ge9o`-%Oo~isF4`~fG`Hd80yo5 z<=cu?iBzI{SG6*PR*d-P7YL@85gIw9^X<)S~V?ahYWCSbQ5BBiSg^b%}1#OQX#T_uVNcn$)*`2r5Sa%l5g%fe!skNJ}T3&4NzI4cz@U3|DRm4*8>nD%CLiKd= z4V%gL%NM`*FAc!}a=gCuVtLJ?meT$c2N!q!h68HOH~R+U;F#an8 z*;tY0h)BLy7z-GMuu#xhYr2$|o?^+!E`>!Ny2i9;yQlKup4N+6gLtQ#>nm!Lo&9}$ zMMcGhR$r>&;o%fstC-$1ke=HuZ{%*|ikH8sMNgoLnp#!caW0+b4(Y^S^V!+kSK3V8 z*EcYz>e7s17EW2KXnFlBVgS<_7`I@zUVD8s{v;r3vX(OXS{aW*c=Jaqt5qEJsX?s` z13i5JjpdJ3&{r1PT3W=*>nAk9`#oIk!;c26;F`3ww2vm3{AS*;D#m4)Hy$ecrLhX{ zIsX)K{PvG%xtv9__wN{wkrGqr9TAVcsN@HRJWewHPpe)IMRW$=ebEy^3Y)88zxTkP z+uv&N9oGn6UFY(X6f${kt-;97R2E?_qlWevkIDCsXDfdlA6w`s%`Ge}SR86^{^*US zQbQ-iD^Pv+y%pA~p`no?=s+uaF>Z9L|dUyeKmFURM+^z z3|Qy3CP3sb31Z~1t>3szTT!s| z9J(ELt?ccgdw4usU})gn>)4G!0wq&oqHWk3}C=!Um8O*a1;}032VPSX%H$4tjck3XLg9Ep-Kl%*Zj)A|XEN!15 z;$*kJ9zZgWBE4b zY|xNPj!?DJkG^H<-!0co?^eFUat+0!q2JAr^f!?==pSwNa^YvFtql!iw1jV>PW$CVomRRFl_(mQBBYpp zn=ErCc%(l6yfMdcwK-p3P9(Gz_p^Ww>K8_zbEd=vn_00`MaG3uG=?rUd+A0q7@iF7 zCx@w5`A1MI{5E4}O?da__MP|6D-MlCzqh9~XUn5s5IW0=A+M*-xkhO^4Z8U#f}ot? zT-^#KhN`wi46?l`0xV(e#fthFuE+>g(FF)wG$bsOz_L_0E zv8uG}YX;(nSd;Jc?mXK{-n@IJ#^TEJC@EsF!!D;jGqio4TT*b}+UI~uk^fR<`3Q|uq7bfW zW6qU+GQls(@1P0~IPN$?<|@{V1Q+DXY>rHi(XnV`z^az2Xu|^@tuStJLx;EVw~9`; z4wk#_L*@27#$p6!lWz=Z-(ruo`TO%qOZ|B-Ot1_gCz)^zsDbiv5Ov)S=_jW10RFXVc*Ku5C93cW0(U58 z`0iVg+-T)9V^Sx%-XKmprTlp6F@N>2v0UC7+kQ$b<(AR1@|QztCD6B+%q3jVFG`6K z5PfLc0XV3rA~ql&{&{(8%X$fmlN5@cp@M|O(^eC{gVIZqgHgs5s^8YBTVk5tNMI_;6i|5*%_$ zV1oq<3{fkz=rK6?tQ1H8CW0l-m);_fmDna+Aa3BvLW5IM4uYoKq(#RYzu(K?IMAo) z#z;q>DRIfU)@Ts4j+`uPLuKTepaM4abGfko*&@>cnyfL9V$rCL7o zZE4}Tw5`M{s-zMhhJLKuuW{sCANd9faMYQpg!$jEYeD&H`VDeMpsD)zSI z&njh338`M3`-&Y)#BI}SW{IgNCGtUOQD;_s8w7J(L)CAYH@j6(bNhn9qc{CigGplO z zog1aT4~eZ)b0VawUi9HHyMBi2u&mflm&t>-qSu|Fb}r9N=(Oq$<+H%QYq+aw6PjNQ zM#qT*HxZu&(Svj)m5Av4p0G@41lUpEplv6J$lt=d>J#-|pdSByXhXte;zy=W{EIck21jk3zdb@#~vHLlWaF8N5xy zikWsSs%ODIX7&M0$&~qr(k1%~lX)`PllWAA+hlW=3j^4yA)GwiJpNig2LEbaAKn6? zdiw-A9n3$>6LTCG62?e$l@|^llPr-nv~$YKrgRSmXD%fYI^LHKTKzAXNC{MmY7pL)9v7Y^=iRSTYsnYt7iiDK`MEbiVb|7SF%B#LxXYl8 zNk@jI0n8fV%6y z>Sc#U?yQ?xltM(XN2j+^vGsD!6 z8X-(jSL=(ZhRIl5QCYMOxd3+74s?OINs*+S4&3i(Tk3*h#HZ{4c**S-zD{A|VBq4pS<%yfD1mklyw8C<%yZajAssRI z8)I-u_>B6V*8DjZROy=`8VO&9iyrt)qYZfpvsDd{d@8mSMcpS6#oUlQ|0!Ev!E zV@{~&S^pmVc8lng7Hz4TLIfjvpMmCbgIk92l%g5~b^DGsXeZc|$(;(I!fW+fFVOW! z?EQNh+5f}dTX9p1DIwAzrF5q# zAxJlO9gOGsy>Z|B2i!5v&@m4C?6vk^)<*?CsIJa-I$UIqG%c1+uhr{RlSi~xr9Z;d6Kq& zZMR1d}|2xrRNko=E8mW1Aakk@x*#GDm z2<%rAbyS)54omBPLWfDCD0AU+`a$T5X%GczBA)q){S$M|ecnxmKTQsSAX(pkev=d_ zlaRyoHay^#d5svN5QT^H?gj=O5X^kX+nB$S0fOqXQ-6S99+Hycb58fBZ(t&t!-L3| zjK_vr@7;5x2Z?@CK6aJte_II;(Hs@i;^GHbvHrIi73L*gql0nNOZ7GoF*bAquid~* zv?su%|5KYIZ{!PfQYl-$Ot4WG5B~L{AUfLT3m({Y@3Ch2A_nZQjH@V3&WM`#^TmO^ zeZB#l08uWlAQ+$uTQE|Y-RpP%(n;%6Kwg--YcUaTX(ro%VM;@&1_yv~W z90;y2Sxi4=1A~f*h8+x@5!Ixl_t^!wa?@Qlzf!D$TGiqDZcW^piFmiAp$8sKdG~9|$ST3_f%VpDVRb{B6+h7v|~}QAfO!>H|ZFbdZVE)P8~K>FJ^Uc09_UAzAOC zJf-{7$|5*8hCQ-4BDVsW!LYinrXh=Xa#AGu!RY20`QiDz4D!{#jh6Z18qW%{GMh_~&`6p4J(M44Lge^*$vMdE=K;clW)2j;~?Ca(knW%jwn= zx&ig^ACrnK$pPNB25;7=+?j`-J)Wp1plI?s_6t_dp7M-r%1Db1-?zi1h3N*!%&8~RDsa4W zdV7yjDP?c}1xgf1A!ale=UX6b?LU(@NeizlDxl#x5cNl$=<~brD|hDaVY&fk(WD>W zzke^%vwQe{Ms9Er#+l3}7=DZywz$7^8XUWb#+h18puEJu&Znf&Jy4Q{A>3RZOvpo1 zbp8NM)2S3hz-@-P`Oz`WIP!AD6$Q>Loy?zH^nuNNXhZvO(MgD8Y+rADStRW{c1wW-xtC7`E z0Mm39Q6w92lmfqvq4D?OJV*paN@9^y^NE*KxJ0))l`)+hYSOk)gPj~YzhaqzNQp_I z@{?gPg80m+!8%ts|A8V&0mE|T%RH%-mist&^m&zk zQJa-dL8A1IFDZ%J-`4nY?rIpBH^{*(cleF3`cYA+W0pT|* zz3=St?%^!ZO8Sl;t}V<3zZR%AYBYPjzx!q_z4?;Y&r#UDg;v+aqe7%e68PL(gT8+ylFD3BKgHUK7omx3hw?M!yx#&xm@F@h-}&KvR@f_veh!aeBG7s#Q|{=I_pJmo)c5 zr1-3|Qq_m~H15?>%xe;nD-=a37hZ1bH*2t-mX|#x`Jx`w-x2@XvahWD85v;`OhaOD zTBzavY~zt;h!FjQTT<_!@oHpJOR`(emSdu`Zkz1yohUtA&Y`kZUnN_$b+*rbU|La( zq6-aB(s(Sxna12%Ual||bR-?A!`jO>XM%xHY6JvjnFN-kKC6Iq07EYngR_|tg z&iKwOw$$NUI?m&I?J*0#A3MbVI3lzdCWa^4*- zxb5hV9V}tkv--zf`i+$*9*=V$6>!;*%nG=|$NNzROu)AxP%RvWTW{lH?P9VFqW<|6Rp?e zCyO1&%Yi37X%yM-?!@HFtj``D9dOm$x{wW_z|C%F-746 zehkOk-|_4)Qr@lV+8R;LRY}zs!V5CP;>pFdCYvRRdD5XQ7IJ35ut;#K*dm?I1As2# zo;~u2hf^9b`OME*uV4O zl5!k;*3H6UW2I3(#1bPVPC}?WQbzB?mqFoC zl!yS5swWDKeg!YL6P8wuVgt}BR zStV|155J}Jn)E;io~u>8TT){!n@jH7tM6_A4*KV_va}j)DpoiywD`z^ z%%ddU@2|2~s>wayoFo7?ju*AHwatgLp~+S%=}$q1#B-#y*FiAV ziDV0h5{i@Vm`8EKjfbKEOhdzCU_YcLvXIdDi9tC9tyX+G?z?hU zjpt|gEO@#<*Cm{v(8s^3Y_EJr8MPf6=;G_H-Nt7%| zOt#^d58N`Zg!pfAe$QGjQ8FHe?lb>bKpD)E|E1aN~5>F z^x89_VRX_(Nm*skVJs%q>I|&}XpbaIatc2&NJER7%*6}@-h0Z+hnnO*7VD>XXmNn< z&!5T0ZvO;!K#7RhYj+bUxyjhK3CO=~|^6myTAoC%PKP^fMA-^OD5T7u^8{Sk@zTyZtE)qT-*~vHHo&4weQZqihWd zL%f!??A7sGa;@(Sr%}8}cI1)Qs;+t&XgJ&i!NKlN7um<%Rku%iKGiC{AtYdl;vwwm zC-HRosIu^Z!J50v0$mYi`eEN(^GZ~y72#Yr4KAu;S-NxxmcQDAfM7X}+ZXoB$Q#5H zn5OO#8imw-gj5yU<+JuRh%tmynbkUaC$FdjKQt=gPUJ$z#a%N$sgsxL>W^QTT`pO! zMl(H85Wm!&Z0ae>pgE6wOUGxTsM=`(+sg@S(o!8-%x1m@GDvRmmgO5i*2oH zSFcA^82xM?8*3q@ub@MzP^5b-xs z1q7Q9@HCUq37|1F#UIpB2bA2Atw=soR=&(RlNyX3ptlxRp*!PGs4mV@xoPs^{mHhg zOter+=o!0(G)jb7*kN%a_3|J}L^Hr*hU%ZF^~mIWyEoE=w#viGjZW&UNTGBdBd(Ej z7{2ANaYr#3CE|TPwkU&HHDgJ|)1BRBTVvVv(4sPlB!}c4If-E9(*)#zMh^+FDYGk< zBhj9$;YeoV%YhkNp_4*0l`S_^s?&YR-5K|T<`UVMW<2T4)cOLU6ljYJFTOB++2#3E zmZT#|(olCuB*2eZsWVuyLZ#!^v6x^wv^(gHiwx!3(edT|R&yx?QlObI!086`BsSG) zf^t|r-Bz>9xNMG6_*SFx6R^`)G8SLT?g_?FTWfq-t&QAh^L;8l zX#5>F#V=5B>{Tu`EXzNKL^e5?JS;h0Y8G{q8kZF84+0mLF%$mwY*7jkHF;}6K81WCP6 z9U048z^p&~HL?oZ_fgHuuS^*XDOL4(C35D>ns{LEc>j*`(g#BFWG-k#Lv1mBG@$4U zF-5D@og1e>j><3{Y3GHx=CZ7r6X6lfn#p@mkW_UlC+ogR72ObtrMqk49`Qu2SlDxT z{bC&yr3P!$rvew2<&pmLognAO*7piR)t!D4-d#nf0P`6R3ffwin8AfcEcixXK1|QnO z1-xsaFTT1GsHarPg$5ppcZBM)t-J9kyT5@*XG1V?k2V@R_HZe6f*d7=d4H-0L^MF1 z6=wVj1Y90)Bo>#t;KM(w>uFY1=0omO1qEItb2Zjfs|bROBtA5!PlJUMYDxprx%JyX zYI`sa!pp8t?0y|@u!q01E8m%D2$YOdVt|E9>~@(ogPB=5D@cO8TFXgI5+|erpVgSo zkpguY*voJpvj7XqpDk>Qw2zzQIO!4poh zOZdhs(`aw3dN~Rq6B2-$_l4b}FD}C5F2+i^E{iePvr>8)8>$h_!zI%uof`L4*>wAn z%;Vq{8a&F0R#AuJhi|D0nwq?S)EV)APOAp(z_g4>LZ#m*vKJsboT6fK|0B6Dx3arm zuqQ`LVjqs=aCC^{a27iV3Osz3ETWf`oO$9pp=5%#gxYYDuK04KgpT~F+ zSF;K1)0{g*3VQ6}IpW?X2F+O8@PcY|@*gd__E!>!hcrdq!{K2uRAwyq*@#62iuof_T_sGSwVS zO4Sq_16|h#512o~6ndUB$BPFbpL5+gb??e+gsvF5^>+7~%gHd!SO-jIjOaI$>7juG z%oDF{sGLmfTQb+)5Q(1+UV9>XY{z8IVm~HJ<0_a6HD3A|yMoR~OYxHufmgEpS?8~z zqfh~fp{0|Xo$Gyk1}-st?!pu5xA`Qe<)@eV!Rxz+C1z5?BPhl~^!rW`zLuSV%So<{f&L0Y+{iHwje&CZ3@hQ@5SRu5R9-T$3xdBByMda>Q1har z1ZDhO=_LxlAj0zU%6yi8F~3K&4rEG*0Jxj4IWakx(jWw@sN}vHPy%s_X38HZiuyl6 zh{+=mwQ_xm6mDx3W3jjJq3tmPZyK+&BPQ*toZH4QtMLl0H)fc?YK!WQ67ebN{?gjW z<*d@m!h$c@BxM6d13^tp)>vcUveUiGmX4w%2Dk`JIx-xaN~mC9wA=u@@+mM(F{u~D z6^RQaHb;^3W$8bqVAB^6@V+27X!epkJ=nbGxTdDrs^AWk5#3;*n5{l11pn+|Ak;-_ zBwlxrQ)Apzr3L6#AF9zert9%mdJ{WAozx~}HpVMsI85bO^xomS?tWFj48@^I4=bRC z`FZ>q-0aw4^=4Dvdz;h*e%_Xo6zaepdbfogsv^s&VF{FN5hB54RPnj#BD#esQ?#nW}i ztlD``Z^ovrM>k35IwQ;xf~!aco}H5@?Kn~ecmQ7<8fxrS1m!S<>oEP^89N3hse(%i z;|Qjv2@0kJ&yN63&ImXug*<|DWMg3lO&+hft&>kQBLSH<-$~`&27Zi7BZjh{1R1^x$rmbX3`iQig~xj!}oym8et*0m@%Ezl7_S`)Fr{eKGbfVeY=; zzvR7&iOk=+U2q!vZE)_M*$Y)aUzH;GCf>byn@X$7?bK?-)VBV^WVY~}{;ZU{KT<^uTU@QTSWoLMzhe6P`NpJf1%m zj9i+$7OfPIMPl}M@4p$SV0G`k(L=l&yHXv$WB3Hy_9p4&K!0HR@_xb2b>GNEU%FD5O73+`!CB<+ZV@FoiINnG@a?*7lpHiuo!~(PMaU=pRGpU z)!~j~)guKAN|nywpq0n|8UX@D0{vGs9L=m=q&0J-`cQQ!KUL<`v?osaDXaZbhq^l) zewz_+Ys~M&*W`f_Ud|lgT#8CrMtX?l#Sr!(4sVtGmQB1vRunOB^>fm+={m?<6xH_P zg|)~2zGSKWL)`oJH{me^{9fz6>!&=lUdiDFJdT29BcH58r|T+QZ%2M2i70PM$E8(v zP&;+R0*ahj%|yD!K(n`A`=}9xJf+wEz#N?2lU9X@IU*&f=E-H8@6paNo-w_AM328Y z_2UK!vux3(DUM~vX<=t%FM}p(?S8P_!XJ!YexCj$lx`kXY3)Mf;j#-|fJsOk3*kj` z7}%Hm(4Q+w-Re9Xe^7$wktQg1)HS43cBn=s!_z>%?A zrsLvu7&ziYsdtQrI7s`wP5Sw@TRd)!d(j*Bo|o6SU|Cd^IK%4g9YR&lZ3r0_4BjUH zn$nwMCZhIINGe$@cVwOA#nfb)eqJ!?f~S42inh6E5i-4UUMEukCKXa`)Mm?%pPuul zKWRS(Vn=I(K~fSOp}s z=J>{U!Q!)WEMZ~tSNCoa)v^RVI@#sg@Ij8?FK!}Fy>|-%IrV3~beT{)ls0-LLxeOL2j=ojv(@HuK&0tz!4l zr4Pd`M|=I!#|=9lzH5!;7beZMN=Jrn(!QJjIr0p$go~=pJgR#Bb}uFI&}sFID@%P2 z2TUm95=KoCKGzdbbkjcox>Bi7g`VD!({@%D9Sw{qX#h^xAMcpPhr(b%8}2lxYQfR- zbCjfjd~FXv$Fp1Qr=WZEO39klZSR{heR42_2G$-)!9Q%i(i7i$yt}H-56G4l04>BY zt20-}6l}yUY;Kme#WjLFo8W{nQfa>=wf{Qjqa8Np7**H?5<2gFB%7OSsk8PvqDy-_ zNg(7O?_>#IM1eh?@zi#IF!_Bf0e&oSP%)yF`WUzKeeksH*$Rbx288JL&AhmW#r^Ri zKWhy5Wm#yM4^phu2cw4xHjkdoAgAhcQ}iX0cLa^Gu@_6Ra;DXi{rqCq5z-!O4>UKR z=Os)yFj01uQf)cY4xdLCPCUR_E>q>DLnwr1&%+Dx-vm&%EQlO#0AJVIp3(R*>G z*W7Fcp^6DOKBhJA`+BtcnM!MKWb&b#%152^6Bl0WD`z?aFngid=j|qDv#$?r=_XG=YzS z?SA0v0!cQ||FN=zI5Z}gaiU=3Nd=P#3FQ)O9Bvad@ITltdrd@eA}_V0`yyV8rinmQ z!aF#_-tL5*^Yom0F}L5R&wrBnUar_&$SXAl%!YO!=(eqNT!8i@P&r|sAW?`*W z`8}Xl#_L_^etAPPE9f1HdY>Dn+ty`Lh%;jz^q3k7^jZZH!7eDU@hgYm#TsbNby z)Vw0m)7z2FsF@?yB^yUX)qmLja(17LPWUuR6O1v+%{ z1I@PU;8ZwoXo-y(^6AxB286VKIy`{nnMjrZtF=zZ$(PY$w|~iUW=5BX{gmw|Lh>UL zoa#u50bw|@@wNEObUR`a=&~fh;6?kW^cAkBLL=uLM!W-UuHB5)1?fEez+?S(rFW;0 zoG#T*R`@-~j*O*ZetQ!YMV1sM%sLp_cgBL0`4w&THN+&#vlZ25%-rEe#|vNFsU(7T zsP5zX(vjxb_3Fz3*{ku|O|Gr7CcFhPhH0RM+y^h~F)oj(n<`0&6<;MBMwe`;{hm2E zmV;u4=SKh#Mzk7Vwy{BxvMM!{DVE%dyKW_YlMbcC07q8!i&zLM3EGJ`3J$Ko4$oSS z5aB6IBGOO(`V_U%$p+J4PS}a=B+f$K%FdG>r;xm%e(f*WNS_JH1IBNARrIDicrCKj z1922vy;cY&epxH=S$F153ul6`w+k9}RK?NJkJ6+JFtFW*DVxbd?yqX>Z>9*~7zlYv zL{g>vGkSio!oTs5EkmyCqweTu2UE6kgN71;jcrZ2_NA?};MUR};toIic;GWYclX{oDdLgYjK%-j!m}$zAQ^?!JmfNVY;R(To);%%< z58+~bQrY~Ym8dZ@04m+WAa#cYvRW+fx>Kor<3T6s9cH(D1`f!<`1iSQa;~8y-Ey9G zlMmsQ+6>O9Eb-)eKI^JFl1;=3%r8VQJ9ah>xCr`SF_^CF)#5Rujqw>Y-zg@Fc{EmK z>DquKrd08iCpM$9>e*cZUhmIz!5-%)HAONY@}e~yMdv8GJ4z1 z&YvxVa+NYA^MXsg)A_>X&vlBK;^%bKm9j!idwN0@xMB()Va7&0Zu~sQva~+3^JRX45E7^{Yv#9?iUx1Y`sT&D+`tBIIvQ-|d7d-Bmq2C} z0T|1(2Y}v*kBj}IXBGX@vu*-CtL8R5nAqN?6~?LKA(H!`i)OXEt>z$rz{%aGvDjgR~SH?;3%TX4VC@>0uHYj8cgrh3Qp1fiyxkM)O923`EDm6!FN;QDHg zZ!si3!CZV=GYN=2h8d2eL~%B_j6~p8epCU6OJFGV1V@gbF}>_lDCf&n)~th^=qJq^ zHx_mY`Mj)*p`1AlIV;PDCv*CAqYv!XyA-UxO_9$W%-o-nP(=uBSOsw;d~^@Y2abf` z)jK(It*;*6rtLW-iMs3Wt#TUR5`bZ=(FpwW9=^uCk>aSDjv*~8ILVdA2+irU0I!fR zyEWu7i=eUU;2<;4PJZq+n0HtK43|Tjd00K4%|e6JmF%sm`)l3VO49wji`u9lh!u1b zk5y&a7RG52p%+T0y<6k}v^q`qDqyJ`|73fCb&JrB6!`}O{M(CLm>f4S^GR$t1O82k z)WQ&R;0#I8jR=Tx_7jf~lW(p^R9%6gkC!_@LU^_3QZur}Bg^*IY;5rgLsmmx#kizR z>B2>xG^BkKg$yf`2qg$CK;p4C+z}FVbvnYq>=Bv0#R?t8*VbYprqP=66L|07frZ1} zJBP`hY*f?jFGmUDUd`jOrpht$Ho>7@LI?N;Lf+i#K^wj_x1k*O61qb4vfVQmeQxJ{ zLyPsj6dqE1MT5qnSy0*&1~(e+4|A`@ynz|QEt8z>1HEgKjuB$qrB=ou=R}zYPyk^B5IuX!2IQ0p{>(e^2JXJ-n?qE@N*KF{xLwmMP zBzAZmy9t-mL@@>^(Zycdx;mj&4bVSCgDw zeWZ{mD|I(NSVNn3p4Bgqi^ks%wNv1a7U6RTaU@Eey*UE7-sr@0HTA|DVj9h+i_-H0i9kUcoXs7&pBX|I(V68D1}3fz`8V{MbH z)_Y~uLL?r^6!{G^x+^Z1&Gf86rjKLK4t(lB`6c+*G{E^OI_W8l` zUAmArUnK*9-k)BCmE!-#a+O9okX(-LWuq|NiG+$_KsMMTKnZAV&!TyH+hmgnfm|Mt zbD27K^gi@e%-Bv~6h2A&6t-GQ-Wv9Vo%&}j6WHb6C;`32@R4=q?P}?RzUV#`Q!4!4 z=LCh)vu3auBBB}=Ra}ayV~dtN7|yp@hs9O25YF3&yTtT9B)O?MmzPbas{>0I46kxa zf@iJ-rbl?doWIYVax1rp)-uk)+4f_g9918iX6ajhO0KtNpi-Xi@mK)T`rjr zLw}SI!_|MRDp5SGJXq&-4l?(!m8(1q$C85EZTx1@0KSBL#i!5%J|z|JUa12rvTgc> zzpH-x;L}sCH<;S5Tev4hkfl&k1;xZ|HUMpR(&6DAaBGw9CMo&lUjedA1R*>0JW^Ct zT)ofZKs?`=zB@sHOsG5vLF%&l*n)C)R^i&EXz}J#R_Q74I5D>(@&Q7GXsu#I!sTFs zj|*dIz5E0I_@G!p!7zG-1lywjmr8J6+c7? z!qL#jVe?jgfoHQjf7m6UZVbv$1DCZr&(oS`POtp}pPVhG69abXo|jQEsC>WZ2a+3@ zmlE*$iK)r^tuI+r6;hv+4Bh-oKTV;$a#&2I4YXsj6hxeL5PyA_A5j?>%R8Ez#t`cYXs0}>j)a%=5J8bcTg0u6rLpgi_ddJwxI^| zb_B>tHT30gm($H75s_to7zg%;!e)>z;yE06HYNQ5y zxHvzJrw8Djf^kg$Ei|d^2c!U@l6LpmMBGP21j4%=NH8uhdj0diOdp_wD`I`Fr(P za+8$iJG|zKzC=`T9*(@Tu{5`^y1Hr*Gk-Y?e%CU`H+2`NR&UY9g|0-s9VhOA71~r!x)t0+ zj2=`e^Yi8g=bv#Z(uHT2)J_hbD6m2ElTJ>?0vx--6Y(Me3>9s*!f!o~@~#=nQG*u& z&dq;p7xPjmh-2vor0eYPLF)W9bvwaR6jw%yBM;(bPjtfsTH_dt)_*tf0AGA-!?C zs*VaWR)f)vG5Hyg>m!cp&R+o5?g29Fmdx{xXd3ZUx#8_Ck{5n~OF_^Zn4PCSgb1U$ z&p8X zz*Qoer{8isggH2>$a-Jkl?4_cWOB3QW9#8Tp4Xp+>5u zH(a--iNyau%RD+fQI-Dnq+I|p|MR3^!uj}Q{%7^B*3=gaVe5B>qdUTh1BRYkd2=2% z<*v1g8{!RW-*E+vRNNKOcN|uEE+RKZNN>0{mZ>d(4b=9*T&GN0F}ng=dPoqFLe}KA ztLW$p5IUUZNfIgGioBvn3d>~-E`L*-L6NunYUaO^vapL+e4eKyq8TUPmP8g+_#iAQ zPWn6mfT^@ydKAk9DuWP*oRX^+_KOi=a3GuziaI`))&TZgW^ImQg?)61FF)gJT~pH%EMF$SxTnFiHRKTCSGmWlxzSFYm6Vgkm&G?t3ih zK$VdScU>yoxn>d$ZP}PwVrJ}Qtf-0w_9ho-+=jp{&BKz;-c#r3rA`K>>8ivkRUv+M zM|PpcUC|zoc4gQ3=K&EoL(#Eb!=*p;4XTNR<)wyG-cc^lO1Fdk<@V7UAE03WAdyWZ z1$W*LOEqwk{QS0~eI(%R2M~eG9g**sxwdDHrelxE)3}fgJUj_bL$Bvi+Z65jq*H;Y zaDibu3E~ny9+>6rT-IA>T?A#9SLopAK@Cc21$Rnq1_fA+6hcw$0W)_Gr#3; zmq;s{FwHdQ485p-j=Ju3;3r+T-KAezxzl+r9Cat<$7{jRf}hZ=gQJz6sC186h{b;dZETEC&pXurGACoysG@)*ee0SK4`^F9#JFsaBbN9})Dj!1N zUum=N!YdX&X3q{g2IJ5t_st>qMMj)Irk0#t?9$FA0U~)z4o4vX;TUp2bd=_M>;;&Q zS%qyJQ_*VypiF&%on-Iz1?p&>t|vG~2S0OsPUIB65y^mY2=i#`(&$VYzgGhn!?zj2 z=p+i4&ZpoR=IZrFnU3*k^b6b*q@%r>VBr6n)V76@ZCK%5D6U3dUal<&KRX8R{sHkD zi)FDT*Y~)>)@G?%0wjl*0)YUbCAmq;k(8t@$p65mF%X;(MO9dU`Vl1HIAv(#|FKIo zpUYv?D$<+v+*f^OEweSDS1wu$pm~#J{9y(5P{Sve2`+XQaq^<#Dg;=4@V%|Feh(BsE)l{A|pa$z?FaK`qoX7A>Q_iWcI}{%WqE4#X4GZNg%3VURAHyt^=A zUjcQ>)9m*9fkrjIl%n}cwVXQn(hKJvlwS3KaX?!pH=5Y~vw_f-zEJCLZF#?PoJCB$ z{X%tnXcd!?lR1z>PbTA z0|y8=K>7eeV*wx?@R}nZsBt7)YOvJo)b?>J+NZ)4Dy~jeWxRGUZDfA`=m|QPd30}2 zMJ(cx(hJQOUa{&HU9sy#YMd`1*QL_eBbtC<6nXcR55uk#jYkgCI6x%jdYML6G&hJx zDDFWSxFq4^$;WrV(bC_g@V;RyTOaQIr*iFHKss_c+%Zri<3`&8k)r^spltr}CUtm_ zcJp2Dtbnm%>BB*IKbA`XX4s90&}^kJeFVRpnPJ44))wj5m_n`bS&eDJwE!CPm4`Vm zfTc0x=Ro`$q_1!`HreKvm0#?>G7!+u>oE%V9FS%+9GuWr=;GxJm}#WPq#uS~-k_Ro zkXRADPi;I0 zJc`v(wvXq}I@S0!oy%31G|M84H%}H23hVwn>r0bfOWyv`J*!~nf%L4F@CXReJ&w0s z$!*#@hOQv(B){q1#mne!yHxZj^tA5iMmIcqKiDJUSxi~545LU4LSzx13%l5SU@hLF zeSGDck|{Ol;g{8`n#ufGOahh9q)HGVfO`UGUHI<=e?fr0z!Rahw3Q#B46AEsj5>#Jd@awgKRwd_ z!pG4P0|^X0XVkGaSR270h0%GqZDnhO#n)|HG7=C_#}ISew&Z=nl$Y+Y=oT9;n~7<|k7cf+RN#?8tVhl2Mt*|!zGF`XIanhI!h{QxZb`_b zzyVq>zQwHGYg$}BplYfO@%2usi>AhXk8ZYSY*R?+Fo_*nb<38Fllw(l8t)WBWbP(X zok|VXd3s>4J<`qXdFoa2;FAuztZ<~R58cF0;nu;UZ^dWvm3vWU=QU>`P<`1qMC|*r zAdqo61V;*3ui;u`ipD__sKe>+W;rP(lzLRlfwW6A#0I8SHa1*NK!U(`Ugs#}0E81r zn%wK}ax-w1%GxInxQYk5240@*j~u>%qA$NXexjg8!l5du(_Q@q{zHKtf10nJieK7M zI;GlNtsWuG$Yh6`dU2lJm5IrJ)g73K{lm~SKY7gqR|;y{{~|0UF{|ZMpVr~CCnm+8 zaT1-&xXH~w-nCDdmg9{T?|2a`0!3%|n8^SkiMYQ&b6&`A8nN_wm<5gJlEK#yhNC1Z z{s|Hy1GafB%iGi?34T&IA5GK|`$3!ylxYDmag+$t&EiY|Q{K$Yu>px#rmY4-5h$h1 z&5#CmIldVDo3b=9=<$z6?_j;UiLPoeIIddy$+C0xP^e(g77F$YA?D7u8EbceSTM)+{UTXU{K23n|bbO zy~QBV=!(wKJwjE(E4vr$1V#~pz|;!0b_kZ)TRu6q!&e$8CO(D@Y(|Q@Wnfzm`hBd#AP=&m>8$!$!L$* z^08cwuA#GN%3=>uNqKoaUts}xp-}4sC5}9R4i)MbEkjwaVX^@!Mu3f3e9N|c0m`Y% z!UcP**>lJfpVYmB$El4MtI~@;S{`t-SgnNs4ASs17!audzLhSayKQ}wv}&vsxUGx`U2~9$3y=W8`8tin6;7HSgQSpL zPPbp0^c9D0z8B_w>^U~jrImA$yMBKCc|9Nu|Asr`f$DdOi2O!)L>qPlT9gcg8~k^r`{GafPl9w3^z`QXid zXv4v3!Xmr&B}x5R9Q?L%(p}1gMYHW$LoI&vItJgZkRKJb`S_e0^ec4^;N;$!cK1sA z_<$b3Au4{Lxm^~EEl;?id?&7|$IED!qdF-+C+n=zfjjxr9OhiqR%M2)EL~rkfg~l$_S1 zFUvE|;bq3;d-1m(fTc3$Kr|ZtVaOKf-qGZ((~S=(lcO zy&AFs;_(qf!sxA;Ge=+1TOcznTz*t+E6d$1^4N6++WDsdte^D~;a%e~_SC0JU6cr6 zR>_?Z96>}YYv}2^@=9SnY8(9MFTYkef4&A-J#3>^={;a%{f69*!4BpKh-}oiy7PCT zXE@3aH8#%iI89*WJdUVbPA2}J+wb-kdQyNe09L%DIFU#TrsWy9T4hxeB~B1$%L}6S zj0tjuFpivt;52?IC&`GRun|X7#el#wRN`|&_-(^vik(w?v)o!ap4)Tg4RW&Ys-9q~ zd}Z^6ZlV><5oX()$qx?duio){l|$d{Xu{c_;|&B5uasqy%qU>2x(4Ko=X=3m2mN2P z70Y-rgYxQlTL4L>&oyh}%>s9>!g#HuZIJd`(^`WVkT>yUmlrGdx%d**utM;o+0|TW zL#^m;&-Lmkm_UzDhP-19CQ?KXe504<_l$VoI|;!gqkMI%bryzSKK)j)ADZNRQSX7e z?#`nw7BnPLw^qRyxJi{JsFFJr2dGl{4(@&1(%IPR7c3x)M#V=umEPu*7#5wd!t^CYTUwx zl^nw_WkpHZ&pHcuL}er%{zh4=Te5Qqx}9MR0l&3++n{?Vy0&j75Ok+nfNXtwClie{ zjCOSI%$5q0q(++SCZpCKsXQ~Rqtq7ow*8LS0o>JTH4q#jW^cwaiDR8PKALsd5!SCF zWxI{DGaB(qR)yNS@+NPi6@%BL%WPf+Y{}`ItxsJKHrdTkWgHgld{UcwwZLo;a3c4r z&kAGEN@=}8{jMi&KY+Jz@;9j*cDt&Y)4s%w~ z1{v*l#wyp+QhPHHzQ=8qi>U-=yZqoScYgrdjR>D}-~S5#Qk z8I3k7D&#J|6)KmTT?*aopyy##+6i6+DFq=ceO9Bu#y{dvB_~zr{;Gu0J|VwLI`Y5zO_>-F?f?Gp#g0QVVD&{CN!!Z&=+1G0woKuP5FP<7oq!if)SVKNP!7t zcSt;_Qh;3Tn}3W!G>RWg37;VvZAbr}ZOnhQC*15waeMo!a(?$jUBs>bUoDXVL5Kyu z*HH}zGeleD=$O(y{YI345dqb*d7~iJFOVWy>uN$s4Mf;nL|s)6%USamcA@NLN$ndHD9EIhd{2P}6#0VJ#KN*B5{5tzn{p;+1Ned>w zY~R0q6Nfh_ErpVgh?sDUMhevbL)^P`h?2qldi|dqG{CJVDjIGom`Gu>B|!xT4WI?NIr3o%m?J0p~{( zRs%Eus9_&<6Pl-LaiU+&Hkmn+PpQ_=UWz_V6toTs{1L%PHbrn%g60#y-rDKgb*f|Eys}n1iL?$ zcKu7>eAo0n2wq2P$wn0IDlI^>F4EX;>>XOXSEyjZA3ikfl@~3Yz{ht7-El-|8??cP z&ncF`qk}L!`F_B)YaOfUq1$Rr(F}&(VO`F&*M8sgtzn(;qf0<_*vLPIi8(V@_uh$- z0*V)BOM8%NBnco zYz$E%uI=|OBAV$!uwic%IE7z#R-_UGEb(R;Gxbupz9=owyyf-)u*vl!bv%fod0)h< zJd$Y~BwKnfF$nO&r7c3TT!-_x2Kk@BJKp@#SW_Xra~-T1Cyg=lqnVGJpWcl0QvzlL|T-`%kzMJ>D; zCy_*%jl=DMqUz3(nXJoz`M{<*kg0O1aKJg@#|yg{Sf)vSrpDAyz<56iV~|)NY_#)e zad|#ze;w~(KDQ(sWXe*jJ_QQVQIv$!yN z{jUHM!EEWmG#a0Wfrl7sR{O&MK*1U>`HZ@p?%|p&#=J3m1~l{CluD=Pvs$UzD-kmH zjA?f*2j^APs>5O!Mob7YLDy5p|or?%VM z7b6%}R6{p7o?`!kw8e+Gn++WgPkN|+eld|~EL>Ojt`>(m)Z98jKfDmg6H<)3$7lkC z^wZ^C!YOGdb=}{F3{lFjRC;`25$8HIjxC=p+PD7+EsgM z*Mmyp$*y^M1um&mq2K%O<|hXCz0JtGFL5Lv?{^q(i!D|3!4 zPU%_#+7mQOoBK;>Kp}>Dk>4}WGbnyHDtbo6v2>P4Z!t~ZojkC{o8V?cHRTUNFSOEH zX)PDcKsyN^iF~=?K0H46FFPMlgv4h#u<*>fu51?hwPn=O`yB}B*@j}atV1MXj*wRL z#A;6Wy;L(rO~*d)!@q29r0h~D6jESK3m+|3@Wk7sudyPVy^_W2Ah^o^bVjmV~i0U@*AVSueQXy)~0SBfM}?kGFlgbEDgVP^}4SQ&TtII`Pd}YA$`M z^RQ?x6^v8ng$7_AVn4ZCL9Qr_f7|@)Lvi=b@WDY8%u*Fx?d8WYlGkvd5gS$X@Ge2= zA5`B%HQO5ROONofjJcD8;0#a9-V2Gfj@Ds(YJB`7v&JiEx%T^ScLF_2Dj^S%wqV>! z(_X@}onE%1z3+IRb7cL2>k16SF0FgEij~ay)bC*;>jD7&ne^UqI&IBTWUR7-5LO>o zqCbLtvft|n#HQVChI zWow%jcyhkj7$GYj$XD+P0QRRswASGWc}@Jfz!VEK8X(G&h%Y|O|mpW zRNIksXiI)S`zKYp90ql%-ShJw#x}$w5ZlqqXZv6JFwv+!cEAjPi=($^BJ+yxP5@l` z3*&*jc?|TbbfS+wm`hK5@|)@G`IMW0r{hSlE9K2ZweXFOHm|CLDX8dj!U1IfQn{>*V5i6+r1>mlhb*xp_DqQaK#GiMGZHZ z$4Ng=mR3dAc%gn_MkWUn##|+8i);Z5K!nse^S0CQ^T>jG_WVfUG&<2f5rIkp=GX_W zmiL=XiI(*9QdCO^mwV5Te(vzf*q7>CoLL)QSlFie?L9Zu@^P9>H4wfJH|`j{o(jKA z@F3-Nl}gdyn6)_&G-Y61ZRODe=}K^Gi=uLgOFY@m87NTpSr7*~=C&Q)&ZJCAsyX0# zqGI1kl*6E}If<>vmJ&{fc?1}wQuD09EG3{pi`ZbBe7a86jpzC@6Zfn1Qg{GR&SmRV zJncwez`IK-5LBd7VFHX>`J5KeM@kGLfrN)jB^$Of-PnjLBo##zEaG#PzrP_2h%|DY zj*B4U5G||+XE6o%=?jBET`34?ksf7q1Rb_D+2#xTe%3A4KX#tKK1%)AA1(B9 z_-~UF2~8x)*fHU|hh?cj`dJ6XZ6`ps&IUNrC6F%?^t!;MLeGsLxqmj=bMM-wy z@SzV>07fA{JUWnd$NSg#KG}THJuMOVK@6|1rRMnXA1RQ5kU%8cp)dmrCF;;8`GX~j z&g5v%^W(WE+#Ctt4`QhAQ^;0%uhjyX8e?PbaZ%v3UZQl2#C3iz?Hy0`2JMwE$0f-t zgr^#0U(L<6tgV6Hj*M0vuD7Tk*FyGo-OFIm^d=jW=GIJgHpvYrO3EcP4WhuN1LkdE z*Ldcg1rbqfarc0iN0afkid&S@Zu_(!iBva7W8wb&>W7hahZ6;<9uS(Rqd%g^LdnNS zNpj={J|Gf0C7wUD?uE0vV}&L@5K2z1IUa7Y#U3c3SmNVfw{|YVnotp|~=tQ^()cp?RhK=liNzmYcsK}fi7}oz|un9&;FwJ76 zxz;$1AtpInyrEA_T5Z7n=m~k2G1nF#l9R4@eflL)Y_NUYL(()XOe{Ydnv*IM)CMe# zKY|_tHWO6}V?cwe*Ah*t=R=R!24t!1o(Gl--6{7aLhrT$u9`YgXD0Lo^Q7q^P?5SB zJ6Tc!2fo(cHaOm+06D0FOv(hBM5rnfaOZhP*9|-cg5@g@#DnES%;(PG5|2xY0zx41 zP>aI^S7NxA!&Q##$;sJaWQ49|*}=!R5~LIDaF2cDui}*E z@cY|D{d4r1gda&O9;lyav6xia^jJrR$fHJ9@1 zKt7udewEMToqnc$eQbYf%A{BMeS0?;Z+GGBdBQ4pn{uD_k(x$ga<(lY<1(BUbDQWJ zi|WA}MpenXa5tW{>V(A5>$`fTZ{&&99*XFe@a1YwMO`pr&peArknADq9(q1LT`pkX zB025;z>Bkli=1JaARmVi2$_s0QXurZ>UhSldMq+<;yO=T4 z=oPDsUR`#wgHma-fTKIMLu5D)m}0$xkI`9L&V(y9u$1? z_iQ@DKdNI&)70jjfZ0n+pY7cFX+1XMK}ya5wCH`R_Er;)vf|p=7q=;!4LS8N{U=r= z7r?q7fllKvWY;~MHc4tVDHaDxlo)FVRTURkeT;k?C5hmDoLwxzs-0}OXd33gWEt1S z^{Ghe*jsZ8i>IbbJzHMs?b2>bznmM+&wGA(^?EgKr)IzB)`dkBJ6_0c5@=sJI~Pj2 zythkPlm_C*>DWqoTAHbWjFWb*C3Dr0=j~Ya1m9D3-!Uz}m3%b#GcJK$Z}hI-yzYQc zlAHZr&vwpm{EKaLqJizJQ@q`?VL1hAJk7~tQS~R<>96%oZ46#8-*>pH*oSVXpEgzZ zE$b-g#f95ZZ$@VJYjijBFBc!aNy+Gw%-XCAP;crH$U&)dteckN>WkW5nG;i)%(!5JQ+tvKTZ|e zUQQ|5>PG8XM87R`5_*wibJ*;M?abKcb3u zAA3OjA$^qsVZMEwRg|_RtvXQY$a^!LM-NsjGM!gKv(;CpAzTa2vJoiy5u+ z>tM|4#OxaT6B29mE6s0!W0r?eXfB#eXPEqu`J9u-7;{CYqw%(;Fge3smEZhFyA!u2 zvWiT~&jWkPPn=IKa1@f*kH)&0EcJ4EubupZQsvY`R!<)Jm4?WtDnN+7b1sX|woO}4 zShyFx2zn@2>2H2pwl9`wutIZt4CJuES{hozR zH2=Xnh)Ck;x+hYSB%NMGaw2ruc2uJ>{$XrWaQw_U%WH1&=AlfIl5I+J`}ONMADG%8~QqWC;^r)$E%Y zgJsU;g4#W+pN8u)7*PxLazSk^KtmUYl=RZJILp{NaQ-3vGp%e+tC)-Z&28&ZDI1b0 z1HUKG_TXYi=4WOVjVz(jDQs56SMGKuW~(ok(6PF18_Cp30JT2X>tgHBV#hQ(B?#~= zl;o6!y)N>>1tAU9^M&dWmnlUBcQ2Ruh+NjU$XH$b@zzPyqZHC8ShCNZbA-D%s76Cu z4T!C*H0qAkf;w6u(B=_Wx)n(t%b<{A>wp|gUBv^IiQC^}vhcYU3@L>K?1W0=OsY1< zxqb4xC7~}c2#xW%zIm2b9eOo-JMhAO@*2EYeDD9#8sbFqKsxDIO12{YfL0m4bStM+1BT=}*wXgW^=78d=M#pli7r+wVs{+{apETeL&E zbSmY2Z($G2*z_ds3se>@SG&@J<(Ff;k9)Yx|-c6DA72n=1Jo61*`NB8N9?8vRvPOg% zKg!%sjw|VC)~auytU|@&q?O3qZO(;nVl|naKj2r+e+le)$o#n2VhAFStUb=>$ca}| zv>{wU@9SI>s)9?=u{b}+p)j$?$pp3~2as|H8Gie=ctH~-mTjrJ)DdaiI+(eCWF7XP zz5RnQzrE~yzNj&6y0CBlI@gT@5vCLAlNPOPY7fnPw?j=X4QB|UQT zliPJ=!j`1s9v9{^N{xMy8q<@YuA}0Dg<-ANkB@l7aof@PCMu0KJWi&> zByhi!+GcZNaXOlMI7$uD*}}d=pLtNQsJ%j~ZahPj5mnUtVwDYKY2&Jua2Xi|y*k7_ zaWz>mSHCxV?pD^M{4>gpv_hceoF6{WJ7o>|Upt6+j ztOPXXQo3Cr$8sYa^^m=zT-B^iMMf5zDtdjnTkVb<7K@BkWhP6b{y1V%^3 zww2{PwIS+Pqb^$Trp1s&7Uyqs5+}snrtY#3Omm_drO+FVu3f$iRCpin$by&*Gya}T zws(9L{K`d(K0!G6F~?>2DEc#eM3ltB{a}dDAbQBsqysG zwouOikEQ)B^`>k~sJ@ncy2yulNtlezfwr3-MtCH3U`X-$%Z)Zuoj^MV+jU)>_)JJV zxqXIkmPy9NU0d5y!3x=D`odq79(X*L9F(nx4TrN^GFELkC1&nAev&e7&_b&Z)s4On znXZmrEyPr^VAjV?^o-Il?i%4MceQsxWkJ3En+xF5r8V1$jx{eDq1MOV@LLvvj2jjd zh&m~U*n`uT=J16}Y~4IGjsK-$1!9|F$=b%Jy zWNrOKNt8{ge0qe~tjqc1ohA0Pb4(}ds%kmj%AER4B46hSG<$i_}s{) zVHOJb3(;`N+(U-md{FLrbOB?6$1gK1@FLUge~L^vR$$=b;F3U7?PJf;v%Ckd1^uJ| z1uKa^w%Y(QfbzIhe*c3#F$|&*7tAQQ&rv`=wWYq`l_>M-S(M#5p(@8F0j}tCD-y$_ z(C3jFUA?xX_HQM&WqhRDGv1Rz-NPIzu`Jg4JDDtN^LW|XtHNKnSc@_+I4=#_GMN*1 z#dg-qS?r!1VNEIo-;fBv{0XIjBAe22=BGJ#o&3t}3D6Xf7#t?Kb?E?3A>O)c@DFg&fx0=UbAG}`V7pifCsI3t| z>&v>-J3KDu_^U3qBIVHBvvz;5P^NFCi%QJ{LzFMv^q9X(VjcouzS6g#7({VhFNBEU z-&&Fa7c?h;pm{C7Y*J`^=W%%PV9P)xl(*DywlK>%mfc+74HeVK-7AE=>Y}gbZ$)5f z)31%hqm=rrb`fM2+T!q?Kbr29dhD;!;^xRiez>HkhDLCqgHG~Cjr3(7p6KnL#K|_w zpFAR9FrOMUq}vNuwSdu1JBA$_VicAgS>%1ubPF?5E>cjsuy2A1HTaMQ13%{erv?Xa z9!-oGb*&gO-qEh2UuKHbZ}8nktnML`xL9}=TdL{2ks^xV&Xx`>(-fuDdhe+TH~{} z;C(LSfo+`Jb_D*_>Lc2& z9$0EG=fZc&UO|~ZFo1EU>vn(g_yXhPakPw_?`581UM;df6Fh#Fj1sj{ zW@75{nR(@`*TCEkSHPqtIiMx| zuQ$rVY($Ea{>P7&`-&d`X?JjGNYAIe-B;)CMH{i`Q6!7jS^A?{HO{kl=!O~n$*T-n zF|Vb0CSL7N@-OmMo~-+YE#hG}YW;|omwfzAb-S$Qq;XSwvXn)a_3k~&mRkUU&=@AQ zHSGAk4Cmgb>~0UZa)-OIa zx`+oZ6o80Y^(#7~Wd~1MFWHpzrEDPtX9x8WQ6enh%r>sjgmmpl^s?WKJzgtVRWKi5$51)qkG@63HSy~C75WN`c|SgUen2}+on zN$K;xBVBs^4pTaj!t%z~_eg@aRn>k?H2S|9Nx6N`C1TzVSKiqhwGG}dZ~XQyMT?`X zjsC>=+_f-Z`(bHjV-o$boy352kBwf`#Iho_MAO{Eu?F&Zi>@gAhT>Q*AI*Eu24t~c zg~j_ljTXCK1@d^7X%ywZ0Wx` zR+LcUpips@kd3|gfRxLq6fVYoX&ce<5kCvEeVITsxw~D0EuB_p?YJdMO4|3n%tS9wyAiD8lJ@k>aJ;~LHftkk9C zX;iNpRuA9+paJ5M!>xEU4bV9P+LK)X-#j?@R9xarmij^;B?ftEG+-m1d|#Z%?y{xe zXJ?iHvvts%7ca`fR_;dJxC5bL>j%vkyZ!Vl7!tWUzZKekX|1x|F|-U=FVL#t*SWS zyT^dRZF7B8+h4ys7cScAM*5AM_4gS`2yg7sTtlUQCo2=eN;r1IqN)kIMa?y{iAUZX zkq2D{cKwf{WBBnGhn5zJ(p0<_GKkW4wO*fj><&F~-$l2>iqc}sF!g+4V#)qIO>s$~ z^}0vF;5xhiAW9~l^-&4wL<}|4Qo|z-zpIn1L~xn#b6jqMFu40jpVpE@ms72kOvi{@ zXC%g@e+VfucTGIgRx6`rjCB2@?KcXpH%D%NJhri`b7LktdaGWs{@T{{48)0nz2B#eytg zTX@7*TQb4tK)J7PQhuqbBygSnZg$UhBuA~J^kQ;4Avt&}LR|MyK z2ckhVMO~ACCky4+bi%C^*1An(Qj_zBM4L~?-3U@9US-eHU&0n|{=HzCHn4qmu|i`* zCSK$b<9T#w&TdCCU8KOMM3M=H1j?|6Z`DAy|p>n(| zE$)LpBaZVb@bx}J25yR9&1v=Hv|#JAQfdAk44fi}>LAXS3|!9;FE5p8Qg?d`Cu&Q z+yZ2Oy2*&a-!}@FR*=sZ-z&fx2-|@FZ8z-wi$8QDDqtEDg&wpXlFMLOgCI}~XYvFn zsutv#LcSH!f}|#>N2%kE7?Ih|MO?U4v>YG#@eHLL$}C~!FSk1IJyyc z_x%0po*grTe?Iji^s8yQ&o4kQ+kz9!CdrI$tQuzSAn;Co6k5vNx}!Am$43~>L?VN$ z%RPRp%M;-0^2cA&=oS7@5EL zG2k*v9dAwUE_D-Yrp{pN3Vz-Pykd{gR^G>X(Iy;J5woV+K7|de+{M0 ze*Xdvyeujl0w}fH#?O_Z0zA~aBrD3(Y=(9I)z!SrF?dkZ&IATV9+|O^#z??xBouC$ zJ5-={_mg!g@Hy}b0v`nP1zUY%zHM2;Jq8h^!jEp9-#9H;px5k%EOPL=T6mO$Sim1Q zU$dAfiS0?WzyGb4$otsWSLbX~e%C$;%sRNBFZZwjXiwb^+HWg=veGCFL(zi}@jESi z1xApEKP~&SjX}rkmBG9ijJr-RD?!!()>^DrUAcX-I}a>;3$0!Zfrgow7Z(??3~F?r zzPqIUe9B-cLk`;R1~fK`ZcWyPf(F{pQrx$vhObzzlXyVK>M+okcV@Uy>tyByb|p3g zs7Md{Mbs2%%#pB$5ElMK4~a2iCA=m zQgdmR=!XNx)=bd%6?hn@8$cymVgND5_;77FU%M@(-U#Da2* zNhCTIFoO*&EiH}Ua^Ly|ji-_2O9rci>H$K`^CQRk{j6F8#eZK20qnm%*3o(L58|zZ zOA&^TSaFL4b!Wnl{(?~K&aE;8p}r4T(}qg zJRS5qibkYpdeCs~V{uyjb7DhuV!`_K9PO4L5?EGvp&GX|ZxLEMHbf!Zx!T^gy#6$K zSIqU+6Idcb^FFw4!z>XISmf{Xpm||5?`ORbCaeOa+U74IAvbACcI0mk&A8C!RpOlB z+hqGmfk>e9+E)g|cp@ql9N_v~DWEt~61#$Nsu+1GiX#$#Y1faDX>~wFD+f$;1BfgDtmDZv+eYi1CAtj_ON>b7zrOG5?1Dz;~ z@);xT7;SGjOyN!3Nd#Qb$$btWj}sY#Z6~VunJwQKiFoYX0``*w z+$`zBUQcP|Wr!ltaj0lo!tv0;acRW?`>H!IsE@KXykAh#K>l)Yx@Cw3Akr;`E&3ea=B|GnIOd`FSX1GPnp zWil5P<+69c(MY@2tvLNv(W&Ffj&T&TF+A$i2T>p4bOuOY{H^B(exa!FU6kO@4EGCN zWTh{k2n|E#%z7>BJz8SGVf6LZqymhJi6&wm*cmYj6=|1=-@nI=FDnOuKw*fdhS^MT z?hB9Y_)5$5ug}gMujsMCN&HSc+ zw@ckR12f9K!eM^!zPNa}S;pnO03C$?jBQn3tZR81&KMVlWo*JQqrK%sg&M}L4>H#n*_}SI{RXa?aJ6c{WOQ*smYV8 z+)QKd-_|zhG&teNShp=+lit(*6Bx^U4TDcxKhp@0%0G!$7$ayUJG067j~$qi zIz0Wg23#xJj_>@%_s71+wZipJj2`Agy{agTP|BM3sr>`_>oqZosmv8JTp`k}` z;jW$KO9py7{~p#GCdUx`V8a;38u+FxJ*5izrw*kr2K)gysMIqZ71*HGC=eAG;&oc# z4*wj7X#XY@tj`6`SXj&Yo#TIA+y=LsedAkJ3EG?fne|wIG;|_5Q&I>|O~`-v+K8Lk z|7Av-%_=i8$UBo*h&N2K!1kzt5Ax>cEOKhQPmAtX&yr zgf)0dzx|aEZ+kd&-k$sy%1eiV5$@IlAGeeLF)EP1}oB9E!{riU%p3S0t(>!z=C5nA^z6t&hGbUhZ z=-)}5gkM-UKQImWV%dKF&(aQnV^Kb$f9)<>u)AuIo!oq zhfn^lu+l$Y4F^D9RB5EYmoe~Y1G+V=*G6~M=mF{sia~>K9sfvxg&jB?+@${=Iu5)? zDSN{*!@nmIxJz07fux{yDZDI!`@?uGC{XIn&aTnTn&SW)iWRkI>(9`$;ajZjSO|VO z{AE8ScB6gy#~O=f1{Xdff&B7m0FmQ~klj2|Ryr_stzXAT{~o%SIy^sclAia^LIfuf zLSP7J%;|sbk=2gimy(FKbco=I1{?+p|IBWzf75N)*9s;7v}M-Pa6@#<*@b;|!gOq` zEri-7mA;RWVfgUDeiei7uyWTI_n^&F-5TgM^DMyu9fSA&%FYlt^a?mj5eBBL3ca-IuFS4_Vg%AqE?Z10Ejgn z3~CE%wt`5i6!go;(*(BGyW5S|L~|Dr_t~OBjOt1Aga{-Oa6G{?7VquRkOuNJ{AWiu zO3_)6CzA3|HY28)*+u>)*XMC|z%bVmKCekeWf^Al5ws^T+H|NkqBe>ANr;0pqp7%& z3LexUd@H1Xt`G))DT#3XW&sPr7cjlgkpdz_DpU89m3D*oXT{|C*2uiL+h8~%kCT|6 zMc)Y{6BZy~(G3FKIlckgU9y%6^RAEzi(V^a7zpMa7djw!K(iBJuOoHJc|bvjcSB*+ zDtW0^2Bt7+|%*T1|l{jx%2o>O2oI$h~(E04hot1Y_n>H3J~2GXvgz z-s)z(iT;Q678m{Xi|e{_i9Y`gc9HsDvC9)=gj0R865x5h{xLI-$5vNC;RYOn-~`VJ zLMh<77KDM%m>*W=va0HRy7(Hzny5tFRzcr_F-^gn81IJlQDp~II&O}0f6DVZ-V*2s zta1>^F?xi>QqFw`(B!}a!dq_kJv;9H)zuYaEb{SbgU>nF8L+Lvf)K!CRXBkdM8}MM z9iL?qzSC!TXN$MYHiv3hh7c zZsl|Y2>`$h!uW3i=t!!tgP(QJ9~stp%zgxpF@#iNGLTs@x>el%zAOo}sGgmpqqe*Z zUknJrf;|JT-7m3-dkkE-?_}brWL~qTNx8|ybznmD7S2I<75J!UyZ{oqH3OE z6e0>nei-28Q)P=eWTb3h*kNJ4iZy-A|A5mB-EHgFvnsWgJK@=}VR!h^$`r0_t<*fi z2~Bb7VX-)I)O@wH5fW!*3$%W{DZJR=9pKP;AWEw%i#V$Yc2NyA)-fuefe5Pd++20P zQ=aC7@ce3gHYEco9Q|;L1z|!SEeJAjQr=hj_5do8En+oNRM6fYfiFk>fM#3*8B47N z3c$JJ<(X^Y)gq-K6;Cp+eYAKm#w^%IRpkofxPr`)Ij_UmIsSr7O4Rf;hZ3raYeNO5 z`_9tTTwh#P1M%n;>*i^6yxuu4_wYF{y*wq!c>*XvN^$CV;Ty60Sl{gi`Q;KK?hQ`k zKK}-~j)UqK)9soRa51@ief7NtRv3>=u;2$T`(60uv5Ue&8Q?$#nyUeD%PCDJbM_)f zKqjd9>~P(2rH|74pkG>mVYDgT z7!9MufD}zD4RD5iN(rSRqY;p>i4%yQGcFYR3om^G=hlyYdR5+)Wl&UU4#8>$Joch- z&O;XQDC$6|Y*@I>xB`?q6_qQ6GC<}goJf=X-|t+Q0Ag+eI)ojrY5}ct#77!-c6Xb0 z=i6No*{DL9wHemMIsy};Suq%&G4a|@4ao$THeTtX;65AV6qkTuKm_Ei;Y?qx(YBBJ z%tL0O=x7GDZo|P+=2%#=`>_vk!;NBB&>+$PG?iew&?%+s-{(%`iUa8l%AdWo-K{E6 z2?9C%U5oL`XO_QeIJ|v)G!7t5Eh^A=Agj`obiaQkONfXRlrH7*il~C{y z=FgM=i46Zj4KOFfa>9IcVmKA$>_KY;z8XZrjSNC*{M8@fi^F?nc%y{!mG{NA3XA(2 zV-ug}5dlxnfGMh$M3WWMtoY34_qFc<@UQY~)~%Iz4CEk{rHFYdtFgR>z(Yz zvD*1QSAbqn?gJLnH-Q~NI89&WDxmq5?g*}=m-6hv={k}d9jOavAZdo?}GW=yuwC|*m++*HCL4z5Ue zP!&jv6GEI+Oq@f_t5wz-SA7qFxSBr&j4)i;}2f|*HNy+P4 zk~Nzur<3w`Z1(jHh#m+B*^)H>1yhl6cWyzO&5EeMp^pCd@Q}gs&C8?xeF?DaUKlVZ z{I@7^1_~j9c4I37Yv%tWLJW|hvwWl2uKM48z# z*1tn-Y(@So;(xvD)y=M`hU3`(+|@lz#9b$yuU)yN&f(C7z@}k zFSmi_;BPqoKbQsa{~2caFa7`kZNkkvEz^xCs5l{;E6%ATVd$aJ|5V-=8flU79Z3U+ZpvO!(E%;Q#ayn9=_A zXhNztXI_V*k&z*}u;h~`Pa=roevTb&xpUq{L_{Q^n_0v}J_*sE{3}N>m0Lv+vMh14 zPeGN|^-lK8;}h0-hhNcUz_Mnhu5i(nQY9TS=z zV%i~8o>YHM3epy}iGd;KK4O?RK+F-WEE0>>WQf$)DI~)n*B>D_Q$m3(kEavfmzi&j zlLzyE0EPz2wYR>!QUEXjm$2{#2I#?I0F$rVRTBI=Io%AZL-DHKg-JI$wb3>L+c$xy&G26v;UGfk%kOdDnS0~F(+q!`6d%Zx zIscbT8Tcq-Pd5@7FiJ9=o9we0J{+GYL*Ez7T`0u}$je;426=4f>)+_}6 zwkQWg(vYX1s*XU0eRIgY25k{%ofbBB3=ogVb~U{tiB-k(cc7cTg3X^4wr*R7xXzFcr-d(HQUG6L{1H?c1vN4!AWP8?~3b z66{Za(4#{J9_+FJry;mf+xBm>+}w8)zp=5)!V)8D@UcRg-H|?GlIiUD?zOS(sC}R3 zDWuMG_fAD}QtM^ca-T`R$O(;t-7!t?WyOPS^Nxs?L1i4bLG_ zj#pb8ZP&Oi@i88fohz_-$2q`6?#kGZZ#Hzr<)CWfd=pa7`5u)w__`i=-yi z9;d!aJ2v31)LUQf$Wk2D>APZDc<>nQ_?lOq+kEcYA%Nd0_a^L%Pi-UtkuhiA?gg zLv?890*W`)D-tc_41BBYet`!Uec$?K!Fa(J47f5%keSJM%m$7(`jJ``67KO{MjP7| zKDKPUVy?@UbPpT(7X3-3QpY0aD&HvVkhFMNIq$AIhQ~gtVSjeh^%jdT?`d_h2@RBb zrrSDnOwhmSy@p+?P*68$9>o-MX@|{{P82xvYE*%R{-7dj zEeMSt{1d`GZXvH3c+7JBuhSV3jBv)1{Z_yX6<6|)vFK+{{6a!`S*c~eVDB6TY3#@* z|FGGFQ=I(#6ro~YiCg57Ni@!t+QJVbcaKl5!ZMigAG?zH+dNTk4au5x@a)0}%Zz2c zUM4!+>HVQHa`;%IIr@9!+)5`uRVjr;R{X=uX0|Z0!HHiP@;K6oJwa90ij;P9#|Qg| zjc(AFKg(3g4W^CG!wG~@L%fqnBvag)wpN=DY%ju8A9O>Aq&0u^`%c$+h9ff!tnt&x z#G!I#HF%npVhHjokwde&9;jTten6CH{2(C@gGtQNCknebMLWZ$;T+aBY1y_xjtA5I?%= zi;@`dE1k3K9J)Uo`dD9n?WHez7?YsRFADukMh)qr7rEL{zyG~rvF(%Tx#RlAZEyWz zN|Tm|7r0`c`wLM3aV+2V34gitwcTF3n|#q*Y&e(i zgZ}C6VNl}7RynEw%_F-gz%OOEzDK^))McSG6&_o*_?kHHuIYY$y5-ff$fNm!Cw>vV zx-QZqxQT}q_pf4#Hy8#Nd4=305(MVxXDN?F>l{%y zOj_GsBjv5~aY$O$y=EXzy1ZcFBw$4e_w~G55$SXzjy^aQ#V*SLi$u{*KQP5tdUx&g z(XU$XyY`0l-?Su_DU3T^JbkTEU56ZrS)}ePl#UW*3|)-e(z@Tv0mtHlW6DE2B#o$_ zZVCo~MEvjE=vQTAoA;?5u-($0L+;C#a-q+DX!C`OQWF|^fc1qP`uZ{puh09Fo{?(z zMgC8(A}PxgTZ8v_-MP;0Uoj2ejAXu%vX^<-U@i<1s(QHgOEtHHXi;}Bx2sRK2+>fy{{500PgZnJDR+k!IduGx6?sN=9kX7n+ zcM)84U$ix?*jTzV_z<7#ifjTq>iUFu+x$lee!ZS}OI$>K+ZzG!en>;OC0 zd1c{fugSowcl?P!LC5~H{SrV3q|`jC(k!FatV3Sfe||QT-hFl4O&q$3=LemCt2e|W zecBP^4%mB^?9aFjMZ7}3f5OaeiydS9xRESE5|vvb{o|)&*+E{e!0J`Dgx}$I*Rqm| z#Sqo6dJj~*#?I2^x3yMCHW_^AF?I%Obev5Dj@ef^!;B`~mg=}DYt@o-l@DIvbHQf# zJ{xn~Wlb-E5COf@&xOOr;8zYdhlsZ4zzE|_e?GFRpFW>9l$x(GVVeVyTFzo%{+0Rn zV96<6-awY=n8{lvA|uh|8B<=#9*3ezj>_7CX4^C^gm@k5>>Ytq`S$;ZD4!cCAw@e+cBbjK!X3z60 zChH?}#HCYdk~{Nty5{+Cl=3nUd)s(zG$U(d=af;Zj8Ez-NG0gzIc@jvq@NadNw;nlmPe2gB~vd8}b?*X$kuf%k%~ z6W})&A^)2^y8b=o;x~D;h>uj46Uo=FRI1x}-{G|31$)jX6=v>k_h7*(u)gkgKx{@hu0efE9zwKyY&R@hQxGk0 z7K$$yRpPZ}vOJW?gFCdJ;(H+!;SOr@>KyPBYx z{r$JU+M_ejX(x^~u4O%C0r9t}O-UJQDm5R!Qg<0L64w!mvT2>Yx%SE-q58ntcze22 zy7N?6@8i{IkANaGN|a~=ODLc{4+`Mk-jfS}t<ztKuY!-uRPYi#&FaM2UZ>cHr z5#15{ssojfPr5zdQy}4BcPy1wskL${i2J_Bh*rHLN1aa7y(ev81ugtu$CJE7DOPO^ za?aCQCyyANlYjAMi&unv9JJ`|NO{tmd%ib9*S=4z#-2f-^C@pr;%E$>+IG=cws?uY zZ`izYA4R>?r(2HSq9}ks>09EX%XZEMEE~68>h$q794TE0`}&u(8#DHji;8u%qT$f1 zsLDLNP}#f5Y}Ip^u~BRU{o4 zLX}3wnl+Amony98q}={tQ*c=cjR_z({3PGcaC$kA*m_s*@m-x%)kT<+du#SsT_EFg zG2ch|q_5VjG9{p>NGg@sfcJ@9&WjlZ%lLTm1Kv|TU#2|~;N1US1;47To&`F;7AKr9 zVDzDicE07H`sMo1T|6tAxCB7$r$;Ln3cK>O!zfW%-{X9c zn|!OSaYUMjsaLWwh#Ic?-VYc$_0n=R8eLYoU2;}ym@Bx(qQ9_0%C(a5k+}@}!?Lcj z#pPlp`x%g?<~x`Gd5hlJa=BBk^AtM&ZK~K)M0KI~q|JD8XERV3{;LMtWK|Wo4%Rk_ zEBZzU%TT_|F`5{Lqj#yNsJ?hh`V@%dy&j0d;4)az7&C)Va138n_sAsizdqfs5F9ZE zEd(tEEJ8s_a9l}bn43RGCZaEr{^}&(NtpSv;DIj7^V*P*kGi}yuSyDx87R=7=+zOp z+XfAN#!S9!bkue?Xrx(TePmBf6?hy8TfH#cl}K3UkyrCJqnxnhFws42Q-OkqWD)~2 zA0j(lo^YQ21R)M}A9XZgu>4`AiJDezprx{X7`aTS|L}Ihw37}smrazH1xl2ohRDDH zl@4}r>mjl@-sG$&)7U$B`u@TZo)-FDko>D^_W}W(kIAy*Tg-34sLOpcS^C%^Un(Z4`RU8hHmj$W)JSpPCkWjgSjA~Ax8zfe>1`Jl z)VItjs)}aiLIs)#2F}oI(S^+mAZ<~pTE}5t9U=T&IHHAGZnQA;MN;McwON!I2R?+w z<`8JF9TIxO5B-eWbDAvqY>^NR8%MaQK485G1Q(YzSJY3&D_jKV2r9!!Zk}3T$aPvu zzUZ?=#UFXY*ZPAG2E6qN=No^#00C$rJI&lc>*WMcARbb=DUe2h2x346U#~9cxIm#x z+3(w9A+HPN(dF2Kq#0x^Jzq!(E|!bQ)FxAgexzcLY-o1jM z#|1;pQ-N1`dAj6b(<2vRO^BPD0&(^;2Fkx%1Qr1W_z5^bYqsa%)Ql^W|7l}`5|gr% zm9ShUJLmpA`w2aIwRlt<;ivLBKN88u9&7fa@Dv`T6s=<9dY-OVjaUXZcUi1QQv{lWTgxg;j5xCAoB*y7yS3$L&Yp~S5Os%Nz1eq%DD z$dGf`N^E#(7!fDsp=`3yPXk3m5uXp@@?#|uaXj`a_`I6fE4O6h;x>&%Kd578RG)oc zQ(q9WIb;)JQNvsilRgQ1LbEs5(fRpHN!W&y>PbrU$c=iEily`FgxI8Q*AaAfji?n1h3f<>4Ciq^{dqf4;|1cR*f(C78u zYo)c{^hF*bs_N#jDeIrJzju1`_4~KNstYPt8dAyoGbGjel2Mth17te+LrFl>W)I<_ zF^6qn!>=(copoBEx$Nxd!pt86hEod+*-K-4g%yDzMeXHBueTF{zCB(_J$HY`JzG7s zfp6zlC|qHI{)&o8MTh|mG$S|)ig;cXv2}0geE^0KbKGkX*7WWc*Po|VG#uPyF-81t>dTNB zk{os+a)HsfT`N$MOINalz*opmx_^?G91rX4*IX)=aX<1B2p#{heC`4H_>vI4l&ASq zn*Xh$l|fxKQ4#AY7X6f};Wy|+wA!tayIxu4x9`U;i(W4Dgi>zULlcCg(sgH*eCIrb` z$A4UQm2c$iTMNl5(i_)91g3IRsMBG_Av~Jl8j|nEVerH+HBXg+&EgUgp>I4c; z3cKu_tPNT-sHwvB(bInD#+X1Z8!PG0a5}#W(hpalT>f*v-LlU26j3ty;+C9Z7Khcd z7`iVgw&89*KSCQl_6Pw^_*T8|PN|rEj6RBx{8YQp>VOW$Xsmad%5+vW zHa~1RX6v0z^{h+xNf9>3Z4Nek^@qR-pW0{+1b&Mz?-9r?}~MDo4$#`#qrEX|C^D z-dP%EGPt8kx2{Qx?8UM!8x-`LczbdP-^l!mLj z!9-PBsD3f)p?K@}t(^;;d;^_?r9QBmf?4D^)dKkg=_C`$5OG?Sv(P78oUk)JApR>M z8AICU6M48lk2C4%vKw;d9#AIOzS%ER6=?*4k0NMjbi8D`cR<_Ip$Wg>K8K1 zaX<6uz8bG4?8388FRs(p!yeF+0o4^^2F><-%o2W|UH3C#w{HKtHP4p_i4-O>{n$3N zF+Ue_ecaQNFixoCr{fltC;iIg$M z50Yy#ToKpxdNOZ?*gp)^#m! z9F>=$`Wm;?C0~s)Zl!@MqKND(>xtY(E$o%d#=_%AK4AilY|y}a&q%H=%d^knXM$T} ztI8;#kGCBQ{{sVm^L|h%j86UOC*&0iwC)|Yu}ZXkd$K}fJ-Cz6XSGTR@&>`(?>w!> z1Mkn zR%`K6?1gOM?%voV`4R&h?vNj#b*;Msd9;+;DD~5g8tpSRH3i9&Y>8_l5QKmJ&pJ=n z7kTfR-Wd&)jugOMf*}wX%1}$6>Yki**%u*W zJwQ6O;?^>r885rkKJg*N7jM(JM32g~9lCv7d6r>K42UxiA7!hfR_c4>0MO1ekM$jG zZhcQQYQTjEV$xEI?-g}B5qtD%KsLBiDaE~Yza6`5pqiWUG`hK|G54)xCh$gCtttUg zWUE4>>6?)mXYlAXQLyxutshHpJ}?IrWB>A2ot7dC?Cp2(RL0KVsSJ>)wZR}V>^C*~ zt=~n`hgb$yfSZF)?uXAIAlJv#zY_)(x`cb{@NBt11YU^>yq{f%1^t6h)mY=#c7Nm3 zIXd~k`%0Iyq`c|A)nEJsqaq&u%w+jj`aoSjc5m%;&`1;;Lx>Al0uc`pm5HT{gHm$? zR_TC!{D{mC#lN`#u7d~&P2+l%I`<1b0l$8s`!cD&p~m%ya7{@+3M7oEbdvPJpHCyB zNBrL~{q@$b%s{1z=e2?OT$(3U;sXadDmk+B#eN`#dGW{@8Hfm9a3nfqF7XrI$f|n~z3s`I0pQpF6Z$qLU=U@)E4N z3tKGscgd!|gzOIQXI_q1Fv8wa!L_pY@T?XjfUdH;0H4Y*VT}tXH0cLyskf^z?9d`l zKqi@z$w{jq=I3o(YcnMvJ0(2|2x9>8BeW@GGnP*wcg8RD?wK;x6Ek`5{<20+j`u;@ z(ZzA^A7yWC%l&u>EH||BXvGDlT~4PWcya-L`8mfNDD}lV<86D_NSV)Y#^M(&aFf-& z7QRvg#PoT5V+qSb1zgx zR-B8di+Kv}B)1FLPrf>|tyIDH44AlCv5ksMK1{3y*n)X?n?6qxyNc^%EpMOk{2EzK zvJnT~1H$EK8ffyvPZzoy7N>KoGJe(C4jOh{P3znn5+r~tRy*Cvv*OVACx*q228QEm zbtRZy_t>i8^LGt%`mx5{@pJnS>1KTio*;}JNOfa$a6JFg>8e3ys;Rc$oo09?zBMnS zwY~}J+dftGF!zXBG@uT+Qs|K&#JGs0MbFI12tBJusTU|5g zf@XL$-EZ&*iIbEmTh$Fk4y8{sq~==j{GxJWUEXz}xzE$c35tSDBmOGb8|(T59NgT; zrJ)f|oRdnUVtbB3m7{p0>>ihmW74$JNW8`mOz#MLyHtT7E2KP?zMX`~>{o8^!q^+) zLa^niz0qoOGiNCtYTgtSB>>=r7~6%B3i7}iUI9pGAs=E!OP^3nfglw8H@utfq71T2 zniZTYb(>!z2=+#v>|HcYJ3Y{oI{sF8zxV$l?Ja<++}gfzMO0EsKsM4PQc8ETX%Li@ zIP?Ytq#Gm!5h(#_wscE3s30NT(%sz+-`XhWJnwm*Z@&M`e}-X3x$k?eb**b%@mp6Y z4LioCOXmJY0MtYpUQ7VU-vaNBnV~Siw$?qE@hN4xLs4*D4`eRCfof07LV?|(fv&(= zUu_Pt)B^Vi|J7TU{NnAu6|Sv_HfVL)xdrm}SA0w;Rr1eH#uIiuq&_|-;lx=yRs!2?N3h6y)Z%TUx9F<>!reC1!hL6-mPuWo+MLItbMW>W_qnZRk zHHh%4Fiu4re0OS5gT86kr8RIJww3ib^j4a4Z2N;G{AeFRoWQkYL?2ZLPJPN z{zMsuGQH#+tFlLT+^p;1<2p}Z)L@P0wMW~TQU(cCpcv~kh`_@Rn*paRfPe`S#i;SQ zhgjKUwkh~%CaB!XU*5CLu`5xauRblm1W!X2G!mnA0kxIh)FedtJ#myL_FnMZ=v$Y( z!Vb)KqI5pv6h-I3tx`aB?+<1^>C5+sz)jA#K>MHHSQf%^$X$ zU`5d^K_DN*VZVyX4NpB(Ek^O!6>=}p`Dihr4W`5|*gt7rvVj6eKyKFGPNu`wJ{PUmy&?5) zX56=Jzu%$*;2wfVmykXVc2Qp4wq5HNz_zx<+o#J@R(b|n619<@xI%Q-4L+_uMIu6v z?4KOm(dp>%7$<5Bz`y%X!n~qkZNK=)E(k8>$xj2(!@B$7f&$~y zbN(~|1Oioc?ok|&x%JxAOyqMdhc{msqwx0E@$N1+);f(&wCI+XR%(LES1%7#m$@{$ zx?Ze-o%xh5uJcfXM~!giqqYL|_itv=5pQk;t3NuOwc@2HEp2P7Ky zzLoU2)A1fEL{4Y7lZ6k+;_o@%#d;<5n?CsHTewh+5)OWcrwD$uBO9g{AGz;JqdB*$ zDx?u8Toln7z9Db77!6RumG5N$2Yx0ZT~};h2L~+$4O>3zzuSwDgmJLHggzhJMJxk~> z>FJbsM8&?J(R3kVkwGY3`qD#bBfUTfu>71pT!r7MA%^(V8xsWDJ-7udFlOy4Uyy$# zLdguGvHi`5@~0X03bMo~g1ln~yIbP8SU!*nyo+Yh z{R}clf0dZx0V?|Cs1;-g z8_bPKPyYB=MP?KaV0@Z33+;g*#Fawq&T1Ze!jnm=>=D2$b$FB zN2(`zR%Kaq$-itgF%%>d zOV$gy9^|5tQThh_Ds))a93K{UVAH5|DfNh=ktK-dwib1Kk@XCYR8w19>H!o)Pr21( z0KY_enS~W&#Q?X%Rg}&hx~VnlUsD-)SslfW^|pi2e*!0@K=j1_2%JFm3Jt#G)2X2g+IU~;fu7!HF+~yxR4afR(3J862q}F#+$8%6 zJhb6Kr_~Zn@cNCrxzD9vOaf$s-B(QHj3H+3!CoLK|Dup2N=`$ z198ajNKV8@dyx&jHbGLP!FZMZJ#=EgtwM(XD+J?r9M;wE4Al2zLZsl~29SYi$Hgk? ziwXK)88{r@1bHkU;z_RROFRJX4azNkQklPVY6eibsa^q}=K=?exVpGHtC2R$OiKUg}0|MK?RQ2jcZ``;O}9e1jTFL)^tR$)YZ>tUawHZ44TMD z{^Owi4i7_c)$bqS3Op2rUo9YI3z2s=OjvG#>Bz-BPJSCRBW(M|6=XX#Jwi=7c8qo@HM`;JdX{7>*51lc=1%9q;kQvhQ7K!E1|ef#2RLmw$oQY*wi_<%wL zgk95W*)0&Bg(<*?n1a_cgz1e2XIZ>R;7l`AwlG{9Y(d2dG<^$GNq?WlGm`Jtsl-8mzSkQX&m~0#c_4!V(~I zyhZJON|K2utmr>S%{rC+KBeGyBJ?Xsmmu6l2OZ>6XB<;!)&kOv`;>sv0wBE&r2&)S z1(y^WM9JK3`_=C=1~{%Ak*f2lV$wR3>CjbARFIB}{p9eb$Hux79qF@T3Bl&~V80&F z4PKou1H|1HhEO-!-`gssNDh+0Bqf5+w?qZG>0#omiUFSh`NMf4Gp@3NKn~lTyGYk; z`*+5gM+OLx{QD5k0JRz5)U@_@qop^|-CwCF!Ns%nqGzvYLURrIn!ntwxmoj>N0sY-DG)0uGx%s1zc-gZ0&zjGN1XRJ z>uqr(LBTz90cpUhxrfLeH)_~kU%He^C09Q4s-NpA;p4svA~f>xAp;x%2c(BP6ORJ#LqT)@(b0?>Ryq{!4Xsk4#af@v z+fjm_P#IqJqx$O^V1JC22eA@JdUQ|RPwdz&$JB{IHWw*KYYJpH?e(qI<0^T*H6_xY ztAS_I8UHN+hwAm-l9F5^KTmtqM+UWf#9WsB4m$N7!5|YTZgdP(o2WZKc)1>N761O- z=Zz9FCQo#+LB05l8%J2`vs+{1d{|tA7w=wamr=wzYQ@Km+?WXLBexI!#@$b!^^Vlk z>@54UpHZbJj6R=n!wai=L%c+Ndu!7-_VF|7)6W757%guN(|E~Er=b_o0|ql1l_|*po~H7y+tpu&}W34dryXKu|xzTnwaYG`eoqzJ>LC=hEWNM8W3F zu^=M>5q?G?lm}DAMzEa6#VN3FTt6^k^GwJ-Bg){ ziNn>=ETvOcoOsq?U~X+>&1HCbeB++I3s%f%{u{;g@*!xOjR*93qe2)a{ZIRW5VD6F zfyAr+pH5q8b-+}hup~)`$E9zBt^fq%XULcu@ z%JpCk0=Hi5ByrpA;K~GM!^Op^)mXJ77VtA0wx%0^j(v+uBl8z1nJ{t;tgN0|SH~Ks zZKN1hqwgU9=>;_r=c@-F9`bmD;vC7!X)>^h8XiRU7_jzCKl+?8HA{Pf>IBMqGl(cHFavK&>MkxbWH$!@|Fl9n4a{!l;}&B2Ng)L7>YsR0#HC-3@B?3Q8A|lU?Q#K zjmCkD9B%8GzXq~ZTqZ!lf#zm1&-XxqO;>N$HGujeIKZ?jF(0~@r&-$T(Rgoq2W05a z?98`qZ5Q4fy%*Q)%nqaj9rly#o4yD=&L5@^v*D+)$t?vXC^=&FV}yRZVW~ z{Zul9u{LDaCY!^27h^#`N&e{}k*g4Xa(0C2;tvU-xY>YL;eJ|{9?kE1AZNjm7BAK4 zCD$&|{z$XrQQgIfIX<1?XJ#c5kRjNx+?Qc*E`lRcW-*%nxe+9>6&f_(JKG=9eIVe% z4h;N^2nQUhkur-Qgk!OoT%}ec6ok?eZXeY z*$669gb8_iFlrQucX5FdFppCMg1<)tQ9{0g12%vH2V$-4>3e_P<3$EPcNIh}`r`eX z(E6of_pe~}R@VC2FX(Lb67l_&p}8+4b2fD*>b+BhOrQrz7jJ*j7tj8vZQ~xH0%i~B zLv1zbW@uS16XJY6mzbZneX@KjM_U0j+1@>U@M%cWYVc2O%S({={S6K(UW5xv0o0iv zz%;U5)u1v6q?95ILm4DHdV>-#nJpCF-r|l@&Syu3P@Vu`N*of_Lc*J(?*fp46d4Mv zlABsjE7huaEkbuhnJ?uGa!nJ3Jo7&X1>BjKn0V|jH79nmx6C2hh$E7s*Wf>5x!7qX zmJX^-1X+$(nNkOWYA*&8HO~9h)Y9R9U0$3+@bw=bfPf>~9zt9{Tq-CA=)xj)J9)U+ zgn=zE00%B!f&ci0RNQ)RT2P`INZlN`Gi@wm8u;;62%QAD;ibHpf(AJ}VEVex;bT|( z4))Ywb~4cZMSzq_sa1e?)45pNYYQV?!}SuP9)so~dlu>rYD@n!Zh?}yQ6}}2A73MX zj9VL=W6zPShYVdfTDNm$}ctcpRzDt>vf)qu4PPUc^2QA{TZ^CvjC?x0T;s`w+24m$B`Eot}NO!sR3=r zwu_@VH*G{k8rskW?w!@w?%oLbta-V`f=aShORoMEiJ)*O=4(mr{8qsWLq5HjE{-m@AK-0 z#dvZraUhdFyL;33PQxvvj&NZEKmTZFbuFh#@s5U5P4#NRdbu@tPxxnfU5XoCQ4%-cJw-F} zRxdGbwjpf}PhB1^M8vql=EC7Upg9=%-5M)@HnjNl*P|BYroX=E>7{ha!3#$%tNntZ zr~9jfQK{;NPFScWu7;yx4Mwh}0c@nq)oVrMWbyNQMEYUV7}pxV!3Cc#r^zyo63qyTj1f+)=P!>JfJoYv&-Yv-}7Id z?dpY~w_!xj3NX}Qq}9>T#aQuJ|J=Jn{0iGJ&`+Vz*FPHW6!Ah9A04y!fY|V?3C|Zg zCP-0o7wHF`us?J=rW)!CWgpAJM*@`Y!xM(F3mBMz`I6tbi6VRhN!h7EiK0xZVuPvm zvTfc;mS!a{>Rcbg7s^cKXN!1-I!u@i7jj|TKS^l|o<}T|{Dv1dt}sj-5C1>E47(%R zKsh@T#4-r%0I}F|-QAX(S3@9PnSqkii_VUxEDWOgG-r442TA&G9jaLlx~;A~2XUN` z+ux)p5YOpLk$)4VVq&d9P3FShi#-Mj{}k-{-0d*j>j}4N*x*)nOQr2yYzM*8&^;cv zd=Y`&N2c;aOfG=J3mGjqma@T0cx$?x`#lLC0UW164;t}jwb%hk-AC4Y8Yc$r7M7nD z)GeRePb}+4mYAi;!Jkh1D4kqf;6=uAh*=P-%V-}|iR3M6ir4eI$E3gaRLhF#kS#ObH8gVUvZSW@@5=Bgz%?lYdxTJ;qaLiUkRh2+z{*yB~`wV|K zjQD!V)|xi@dRyn`bf%Re1(_ulbnedUA7TxLgG;AqZ{_$xr8OAbxt*MuT*rdU>S2oK zXLAyg0mdG9}LmHbXP-nVI}d@aKPoh8Tb^d8!}Ep1bmqE_ZcGyh8( z_KS--qVw$&6no+$7U%ox>^k| zMYman^Q{)_(iL`3{K1_pyncfv$&=kYSC08Ot^xa`xQ0!^S{M~m$#G$}FAeO1!4bb?-J`*4|6z)hAq{x8OL7A*>cYfW^hg7epK4+&g1A80fS> zU9JI;{ASA={Wp)k5x$SC;F56?KHSbZcXIp8-b(sHm$~`nVB*k8hoK%1?W?ND&R?m|Y>x&FWu@lIJE@py!aja19jxe6>$hpH&n-iD z=$k8;M2QPB}PRES5gxg_J!u~5Oljrf}V)X3CX=>B*X#^zr<2{Q0NhoLRb#9 zQ48i2L+%P|v|HefnR?ZlW>KXbGnUxk(?y7p*J z+ROy=G70i-x>WKq73*}0rsTU6p<=R#9Iy2X#_p_-Y8p?$?ANNGI`QU-02971$n1|` zNCN~zn!4-`S`AuIi zo(z}*%pXaV3bN6U21ZF<21CRqKK+j`A+hkZ;xCCHH{ zWRVrtrWC%D{|bl1k5vbUBgqUwi~C78L{pC1vv97p0D(05yJNFfdm0&%PmA|!ak}10 zV}b6C%Q&g$VE5Z7y90<-eJn$X>{2SS%5$fBUtV*-j5zr_lcYNW1spTSPZVm1dkK7@ zVwu*M2vTLDu+R7pBokAMQe{g$XSrrp9H6?yAHI2*EeV^u5%?S-&>Q`M{^f)xUuh1+ zf3RDDWgj&Y!f`Gi^oSw`y99db3FKP(zY$hKA!mQ*U8xjF&JOVl3A$QU3|Ljm*Q|Tj ztBR7j0Y1<|#XdturHKNFH>Hb&f(gBKJrnFl2<5Nzgex1@7n-tV{S+~4N-(|qMBD?{ zThOfx2tGsR>gzw~lC1N=B;_e^r z5|*U_4D30F1KAG2Ve$p%WQzrudMP*mEEsI8A0WY_PsA#yqA5y86%W84gu}oUL^LkE# ztV!z?sj4YG>2aEOak9#C>~#?)N!p|O(n1BZsfpMscQMgbA&9vub0OB8`5ajJ(`T=m zmot*`f$+}!{F!oPihz|AfjyrgWBD&T@lKeW5Z+Vg?#fg?X5+G$;J7#LIB5!sa86GS z-;;fZC3N=8JipS|$FIqs*zy5#1F-rB3Uq~Px4xr`d{)oAKLZx*O=%>Yd<>S}C=`WY zz5oJ*RtH>n2kaIU=a*&m`W{#hL!?aB^9+R(Q0lexD-wE(1_3Aw>h`EQH&>2I@OGDv#|>v&qZ`A$fBF*joEeytbh)i@$fJLNdDoFuIGR^I&&r{rK22Y zQKr#CUAXqkFhX9my7$rp80{Re?&MJEojye&Kizhbj zg*G$8a~Suife$8UH-O{RXe5eIg8{a&(n1x|uj9I&_R_MF#?XO(zyG;NS#I6Q3S1B! zqr4FK>9N50&p{22P!!aNv#iPjz5*-eL+)n+E_*1VE#LiaAz{NNuGpXDANI$FyhjMu zZ^)b@Fsfi8HYh+w(2dmZsYAH|neaNxw)B>_!Dg9(0$9NW5R)YLG6nwa#%K9-$c_`v zCM`8m=$Iyk2pnxX{<#8=gvCe+>LRaLgE!Uqpo*#nO^Xjeu!c};C0~uEo zA$f%<{+7HzVm<(p1|s*B2vPtxTE+hxz~hLc)U=%EBlRv(Sw_{Zf8aM1y(83hD|{qM zRm{dU=~jF&?x>q!Kd7-Y0RhfZD=mBPkSi9Z$+I(1J4OVSdf2yB6mz@~$Fi2C#3n`s zSVpX2m;e+r@_1x3?MHZTm?DMZ$!YTuIm8e{}-zenTR`#3$P@ z2?Eq%Vl4hG;ZD(p0`qae8zqC9=6*C36T)@?<{APXlutZ3cP8L$Y-AL(Fp3La9L(6#5PQQyb5if+xq%p5kVow65#_)4^^b0mpaq(cb?eTHx81NC?95lL1<_5;+rbzD_y{e6(3#!8 zrMT69k>b~1`LFu4Lki}kT|HF!FWEnS>-~Uvr?6=^0VLc#n;>*pcmN2S;dQVSE zj6WI_810f^1=XSLgXI5AfE^tkM$g(yA_{v_@yAre4qyNZPm5~Tsv4mze{0)N7+fi? z1!#oKzn|g*PmNJg|HJ6yVWP7@{HlNgRSmu2ukI2-Ff6Z)fEk26GSqT5%KXpR)H1L+ z%KtvL4_KAqZ)0HurP1wqH$Y+kR>y#AG6Rj68uicvTSE4)(+gI`#!( z=hF^@fyN7=@Sdp_8-%%XMZc$1p>4%jqEsWv6zrep71t5_xkH6*3!jeT1&|J*awo!e zM?6cOh!C>xWsP(Gq23mY7cxq)oq;V5@f!Tw)ZdThgGa4{+w0dw|5iJ~lzz~vo+S5! z7$EhYuNuQqFz;|(&*FbA_!+SmvJtHRm4}7?E%>)5K>Ylk!|xAc01+7mzp`^;W4fgr z>7Ks=hS$OsFM_EyMgkd{A0`6EX1!8*!fg1E82uDb=)95WZ@a2796S}Jmo@kZxFi2q zaMVt%sCo!H>^FU`{0q;Lf0q303H+wa_3ne?J5Y|2s8=R%7Gm%rcd295goPchuhGyQ8K<#@taOoE7gp>_S-WC$}a z9GH3G4+Tc<{b6-sZ(uFOa;k#}@_E^1#Byg4tY81BoAUaLu{3~t`~30YJtc1RB#4ik zAuKFHJk4u=M!1+7n*qKZmMTjVIVfD~l{&8PeW@ueK%E!;(iIz0X+3l03wFE(X6JEZ zc;dAUiEWNlhzBW`$pF ze<O{R)(htycvD!Kicg(D(W?J#=i6I9|J#jZ;ew&;UkCy&64JyL z>`i1xl$5!o*{DnNYC1I@?n;k{_DYA}3+u1kaqRrH^ehHGz4oEwv8RW;QB(^rN^pSx zlL5pi0YqS(?MR}we^?V6mXEAY$8MR^UebcRVMIAggIUo2S?Y4bM!J@| zST!BL5)h?2kHRxCce=zxe~(G+PO(fYo%fjSscM*AP^vq_q|VGdOOmV^i|No$cYbvn zW%v8rBdQnqh$E3H0_>mAKLi5*AwdC7mC{F2!gAb$?nlg4W&lkh#})wd*n66M`JhDs5|JE9i@k4V;^M&r7njLhR$JxwqpeMgf~e&4l9XI zO?bT8UOP`IMunvwe}$Q7Ah4F8C%vfGl5a+l{Hf?HmUOesy^D)2yqA!Q2>@J*0i8OC#;_QjKUV!gWD~iR0~=S66GN z16=BXv3&EIg#yU|mll?!)G*d9kG9Subf70opgk?6{psX5n1=W_SWrk;I&*WD6A zXo9fBqyz!8?HEX5UCRe4WUoe);Eiz39Pt&5VAF7iNO0oWXgX+2qf#4a5?ES^Aj9B> zBL86wK>KLu;s#r~w`?MRJocOjkfLOV+zgRL*hJI_JsGB#(sRwj)FI#~HBE+p$e%O6 zSs0wELX+^=)KR)k)5=~}tfpi+y%v@~MmxAo>hu0#f8TgJ}pR13{T9~BG^$N zA%Y0WR>6yMX&5H;i}cfSTin`OQ^Iv3Uoe}(WtN{hdTfis=9+_wuVxM)F{g^-hhj@xT07!C?UlrE}5I9-@3<_f@JnIB#*8lxe;4RYz-)@*@ z-A9CeCig&?M?t7ef8*>L0N!G^P{)DDw?+3NMrsTJ^YM5Jm$;q}yX=)TVEEtCAX5W( zOKc5zV0i#{ks`LLDNJ&JuKd;*3=qXnK|2l zA_oucc^#Wp8s3eH*=*MP2uJ-1JkR;a>RWbBem0iikuU=5u_*fZ6pJj$<0LCe(~RDT zVG{rugIO7g5b-8(TOmet>kdXY2=e;A59;C$f{j(FLdt0fn^5Ovdb1 z`sqKew}-qUf-Ec+PVvs}L@=2jjynzk)G3Rl?ub%uExt|&+(GzDn`;9ud~M?E6kgLx z8|NG7-28#0VUBGVfGHsOLCZX^RP|G%bb=9EOeHoumRTzNj`X-{bS$z~wscP776o?=YT;VqF5h!E!MGApE3HK*E6!_mctu zOsQ_xo#qp0!YJfpSZNsBoB^OH=Gh4-M2`88j*XM0xoJ`3MZZeo>=SUsc~{!VbheHg z%G~Vz!+i-v@^Baf&eRTNPn^00yeg9IjsB>cE--%WAEmw}W=Cn}Qy}BQ8S!#Xy=sHsdWYyiU>7?Udj+5^8t9j-ARtx4J z+8~KdF_VeZEx6w6u<+;O9N*T0=a=Sq5JAVn9|Y)l0FtX=UG07IYF@{Ee?U*!jAzPf z%e}&V)T-XS1*4UJ4BB>^MS^=N*@xZjXe&GSj^f_y)N-1) zm+KAZcB(=1q6;=3OS+{Lr)uj1qYsWIqi{OOrA#upXB$3$#d{s(9ihN0ZIbA0{AhAf zqm1#(S0onzFXcSo<0PP2&{cEZs4tddr0$JhdC5Ds6Q!;n7&AA9<={Sl82?<4jHbv1QnPE zOfwP)e(qN6Rul@`Xb?D)ibXMX{(S8pQLw$q-(*8Hbf)Jmtc60oFiiJ-u_4pIuRDMj zsUHg=xyg`lCcfa>KJR)kFk2`)=4B z@f!UW%gxp?^9DRE7TqoJzF~r8qlX-q36>5E;Sm7Ql{AUBhPJ$Vo$ebQzgE=7JmB&v zPpd5uVB_T@9vI-}6*J~e6lToZVRf6eirEjmXIl+MMBx)-{gV?cHCx|pprU0vHChcI z9)Wi3t_irJ?j5FIP8zxoiMe!mDIn6t&;QM_dTb*0t&u;C58arl^xQ8eHZ{eORR)%E z&*)X#K3Rn#_hh#ZY+}8$XynT(s?3Nj$=9OWcpR&wbrR5PEt+SvXGktj&M(Sz` z598?s*PJF!>K8|lq1iO4<-++Rp53{HIIDX?wzO~Age>dN4?^W88N5UaKAJ`u6tI*a}P? z*4Z`c!`4fH^OuSdyz`UE|Nnfd?>~b%Wi{U8Tf?q{I}G|oiyf`gO<;YC3v>y&rKp}j zGKH$+4uGl%xb+hJt^fJM-u-^9rmzj;GSoDhfMQ<$Zyqy&baPxDPQw9o={g(X&U2LP zjA%uXi;R8%h__NTrI_jvI2<-f3jHx-#74jxBMIa?@h2B7OqTiAoZiLy@%Yn49pkc! zT8W^`08S>GEF*ReYBt?zr|%c3FJoJCHTgR=_Zt^ou!0d61ttmhs*jLa%IGO==^tt?_EcVVTs>YGEa~N%9rs^~qdt z$^_Oy;Z!bq2v&&I&}`kQ%Vya0j+TEkDGy%5Ow%y!#Qpx$%uj{G02aR}Vi$vL(Q%m~ zha`{ia=c0Ez~jW0_r{424~jO!vBjhqw%K{TxgrzqH3#pL2hwdZmb z-tFb_6=x|Qcx5K2sMO%oyNs+gFB;y?ZUNQ*_SMZzvgtIoC?Xxpu+BGIbEBu~PL;l^ zKp;xUxzNj%?O4`qH+}J?1dg}v>MN|ai1sv@7>h&hEoIywG%^pWKq8I5)D4feXwxP> z?$@1epg*R+aJh~J!2M>$3=U^t!8R@p`YjW=waUfB(u2ODMk^X?*93X}Fn!e7c8a#r zSRfP)HW;aj?Ks1*LUhX+YE5%aXL#M5Y+r14=}8;%MCmBZtzn`Q`rh%hiGY%WUX3_{ zWKfdlzR-T0WDkS`c#IveHNlSvXv5>^T+|G(^soNliDcJ0Gm zk*}542EK6e17x)GwLU|O55UlDJ^5(x4Rkjbv4$WFO?AWoDw%Or68?Q(YX>LICqVm2 zGV~I5q<==kebis981Woorna7{D&GG0kcC|Pa$#W`tfEv&Q?|_4uIdl!?q4QJpG!G_ z4FA0UR952Oufe(YN5t=dmy-W^NeLUaRIJ!D3IR;ibE=sl`fpNf%8&Cc9$b|`;|rd|9*W{UV!kwO*M42Sn=uT3v+G628=uY z`?WUtS`dG3Z*17V_W0kc1kg2rlpY&Lm=!P})D<7Ni~k?$$_B80AAYCbZx{<~sx{pe za?ZI8kaM8hIK4q6#_@AL#CDvafOr_NEM0CM;%hAC2)AXv?V&HUWLpd--j+0SWpqd) zLWH>N54u}eQm_1ZAkyCm^8-v~hA6sS^`CnK5SjoYUY45{C^^9u-ZzBnfAy#q;!#M9 z=O~!LHSjEs7HpRybh^g24+6jkJ4A8*_Zu;?uY;qURxv(wv^D+;4qs zc{(gM5H4)U6Iq4-2(Ccw?9L>D1_gsKW*iX3)qf})DTWh|aN5bskk3|R$v}fY5wk{4 z8Kz>r{c%rc3-i5MBsh*c1`L6mc#Wa5Q^CVrGkA*Y?mk9AAXrkYHxW0GkbfY+4dj*m%KM*~a}URbZAS!bmX@qNz~yhc}*X*tj3+g2%ss z%_O^}2@FVKa!25gj%jeDu&uf#n?beH7{FGdQ$Mp+4(Nw@`H-h;=WkvFbq5H({_VCR zkc&A5AV~RgjO~owv0ij!=Gff^mn>e$W^wl^IW(t)1N5R-t1l_pKVv~esZJhrBO!LZ z5`f}!o!LYQ8k4tF*j65EgL^bha<5mvr-t!#_0QC9{=cU-l!~3zgLGH?eU;X4x_!a^ zf6=XvJ^mJHeP3tP+J8%m#~|vwKDsuTfbUoaCa&6tx!wM_gtbryHOAKy`iH5 zyQvng0qZ)y?ELeUQWFl(p^aDr_zxTAf_ywsG;4WmU_n_Uu57wwaDBvlJ94PkS88 zx{`VO%B;}FcGybgJTuSGDxPlrIokQjF}DTBQFcwfc}wf2(4MEJ>8Jq2#?+_acq)u< zud!0nd>{d@`oRw$tHPk`mF$gnru`ZIe^`6Wf;oT|+EM?wQs1E<+ZyDvIx5 zkR|N1CCDzG@mREQXK6SvBestjXF6UKPpZ9G5lB8i<3uZGlwh|&Ye59}KSKS3CHy>M zR~qLS7P4J#JvmywTRNYm=G%HcS$~$C!Td1wod#R!C`U;D@Vh3UWPPCzYa;Im9{b72 zv}F(2b}8|hC!B{2=riQ^z|gYZ*`z6Fk+XkrTI(a1raeE1X!=m~oCQ-zvRY^R4S}xF zI&>&P_GaQ2`BPVK-gW_9=8T+f(b@gWYOQ%T6WF?YNNbiy(nq2U-TEVL^Y$Wq*hRe? zuWFZ$N328b&bRp0{XhvkNY+Q@QE+{9+^^ zRAJF2zSL|~HE*QETp7N-kf632$EDkUk8ncCBDeocZn1YMNywu>WHyyTv|)&x!zK`I z@qYLpZ=8{O)BmGiHw=X%|LR5=|Bs;}i62Ls%N21xC*(1Je_mx?5SKJa$&jn5viiO? zg8iU}uk@$qfvTwngMfVxDhREVoI@SJ&B-IHdwSSqivuAOa8A;xI(QY?6`Q88%aqEb zP|I&;4Zl#cs#0gsvQ1x5f30NU(yyWE(6_#~v%hG* zmdQsQnYkKEY+h{8siLvUd9vM5zPF?yX*M$HnWYj}w&MDoW~J!l&dG9$<9dS*^DeV4;v@`r*Xb3uy`G4}x4JAN1}ggk%d7$(2Vt7}rI;l@ z4^Ck_oApaY9Z~(3YC^d*LnCuZE_cW>ExKw%uo7I}R9SrSt8+b#B$qqEsFIS+E7mBL zGB$>B6E_XXo$S*&_twi?x|eo(G^_Fnp#;w!JUa|73+IU{wIptgI9;M5jxNw~*HqMY z^EQ@G$}bwsD_y?qRMm)csJcA5`PgDWlfj}ogOk+BrL;=WbNi`)DOKc*!ZnGD&{`LB2Y|0@yXiL>;)|+Yo?m9$FV#X zG7qet)175CcM2*e?nQl!%4{a6_Qi;cZLXEv%$A7SOasm7((64*3n~e9W%#t5Ttkhc z*h))>hMe43zVzFS%)h%UI=(#H*-(ZaL)_`G6?VKiWLYa`HRNVPV7erBVl%y~8AI-2f>4Mhi=-}ec6 zc~40)o^t8tQjuQfYA(F6DAaMOt^3*DeY6vqv3FD{nNyPcLZEC+rC^d%?(WzwG}N{9 zM{VC3dRJ3v*>1eRZa4j<%5nR$rs1*5{?MVW+Nqpt&CtW=@U<5*UApej(08P#8?FTx zr^{6&u4=&%+W8f_DgLCNw>N9kUpQBSgPix|9zX3WpG_+$JslPW0cl?MynSvV$xHt+ zRyk=dwSEFTT{Q6OLJ$C5zn#ii+#IAbs)%!wu8>~5MY4H<}U6X@vTO?|IQ9cD| zM36+7pyTmh>+WsEW_|D!C@CMg+>+Yl`m^zh(<(jIl9|Ek)rRJcdoy#%r<0g?LoW{| zxaHocE>)+12(exw$a3e!!2lhkZgz#&quuIcF6Qq3N=G!TfH>*JssqX1RO_i9>ZF@? zmHC-#fmzNswbj{-jiFC3nFo4T4|-1SCgpNLwbD>0JyRkti_7yTjhNPLj-{ANI~O2_ zM*=RkjT|EPjYwkvG$*I%Es&KYX!;q72|611mL)9YRyB&wrr%t>8(7%u;JV`T)(Q)E zcIxhAPDIf0KA*F?y5>$>2H8>vUqMe)hBgsxsV#HKF923XcwTA@Sx(BjP8@wCP&f7? zr*5i0&334OiI;L48NXM*QJH9u#V;aovYrd1sZ{w!ZQ(uVIk8|{#(X}t)3Fn+Pg8ND z)rXT>k#N0x7JQv>=W4s0rp0wv|J4S`&A~U*A!ZBx^i@U$!6ms>mRhG?Bc^{oq zgR(9WPI!-fhrC^f1H)xl ztsb(R5|x}>>#p3?{Ozz9i3lFSm8yl=3^J6Y6oH3}rhRtjo#vUBe&ws1N%aK~I#RaH zF;}tBx1C>XXorj9Ud27__yhQ<$K}TSkk%{Ona_w ztbT7~TMd&jZR&3Y9Qd4jzK1QOTPuhWZhs(@uLIX}-fm^XGF{Z8YAC(9b8*J8*!Ps> z;z3Jch^f9|;1 zbQ<5qbdAc|UC&nph1yP8%uy!ckOB!%+*IiADR<=i+H zh%9XSV<`C2%hHe&$tS8djd>5!$i&FnOyq`~%{wwu5|3%+wu-EYthWY52yzP-L8`@e zTJ{J)shu9GQ?I$q-oNp;XSS1r==f9_LhoE#W(kJO2K;j2kK=xS8@|r$TU`Qb6*uIpa)C1OSo|1D@WNum5yg|C zcV@NPQ3o}KpFmxEXe@IujcvF|sZO+Eo}2^M=9a7z8{QuvgaY)I{NF$bFl~*CE@#r3 zIZoa;h(TP-8t{H*u<%INd+tkrs&L6_G^>92kQE(7M>@?np*Pr%#y_peK0+VV5x%T- zeQzum6CcsN)wp(gs9S*h7$1Lrg?rO`Wb;F`C~f0QN9~Lq4K|gh=Bh2U+}nA9dfpr} z-4+~Mfn8#?8f@%!XWVXag0S{Yi1jarl^(~8P}J#$$y`@@?_sS{d#rWG<~E3R3n%3D zcuJMgX7Ju*wQgVJ%l+<~hjs8kArHbR%2b}Ji+0Zpz9f3~Fz+e7dPxzlw6rGL8oDs?#7*3|#lFc30r;|f*3Y~%yR5u4C%fkX8 zOwrjIjs?{97}vslxZLGNwBqDYnaMbJT(`Q}ct-;+1({{_W$oAAHWLf0!{GAPmmnOZ zwN015sUEW16`lu7GEt!O;b89{8GcUx4DwTwr*>@Gk)ZPYPWR93@P2HUssA%5ofE>2}pZ3l?9P0P&`z;blL=h??YnEcN zuR~%6lO-{>WZyy}OURyt>{&DRJu(tmvV^it)=1WD2{T!<&3(<(_wTVJeQKYFh z{ygeaUm!2B2YWWVeuOJh1Nq0Njz!@j^tNx=b zQ{NqQ`+OeMW&BDo6wN9$DVzQYWK(S(b9&}(qbnE9wSPPus`Z?U-31FxnTi2fJk{Dm z+)sMXy^kmbb7|qt^wQYo80)Z8zAiN*GhY@e3_@&{4bmv-s~-j_TUzaHuDg8odB-q3 z-{XbG{w&nl{Ro}HuP?83S1xdJn)-&V45Zw`d#%5HhSFq7NNVcXb;o9TH(f=f_}Ul4 zHEI(4xFe;E?+w~#LRJhmhJ_&`ukUmDiyR!-U1n3?#2M^j8y0T9x^N5Sj8Csce}>uV zyx*D+LmW@e%MpRp;LNv*A>D>0mvi>HR?p0W#lRkgAf-=1l7Wjx06%~Pb?9?Vg>V2*P5plUR@xdOpQx8L+0 z*co!@FA1pgn5wdc$MiU0&%E))q#=Sjia+cH6?Rd?P%QV3$8M_D@18MNE_7RLN~V%lSPkr^r(A9wFAAF8UZr`NSR|y_ zM1LBw#D7qlX@%uv&HPN7n6&2Xd1kR6)o!2BJ#-D*4{O{PC5Fv5%}ie&T5g)K;M!B( zSwFlLR7A0SWSo=246JNh>$eK`c9sfl{ZTm>Y}V3l$lczRTgRzc`~^RIzb|#pu6Nj| zWAqUT9@8GGDGb7GSt_tDp*Uqt!tVd zRuxjfl4lyXOgw#DVrTyxq+jyNp<%lVF1X;ZkZhT&CO3Z^&xu$OOsA${;jdp`-%1_q z(N!!t*8a3*udMWN!fCCZGjoURX2+XP>w0N^zfgrR};o7uiT&HZ++=SplNSn z1=v^d+cFLh2BFnY&R`h%BLO{$ktF5`x=*oKL?cpr}D zZJM$3b(AkE+l5U7nlz~RGeQtD@?&*t(ZiirBSm+|Bs{?;wvp^Z%qSS{ABWM@n)gf? z7OFfyn!Q2ECw$W$#~^{`{_5z`Zk?+0>5M>GV|;hV(dE zug7f>W+#EI5GpT;9{8;$MTh+mp!8>bjkrfV%qdAk=0;7Qt zQ0LB~G>UEwR#4HCxA1BYO|abT6q*`4ms!OyUnE%QcxGLBXl8Z6zZu(-IA57k)Z<%^ ztS|14M@>8p2Pk)+^-CuAs<1KSlzqm1hR423of@pt&db=>(Z3Rqf;E~}L+Y=!BYmWT zvh}*;`2;S-jEzo&-|~(CBxJ26*WRl4&-RR<%}!P!Y4{BAm7P(?9MEd^IkU`A(cRZ+VX;8OKPF z!<_%=fOFt~=V5cJW(K~6($n=IfA^3(nPitl0+;$`q5Ekq?J)Lzd5QKvM4SC`6;gR$ z8gXB+pi?@!~!DtZ?$|QcLN1^xiQh-3VDRz(p$Yx?_r=|SqklL4 z5Pwm)D8ic~z=PgVyxs&*Y6e0vF?*9nE~)Q$Wc`{SCK7~VT&7knaG&@e%bou%9zl4y z9S3dBBFPLCDYVKV2IiCJID4~VMiWyi*2}$Ibqb7>!ZVTkL3!ggA&hu=qcoVb#z$Lg z-o$87Q7c(DOKU)A2$Gs1#WU2&1QiT`sD4M&69q!VTfGqckPmsyn9A2RN%=g~RRr0j$sU2o^=S zVv>X(sN_iYI6HTKzjuimF%Xsl4C1UKbvcusMW+cO<{vYimMn11kFYbU>?_5HyDJCD zI+|$PKun*~GOJT0WDo^A?L$DW7t`LmPqjZZi5dT3ka7~%Wt!JPrWh)BpQks4vqz>} zi&;`6I}v6x5lv_GB}b<{4hCutH~6%My*yzC393T6Dsnkm)ivt*Tk-uberZ}|on(H$ z%XWj{UEwz)|4wXeD?*3w<{0k|iPZqrzW(xX2IdnOsP@OJ84mgJyaJy-YPYW`zZDkH z<}(tzl+9w9vx?mAkqm12{3KR45sA7!99LkXAG(wh*E21m?OEl`k&pqR0QU<;Y@a*o z@5DUFl#3LVrbvm)n#?a=5aOv`jIk+(uZCo8TV?w{$6is_+Ws)TRj8Oanx~RKhCbG5 zGUfHMp%`A_lZakIA5)*hcHeWIf0BRT0ONwHl7dd`#wD#`OJq=8{?3B{#Q==;Wl2FI zxI+N!se=!Zx}!5u9?l4CMc>prCaI4KKqj~%NieV60&b1U|iP)tL3|98cMr`gX^z9bl96A821~ET%k3i#t z+GR>Vlwt@$zg5=~4I*6;VEUBkizu9@2h8}!BwoM(nK5OMU|p$;zx&jAxhOWcL(L{w z3z?CnG*}_fuf^$FKOJ~H-1yv(li+v66=I3fej+1Eo5&{^6+`PGMGof+nGK`?Mrw!~ zN+8nN8@qD(0+nUa;=fBJY&qUV)3+S?R~LX8Am1uCXSmkShem4tz-Penb))I!;yVqb+8ZT?r zB@6{GK<+g*s}diOWVrPEY5iu@(yOS#O|*J;IPo4y8I5v#0ebm2$6vCp)_&1eZ@M}rRw?B39)nUm`-o;#%XcK zS?eX!6>BF}T)C$U4GLUyb>{*>A}Z^b6aF^wGgc!xsz+)Oe)G#O z*8Vn(xSb`eva{gX%TaWZ5@RLr&E#r!PHz<`Ci=it3)KV$7Z%(5s+1;X7vkf{eqpsd zU>g-KB_Zgs)nl$g0-ubFVYV{$CW!^{DbjjrJ*pyntFX7iu6$8$DY_?>KD--nHGcvi zm6g_rI>Np1oB%!LEUryOiil6N)}hc$XwNZ_XUfd<^`1vEvO%qzDp}Nrtn{R+Q zU%3|jU}jT>(=}PEgjI5fgOedmt7H(2*lX|zWLI|wrgTT|HCicv??_AD&sf2f#Gdjv z*FI)o>i}V;uq70IU{WZK15h7mCwDcBGKBU+7kJ>nPca)?z8dIz@+?0}mJ428t)5>8 zQ#HG!Rh{jCblvgl;_Br?TNTo|3Ph22s?4vgXIJ00EErf3*}#1EL2_Y`T>LyB0oXQ_mb%K7ACTebK7^0tBI-Nd*aBapg{5@m2!u%wGY z)7*pIu=!DaURR;G>e$%1=L4J{-S(FAeaG(o9GgG=T_2f|`5k#jOGF7cajkhj0<4{P zF4k-R6ZThS3Y2Ii)I`VAQP8dS3v`PE1xocfSsYe6bJe2RNo@J8efMg6N+A67L$01d zo(b`R)CvM()oKDXi411L2{xg0^V>R$vsO!}iB1*73tTr{j07xe!7{rs?d-PB-OETj z`6+gYt!TvRw7<_D&vC*qj-eZObc>nHb|T#kI~T?vGrSoMUvyaD%mA$ zw{m)7vcC`V!Z}DLbSzuTlr=olAmRHstifrcC{`m;A62j2V;SctVPl|=bp!w2f)cXg zXUt#RNph5s@B)F}QGn@)ZM4h~1H(7NwN^&71R6j+!@k$7B?E%WPnrTNCiMpnBCb&< zDIiBtnrd@L)n8&hW^+nHUQBaP+ps}1FZ8`3Cw7xCSeBikAq54*ickuYxovt4wwrTv zu4&yOsl4RJlaa6vA1-Jn?I~UY4=3LaaA@lxTgK|{9b0Pqd2Pw=!UCQnIq0IJGKg{+U|FxR9wp8one@iHqhrw9|Q4b0Vkw&n+-q2jElY`)e~}CpaKT zC+s|tD{CR}QGaZ#2-%#LQ0emKI*%n2GocQ6ydhu;5zauIG9GOvce0T(J~8Pw%Tq#6Xd(gM}X%Dah=`<^6545(MR9GH2kQA zllaJN7a(W$i8^ApZC?M2txOr%Vm!9c1HNO#OB^_K1b77T(6({$lcPQbI`I-VJ@PJu ze(d^`?MP~KP+cVlHVY&!oR)y*=miJu+xFaG1AjQ=XWw||FRL{|$jqEIBBp*~1AwM6 zBjEIZ9B%&kR=9MbW^))6v$VStz1#1#pM3favdv^8p6J-a;K_+Gd=`Yt1nNoH+^EkO7!Fe9CA-%o5*}e*O>vykDxCJ)i2- zNXaK4P%J4G2b%{@H2r`DD@ri-RJ~M;js||hH*cf?5GmPn{fC4>6+#*pm_gIq426jOzd!S^(Ht4=Bz~Cm=YgTMN}4YrqDk4*+W@z;bAxle~|) z*DjC?TxJT)S|eQ77Xc#Ew^%t=0vbRx76&dFxXu8mKY-m37#~Ib#8$3k`wVrluV~fh zj2Rf zY-tMX%G}~xfEC7Cr`TAasx5F>C75D(WGt>k%5Kd$Rn^ZU@Lw9u`|>oI0Ta`vp$BYe z{PUbdRZ5^u7F%1O+a%Sta@7(ufMtt8J+L6=aj9J!=#^Y3%4GG~ywLX+1Eo5m39!GT zbD+Bh08di*^XENKDL*nyu)C?Z>`Xe{1F&K@p-uGwmKa0> zQ?4VDpilEWSLaR&lYs{Vzo-zvE1vdNz!x&rQJMh1ofSG}Fu?k$glf_iX=EU3KDz?d z+_T27ok7$u0DgulW9ob1%M)^y4{`O~60>J_mwoY6F~0!jCWr1HIvIZoSXLQ7JI}yW z*EQBBmi^z~`)20jjn@hJdgJbsMk{%6>&=l6Rv+=Urxk<7Qq2UWfaZ=lm@5^U^&aY5 z>wSJ+d8ALITbh2-VtsD|SX7lLo{+d0JLSU<47etS;X<{L=4StnjAr|FeVSG z0j_#6?SOANJu6g_uoMA^KifEEvB+`h&=sXI`i6iLl01DWY2%&XUQFj(LH1&;Germ~ zeCTMy{<lQKbq8 z{^`Qz8k?n?So1OV_lp(gOv2_W6>T0FdlMn7-|O+a%l0QLc(%p^Xd0KLhn9P?;cfS^ z6Jqv~mu`Q$#@NXRiT^T95y$`5Qe1BBr%X@A*Q{u*`7dRXvu%f=Tw&X0qJYq9v2S-i zi^om1cE1i$Mw0(R!i6Fa!uhRSHk~};p?3|EnyME{SiZG@v|i-B@l|YRq_N#g$NtSk z@Wj|&>XERQYv<2^fbpGsc?7xDw!Wg~4$fh4Ei8(b*Jn%*PF#tCr<7Atw-ars#B*9> zf`YflJD4KF0o8d)XBz-{b09(oHKZ0H2KKuFdcYE980tf?{rk(Blnq5zqncgLCVjBt z9X87q3}33+4Z6b`iR~tbjt&f#E=q21Gt*Xx#!Afna6cMTdO!tn`{mmuNr%z~a#N3MlG+V1yB$r9 zjf*9TRTC~F8iVz~3;Bo*!F|6N_?%7rT&)KVS-2livq&?YSABBHuXBv+ZXwNWY+4}M zvJZ{veeePlbgW(W&bwC=Qu`866Lz(b4{+6#4>1fnOPL86z5APQfMr(GXPnmYFaaT) zYUBmHly|Dx)KhvZPrZv0$xW|XwLT8<#NQh7N&A2XHpNxV63LiS!!lYNUFK?3vG1Om z`FiEd3y_?k_{%*+{{zyxSe|WIWxa{p=Y0@F5f3e6>BWxi?f6YYc(KI+S20!+O! z8{45LWH-kzYp(B6cVS8~Sf&24cT&mkLbAhxu7heVR*~|{H68C_wt|mQBQTwtrA;Ra zKUz~9Vvjr44og9e%br%Vj;)BU+g6!9TjrRTS&}J>nkb@wF=fF%^P`sQp`JAmr)M{v z$OGO-ba$tWOD%!a=yZD7-U0aVAsY20fzMo_MVR@vhc}cXL3|eQWBoU81^r-eFX`9= z2nSv);`ioge|yvU{1JCF1Lf%1D=q;TGzJ?uwyl+AGUYMmy%E$YkUD?+!*~Zp*kSnb zWe!sm0Q=rI0N$QeSRy5E!h&gcYpvY_FooZYg(sdKM`Q@_o(Hygr5l0eLc$^t=?|-x zaljPs!L!qM!xA*yO^!N#pR=b&vDgXow!!`yDYHyfNAHcbdrqqmBEo=|RUdwz{Fd^L?RZHI6% zzj^jBe!9Tma>JCH6OuKvfg8_3a|7uX{yw?dGc;5md?UdE>g9Q=rVtt$_=B@@D{I0u zk=XoGcJk(GRNA=5tJzxGn&u|BObuCM;z?uW@#9Ak$W-qM3&v6Hcl7nE!qf<3 z@45sI_o3jR&@rx0_~b z^)62HGOA4HB~XLKxgvUjg&NTOOLUKp3CU7U2NGg86Q7X2t&K`<>^KWDZh2UcETh zjz}?`P`k6DW|GF2$V!dfWfM0E;5h0RR91 diff --git a/website/src/images/dofn-sequence-diagram.svg b/website/src/images/dofn-sequence-diagram.svg new file mode 100644 index 0000000000000..898b1aeb65d99 --- /dev/null +++ b/website/src/images/dofn-sequence-diagram.svg @@ -0,0 +1,94 @@ + + +User pipeline«Serializable»DoFnRunnercan have non-transient instancevariable state that will be deserializeddo not include enclosing class serializable state; use staticnested DoFn or define as anonymous class in static methodno shared (global) static variable access (no sync mechanism) but a beamstate (based on engine mechanisms) can be injected to processElementkeep as pure function as possible or idempotent side effectsbecause DoFns can be retried on failed bundlescreate DoFnpassed instance or deserialized on workersIf state variables are known at pipeline construction stepinitialize state variables by constructorDoFn Lifecyclecall setupreused instance to process other bundles on the same workerIf state variables do not depend on the main pipeline program and are thesame for all DoFn instances initialize them in setupFor each bundlecall startBundleFor each elementcall processElementIf state variables are computed by the pipelinepass it in a PcollectionView as a side inputoutputcall onTimercall finishBundleIf DoFn is no more needed: call tearDownCall of teardown is best effort; do not use for side effects diff --git a/website/src/images/source-sequence-diagram.png b/website/src/images/source-sequence-diagram.png deleted file mode 100644 index d544eea3d851e5ca1075edcb6d66c3e89a91a981..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 73657 zcmeFYWn7fq+cpY_0s?}RBHhvo($d{1-O}A9Go&D$BFz9wcjwU3Nas+aGz=j%L$fdM z`+0uv|9!vg{c-R8d%nRn>sr@Z>pYM1JlC;~6{(>nkN^0^V>C20d_@JB_h@JjcG1u< z60jcuPu6WjdeG2_&Ouswo=zX--OXHWRb4Et9MRC2^adKs1@Ue&npnb_gtmXi;i-=O zMqnLgO-J{CfYmA&eEIb=JniZDeOfaYiJf$Z_M~BFHP}* zm{aD(BLZzIH1aQZrqtL`lY63U@INPI)}!mzV0Uw@CiW2jvCkOv zH{uLA#3j&&s$AK<7daYug;Mp6%)u^SO1Gkb5YPw~othA#y+5t8yk2 zXIe(E)rFyR6Z=x9_xVcQ%ac!^NX8d=e_j*tgS^S~)c&A`%D#Zw7zV!6RT(Ck?~9z? z)mZq|vKD_P+4OfG<Yaw3KkeZy^>hoeO2R(ov2rN=hq7-rhcDs^8_)rfN{4Q1Mes z8kOFAPKLOidhSE_b0nL)*=d$9oP#`YDZ=S1zn|+-u=$H9(^nidPc)_TXt5N#LU3hD z;~rbce8L~d@1)gG=kJx=o~)iyK9R9vBpuzZ2`2*;45$|?K$e&JBxmhQp@c9M>-y<= zH-X-TOtwhM1My{0xHn78wdL8jW{*tPZ+Y(gTkxcL!O=c#SQVLaboh2V%{jvt+hBM9 zFCE?wOSUJ^ko|%2VR5_3yu_-z>r{pIY+aHISzz@duA-hh04f38IiU%y%_Vfkl^<5my?Op_5nZp=dV~a;RG3yY#M=4g^jx_yK*4SW-$UQ zz2I&FrCN1V_cn_eTI*}yqmdV6;%b6c-aPr_8f_Km4!EyCvBiY2-#;F6Tr|HD0AX|u z<-}yYcA(ItYn#zZEhSOz-^oBvcpu?;+jD`YF3ue-Db-eLX1b2g)A~5$uv}?-x&653 zQpkeGDbTmp_5`XM_mE%gYV}(w9qMt@Pe{*?$7M~$&}?YJ>s7G!TTW}XEnjnvHXSSZ zVuy8K*Me5?Yq5|Z^~)6uTuUZNLT0Zd1LJb@3m&p0veo5;?@Ke%nPO?NGp`)ft)#^! z6oz}B7+_H8Gsn0^icPxDfS*uuz#Pu zoMHuMShff2(qup$CO8C2_gblM5Gc!UKd=jmj~}fl@?(OIH;uWKABZ^HYlP!+7JQWb z{PIk;zi$&9s`5DKa0hdnYcK3jW3AIc>Z;34z1h|Xet*&39X18FFdoE=Jd{Ug+UfdC zX+6=xxU=%Jpw2q9Wpu%m`g?L3@4hNA*5`ql6Q6F7hj72x?Q(xf#CRKbj~K{KUMl9J zz>Tv`lg4*`QkiA{_EJ)fp$9vseIY+?xSUU}w9mCj4n^Hfy5n%YF`V98yq$yyx%JK; z2pomuZlnk|Y-Rqq?vctS@XfGU>fBTmZVN9Qcpw=8jeQssp|;k7A0p0U1l=OCJTW3(B1!%7$mnt50f|!Ca|O|HtBtn({ypAT6AxD{9KI907#S(hQL{Qa>Z0q?m*{yOCl>_f=Xw6(g@?((*uZdgqaE2K-iJh$ zCzf14IdRop_o0g9+j`$_PDq`&^tR1kvVH{(!9SYKD7&Z~uf@lH7wam;J;qhlkUsKD zIgg=mptpSS9eWH94YxLx;%2@{=!rJR=`hxdXH$;Si4I|{#4bIqsd!<$r0u zfO6?&gG|ZmFln(J?%q)D5BjeOsL}9*6&F%a1eHQ7lm+khr1XuEL#2MV38Ckr!HOrpaT~ zEH#?8LWgxfTGC*goiG_tOA1rUkBz7Pfh9)e&+gaDV_>tWIwRWaGKtCpQK)7_wO!|;Z2POo4SK;Gc z(J4$$Z{hNF>iBp{_ZaE?zO?T0$4-A_vJTUf@~ODw5S7`JizXzS>Y{xB4gtEb{%L6i ze{4NLP6QLLgj25Zk#YAvjvA3X!JD6sR;t6SxU^eAa=)a#<5yOl=Cw31Ufb2GQT2DG{{v%Y6T}%2f5i#Ef*Zft!jahihnomOA9D zjD9C;=!JINonFkw@8z-u&9Jvx)J$zrkXALKgLPWW70Zx@blWp5&nQe$@Ht#EGV4jx z_!Af7T07{fP!}hU^(bD#kC*6g{tp7)H({ z5+5uVFDQgAo9?2=Q!z56EaE3S6ucgbAn;fXrERnd{5gAs8fz`!`Y=HTD#x?=>|xRv z{nPFKr^5T~m%gpSN9GqXWm)9chlP1)3O3@9rBk8bn`MP*4n?{G-i@Jgy>@4x%T#;v z`dCY>t-L~)c)zjmT0XqKlm5y$V7p|$zU7!h@m|owJcf_C8--BYK>GI z-c0Jrc<$^?`;C0y9O9kk=haC^hd^OpXmGw3t~!%xK9f^++>%=eF8aX%+p_I5(Uq<# zd@zH@kaM3aUG(uioU=w1pA)OU4R%M%xA}%li6G+#th_VP1WmeBUQ{&AKVHp)od&sX z_jXn4CR#9x)e7AsriRv%B2+M1BJS6u{CoM-rOj6YLK39x0qGXhT&2wVM`BM%bTZh; z!u%e?I(Fg*=!UpKte1ST!GtePKUVEZXB#Aac*OKjOH1P6ZM}TSPl&sEOo<>$+n%&q zf`1(%)ihH@vo2?MImoT-k+yc!!T~c8hCY#B-48BKPh7OY?U6E>CoNE&Ei^e*nJ}Gl ziX^fRlVD!8x3a>z7vvo+|Cwre;AnZ_E7_sM6Z)0)1=pXhlm&^An?|umc*%jPMiXu-tJvMuPdXx{m@=?>>W5gSsm7*)+@Sy0g(`6K!q!Cv{N%%ra;qyYG zFji*S;=-hPj5|K;81O0;ecVgayC)RfQe2YRM%Fboif3#cLekw4Za%g4)ow6$cZaI4%sCnRJG!pV~51 z$}yj;t26wr48SzTeP~N()|PFs^GTOqKaG^qyM(XL@&kN4#U(ziRWG+x)1!}wryKZ^R*E@kB?mJJ(My^}Y`iZh21n4P-NxJ8AYd(vlG;0W_3S6{wZJ$?&Om}vampSBx#)4ts>d{P6XWNf(3zl@qYE5<-d;#dEoN+nX#!<`WDxu56O1!x~<%i zXY?!hQxXIF?f0i4pW1AiGz%Yq8-fyx3fx;SN`I6$VY=0t1+Er`e=FT`ASmgr4X7-i z<4Sc)5!1L{SDAh_C)TZ8Yd;SokLtHBY;)ijBAGQqfqH_2``%V!#S*Z51ad<^(* z8mxgt+a1PRM8dzSscmaE+YfWDH-r0*#^fW2)AK@AKaF z`|&Xz)w)`h94wn`9+TqL?YiyifkrSptrLDN*>*;q+!8P7gg+K<>w5TjwRUL!H>_vo zg@Lu%jI)1eFz>UGpM&VZIoW+L;V)f5itNX5``A{RK+a&<@svLZ#;1EHUW%=z_H8ML zzQfU2^d7FSIw@kqhj`C2@i{{TL$K=xR41nFG`_vlR|h;<;zex9VPgex*%= zwmFkIvF+m})w#*h6aDHM{GC}a1k|UTW^#1bS`$90x3EK&tD31Km9K2I zUg8w*Vsq*H*njNEz2Ikm`UmaJo7A}40D;tp@?8?mN7`*Am7o!Bqk-e;lOOdj>C>u5 z9}(QVkkTVEvedJ+j$Udsnjkg@cyS9ptxwYn4gAfU{^Ju5UDw)?D9tUC)z) z$$rf6XAb-MMfh;w@4;I5^1r)GL`J=X7;zs{{#KJqv7`snd&Vex1ICp_a4y9bmQ6cXsia5gkuen@lD61 zt*mJ(&T47Df*tOz2L5(=M*nBXz#qTTbYlSH10LHspSzljsqwF9_A$(KZ3#0xtquRy z{T-U0pKKd;X)$lGIAl!Up`p>DDauG{eKgsJ;52JbUk{k7zJHfkLjPTgoFP3^S(Xl8 ziH9+tF`wuuDJu!S4BJ!E7mUg2!x~XV3fT1X}_PgD^-U>gmgOwO^fXuB%8Wv{%D4b3aHz z5o011F7OUthd~&jf*VGI!+c{CJV2yRp8V%;Qci;zey2qemG$b!80d7>&*(*Iv%B#A z=}qBfa658w1jN0XHX-(74i=OYEs}K9aqS!mTF44-S7f41ygLu0nsYvLP;7q!v0;Ao z&4S_j{51X`LHbDzpTk@b76xI2BwC`H?Z1OlB9h3@RLp0O4j30j(avb{43mG=#fb2% z+PG;*QmE4Vo=d!Q6)RVKy{?FuVG2@EH-_`@{2hv&Dq7-Qj>3E?ztz;!J-b`44}bKr zX{S&s3NkWRhA!<^@|13vQvT!3!DV4VX!N3(YCSUlOkq$2UFPaurgevI=9F5xUcne6 z;!JdX$u^M-mv9f_Jz0Y{wuq^J6JNEc!Ir55-J+~1%tD-3jf z5nT2t2k81jtVimfuK!4&C3Zj+0`W(}aM}YrZBDmF>r4YK`?y**h?*|dc>2o_kqGcnyA4g?P0j7) zc8*TD-fNMqG>dYxA7MIe7lF{ze&K+-OB*l}W^*uS=;GynbA_lHGwB={gNPi&i5w1Y z+$-eF8+pW1`EDcza3|Iyf&L3b&u27Yi>sZ1_bZhYBjeFzoV9ziWzf^4-TVCDTnp^{ z*XS}JB)u`6#k`%;m&!!JHlGMFZzX($CTE{flPWF^)l@QHu zGQG`Xp>lU1;auf+TuAtOJRG(=U6dJW1uR5`QHwhg1rJ}1ZZG;~uEXh%%2H9igS`(^0ty{!@{;Eq= zYe%8*e!V8qgBC5g?Wsim0NE~mq{g`6TQ_fe;NK(iz%53=@4)>mMto_Q?i@Zjyf$$a zrk$%+qYUAgt(bSOn#wvOXd2!^GE99w;Pq< zKiya&$4fO9^4GvBsfZ{aF4X)`N@laFFlcaJU+nOQis)KQb~2>_9@J{cd<%TPkwEOC z%j(o&rug?&>n(Cg)m(jDN1SM3FVA{3qwzyG#x>BW+@u{uo`m&-Att80@;0^{_e{=eq6^NP;WH$oRszc6^tc(@8Gw054eU8Jly+W zvE})C5E|x5Dga%Z#5j#{_{R&oG0EF>SX0gT3)`iiwTJ|Lh%Nx+tEM z%V^X;3+E+(=QVcEc zN(aion`79WF)^L*)7|3>ePU6AMY=d)`pv_PHxnB}u>z|*9az9xCvqA!PsA$TrH*eX zNBXF~V3ph81@aheAP%EJexP7L@onQ4uEUg%%+KKR+cjSB?p&kyZA<~#_+^c7)FBH( zh(`1}t>W6w0Y^4GA@_D` z;}IS8UpI)Q&nbEkY$OmjE$oVuo1`W@6sa4TsRj|YBH}cV7Ij;bNmbQie)B$q3L>N% zQl9P^$%Vs9MN~D+zbDU|tD#;tL| zbvywfS~E_!a;)z=(miT0w~Es-knM@IzM(&J&ByauT|-!{)ruVuZak7GiB@W4K;GL9 zwG#ZwGKdMw(1qgrodUF{iP;S^CbQ^?JnmM#$$axI847|M3DQ}n*&o+%SnR!acva7e z9dGK!#XR(H1{?Sy^YG4mw;|hOvo;xT0OW_T^UY%hg|>uNKDiWxcG_$fs_NiF{hZ&E&q3seCTHVCT*q>teHy$jd{jn6=JR3#X zpi&E>_QaI2#D%Z6NyV+dKH&mkdLD$-HT9pJt1#TY`ZAH4R-sv>7Dq1%S{ID$^B5Bv zuuAokVScLhiNow=P4Ul886c8ZnY1ta5?u^?E2->i__kerA)NCia!(xXS@lO5+gQ{q z=$5n%k1}M|vEo*!BI)Uk`Q>ENd-qzlvk^*AKDXRjZ=AIqX?i1iafdMD?Ljr;P9U7C z$*>!i?N`JNb~4pkM4s)B=^_nouP(`ucity+&z-@7YYiFXEw4WENUY=~O5JgQDC=c9#-%KwLudM%L z8#=BW8C;jxqryz*S|CnYy~ybxo?KVmq#^Eq6rB4sf^1i$(WxOFW#l66clGUHvF68c zj@Go*cie;ac>l{)2P87jCw+`n`%uBb9i_njZj12xPr6y-;fNJ-WLW)R$KwS|>^gaD zF#o>IF>!P+Fv;o8rk#|nwux{5vRYg4z-WZ4O#5gKYkx@j9);HV8`a7-6SVX1TkT6!JL~eag;9RjcS)Cxzne^f=F;VB>F18~V-V!wAlRvI zITfO4RhDgz4<=|(XO3<3kzcX7rq-I<^G3L6@9~ek5;6|-a6i>Ik!ih>D&Cg+I4 zQAux1Zj&6Q{DzgHLbaEkd}rSqnz8^sY}qb%-yA-SKJUThZsxI}A~NK){^@G2k?vtd z%Yc|@C^W|45J;qMcQdNj{8|tIq-$eqeTRtqD+IUhlWd4WYMQQMt5O|MNA1<-4CgE7 zv;QbP@lGB^Ovrqa@vB!z9gH1IVHYVszc!I=ZReUdo-7QUfSs1LKQi6^T#fz}>9U4@b7D?Z~i z)*$z1l&&~6zctR;WS#>2LoMC&ShLYIy!M)C7t}S2sdjcjxLa~HJ=EMIL2Hm-<#%gk z_;!=kf!96k+R6VkQKVc6f?IKBMtX1@t=2d)(B_Oq?Vm-wCC;J$Ar&v(7B`2+?nvCN z)9d+tV=`NyW#P&B>r{wUk?|ZX&IOdt33mF@et&_6I6L{`C=2rRy=kq|SqLpd7>45< z8SI{raux+-JA>nhw^-glms7?*&TWYb4OV6%?U)nqv@&@st_r9Q@}sLfKLwtQR+OLq z+|;T56R2$uZ`5If{rxh8gKSd`p&LUwd=;~K+y;HTh@}%)lGE?rcXI*z6o;M9 zV*D2t7zxmrj+;L0OqrrND+UC{SL-O)GP^Z-lyA8rSeF5d9$9VrxiI(B0BC*GqSe z6s|z%C^ipguCw}#?-5#!$j4Jdj)w)4OUfJ#{86Xt1wz zsnDIb(U&blGAE$TQ=`=?BcOab$z94fK-IX&nUip#-52a!3cD4352CIS2i!S6$0|OJ zeJpu5z86|a7m&!e%VwAARLVFgEG7dgz4PR$d_@QZ6F7Iaize8_Y#Ze}6DQc7Ldy1j z2H$L#eH>P+pHzPSr{oZBc)#+20Ae`ie7{3r_{Odth%y`wqJA0I93cN4Bxhz8;cA(L zWU3o@|Hh(xdjR1^y6D+vLbm6wq&~Kkn>*Z{rh7W!SXvfRCWs z*(JasXFO|XEc4XMVE@nF_D7;!?DJPL{QS(HB)sYU z{lbO4N3J`Y4+D*FTU7TJdz`ZxrV3Fli%%;3K0#LRm{G9#Q`4rVuoXL9@lIe9r#=rP zAis6l{4#Z(y_UqyQ!NIPV%Iz1sS%s>Zt=7X)$_w@- zL732+3dBq1HF&QZ6F=9gL@N`AH7+<&8DJL7 zKAx%|zB%QPwFBq2|556c$#0M+61MpIoXdSQEL(Su|L!)GMS$$3-h{U!{IG;GmX^>H z0~UF^?QyP5+0)oFh0XRycJa(P` z{xr4S?|M_`I0mHScOSAG;u#%ox-k0l^LTu1PD+5~GbGKrSaighN7a!91GnI6B`BaH zS*SMZwhk!|E!S*cRq%EMp#P`Vvlkcy+x!pX0u7(=&^|pMnbT{$zqMq!IKF zgkDx0IRWq!Mt3{(@>w9}RJ%ROVx;utOjiGd` zZ7QAN)v!zxx*(JEqr~OISXz7RPpXeUd3vs zsIg^Lt#zlfox@YXb{nvJndw{GPfIa-r@E%q4=t$c<21>v4U#H7v_2U&yi>kLjl&%e zbC>pSL}(i5`;;z_H1sP1Qo`7;g4-g_NIgvIFD)KH;W(VhzL!X(^QXgQio^OMgB-I4 zDAEJ5kVgPa|Hw}Uw3a?);fg~$C3+=iwY%tRw7I%Dob180I0-EP@AETd(EaDs?~U)F z?Nd!jx2<~~O#x8H**)iHK9R7&!^0(hr&{{H2I(t8k^K!`izrVgOo8>WIONKARE6!( zln!bu7`eKByHa_K4;F39RHy<4u&2t-Kbc6(bFU~oT-Lhci=g+&dAFzse_^7 zM$u-!5qe#I-MRqpyxHm3UO(aTR*GIR-QMj>kMQK4u_Y$M05W8{w-3Bw+cXcEHZ04>!c?T;{G9y9wgAgxoEj@wrkzDJPT5WA6TrI>V}5a#0(d>@#{y=In6zl<*c-{f{<^A{CnsJ3 zH4s17%^aja&=^=-_au>8EXS@qt((A8ScG;_S~QRy46PT4J_d_SvIJHAy~4C|Z=A9v*llc2<8oPq~j9|(?Ze5HL_ ztX-DUem-X?$%A7^n?M3k(fJ<=H2)!+|DqCr*}pXUDiD@V$70C-fysYA{o5copHL#7 z3R2)(r|W+v^+Fn;*cb9J&(QjAoqV3T1}^294fQ1{1RePi{Vued7y8?Q-3t;$ zN^j3Lx+#mGEjs4Nrwa3bhjkK+MxX(c-nvU6=pJA~VxxDQf-%8LrX-2kQ-}D=zlhkT zmW)5Pa2mJGJl@yt!ml4(l`UUXoZXK@Z?7;fhzh3gYkPD|NX*ST76MF?Pz59M;|J4r z_XhmdYsqVK*t15D0X258o<3p@nhEh;CGV#6q||W>jDA2=6#7MizT;Gq>f)2nGra+w zd4u0QpLtc@{_1*AK1<43VKEblD#uIgu|Ict+L_oHX8v~zz25@VS>p*C;R$ix5Z z0lx?pdLMw-c{BCG5tY?s2L@?ZnWw{)(DgzwWIs!l;m~^~e|+!lF^EoX5bFBte6&a4 zsZyM`BuhfXk)OuE%V00{_yFW0RvnLuQG;KySL?no^z=?lr|2CdG%Z^eR1U>nr!UU-h1{W4W{*xgGsmqr61#gkJ_d8A(ruD9poMj+Uoyw~Tx|SeBa2 z@>b>%V%MINHaf)`bPixI!&JNZ4P3{*PA8_{K`j3>zI^<23cr za5hR0Q<)D?1!g&|)(=^|^8l#fqXn@(Bo2?*cw<3^0uAq)-8Hhz<)9^f+#o$_}XKnRj`+Mh``3$4wUM8FEcE0gL#0vZ|*~<^A+;;XB zG&DBTtJxW>N!09NBuzyS-;4%ZaO7>rp^pga;?(gBH_teE@uXk`qT{zan(C+lmqczk zxGauApv#viv6=D5WNfyOCAe*JT#4m0vua%c!HPxEj3ATsKG^$?+BQuqu@aUl|LCpx zh?t@CxC%v%;#V8e-SJr8u`qGLp2BwC*@scq*viRVyS?Z71KCcTOQw5w;Dhl_gC8&O z4??tY9e;>ZEMo>;rFitmQ94A6n>b1j;5FFffac^x-wZQ%2u}{{zC6O^BzWENXYx4^ z3cIKQNMLze;m_T*%`KXOTOa|*-Spn8aq&CHXR|E}%cwc3+(u5X3qEsxu8STGFOb0xGC(FUWs~$`KNo~gVOg+{isO#=#U7g}$N8;My z)^x5`k{#^1|ElbLLzJ;WXF&sEVJxGeZNkL=Qk&xS$57>5QE}wM%*zuMitDv$Lb7T^!_tW<=oo0r`+Go26060_q=-%T@ zf1Bdq;+3(tCDPg=V_OzhVdVIxrMPCc!esmqd|@I|hScYZyFwkhRB>kjhr0t&bi2KM zH2&r&F~6!Z4GP{HJCUM8p}EUB3df$L@x^%f?XSZYf)!M?YPQIbc5Q?G%p&K3?edKu z!rM#W`nE|ttD4)u^N7-duNc0&zpvD+pqnlOgxuu@{+qhVJ-!l~_QL}^EENCwixcf{|;F)eiyC`A$FvE7LS>7|5a6ysIRqvRX;? zgDzy#pxW9+QB)|`b<{7qes0Odf3i~@Ced+MrQ=n;4Kr+i*TcwaAu;)GM9)q&8}pK) zs>7LqImY#Yw%p#=$FqP}(JbJuXUC^2&fhxqhW_?8KPNsYq6u)PH+J&e0U0(qdl4)= ztJPBG`fn{jmd7{$v(0dFrejWu#@eOBj!`v{buH5mEuEJ=^6O~C3cWsJ#Xpi1Ivc%M z*iF8t@wy>WoOG^&rVdd%QzP77ftpf??so)>I7L~%?nIm6AAnQDv=vtbe&J@Z?Bi$N zq!Ng^$~sWdoDXSLlXvSZ1m2+up5OX?7r&6xdH*|9?;oo_OxCC0SCUQS>h^aTzux~m zpY20R!a)7PZr=OAR{DshF;hV|%rJxf`v8~2)ZC&GtiKeZHM{u6ImzuPYWZHvm#m#x zR0q#^p*GzZjs#^MuX<7}*yK|ANNH!jwhEgLSP@xWV@?{Y@TqjV@<}RVoOVMwp_=U- zv~_%-Th?XX=GxH05x$oG0QxoB&MSpxg?QjdME}XHUa_B~yr(aCE|5K`@=F$082o4~ z_O7YphS=19%!60?V&%gRUP4 zq<;&UYIz*St6*b-ma6eA1T{&&itn&h+q_>(zbc1(+@5^4J$L8j7bs01o6r2MNk`D(LK$a>wPvD!#=bx-181Cv&F zn9}wBz<9YuSzM)`P;Fw`@VCxO_gLaxq4Q_AB4mtek&R5GvR9f(Id2Y{c-(uiQ)!cb zrCf}v<>!1{P(<$Cr0X13yFWQNcXFxjf5zANgY9e6Qst4E$Fs2^&->|?NqW(u=U;+! zF?2FvqPJ_$r!j*LbMWoo)R(w)Is!gf*Nwt!tkMlU;Mrws@7ymPS?>KRa|LS{b5FJw zc5hYtZb6@Z%cPx;_*O80-?UWzu&OnkKs3m9rTgc-X;S6S%b?X;WhLd^7`{}(()aLkNCp8KfmWM4a56wQ|DOF8Dl-q?&z3+f|(qh4|T= zv5^cFTpkhe#)49X!2RyuI{6AuGkMIS+RwJ43rlzQWv+x~#~LQb#S@=ua#GaSk5oCi zXsf>+Cq@E>KCJ$!8@ew^`MwY&{nvW6#+X+JkjKOKPKN*ZT@)u{>C{INPFF$j{X%?f$#Yci*;qU zS0hz(8|8(_Ef$xB4knaW*R!$vg*pXplkd%ZRkO>?KhqKtKc|WLhMgb69RwziusHq? z!}yQt!kf>1hQH3+X)TRza9_D5dL6hXWu9|41hTT^vy5hMly=l}M^UTwt5ezVnr7G$ z4~X5v)mk!}n#0MHn}DoqK(Np@CwoX|n9ku;e6^Z+)M905AYIzY?r;p0J*{85z>naN zcURlVuTTP28@A6pF#-U%P@4ke>(GGmeyjI9v^R}Sg=R-&P?c=;ybIE3as7nC9WL!G zT}3bZcq#drsg(-!`dVy>{Nzf2Cg~U|1`?)UhySrkQW| z^;H?D$1TE>*GF9s*m6id$y8}%4Fd@Ebk7xdrL$YI;ohG?&G?0ZQ}HpKd-7K3nvNb`KIcYAFaUt4ve*OA9*VEWn1#B9y-uzpGF z6wky;<4dm2>h_Urn)gUT0d^EJt7+xhYP`%R8=Npn5S6OZN*ziMg}N-(!x$HjcHm2D zLboI`ptSFVhIGK*jxk+=o;p|Swtz{;L8r=%z;aPS9`SoG+jL&PYM9F@s1)zw+vXE5 zdV}p9$;*EB)@IeLGgKZMjTPelV)~HWecYL#X)#8g9K&ihUT`nsClm_zj%UrQ;n*?t~@5? zvTTPU8{@aR>%SBzFFN_VH2e;cUx=d;*&li)XIRD6?sz<)yzG+9kC@6hNB?oPy+D(G z<{ItU*TVcsWYsg)P>NJJAHm5W3WO{~!0o77dBCSfP7QlG+sw%gh^t#4v0wZ-(EI}? zHCOet;}|=;6JS?m(?&h$L1~Y&pBef_4B>K)eg3-#e2Yn5>A@-H3M^+1GPwCi`Iog!_Qj~I@|bepmaC<9AEjDRMQ8p{=Eo*?)T2(1!9IDfk0 z6bM{Dni~*d|ADAu%)B`wO3dCuPZ<9ppFe<32{dO^3;sK~4;XK&8KxdkW~ZfK0y9fO z;~Ry_lf=pweJO&D+`FW%xDn0GAG%5rTHSh5wpzV4RZ}(wBspAS%9{R z%t5wFJRe%eWr9AQk`YFDx-8p1{Fk_k$OM9fvsM=N->8oEopmT^mpU;!=s7^yq0t-v zL+znqx&zWQ7BPUx$pxr9tUlshGpysS(eeMAezH|S80G-MhZf<9{_n?Vlhp8lF3o5G zB0ePb{eaxsXeRg5-PPvF$NvnOmQVnlXV)!n4baKb4q+0iDn3#{SxylXJ_sh-y_)>x zMTqlu*AUQ!=FoOV3z$t8uTkmg#SIw{~!?+ z@Bt2U<_|;Oe?0r=&tMW-!k39^LLVN>kVyagWCi#L!^?jro_izsGFyQ_BtpO?;K^47 z8w)BIwS3Zl2vE#Sz$_oGu@JNgYM2XxT#&Z*?E$4W-&6GeKii{Co*$jM0QVn<#->C+ zbP4vrc_c~eg!5;0uiopHnp_a@NN}!9YZKkXf0_3`XyJqxz_CinZ+W@@a*h9G3|0wR z4SI-s7fHm+uzz2GJ_UfH0XU8%oy4V^#@j?{Z$g*V8_Ji7WBtMatIPiH82EuPbR!Ao zBxxbo)ow<@zTolrJU}~k}E)?w7(uHGx=S%0*pH3|Ime| zen1Qx92X?A!}6D;O$I1*O&Bx=ALBQDknQ8t&K_q@0B0fp;4JwOW(J>wH6Ze-2DJ71 zkwi@Uh^o%U$vpWbh|lV7?_~<7p(CI&o@;b6%oKF{&hWn3)hrW`_Rf~+j(hJfm|2aa zGng=_1@!q$H1wan&G+U~Dc%K*l zR~?9y!V|_LT6JHOcuF-YLqDtsmJ}!GP)|y>=8J&b=YPBBKh4ysUn@wC*ye%5uNYkC?4AUfLSZIl8Nm z#xT}D09wrZ-vPnRpf8Ga#fIpLXt7MUYK(D;sgywUOvA9ny+A}g-8T|!wVOv^H8yJs z2nEa4bHpkGsfH(%vy__=%_qa`PB$0FT^sS_qp z?+U{oCZ$@eBENzjcoXklT>~0s=Nx~e{<0Xkh);}a_3W{$>1+d4s~Uxj>IBPpD9 z-62?pz%>LTeJKA&cD?Fc;6}_Cu%0$-tn(S3$xz)=ts$$vWMf_Ur2O$(UxjQmS!HZ! z=hf7?*v%j5rZHn;Yy%Y6WEDPK>qS+}4FHq_{no+oSc$H~q2OzLjo-5U6zXpPc<7ft-_mmi$oHsg)M$?V$-4-&H< zb@vd`5fWiU6eb9=$^K+qZ4)9&JdCBN`0E}LwD{N?fYkB-S>Gm!*_nFs`9+e|uVk=8 zLxG}baU;@#P zQKCTkA5Z`l7H~D9eC;idy$s0yBIn(IAq9!~_dZhY7~fwGL4uY9v#o=L3eany%o#DK zzMyRm*zF=k`EA!mDM0_HS=c?qY{n?dv`XH`<^5e6*S|~i6^+98^NbrEUnXEl!T;?Q zLO|mmb&5^&*XUS(jgA8teXltoL$%Gz#m=O^r9x(lm#OhA)ToA=XPKn44MQTHBDQLW9|ARtG;Kr|saM?p|Q z$(aTuND#?Lkeo9}6om!>Ns=U}t&&j$n+z%vB?w9mN{|eaGfZvJd(OT0`@fm_|Cw3; zti|$pqN(hRDfxi)n<4+^ zBM!3wa-_uWh;cpiS5qnhgM*!-c<00qz>q`@bfT4cxEBE$LhRHGG%PB{2ls~nEz#Iy zcR(;ZVTTCjzy${;LKMd*fMi}1&vxvzW*g?EL2Zg)HRIoQ0xf6uaPIJAF9B>FSGI4IqcqClcBSDdhFSi#5o0GI77 z7697|Vb(g5-!Vf%0QNBWufs<`{~m!wM-@F_Lp~6y`8~N9=X#0&kOMIaw(QZL1g7CQ z1U^GcCmuL#>G(YjT;f(P7vO-Nc%*OuFaO)a34NL2&3a{yaj0VR^#7E3;e0tIlyJY# z7%Y7IpV;_+dkbI3yz;@n=yw>O!NdJS-X3Y*vtBz|!cy-XT7K;Y3KwPbRX@ znQ02j)R6{|1}%FP5+dO;G)O_0*<-Y1WlZfg)IgE`eK+9oRw5va8I^W_zib z*LOXKxS~%7u+jF;a@#1aQRhqL)In%lR0|awul@Q+qBmUSd(d?W8Asozvqn$Q4WE-PrA`i&x#ei( z-ujZ}J&If#{r7s2RES2{y@<-8;!F0;JZor?dX?(8HN~pOn6&a#I;1a# zMadTM7*CFG&+U-j)ovq)k0B&j;1+rTW!gbYTq$f(bD772J?z)pRACQYXnCA%WJ@n% zkbW?G$HIM-1CV@6nrMHo`fFp!gfHD!6L<`u2htak8~5f5*q)y)Davz~_W<1KF~`#0 zK8L#m5WlR7>awNrn-a{et9;aiCBxxE;evI{eBZ}L=6#y~PLd0uTbY8wO4S&Ew>xyu zB4FMY5(#WLPgz6N#>jDPd!8@-^DB>7Fpg#nyZma37P?)khUf(?2jiH z&Gwi^6$3zTOJ#a0;xHuQGxZ|j385G?Wg!^^BHrjlFeLsCWIzT+O&Ixva`CkuQ~M9b z(BktBAbTG`w%aGE1Y9SopwZG^>NZGG1x)Ks%s{4fLAR&@eee7g>kj%Q(wmC3tf6%-?qOF z9eeWEd93JyaZuO5!N$389QmEbk5Pd;ta|!WN|NQ*#%NUe^#YlqNJ#32(3a?xocyvT z=+!lWe)96|>q*}MS@%Of_Gt}F6rK1GtI8A0w|82c?Vx?v_=Qq}dy<8DBTECp$@6AN z$JS|fwVKpRV84lsyO3DFX^dqnUv9lNK|Ry)BD~nPx5Ez>B|Kj;`lLr&&K{KV_KcG9 z(7c-~b^n-s5Q8qJ{(O{7DJzv`VZQXcd1z=`gSOHI){dF(0*81mZTF;YXien&n5WZ? zQTnb8=}^QvDAwOSDPi|%!v#nO6a$L`{~1Ee8Qi8QjplSox|0w-r4PbqycB+1)*c-0 zR8-f~+)~93mJSvX@s02IPAuk-uihgO>oI>|YTm&zqV<5b@}<6%E~U?|a=Q5XfXf7cJh}+7>RRRj=o?@? zxq@|R6YWM?#w>hP^fe7?{>%LdEGb^_Kk9oKf;Y)Q@YXs;*1l!mvLsvhtqL zO#u-DTeK zwK;9bk&>6vOZnb-ruStowyT7Ti7r(EX)w-%c2RqBDgU!g{b7J&Lvf^j$}20Z>)cTObu4lG$0#bn)^~ROKf6r0 zr1js~>aYzJa}4;d4^DKU<>{ggKC@V}%p+~e*e}WWjrw3G`9V)*K=EZ27Vg&yM|UxKh}CijRT6@i3keJf~aT7|d;W zPT0fe6h_u=)6AGslC@F^ZBJ`ozCiq2N`ZLX1_W5ojeF1X;M~G2xZtznBoTvuuzZ`zK8%kkIia<+v0qQY= z_HXDbkxaS)ZcN`z*^u}jcn@NwU&l|~+7v7h^Z$_wppM{y@#Xce+(_YYUlZh9)FXy zi-YNFi5oz{*uqPywvX8mguZ<3&GW<^QjDE5x1#rkMvu%^-q@{-2)p@v8}Z3{V4j5? zU;?LAMCT|xSJY|yKIpH|m6A{=^=#5}aJ(sgZsnt1^Xk3(tOAoGf=?7!etBlE(-?qz zT zSOmIJ2v(CPc=bzOw>-Jl2@cOf}PKyU>BpVIv(vBI|+6SDkMba#IR(m(@WQePaGcB|jU z4+1I}j^Q@_wXqj7GvwTSLBO&-U;9((Je%(Po5j{zY+-_SeP$4V-vAQpgH~zt<^*03MsGl__7tCx@ebF$ zT}!t)UOh1Zt9;icw*47BWAZ7ibd6JRp(`&1?Dcc}C!~RpRXN!VSmU=yNGBHCFt%yW zKxW6-%nwf+C9Ocu!3|?is zo@ku>gdIq^Z<#kfYSm^6NidFruN}5yq|aplYt5PXU?m9F>hPbe70I=08c#S09!$5{ zT%NGtUVbDYKArhopLGWUh@dYo0-&1`&vK~O7HdBievC2#O4lYwT1o1>fbC@A0^pQF zu$=PGrG;J7Sh&gg3TUk^Ckg~o-vSzNH$rH=GqY4_$iI=VH#N!AC6hL^wnCGeR&s%u zs?Zd(UXf)qgqM~)|86sBGh6aGfUE8l$Fdyhlhxmc|DZeqYu5hHYliG8FqU$}9-r9^ z-0z$I;bE9$k#IZTu(FJzjl3K22mqQuR9d~Wq_n-gP;KDP>7gy4_hKdbVdd5rj0E7h z0dFU*XQ(2j*IcF$mAYxkAbhu&rHDnv4akkoSR&6~pA3dSp@wOwp*ZXEg8^ z-}&{?VDoN;d{HW=Rn+YK9>kaue((4;o74(o^9?9|H1(eyx!>#sPE;Lycx+I-${~Yw znh=Fack9x5sV(t|sg=o84)ZE#z0>2wLd9}qfBW1YqSvh#G6;{9GAp{H<{#=|HUNIH zV%utI;S_K6P9YaXf@M@ikg`q3DA|;5PX3jjeW(3?eShEeT%M$RJxY9Ul79da4wr7* z-oh6Y-#9%AG_zv9oPk>?hMq{fS_L+e9^t{(6WuULm$Q;A@q^LcX(st6_+$Oi=X-XsIv~96 z$Tmo~TN*BN<>U?aGrgFZma=Imf;Vrg}YgS2A0yI{dRz~NhVJa{4bcXwLOW9{`* z+7+JUoG=V&ny~ns_#@bLg{-+13Ei1=SFAN(3iJi5P}R!9hfowBYGXJ{V#c_|H1%0eBXCbJ)Ya^S}3-neQUV%A2tmo>wmIM z66bQq*%I`f(KJrAWC9kgPf9)s!72DR{FKBwYO*OZ(~;9NV2%HVWx~7dcr%)|;1=7z zXY>f~!L$X-H~HOXZszM>i>>@Wy{J8-=;MEL(IGt=p@hFaeD{>~zWHXaQ&qfF91bZL z=_p}1S?J1Z&tt-e#(Ejw*;{q%7i)c=k+|>K_(%wy6 zoB#<@!ZGF!^I@2;MMc6c`RkOx;XjMuFC{DRYP}08rsW%4D{x$~A9Nx4oO+c})I;vc z?>x1E0X*R3Fd$WILOCG?4?LuJ3v)>Nfz(LSJT+P$g$1Ib2cbM5 zxcPs!xu8>63;aQJjws-PG8&i9d^fA^ZeseBCrD9(cxGfBEu#<5;{9NdDE0BYd-|qQ zDZ$n{kn6|?2OxbmU58AN0%wF0Pid|7x9ss?Zx#6p#!8*Hce--4d2`^u>T@q$2r*Q+ zv}%8@R#WHnV^QnYg>l-+VV8q_WQZWNh01e&)s{6vYs|r@saByGZl9$_HanO-Lz%SV zx4YE~%mQ^hM3nx%Vw=Hz$VrC~_wNHUqXu+S5s)EGB5EcZcB)av5n7GhyA3>g#~Nb? zhgF5=E>rQD1d9T_!Q*3CXr@pOGnKfVV1NY7WG*xf=Xg(V<1A-%$CmP-ONj-m~FyLxEh`>O?)av&w%eKp#bIrSxXEs z`)T-@u1RQ|b%}l4K-7BMBifG88UO(cwRyp1{6FAQZqM87mABwiX8Ajn5_sI6>Gv*h zYGwBT#Wmme8Bpv|5?Wh_%GH~!LF7GbX`aspy=U^~7oe8r&gcUyA|EgZxG)fGoE4AJ zPRL{aNwmUniPQ79f0Ha9U?zx72Sb;a)tJeHV}Ze>-*H(O&Ot^?ioyfGL!QyW!YKM4 zvGR**WlFwyO)p81LhMfvNc1s9LAOoZ;*3Ht2 z!XfgFFY+HBoHQ`fHuQS2y2#1^op(HBWQ#Lp2M5$psNsTpE~H? zZ^zTj^Wzuzw;qr}So&_QG%|O%2=T5~HW4yBx&`i}b_pX07T| zigYr0yPOjciTvnkAWZ~gQQgVEfZ9WV@uf+9mzFCkr#xo5F36j}sFv?JafTuMW8_~s zwWGA=^NS~&N>o#Yz7+%D(H?MP?ufrOTxcGS49vR;1Y|4?%pt6WAo6Ot8)a@q=hE%*jp(kVt?!V`d?_clgJus@QCiH1z0M7zwJnQS_W zU+Pm;TZOpgaYRvfzETym<9xNW1}s1~GQ$aLr5{2}!;kyqvQ>-~MTyt)&r)Y^W&FwW z)>KLu|E;N4_P*r;uZ1Tw?&(t*7%>{y^dN(!%Euq;U7z6KDmaE?C*=Xhx!eXJO6r$8 z>OVd_g@**^?B?MbhNBlEWzY8`vPh=%T5zH{P@Vq@W9Bl{gdVd04-2UWfXH+w2Iq$%KD~8wu9K0-0iW+dWk?^D2v2~CA8|q zrad}`B7&$qlp%R!d$~>A4pM0%Txqu7H&6ifZP@sQ7|tWoNMA_QDGoQuM`|5SOE%$s zwrJ?2FC>=b>(U2&3f#*YGkE_sX5ajiz{bcxT)oQ-iaLq?U}=ty=byZ?)L+v& z_Y1!m-oTO3(YQ{Xz_U|upn{v9FtzRqr~H10KXP^PC#A;@umOG)vV1Y33RFTYZ}}`g z$bHf%W`vjrx`8F!+lFLwY{8M9cEj+OaLs~5h97~e|1=cnKbNln+Uf_UerNd8AMikW z5eiuQ#BeRP*!khc->7C&2_nkQc*{4Sp*D!r=4r*bT36s3A~FQ+zvSWZcEnp5E0_7p zcOGJuF*?}brKl^~z@9J3fmtlog}+@^3GgudYZ)B@EqPRrq z@)u~CfM-Y4b7AQ?>oprXpM)-HGK%ttPOf9#K7F!JpK*xAP~32R^5)nfSWPWLIz_UG znyVtuqk-@RN=L4;T#i%yUE>L-?>Q(#s z+ME7#`3>~c!C}Q{R=5(wn*dC$gFQNteWyKl_9eBdB2hH$g)T0w?6LG?ho6Cie-|Rc zCjyaVMEQygY~?f*61+4Yxc8C%1)KuvKfv&pDjdW^l!hV(Lv=3cnAs;pqVmaq9;ZiW zIM?|22XrZ%YN!X&l*6H3XZar>tnqmwG+a&UVh7P7Q2g$X-@r(E^5Aql%10c+y2wA@ z`$*wHGN4A9F(7)%{#Y?cP4=PU|KD6fa1ahv{r&$%0QtWTLH|Fv!vBLP7wJ2IL|`Wh zM+g5I;Bi7u+@?WD7QVg-hlIii(EooV6gWwT%8S2n1tO2%Pm!`?{`Zjo|MxGb@j)`8 z|6zPM*ttt!h$Nt({jb5};FnapeTxE@hdX1y5K7 z>B`>$j{i@BWlMebnBe_B&|vwm|8*0iI@!}urMHo`PnN6$^r~I;?%LaIt(?9q-TJ05 z*ilw$>jlMckKhgtnt=zrO$IOpe}Bx0TvT`aZ%%mJCaxdrfCa9qZ?^ZX-PX#~8dP5N zU^p29rJ-DN$;h`8>8buZLAwa9E2L5HM-d1nh8;L~JKjn&JMp|9!a5k0L%`qF_>a`{ z6!<$r@O>Z=Af1w2y*}OZNwmG1w^JQ+Jf*hO5A#wbRZ(uxpZd$8|NX7`!G{+!II1@& zBiWmAUzqDt4Y}Rtz^x*6X>o-=soah@a&>UN|Mlv+nBzvkg3dv$`NZxWXVdz~V$cz8 zL=j%%y74R+%HSPM3B2i#_JEsl0O4z4tg6^GhTZY5Mj%pi0h+Ht+4rZrwUk(Q8UYj; zs%U_}74YBN30_IcR3WE4i!Ma_(LW+af*%3zB{{-iq=D#(-|^UQ14nj!xaMTHk}T4u z-2MF*Dr;w%7lv8979>+yX1BCM+N0=4hrr?bnZo=!#|mGnWz_R*K_Dx6h+1ad%J0LTH+B5h^t z&5Macw@hTk+X0h({SwA^Ves#gj7HEb)CgY#;d=`hG6MlbSNIO8*A~x%)Zwi(@wHlH zOf52e+7@PrvM#cJ*#dEJ|LqW(J5b-QMD&?Ji;s6Tz<}v-0kZ~DgfF!ls^~?CZ7vWk zh(hU18{al~So-fSMD+?nq&l`pbpa&QX zw8)4cq90fY$l-J&UtLAS&Pa_@$iIPCRb(-EIW`hR6yOaGBbV0Vs2xG(Q<{Z^Bh}(Wy$a7aDFG!= zK^{mH+Z>}n^6Y^gq{=@r3X{0|wEjXBA$)3NJ=}ims|7+fY5G(o__q#O%pxbX;{X@# zsro>lIOBVk=yMB;KzQ{ku%(W5y;ZUm1eq8SC7Kq^UHMbF*OW*atxZ%~PwVsf=!tX2 zfIOazT;HkX@$=wp%sb0kSD+Bl; zbxw7Z7A3c!Q6OF!*o#yqVxej%v$}tgJzIKi0-n|g6X3|ZPp%$6@UrTMpja86om0Ns zl$RG>xlB5K7JfW2S`Yn-h=R2Tkq{JAEQM{C%e*a&Y6{?V*=`kHFf%i01NlYlq%{!W z&4!%E^B-p8_kf-w{tK4>i{PMZEJYy9!-F|}=Ht^!jNIz^Jnyq}H(`2k0vTV^yMzmR zT_EqCENmrjh-R8eb|u<`+d(F@50))0j@I|rqA3i1YKO3rDJJmD)uj|`Zb4tf6?4}p zq3O&)n8X@>@hg4+Ct1&j&rkwC&-{e`f0oks#oEV7#)5Rv3`uP;I@%5ZrCb^WDMwfn&fuH8*3K;#J zA0{bf5PvjG7*VP@fX{5C;%$8Z_&y3}e15ZD$e^GWLefO1A?El~JPuyQ-S-4^uFrIW z__rQ?*AfufTHxf{Tj^AVN#TYrI+o_SZkd<$f(ooUU*>ZgBzd5jiNxY<f>su6S&ohx$i#T zJtj5wh^|&Eo+#Sc$@iu6JfdmYl)D?3dh;)-E@}S+N3A=CAv@1z3m-Grl24U7Z6U?- zwS7dX@{L01a@DOhs}o_uo9tVm+L)YKqeV)qf~z-etqUbbH6F?mn|BtwO7J~HGe_Jw zjgQn9?!vyIqbMw(Wo}ay$&1px$_%0b7)CL5h=#hN9nC|SW}@Q1#NE>LdflZZzn6ca zJUG6v&zhnD#su80dq~+@KCNM5ID`F`;LhFM=aBbTf~EG9uPdv;vm53=nuh9qEO#sX zUb{uWcLX2>J8RRrhfXZpWs$^0x6xNCO~qbG&tT)7T6K@+3tU65K*h4ec?qfS&THMt z%RUmpWf+~Gjmy^_B@-O4xD&6SEk0Q0*5Bfab|o6J$uGTw1tv=zxEcc=nx=sDsm^-y zeW)%BTx~2MpAI{_sAYaHpQc$1ZSyI$a7d_#S*n?p2Sj4FUgil3+IQvpa*`$DnUg)+ zOHK>xknm{|Rmj{0igENL45X6iqX1rQP8tV?l#mxyTv!P#AEN^x~u9jl{R#ZBbq%$MNwlM8i@AqWC16N>>=7 zd@i&SAC9Hi4Cm3f`(4bzgdu!XviY6F(_^(3)xarp+ClM!{zO z+D%mXp(Udk;7|J?HnwdbAZMF8i^T)8@lFvWW4?HA&3ammmQR+n`yI)qR--Iueatws z_Dw(+(;Sr@G}(_Wa7i*zGmVJb+TQPdNp?1?>3~OBv2?e6+{rRqE&Rh za&ck5lL*(lipS+Sl`<*(Sxv!;+_LT-+7EQhdzyx(D|^WU`*uZm84ZGzS*D?G?^2#> z)5&j_|I)^~^U^9DMpG5^Y}pI4Eb79(_{Ej3%jo`TN;b%Zc(b)oj>DiiFJk{ z@zQzyXV}o?2X~IP!k-nMJdXaaCnw_U#-6f?YjE@xsL<0c#EOdz^%q7oRHPn!j-&pMtg! z8=`=weV_?r8Q;j1)jgqMvpe@HqOXWmKc%XXI6^rVhqE)vJpF3sDI?nox^XUA!LK6 z@nK3Z_kn~E6k}f9OIV*h2StmCbah5i##W~Co5!LGBYP{EM9!xw7HaKL2Q$6OwI!YX zc=RCeXh;Pnp!CuRd=ZUt8@zvka(>EH3qHIum7Gyj*oo)zG15vEPBwV5gPOGY&Ohm? zLs#fMajB3DbOVB67z~#L7ab!HNfVGDJ9N14W+sNOj0P!ptOo5|e~O$}5MeZ!2`HWy z;w^u=U9a++?`lFYx%mRW0AX+n&Vm97cK?I1D43Dqu@|T||M4-&g8`SE9+&)= z2;SYJ$x&!|^vZ3o-Ot#D8Y!qnedWsh)pxPN-5QD<_SLWA?TzIyrj0m543YaqM{IT% z&-2QYI8aP^EDXg%oEQF7bK|B+6h5lPg~%~rr1Z^G3+C@L__yZgXGCju=O!+-iRKNioB`pd`hje? z>6vW!*PkTNWT}-d8%1jXGUKO11-nb*N~_D0U!H&S>0eD+DRCOjUJr;me?>9hsG3L6 ze&7)Rd`0`=s=Og=gvVV|Y4ILQ06Bg4#i1o9Na0k456wbqH(i5kcb0}cq{6H@H}hi7 z@tI_Lw3}M^U4M7kVMyP6(P`3O`0_rZBjx*f=OzgGUr_oD8JGKTV0%*ESM=04C7 z$Ei`!hA}E}WRR)0V+zZDNO|c_=1Iau6(2{XbTQv{;fc01(VSUJf^5MW+J$S~LofEW zR&Cr)4oyX_cP+#AzvjUuR?DRl_Ey0gQVF1{_wos5s`5piNZ>dG2jh&Zx-jHE*>#yv zv`q?@8OL5MN=z%l4&}MyTW%5(0T`83T~KCm!I!0I2AvL~!CTr`998@b6*dVXasK(414;`C_WdZH#<7etDCA_!0Zq)eX+S@|(u?P_ES?G)?BQCjPBA71m^ATvP z!~1bP0Fvmy?eB9NCoa>uDuub`^~#wHYRi^A9~{})Tvo@RqBBzGqQBLh-vxDw!v5Ya zxJGXHZew0^;$B0uaE6yququ$Tg22oI$_z0z55oTU1?u;>DXoFMcHqP|)!hmI@BIlsk``>;$Y^Pkbt_4xPQd-4@R z9<`;5`#(~J5^@|EJ%^E_aW%f#KH2f2SOTI)0axIBnY5IAu_6EU7v8C9v%%F_f{z3Rn|9Fo9}^R7h!!7IZ#T^!Gw?gn&qO}FR9H<~2OIqu zdV3WrJQwk28W;xbWQF+GfyYtrmdKrX+t`pSoxya8n2x^CDZ`a^orEJ6GV4|2l%4n5 zaqm5*zjA0sN4lGRPQ1|@Vp-n?UlF9K<7rmuL|;A(Ra0X|QAp_Acje3j_iDrlNT(7> z+0`-vthFH{3Nr7w@fV&7>3$;q`#^M9;1L4O>Y^hQWDUQb+ot{7oAkMlB2mtL?t0F= zrONfH?mRI==#ujs&0@pIGkvNgS3>zXBagYhE1IqiIHnhb6|3GD>h}*CP<$k$mJ|)k3zPY0~t%hT2kp3O}p;fu?G5w;c?^ z%B&6gE(-D`yXeFwyD_HwM|}MH?=Agop?d#u0m|Hkkl`CM`ZmRJSX;gS@iBzO`fFxW zEo9~9tJB{k{WjFa&Gi&&QV_S;^2NL#Rnv#h-jPQm&gI!=k+I3{6_3K6p1C!FB>P)W zP##5k5^8?H;snLmfY>mgK=la(FD<$h%iSJQ}tZV2ezTmtdTsv zv~FhPH3QX(prBBiC+QA3W7Rg=_mvydT$*Q`{OW^pF|o8t-D_%uO$AOPUE z?YUSiNmVP-mBDoUVR1J*S-k})0pfx9$?8}K&Za_pKA-oJywx=UN9o73GaYZzXjoQB zo8kOwQGLBlA$%Rjcr)ncgM6(=U26E% z-;81U7!~7U>lqT`yAf3AdtI$(ex5dF*gL!D3k?NnPcF$!!j@KN-434e)B{rK3MuVu3n-_%deFhp>gS zq=)}kr5MmAu^jyhCD``p3@QNs;D<5Q_xIoMWxf`E6=U0(D?InOTlGTm`0aEYSs@=d zRXE5`3F1I`jw-sq#dw%%oIMhrVi<(+!GjK85mCWVT|vL)kqh3qcJOiu8CW38Gp3sD zSRA4rrkaS+9>D=+P@4XcdmZA-XCTz&OaKBEmHb zS3V&uJf|4ux_rVa@NlvCL1f=??$f|s*Ju!_VSKF3D(ZQW?TKL=6p_}2n0W9L=DLRb zO^GxEg_TLszB?2qv^!gJW54{W&c47KmK zYgQA0%i|y_Th<)ik#Q0i-e?f?pLRPT-1WI6aWp(=Pu@VlftsWA9R$GcbUaYUl%HaY zxGzyNxp{w&=G-}y`%`!%7VCeW(^(3~PdJ|%9)ugbyapRa!iYn5B#>Bg&ibiJ^t@!? zn!5JGG^0~fb%9{AcWN*FHrW~uy!>++=XxER7}Bo8GvvTa4wXDJ_)RL6aZugheN5|l z(Q5XNh>wcV_Mby7es5HO2EIO@1YhBgIBAh2FdudN6~?;%yKk|TeaLlRNVe=nSrAL= z6>q;A7^;wX#=%6gwm*0Dim=2az{Kn$M~EjsSjG#&U}h72mam&>u1aoyDGAxE>^wsW z5_8Fx@iC=I{yxhpWQV(;LA+>&SS{IWHsn1Q{<_zy8yWCrIh=9uDh77pq36-xNAap` z&FLejn5a~))y4uU=a!|vOeX!*K1*#2^H#2nH_Ua^Uh?1H{6m?CluuioJ^rkUe|q?_ zZBcmYOOywnS~An`bEJJj?V+t@oT|M(H=VW(^?W-C?-Tlc!#$1c6N%^N*WXqCNEKIY z>Z6Uo3`V_@S?C`BD6TY8l*FRa5fxL!adZlI{AImg zu{!Gp-j$VN;~goXp-hadxl*ZA^`%0(MHWh6E90pBHMeX)C__hj{0QCi=XLtA2UYy4qpI(+qUu zoHEh#gec*O5-E&uCtK8obXM)Cxp-aE$H~9?L=*aEG}h*SUN2u2pgzran{4{j)ch1x z(Uv$KY^MA+miN*N)cW}3+qpwGYR~XJ3^!~@t|q0saC6(;c{l1K`8iU0j^$G~C}&qI z^&0LQ2`I#@=)4#jTjhW2?#Izp_=-ANTt&#Pnq75aDDJ{NVf6lPKl$9f9`&c)=M<94 zL#>yyl$Ff)XKyUc&r!v{$rUhrSF8M3J(4Gtk<=Wwz^J*hx()RN!@=U8+U0Ef>4ev> zHY4@IIggmPiua5uXPj6xP)V)##&_mdmnOGPH}1-6aCBwADr-t7DHfDd@`U{Bla9j9>8lLYvTBQ`9WI>xi8_zn)j&yQ=t1CY;s0{ABD2 z7S1M{eTzanPwi5#=}ei(^IFF)Te59h`)AF27|);EwEM=5rW6b`<1*4vvA>1k-+M=} z(<5k_h+pBG)3>8jtq{X*cjt6e`B~NGer?|IAFIu~%7BzIb_N$N`pg{&T*9sj#?i{v+U zCs*zePNQPpIZ|@0uM(okFO=m@>&nNNHSnpwd;BG!ldu3CkPd%`_mqgpYVDP42+ zk|f_i&$<$k=dz-5{@0Y9;fcP*?jgISX3SntHF|^YMJ(r$R|>PWjb`QveKGHhIlj>u ziuy2KvL1`y(&%aqNpk63T^R4IFLZO1FLjy@mD#URZSub)*KvCDa%j|rf_lM%zh*SD zj>pB>eJpE8UG_&|8lsu&EmkqL+LiG+ri#ob*RUJ?*}V*FRz5}&t)b^E5U-_IwPvL$ zUvs)m!iYdI8q({4Iktm??syFsPWp;K>w+#D+s(G2<+c%M{c)p%V|;X4Ii5Wp6>X++&qUF9BCA)*KJ^+;PT99u;*_?gnTb9AEGfB7 z>lQv1KP>SSrgeX|9PMno>B@qs*4<8lBYGj0%OZ9~wwWV(wasBWF4OAm{5l}vyHlDm zSa&XtL8px5bBSAC#8Z1SC%x{@By(HexLKv^qL2RiZE&Xf9@p7;l<)3NlD=q}Jb!#P z`NREX`E_&e)uGV zn!TQc0zEk^gE9*7*u;TKN5wbU1O8#N=R1|s#J9AwPo`bqO%sVLJ*k9BUF2eRE}ATS ztwRa77QDf-qb9Is$0Ddg>#K?pFZETv(|v*a!sFR^{4D+k^c9!!sMX_`&1a(1v_7v$ zwB9T!#c94Hp5vygRm!|I6V=}OmaQY!tbyU}%;{)W7mv)wl#}3Q2CG%L#4qYBetczh znqq(3;}b3K^*cNKRsrB(-^xUdv)OoGR*$9mi>fH$is<>KWrB6g`=SW)D|H0#cSOs5 zw{?OUfza8=S*Q<)2P(wlbb(ovQ`PjuNt`~6=j zH!fAL^mqtouFy#3*skA>5!z?t+&)hBaP%{L``pKH1)R`I|Hy^d6lSZgf$3zY7h*AwmDY1UFTc1~9vAun_x3xBr3}(b zjk=^mRo)B~%@&V3QVFs?U&7CFm%s?76aZRxY=F5s+j;?wwdu{5?!eeqP z@a2sSjA+b^sy55bir}TK`_(d0t8`S?jp=P7ngY!2#1n$8_!XYArZP)5tw{RaRv!uf zvCJZyA2jVzUcY{$dz)oTZEpxGJ-zcEpPx_+PzO%WSg;#C)W(iPKK!4l~-XwH{Z-@LCxtl$rN1J7L7n32n&<5g>^G7>s~5?7xSix53^}_;A92S{gHIuiDgBh);;N6o zY2s^C@}?=rK#>wD)%D0CUBC8x6hZhUJ?-oh*NCsLJmlJpUJ+W7_(b*%-+y74A#4aK~e`9|xoDGJ!rQ?Z=pkNZX+D=p1##)*ro&K7K` zm(nu*MWNj{Yytva(Tn^0$1V7$l26JbK23Cbr!t4IWQ1~((Ib zPR=Yh{$=7&inubtCvSAnuKIdz?bla(M|imuqF)*?n>M!LCw$1(!WaI3$-^?tzn-g#Q5z&tJXU*y4^P2Bewnp``5ry3tcW0 z@8A$@;RieP&VT&Y20`h@x@`IEqSWd45wY??tXGCAkE>}NxsWgqmSB|oBzx?&=;Pu7 z)@z(yH{wrRtvMQh=GM#Gys?qyq}3{PoNq?hj`6Qg_m1v!o+f9}qEHpTn=BW0d4`jv ze_dlD!*^}x$Y@tiRCd)1@!7?~(^E{xS=83M^2Lk36O{V=z?eL!#+`HED-yF?P_mWE zItA&8Q)1QVGZ{1|Wb|)*F2JD)d$C^~ao}BR}&A;B*!) zSzrqui^f(9o|;OozQtSZQFpPJwY-%iCl2Ls*(IF0`LT9tVMgyOYdl<*?bbUTXR+D8 z=u%M8U$=fn-=|Q-rWa(bwvzUylRSA@_%@~5fp<{@a|4>&=uOL?svdW1S(z?<@iUPn z=5llW`_}imt`{>!{qAq=8dclJr#`PH;*0h3>)y&(?PgyH37CDe5cyn&$KJZUN`TS+whl#`T+4B6bn;wB35j{5!=CgaRNB%8v60FZ#4U8Z-$L3n%y-$H#IU9joPUL&_Dggr zv}AuxZkZ=(y?Ru5|4JtVdga<`xFP^(X5TkU?@b;?wAZ}+nfb|Dy0la8X2bi^=Py>L zO`BTfOOGr4U`_p2=3?gYRx2}00AFuO%RK9fH(FPCce3tu;O@MSedcH3;8hD>)`s!` z>{4g2U(8!)(fB~afwCCg*r?pCGKb`BP#ZvN+N4K;)3*|M$F2^uMnvrdwcapi_MR8GI9HsaIB zPZ!B`4*sc|>gVk6niwuLHH#wklElnxe|>k`faRA#fbN5!s>pHX z%q^5&rsnJFPwiVp&DD3QJ$8^SVu<|Pox|ziY`B^ zlCwI_;XI~Yu4_8DTY)pfD`xQo`BH`^_ofROT>Grp1t$hm_xJ zw=Q};yxLxt-vF{%R>A-t0J&s|I&SfpguvBE{lak;nIXNJ4nH7qt4#Np_u(1 z*%SgWplRaBs~m23OGZ)P6axceIE|dLezjO$=`~Zuj2iaM&pQ}asKw%)_Kwg>8M~UQR&Kr`|iEhL;RWd^tWl6sYev=JJczbJu^p;V#VaPW^2>Q zuX28HAusE9nz6!HSCUrX@d{V*MDS0|m$2Ez;K-KQY$Jbob)5f01L|Y+D6{rNzp(qt zd%1>xs(z5AKswc+I9hBi0tAf|D^P1`WM9DX-k|wn7dpAEfsuE`_w+x^;|rLO+&TIr zmxOekpY=ED4%_#XNBN`M z{`o}04Z4%!s-o*Er>4@VsJsu&*7(LDtMr=ve3>%PX?O7C@In~}TW~YH61X>)+#dcH zAbOF!NA8~GRWq}X<0_r>!1ZX8Ufl9~idql5=;O)+yp^2q*XLAcRa$RgLMSYU9vl#u zH6FkDArH&K7jS#6q!-C0kVTG_-exdf+2Nmle!KJ))<7Dje3F^_91>oPPXf4(vdalD zRz$l3Ra?VEbdHEV%r)vniuN`0gZCKz#q==ZDT;*Ys^OnAxOVtK_f~)(LeVoH+)V0j z-5~zq$_Dpz-iyhWoqi@mhr9sckPa!O$9Pf}NTD|(AR6ZxiUg~+F&lCC7K`^&=T)Lx z4gXNkJgkS0@@)Bagc3HCf=yl&oiZf1B(Z<+Ww|*s%G?>uCMt9ddP5{ zi^WQ9Y}b#<$h7G>d>mT%=uOu@v$9qVvz0zQg(}L+&ve4>*Fj@T$C{>{uOHnezON5u zKrSM3=8NM5ynA#=g8R_+ksxBk!7>4Op}UOFh|Y;t_sB;u$sC%LLo81wmd-IYAOHvY{lccUB?DkcjDoUA?L*4W7mNm@6xW=`=*wHT=9C$C zVVV^{F{W=8ADpkJiReVqiSvNU@Oblh1uOocZP9xorvTxEf5{-+5)gHnz#(25p=ADk zIS^_^N>HU8s|h@!xN?TqP~W@(3cy@Ym57Im>hZ#K(wn3xs3j9W?)?VRJ|VXlP}!o^ zW_!(HbVLmF^U)hU!o3eQL;EbBBKtvIDBxUJ7^;i~foU@os=t=PXx6wajaQ1$aut88 z&J85U8tGQz!xdf{y69#z40v zd7tlUlB;}4-;6f2k`pyyNX}nIL76*e=VRr6+1XhqqwAU?Z}aw*!C93vjPw8HWsXjTIg=WZfKbu8*W4KX;BDnd|}5b?Kd*ow%z5Rleoq z)1==)Q5pv|*kJY!hGz`K428B)P>tsN(FdA!7Ir=kocAaA#G#{3tBRWICbyR2Nh&A{ z6>)Ve43@KjB3!GxJ4aiUzHfP~5&oU}M$j1(a*C6rnWUK4Cnd$*XK%51cjp^WeHJaE ztfCR|R6|!o0Z!qK$)~T?pi_WHDDD#%v79b~6^putu z4Y(>tv=erk%M4@@){Hm=t@T28t}{+5je$LKgU%p;TcNX*=4S0!+g?T(FQ-Q7X*NW`qIl}7WYKk@tJAxNC5}%G_MzO!eE;~_#~|muWF@vrL+1DE!-w9z` zIWbqJ;Dl@Vr3<+!61n+2;j;=nvVAlo%9uYm#0!r)Bk{1_H_retj5Z(Ptw95_a1!cO zlpT05CK*Sk4b(pDKQyrR2Jsy7b<`hDuuSv?&I;8E*JeSL_9@zx@NTG%SqQ@tgEND~ z4U?0%Lr+|Up{`Nc++qTbZ?)%Pi8E@`kw>Vi-k%M1+YQ`4NqzNVPoD*$hFL=~x%ZPO z?#ryl_n=G6ZS(c*kDC2ga0%1@5BA<~J7p4?YlvG3-X$2IeQ|S(sE(uW@q#Hz1 zz)dQtu#rybR#8eqq*Lkc&U0;X=FFVm_rxFPk9V#2tmT?DGs|Z`_jBiUU!SOHQ|Dt{m0h%@x%M)`*#gfW zY7`sg&^GQr9wtv)B5-g;;Q9pTR511{tA*w4!NwUStpk#|49l;Bj{^-Akg1d815cOu zqh0{Z;30?)V)e7STn;T(z;GzkK-mY6YcQh&xp3s)R6Ij%=uPCbB%EYE=07LrOU_RWvpm;=ui$lYv#vD%RJP4qc`D2PnIDdBr-k;7Gy>`=fT>Rm@5D2Na$gqfd zc_5VH{G{;#&~t+5YmzUO#|4q<1&`RcTdHa2X$!Wn{ph%_Kj#&%;Ongvo7JzhmoZ5$ zq`FE+K}Wdr)!$HKB2;1WC-!pa0)n>m(%i0Y7zJN9zim{C7+Q)u<$#MkTD{|Y`8N+@ zsNyT$)$C&mzfIJt5a2fWOpn3MdjOJ&Xr&A)Z|Q(5{?^P@l5PmLMk4PV%@0|so`F)u zn3gD$b*&vsC8Sa09k*j24%uBGljS0 z&1oFCPT6{{AWY;Af@%;ola6pHewpR#(`x6z6`0L}0_UWV$;YvOBIn^{(N2^b-O59y z)eZp*EM8GuavnX%Dg5*Vym;Akalz^5?M`=mI22Gw`cEMYf*OM2rdq>L9!HJFupt6R zW{*Z!lRyRcpu3Z_*UVRh=9pvso$slUFXWeCVpM=v0qgm2x3|?43_tcqMhbn%i*g7I z2=)jn`jHm|oMA?5E~Q))Xo_06{jnUoUE=I9P?cr7hv35FFi+6aPji7hYz}{zM<#+k zYmt(?@t3zv!?ZaF^&>$_owaRda>5)SwM|9#OrGBRQp*ft=_~t&#!A?tNM57j&bB!J za_Ulq)dDqlh?pZaG5!;gtg8j*tCp@>SomZ6oM(Lfz*{+B^@YeTS>&~Hk;8gEa1(uC zIHJcWRD9rHT;=veXK5>YW2OVBR}JLHMm%0iqG!bY@yNQ1Ftyv8$|vgMJmhr@#O_f| zqX_{!@+hBz@jq{(2Qo#`r3#J=;j=B|)%dmz4Vh1Ezas*k zU?I5|95lJZB;jj`m?Yu2XDESmSO9xmJNqvy)DFiae~OcPf5xKeQkl-Hp&tHqp3RxA zH`V9@kL+H_$P!yWUa}2Y+F^f2xIk4E`!L1Z@-PNA1aKK*(BSq8SJrLe-|OJqpLIaN z_*0IKQn}zg#lp}Q`D;GB5GMcg=r{Q4+gIw2_p|9CFx6RpS`z1X)G9_|G9?(_V;cTRR*A_eKf!W@&CgE`I&}g z0h~2M=pT+H0*lMI&gke?6fCnbQ#2k05-fKUI;4ylgvMo5=2_40|XbzS`U7~xF{d+ClZjjJ;;>RaIw@;Ao- zzLl4?%kqmWDje*<^{RT{KeZJJC=y57*T>M~&YZv;Pznok+Lc6D35QW#?@E8r%ouKc zY*hQ=Z&u4MX&TumW+DOtQ3O@kH<#H+fVaf``}1ucASZe(_AgBG{vUnLl^}a<+q_4> zvyl_PikHC96ZKw^`Ja9~P;m6FYH){~Uh^!0yVI7M_V2FDm6n=zhqy5U4r68QZ|K3X zjyQik@Cq&2$3GBH33e3m2IA9upL?`7V;AM6nX63yCsW}I7Q0^clh=G@mcrP~dc(B+ zcZdpjVK$hqp=s57z{!Ul^iS4?j{)pzj{;C(*aO(plCF}itcAe%7XlIH@9~%RK4k~h z6T$be5i-gN$UwKc|D#*sHWeZrej3)v55fT>EKOq<1x950B4jrWyfb@&jr6H8gpvl1 z9W?sq_y0wX!68Lh-C2Ef1wOYrw!Wa+Gs~7>4ioz#-ZL;6`1~@AlwZ1jR`W$I#g!|w zZ@I<2@bfQ253<7R~S%QJbe*6-ySlC_IR@R$2 zFqxOJ{zvMBE>I_`6#1G@sS~2a+)XL*D9k_RTosDdq4pnH9p=F5=wo4WIc0Usz6b^S z#12xMbuq@icUBW-8--f1y^*Bg^FKtT1iGJjkm%4&Pnc|mZi%5YxNh+c$3c!O;Ww${ zL-2nhb^N>M-?dUUn0Pd*xaaHuw#{UJy@? zwR*`$qu>90G^$qdPwvR4$@S0XUacRzR;074zfAV8&)IEs6J~PTTfMe$*_QxNBT9np zdQ<8`>{K>aV*`gumyMHVhrqCPUKQ9kVFglMbh+p1F_`(A+v4=XJ~c zJQE-#u*gTe%PvYM5jI|sPn-Be4N0vjD1$Lt9h1UVyZr>Qd```YBMD(*nJqn&=q2!N zH?yhzvoxvf9T@brF$8c$F+$%Cldu& zXR)MP1SKAvb@N!cZ1*cxA%fmREtAKk=37ppK4ke=0;aaSF z>OwpBDfysinxM(t9@UHA$k-@L6Cw}P*_*y;ok8xbF|Djo-KTq6;P~=-qAb4^A8C{> zr6%vY{ehEX-GTo65@XiAH#!GP-hu}^sOGA5kzP+9wA~qh>T)ojCRpvMIa=d!9YE%2 z=jQvZ&$CIzChZP}SBd~K+`DPy_$BXqj^#&px3yg#i^YSqb;|J^<%uxWg1c@ENU`9K zqwA#4Ip=;>7h67T)<{VqvFN64*hu}lOEc@K9=3>Z7`qeEq>|d)*b^t6fPcSfYYZP( ze7J-dyMe=2HB-o{!mQ&z(@^%P`dYD_TJvnjT87l~>1B(bsFn`hY*tQ5sdhe8F1@fo z=hS{e3$rMKBDp7$NPjDaJUy)G{jt7GyhGh}?fqnYK+geHzgE_9LQ5u+NuMTw_e$u; z+19zX*68pv#@=y=q-P7sxB`O5;16P0^ds5H>jrge?(N2@(XknNr!7WbjpG{Y`1wdKN%jc*+cQLNM~$Up(F?a&rIB?m z8kURE_EVF{wL^GV?her zs_%8g03tjD1eLA830|tKHt%C^j^oNk%D?G~jF%XL6M}*swV{qK@98c~Hq7&N84QMJ zVQr^V*GAq*R*Up|db<@F{07(a*$?nI5r(qctEC&Gne0c%y{e2uywL!fZpS~-3jiS zvF+yGkCf3*euQkoFUyP$<|6GIePo9fj|2nkPqzF8YaJ9WmoQ(~&*fvkHNp7m4@KqS z-p;~I$K?EXp@_@}(L2&ZrDDWWTDZmA8yf*TCXO|9s@7ds0YR-!7^#&FV~b~3m|0O< z>KE{MqCB_R`z~?nxifBb_;$JI0}e>-mvFg#8C=;banrGO+1o@sAiJqRHCD|E@NDFE zykhH?$i){|lAloTo(bBt6stm^PpTCsB!#3MQ7*w}t7f(dM;NStmB%>;1QP_~hNIlL zy{56)%4MRJ)H_*1RbMwFX2Or7&7PaKhA`oqSbZi^H9b7XfVb6{Gk6Bs%xGG$ck}L#@@3)Y`%d+ieuIZKTI3`viM=AozqM@o_HH+`j^D%O$&)9L~QAb8^dR1i8`bL9im)(G8 z5Iw?JZz4i9=OKRcZ&&@MIP0ANJEz_I0JJ(4KAb5Ft33dLq@2~ERVIGOw5fp+E=Lw2xwGNc=bP3 zfD1!2Op5xg&*0yC!#XYH|tp#op^glt)=-wNH;y2-;hY-j!fJ4 zkRH9C_ALSL50tR)>wQ0*Na%&}f0S9QPhOS4>)l&|htVs~=o zT{@|tEY4rFiPl59mJ<6@VpVB?uKSjx&;QgkQ=R;a`SM|8;y`*+qi`yc+r#&bPVe?J znb(YwBxYSQL`Zq+O)}2TrWU4-B&DO}nV=(vpIg_P)e6;^={zbAOPsR1lZ)=^et0TM zzIXG!B1?C985Pdg;Cf)>0Gl;;bb!d5Qw1XDcA=qBbLhvI!x|0y* z_w4ZWzqkMjl_r_3EM07u*!j&bVQh2GHDz_Yz9lwVKv9GJY?*KP6Ljt+LzOKnJQ zXefh+N*3c;!>=z{AV+~q)HT=v_6|{C;wKArG^W56c^$0BTA?Q5B>0O&nsy|~fF|g{ zwg|XZ6uE3XcA(yk0x#h=;HOBhl%xY!MLMu4i$rDMYLEBLn`7AGAKnuek*Uwcg3z|G zBxG@6?b7WBF1N6r?Iop7wzZP31Z_S(?-k>hdAsEgdFSC{Wr5M^)mYgvg;#JAsB1pP zc|rm3foj_F1yc*P@ymT`ly~lXl1Z=4(Asq7B`vX3Aay7$D=f_fGanuoa8u1Go4foZ5R2Cw8| zxoYjq6B+f%`(&ETK12m;BC^?vQj&+uFCruAJcpDe_Cq5B()J5n=Ho6Un#q>haZLh6 z(|6>V$&8lW#3X`7<#1Wo8^uD@Vjm%06s(aJx#{$Bxb86B0OOv3B&hfp;DmK*5xe$Qb(-&Q;{I8qGSJp1K=gP2Abvr0+}SO9gW zxhuJ9;H+OY#VKDPn|+C|CK5|GGEay+p^~AYM+h4>;GmR#IyJ5j0gSr_aRI6E06drP z$BA`|a}G*|sNJLbFa@H#2McX>lTGgi2MUd(7)l|xk=lLeTX*A+tm%jj>^5aO zU4hLt8SRYX8OUlx!@}G~hWIQ^|+fub&=Xbck%_2@c`}YB_(?~V1QTpMYO8HY#7w4sjsO8O?mINcBi4C70-De&CP@`Oc)v&XDs76=g?9zNc zwPL+P1!0wcA_`=bTN+ok;yVrOQrV?W&N(Mt|2=EYjpZ0VryF5Kq$J1C5kyEQgUuyM z5$TUl%?_X*0)VG8B{Cdcwe9S7S~tIXHmHHmB(Q1{JEN0qFj^|jA$*QbEvbp5WM{Ajr<3moFw zzFQqyXSG|ObXaQW;6_nI&u8I{x2p*CoYir*BrlqEV9HU`+$<3y&m=O);czrzq%1`| z>S=kYZl6ONZl&X3ps;_!NKTMF*Yiy7i?E1vmL?<`%i09J2CuThzvgE$)p4ogyC^l`n` z2J`pSY=ynWhiBgk_2XQ)xBWqw_<_up67jxFd&^c-@}N>ouVvvrs)c+Xzp3XpTyJ zq{G-VF_Tv+??b0Z0?A?|+JHuRlQ|=Mq_irSIZ-g@U)uI0tO@U&YLnh z+{21cS?R5J&mPnb2^f{%Nh(ob%dg+Jv$$${Feq9h-N*R)m8F#aSN_ds0(~-(d;)k* z6;f~ZkL0{$f4xs+Ft503#>U_KCDvk0>E=^kXZNV8g2C>K3+o7%4z+jh4RdBZY&#xT zc?LX04lRh-3Lnl2A*fXA;PzT5rmssfck$7<3_E0MOy7nKfAAvm*GV!uT=tYlAxr?DLR>?lxG)4}*VVx;$Cj-^!dAz1%UEss>8q`@eO?}Ho-6`&z- zuxTBtKX%%jRmoII^#>eC2n4Uc5wx^z07umfb-jLiYW#@dT{}68(KMbc;KZKccMND| z7N-GJhP`Vk^ZMqyo==iqGW8~?qwxFCm%(SotSWk;oC_p;>vGXPKOXQc0iEhhmDsff z9KxIqQ6ti4IN3~P))@;0m&vL;#dvcURkd-y@XCNRMDcm|d%ON<$^kR==qfRTM;~$^ zQk1DS&ZM82xiHx@-rQw1(X8y`RF-H_A2+Jccp<91`M&H|qUdhLk^TpGzFh-=)8z`W zOTMvRR1H^Md|fz^Wx?;b@YN5SJMX}x;j&vK~x>q3N|obkAzu zOJO!YC@ON_wEbm3W?b{6@fZUbRQIdu#E_|*mG0pPGgIdybbXb(KF?CV9E!@8uH|Nv=(Zp$zAZ5BwAPnP#35l;5+cAoEp}+ZbfJLVt0QV$S8LChb?mM@d}1hFcA;060G{IfC=XY=k#2fVUm)&7IjAueKy*4#5QJ zOGk&&5@$HqWK*Y+0>A7KLIzc3fM}hUrrlP_;%vYKkQZ3ZYfU?-N@Uvhw-&~XPJVvG zEZL?^+jqVs4~N{tW&wPnRR@gU>ARA1PDgG?mHn{wnad$wCBX-qkytmR;PL{wzo}2A zd;yUA%UQg6k>!sxO_3N!2Kif5D^ICugJY62p$DLw#bB|_(r_4_fcS@{Lbn~OXrV@} zRVd{p2NaBM7iONu_W6Qmb6pG$LM`7Ok(+-j40G>%d)JIo(9a$WQH%khns+ts0I|4d zK*#AAA4Eqd$5VXx*-Fa%?Oz0KTZ>;ovMG6 zY+i{&R~9~|SST&*C4XaCJwak#M*E)jQvevrGD5k~T{uVC z+ZPKJ8J@CNdtf8-&u`JG=Tj~D*|S?|k02DJM1Q?t_{)%21? zrL8g>=(fiNJr8%y)XjsRO&Dz8{YJffR%w4{%STq9Qm)(sNhcSEyHaHj-D69y)0FgD zNtAS*%`acn_WFu1otNh#`=56PUOAoLp|5+_^Wc?9iry(rKdw#c9+L@W_fv;XSmGk` zNM~-C^IT;8>k*AVkMyf;ETbNgg-7mRn@IHh^T_fVoe;>#S)VD|T2aXV(P{o9Lv6&6 zZYRr4Cq+Fa={>cabeV8vyYV#t?-fouhhqms8xydLkjT8q8vKL6~~*z$ee988i_pYS!KWYh@k(xJe+5S8$9ZgQdQPy2W;%P+q; zNfgpU-j1X?PV{x6KZOmYP8Y#+^>h!kV^kLAy7sk&H%pwqQ_GEQ6A~!z4;zI_TN;;B z#i;iT0gMa7sl2yv=1x=+yiUW{TYuPjeR1vD)B+^p@{cJ2uwigdkLSq#Y>$-*?I(jOuC^0j*O;Di1y6v%@5 z@#)<)Gb8w$S8sFjf9|3iQOgl=-&Lj(+PAK`fLc1JKY`59B5@Lz(O76vAfU{Iw4PQm z0H;014R(GZ-50V2HvVtY@BO8SH}FNnW@h}krV0<25$ysGBkBlMzh_JM&T`dn2-hv2 zXl~lJ6XOq(@tO%5c{uq$P*_onk2p5opDv#2d4FdA{t=gdV58oEp8cgI zpIZLk%FW3eHUD>Fa#Wzw+@+|BjK`<`S4&9d7BOK`!~ouA^`CX#{|AtsGV%ld?+F6E z$?(beDHBUqTR`|f(L>l#{;w+RS7iRzI{n}H4c$G2RW11+=D;5o>Zy$X_kWPgvCm<` zKR!xdgC0QrSrx(fMpwWCpW@7l9#i!{a|?L<_u%}$p8xO85q}&Ke)r`6_j{g}=*dWB z3Dj02RZWw%q*Ar0uCyM-6?eG5$bB}}^)I+_2 zN$@trxqlw82$qp5r`SiI{a?(vN^x*bq#jc}BAme3PgogBnD2qRF(8x5D6%uun~i|dDl*_}8pZ1cJTox*kwlRL!9(soTj*uB zIE&G0P4L}eLPtlBpp!UP;k1%eM98E_QQ@-jo`hLh6veOG;!lB+EzJ<_p#YN>;V?g3 z;ufG6cGvBHP7>OEFM@Jw0(vMa_m+7uak4Z(4dqh6|5jr!AQiIOoF*+KTPJX*iGi<` zMWv*`@1;eAEz+ukijkU#N%02C0Sh>!O`l$zXB)OW%T_{5uZ7^2jRu;wj zP!_^$Ca_0RPku4Y!N_sbb`5yW8Mqi!y}&54O4QKoxep(QIJLyk5r;gz_&S8snYvY_ zS8qscA~^JgqkoC|`${!oz2e>7BBwG)9yphEq}Q&PzpC3OM8h+36iAS#>LmDS)!9 zSL>eHewX{IrlIk-D*L&VqFLrC_WSQ&Qh>SE(MwDWA@Gfo?Hu0#I^E_m z4?I;silH$=j3S0#LH91%gX{=|QH9}x7FBvzHu-*gW4%hE-{3wC7 ziY)}3Y}Cb7?6PjjarkeTW;&B%LWX#y8dcZ^WLI|dKELwu&p_Q*P-NmR$3P35ZJ_fD zXfO&v)HpA2-;3r)*v1w%BPPIom*)1HCp#BEQ#J5iEXJjyICC9^zSLEP;b^##S< z_MnOr3fu&D*jBh|~)-l<0KXYIHfNX$bX!}L68Yj^79Qz~({XgrPvk_89$&8C;UK=D+F=q>> z5@KbLjlyY>^f+8=ygQYuV?O6Aq8w)v2QiA`wUC1IuVex`yE!MBsikmt;CkpAWFJ+z zGFB(TJdxvch0IJ4lg4pPFoas9_AUQ6HAtTUyrQb;g{QNN=brQmEOM{e`aw}L_Lwy z(%Ut8tcYc%zZB3(lY;D|N2Jn^8;v6_9Jy(`t16( zIozw?dzT)e`5nBgrlPxyc^R*zAcad-0We{tdna>BCb3c&?e8^X&W(T`|gM*e(Mb( z!kjIVLGJz8*U*swL3{6bJxSCE-<{dx*Z&k|*x1GGfEBM4s&Z8ei1W9kr1byry2XS0cisILZgt)ppzA$rsO*OsKW(r)PD!(`K3TB z%S1ZPvjiTc42UKwqa_+Y51{xz*Asg_;&H3<`2~tAhmS6l#Y}-4TUeXlra8@0ZgfL^ z^XQ=Tk-6`1O|L3P*EPFbhV2WYI>l`v%3GaK?eJk&V@VAAK4uzOI>fl+BzZU@WaMA( z<}5fl_4Ojb$je3*ZpVhhCFbfSOxWUlu!I8*+otJEI)Q}Wq~E09c(`3C6=2HbQB z!4zmPQg^Z7Oc%NLAs~x$3FoP1jL^Yrde8mM%&c<_2-1Ym0+yd&!EVtjRd zrX%06vnL<=@T&PNIE}9+@xTf7aNr@{9p$bt%;)$xUV--g>`=t@5Y$B<^GYZW`?XUq z`rwS+#6_zbM8hV>rIkwooy`GnqaLF!@!USwE85&>JWETY+P(0vbCbtsauE zw3MV^_y+s*2S9?%7cF$V^mv}z*;zRHtFD1l;8~L{gijAQmM>+L5+HK?AOj4upy%;A zc8|UVvX~4!YGEDK_wr=M;Gy^)tl1)fPcM6Z%~LmOIJKJKje|DA$~u=`58c?>5?Q;n ztY@A{I!9R2OSB~MI0K5|1_y25=B8=D!tK+loW+lNoInRRo%`h|=OR&FkSXCy^$WXR zM^Oj=Zn6T>2wN)(?Owg10+-|D*9CKBMx5^^pkd%q9t~S4pdy?4$xJI=aa(WUeRQ7D zeVQEYMppr2y+W6ZqdwUl7jKQCEEL7+%=3g0%QG|Si$*oH+!sFaiyOB=J+PeK5~B^; z>f5Arm&t42-Jwb;f-&PX@4dnG$a`nSpS}wwYy#i2_ah~UeH$18hQSxXSUKh|GfHAK zT7znz%bv5SOkZn2;xHPQqekKFoe2`uD17>YqWXP80E3kG)Ao05l!e^t>8TNd1HNqL zn2q7o)WuZ-p5cpkdOHSci5jTRG)fM zRbbj0qk!uu)jC4kq+*pDonho5;8y{nR1N$YS@4faB}{c%#K?Ys@e1Ct?VBu)ZP1=6hu>`cK^IsbkNMNoG6O;D~b7Aw%C|9J*N z!5W%Cs!=S|TmjMK-&Sm^su-noBPxaMf013Qy-~#L|0O}0(%QEPM)@Y}-Of(~3_eKL zW8(OkB;(=R#Ix@bpy=Az@&A5>B;81o#>`(lT|#-9oC9O=f}!e~a6(8-xYz%?`9CvI zwuzcW#t4+xohn&OAIEG-s9tawik5AI=~J*A`OfLZ)Up*0p2As_-n3TCIY&BKl_6db z$WwclK~e)ZO8Db<%?IF2X;D$syyBe6m9AIE@pDMV6QYJR&h;9G@j+nACe<_s+usR& zb9Mv${i!LOPXVz3qtN1_GXrEf7I3e4F#WnfLc0`s7qPsr(V60b%6qtB|KnB23&ERh5e4sv&_TKfc<Y9lS3>2E(z3=e>>tivq0}5|7JGRy%yEDJ?B0ljs>DBE0kxY_doARda^XYBbxq4 zyHpVE9-MT%%EF#*Fw9P9|DM@8pv8gv%jARee4@7JNKx5Pn^!$3F3p!c=r`#wRZ@0ORs_L3nt$8Gh>inOK5bKw)%9Z?w2sh5MqjD%z zbpIF#Q5c%v=@l!cdJoRwu-7~`=#UB7Ao%A5`z`N;yc7;8#xT=0otqnsj9?EK@4a&p z3+bBwa=-prh{YTy$bCm|?#e;@UcvwSD}VA1|NrTi%e3ABU*9XlBcaOQ<+ayVTG0^v ztA$b=F`}=~*RRm8yv$B74L)jfH~yPE&qL%4@>O@JYRX>m`~%%c^p2wp=930ap!65Y zXUsC=?qZ>PGVO`K7J83Zw4N3NGr&JGoY(cB&qIQ)Q#JwRJq&l~Yfv~O0^EW#1webM z02<#LP^=Lpc$PuXy%?@?Qo_K*WB^FqZev;@h*F@m_>X^co=hxzqE4kf3p3nJ^-%O5 zU`edQt;@s-5X>$pL+C(F0gl!vC2V>5mpe0-(7LtKz#<`f151oU#UjVUoM|DL^MQXG zbFwp{JsPfYbJ|<{&|wT|k#(peBLg6r$tOSDE+|sejJOs+Gz`w$`TF`$)|CZ9q6nxT zBKKG703xKIOV?vi4%`scGoj?pU<%EGxVUEBp$DHjh2oonfST1diPzRJh?FDg@Mveu zZhKLa1!SW|kwR{cq-Al_)j==-qa+HYFR*fyyvJmE!{0?~)gM0=*7|F2@eaY2PvBJn zSQxJL-&3VxVOwefFXHYFUYgF^#!WQ86Ak*nUg=6#$`_UcCvc)(lqwi?_gv%;1D|JU z2-l*dsc%1isp|%~Z!Fm6VvU@C^Q<3Kz-60DL+T^p*tl1~?et3^fe6^N;V40$ol_dD zCWS5ozm2~K6f`1SKA|~VAb1_EY6U2o0i@NcnW_wHJ3FL;B+4-tKk8+xv5^V6+P{)+ z{4yA4kK^pNjUuqL45%!JVV+EPBxHdV8wSm5Jo#Cv}j zlPR3|LVCg!o;&iybww2ekt&WjKH6{gM#sck;vX8ryRiQg7f`7y_S|o<1QWeM=<;+Q z@Utl(MyynXedF`vb^j<{dF&5F zAN7S{@VaINpnc8$>9;47P9NUAqB)+lJoYdKaVtuEB__nYR{xrN9pWD7{HJ%~lqQQP z`8JA)mP;LAU_DrzIVh;u`c({Rbf3>{8ZxNpdEZEy6Wg2xCt4S z1b){iTD8Jx_hceVTp20uuY83gsv-8)3(9Ev-y6v|0T6HJ79a`CQf*Nzk+#_=D8^fg zxZu*9janVKKmkN>AOSs z;i_qSvUUz}(Kc)(fm+y|87|74|AOpbj)mGhjRW}P>7UOj#;mSi$J+cfD4jvRMGeZ6 zWrL@Pyr5^sC9A@-jROC0QfZ7|w~Ag1CKuOC>UPu-xbWKmjY z#h}GCcl*7Ya5%;v`_V%fKI{iv=|1|li5wJ86AwHZ= z7$4?+{YTtWe;aibCbNBuf&otm|GaQZ2t)Fd7DspCfm<^1u_GCR-=YJVo2A^vB0_>h}qeNt^bI~JKY zo>~SD_;GJnjBnf8_lA9#shW4=EIt??@I!BjVjY-TYdbmC8&8vF148@#lW_E4N8SY` z@sd7G5f|ZPsmHb0{a!jlUaKAxFQ3>wKqe2DYY6A{@(y$L{9A}4$s<_mUO6o2Af9z0 z(cC`k*NoG#Ad9)%od`#fBu>FT1hJZ99>c)(UIaKz4GA6T)@$=sd4x{p=y3G&h2fYh zWC#q6QLK&hwXZoN+T|UgcHI(m%UrasH_p;+(tm4o8Xiq z*L~#{z1ik#$07TfL+bSnk$Vb?}5;+-W^lb(; zU;2nLH95YUrp5OCc8cJ~MRVKy(yP(NY3hzP-5MF z^Oo^WvHX6(8@r~jp1BP7n>!{4t+6M;gEQ@3ej%%H;6y7^IK4zWMwiXk>9?f<8^iZb zK0Z>)5HM;bJr))-OJxwRoaX&fneTYNH2(KwlBgd*FD%iLvWA;{jwEW6INF zo{@h3RODQQsmRf4qVZmF9KC$JCga0z^febnlzG3#DzZ?kTeF%}UHn4l{N~5o*F^nb zcJxZyH0(s*Z3dn3A)@5h1||j(mI^7B3VrZvTth4@gaz6y0xV)yR#{bbLWx@jTDB^E zXufaCain`C=mXKk&qPU(zIuBjnE9^nVTXE^Q)#-XYo)bPhyB>{j&OLmywi*OE14C~ z>iE;=2g`>zK8*fQ<0ageZ;f+2@>9w=RB0UC#J0Ab(jF5U#?`@{TVV2h%peGIclZ zvIQo0GX{LV?yoyF!=C$nX0iOWrZd*1+Y>Ho*)l`r5|!dxJyC_l`rcj%m9-VFX@&gA^r0!7Zpj<3hX;Qw&&&$^vvgTP# zHMVYdbP&zm_TRphmQq%+BXXY1Y$4cmtk8%&dFHa*sN1|%)uZ*9Qp0Ip?ah|^y&jVm z0vX0E@$Nf4YNdvk1oAtcYtreS$Ddy*GG#0melQcslU%l(|BY+UZ2+l09)>)hF&&Pz ze^@6Jm8~p=l&X7#XYiqS@_Eke$c<*5u4h^Dq`3L?|M5ErbF(_HF*ZXswJjbdvOwqf} zruk{_%I7vd@%`uyrIO&r#Kn2R0c5c5(N~NEtM57oj^8O3b{pcCpW%LY@uG~FIku8m zFiFpYPS-3PXGl0Ii7)-IjbHuRa!{UfxHauF&%A5M!Ql>Noz~iOxq);e{o<9&9VK!n zob4XRYw;a})0gCOWmep-aqgK+@N08973uEB7M?7*rC(xEy>mUnbN``-MbDy)kKD=qpu za-h-nOnWFSSET>Ll2S4Hd9QD@U2fI;_yWad7u2N?4oB5?z2XOZ2P;JvV-JouzHm<^ zEf@;>v1wHv43+5^I%w-yFzNQQ4Lvvq$)=;>g4Q;|Hx@$$y`_g6*%ar|i6)F(OAF{E zf;DpATjqX#&QVpnx#w)cSbdQES=vF!@hhQY+fazb^fiy%(KE4+2hwhwH6eU*-ZyqR zPkLXwhQmI2Zg9QhlBS4MFj8a$S+vxDET>b)t))@E5qZ)hfs|^z0>aP>9mESFCx;&Z z3uW8SU_n2v#>*^<;Yl5MzLO*licZnIDo`(&`&lUCWdDq1mZ@|lLJa96-Wt+d^9MU-OMlA8j4`1=&k<;frkZ znh#Gn@(6}jXf&&C_iUDZNL*eHvR{f1ZP6wt{wvYIJc3*<^(TugD_L=RKy%8#*8sCxQ@D4YBS^kqG~{;$ss+{Eyc`$6FCZGF02qw{JG zVv|Qk4RwF`4^K8^ZpVoXOftjjWE7bF*`-Nn!I2`g-ts}G#i2$m(Q>Baq;%l(S&fCe z9k2C<-ZQ(-q&EuO;0$67tqbL?WaS}Ws-Z8W9J zVdlU#(yPt_p)fo4C6BzUA?-N4dflRG_xpE8osf``dyOe7;X@7+B8AZ(82djV!mEy^ z2IFqdg)LiIA6D>O2$n&bdmW@8l=>g0jy*e%7F13!cg*rWT6#ys-R$=wLc=0bs$aph z9cOzhfvRD4a_#3%5{Z_cTgN3XSojtPo8GamQ zS}DqZ{PKfcbk-p5QeFW={!x#WyQzzOdX-eX2)~3b;XKFtnmIkG^dBSS`VR5hPjEbx zTmk2Ooxu^b;Bw8syk@D9+bU+l-YNRcMnht$#}8Mh`9q}7NRC=0cj^A~IYqfL%EJUo zhZeE{B_|P^`m7|!m=cQJ*V@uS3=w&jXHqMdd5I>+oP=zYDuMo~1zfn0Y zf=2U-3B3-#;in(02jy3&EhQzILt2}7z;RY!D0)9Ha?ZimwUZ;QN{OPghm9BO*FEIH z4iOYYLj%i_Ng7kLo2zrYRPlNGm9LL z>l=LyJd?Qj5u$ii4zgmfTdqpguL!Yd zxp>vw=l_9AAhRQa2~*)8pY8Jmzr5Ph9~TQ9hlCNmZU^~f>iiGK{d2QSb0mEUW9M;p@q7MFSEP4 zl;P*g{$op_4&W8_w6u5d^H$JOQ^^abb;8l;68WGhSpLv)H}hoW#Ng*gprsuzuM(U# zQ=I=9^d1f#s=%dAbc{dzd=*sxPA21`A$WojKh&M!UhM=>t4t z?nY3YT>IV7I%wPmb)7~`P&JI-Z3{jAcIDu3qhl$EDQUIGx_Zmqfk?F+urkM9P16A& z6--#t87=I2(6nnP<56G`Fq5FRsqZbM!Qbs8xS=}uv2M_5>u~O%AN}%JZ)REoYoPF_ z6mWqr0=^LM`=T|HU@0V1%X??>eK3B?f#lp89#?idyzzptOQ<2K#W;}@87^zLAyz5&^W~|jn$2n z^yKS{fD-2Q^&e1XJ)Xxp>A(q$`79x|*85HjTYbCVD7*v;d5;D~0=Z?hc7OhQq;#xC zk13H0K;E{5$8ziy^xDZx2{yi{a?%pRP_%3DSQm7J2hbb36+kC053D}+f@+@wP_p?I zgRfQfnk4xxL%-|lRgRY0HLfKKQrj7|q%z^%Ne%PA0;sK^H3h)^c|$iiiYiFWd$|HP z4^?qm&t)n)r$}ATB`q521Sdh+9%-051p$H*IOy^w{nY2lRZj9kWywG zJat&xr1U=iQf~DUNG1Yq1t>VHg1cM8BypX(MC_a>jUO7_$ceq^>D>-#C7_uc0`X(KQ8m#R(Jr*W&UY6`OJ(58 zIOsIyVS3@}odTXw=g)cNzUqYWpa29=%0F zS?d4>-wI!N_bo&%ZMfE!0-fW2;)V0UK`rVz8sEVta<1HOhg^%J_ybf(XN|8Lbz2cK zAFZwsf+g9W4&vu$A%t33&DnK8snv#*)`+2B-!zncHib`jYS{jCL^P`(hIN&C37r&? zB-(%H3_6ZD1M2w8b??5&f(A>zb{aMY{oM>apPz)Q;*|}Cz-F@q9)n`{JgZvHYhB>@ zIkxl7Xh4NT9TFkI61VlqW>k4=lLN&qSRt051Mr5 z@BXV+yTmM-^4YgrRvjDQ_^0Ei-^Y@kFpf0^gq9B@ zS^4WkZ0|S_0;cU0Wd!e-ODK^GF8)ZfR$I<^}{5xpM{g1 zy{Mz=-j~vp9`nAu63FXR<(rh%k~Xe-UJJ4>p`Pb3x&7qb7bAuwSj9HyiHcZMz2eA& zTg=xQ;_L|UN!%r{4eJ9a-%uTg;-NxUW8biG6sB>RQ z^K^F?eT(Z%ck~%wq*#i|gU}Fq|=mDLHfJz2GH0C;8UhxX7#oH;; z>MgFYE!$mM)ibM5Wbga+M7APH#LH8SUf1oNP+__wxTG5Q={o6hBS^EQ)wN3LHiBOU z=z(;j3{vyPX&nKBLr{pGw0S3GTejKjV5uxo8uqKf7!5T)eP903Y?xu&z*DK5E#-Bd zTMkZ;du_fZitrkEw;~9pa@IL?y2GS5G+uGFE5~olD?JlC&*L;~8<3r{*IE4Z{?3Z@ z(wAqe`qg;PA8-s}veOAkk2a{`U?rc9>qn+fIOif^(-Ex;zk zT;?%z$Y)Ery~j2ZRo5iCgx21>4z{xY$`a}(Y7MS;&0H8H&)c-aHAw50>o@nR+No(s zYbZmTBpxw*V%QAhpJyf|Q@dK` z#OeZUz(j*^o)O(6*@SFeT`il}oF>8AH+d&ti7oc&dpTei>Ute*S;-R>Xx-=(cBVv2 zUz<0!ljs$)CC{v>7I)Q4uQC@Y&r}v?ygk}r zP5rSm2AiG*;bEuiyOrpUHvM<$h2}YBBaZO`^@<^Ah<#!KsPmg%m7P?CPrrYyBtTjU ziM{V1xJJ-Kfbm`2Y4_8k;&n3L&_|4csQDoM&meA6h6d&i!k&=;;^o(W;$``vx$C>A zZt@6RyL^@HFD}4;4XdQj`E-_LG?mYk?{N2AgKQOdjyE2d_HgK4pL!ORNxF@pdAqe(s*L1pDrW_r0){@QmJRD z1xwqaZSVI99g-X6<1OVaU3pRgjrlux^by1rcdMVWcS7|J0lp7-r$=b){GLfvLz$5k zzq=^>9}l3PMw`s`7=>U|U8o1f#RnT*l!QxD_B;2SOe7rwA&6d%g2EJoy(RkGc^e{h ze_cC%XQa?D!bDMV6zpBl2Vz|iS|6nQhzvW`-=31bOqA}02ute1k#%e(MIsrp?-j-xS&HI0FZ6rv`}zGo&->5&*ZcNgpU>r*>-sL& z_c)I8IM0&-1$2Ne$Oun1YeIVmCar(-w@ONNpGH1@YgPXNOv7lxi<%_)Y&*r;H>oD! zW_bB*2wXc2zwkeSH=8VNf3(?HJ@pVRG8laS0le=>RxBkLe(`P3K4$>or(j@Y`2L-K zgKm3lkSCZ{{@Yy@m3Ts5K`B&4UNkC&5UHQ!;{eY%?Lmdkzbq9ITc>rR|8EV1A|LC=*KZ(fnFFv= za^ctC8QSTuJJUWw+Wln!X!(Fz%Xt%;ZI2YD@+sfy^jqp1ed|Y2!*)T_T6+jOZtV`p zzQcizG@2AXS3pX3Z$BpmR)3@LI{mjGsF);cGaw@6ZRDfa5F_d+SG>k@X@c#BVn3aF zjF|Orh0X{1gtXsKVEw}T8?e==KzrHfgCH-W>(zFiq2ht>Z zZEeBp-~a(}1F&>XbiK&9!;S0yBm>NB3MyDP9g9-fHv<7$Zi*E#P=um6A zk1TRF^8ymiUa1~W^60*@_=dy$xUodp7ZwiF*v?PsF0kQB2*2f5e|+|+g z)n~d4Ys>!|T*}2ob{*YSek=AsO+Tc7Rk|!cV40fX0Zb%;ExSi>8mU;gs{phP=*q;cUVdkh4+&pj@eoHFPU;?QenAXjL~hTzfVl4-h*L;@VjEM+paxBi^tGx0oaKZRAYBWl`4lYO1X}lE-`u( zki~V!7&Qx9qZmkW*QH1tI-20v`b6{e@7k#8TznYV7-WOI`@KeGxX&Los`js3DN$Jg zH@`i6q6-ktPWG6x1$G(Q8DQqCyq%-r(jR7&k6wv9vD%r>P{|^hT5I}Q)V!*~q4W8x z3aMirl)f@|AinWRG%Yd0DQE6DA0C*S*{POlIrq7!JgYE+4=oI+lG!cVyI7mPDGSA?s#w@WyW0lkvtU!g=i~ zQgXE9XM-1J1_plb7g1*)3P41d2GKmXwT|m5R}D!7GM4gqfi;U@o^Nq8B5gwUYm8oR z%{!nwT)V~GdVOXpr{$d6uD|Z{9N=9G|hGDpE_y+Jq0Wjgv0Zk zg+=mQceuggXoNJT>x9t{Ku|3~iPC#pDl4$9!S{XjJ|^l5F zoJP|sRP~}I-DUQ3OaACl&>`N7mrowHzW*!Z^(ep7fjdVes)#D&VwBC! zK=if>4L_fAyM{oq|9%BN7+3wh{0({v;@v#28pEpyGAkaniYkddVK8F6>IX>dgz|Yn zVVF?2^au%~=e(k}`*x@kT^X$KIz=I-H4D#QGrAJz_c>NbYLI+dmcF(26_uE-B)H0P z9d*+pck6qXsW|^&7wN}@xryyj8ONSIm^D?QLhdM8tOoPez*LLD4*yaD;<(X>?nn>K zL!)XAP`3(G+@A#2R*6r=8(t1t>{Ml;u20qL1Mr@4k%dOuu+p<_hkQTBYo!nZz7SP= ztNcy7t9U<<%3UX})6gBhNs}h5C!fsu*|O^E!0yobU*L5HWOAL53VIi-a<_3VZlqf} zcT0_%QPN_vU1X_llKY`9I1WtDlXde2epUGOn;lgsF?_COuU4zxCX!?3(=G6p{n!GA znk?t(;X{d?7N^bp-5gG8C^k|dPlmNc+K2O-6&V_@OvF;ebaD+JsT~@pSOn8O?`AYF z{rJj@B?r>Ox(2^|?gV&3u z=S?{LuIC+(=y$O-1x3xrDXGa7H5;%s3={b7+dr~ckxM*!(f;79+$RQYZn%6Aqj z%SgD34{}F4W`R8ChDCy}a)6}L)O_(48=`XD;cfh#d8NbQ@_6rx?`l<5)8vVQGoEp~ z-FP3V1e`RLf5^smuKOkp-7w!voG`a-TUZ>;cQG@UwXBdo>l8uamIq#L*};ke5mf-y z8?F0pAJuUuG9IV?j0Zw1HX{XA47>@t?Z>WM*#lRAI=wk!|G2`kz=q<1tyC{UAj`@q zDf*0yqjZbc;dkWWe7SSQx0Vv$eHYV#;l_sZPi;^ex>b$9?QBoryLnHY?(okPlIw-# z#i$+-7le%==<4xLvalDPgMiIaGP+?X{X>CriFqK!67mS+BO zq(Xx?yB+M;zLPJI6MnEE*=rXty!1BKRLb}(bfz~jy)n(fqYrk z@oD^mK37bZVd{vL=Z-V^_JZl}eKiEFRlE;VTcy{d2qXboZfy!BXV(r?kib30B^bVd zenjm}l=I9@c4-ivF3RzCe`~I|~W~@oryE03KOjp@|9&<8aG}&_`l;q>cSJ$a`U2hL|Uv6bxaxTSYpLPZis=V3xMASE}KMSBg5!Tpi7jQqrtb-lFf}$Z>!p zaa!ycv_Z)4dSP9@3R3pfvW3UexzN0%?T%XBw%9&6DLwKm?pS&<30{Ak0)h3 z(%Z|jKFa)Yin8)d*f`!wpu}a&4~{#hxr1l}jeJIG&9g0V@Wh-GfM5z3UH3tXuZBz< zK|&`jSVMAmC4Urq%Uym7ocY(FQ-T#^W4P^rM~}l9lAXlL;Hw+1F;@nvDm5Z@vs357 zY8SyOYwER-Rlj2FSjnEB3O2{`!e0aaD@~ttEc;gC3Er0nAZ`^u6>w@}0Jkdga&d?5 zq}i#dM1%Ut4+>RYywMwP+1_Xl7Z(NXeKUnDeoEan zU^onmNtw>;LDCtcGREBJAdv z@aF1Djhha&SyRrq|G|C0ow9wuMWAkzJ0}^_S<>qgA=hGh-IPi-d5RCL4*I#I+7Ld^ z>@d+Sv9IH6Kycv(p+!EUfXjCYJZy8qC*BaJd{vk_X87zU7^HhszC%YHV<0X^+Fsl| zGI`qXtNGB!JG~kW;coH!gY_#d{Df>ax)ei^7~uxQdpZ~&>L62hAWTw zq7M1evoX;!&WZ3oKbj~CT90?BkyeP|s(_C=FC}`4b~$8y2n!Dc613#v4)fBJb_J7kpCDKc>_|sdgYFga> zD&BLQtw+}P5J{MpJ9OhYX$9aceRx-+pbLgLWdk z9#~UtROk$7 z35rama5OtK#0X!F)Pu6AlR)~5pfcrUJVoFk95WXoUh4E&7)TXDQ9XK{>$;%4gd<>L zmsAd6-C8rh1^x2ja8Z-rW|iJu@EJF?EQFY|6p-5e^7E!}rk!3Xu3DLv13I*XX@?eX z@lA=+%|$weHadxiE?$S$R|l&_ij*1IQ@%W&^+LrP zp4uLAeTy$Qab}$_dIoOx4~#Njm=d9c1i6j48y>>=tU=4xgS_dM7P)0Y_4`7`)QT z9%9Po)Ws_T_Hcl-?qVo?W{ni#$N{L_g`q*tmxPZ;%f0BpzW9b(Dr+l zD0M?aHXopzT?ZNZqQxKOg#O&l<2%@Jjp&?S2bC~iFTUKe`@y0+kK4zS;(fotqfiXB z6-M`*gFl&qpzu3Bis~IsE1dWIa}eTGvXKxvM^0Q0xy=*GsU{KP{H&9lPZ@|*ye)M8 ze2POj(oz)(wi8B=396+8mbo==Q6K+F7r}e)9bGBbs$gknz$A?25!F9XTTSpGfvihm zUuZ9~?>ZcJxC#P;*>0YAN+g79mf~|EkEI75YI-rzr$eIp*D$3Kx~vT^hL-?h5TwmM zaVq>;Rui1$`0NZY$W4RK&Q)%mnh7fFy+(i{JH-pguR{ab>EP+BBG zYwkxxfU22*05rf7XAjd%Px3ZP?y^hTY&0QET9pJ?uj%gsHy_?l6! zF}*lsTCKxu0n(W|t5+8!vW)Vqp53l39) z%lFe6pTP~tm=GC+pOmO9o zCtEN$g{y5KaUhre3D2>t$HkT#yzC64VPrpzF*G-)NybsPrF82fUI z{8OPYQ`xkaZvL zT*rhIz=Xxno`)^?U19tXjP0yu;C~0kA{@{DdB9CJoDC%giUZdCA;QKdOm|Jo+qYBC zfMj<(d;HDl*^Mz$bOB$SmUw`7h$5MVH!3AJ#ma=K{tQ^~Zf`z3{{ygtyi&Zvh+=Z0 z1I5exnDH2`OK>Zo9GGa3bN;2;1NtBumQX>;>o;Z6D${DTZXO8_ zPLq9n@2ZgM>h7Fhzm_H_6qe3q7kJRECj*yA9CZ5K2MMVC`#E?_RlI$BGePx3USiV@ zrzF56PGOfxg$O*=kWoWGQkTbK&kUnO18N?~mrb!!%H*97yn%Z)^M?GovCK z4W3DYJ_zAw7fyS3zwwP|&V%()sss8sGGKq~c?SY*b|k$=IUM>frm)0E`TY452u16Q za?^cx7qG08_|C#tjU_*NL;iggG95=%l*s;XS8Tip0~TQCaa~Ttyh#&u8vF8yQ8E%rH?;Gr}pX&){^=@R_$D zZpxzS&8Xpjn3tEJT5kPd6(sfGCpvZI?_>4H!>LNs!gf4vYxf2C83t7LGR{CtS%5E^o zS$p<-Wo+X({DOvN7mu3C$+M6(z;9Q-BO5nYf1n+*X^}RIQwYq})50BlL_px99N;6D zLOB3#psJYN#{si0gvc8bcSlT(3`MW(?4fOHF_iM3=*)i%@hP(6+pMZucPQYx2w=1a z2}h)>#}*)w5b@GMlqggeOS#ko{bDPji*tK*KtRV^73u z?dUnE$HUr!4=S8(4p?mkW5ri{*NzFBv5${rmT!NtE^@79Rz-|#J8IX21t>+<7rza@ z+B(d{3ECj?^867r0RvI^6n_BIPN<)nC+^!I&adRoEGQ#l@ErheK9tq%0l9Yy5`jw~ zk+5uy#p|m81w0=e2Wg)LRD%w0P-Lrb2R@P0f>4o(r))7%({arJN!};L(>uyoyao9Sv76UB@!uTq2_o+&I*XNs_g?6=S z>pSDl9YOPg_d;(}u$A=}J0wOV_{7nX)51Y`^Ea^6X_<%0w4IQCovfH`kdt8Q@=ic- zHggW-JoiuR#0CHR=m${!DaRKOX>!B-Vf zylFppzbGIM8v}dE1|csuM{c_GHUHw#t3?Nba-=w=ui67hEI>5Nc8IL)zixON_?B}y z*jJ~t0+#jAcKN=b3ij>ck9mV%4E+qi&w^d#>~m-oTk7uurcDuvw@9!Hte523EUtVB z<6`@}=HV48%K*hqtLJ7Q1U30KyYZH2UIk^VCA-5K#9L2C2wU(62(2i^Gt7k@-ilZl zJ0ZB)wt#q7?ZDB*E_S{xrs&a-#jS8s>%C-n088^Y zDvXKB65A~Q{dS*gGtT4V$?frJh3f40c@-v~zb;TASFJ$cZxWnk9)4P>3Vkq3qq?L6 zVe>$Yx}jaRi7^?98x3LZV^e(#B#5g}wX_@X{DpPbkz2Tjx|4|2FOYEyX}4@UKD!Fg zuB^G};loq`5s0jNV70ND3195>*^Fyepg)JFlG(7y0vrA>P0u*()t-UX%ZM>aLnCev zS`_af@K!)pz$Eti3H#1NabXXafpR5Jyf-fV_GBEar>E1*r*4|FgsGdyeT(J2FH|v- zV3$D@U9l zDr`(kv2&)x!c&<+2hZb3pl|ei@JHWIn6Qzsk#4Alm;5|V;>}4VGglQ1H}ON$-p#Xz z4vr#z#S9HNSqzccur0%0ywz_qxHJqjTgeh22&iC3zeUxFY_Lx?Ix zV7apL3rk+OloWgzFB5YKqX@xNdA_JBYWuYQIr=X*98eF$kKWqvE`|yXD=9hublFV3 z^z8z;E6Mn65`Jbn=B6z75P!N9QC#wwP#)G+-~1^xg@AOKH4Qe7tNpdtI%x>K-7>Fs z7F-W;P1Y=_i0#1CXR1iX_vxERXq;W+ho$vtl&mWaEzIpO^J&660^Z-RLmD+O+hyqr z#M4xcGqO|u+siF7#i;tVfq*r?yEtk;DrDT2ZxqyRmzHVZa-~wAQqZ!NiFHHSFFv?j zH`VdYa}*o4Y4!xM(?rHE1Ha|dr<6G)n0v=CLaCpf`d#~)x+f3R(|jR3AC=jj6#Awa z!isb&--cc%^Hl!iC+12Dy*}ys7_5=GJwSlEZ~e7zlA?sSqD~;HcYiUJ}0}9Q+2ZLex znZ_mLY7ia6&f&3_aL1=1Fzn%XSw>=yPQnV<@iJ55C%75 z%(-=)8quI;fd*|r8O31+esz-QaCVzhm>~YKdv`fU(1)o)*DxHuM1V2&$uC%5#!zvz zmqs8wmRn3Rv%ZWjv1GR?gf+)N`}}z0bIns>mt31CC7}=5?C_ee@#M66mRR+r`#)O! zZ9sUU2vo5N>$e%kp;`EcmGR`adpN}m!(Rm3R$xri^lrnPC$oDQ46tOYUoqrAcL}OH zbX>oM)`xQ)?LLfWr8Y2X@BM8au!a3jNBU1}N;l_24dP`ngG`f5o7jO^|C!mlmLI`+ zIW+pjI{K% zXLxlmra1NeYWpW>ay}}5ktd2XXZiK_i#AerzBVb|1GgBbavQQ||SEls)PaS`lGs|ZSQ zk=FU`(2PO-g8FR@6K>~2hxBFkL_affIc3PIoQ{mZA`-Jvu@WWoe^t}Bp9}t*z@ZO*s}lTV6DWOer}0{?}D4xjGO0-NgyMa zY-wXiuZs6oXuHeksh3byaIlcWbd+|CSC$Kvv6VHDec%C(MwDQxHzxZF>h|XMO{7Y}V z95WPt&3c?&VPJkqJLJ8Zy(?buo~{qYneAT~eVxMw@v><1ujbcryE0$G-(^oXeCBbL zI(%}!ImZ4zp~J2|=Gljv8(CEw)3iZ7g|t!X8iw;md48?j*V1i5Qtq2QOy2&`S;gD_ z3?_ + +Runner«Serializable»SourceReaderMiddlewarecreate sourceget estimated sizeestimate sizesize of datacompute size / number of executors= desired bundle sizesplit source (desired bundle size)streaming: split based on number of executorslist<source>streaming: runner asks the source for watermarkfor each source create a readercreate a readerreadersstreaming: + checkpointfor each reader : start readerread elements until none to readget elementclose reader