From 6aac47c8354eacaede1eb50c80288c36e734e6f6 Mon Sep 17 00:00:00 2001 From: lahariguduru Date: Thu, 1 Aug 2024 20:56:28 +0000 Subject: [PATCH 01/78] [CsvIO] Create CsvIOParse Class (#32028) * [CsvIO] Create CsvIOParse Class Co-authored-by: Francis O'Hara * rough draft * [CsvIO] Create CsvIOParse Class Co-authored-by: Francis O'Hara * Deleted changes made to CsvIOStringToRecord Class Co-authored-by: Francis O'Hara * [CsvIO] update tests for CsvIO for more coverage Co-authored-by: Francis O'Hara * added more tests for CsvIOParse Co-authored-by: Francis O'Hara * Added documentation for CsvIOParse Co-authored-by: Francis O'Hara --------- Co-authored-by: Francis O'Hara --- .../org/apache/beam/sdk/io/csv/CsvIO.java | 162 ++++++++++ .../apache/beam/sdk/io/csv/CsvIOParse.java | 84 +++++ .../sdk/io/csv/CsvIOStringToCsvRecord.java | 2 +- .../beam/sdk/io/csv/CsvIOParseTest.java | 197 ++++++++++++ .../io/csv/CsvIOStringToCsvRecordTest.java | 12 + .../org/apache/beam/sdk/io/csv/CsvIOTest.java | 300 ++++++++++++++++++ 6 files changed, 756 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java create mode 100644 sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java create mode 100644 sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOTest.java diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIO.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIO.java index 04141e5c677aa..fc2b68c0a8936 100644 --- a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIO.java +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIO.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.csv; import static java.util.Objects.requireNonNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.sdk.values.TypeDescriptors.rows; import static org.apache.beam.sdk.values.TypeDescriptors.strings; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -35,8 +36,13 @@ import org.apache.beam.sdk.io.WriteFiles; import org.apache.beam.sdk.io.WriteFilesResult; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.JavaBeanSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaProvider; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -44,6 +50,7 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.commons.csv.CSVFormat; @@ -340,6 +347,161 @@ public static Write writeRows(String to, CSVFormat csvFormat) { .build(); } + /** + * Instantiates a {@link CsvIOParse} for parsing CSV string records into custom {@link + * Schema}-mapped {@code Class}es from the records' assumed CsvFormat. + * See the Beam + * Programming Guide on how to configure your custom {@code Class} for Beam to infer its + * {@link Schema} using a {@link SchemaProvider} annotation such as {@link AutoValueSchema} or + * {@link JavaBeanSchema}. + * + *

Example usage

+ * + * The example below illustrates parsing CsvFormat#DEFAULT + * formatted CSV string records, read from {@link TextIO.Read}, into an {@link AutoValueSchema} + * annotated AutoValue data + * class {@link PCollection}. + * + *
{@code
+   * // SomeDataClass is a data class configured for Beam to automatically infer its Schema.
+   * @DefaultSchema(AutoValueSchema.class)
+   * @AutoValue
+   * abstract class SomeDataClass {
+   *
+   *    abstract String getSomeString();
+   *    abstract Integer getSomeInteger();
+   *
+   *    @AutoValue.Builder
+   *    abstract static class Builder {
+   *      abstract Builder setSomeString(String value);
+   *      abstract Builder setSomeInteger(Integer value);
+   *
+   *      abstract SomeDataClass build();
+   *    }
+   * }
+   *
+   * // Pipeline example reads CSV string records from Google Cloud storage and writes to BigQuery.
+   * Pipeline pipeline = Pipeline.create();
+   *
+   * // Read CSV records from Google Cloud storage using TextIO.
+   * PCollection csvRecords = pipeline
+   *  .apply(TextIO.read().from("gs://bucket/folder/*.csv");
+   *
+   * // Apply the CSV records PCollection to the CsvIOParse transform instantiated using CsvIO.parse.
+   * CsvIOParseResult result = csvRecords.apply(CsvIO.parse(
+   *      SomeDataClass.class,
+   *      CsvFormat.DEFAULT.withHeader("someString", "someInteger")
+   * ));
+   *
+   * // Acquire any processing errors to either write to logs or apply to a downstream dead letter queue such as BigQuery.
+   * result.getErrors().apply(BigQueryIO.write()
+   *  .to("project:dataset.table_of_errors")
+   *  .useBeamSchema()
+   *  .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+   *  .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+   *
+   * // Acquire the successful PCollection output.
+   * PCollection output = result.getOutput();
+   *
+   * // Do something with the output such as write to BigQuery.
+   * output.apply(BigQueryIO.write()
+   *  .to("project:dataset.table_of_output")
+   *  .useBeamSchema()
+   *  .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+   *  .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+   * }
+ */ + public static CsvIOParse parse(Class klass, CSVFormat csvFormat) { + CsvIOParseHelpers.validateCsvFormat(csvFormat); + SchemaProvider provider = new DefaultSchema.DefaultSchemaProvider(); + TypeDescriptor type = TypeDescriptor.of(klass); + Schema schema = + checkStateNotNull( + provider.schemaFor(type), + "Illegal %s: Schema could not be generated from given %s class", + Schema.class, + klass); + CsvIOParseHelpers.validateCsvFormatWithSchema(csvFormat, schema); + SerializableFunction fromRowFn = + checkStateNotNull( + provider.fromRowFunction(type), + "FromRowFn could not be generated from the given %s class", + klass); + SerializableFunction toRowFn = + checkStateNotNull( + provider.toRowFunction(type), + "ToRowFn could not be generated from the given %s class", + klass); + SchemaCoder coder = SchemaCoder.of(schema, type, toRowFn, fromRowFn); + CsvIOParseConfiguration.Builder builder = CsvIOParseConfiguration.builder(); + builder.setCsvFormat(csvFormat).setSchema(schema).setCoder(coder).setFromRowFn(fromRowFn); + return CsvIOParse.builder().setConfigBuilder(builder).build(); + } + + /** + * Instantiates a {@link CsvIOParse} for parsing CSV string records into {@link Row}s from the + * records' assumed CsvFormat + * and expected {@link Schema}. + * + *

Example usage

+ * + * The example below illustrates parsing CsvFormat#DEFAULT + * formatted CSV string records, read from {@link TextIO.Read}, into a {@link Row} {@link + * PCollection}. + * + *
{@code
+   * // Define the expected Schema.
+   * Schema schema = Schema.of(
+   *  Schema.Field.of("someString", FieldType.STRING),
+   *  Schema.Field.of("someInteger", FieldType.INT32)
+   * );
+   *
+   * // Pipeline example reads CSV string records from Google Cloud storage and writes to BigQuery.
+   * Pipeline pipeline = Pipeline.create();
+   *
+   * // Read CSV records from Google Cloud storage using TextIO.
+   * PCollection csvRecords = pipeline
+   *  .apply(TextIO.read().from("gs://bucket/folder/*.csv");
+   *
+   * // Apply the CSV records PCollection to the CsvIOParse transform instantiated using CsvIO.parseRows.
+   * CsvIOParseResult result = csvRecords.apply(CsvIO.parseRow(
+   *      schema,
+   *      CsvFormat.DEFAULT.withHeader("someString", "someInteger")
+   * ));
+   *
+   * // Acquire any processing errors to either write to logs or apply to a downstream dead letter queue such as BigQuery.
+   * result.getErrors().apply(BigQueryIO.write()
+   *  .to("project:dataset.table_of_errors")
+   *  .useBeamSchema()
+   *  .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+   *  .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+   *
+   * // Acquire the successful PCollection output.
+   * PCollection output = result.getOutput();
+   *
+   * // Do something with the output such as write to BigQuery.
+   * output.apply(BigQueryIO.write()
+   *  .to("project:dataset.table_of_output")
+   *  .useBeamSchema()
+   *  .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+   *  .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+   * }
+ */ + public static CsvIOParse parseRows(Schema schema, CSVFormat csvFormat) { + CsvIOParseHelpers.validateCsvFormat(csvFormat); + CsvIOParseHelpers.validateCsvFormatWithSchema(csvFormat, schema); + RowCoder coder = RowCoder.of(schema); + CsvIOParseConfiguration.Builder builder = CsvIOParseConfiguration.builder(); + builder.setCsvFormat(csvFormat).setSchema(schema).setCoder(coder).setFromRowFn(row -> row); + return CsvIOParse.builder().setConfigBuilder(builder).build(); + } + /** {@link PTransform} for writing CSV files. */ @AutoValue public abstract static class Write extends PTransform, WriteFilesResult> diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java new file mode 100644 index 0000000000000..0a27cdbc57eca --- /dev/null +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.csv; + +import com.google.auto.value.AutoValue; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; + +/** + * {@link PTransform} for Parsing CSV Record Strings into {@link Schema}-mapped target types. {@link + * CsvIOParse} is not instantiated directly but via {@link CsvIO#parse} or {@link CsvIO#parseRows}. + */ +@AutoValue +public abstract class CsvIOParse extends PTransform, CsvIOParseResult> { + + final TupleTag outputTag = new TupleTag() {}; + final TupleTag errorTag = new TupleTag() {}; + + static CsvIOParse.Builder builder() { + return new AutoValue_CsvIOParse.Builder<>(); + } + + // TODO(https://github.com/apache/beam/issues/31875): Implement in future PR. + public CsvIOParse withCustomRecordParsing( + Map> customProcessingMap) { + return this; + } + + /** Contains all configuration parameters for {@link CsvIOParse}. */ + abstract CsvIOParseConfiguration.Builder getConfigBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setConfigBuilder(CsvIOParseConfiguration.Builder configBuilder); + + abstract CsvIOParse build(); + } + + @Override + public CsvIOParseResult expand(PCollection input) { + CsvIOParseConfiguration configuration = getConfigBuilder().build(); + + CsvIOStringToCsvRecord stringToCsvRecord = + new CsvIOStringToCsvRecord(configuration.getCsvFormat()); + CsvIOParseResult> stringToCsvRecordResult = input.apply(stringToCsvRecord); + PCollection> stringToRecordOutput = stringToCsvRecordResult.getOutput(); + PCollection stringToRecordErrors = stringToCsvRecordResult.getErrors(); + + CsvIORecordToObjects recordToObjects = new CsvIORecordToObjects(configuration); + CsvIOParseResult recordToObjectsResult = stringToRecordOutput.apply(recordToObjects); + PCollection output = recordToObjectsResult.getOutput(); + PCollection recordToObjectsErrors = recordToObjectsResult.getErrors(); + + PCollectionList errorList = + PCollectionList.of(stringToRecordErrors).and(recordToObjectsErrors); + PCollection errors = errorList.apply(Flatten.pCollections()); + + PCollectionTuple result = PCollectionTuple.of(outputTag, output).and(errorTag, errors); + return CsvIOParseResult.of(outputTag, configuration.getCoder(), errorTag, result); + } +} diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecord.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecord.java index 5fc4954cb450c..7fe0f5090d677 100644 --- a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecord.java +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecord.java @@ -84,7 +84,7 @@ public void process(@Element String line, MultiOutputReceiver receiver) { for (CSVRecord record : csvParser.getRecords()) { receiver.get(outputTag).output(csvRecordtoList(record)); } - } catch (IOException e) { + } catch (RuntimeException | IOException e) { receiver .get(errorTag) .output( diff --git a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java new file mode 100644 index 0000000000000..05d6982004f45 --- /dev/null +++ b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.csv; + +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NULLABLE_ALL_PRIMITIVE_DATA_TYPES_TYPE_DESCRIPTOR; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypes; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypesFromRowFn; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypesToRowFn; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NullableAllPrimitiveDataTypes; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.commons.csv.CSVFormat; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class CsvIOParseTest { + + private static final String[] HEADER = + new String[] {"aBoolean", "aDouble", "aFloat", "anInteger", "aLong", "aString"}; + private static final Coder + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_CODER = + SchemaCoder.of( + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_TYPE_DESCRIPTOR, + nullableAllPrimitiveDataTypesToRowFn(), + nullableAllPrimitiveDataTypesFromRowFn()); + private static final SerializableFunction ROW_ROW_SERIALIZABLE_FUNCTION = row -> row; + @Rule public final TestPipeline pipeline = TestPipeline.create(); + + @Test + public void isSerializable() throws Exception { + SerializableUtils.ensureSerializable(CsvIOParse.class); + } + + @Test + public void parseRows() { + PCollection records = + csvRecords( + pipeline, + "# This is a comment", + "aBoolean,aDouble,aFloat,anInteger,aLong,aString", + "true,1.0,2.0,3,4,foo", + "🏵,6.0,7.0,8,9,bar", + "false,12.0,14.0,8,24,\"foo\nbar\"", + "true,1.0,2.0,3,4,foo$,bar"); + List want = + Arrays.asList( + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", true) + .withFieldValue("aDouble", 1.0) + .withFieldValue("aFloat", 2.0f) + .withFieldValue("anInteger", 3) + .withFieldValue("aLong", 4L) + .withFieldValue("aString", "foo") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", null) + .withFieldValue("aDouble", 6.0) + .withFieldValue("aFloat", 7.0f) + .withFieldValue("anInteger", 8) + .withFieldValue("aLong", 9L) + .withFieldValue("aString", "bar") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", false) + .withFieldValue("aDouble", 12.0) + .withFieldValue("aFloat", 14.0f) + .withFieldValue("anInteger", 8) + .withFieldValue("aLong", 24L) + .withFieldValue("aString", "foo\nbar") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", true) + .withFieldValue("aDouble", 1.0) + .withFieldValue("aFloat", 2.0f) + .withFieldValue("anInteger", 3) + .withFieldValue("aLong", 4L) + .withFieldValue("aString", "foo,bar") + .build()); + + CsvIOParseResult result = + records.apply( + underTest( + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, + csvFormat(), + emptyCustomProcessingMap(), + ROW_ROW_SERIALIZABLE_FUNCTION, + RowCoder.of(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA))); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + @Test + public void parsePOJOs() { + PCollection records = + csvRecords( + pipeline, + "# This is a comment", + "aBoolean,aDouble,aFloat,anInteger,aLong,aString", + "true,1.0,2.0,3,4,foo", + "🏵,6.0,7.0,8,9,bar", + "false,12.0,14.0,8,24,\"foo\nbar\"", + "true,1.0,2.0,3,4,foo$,bar"); + List want = + Arrays.asList( + nullableAllPrimitiveDataTypes(true, 1.0d, 2.0f, 3, 4L, "foo"), + nullableAllPrimitiveDataTypes(null, 6.0d, 7.0f, 8, 9L, "bar"), + nullableAllPrimitiveDataTypes(false, 12.0d, 14.0f, 8, 24L, "foo\nbar"), + nullableAllPrimitiveDataTypes(true, 1.0d, 2.0f, 3, 4L, "foo,bar")); + + CsvIOParseResult result = + records.apply( + underTest( + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, + csvFormat(), + emptyCustomProcessingMap(), + nullableAllPrimitiveDataTypesFromRowFn(), + NULLABLE_ALL_PRIMITIVE_DATA_TYPES_CODER)); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + private static CSVFormat csvFormat() { + return CSVFormat.DEFAULT + .withAllowDuplicateHeaderNames(false) + .withHeader(HEADER) + .withCommentMarker('#') + .withNullString("🏵") + .withEscape('$'); + } + + private static PCollection csvRecords(Pipeline pipeline, String... lines) { + return pipeline.apply( + Create.of(Arrays.asList(lines)).withCoder(NullableCoder.of(StringUtf8Coder.of()))); + } + + private static CsvIOParse underTest( + Schema schema, + CSVFormat csvFormat, + Map> customProcessingMap, + SerializableFunction fromRowFn, + Coder coder) { + CsvIOParseConfiguration.Builder configBuilder = + CsvIOParseConfiguration.builder() + .setSchema(schema) + .setCsvFormat(csvFormat) + .setCustomProcessingMap(customProcessingMap) + .setFromRowFn(fromRowFn) + .setCoder(coder); + return CsvIOParse.builder().setConfigBuilder(configBuilder).build(); + } + + private static Map> emptyCustomProcessingMap() { + return new HashMap<>(); + } +} diff --git a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecordTest.java b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecordTest.java index 7cbba3335dd28..1618962ef394b 100644 --- a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecordTest.java +++ b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOStringToCsvRecordTest.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.apache.commons.csv.CSVFormat; @@ -547,6 +548,17 @@ public void testMultiLineCsvRecord() { pipeline.run(); } + @Test + public void givenInvalidCsvRecord_throws() { + CSVFormat csvFormat = csvFormat().withQuote('"'); + PCollection input = + pipeline.apply(Create.of(headerLine(csvFormat), "a,\"1,1.1", "b,2,2.2", "c,3,3.3")); + CsvIOStringToCsvRecord underTest = new CsvIOStringToCsvRecord(csvFormat); + CsvIOParseResult> result = input.apply(underTest); + PAssert.thatSingleton(result.getErrors().apply(Count.globally())).isEqualTo(1L); + pipeline.run(); + } + private static CSVFormat csvFormat() { return CSVFormat.DEFAULT.withAllowDuplicateHeaderNames(false).withHeader(header); } diff --git a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOTest.java b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOTest.java new file mode 100644 index 0000000000000..13e09725e952d --- /dev/null +++ b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOTest.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.csv; + +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypes; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertThrows; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NullableAllPrimitiveDataTypes; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.commons.csv.CSVFormat; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class CsvIOTest { + private static final String[] HEADER = + new String[] {"aBoolean", "aDouble", "aFloat", "anInteger", "aLong", "aString"}; + + @Test + public void parseRows() { + Pipeline pipeline = Pipeline.create(); + PCollection input = + csvRecords( + pipeline, + "# This is a comment", + "aBoolean,aDouble,aFloat,anInteger,aLong,aString", + "true,1.0,2.0,3,4,foo", + "N/A,6.0,7.0,8,9,bar", + "false,12.0,14.0,8,24,\"foo\nbar\"", + "true,1.0,2.0,3,4,foo$,bar"); + List want = + Arrays.asList( + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", true) + .withFieldValue("aDouble", 1.0) + .withFieldValue("aFloat", 2.0f) + .withFieldValue("anInteger", 3) + .withFieldValue("aLong", 4L) + .withFieldValue("aString", "foo") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", null) + .withFieldValue("aDouble", 6.0) + .withFieldValue("aFloat", 7.0f) + .withFieldValue("anInteger", 8) + .withFieldValue("aLong", 9L) + .withFieldValue("aString", "bar") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", false) + .withFieldValue("aDouble", 12.0) + .withFieldValue("aFloat", 14.0f) + .withFieldValue("anInteger", 8) + .withFieldValue("aLong", 24L) + .withFieldValue("aString", "foo\nbar") + .build(), + Row.withSchema(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA) + .withFieldValue("aBoolean", true) + .withFieldValue("aDouble", 1.0) + .withFieldValue("aFloat", 2.0f) + .withFieldValue("anInteger", 3) + .withFieldValue("aLong", 4L) + .withFieldValue("aString", "foo,bar") + .build()); + + CsvIOParse underTest = + CsvIO.parseRows(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, csvFormat()); + CsvIOParseResult result = input.apply(underTest); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + @Test + public void parsesPOJOs() { + Pipeline pipeline = Pipeline.create(); + PCollection input = + csvRecords( + pipeline, + "# This is a comment", + "aBoolean,aDouble,aFloat,anInteger,aLong,aString", + "true,1.0,2.0,3,4,foo", + "N/A,6.0,7.0,8,9,bar", + "false,12.0,14.0,8,24,\"foo\nbar\"", + "true,1.0,2.0,3,4,foo$,bar"); + List want = + Arrays.asList( + nullableAllPrimitiveDataTypes(true, 1.0d, 2.0f, 3, 4L, "foo"), + nullableAllPrimitiveDataTypes(null, 6.0d, 7.0f, 8, 9L, "bar"), + nullableAllPrimitiveDataTypes(false, 12.0d, 14.0f, 8, 24L, "foo\nbar"), + nullableAllPrimitiveDataTypes(true, 1.0d, 2.0f, 3, 4L, "foo,bar")); + + CsvIOParse underTest = + CsvIO.parse(NullableAllPrimitiveDataTypes.class, csvFormat()); + CsvIOParseResult result = input.apply(underTest); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + @Test + public void givenInvalidCsvFormat_throws() { + Pipeline pipeline = Pipeline.create(); + CSVFormat csvFormat = + CSVFormat.DEFAULT + .withHeader("a_string", "an_integer", "a_double") + .withAllowDuplicateHeaderNames(true); + Schema schema = + Schema.builder() + .addStringField("a_string") + .addInt32Field("an_integer") + .addDoubleField("a_double") + .build(); + assertThrows(IllegalArgumentException.class, () -> CsvIO.parseRows(schema, csvFormat)); + pipeline.run(); + } + + @Test + public void givenMismatchedCsvFormatAndSchema_throws() { + Pipeline pipeline = Pipeline.create(); + CSVFormat csvFormat = + CSVFormat.DEFAULT + .withHeader("a_string", "an_integer", "a_double") + .withAllowDuplicateHeaderNames(true); + Schema schema = Schema.builder().addStringField("a_string").addDoubleField("a_double").build(); + assertThrows(IllegalArgumentException.class, () -> CsvIO.parseRows(schema, csvFormat)); + pipeline.run(); + } + + @Test + public void givenNullSchema_throws() { + Pipeline pipeline = Pipeline.create(); + assertThrows(NullPointerException.class, () -> CsvIO.parseRows(null, csvFormat())); + pipeline.run(); + } + + @Test + public void givenNonSchemaMappedClass_throws() { + Pipeline pipeline = Pipeline.create(); + CSVFormat csvFormat = + CSVFormat.DEFAULT + .withHeader("a_string", "an_integer", "a_double") + .withAllowDuplicateHeaderNames(false); + assertThrows( + IllegalStateException.class, () -> CsvIO.parse(NonSchemaMappedPojo.class, csvFormat)); + pipeline.run(); + } + + @Test + public void givenStringToRecordError_emits() { + Pipeline pipeline = Pipeline.create(); + PCollection input = pipeline.apply(Create.of("true,\"1.1,3.141592,1,5,foo")); + Schema schema = + Schema.builder() + .addBooleanField("aBoolean") + .addDoubleField("aDouble") + .addFloatField("aFloat") + .addInt32Field("anInteger") + .addInt64Field("aLong") + .addStringField("aString") + .build(); + CsvIOParse underTest = CsvIO.parseRows(schema, csvFormat().withQuote('"')); + CsvIOParseResult result = input.apply(underTest); + PAssert.thatSingleton(result.getErrors().apply("Total Errors", Count.globally())).isEqualTo(1L); + PAssert.thatSingleton( + stackTraceContains(result.getErrors(), CsvIOStringToCsvRecord.class.getName())) + .isEqualTo(1L); + + pipeline.run(); + } + + @Test + public void givenRecordToObjectError_emits() { + Pipeline pipeline = Pipeline.create(); + PCollection input = + pipeline.apply(Create.of("true,1.1,3.141592,this_is_an_error,5,foo")); + Schema schema = + Schema.builder() + .addBooleanField("aBoolean") + .addDoubleField("aDouble") + .addFloatField("aFloat") + .addInt32Field("anInteger") + .addInt64Field("aLong") + .addStringField("aString") + .build(); + CsvIOParse underTest = CsvIO.parseRows(schema, csvFormat().withQuote('"')); + CsvIOParseResult result = input.apply(underTest); + PAssert.thatSingleton(result.getErrors().apply(Count.globally())).isEqualTo(1L); + PAssert.thatSingleton( + stackTraceContains(result.getErrors(), CsvIORecordToObjects.class.getName())) + .isEqualTo(1L); + pipeline.run(); + } + + @Test + public void givenStringToRecordError_RecordToObjectError_emits() { + Pipeline pipeline = Pipeline.create(); + PCollection input = + pipeline.apply( + Create.of("true,\"1.1,3.141592,1,5,foo", "true,1.1,3.141592,this_is_an_error,5,foo")); + Schema schema = + Schema.builder() + .addBooleanField("aBoolean") + .addDoubleField("aDouble") + .addFloatField("aFloat") + .addInt32Field("anInteger") + .addInt64Field("aLong") + .addStringField("aString") + .build(); + CsvIOParse underTest = CsvIO.parseRows(schema, csvFormat().withQuote('"')); + CsvIOParseResult result = input.apply(underTest); + PAssert.thatSingleton(result.getErrors().apply(Count.globally())).isEqualTo(2L); + PAssert.thatSingleton( + stackTraceContains(result.getErrors(), CsvIOStringToCsvRecord.class.getName())) + .isEqualTo(1L); + PAssert.thatSingleton( + stackTraceContains(result.getErrors(), CsvIORecordToObjects.class.getName())) + .isEqualTo(1L); + + pipeline.run(); + } + + private static PCollection stackTraceContains( + PCollection errors, String match) { + return errors + .apply(match, Filter.by(input -> checkStateNotNull(input).getStackTrace().contains(match))) + .apply(match, Count.globally()); + } + + private static CSVFormat csvFormat() { + return CSVFormat.DEFAULT + .withAllowDuplicateHeaderNames(false) + .withHeader(HEADER) + .withCommentMarker('#') + .withNullString("N/A") + .withEscape('$'); + } + + private static PCollection csvRecords(Pipeline pipeline, String... lines) { + return pipeline.apply( + Create.of(Arrays.asList(lines)).withCoder(NullableCoder.of(StringUtf8Coder.of()))); + } + + private static class NonSchemaMappedPojo implements Serializable { + private final String aString; + private final Integer anInteger; + private final Double aDouble; + + private NonSchemaMappedPojo(String aString, Integer anInteger, Double aDouble) { + this.aString = aString; + this.anInteger = anInteger; + this.aDouble = aDouble; + } + + public String getAString() { + return aString; + } + + public Integer getAnInteger() { + return anInteger; + } + + public Double getADouble() { + return aDouble; + } + } +} From b795a61f09432f6ead573f15aa99436865565ec8 Mon Sep 17 00:00:00 2001 From: Andrew Crites Date: Thu, 1 Aug 2024 16:01:35 -0700 Subject: [PATCH 02/78] Adds null checks when accessing OperationalLimits in config since they might not have been set yet. (#32053) --- .../config/StreamingEngineComputationConfigFetcher.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java index 850e8c3f24bdc..d230aac54c636 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java @@ -188,11 +188,13 @@ private StreamingEnginePipelineConfig createPipelineConfig(StreamingConfigTask c } if (config.getOperationalLimits() != null) { - if (config.getOperationalLimits().getMaxKeyBytes() > 0 + if (config.getOperationalLimits().getMaxKeyBytes() != null + && config.getOperationalLimits().getMaxKeyBytes() > 0 && config.getOperationalLimits().getMaxKeyBytes() <= Integer.MAX_VALUE) { pipelineConfig.setMaxOutputKeyBytes(config.getOperationalLimits().getMaxKeyBytes()); } - if (config.getOperationalLimits().getMaxProductionOutputBytes() > 0 + if (config.getOperationalLimits().getMaxProductionOutputBytes() != null + && config.getOperationalLimits().getMaxProductionOutputBytes() > 0 && config.getOperationalLimits().getMaxProductionOutputBytes() <= Integer.MAX_VALUE) { pipelineConfig.setMaxOutputValueBytes( config.getOperationalLimits().getMaxProductionOutputBytes()); From 202fa56be771167495cbcc3fd84e60fcc77147c9 Mon Sep 17 00:00:00 2001 From: Damon Date: Thu, 1 Aug 2024 18:00:17 -0700 Subject: [PATCH 03/78] Enable ExternalWorkerService during Prism Runner lifecycle (#32057) --- runners/prism/java/build.gradle | 2 + .../beam/runners/prism/WorkerService.java | 116 ++++++++++++++++++ .../beam/runners/prism/WorkerServiceTest.java | 85 +++++++++++++ 3 files changed, 203 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/WorkerService.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/WorkerServiceTest.java diff --git a/runners/prism/java/build.gradle b/runners/prism/java/build.gradle index 23f4a024569b4..2b0635ca61255 100644 --- a/runners/prism/java/build.gradle +++ b/runners/prism/java/build.gradle @@ -27,7 +27,9 @@ ext.summary = "Support for executing a pipeline on Prism." dependencies { implementation project(path: ":model:job-management", configuration: "shadow") + implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(path: ":sdks:java:harness", configuration: "shadow") implementation project(":runners:portability:java") implementation library.java.joda_time diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/WorkerService.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/WorkerService.java new file mode 100644 index 0000000000000..289ffac64f8a4 --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/WorkerService.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import org.apache.beam.fn.harness.ExternalWorkerService; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.fn.server.GrpcFnServer; +import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An {@link ExternalWorkerService} {@link GrpcFnServer} encapsulation that {@link #stop}s when + * {@link StateListener#onStateChanged} is invoked with a {@link PipelineResult.State} that is + * {@link PipelineResult.State#isTerminal}. + */ +class WorkerService implements StateListener { + + private static final Logger LOG = LoggerFactory.getLogger(WorkerService.class); + + private final ExternalWorkerService worker; + private @MonotonicNonNull GrpcFnServer server; + + WorkerService(PortablePipelineOptions options) { + this.worker = new ExternalWorkerService(options); + } + + /** Start the {@link ExternalWorkerService}. */ + void start() throws Exception { + if (server != null && !server.getServer().isShutdown()) { + return; + } + + server = worker.start(); + LOG.info("Starting worker service at {}", getApiServiceDescriptorUrl()); + } + + /** + * Queries whether the {@link ExternalWorkerService} {@link GrpcFnServer}'s {@link Server} is + * running. + */ + boolean isRunning() { + if (server == null) { + return false; + } + return !server.getServer().isShutdown(); + } + + /** + * Queries the {@link Endpoints.ApiServiceDescriptor#getUrl} of the {@link ExternalWorkerService} + * {@link GrpcFnServer}'s {@link Server}. Throws an exception if the {@link WorkerService} has not + * {@link WorkerService#start}ed. + */ + String getApiServiceDescriptorUrl() { + return checkStateNotNull(server, "worker service not started") + .getApiServiceDescriptor() + .getUrl(); + } + + /** + * Updates {@link PortablePipelineOptions#getDefaultEnvironmentConfig} with {@link + * #getApiServiceDescriptorUrl}. Throws an exception if the {@link WorkerService} has not {@link + * WorkerService#start}ed. + */ + PortablePipelineOptions updateDefaultEnvironmentConfig(PortablePipelineOptions options) { + options.setDefaultEnvironmentConfig(getApiServiceDescriptorUrl()); + return options; + } + + /** + * Overrides {@link StateListener#onStateChanged}, invoking {@link #stop} when {@link + * PipelineResult.State#isTerminal}. + */ + @Override + public void onStateChanged(PipelineResult.State state) { + if (state.isTerminal()) { + stop(); + } + } + + /** + * Stops the {@link ExternalWorkerService} {@link GrpcFnServer}'s {@link Server}. If not {@link + * WorkerService#isRunning()}, then calling stop is a noop. + */ + void stop() { + if (server == null || server.getServer().isShutdown()) { + return; + } + LOG.info("Stopping worker service at {}", getApiServiceDescriptorUrl()); + try { + server.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/WorkerServiceTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/WorkerServiceTest.java new file mode 100644 index 0000000000000..7fc05d7747cd6 --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/WorkerServiceTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link WorkerService}. */ +@RunWith(JUnit4.class) +public class WorkerServiceTest { + @Test + public void testStartStop() throws Exception { + PortablePipelineOptions options = + PipelineOptionsFactory.create().as(PortablePipelineOptions.class); + WorkerService underTest = new WorkerService(options); + underTest.start(); + assertThat(underTest.isRunning()).isTrue(); + assertThat(underTest.getApiServiceDescriptorUrl()).matches("localhost:\\d+"); + underTest.stop(); + assertThat(underTest.isRunning()).isFalse(); + } + + @Test + public void givenStarted_updateDefaultEnvironmentConfig() throws Exception { + PortablePipelineOptions options = + PipelineOptionsFactory.create().as(PortablePipelineOptions.class); + assertThat(options.getDefaultEnvironmentConfig()).isNull(); + WorkerService underTest = new WorkerService(options); + underTest.start(); + options = underTest.updateDefaultEnvironmentConfig(options); + assertThat(options.getDefaultEnvironmentConfig()) + .isEqualTo(underTest.getApiServiceDescriptorUrl()); + underTest.stop(); + } + + @Test + public void givenNotStarted_updateDefaultEnvironmentConfig_throws() { + PortablePipelineOptions options = + PipelineOptionsFactory.create().as(PortablePipelineOptions.class); + WorkerService underTest = new WorkerService(options); + assertThrows( + IllegalStateException.class, () -> underTest.updateDefaultEnvironmentConfig(options)); + } + + @Test + public void whenStateIsTerminal_thenStop() throws Exception { + PortablePipelineOptions options = + PipelineOptionsFactory.create().as(PortablePipelineOptions.class); + WorkerService underTest = new WorkerService(options); + assertThat(underTest.isRunning()).isFalse(); + underTest.start(); + assertThat(underTest.isRunning()).isTrue(); + + underTest.onStateChanged(PipelineResult.State.RUNNING); + assertThat(underTest.isRunning()).isTrue(); + + underTest.onStateChanged(PipelineResult.State.RUNNING); + assertThat(underTest.isRunning()).isTrue(); + + underTest.onStateChanged(PipelineResult.State.CANCELLED); + assertThat(underTest.isRunning()).isFalse(); + } +} From 0b4b8ea9423dce255b38f2e2533307b1930fbd13 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 2 Aug 2024 09:52:16 -0400 Subject: [PATCH 04/78] Handle rc container in _update_container_image_for_dataflow (#32049) --- .../runners/dataflow/internal/apiclient.py | 6 +++ .../dataflow/internal/apiclient_test.py | 37 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index badc3683bb285..20cae582f320d 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -733,6 +733,12 @@ def _update_container_image_for_dataflow(beam_container_image_url): # By default Dataflow pipelines use containers hosted in Dataflow GCR # instead of Docker Hub. image_suffix = beam_container_image_url.rsplit('/', 1)[1] + + # trim "RCX" as release candidate tag exists on Docker Hub but not GCR + check_rc = image_suffix.lower().split('rc') + if len(check_rc) == 2: + image_suffix = image_suffix[:-2 - len(check_rc[1])] + return names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/' + image_suffix @staticmethod diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py index a4e1a5253a736..8331d9cf3919a 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py @@ -321,6 +321,43 @@ def test_dataflow_container_image_override_prime(self): self._verify_dataflow_container_image_override(pipeline_options) + def _verify_dataflow_container_image_override_rc(self, pipeline_options): + pipeline = Pipeline(options=pipeline_options) + pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned + + dummy_env = DockerEnvironment( + container_image='apache/beam_dummy_name:2.00.0RC10') + proto_pipeline, _ = pipeline.to_runner_api( + return_context=True, default_environment=dummy_env) + + # Accessing non-public method for testing. + apiclient.DataflowApplicationClient._apply_sdk_environment_overrides( + proto_pipeline, {}, pipeline_options) + + from apache_beam.utils import proto_utils + found_override = False + trimed_rc = True + for env in proto_pipeline.components.environments.values(): + docker_payload = proto_utils.parse_Bytes( + env.payload, beam_runner_api_pb2.DockerPayload) + if docker_payload.container_image.startswith( + names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY): + found_override = True + if docker_payload.container_image.split(':')[-1] != '2.00.0': + trimed_rc = False + + self.assertTrue(found_override) + self.assertTrue(trimed_rc) + + def test_dataflow_container_image_override_rc(self): + pipeline_options = PipelineOptions([ + '--experiments=use_runner_v2', + '--temp_location', + 'gs://any-location/temp' + ]) + + self._verify_dataflow_container_image_override_rc(pipeline_options) + def _verify_non_apache_container_not_overridden(self, pipeline_options): pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned From d96fa7d4009a9638e0cdfc8b107710eaf8362f00 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 2 Aug 2024 17:04:40 +0200 Subject: [PATCH 05/78] Add some large model troubleshooting steps (#31862) --- .../ml/large-language-modeling.md | 50 ++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/ml/large-language-modeling.md b/website/www/site/content/en/documentation/ml/large-language-modeling.md index 79ef58e6de319..90bbd43383c06 100644 --- a/website/www/site/content/en/documentation/ml/large-language-modeling.md +++ b/website/www/site/content/en/documentation/ml/large-language-modeling.md @@ -27,7 +27,7 @@ RunInference has several mechanisms for reducing memory utilization. For example Many Beam runners, however, run multiple Beam processes per machine at once. This can cause problems since the memory footprint of loading large models like LLMs multiple times can be too large to fit into a single machine. For memory-intensive models, RunInference provides a mechanism for more intelligently sharing memory across multiple processes to reduce the overall memory footprint. To enable this mode, users just have -to set the parameter `large_model` to True in their model configuration (see below for an example), and Beam will take care of the memory management. +to set the parameter `large_model` to True in their model configuration (see below for an example), and Beam will take care of the memory management. When using a custom model handler, you can override the `share_model_across_processes` function or the `model_copies` function for a similar effect. ### Running an Example Pipeline with T5 @@ -122,3 +122,51 @@ A `ModelHandler` requires parameters like: * `device` – The device on which you wish to run the model. If device = GPU then a GPU device will be used if it is available. Otherwise, it will be CPU. * `inference_fn` - The inference function to use during RunInference. * `large_model` - (see `Memory Management` above). Whether to use memory minimization techniques to lower the memory footprint of your model. + +### Troubleshooting Large Models + +#### Pickling errors + +When sharing a model across processes with `large_model=True` or using a custom model handler, Beam sends the input and output data across a process boundary. +To do this, it uses a serialization method known as [pickling](https://docs.python.org/3/library/pickle.html). +For example, if you call `output=model.my_inference_fn(input_1, input_2)`, `input_1`, `input_2`, and `output` will all need to be pickled. +The model itself does not need to be pickled since it is not passed across process boundaries. + +While most objects can be pickled without issue, if one of these objects is unpickleable you may run into errors like `error: can't pickle fasttext_pybind.fasttext objects`. +To work around this, there are a few options: + +First of all, if possible you can choose not to share your model across processes. This will incur additional memory pressure, but it may be tolerable in some cases. + +Second, using a custom model handler you can wrap your model to take in and return serializable types. For example, if your model handler looks like: + +``` +class MyModelHandler(): + def load_model(self): + return model_loading_logic() + + def run_inference(self, batch: Sequence[str], model, inference_args): + unpickleable_object = Unpickleable(batch) + unpickleable_returned = model.predict(unpickleable_object) + my_output = int(unpickleable_returned[0]) + return my_output +``` + +you could instead wrap the unpickleable pieces in a model wrapper. Since the model wrapper will sit in the inference process, this will work as long as it only takes in/returns pickleable objects. + +``` +class MyWrapper(): + def __init__(self, model): + self._model = model + + def predict(self, batch: Sequence[str]): + unpickleable_object = Unpickleable(batch) + unpickleable_returned = model.predict(unpickleable_object) + return int(prediction[0]) + +class MyModelHandler(): + def load_model(self): + return MyWrapper(model_loading_logic()) + + def run_inference(self, batch: Sequence[str], model: MyWrapper, inference_args): + return model.predict(unpickleable_object) +``` From bf42a8153af582e4dd97140bebf1a829f35dfe20 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 2 Aug 2024 14:12:26 -0700 Subject: [PATCH 06/78] [#32064] Keep elements heap in sequence order. (#32065) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../prism/internal/engine/elementmanager.go | 15 ++++++++++++++- .../beam/runners/prism/internal/engine/timers.go | 1 + 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index a632318e02c7e..bc8449c72b39a 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -45,6 +45,14 @@ type element struct { holdTimestamp mtime.Time // only used for Timers pane typex.PaneInfo transform, family, tag string // only used for Timers. + // Used to ensure ordering within a key when sorting the heap, + // which isn't using a stable sort. + // Since ordering is weak across multiple bundles, it needs only + // be consistent between exiting a stage and entering a stateful stage. + // No synchronization is required in specifying this, + // since keyed elements are only processed by a single bundle at a time, + // if stateful stages are concerned. + sequence int elmBytes []byte // When nil, indicates this is a timer. keyBytes []byte @@ -103,7 +111,8 @@ func (h elementHeap) Less(i, j int) bool { } else if h[i].IsData() && h[j].IsTimer() { return true // i before j. } - // They're the same kind, fall through to timestamp less for consistency. + // They're the same kind, so compare by the sequence value. + return h[i].sequence < h[j].sequence } // Otherwise compare by timestamp. return h[i].timestamp < h[j].timestamp @@ -688,6 +697,7 @@ func reElementResiduals(residuals []Residual, inputInfo PColInfo, rb RunBundle) pane: pn, elmBytes: elmBytes, keyBytes: keyBytes, + sequence: len(unprocessedElements), }) } } @@ -704,6 +714,7 @@ func reElementResiduals(residuals []Residual, inputInfo PColInfo, rb RunBundle) // PersistBundle takes in the stage ID, ID of the bundle associated with the pending // input elements, and the committed output elements. func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PColInfo, d TentativeData, inputInfo PColInfo, residuals Residuals) { + var seq int for output, data := range d.Raw { info := col2Coders[output] var newPending []element @@ -743,7 +754,9 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol pane: pn, elmBytes: elmBytes, keyBytes: keyBytes, + sequence: seq, }) + seq++ } } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go index 3f52ebc4510cf..787d27858a0e5 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go @@ -74,6 +74,7 @@ func decodeTimer(keyDec func(io.Reader) []byte, usesGlobalWindow bool, raw []byt timestamp: firing, holdTimestamp: hold, pane: pane, + sequence: len(ret), }) } return keyBytes, tag, ret From 7e7508731521862097c6e6230ac4bb6c53477965 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Sat, 3 Aug 2024 22:06:12 -0700 Subject: [PATCH 07/78] Update top_wikipedia_sessions to be more idiomatic with beam.Map. (#32041) * Update top_wikipedia_sessions to be more idiomatic with beam.Map. * lint --- .../complete/top_wikipedia_sessions.py | 34 ++++++++----------- .../complete/top_wikipedia_sessions_test.py | 2 ++ 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py index 7064a5add13c7..50b026edf2402 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py @@ -61,14 +61,13 @@ MAX_TIMESTAMP = 0x7fffffffffffffff -class ExtractUserAndTimestampDoFn(beam.DoFn): +def extract_user_and_timestamp(element): """Extracts user and timestamp representing a Wikipedia edit.""" - def process(self, element): - table_row = json.loads(element) - if 'contributor_username' in table_row: - user_name = table_row['contributor_username'] - timestamp = table_row['timestamp'] - yield TimestampedValue(user_name, timestamp) + table_row = json.loads(element) + if 'contributor_username' in table_row: + user_name = table_row['contributor_username'] + timestamp = table_row['timestamp'] + return TimestampedValue(user_name, timestamp) class ComputeSessions(beam.PTransform): @@ -98,19 +97,15 @@ def expand(self, pcoll): without_defaults()) -class SessionsToStringsDoFn(beam.DoFn): +def sessions_to_strings(element, window=beam.DoFn.WindowParam): """Adds the session information to be part of the key.""" - def process(self, element, window=beam.DoFn.WindowParam): - yield (element[0] + ' : ' + str(window), element[1]) + return (element[0] + ' : ' + str(window), element[1]) -class FormatOutputDoFn(beam.DoFn): +def format_output(element, window=beam.DoFn.WindowParam): """Formats a string containing the user, count, and session.""" - def process(self, element, window=beam.DoFn.WindowParam): - for kv in element: - session = kv[0] - count = kv[1] - yield session + ' : ' + str(count) + ' : ' + str(window) + for session, count in element: + yield session + ' : ' + str(count) + ' : ' + str(window) class ComputeTopSessions(beam.PTransform): @@ -124,14 +119,13 @@ def __init__(self, sampling_threshold): def expand(self, pcoll): return ( pcoll - | - 'ExtractUserAndTimestamp' >> beam.ParDo(ExtractUserAndTimestampDoFn()) + | 'ExtractUserAndTimestamp' >> beam.Map(extract_user_and_timestamp) | beam.Filter( lambda x: (abs(hash(x)) <= MAX_TIMESTAMP * self.sampling_threshold)) | ComputeSessions() - | 'SessionsToStrings' >> beam.ParDo(SessionsToStringsDoFn()) + | 'SessionsToStrings' >> beam.Map(sessions_to_strings) | TopPerMonth() - | 'FormatOutput' >> beam.ParDo(FormatOutputDoFn())) + | 'FormatOutput' >> beam.FlatMap(format_output)) def run(argv=None): diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py index 3c171664e45d2..92d1d196fe055 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py @@ -28,6 +28,8 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +# TODO: Unit test top_wikipedia_sessions.extract_user_and_timestamp, etc. + class ComputeTopSessionsTest(unittest.TestCase): From ca744ae9f6511c58ee08c391bc06e63c0f90ecce Mon Sep 17 00:00:00 2001 From: martin trieu Date: Mon, 5 Aug 2024 16:04:28 +0700 Subject: [PATCH 08/78] Add WorkProvider interfaces and implementations (#31883) --- .../worker/StreamingDataflowWorker.java | 342 +++++++----------- .../FanOutStreamingEngineWorkerHarness.java} | 63 ++-- .../harness/SingleSourceWorkerHarness.java | 284 +++++++++++++++ .../StreamingEngineConnectionState.java | 2 +- .../harness/StreamingWorkerHarness.java | 28 ++ .../harness}/WindmillStreamSender.java | 20 +- .../windmill/client/grpc/ChannelzServlet.java | 27 +- .../grpc/GetWorkResponseChunkAssembler.java | 139 +++++++ .../client/grpc/GrpcDirectGetWorkStream.java | 148 +++----- .../client/grpc/GrpcDispatcherClient.java | 4 +- .../client/grpc/GrpcGetWorkStream.java | 131 ++----- .../windmill/work/WorkItemScheduler.java | 4 - .../budget/EvenGetWorkBudgetDistributor.java | 27 +- .../work/budget/GetWorkBudgetDistributor.java | 5 +- .../work/budget/GetWorkBudgetSpender.java | 32 ++ .../worker/StreamingDataflowWorkerTest.java | 2 +- ...nOutStreamingEngineWorkerHarnessTest.java} | 40 +- .../harness}/WindmillStreamSenderTest.java | 5 +- .../client/grpc/ChannelzServletTest.java | 6 +- .../client/grpc/GrpcWindmillServerTest.java | 12 +- .../EvenGetWorkBudgetDistributorTest.java | 126 +++---- 21 files changed, 839 insertions(+), 608 deletions(-) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc/StreamingEngineClient.java => streaming/harness/FanOutStreamingEngineWorkerHarness.java} (91%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/StreamingEngineConnectionState.java (97%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/WindmillStreamSender.java (93%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc/StreamingEngineClientTest.java => streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java} (93%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/WindmillStreamSenderTest.java (97%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index f196852b22532..90f072be997ed 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -23,14 +23,13 @@ import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.MapTask; import com.google.auto.value.AutoValue; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -48,26 +47,25 @@ import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ComputationStateCache; import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; -import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; -import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.streaming.WorkHeartbeatResponseProcessor; import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingApplianceComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEngineComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness; +import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness.GetWorkSender; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; +import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerHarness; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerStatusPages; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerStatusReporter; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.appliance.JniWindmillApplianceServer; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.CompleteCommit; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.StreamingApplianceWorkCommitter; @@ -104,12 +102,11 @@ import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheStats; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -119,7 +116,8 @@ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -public class StreamingDataflowWorker { +public final class StreamingDataflowWorker { + /** * Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked * (across all the sinks, if there are more than one) reaches this limit. This serves as hint for @@ -128,47 +126,44 @@ public class StreamingDataflowWorker { */ public static final int MAX_SINK_BYTES = 10_000_000; - // Maximum number of threads for processing. Currently, each thread processes one key at a time. - static final int MAX_PROCESSING_THREADS = 300; - static final long THREAD_EXPIRATION_TIME_SEC = 60; - static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; - static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); - private static final Duration GET_DATA_STREAM_TIMEOUT = Duration.standardSeconds(30); + + /** + * Maximum number of threads for processing. Currently, each thread processes one key at a time. + */ + private static final int MAX_PROCESSING_THREADS = 300; /** The idGenerator to generate unique id globally. */ private static final IdGenerator ID_GENERATOR = IdGenerators.decrementingLongs(); - private static final int DEFAULT_STATUS_PORT = 8081; - // Maximum size of the result of a GetWork request. + /** Maximum size of the result of a GetWork request. */ private static final long MAX_GET_WORK_FETCH_BYTES = 64L << 20; // 64m /** Maximum number of failure stacktraces to report in each update sent to backend. */ private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000; - private static final Random clientIdGenerator = new Random(); + private static final long THREAD_EXPIRATION_TIME_SEC = 60; + private static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); + private static final Duration GET_DATA_STREAM_TIMEOUT = Duration.standardSeconds(30); + private static final int DEFAULT_STATUS_PORT = 8081; + private static final Random CLIENT_ID_GENERATOR = new Random(); private static final String CHANNELZ_PATH = "/channelz"; - final WindmillStateCache stateCache; + + private final WindmillStateCache stateCache; private final StreamingWorkerStatusPages statusPages; private final ComputationConfig.Fetcher configFetcher; private final ComputationStateCache computationStateCache; private final BoundedQueueExecutor workUnitExecutor; - private final WindmillServerStub windmillServer; - private final Thread dispatchThread; + private final StreamingWorkerHarness streamingWorkerHarness; private final AtomicBoolean running = new AtomicBoolean(); private final DataflowWorkerHarnessOptions options; - private final long clientId; - private final GetDataClient getDataClient; - private final MemoryMonitor memoryMonitor; - private final Thread memoryMonitorThread; + private final BackgroundMemoryMonitor memoryMonitor; private final ReaderCache readerCache; private final DataflowExecutionStateSampler sampler = DataflowExecutionStateSampler.instance(); private final ActiveWorkRefresher activeWorkRefresher; private final WorkCommitter workCommitter; private final StreamingWorkerStatusReporter workerStatusReporter; private final StreamingCounters streamingCounters; - private final StreamingWorkScheduler streamingWorkScheduler; - private final HeartbeatSender heartbeatSender; private StreamingDataflowWorker( WindmillServerStub windmillServer, @@ -226,39 +221,42 @@ private StreamingDataflowWorker( this.workUnitExecutor = workUnitExecutor; - memoryMonitorThread = new Thread(memoryMonitor); - memoryMonitorThread.setPriority(Thread.MIN_PRIORITY); - memoryMonitorThread.setName("MemoryMonitor"); - - dispatchThread = - new Thread( - () -> { - LOG.info("Dispatch starting"); - if (windmillServiceEnabled) { - streamingDispatchLoop(); - } else { - dispatchLoop(); - } - LOG.info("Dispatch done"); - }); - dispatchThread.setDaemon(true); - dispatchThread.setPriority(Thread.MIN_PRIORITY); - dispatchThread.setName("DispatchThread"); - this.clientId = clientId; - this.windmillServer = windmillServer; + this.workerStatusReporter = workerStatusReporter; + this.streamingCounters = streamingCounters; + this.memoryMonitor = BackgroundMemoryMonitor.create(memoryMonitor); + StreamingWorkScheduler streamingWorkScheduler = + StreamingWorkScheduler.create( + options, + clock, + readerCache, + mapTaskExecutorFactory, + workUnitExecutor, + stateCache::forComputation, + failureTracker, + workFailureProcessor, + streamingCounters, + hotKeyLogger, + sampler, + operationalLimits, + ID_GENERATOR, + stageInfoMap); ThrottlingGetDataMetricTracker getDataMetricTracker = new ThrottlingGetDataMetricTracker(memoryMonitor); - + WorkerStatusPages workerStatusPages = + WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); + StreamingWorkerStatusPages.Builder statusPagesBuilder = StreamingWorkerStatusPages.builder(); int stuckCommitDurationMillis; + GetDataClient getDataClient; + HeartbeatSender heartbeatSender; if (windmillServiceEnabled) { WindmillStreamPool getDataStreamPool = WindmillStreamPool.create( Math.max(1, options.getWindmillGetDataStreamCount()), GET_DATA_STREAM_TIMEOUT, windmillServer::getDataStream); - this.getDataClient = new StreamPoolGetDataClient(getDataMetricTracker, getDataStreamPool); - this.heartbeatSender = + getDataClient = new StreamPoolGetDataClient(getDataMetricTracker, getDataStreamPool); + heartbeatSender = new StreamPoolHeartbeatSender( options.getUseSeparateWindmillHeartbeatStreams() ? WindmillStreamPool.create( @@ -266,9 +264,16 @@ private StreamingDataflowWorker( : getDataStreamPool); stuckCommitDurationMillis = options.getStuckCommitDurationMillis() > 0 ? options.getStuckCommitDurationMillis() : 0; + statusPagesBuilder + .setDebugCapture( + new DebugCapture.Manager(options, workerStatusPages.getDebugCapturePages())) + .setChannelzServlet( + new ChannelzServlet( + CHANNELZ_PATH, options, windmillServer::getWindmillServiceEndpoints)) + .setWindmillStreamFactory(windmillStreamFactory); } else { - this.getDataClient = new ApplianceGetDataClient(windmillServer, getDataMetricTracker); - this.heartbeatSender = new ApplianceHeartbeatSender(windmillServer::getData); + getDataClient = new ApplianceGetDataClient(windmillServer, getDataMetricTracker); + heartbeatSender = new ApplianceHeartbeatSender(windmillServer::getData); stuckCommitDurationMillis = 0; } @@ -282,49 +287,40 @@ private StreamingDataflowWorker( executorSupplier.apply("RefreshWork"), getDataMetricTracker::trackHeartbeats); - WorkerStatusPages workerStatusPages = - WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); - StreamingWorkerStatusPages.Builder statusPagesBuilder = - StreamingWorkerStatusPages.builder() + this.statusPages = + statusPagesBuilder .setClock(clock) .setClientId(clientId) .setIsRunning(running) .setStatusPages(workerStatusPages) .setStateCache(stateCache) - .setComputationStateCache(computationStateCache) + .setComputationStateCache(this.computationStateCache) .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(getDataClient::printHtml) - .setWorkUnitExecutor(workUnitExecutor); + .setWorkUnitExecutor(workUnitExecutor) + .build(); - this.statusPages = - windmillServiceEnabled - ? statusPagesBuilder - .setDebugCapture( - new DebugCapture.Manager(options, workerStatusPages.getDebugCapturePages())) - .setChannelzServlet(new ChannelzServlet(CHANNELZ_PATH, options, windmillServer)) - .setWindmillStreamFactory(windmillStreamFactory) - .build() - : statusPagesBuilder.build(); + Windmill.GetWorkRequest request = + Windmill.GetWorkRequest.newBuilder() + .setClientId(clientId) + .setMaxItems(chooseMaximumBundlesOutstanding()) + .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) + .build(); - this.workerStatusReporter = workerStatusReporter; - this.streamingCounters = streamingCounters; - this.memoryMonitor = memoryMonitor; - this.streamingWorkScheduler = - StreamingWorkScheduler.create( - options, - clock, - readerCache, - mapTaskExecutorFactory, - workUnitExecutor, - stateCache::forComputation, - failureTracker, - workFailureProcessor, - streamingCounters, - hotKeyLogger, - sampler, - operationalLimits, - ID_GENERATOR, - stageInfoMap); + this.streamingWorkerHarness = + SingleSourceWorkerHarness.builder() + .setStreamingWorkScheduler(streamingWorkScheduler) + .setWorkCommitter(workCommitter) + .setGetDataClient(getDataClient) + .setComputationStateFetcher(this.computationStateCache::get) + .setWaitForResources(() -> memoryMonitor.waitForResources("GetWork")) + .setHeartbeatSender(heartbeatSender) + .setGetWorkSender( + windmillServiceEnabled + ? GetWorkSender.forStreamingEngine( + receiver -> windmillServer.getWorkStream(request, receiver)) + : GetWorkSender.forAppliance(() -> windmillServer.getWork(request))) + .build(); LOG.debug("windmillServiceEnabled: {}", windmillServiceEnabled); LOG.debug("WindmillServiceEndpoint: {}", options.getWindmillServiceEndpoint()); @@ -333,7 +329,7 @@ private StreamingDataflowWorker( } public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions options) { - long clientId = clientIdGenerator.nextLong(); + long clientId = CLIENT_ID_GENERATOR.nextLong(); MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options); ConcurrentMap stageInfo = new ConcurrentHashMap<>(); StreamingCounters streamingCounters = StreamingCounters.create(); @@ -438,9 +434,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o ComputationConfig.Fetcher configFetcher; WindmillServerStub windmillServer; ComputationStateCache computationStateCache; - GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(createStubFactory(options)); GrpcWindmillStreamFactory windmillStreamFactory; if (options.isEnableStreamingEngine()) { + GrpcDispatcherClient dispatcherClient = + GrpcDispatcherClient.create(createStubFactory(options)); configFetcher = StreamingEngineComputationConfigFetcher.create( options.getGlobalConfigRefreshPeriod().getMillis(), @@ -469,7 +466,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o options.getWindmillServiceStreamingRpcHealthCheckPeriodMs()) .build(); windmillServer = - GrpcWindmillServer.create(options, windmillStreamFactory, dispatcherClient); + GrpcWindmillServer.create( + options, + windmillStreamFactory, + GrpcDispatcherClient.create(createStubFactory(options))); } else { windmillStreamFactory = windmillStreamFactoryBuilder.build(); windmillServer = new JniWindmillApplianceServer(options.getLocalWindmillHostport()); @@ -704,10 +704,6 @@ private static ChannelCachingStubFactory createStubFactory( return ChannelCachingRemoteStubFactory.create(workerOptions.getGcpCredential(), channelCache); } - private static void sleep(int millis) { - Uninterruptibles.sleepUninterruptibly(millis, TimeUnit.MILLISECONDS); - } - private static int chooseMaxThreads(DataflowWorkerHarnessOptions options) { if (options.getNumberOfWorkerHarnessThreads() != 0) { return options.getNumberOfWorkerHarnessThreads(); @@ -736,7 +732,7 @@ private static void enableBigQueryMetrics() { } @VisibleForTesting - final void reportPeriodicWorkerUpdatesForTest() { + void reportPeriodicWorkerUpdatesForTest() { workerStatusReporter.reportPeriodicWorkerUpdates(); } @@ -765,6 +761,11 @@ int numCommitThreads() { return workCommitter.parallelism(); } + @VisibleForTesting + CacheStats getStateCacheStats() { + return stateCache.getCacheStats(); + } + @VisibleForTesting ComputationStateCache getComputationStateCache() { return computationStateCache; @@ -773,14 +774,10 @@ ComputationStateCache getComputationStateCache() { @SuppressWarnings("FutureReturnValueIgnored") public void start() { running.set(true); - configFetcher.start(); - - memoryMonitorThread.start(); - dispatchThread.start(); + memoryMonitor.start(); + streamingWorkerHarness.start(); sampler.start(); - - workCommitter.start(); workerStatusReporter.start(); activeWorkRefresher.start(); } @@ -794,121 +791,19 @@ private void startStatusPages() { void stop() { try { configFetcher.stop(); - activeWorkRefresher.stop(); statusPages.stop(); running.set(false); - dispatchThread.interrupt(); - dispatchThread.join(); - - workCommitter.stop(); - memoryMonitor.stop(); - memoryMonitorThread.join(); + streamingWorkerHarness.shutdown(); + memoryMonitor.shutdown(); workUnitExecutor.shutdown(); - computationStateCache.closeAndInvalidateAll(); - workerStatusReporter.stop(); } catch (Exception e) { LOG.warn("Exception while shutting down: ", e); } } - private void dispatchLoop() { - while (running.get()) { - memoryMonitor.waitForResources("GetWork"); - - int backoff = 1; - Windmill.GetWorkResponse workResponse = null; - do { - try { - workResponse = getWork(); - if (workResponse.getWorkCount() > 0) { - break; - } - } catch (WindmillServerStub.RpcException e) { - LOG.warn("GetWork failed, retrying:", e); - } - sleep(backoff); - backoff = Math.min(1000, backoff * 2); - } while (running.get()); - for (final Windmill.ComputationWorkItems computationWork : workResponse.getWorkList()) { - final String computationId = computationWork.getComputationId(); - Optional maybeComputationState = computationStateCache.get(computationId); - if (!maybeComputationState.isPresent()) { - continue; - } - - final ComputationState computationState = maybeComputationState.get(); - final Instant inputDataWatermark = - WindmillTimeUtils.windmillToHarnessWatermark(computationWork.getInputDataWatermark()); - Watermarks.Builder watermarks = - Watermarks.builder() - .setInputDataWatermark(Preconditions.checkNotNull(inputDataWatermark)) - .setSynchronizedProcessingTime( - WindmillTimeUtils.windmillToHarnessWatermark( - computationWork.getDependentRealtimeInputWatermark())); - - for (final Windmill.WorkItem workItem : computationWork.getWorkList()) { - streamingWorkScheduler.scheduleWork( - computationState, - workItem, - watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(), - Work.createProcessingContext( - computationId, getDataClient, workCommitter::commit, heartbeatSender), - /* getWorkStreamLatencies= */ Collections.emptyList()); - } - } - } - } - - void streamingDispatchLoop() { - while (running.get()) { - GetWorkStream stream = - windmillServer.getWorkStream( - Windmill.GetWorkRequest.newBuilder() - .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) - .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) - .build(), - (String computation, - Instant inputDataWatermark, - Instant synchronizedProcessingTime, - Windmill.WorkItem workItem, - Collection getWorkStreamLatencies) -> - computationStateCache - .get(computation) - .ifPresent( - computationState -> { - memoryMonitor.waitForResources("GetWork"); - streamingWorkScheduler.scheduleWork( - computationState, - workItem, - Watermarks.builder() - .setInputDataWatermark(inputDataWatermark) - .setSynchronizedProcessingTime(synchronizedProcessingTime) - .setOutputDataWatermark(workItem.getOutputDataWatermark()) - .build(), - Work.createProcessingContext( - computationState.getComputationId(), - getDataClient, - workCommitter::commit, - heartbeatSender), - getWorkStreamLatencies); - })); - try { - // Reconnect every now and again to enable better load balancing. - // If at any point the server closes the stream, we will reconnect immediately; otherwise - // we half-close the stream after some time and create a new one. - if (!stream.awaitTermination(GET_WORK_STREAM_TIMEOUT_MINUTES, TimeUnit.MINUTES)) { - stream.halfClose(); - } - } catch (InterruptedException e) { - // Continue processing until !running.get() - } - } - } - private void onCompleteCommit(CompleteCommit completeCommit) { if (completeCommit.status() != Windmill.CommitStatus.OK) { readerCache.invalidateReader( @@ -927,15 +822,6 @@ private void onCompleteCommit(CompleteCommit completeCommit) { completeCommit.shardedKey(), completeCommit.workId())); } - private Windmill.GetWorkResponse getWork() { - return windmillServer.getWork( - Windmill.GetWorkRequest.newBuilder() - .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) - .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) - .build()); - } - @VisibleForTesting public Iterable buildCounters() { return Iterables.concat( @@ -967,4 +853,34 @@ private static ConfigFetcherComputationStateCacheAndWindmillClient create( abstract GrpcWindmillStreamFactory windmillStreamFactory(); } + + /** + * Monitors memory pressure on a background executor. May be used to throttle calls, blocking if + * there is memory pressure. + */ + @AutoValue + abstract static class BackgroundMemoryMonitor { + private static BackgroundMemoryMonitor create(MemoryMonitor memoryMonitor) { + return new AutoValue_StreamingDataflowWorker_BackgroundMemoryMonitor( + memoryMonitor, + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("MemoryMonitor") + .setPriority(Thread.MIN_PRIORITY) + .build())); + } + + abstract MemoryMonitor memoryMonitor(); + + abstract ExecutorService executor(); + + private void start() { + executor().execute(memoryMonitor()); + } + + private void shutdown() { + memoryMonitor().stop(); + executor().shutdown(); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java similarity index 91% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java index b9573ff94cc9a..3556b7ce29198 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet.toImmutableSet; @@ -47,6 +47,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.StreamGetDataClient; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.ThrottlingGetDataMetricTracker; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; @@ -68,16 +70,19 @@ import org.slf4j.LoggerFactory; /** - * Client for StreamingEngine. Given a {@link GetWorkBudget}, divides the budget and starts the - * {@link WindmillStream.GetWorkStream}(s). + * {@link StreamingWorkerHarness} implementation that manages fan out to multiple backend + * destinations. Given a {@link GetWorkBudget}, divides the budget and starts the {@link + * WindmillStream.GetWorkStream}(s). */ @Internal @CheckReturnValue @ThreadSafe -public final class StreamingEngineClient { - private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineClient.class); +public final class FanOutStreamingEngineWorkerHarness implements StreamingWorkerHarness { + private static final Logger LOG = + LoggerFactory.getLogger(FanOutStreamingEngineWorkerHarness.class); private static final String PUBLISH_NEW_WORKER_METADATA_THREAD = "PublishNewWorkerMetadataThread"; private static final String CONSUME_NEW_WORKER_METADATA_THREAD = "ConsumeNewWorkerMetadataThread"; + private final JobHeader jobHeader; private final GrpcWindmillStreamFactory streamFactory; private final WorkItemScheduler workItemScheduler; @@ -101,7 +106,7 @@ public final class StreamingEngineClient { private volatile boolean started; @SuppressWarnings("FutureReturnValueIgnored") - private StreamingEngineClient( + private FanOutStreamingEngineWorkerHarness( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamFactory, @@ -152,23 +157,15 @@ private StreamingEngineClient( private static ExecutorService singleThreadedExecutorServiceOf(String threadName) { return Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setNameFormat(threadName) - .setUncaughtExceptionHandler( - (t, e) -> { - LOG.error( - "{} failed due to uncaught exception during execution. ", t.getName(), e); - throw new StreamingEngineClientException(e); - }) - .build()); + new ThreadFactoryBuilder().setNameFormat(threadName).build()); } /** - * Creates an instance of {@link StreamingEngineClient} in a non-started state. + * Creates an instance of {@link FanOutStreamingEngineWorkerHarness} in a non-started state. * * @implNote Does not block the calling thread. Callers must explicitly call {@link #start()}. */ - public static StreamingEngineClient create( + public static FanOutStreamingEngineWorkerHarness create( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamingEngineStreamFactory, @@ -178,7 +175,7 @@ public static StreamingEngineClient create( GrpcDispatcherClient dispatcherClient, Function workCommitterFactory, ThrottlingGetDataMetricTracker getDataMetricTracker) { - return new StreamingEngineClient( + return new FanOutStreamingEngineWorkerHarness( jobHeader, totalGetWorkBudget, streamingEngineStreamFactory, @@ -192,7 +189,7 @@ public static StreamingEngineClient create( } @VisibleForTesting - static StreamingEngineClient forTesting( + static FanOutStreamingEngineWorkerHarness forTesting( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamFactory, @@ -203,8 +200,8 @@ static StreamingEngineClient forTesting( long clientId, Function workCommitterFactory, ThrottlingGetDataMetricTracker getDataMetricTracker) { - StreamingEngineClient streamingEngineClient = - new StreamingEngineClient( + FanOutStreamingEngineWorkerHarness fanOutStreamingEngineWorkProvider = + new FanOutStreamingEngineWorkerHarness( jobHeader, totalGetWorkBudget, streamFactory, @@ -215,11 +212,12 @@ static StreamingEngineClient forTesting( clientId, workCommitterFactory, getDataMetricTracker); - streamingEngineClient.start(); - return streamingEngineClient; + fanOutStreamingEngineWorkProvider.start(); + return fanOutStreamingEngineWorkProvider; } @SuppressWarnings("ReturnValueIgnored") + @Override public synchronized void start() { Preconditions.checkState(!started, "StreamingEngineClient cannot start twice."); // Starts the stream, this value is memoized. @@ -270,7 +268,8 @@ private void startWorkerMetadataConsumer() { } @VisibleForTesting - public synchronized void finish() { + @Override + public synchronized void shutdown() { Preconditions.checkState(started, "StreamingEngineClient never started."); getWorkerMetadataStream.get().halfClose(); getWorkBudgetRefresher.stop(); @@ -334,10 +333,13 @@ private synchronized ImmutableMap createNewWindmil .collect( toImmutableMap( Function.identity(), - // Reuse existing stubs if they exist. endpoint -> - currentConnections.getOrDefault( - endpoint, WindmillConnection.from(endpoint, this::createWindmillStub)))); + // Reuse existing stubs if they exist. Optional.orElseGet only calls the + // supplier if the value is not present, preventing constructing expensive + // objects. + Optional.ofNullable(currentConnections.get(endpoint)) + .orElseGet( + () -> WindmillConnection.from(endpoint, this::createWindmillStub)))); } private synchronized ImmutableMap @@ -423,11 +425,4 @@ private CloudWindmillServiceV1Alpha1Stub createWindmillStub(Endpoint endpoint) { .map(channelCachingStubFactory::createWindmillServiceStub) .orElseGet(dispatcherClient::getWindmillServiceStub); } - - private static class StreamingEngineClientException extends IllegalStateException { - - private StreamingEngineClientException(Throwable exception) { - super(exception); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java new file mode 100644 index 0000000000000..bc93e6d89c415 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.harness; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; + +import com.google.auto.value.AutoBuilder; +import com.google.auto.value.AutoOneOf; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.RpcException; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; +import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link StreamingWorkerHarness} implementations that fetch {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from a single source. + */ +@Internal +public final class SingleSourceWorkerHarness implements StreamingWorkerHarness { + private static final Logger LOG = LoggerFactory.getLogger(SingleSourceWorkerHarness.class); + private static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; + + private final AtomicBoolean isRunning; + private final WorkCommitter workCommitter; + private final GetDataClient getDataClient; + private final HeartbeatSender heartbeatSender; + private final StreamingWorkScheduler streamingWorkScheduler; + private final Runnable waitForResources; + private final Function> computationStateFetcher; + private final ExecutorService workProviderExecutor; + private final GetWorkSender getWorkSender; + + SingleSourceWorkerHarness( + WorkCommitter workCommitter, + GetDataClient getDataClient, + HeartbeatSender heartbeatSender, + StreamingWorkScheduler streamingWorkScheduler, + Runnable waitForResources, + Function> computationStateFetcher, + GetWorkSender getWorkSender) { + this.workCommitter = workCommitter; + this.getDataClient = getDataClient; + this.heartbeatSender = heartbeatSender; + this.streamingWorkScheduler = streamingWorkScheduler; + this.waitForResources = waitForResources; + this.computationStateFetcher = computationStateFetcher; + this.workProviderExecutor = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("DispatchThread") + .build()); + this.isRunning = new AtomicBoolean(false); + this.getWorkSender = getWorkSender; + } + + public static SingleSourceWorkerHarness.Builder builder() { + return new AutoBuilder_SingleSourceWorkerHarness_Builder(); + } + + @Override + public void start() { + Preconditions.checkState( + isRunning.compareAndSet(false, true), + "Multiple calls to {}.start() are not allowed.", + getClass()); + workCommitter.start(); + workProviderExecutor.execute( + () -> { + getDispatchLoop().run(); + LOG.info("Dispatch done"); + }); + } + + private Runnable getDispatchLoop() { + switch (getWorkSender.getKind()) { + case APPLIANCE: + LOG.info("Starting Dispatch in Appliance mode."); + return () -> applianceDispatchLoop(getWorkSender.appliance()); + case STREAMING_ENGINE: + LOG.info("Starting Dispatch in Streaming Engine mode."); + return () -> streamingEngineDispatchLoop(getWorkSender.streamingEngine()); + default: + // Will never happen switch is exhaustive. + throw new IllegalStateException("Invalid GetWorkSender.Kind: " + getWorkSender.getKind()); + } + } + + @Override + public void shutdown() { + Preconditions.checkState( + isRunning.compareAndSet(true, false), + "Multiple calls to {}.shutdown() are not allowed.", + getClass()); + workProviderExecutor.shutdown(); + boolean isTerminated = false; + try { + isTerminated = workProviderExecutor.awaitTermination(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("Unable to shutdown {}", getClass()); + } + + if (!isTerminated) { + workProviderExecutor.shutdownNow(); + } + workCommitter.stop(); + } + + private void streamingEngineDispatchLoop( + Function getWorkStreamFactory) { + while (isRunning.get()) { + WindmillStream.GetWorkStream stream = + getWorkStreamFactory.apply( + (computationId, + inputDataWatermark, + synchronizedProcessingTime, + workItem, + getWorkStreamLatencies) -> + computationStateFetcher + .apply(computationId) + .ifPresent( + computationState -> { + waitForResources.run(); + streamingWorkScheduler.scheduleWork( + computationState, + workItem, + Watermarks.builder() + .setInputDataWatermark( + Preconditions.checkNotNull(inputDataWatermark)) + .setSynchronizedProcessingTime(synchronizedProcessingTime) + .setOutputDataWatermark(workItem.getOutputDataWatermark()) + .build(), + Work.createProcessingContext( + computationId, + getDataClient, + workCommitter::commit, + heartbeatSender), + getWorkStreamLatencies); + })); + try { + // Reconnect every now and again to enable better load balancing. + // If at any point the server closes the stream, we will reconnect immediately; otherwise + // we half-close the stream after some time and create a new one. + if (!stream.awaitTermination(GET_WORK_STREAM_TIMEOUT_MINUTES, TimeUnit.MINUTES)) { + stream.halfClose(); + } + } catch (InterruptedException e) { + // Continue processing until !running.get() + } + } + } + + private void applianceDispatchLoop(Supplier getWorkFn) { + while (isRunning.get()) { + waitForResources.run(); + int backoff = 1; + Windmill.GetWorkResponse workResponse = null; + do { + try { + workResponse = getWorkFn.get(); + if (workResponse.getWorkCount() > 0) { + break; + } + } catch (RpcException e) { + LOG.warn("GetWork failed, retrying:", e); + } + sleepUninterruptibly(backoff, TimeUnit.MILLISECONDS); + backoff = Math.min(1000, backoff * 2); + } while (isRunning.get()); + for (Windmill.ComputationWorkItems computationWork : + Preconditions.checkNotNull(workResponse).getWorkList()) { + String computationId = computationWork.getComputationId(); + Optional maybeComputationState = + computationStateFetcher.apply(computationId); + if (!maybeComputationState.isPresent()) { + continue; + } + + ComputationState computationState = maybeComputationState.get(); + Instant inputDataWatermark = + WindmillTimeUtils.windmillToHarnessWatermark(computationWork.getInputDataWatermark()); + Watermarks.Builder watermarks = + Watermarks.builder() + .setInputDataWatermark(Preconditions.checkNotNull(inputDataWatermark)) + .setSynchronizedProcessingTime( + WindmillTimeUtils.windmillToHarnessWatermark( + computationWork.getDependentRealtimeInputWatermark())); + + for (Windmill.WorkItem workItem : computationWork.getWorkList()) { + streamingWorkScheduler.scheduleWork( + computationState, + workItem, + watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(), + Work.createProcessingContext( + computationId, getDataClient, workCommitter::commit, heartbeatSender), + /* getWorkStreamLatencies= */ Collections.emptyList()); + } + } + } + } + + @AutoBuilder + public interface Builder { + Builder setWorkCommitter(WorkCommitter workCommitter); + + Builder setGetDataClient(GetDataClient getDataClient); + + Builder setHeartbeatSender(HeartbeatSender heartbeatSender); + + Builder setStreamingWorkScheduler(StreamingWorkScheduler streamingWorkScheduler); + + Builder setWaitForResources(Runnable waitForResources); + + Builder setComputationStateFetcher( + Function> computationStateFetcher); + + Builder setGetWorkSender(GetWorkSender getWorkSender); + + SingleSourceWorkerHarness build(); + } + + @AutoOneOf(GetWorkSender.Kind.class) + public abstract static class GetWorkSender { + + public static GetWorkSender forStreamingEngine( + Function getWorkStreamFactory) { + return AutoOneOf_SingleSourceWorkerHarness_GetWorkSender.streamingEngine( + getWorkStreamFactory); + } + + public static GetWorkSender forAppliance(Supplier getWorkFn) { + return AutoOneOf_SingleSourceWorkerHarness_GetWorkSender.appliance(getWorkFn); + } + + abstract Function streamingEngine(); + + abstract Supplier appliance(); + + abstract Kind getKind(); + + enum Kind { + STREAMING_ENGINE, + APPLIANCE + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java index 8d784456d655b..3c85ee6abe1f5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import com.google.auto.value.AutoValue; import java.util.function.Supplier; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java new file mode 100644 index 0000000000000..c1b4570e22600 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.harness; + +import org.apache.beam.sdk.annotations.Internal; + +/** Provides an interface to start streaming worker processing. */ +@Internal +public interface StreamingWorkerHarness { + void start(); + + void shutdown(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java similarity index 93% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java index 7d09726e4b28a..45aa403ee71b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -29,9 +29,11 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.StreamingEngineThrottleTimers; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FixedStreamHeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; @@ -57,7 +59,7 @@ */ @Internal @ThreadSafe -public class WindmillStreamSender { +final class WindmillStreamSender implements GetWorkBudgetSpender { private final AtomicBoolean started; private final AtomicReference getWorkBudget; private final Supplier getWorkStream; @@ -107,7 +109,7 @@ private WindmillStreamSender( workItemScheduler)); } - public static WindmillStreamSender create( + static WindmillStreamSender create( WindmillConnection connection, GetWorkRequest getWorkRequest, GetWorkBudget getWorkBudget, @@ -151,6 +153,7 @@ void closeAllStreams() { } } + @Override public void adjustBudget(long itemsDelta, long bytesDelta) { getWorkBudget.set(getWorkBudget.get().apply(itemsDelta, bytesDelta)); if (started.get()) { @@ -158,19 +161,16 @@ public void adjustBudget(long itemsDelta, long bytesDelta) { } } - public void adjustBudget(GetWorkBudget adjustment) { - adjustBudget(adjustment.items(), adjustment.bytes()); - } - - public GetWorkBudget remainingGetWorkBudget() { + @Override + public GetWorkBudget remainingBudget() { return started.get() ? getWorkStream.get().remainingBudget() : getWorkBudget.get(); } - public long getAndResetThrottleTime() { + long getAndResetThrottleTime() { return streamingEngineThrottleTimers.getAndResetThrottleTime(); } - public long getCurrentActiveCommitBytes() { + long getCurrentActiveCommitBytes() { return started.get() ? workCommitter.get().currentActiveCommitBytes() : 0; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java index e0f823d79ade5..adfb380d21647 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.servlet.ServletException; @@ -31,7 +32,6 @@ import org.apache.beam.runners.dataflow.options.DataflowStreamingPipelineOptions; import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet; import org.apache.beam.runners.dataflow.worker.status.DebugCapture; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.channelz.v1.*; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.protobuf.services.ChannelzService; @@ -47,16 +47,16 @@ public class ChannelzServlet extends BaseStatusServlet implements DebugCapture.C private static final int MAX_TOP_CHANNELS_TO_RETURN = 500; private final ChannelzService channelzService; - private final WindmillServerStub windmillServerStub; + private final Supplier> currentWindmillEndpoints; private final boolean showOnlyWindmillServiceChannels; public ChannelzServlet( String path, DataflowStreamingPipelineOptions options, - WindmillServerStub windmillServerStub) { + Supplier> currentWindmillEndpoints) { super(path); channelzService = ChannelzService.newInstance(MAX_TOP_CHANNELS_TO_RETURN); - this.windmillServerStub = windmillServerStub; + this.currentWindmillEndpoints = currentWindmillEndpoints; showOnlyWindmillServiceChannels = options.getChannelzShowOnlyWindmillServiceChannels(); } @@ -81,14 +81,6 @@ public void captureData(PrintWriter writer) { writer.println(""); } - // channelz proto says there won't be cycles in the ref graph. - // we track visited ids to be defensive and prevent any accidental cycles. - private static class VisitedSets { - - Set channels = new HashSet<>(); - Set subchannels = new HashSet<>(); - } - private void appendTopChannels(PrintWriter writer) { SettableFuture future = SettableFuture.create(); // IDEA: If there are more than MAX_TOP_CHANNELS_TO_RETURN top channels @@ -127,8 +119,7 @@ private void appendTopChannels(PrintWriter writer) { } private List filterWindmillChannels(List channels) { - ImmutableSet windmillServiceEndpoints = - windmillServerStub.getWindmillServiceEndpoints(); + ImmutableSet windmillServiceEndpoints = currentWindmillEndpoints.get(); Set windmillServiceHosts = windmillServiceEndpoints.stream().map(HostAndPort::getHost).collect(Collectors.toSet()); List windmillChannels = new ArrayList<>(); @@ -291,4 +282,12 @@ public void onCompleted() { } }; } + + // channelz proto says there won't be cycles in the ref graph. + // we track visited ids to be defensive and prevent any accidental cycles. + private static class VisitedSets { + + Set channels = new HashSet<>(); + Set subchannels = new HashSet<>(); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java new file mode 100644 index 0000000000000..9f30f75919f97 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link ByteString} buffer of {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk}(s). + * + *

Once all serialized chunks of an {@link WorkItem} have been received flushes (deserializes) + * the chunk of bytes and metadata into an {@link AssembledWorkItem}. + * + * @implNote This class is not thread safe, and provides no synchronization underneath. + */ +@NotThreadSafe +final class GetWorkResponseChunkAssembler { + private static final Logger LOG = LoggerFactory.getLogger(GetWorkResponseChunkAssembler.class); + + private final GetWorkTimingInfosTracker workTimingInfosTracker; + private @Nullable ComputationMetadata metadata; + private ByteString data; + private long bufferedSize; + + GetWorkResponseChunkAssembler() { + workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); + data = ByteString.EMPTY; + bufferedSize = 0; + metadata = null; + } + + /** + * Appends the response chunk bytes to the {@link #data }byte buffer. Return the assembled + * WorkItem if all response chunks for a WorkItem have been received. + */ + Optional append(Windmill.StreamingGetWorkResponseChunk chunk) { + if (chunk.hasComputationMetadata()) { + metadata = ComputationMetadata.fromProto(chunk.getComputationMetadata()); + } + + data = data.concat(chunk.getSerializedWorkItem()); + bufferedSize += chunk.getSerializedWorkItem().size(); + workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); + + // If the entire WorkItem has been received, assemble the WorkItem. + return chunk.getRemainingBytesForWorkItem() == 0 ? flushToWorkItem() : Optional.empty(); + } + + /** + * Attempt to flush the {@link #data} bytes into a {@link WorkItem} w/ it's metadata. Resets the + * data byte string and tracking metadata afterwards, whether the {@link WorkItem} deserialization + * was successful or not. + */ + private Optional flushToWorkItem() { + try { + return Optional.of( + AssembledWorkItem.create( + WorkItem.parseFrom(data.newInput()), + Preconditions.checkNotNull(metadata), + workTimingInfosTracker.getLatencyAttributions(), + bufferedSize)); + } catch (IOException e) { + LOG.error("Failed to parse work item from stream: ", e); + } finally { + workTimingInfosTracker.reset(); + data = ByteString.EMPTY; + bufferedSize = 0; + } + + return Optional.empty(); + } + + @AutoValue + abstract static class ComputationMetadata { + private static ComputationMetadata fromProto( + Windmill.ComputationWorkItemMetadata metadataProto) { + return new AutoValue_GetWorkResponseChunkAssembler_ComputationMetadata( + metadataProto.getComputationId(), + WindmillTimeUtils.windmillToHarnessWatermark(metadataProto.getInputDataWatermark()), + WindmillTimeUtils.windmillToHarnessWatermark( + metadataProto.getDependentRealtimeInputWatermark())); + } + + abstract String computationId(); + + abstract Instant inputDataWatermark(); + + abstract Instant synchronizedProcessingTime(); + } + + @AutoValue + abstract static class AssembledWorkItem { + + private static AssembledWorkItem create( + WorkItem workItem, + ComputationMetadata computationMetadata, + List latencyAttributions, + long size) { + return new AutoValue_GetWorkResponseChunkAssembler_AssembledWorkItem( + workItem, computationMetadata, latencyAttributions, size); + } + + abstract WorkItem workItem(); + + abstract ComputationMetadata computationMetadata(); + + abstract List latencyAttributions(); + + abstract long bufferedSize(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 58f72610e2d35..45d010d7cfac5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; -import com.google.auto.value.AutoValue; -import java.io.IOException; import java.io.PrintWriter; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -26,12 +24,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; -import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationWorkItemMetadata; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; @@ -40,6 +35,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GetWorkResponseChunkAssembler.AssembledWorkItem; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; @@ -47,13 +43,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Implementation of {@link GetWorkStream} that passes along a specific {@link @@ -66,7 +58,6 @@ public final class GrpcDirectGetWorkStream extends AbstractWindmillStream implements GetWorkStream { - private static final Logger LOG = LoggerFactory.getLogger(GrpcDirectGetWorkStream.class); private static final StreamingGetWorkRequest HEALTH_CHECK_REQUEST = StreamingGetWorkRequest.newBuilder() .setRequestExtension( @@ -90,8 +81,10 @@ public final class GrpcDirectGetWorkStream * Map of stream IDs to their buffers. Used to aggregate streaming gRPC response chunks as they * come in. Once all chunks for a response has been received, the chunk is processed and the * buffer is cleared. + * + * @implNote Buffers are not persisted across stream restarts. */ - private final ConcurrentMap workItemBuffers; + private final ConcurrentMap workItemAssemblers; private GrpcDirectGetWorkStream( String backendWorkerToken, @@ -120,7 +113,7 @@ private GrpcDirectGetWorkStream( this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.workItemScheduler = workItemScheduler; - this.workItemBuffers = new ConcurrentHashMap<>(); + this.workItemAssemblers = new ConcurrentHashMap<>(); this.heartbeatSender = Suppliers.memoize(heartbeatSender::get); this.workCommitter = Suppliers.memoize(workCommitter::get); this.getDataClient = Suppliers.memoize(getDataClient::get); @@ -163,7 +156,8 @@ public static GrpcDirectGetWorkStream create( return getWorkStream; } - private static Watermarks createWatermarks(WorkItem workItem, ComputationMetadata metadata) { + private static Watermarks createWatermarks( + WorkItem workItem, GetWorkResponseChunkAssembler.ComputationMetadata metadata) { return Watermarks.builder() .setInputDataWatermark(metadata.inputDataWatermark()) .setOutputDataWatermark(workItem.getOutputDataWatermark()) @@ -171,14 +165,8 @@ private static Watermarks createWatermarks(WorkItem workItem, ComputationMetadat .build(); } - private synchronized GetWorkBudget getThenResetBudgetAdjustment() { - return nextBudgetAdjustment.getAndUpdate(unused -> GetWorkBudget.noBudget()); - } - - private void sendRequestExtension() { - // Just sent the request extension, reset the nextBudgetAdjustment. This will be set when - // adjustBudget is called. - GetWorkBudget adjustment = getThenResetBudgetAdjustment(); + private void sendRequestExtension(GetWorkBudget adjustment) { + inFlightBudget.getAndUpdate(budget -> budget.apply(adjustment)); StreamingGetWorkRequest extension = StreamingGetWorkRequest.newBuilder() .setRequestExtension( @@ -200,7 +188,7 @@ private void sendRequestExtension() { @Override protected synchronized void onNewStream() { - workItemBuffers.clear(); + workItemAssemblers.clear(); // Add the current in-flight budget to the next adjustment. Only positive values are allowed // here // with negatives defaulting to 0, since GetWorkBudgets cannot be created with negative values. @@ -229,7 +217,7 @@ public void appendSpecificHtml(PrintWriter writer) { // Number of buffers is same as distinct workers that sent work on this stream. writer.format( "GetWorkStream: %d buffers, %s inflight budget allowed.", - workItemBuffers.size(), inFlightBudget.get()); + workItemAssemblers.size(), inFlightBudget.get()); } @Override @@ -240,27 +228,49 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); - WorkItemBuffer workItemBuffer = - workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new WorkItemBuffer()); - workItemBuffer.append(chunk); + workItemAssemblers + .computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkResponseChunkAssembler()) + .append(chunk) + .ifPresent(this::consumeAssembledWorkItem); + } - // The entire WorkItem has been received, it is ready to be processed. - if (chunk.getRemainingBytesForWorkItem() == 0) { - workItemBuffer.runAndReset(); - // Record the fact that there are now fewer outstanding messages and bytes on the stream. - inFlightBudget.updateAndGet(budget -> budget.subtract(1, workItemBuffer.bufferedSize())); + private void consumeAssembledWorkItem(AssembledWorkItem assembledWorkItem) { + // Record the fact that there are now fewer outstanding messages and bytes on the stream. + inFlightBudget.updateAndGet(budget -> budget.subtract(1, assembledWorkItem.bufferedSize())); + WorkItem workItem = assembledWorkItem.workItem(); + GetWorkResponseChunkAssembler.ComputationMetadata metadata = + assembledWorkItem.computationMetadata(); + pendingResponseBudget.getAndUpdate(budget -> budget.apply(1, workItem.getSerializedSize())); + try { + workItemScheduler.scheduleWork( + workItem, + createWatermarks(workItem, Preconditions.checkNotNull(metadata)), + createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), + assembledWorkItem.latencyAttributions()); + } finally { + pendingResponseBudget.getAndUpdate(budget -> budget.apply(-1, -workItem.getSerializedSize())); } } + private Work.ProcessingContext createProcessingContext(String computationId) { + return Work.createProcessingContext( + computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get()); + } + @Override protected void startThrottleTimer() { getWorkThrottleTimer.start(); } @Override - public synchronized void adjustBudget(long itemsDelta, long bytesDelta) { - nextBudgetAdjustment.set(nextBudgetAdjustment.get().apply(itemsDelta, bytesDelta)); - sendRequestExtension(); + public void adjustBudget(long itemsDelta, long bytesDelta) { + GetWorkBudget adjustment = + nextBudgetAdjustment + // Get the current value, and reset the nextBudgetAdjustment. This will be set again + // when adjustBudget is called. + .getAndUpdate(unused -> GetWorkBudget.noBudget()) + .apply(itemsDelta, bytesDelta); + sendRequestExtension(adjustment); } @Override @@ -274,74 +284,4 @@ public GetWorkBudget remainingBudget() { .apply(currentNextBudgetAdjustment) .apply(currentInflightBudget); } - - private synchronized void updatePendingResponseBudget(long itemsDelta, long bytesDelta) { - pendingResponseBudget.set(pendingResponseBudget.get().apply(itemsDelta, bytesDelta)); - } - - @AutoValue - abstract static class ComputationMetadata { - private static ComputationMetadata fromProto(ComputationWorkItemMetadata metadataProto) { - return new AutoValue_GrpcDirectGetWorkStream_ComputationMetadata( - metadataProto.getComputationId(), - WindmillTimeUtils.windmillToHarnessWatermark(metadataProto.getInputDataWatermark()), - WindmillTimeUtils.windmillToHarnessWatermark( - metadataProto.getDependentRealtimeInputWatermark())); - } - - abstract String computationId(); - - abstract Instant inputDataWatermark(); - - abstract Instant synchronizedProcessingTime(); - } - - private class WorkItemBuffer { - private final GetWorkTimingInfosTracker workTimingInfosTracker; - private ByteString data; - private @Nullable ComputationMetadata metadata; - - private WorkItemBuffer() { - workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); - data = ByteString.EMPTY; - this.metadata = null; - } - - private void append(StreamingGetWorkResponseChunk chunk) { - if (chunk.hasComputationMetadata()) { - this.metadata = ComputationMetadata.fromProto(chunk.getComputationMetadata()); - } - - this.data = data.concat(chunk.getSerializedWorkItem()); - workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); - } - - private long bufferedSize() { - return data.size(); - } - - private void runAndReset() { - try { - WorkItem workItem = WorkItem.parseFrom(data.newInput()); - updatePendingResponseBudget(1, workItem.getSerializedSize()); - workItemScheduler.scheduleWork( - workItem, - createWatermarks(workItem, Preconditions.checkNotNull(metadata)), - createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), - // After the work item is successfully queued or dropped by ActiveWorkState, remove it - // from the pendingResponseBudget. - queuedWorkItem -> updatePendingResponseBudget(-1, -workItem.getSerializedSize()), - workTimingInfosTracker.getLatencyAttributions()); - } catch (IOException e) { - LOG.error("Failed to parse work item from stream: ", e); - } - workTimingInfosTracker.reset(); - data = ByteString.EMPTY; - } - - private Work.ProcessingContext createProcessingContext(String computationId) { - return Work.createProcessingContext( - computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get()); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index 033990017b24c..cf2e7260592db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -75,7 +75,7 @@ public static GrpcDispatcherClient create(WindmillStubFactory windmillStubFactor } @VisibleForTesting - static GrpcDispatcherClient forTesting( + public static GrpcDispatcherClient forTesting( WindmillStubFactory windmillGrpcStubFactory, List windmillServiceStubs, List windmillMetadataServiceStubs, @@ -106,7 +106,7 @@ ImmutableSet getDispatcherEndpoints() { } /** Will block the calling thread until the initial endpoints are present. */ - CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBlocking() { + public CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBlocking() { boolean initialized = false; long secondsWaited = 0; while (!initialized) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index 4b392e9190ed2..09ecbf3f30516 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -17,45 +17,34 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; -import java.io.IOException; import java.io.PrintWriter; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequestExtension; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GetWorkResponseChunkAssembler.AssembledWorkItem; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public final class GrpcGetWorkStream +final class GrpcGetWorkStream extends AbstractWindmillStream implements GetWorkStream { - private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkStream.class); - private final GetWorkRequest request; private final WorkItemReceiver receiver; private final ThrottleTimer getWorkThrottleTimer; - private final Map buffers; + private final Map workItemAssemblers; private final AtomicLong inflightMessages; private final AtomicLong inflightBytes; @@ -83,7 +72,7 @@ private GrpcGetWorkStream( this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.receiver = receiver; - this.buffers = new ConcurrentHashMap<>(); + this.workItemAssemblers = new ConcurrentHashMap<>(); this.inflightMessages = new AtomicLong(); this.inflightBytes = new AtomicLong(); } @@ -138,7 +127,7 @@ private void sendRequestExtension(long moreItems, long moreBytes) { @Override protected synchronized void onNewStream() { - buffers.clear(); + workItemAssemblers.clear(); inflightMessages.set(request.getMaxItems()); inflightBytes.set(request.getMaxBytes()); send(StreamingGetWorkRequest.newBuilder().setRequest(request).build()); @@ -154,7 +143,7 @@ public void appendSpecificHtml(PrintWriter writer) { // Number of buffers is same as distinct workers that sent work on this stream. writer.format( "GetWorkStream: %d buffers, %d inflight messages allowed, %d inflight bytes allowed", - buffers.size(), inflightMessages.intValue(), inflightBytes.intValue()); + workItemAssemblers.size(), inflightMessages.intValue(), inflightBytes.intValue()); } @Override @@ -169,30 +158,33 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); + workItemAssemblers + .computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkResponseChunkAssembler()) + .append(chunk) + .ifPresent(this::consumeAssembledWorkItem); + } - GrpcGetWorkStream.WorkItemBuffer buffer = - buffers.computeIfAbsent( - chunk.getStreamId(), unused -> new GrpcGetWorkStream.WorkItemBuffer()); - buffer.append(chunk); - - if (chunk.getRemainingBytesForWorkItem() == 0) { - long size = buffer.bufferedSize(); - buffer.runAndReset(); - - // Record the fact that there are now fewer outstanding messages and bytes on the stream. - long numInflight = inflightMessages.decrementAndGet(); - long bytesInflight = inflightBytes.addAndGet(-size); - - // If the outstanding items or bytes limit has gotten too low, top both off with a - // GetWorkExtension. The goal is to keep the limits relatively close to their maximum - // values without sending too many extension requests. - if (numInflight < request.getMaxItems() / 2 || bytesInflight < request.getMaxBytes() / 2) { - long moreItems = request.getMaxItems() - numInflight; - long moreBytes = request.getMaxBytes() - bytesInflight; - inflightMessages.getAndAdd(moreItems); - inflightBytes.getAndAdd(moreBytes); - sendRequestExtension(moreItems, moreBytes); - } + private void consumeAssembledWorkItem(AssembledWorkItem assembledWorkItem) { + receiver.receiveWork( + assembledWorkItem.computationMetadata().computationId(), + assembledWorkItem.computationMetadata().inputDataWatermark(), + assembledWorkItem.computationMetadata().synchronizedProcessingTime(), + assembledWorkItem.workItem(), + assembledWorkItem.latencyAttributions()); + + // Record the fact that there are now fewer outstanding messages and bytes on the stream. + long numInflight = inflightMessages.decrementAndGet(); + long bytesInflight = inflightBytes.addAndGet(-assembledWorkItem.bufferedSize()); + + // If the outstanding items or bytes limit has gotten too low, top both off with a + // GetWorkExtension. The goal is to keep the limits relatively close to their maximum + // values without sending too many extension requests. + if (numInflight < request.getMaxItems() / 2 || bytesInflight < request.getMaxBytes() / 2) { + long moreItems = request.getMaxItems() - numInflight; + long moreBytes = request.getMaxBytes() - bytesInflight; + inflightMessages.getAndAdd(moreItems); + inflightBytes.getAndAdd(moreBytes); + sendRequestExtension(moreItems, moreBytes); } } @@ -213,63 +205,4 @@ public GetWorkBudget remainingBudget() { .setItems(request.getMaxItems() - inflightMessages.get()) .build(); } - - private class WorkItemBuffer { - private final GetWorkTimingInfosTracker workTimingInfosTracker; - private String computation; - @Nullable private Instant inputDataWatermark; - @Nullable private Instant synchronizedProcessingTime; - private ByteString data; - private long bufferedSize; - - @SuppressWarnings("initialization.fields.uninitialized") - WorkItemBuffer() { - workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); - data = ByteString.EMPTY; - bufferedSize = 0; - } - - @SuppressWarnings("NullableProblems") - private void setMetadata(Windmill.ComputationWorkItemMetadata metadata) { - this.computation = metadata.getComputationId(); - this.inputDataWatermark = - WindmillTimeUtils.windmillToHarnessWatermark(metadata.getInputDataWatermark()); - this.synchronizedProcessingTime = - WindmillTimeUtils.windmillToHarnessWatermark( - metadata.getDependentRealtimeInputWatermark()); - } - - private void append(StreamingGetWorkResponseChunk chunk) { - if (chunk.hasComputationMetadata()) { - setMetadata(chunk.getComputationMetadata()); - } - - this.data = data.concat(chunk.getSerializedWorkItem()); - this.bufferedSize += chunk.getSerializedWorkItem().size(); - workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); - } - - private long bufferedSize() { - return bufferedSize; - } - - private void runAndReset() { - try { - Windmill.WorkItem workItem = Windmill.WorkItem.parseFrom(data.newInput()); - List getWorkStreamLatencies = - workTimingInfosTracker.getLatencyAttributions(); - receiver.receiveWork( - computation, - inputDataWatermark, - synchronizedProcessingTime, - workItem, - getWorkStreamLatencies); - } catch (IOException e) { - LOG.error("Failed to parse work item from stream: ", e); - } - workTimingInfosTracker.reset(); - data = ByteString.EMPTY; - bufferedSize = 0; - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java index 17c9f7d80d5da..00784493fe3df 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.worker.windmill.work; import java.util.Collection; -import java.util.function.Consumer; import javax.annotation.CheckReturnValue; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; @@ -36,8 +35,6 @@ public interface WorkItemScheduler { * @param workItem {@link WorkItem} to be processed. * @param watermarks processing watermarks for the workItem. * @param processingContext for processing the workItem. - * @param ackWorkItemQueued Called after an attempt to queue the work item for processing. Used to - * free up pending budget. * @param getWorkStreamLatencies Latencies per processing stage for the WorkItem for reporting * back to Streaming Engine backend. */ @@ -45,6 +42,5 @@ void scheduleWork( WorkItem workItem, Watermarks watermarks, Work.ProcessingContext processingContext, - Consumer ackWorkItemQueued, Collection getWorkStreamLatencies); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java index 3a17222d3e6bd..403bb99efb4c5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java @@ -26,14 +26,13 @@ import java.util.Map.Entry; import java.util.function.Function; import java.util.function.Supplier; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Evenly distributes the provided budget across the available {@link WindmillStreamSender}(s). */ +/** Evenly distributes the provided budget across the available {@link GetWorkBudgetSpender}(s). */ @Internal final class EvenGetWorkBudgetDistributor implements GetWorkBudgetDistributor { private static final Logger LOG = LoggerFactory.getLogger(EvenGetWorkBudgetDistributor.class); @@ -50,10 +49,10 @@ private static boolean isBelowFiftyPercentOfTarget( } @Override - public void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget) { - if (streams.isEmpty()) { - LOG.debug("Cannot distribute budget to no streams."); + public void distributeBudget( + ImmutableCollection budgetOwners, GetWorkBudget getWorkBudget) { + if (budgetOwners.isEmpty()) { + LOG.debug("Cannot distribute budget to no owners."); return; } @@ -62,23 +61,21 @@ public void distributeBudget( return; } - Map desiredBudgets = - computeDesiredBudgets(streams, getWorkBudget); + Map desiredBudgets = computeDesiredBudgets(budgetOwners, getWorkBudget); - for (Entry streamAndDesiredBudget : - desiredBudgets.entrySet()) { - WindmillStreamSender stream = streamAndDesiredBudget.getKey(); + for (Entry streamAndDesiredBudget : desiredBudgets.entrySet()) { + GetWorkBudgetSpender getWorkBudgetSpender = streamAndDesiredBudget.getKey(); GetWorkBudget desired = streamAndDesiredBudget.getValue(); - GetWorkBudget remaining = stream.remainingGetWorkBudget(); + GetWorkBudget remaining = getWorkBudgetSpender.remainingBudget(); if (isBelowFiftyPercentOfTarget(remaining, desired)) { GetWorkBudget adjustment = desired.subtract(remaining); - stream.adjustBudget(adjustment); + getWorkBudgetSpender.adjustBudget(adjustment); } } } - private ImmutableMap computeDesiredBudgets( - ImmutableCollection streams, GetWorkBudget totalGetWorkBudget) { + private ImmutableMap computeDesiredBudgets( + ImmutableCollection streams, GetWorkBudget totalGetWorkBudget) { GetWorkBudget activeWorkBudget = activeWorkBudgetSupplier.get(); LOG.info("Current active work budget: {}", activeWorkBudget); // TODO: Fix possibly non-deterministic handing out of budgets. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java index 3ec9718e041e5..d21de17e522c5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.work.budget; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; @@ -28,6 +27,6 @@ */ @Internal public interface GetWorkBudgetDistributor { - void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget); + void distributeBudget( + ImmutableCollection streams, GetWorkBudget getWorkBudget); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java new file mode 100644 index 0000000000000..254b2589062ef --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.windmill.work.budget; + +/** + * Represents something that spends {@link + * org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget} + */ +public interface GetWorkBudgetSpender { + void adjustBudget(long itemsDelta, long bytesDelta); + + default void adjustBudget(GetWorkBudget adjustment) { + adjustBudget(adjustment.items(), adjustment.bytes()); + } + + GetWorkBudget remainingBudget(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 5855057c4210c..d16ed2942fd9c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -2182,7 +2182,7 @@ public void testMergeWindowsCaching() throws Exception { // No input messages assertEquals(0L, splitIntToLong(getCounter(counters, "WindmillShuffleBytesRead").getInteger())); - CacheStats stats = worker.stateCache.getCacheStats(); + CacheStats stats = worker.getStateCacheStats(); LOG.info("cache stats {}", stats); assertEquals(1, stats.hitCount()); assertEquals(4, stats.missCount()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java similarity index 93% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java index 1999dbe319027..aaa71b6598ea2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -49,12 +49,15 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.ThrottlingGetDataMetricTracker; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory; import org.apache.beam.runners.dataflow.worker.windmill.testing.FakeWindmillStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributor; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessSocketAddress; @@ -76,7 +79,7 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class StreamingEngineClientTest { +public class FanOutStreamingEngineWorkerHarnessTest { private static final WindmillServiceAddress DEFAULT_WINDMILL_SERVICE_ADDRESS = WindmillServiceAddress.create(HostAndPort.fromParts(WindmillChannelFactory.LOCALHOST, 443)); private static final ImmutableMap DEFAULT = @@ -113,14 +116,10 @@ public class StreamingEngineClientTest { private Server fakeStreamingEngineServer; private CountDownLatch getWorkerMetadataReady; private GetWorkerMetadataTestStub fakeGetWorkerMetadataStub; - private StreamingEngineClient streamingEngineClient; + private FanOutStreamingEngineWorkerHarness fanOutStreamingEngineWorkProvider; private static WorkItemScheduler noOpProcessWorkItemFn() { - return (workItem, - watermarks, - processingContext, - ackWorkItemQueued, - getWorkStreamLatencies) -> {}; + return (workItem, watermarks, processingContext, getWorkStreamLatencies) -> {}; } private static GetWorkRequest getWorkRequest(long items, long bytes) { @@ -163,16 +162,16 @@ public void setUp() throws IOException { @After public void cleanUp() { - Preconditions.checkNotNull(streamingEngineClient).finish(); + Preconditions.checkNotNull(fanOutStreamingEngineWorkProvider).shutdown(); fakeStreamingEngineServer.shutdownNow(); stubFactory.shutdown(); } - private StreamingEngineClient newStreamingEngineClient( + private FanOutStreamingEngineWorkerHarness newStreamingEngineClient( GetWorkBudget getWorkBudget, GetWorkBudgetDistributor getWorkBudgetDistributor, WorkItemScheduler workItemScheduler) { - return StreamingEngineClient.forTesting( + return FanOutStreamingEngineWorkerHarness.forTesting( JOB_HEADER, getWorkBudget, streamFactory, @@ -194,7 +193,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(numBudgetDistributionsExpected)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(items).setBytes(bytes).build(), getWorkBudgetDistributor, @@ -216,7 +215,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor); StreamingEngineConnectionState currentConnections = - streamingEngineClient.getCurrentConnections(); + fanOutStreamingEngineWorkProvider.getCurrentConnections(); assertEquals(2, currentConnections.windmillConnections().size()); assertEquals(2, currentConnections.windmillStreams().size()); @@ -250,7 +249,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { public void testScheduledBudgetRefresh() throws InterruptedException { TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(2)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1L).setBytes(1L).build(), getWorkBudgetDistributor, @@ -273,7 +272,7 @@ public void testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers() int metadataCount = 2; TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(metadataCount)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1).setBytes(1).build(), getWorkBudgetDistributor, @@ -311,11 +310,12 @@ public void testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers() fakeGetWorkerMetadataStub.injectWorkerMetadata(secondWorkerMetadata); waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor); StreamingEngineConnectionState currentConnections = - streamingEngineClient.getCurrentConnections(); + fanOutStreamingEngineWorkProvider.getCurrentConnections(); assertEquals(1, currentConnections.windmillConnections().size()); assertEquals(1, currentConnections.windmillStreams().size()); Set workerTokens = - streamingEngineClient.getCurrentConnections().windmillConnections().values().stream() + fanOutStreamingEngineWorkProvider.getCurrentConnections().windmillConnections().values() + .stream() .map(WindmillConnection::backendWorkerToken) .collect(Collectors.toSet()); @@ -362,7 +362,7 @@ public void testOnNewWorkerMetadata_redistributesBudget() throws InterruptedExce TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(workerMetadataResponses.size())); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1).setBytes(1).build(), getWorkBudgetDistributor, @@ -439,8 +439,8 @@ private void waitForBudgetDistribution() throws InterruptedException { } @Override - public void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget) { + public void distributeBudget( + ImmutableCollection streams, GetWorkBudget getWorkBudget) { streams.forEach(stream -> stream.adjustBudget(getWorkBudget.items(), getWorkBudget.bytes())); getWorkBudgetDistributorTriggered.countDown(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java index 9d49c3ef3146d..dc6cc5641055a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -35,6 +35,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; @@ -64,7 +65,7 @@ public class WindmillStreamSenderTest { .build()) .build()); private final WorkItemScheduler workItemScheduler = - (workItem, watermarks, processingContext, ackWorkItemQueued, getWorkStreamLatencies) -> {}; + (workItem, watermarks, processingContext, getWorkStreamLatencies) -> {}; @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private ManagedChannel inProcessChannel; private WindmillConnection connection; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java index 96c675169a7d2..d234cf424767b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java @@ -56,7 +56,8 @@ public void testRendersAllChannels() throws UnsupportedEncodingException { fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(false); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); @@ -88,7 +89,8 @@ public void testRendersOnlyWindmillChannels() throws UnsupportedEncodingExceptio fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(true); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 5cfc19ac07dfd..7e5801b65de47 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -1142,7 +1142,13 @@ public void onNext(StreamingGetWorkRequest request) { StreamingGetWorkResponseChunk.newBuilder() .setStreamId(id) .setSerializedWorkItem(serializedResponse) - .setRemainingBytesForWorkItem(0); + .setRemainingBytesForWorkItem(0) + .setComputationMetadata( + ComputationWorkItemMetadata.newBuilder() + .setComputationId("computation") + .setInputDataWatermark(1L) + .setDependentRealtimeInputWatermark(1L) + .build()); try { responseObserver.onNext(builder.build()); } catch (IllegalStateException e) { @@ -1175,9 +1181,7 @@ public void onCompleted() { @Nullable Instant inputDataWatermark, Instant synchronizedProcessingTime, Windmill.WorkItem workItem, - Collection getWorkStreamLatencies) -> { - latch.countDown(); - }); + Collection getWorkStreamLatencies) -> latch.countDown()); // Wait for 100 items or 30 seconds. assertTrue(latch.await(30, TimeUnit.SECONDS)); // Confirm that we report at least as much throttle time as our server sent errors for. We will diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index b0c305dc4ec45..3cda4559c100b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -19,7 +19,6 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -28,20 +27,8 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection; -import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; -import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.junit.After; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -52,8 +39,6 @@ public class EvenGetWorkBudgetDistributorTest { @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); @Rule public transient Timeout globalTimeout = Timeout.seconds(600); - private ManagedChannel inProcessChannel; - private CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub; private static GetWorkBudgetDistributor createBudgetDistributor(GetWorkBudget activeWorkBudget) { return GetWorkBudgetDistributors.distributeEvenly(() -> activeWorkBudget); @@ -67,20 +52,6 @@ private static GetWorkBudgetDistributor createBudgetDistributor(long activeWorkI .build()); } - @Before - public void setUp() { - inProcessChannel = - grpcCleanup.register( - InProcessChannelBuilder.forName("WindmillStreamSenderTest").directExecutor().build()); - grpcCleanup.register(inProcessChannel); - stub = CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel); - } - - @After - public void cleanUp() { - inProcessChannel.shutdownNow(); - } - @Test public void testDistributeBudget_doesNothingWhenPassedInStreamsEmpty() { createBudgetDistributor(1L) @@ -90,38 +61,40 @@ public void testDistributeBudget_doesNothingWhenPassedInStreamsEmpty() { @Test public void testDistributeBudget_doesNothingWithNoBudget() { - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(GetWorkBudget.noBudget())); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget())); createBudgetDistributor(1L) - .distributeBudget(ImmutableList.of(windmillStreamSender), GetWorkBudget.noBudget()); - verifyNoInteractions(windmillStreamSender); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.noBudget()); + verifyNoInteractions(getWorkBudgetSpender); } @Test public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHighNoActiveWork() { - WindmillStreamSender windmillStreamSender = + GetWorkBudgetSpender getWorkBudgetSpender = spy( - createWindmillStreamSender( + createGetWorkBudgetOwnerWithRemainingBudgetOf( GetWorkBudget.builder().setItems(10L).setBytes(10L).build())); createBudgetDistributor(0L) .distributeBudget( - ImmutableList.of(windmillStreamSender), + ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.builder().setItems(10L).setBytes(10L).build()); - verify(windmillStreamSender, never()).adjustBudget(anyLong(), anyLong()); + verify(getWorkBudgetSpender, never()).adjustBudget(anyLong(), anyLong()); } @Test public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHighWithActiveWork() { - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(GetWorkBudget.builder().setItems(5L).setBytes(5L).build())); + GetWorkBudgetSpender getWorkBudgetSpender = + spy( + createGetWorkBudgetOwnerWithRemainingBudgetOf( + GetWorkBudget.builder().setItems(5L).setBytes(5L).build())); createBudgetDistributor(10L) .distributeBudget( - ImmutableList.of(windmillStreamSender), + ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.builder().setItems(20L).setBytes(20L).build()); - verify(windmillStreamSender, never()).adjustBudget(anyLong(), anyLong()); + verify(getWorkBudgetSpender, never()).adjustBudget(anyLong(), anyLong()); } @Test @@ -130,12 +103,12 @@ public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHig GetWorkBudget streamRemainingBudget = GetWorkBudget.builder().setItems(1L).setBytes(10L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(0L) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq(totalGetWorkBudget.bytes() - streamRemainingBudget.bytes())); @@ -148,12 +121,12 @@ public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHig GetWorkBudget.builder().setItems(1L).setBytes(10L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); long activeWorkItemsAndBytes = 2L; - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(activeWorkItemsAndBytes) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq( totalGetWorkBudget.items() @@ -167,12 +140,12 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo GetWorkBudget streamRemainingBudget = GetWorkBudget.builder().setItems(10L).setBytes(1L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(0L) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq(totalGetWorkBudget.bytes() - streamRemainingBudget.bytes())); @@ -186,12 +159,12 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); long activeWorkItemsAndBytes = 2L; - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(activeWorkItemsAndBytes) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq( @@ -203,9 +176,9 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo @Test public void testDistributeBudget_distributesBudgetEvenlyIfPossible() { long totalItemsAndBytes = 10L; - List streams = new ArrayList<>(); + List streams = new ArrayList<>(); for (int i = 0; i < totalItemsAndBytes; i++) { - streams.add(spy(createWindmillStreamSender(GetWorkBudget.noBudget()))); + streams.add(spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget()))); } createBudgetDistributor(0L) .distributeBudget( @@ -225,9 +198,9 @@ public void testDistributeBudget_distributesBudgetEvenlyIfPossible() { @Test public void testDistributeBudget_distributesFairlyWhenNotEven() { long totalItemsAndBytes = 10L; - List streams = new ArrayList<>(); + List streams = new ArrayList<>(); for (int i = 0; i < 3; i++) { - streams.add(spy(createWindmillStreamSender(GetWorkBudget.noBudget()))); + streams.add(spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget()))); } createBudgetDistributor(0L) .distributeBudget( @@ -244,24 +217,17 @@ public void testDistributeBudget_distributesFairlyWhenNotEven() { .adjustBudget(eq(itemsAndBytesPerStream), eq(itemsAndBytesPerStream))); } - private WindmillStreamSender createWindmillStreamSender(GetWorkBudget getWorkBudget) { - return WindmillStreamSender.create( - WindmillConnection.builder().setStub(stub).build(), - Windmill.GetWorkRequest.newBuilder() - .setClientId(1L) - .setJobId("job") - .setProjectId("project") - .build(), - getWorkBudget, - GrpcWindmillStreamFactory.of( - JobHeader.newBuilder() - .setJobId("job") - .setProjectId("project") - .setWorkerId("worker") - .build()) - .build(), - (workItem, watermarks, processingContext, ackWorkItemQueued, getWorkStreamLatencies) -> {}, - ignored -> mock(GetDataClient.class), - ignored -> mock(WorkCommitter.class)); + private GetWorkBudgetSpender createGetWorkBudgetOwnerWithRemainingBudgetOf( + GetWorkBudget getWorkBudget) { + return spy( + new GetWorkBudgetSpender() { + @Override + public void adjustBudget(long itemsDelta, long bytesDelta) {} + + @Override + public GetWorkBudget remainingBudget() { + return getWorkBudget; + } + }); } } From bfc64d5c14adea209364d48b0fe9b4e8ba6eaab5 Mon Sep 17 00:00:00 2001 From: scwhittle Date: Mon, 5 Aug 2024 11:34:45 +0200 Subject: [PATCH 09/78] Fix error when ActiveWorkRefresher processed empty heartbeat map. (#32078) --- .../work/refresh/ActiveWorkRefresher.java | 3 ++ .../work/refresh/ActiveWorkRefresherTest.java | 38 +++++++++++++++++-- 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index 499d2e5b6943c..781285def020e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -130,6 +130,9 @@ private void refreshActiveWork() { Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); Map heartbeatsBySender = aggregateHeartbeatsBySender(refreshDeadline); + if (heartbeatsBySender.isEmpty()) { + return; + } List> fanOutRefreshActiveWork = new ArrayList<>(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java index 9dce3392c60c5..5efb2421fe607 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker.windmill.work.refresh; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; @@ -194,10 +195,13 @@ public void testActiveWorkRefresh() throws InterruptedException { assertThat(heartbeatRequests) .comparingElementsUsing( Correspondence.from( - (Windmill.HeartbeatRequest h, Work w) -> - h.getWorkToken() == w.getWorkItem().getWorkToken() - && h.getCacheToken() == w.getWorkItem().getWorkToken() - && h.getShardingKey() == w.getWorkItem().getShardingKey(), + (Windmill.HeartbeatRequest h, Work w) -> { + assert h != null; + assert w != null; + return h.getWorkToken() == w.getWorkItem().getWorkToken() + && h.getCacheToken() == w.getWorkItem().getWorkToken() + && h.getShardingKey() == w.getWorkItem().getShardingKey(); + }, "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) .containsExactlyElementsIn(work); } @@ -207,6 +211,32 @@ public void testActiveWorkRefresh() throws InterruptedException { workIsProcessed.countDown(); } + @Test + public void testEmptyActiveWorkRefresh() throws InterruptedException { + int activeWorkRefreshPeriodMillis = 100; + + List computations = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + ComputationState computationState = createComputationState(i); + computations.add(computationState); + } + + CountDownLatch heartbeatsSent = new CountDownLatch(1); + TestClock fakeClock = new TestClock(Instant.now()); + ActiveWorkRefresher activeWorkRefresher = + createActiveWorkRefresher( + fakeClock::now, + activeWorkRefreshPeriodMillis, + 0, + () -> computations, + heartbeats -> heartbeatsSent::countDown); + + activeWorkRefresher.start(); + fakeClock.advance(Duration.millis(activeWorkRefreshPeriodMillis * 2)); + assertFalse(heartbeatsSent.await(500, TimeUnit.MILLISECONDS)); + activeWorkRefresher.stop(); + } + @Test public void testInvalidateStuckCommits() throws InterruptedException { int stuckCommitDurationMillis = 100; From 80ae93217c5ac74e41cbedaeea7806fb0f05c2a9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 5 Aug 2024 09:49:55 -0700 Subject: [PATCH 10/78] Minor optimization for the common case of merging empty string sets. (#31803) --- .../runners/core/metrics/StringSetData.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java index 93dfb8e3ebc80..466d4ad46eb6f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java @@ -19,7 +19,6 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; -import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -50,12 +49,16 @@ public static StringSetData empty() { * Combines this {@link StringSetData} with other, both original StringSetData are left intact. */ public StringSetData combine(StringSetData other) { - // do not merge other on this as this StringSetData might hold an immutable set like in case - // of EmptyStringSetData - Set combined = new HashSet<>(); - combined.addAll(this.stringSet()); - combined.addAll(other.stringSet()); - return StringSetData.create(combined); + if (this.stringSet().isEmpty()) { + return other; + } else if (other.stringSet().isEmpty()) { + return this; + } else { + ImmutableSet.Builder combined = ImmutableSet.builder(); + combined.addAll(this.stringSet()); + combined.addAll(other.stringSet()); + return StringSetData.create(combined.build()); + } } /** From 5b2bfe96f83a5631c3a8d5c3b92a0f695ffe2d7d Mon Sep 17 00:00:00 2001 From: Damon Date: Mon, 5 Aug 2024 10:25:37 -0700 Subject: [PATCH 11/78] [Prism] Enable an artifact resolver for the Prism runner (#32058) * Enable an ArtifactResolver for the Prism runner * Rename class * spotlessApply * Fix Builder instantiation --- .../runners/prism/PrismArtifactResolver.java | 110 ++++++++++++++++++ .../prism/PrismArtifactResolverTest.java | 45 +++++++ 2 files changed, 155 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java new file mode 100644 index 0000000000000..db56bc6047ca7 --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.DefaultArtifactResolver; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; + +/** + * The {@link PrismArtifactResolver} converts a {@link Pipeline} to a {@link RunnerApi.Pipeline} via + * resolving {@link RunnerApi.ArtifactInformation}. + */ +@AutoValue +abstract class PrismArtifactResolver { + + /** + * Instantiates a {@link PrismArtifactResolver} from the {@param pipeline}, applying defaults to + * the remaining dependencies. + */ + static PrismArtifactResolver of(Pipeline pipeline) { + return PrismArtifactResolver.builder().setPipeline(pipeline).build(); + } + + static Builder builder() { + return new AutoValue_PrismArtifactResolver.Builder(); + } + + /** + * Converts the {@link #getPipeline()} using {@link PipelineTranslation#toProto} and {@link + * #getDelegate()}'s {@link + * org.apache.beam.sdk.util.construction.ArtifactResolver#resolveArtifacts}. + */ + RunnerApi.Pipeline resolvePipelineProto() { + RunnerApi.Pipeline result = PipelineTranslation.toProto(getPipeline(), getSdkComponents()); + return getDelegate().resolveArtifacts(result); + } + + /** + * {@link PrismArtifactResolver} delegates to {@link + * org.apache.beam.sdk.util.construction.ArtifactResolver} to transform {@link + * RunnerApi.ArtifactInformation}. Defaults to {@link DefaultArtifactResolver#INSTANCE} if not + * set. + */ + abstract org.apache.beam.sdk.util.construction.ArtifactResolver getDelegate(); + + /** The {@link Pipeline} from which {@link PrismArtifactResolver#resolvePipelineProto()}. */ + abstract Pipeline getPipeline(); + + /** + * SDK objects that will be represented by {@link + * org.apache.beam.model.pipeline.v1.RunnerApi.Components}. Instantiated via {@link + * SdkComponents#create(PipelineOptions)} by default, where {@link PipelineOptions} are acquired + * from {@link #getPipeline}'s {@link Pipeline#getOptions}. + */ + abstract SdkComponents getSdkComponents(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setDelegate( + org.apache.beam.sdk.util.construction.ArtifactResolver artifactResolver); + + abstract Optional getDelegate(); + + abstract Builder setSdkComponents(SdkComponents sdkComponents); + + abstract Optional getSdkComponents(); + + abstract Builder setPipeline(Pipeline pipeline); + + abstract Optional getPipeline(); + + abstract PrismArtifactResolver autoBuild(); + + final PrismArtifactResolver build() { + if (!getDelegate().isPresent()) { + setDelegate(DefaultArtifactResolver.INSTANCE); + } + + if (!getSdkComponents().isPresent()) { + checkState(getPipeline().isPresent()); + setSdkComponents(SdkComponents.create(getPipeline().get().getOptions())); + } + + return autoBuild(); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java new file mode 100644 index 0000000000000..ef4646f023477 --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static com.google.common.truth.Truth.assertThat; + +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.Impulse; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PrismArtifactResolver}. */ +@RunWith(JUnit4.class) +public class PrismArtifactResolverTest { + @Test + public void resolvesPipeline() { + Pipeline pipeline = Pipeline.create(); + pipeline.apply(Impulse.create()); + PrismArtifactResolver underTest = PrismArtifactResolver.of(pipeline); + RunnerApi.Pipeline pipelineProto = underTest.resolvePipelineProto(); + RunnerApi.Components components = pipelineProto.getComponents(); + assertThat(components.getTransformsMap()).containsKey("Impulse"); + assertThat(components.getCodersMap()).containsKey("ByteArrayCoder"); + assertThat(components.getEnvironmentsMap()) + .containsKey(BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER)); + } +} From fb49e9644a4b81bdca339d98181c6f21256d474a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 5 Aug 2024 18:00:08 -0400 Subject: [PATCH 12/78] Fix load test dataproc cluster name exceeded allowed length (#32062) --- .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 5ded71a7652a1..e2afb2e2cfd70 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-cogbk-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 061a1b9e210ed..bae2f9f82ee1f 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-pardo-flink-batch-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-pardo-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index bec926ab9656c..4485b7187f800 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-pardo-flink-stream-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-pardo-flink-stream-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar From c60623524ae9998cdfb8bfb1985f218e7dfa823a Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 6 Aug 2024 10:51:42 -0400 Subject: [PATCH 13/78] Beam Website Updates for 2.58.0 Release (#31925) * Beam Website Updates for 2.58.0 Release * Update 2.57.0 links to archive links * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * move solace io to highlights * Add SpannerIO breaking change * fix urls * Update CHANGES.md Co-authored-by: tvalentyn * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update release date * add release date to changes.md --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> Co-authored-by: tvalentyn --- CHANGES.md | 21 +-- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.58.0.md | 130 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 13 +- 4 files changed, 143 insertions(+), 23 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.58.0.md diff --git a/CHANGES.md b/CHANGES.md index b127599ae0aa8..7f12b53342602 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -88,15 +88,10 @@ * ([#X](https://github.com/apache/beam/issues/X)). -# [2.58.0] - Unreleased +# [2.58.0] - 2024-08-06 ## Highlights -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). - -## I/Os - * Support for [Solace](https://solace.com/) source (`SolaceIO.Read`) added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). ## New Features / Improvements @@ -110,25 +105,13 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). * [IcebergIO] IcebergCatalogConfig was changed to support specifying catalog properties in a key-store fashion ([#31726](https://github.com/apache/beam/pull/31726)) * [SpannerIO] Added validation that query and table cannot be specified at the same time for SpannerIO.read(). Previously withQuery overrides withTable, if set ([#24956](https://github.com/apache/beam/issues/24956)). -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). - ## Bugfixes * [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). - -## Security Fixes -* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). - -## Known Issues - -* ([#X](https://github.com/apache/beam/issues/X)). +* Fixed a logging issue where Python worker dependency installation logs sometimes were not emitted in a timely manner ([#31977](https://github.com/apache/beam/pull/31977)) # [2.57.0] - 2024-06-26 diff --git a/website/www/site/config.toml b/website/www/site/config.toml index 7fe6df7a2c7a2..6675cf418bdd9 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.57.0" +release_latest = "2.58.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.58.0.md b/website/www/site/content/en/blog/beam-2.58.0.md new file mode 100644 index 0000000000000..603403cd7fdbc --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.58.0.md @@ -0,0 +1,130 @@ +--- +title: "Apache Beam 2.58.0" +date: 2024-08-06 13:00:00 -0800 +categories: + - blog + - release +authors: + - jrmccluskey +--- + + +We are happy to present the new 2.58.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2580-2024-08-06) for this release. + + + +For more information about changes in 2.58.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/22). + +## I/Os + +* Support for [Solace](https://solace.com/) source (`SolaceIO.Read`) added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). + +## New Features / Improvements + +* Multiple RunInference instances can now share the same model instance by setting the model_identifier parameter (Python) ([#31665](https://github.com/apache/beam/issues/31665)). +* Added options to control the number of Storage API multiplexing connections ([#31721](https://github.com/apache/beam/pull/31721)) +* [BigQueryIO] Better handling for batch Storage Write API when it hits AppendRows throughput quota ([#31837](https://github.com/apache/beam/pull/31837)) +* [IcebergIO] All specified catalog properties are passed through to the connector ([#31726](https://github.com/apache/beam/pull/31726)) +* Removed a third-party LGPL dependency from the Go SDK ([#31765](https://github.com/apache/beam/issues/31765)). +* Support for `MapState` and `SetState` when using Dataflow Runner v1 with Streaming Engine (Java) ([[#18200](https://github.com/apache/beam/issues/18200)]) + +## Breaking Changes + +* [IcebergIO] `IcebergCatalogConfig` was changed to support specifying catalog properties in a key-store fashion ([#31726](https://github.com/apache/beam/pull/31726)) +* [SpannerIO] Added validation that query and table cannot be specified at the same time for `SpannerIO.read()`. Previously `withQuery` overrides `withTable`, if set ([#24956](https://github.com/apache/beam/issues/24956)). + +## Bug fixes + +* [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.58.0 release. Thank you to all contributors! + +Ahmed Abualsaud + +Ahmet Altay + +Alexandre Moueddene + +Alexey Romanenko + +Andrew Crites + +Bartosz Zablocki + +Celeste Zeng + +Chamikara Jayalath + +Clay Johnson + +Damon Douglass + +Danny McCormick + +Dilnaz Amanzholova + +Florian Bernard + +Francis O'Hara + +George Ma + +Israel Herraiz + +Jack McCluskey + +Jaehyeon Kim + +James Roseman + +Kenneth Knowles + +Maciej Szwaja + +Michel Davit + +Minh Son Nguyen + +Naireen + +Niel Markwick + +Oliver Cardoza + +Robert Bradshaw + +Robert Burke + +Rohit Sinha + +S. Veyrié + +Sam Whittle + +Shunping Huang + +Svetak Sundhar + +TongruiLi + +Tony Tang + +Valentyn Tymofieiev + +Vitaly Terentyev + +Yi Hu \ No newline at end of file diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 8f3b92ef9f2a0..b7db1ddd65b6f 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,17 @@ versions denoted `0.x.y`. ## Releases +### 2.58.0 (2024-08-06) +Official [source code download](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.58.0) + ### 2.57.0 (2024-06-26) -Official [source code download](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip). +[SHA-512](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.sha512). +[signature](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.57.0) From d09c3237c8aa1ed48351046fe61bc0cc8794521a Mon Sep 17 00:00:00 2001 From: atask-g Date: Tue, 6 Aug 2024 11:53:36 -0400 Subject: [PATCH 14/78] Added support for the TOKENLIST type in Spanner (#32038) --- .../org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java | 3 +++ .../org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java index 3fd09c63da794..fa44cadeba0a4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java @@ -184,6 +184,9 @@ private static Type parseSpannerType(String spannerType, Dialect dialect) { if (spannerType.startsWith("BYTES")) { return Type.bytes(); } + if ("TOKENLIST".equals(spannerType)) { + return Type.bytes(); + } if ("TIMESTAMP".equals(spannerType)) { return Type.timestamp(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java index 166df1704ca8a..1e89326d1e8c9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java @@ -40,16 +40,18 @@ public void testSingleTable() throws Exception { .addColumn("test", "jsonVal", "JSON") .addColumn("test", "protoVal", "PROTO") .addColumn("test", "enumVal", "ENUM") + .addColumn("test", "tokens", "TOKENLIST") .build(); assertEquals(1, schema.getTables().size()); - assertEquals(6, schema.getColumns("test").size()); + assertEquals(7, schema.getColumns("test").size()); assertEquals(1, schema.getKeyParts("test").size()); assertEquals(Type.json(), schema.getColumns("test").get(3).getType()); assertEquals( Type.proto("customer.app.TestMessage"), schema.getColumns("test").get(4).getType()); assertEquals( Type.protoEnum("customer.app.TestEnum"), schema.getColumns("test").get(5).getType()); + assertEquals(Type.bytes(), schema.getColumns("test").get(6).getType()); } @Test From e9b5dc69532865e4ec20faa13a1ff88552bc50ae Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 12:39:03 -0400 Subject: [PATCH 15/78] Enforce java.nio.charset.StandardCharsets against guava Charsets (#32083) * Enforce java.nio.charset.StandardCharsets against guava Charsets * Fix dependency analyzeClassesDependencies --- .../transforms/FormatTransform.java | 5 ++-- .../streaming/io/StreamingImpulseSource.java | 5 ++-- .../flink/FlinkJobServerDriverTest.java | 12 ++++++---- ...FlinkPipelineExecutionEnvironmentTest.java | 4 ++-- .../runners/flink/FlinkSubmissionTest.java | 4 ++-- .../ExecutableStageDoFnOperatorTest.java | 9 ++++--- .../wrappers/streaming/FlinkKeyUtilsTest.java | 4 ++-- .../dataflow/DataflowPipelineTranslator.java | 7 +++--- .../runners/dataflow/worker/graph/Nodes.java | 6 ++--- .../GroupingShuffleEntryIteratorTest.java | 7 +++--- .../state/WindmillStateInternalsTest.java | 11 +++++---- .../state/WindmillStateReaderTest.java | 8 +++---- .../artifact/ArtifactStagingService.java | 4 ++-- .../state/StateRequestHandlers.java | 5 ++-- .../ArtifactRetrievalServiceTest.java | 4 ++-- .../testing/TestUniversalRunner.java | 5 ++-- .../resources/beam/checkstyle/checkstyle.xml | 8 +++++++ .../providers/LoggingTransformProvider.java | 4 ++-- .../transforms/errorhandling/BadRecord.java | 6 ++--- .../ByteBuddyOnTimerInvokerFactory.java | 6 ++--- .../resourcehints/ResourceHints.java | 8 +++---- .../sdk/coders/StructuralByteArrayTest.java | 10 ++++---- .../apache/beam/sdk/io/FileBasedSinkTest.java | 2 +- .../org/apache/beam/sdk/io/FileIOTest.java | 6 ++--- .../apache/beam/sdk/io/TFRecordIOTest.java | 7 +++--- .../apache/beam/sdk/io/TextIOReadTest.java | 8 +++---- .../apache/beam/sdk/io/TextIOWriteTest.java | 6 ++--- .../sdk/io/TextRowCountEstimatorTest.java | 10 ++++---- .../apache/beam/sdk/io/WriteFilesTest.java | 5 ++-- .../options/PipelineOptionsFactoryTest.java | 24 +++++++++---------- .../sdk/schemas/SchemaTranslationTest.java | 5 ++-- ...fferedElementCountingOutputStreamTest.java | 6 ++--- .../util/ExposedByteArrayInputStreamTest.java | 6 ++--- .../ExposedByteArrayOutputStreamTest.java | 4 ++-- .../beam/sdk/util/SerializableUtilsTest.java | 4 ++-- .../service/ExpansionServiceTest.java | 4 ++-- .../avro/AvroGenericCoderTranslator.java | 6 ++--- .../sdk/extensions/avro/io/AvroIOTest.java | 6 ++--- .../python/PythonExternalTransform.java | 5 ++-- .../sdk/extensions/python/PythonService.java | 5 ++-- sdks/java/extensions/sql/jdbc/build.gradle | 2 +- .../sdk/extensions/sql/jdbc/BeamSqlLine.java | 6 ++--- .../provider/text/TextTableProviderTest.java | 20 +++++++++------- .../sdk/io/clickhouse/ClickHouseWriter.java | 4 ++-- .../ContextualTextIOTest.java | 16 ++++++------- ...PubsubReadSchemaTransformProviderTest.java | 6 ++--- .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 6 ++--- .../apache/beam/sdk/tpcds/QueryReader.java | 4 ++-- .../beam/sdk/tpcds/SqlTransformRunner.java | 4 ++-- .../beam/sdk/tpcds/TableSchemaJSONLoader.java | 4 ++-- .../TransformServiceLauncherTest.java | 10 ++++---- 51 files changed, 182 insertions(+), 161 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java index 2d9089fcd29af..296d7e7d2409b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.complete.kafkatopubsub.transforms; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClass; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.kafka.common.serialization.StringDeserializer; @@ -120,7 +120,8 @@ public PDone expand(PCollection input) { MapElements.into(TypeDescriptor.of(PubsubMessage.class)) .via( (String json) -> - new PubsubMessage(json.getBytes(Charsets.UTF_8), ImmutableMap.of()))) + new PubsubMessage( + json.getBytes(StandardCharsets.UTF_8), ImmutableMap.of()))) .apply( "writePubsubMessagesToPubSub", PubsubIO.writeMessages().to(options.getOutputTopic())); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java index 8f21e42d61e66..871d7a5a39895 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +60,8 @@ public void run(SourceContext> ctx) { while (running && (messageCount == 0 || count < subtaskCount)) { synchronized (ctx.getCheckpointLock()) { ctx.collect( - WindowedValue.valueInGlobalWindow(String.valueOf(count).getBytes(Charsets.UTF_8))); + WindowedValue.valueInGlobalWindow( + String.valueOf(count).getBytes(StandardCharsets.UTF_8))); count++; } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java index 4a628eeb4fdf0..22516cbc96331 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java @@ -25,7 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; /** Tests for {@link FlinkJobServerDriver}. */ @@ -104,7 +104,7 @@ public void testJobServerDriver() throws Exception { boolean success = false; while (!success) { newErr.flush(); - String output = baos.toString(Charsets.UTF_8.name()); + String output = baos.toString(StandardCharsets.UTF_8.name()); if (output.contains("JobService started on localhost:") && output.contains("ArtifactStagingService started on localhost:") && output.contains("ExpansionService started on localhost:")) { @@ -114,7 +114,8 @@ public void testJobServerDriver() throws Exception { } } assertThat(driver.getJobServerUrl(), is(not(nullValue()))); - assertThat(baos.toString(Charsets.UTF_8.name()), containsString(driver.getJobServerUrl())); + assertThat( + baos.toString(StandardCharsets.UTF_8.name()), containsString(driver.getJobServerUrl())); assertThat(driverThread.isAlive(), is(true)); } catch (Throwable t) { // restore to print exception @@ -149,7 +150,7 @@ public void testJobServerDriverWithoutExpansionService() throws Exception { boolean success = false; while (!success) { newErr.flush(); - String output = baos.toString(Charsets.UTF_8.name()); + String output = baos.toString(StandardCharsets.UTF_8.name()); if (output.contains("JobService started on localhost:") && output.contains("ArtifactStagingService started on localhost:")) { success = true; @@ -161,7 +162,8 @@ public void testJobServerDriverWithoutExpansionService() throws Exception { } } assertThat(driver.getJobServerUrl(), is(not(nullValue()))); - assertThat(baos.toString(Charsets.UTF_8.name()), containsString(driver.getJobServerUrl())); + assertThat( + baos.toString(StandardCharsets.UTF_8.name()), containsString(driver.getJobServerUrl())); assertThat(driverThread.isAlive(), is(true)); } catch (Throwable t) { // restore to print exception diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java index 9d898ed53a896..3b92c282c38a7 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java @@ -38,6 +38,7 @@ import java.lang.reflect.Method; import java.net.MalformedURLException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.resources.PipelineResources; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.RemoteEnvironment; @@ -373,7 +373,7 @@ public void processElement(ProcessContext ctx) { } replacementStdErr.flush(); assertThat( - new String(byteArrayOutputStream.toByteArray(), Charsets.UTF_8), + new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8), containsString( "UnboundedSources present which rely on checkpointing, but checkpointing is disabled.")); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java index 22a9ce4f39ab6..cf860717def37 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java @@ -20,6 +20,7 @@ import java.io.File; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.security.Permission; import java.util.Collection; @@ -30,7 +31,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.construction.resources.PipelineResources; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -200,7 +200,7 @@ private static void prepareEnvironment() throws Exception { RestOptions.PORT.key(), flinkCluster.getRestPort()); - Files.write(file.toPath(), config.getBytes(Charsets.UTF_8)); + Files.write(file.toPath(), config.getBytes(StandardCharsets.UTF_8)); // Create a new environment with the location of the Flink config for CliFrontend ImmutableMap newEnv = diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index cf5b2b555124b..2eb0545b77940 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -103,7 +103,6 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -818,7 +817,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing BagState state = // State from the SDK Harness is stored as ByteStrings operator.keyedStateInternals.state( stateNamespace, StateTags.bag(stateId, ByteStringCoder.of())); - state.add(ByteString.copyFrom("userstate".getBytes(Charsets.UTF_8))); + state.add(ByteString.copyFrom("userstate".getBytes(StandardCharsets.UTF_8))); assertThat(testHarness.numKeyedStateEntries(), is(1)); // user timer that fires after the end of the window and after state cleanup @@ -966,7 +965,7 @@ public void testEnsureStateCleanupOnFinalWatermark() throws Exception { BagState state = // State from the SDK Harness is stored as ByteStrings operator.keyedStateInternals.state( stateNamespace, StateTags.bag(stateId, ByteStringCoder.of())); - state.add(ByteString.copyFrom("userstate".getBytes(Charsets.UTF_8))); + state.add(ByteString.copyFrom("userstate".getBytes(StandardCharsets.UTF_8))); // No timers have been set for cleanup assertThat(testHarness.numEventTimeTimers(), is(0)); // State has been created @@ -988,8 +987,8 @@ public void testCacheTokenHandling() throws Exception { new ExecutableStageDoFnOperator.BagUserStateFactory<>( test, stateBackend, NoopLock.get(), null); - ByteString key1 = ByteString.copyFrom("key1", Charsets.UTF_8); - ByteString key2 = ByteString.copyFrom("key2", Charsets.UTF_8); + ByteString key1 = ByteString.copyFrom("key1", StandardCharsets.UTF_8); + ByteString key2 = ByteString.copyFrom("key2", StandardCharsets.UTF_8); Map> userStateMapMock = Mockito.mock(Map.class); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java index cab45632ac552..cdf461b5fde83 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java @@ -22,12 +22,12 @@ import static org.hamcrest.core.Is.is; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Test; /** Tests for {@link FlinkKeyUtils}. */ @@ -66,7 +66,7 @@ public void testCoderContext() throws Exception { @Test @SuppressWarnings("ByteBufferBackingArray") public void testFromEncodedKey() { - ByteString input = ByteString.copyFrom("hello world".getBytes(Charsets.UTF_8)); + ByteString input = ByteString.copyFrom("hello world".getBytes(StandardCharsets.UTF_8)); ByteBuffer encodedKey = FlinkKeyUtils.fromEncodedKey(input); assertThat(encodedKey.array(), is(input.toByteArray())); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index f905e136e83ba..1fedcd8f3a290 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -42,6 +42,7 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.Step; import com.google.api.services.dataflow.model.WorkerPool; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; @@ -110,7 +111,6 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.commons.codec.EncoderException; import org.apache.commons.codec.net.PercentCodec; @@ -618,7 +618,7 @@ static class StepTranslator implements StepTranslationContext { // For compatibility with URL encoding implementations that represent space as +, // always encode + as %2b even though we don't encode space as +. private final PercentCodec percentCodec = - new PercentCodec("+".getBytes(Charsets.US_ASCII), false); + new PercentCodec("+".getBytes(StandardCharsets.US_ASCII), false); private StepTranslator(Translator translator, Step step) { this.translator = translator; @@ -764,7 +764,8 @@ private void addResourceHints(ResourceHints hints) { try { urlEncodedHints.put( entry.getKey(), - new String(percentCodec.encode(entry.getValue().toBytes()), Charsets.US_ASCII)); + new String( + percentCodec.encode(entry.getValue().toBytes()), StandardCharsets.US_ASCII)); } catch (EncoderException e) { // Should never happen. throw new RuntimeException("Invalid value for resource hint: " + entry.getKey(), e); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java index 6092d0d64de5a..d824324170005 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java @@ -29,11 +29,11 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.charset.StandardCharsets; import org.apache.beam.runners.dataflow.worker.util.common.worker.Operation; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputReceiver; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.util.Transport; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; /** Container class for different types of network nodes. All nodes only have reference equality. */ @@ -59,7 +59,7 @@ private static String toStringWithTrimmedLiterals(GenericJson json) { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); final JsonGenerator baseGenerator = MoreObjects.firstNonNull(json.getFactory(), Transport.getJsonFactory()) - .createJsonGenerator(byteStream, Charsets.UTF_8); + .createJsonGenerator(byteStream, StandardCharsets.UTF_8); JsonGenerator generator = new JsonGenerator() { @Override @@ -164,7 +164,7 @@ public void enablePrettyPrint() throws IOException { generator.enablePrettyPrint(); generator.serialize(json); generator.flush(); - return byteStream.toString(Charsets.UTF_8.name()); + return byteStream.toString(StandardCharsets.UTF_8.name()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java index 2421d7faf8240..8c6a003cb72bc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.After; @@ -130,10 +129,10 @@ private void setCurrentExecutionState(String mockOriginalName) { private static ShuffleEntry shuffleEntry(String key, String value) { return new ShuffleEntry( /* use key itself as position */ - ByteArrayShufflePosition.of(key.getBytes(Charsets.UTF_8)), - ByteString.copyFrom(key.getBytes(Charsets.UTF_8)), + ByteArrayShufflePosition.of(key.getBytes(StandardCharsets.UTF_8)), + ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)), ByteString.copyFrom(new byte[0]), - ByteString.copyFrom(value.getBytes(Charsets.UTF_8))); + ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8))); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 33e47623cd0ee..d06ed0f526c79 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -80,7 +80,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -225,7 +224,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -240,7 +239,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyNewKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyNewKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -255,7 +254,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyNewKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyNewKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -2004,7 +2003,9 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) } // clear cache and recreate multimapState - cache.forComputation("comp").invalidate(ByteString.copyFrom("dummyKey", Charsets.UTF_8), 123); + cache + .forComputation("comp") + .invalidate(ByteString.copyFrom("dummyKey", StandardCharsets.UTF_8), 123); resetUnderTest(); multimapState = underTest.state(NAMESPACE, addr); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 8dbfc35192b7d..b06d88bf4bc4e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -27,6 +27,7 @@ import com.google.api.client.util.Lists; import com.google.common.collect.Maps; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; @@ -1151,8 +1151,8 @@ public void testReadSortedListWithContinuations() throws Exception { .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end)) .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)); - final ByteString CONT_1 = ByteString.copyFrom("CONTINUATION_1", Charsets.UTF_8); - final ByteString CONT_2 = ByteString.copyFrom("CONTINUATION_2", Charsets.UTF_8); + final ByteString CONT_1 = ByteString.copyFrom("CONTINUATION_1", StandardCharsets.UTF_8); + final ByteString CONT_2 = ByteString.copyFrom("CONTINUATION_2", StandardCharsets.UTF_8); Windmill.KeyedGetDataResponse.Builder response1 = Windmill.KeyedGetDataResponse.newBuilder() .setKey(DATA_KEY) @@ -1327,7 +1327,7 @@ public void testReadTagValuePrefixWithContinuations() throws Exception { .setStateFamily(STATE_FAMILY) .setFetchMaxBytes(WindmillStateReader.MAX_TAG_VALUE_PREFIX_BYTES)); - final ByteString CONT = ByteString.copyFrom("CONTINUATION", Charsets.UTF_8); + final ByteString CONT = ByteString.copyFrom("CONTINUATION", StandardCharsets.UTF_8); Windmill.KeyedGetDataResponse.Builder response1 = Windmill.KeyedGetDataResponse.newBuilder() .setKey(DATA_KEY) diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java index a8e5e2ab6a882..8c7a356b99392 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; @@ -56,7 +57,6 @@ import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -192,7 +192,7 @@ public void removeStagedArtifacts(String stagingToken) throws IOException { private ResourceId stagingDir(String stagingToken) { return FileSystems.matchNewResource(root, true) .resolve( - Hashing.sha256().hashString(stagingToken, Charsets.UTF_8).toString(), + Hashing.sha256().hashString(stagingToken, StandardCharsets.UTF_8).toString(), ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY); } }; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java index e2c45850dba93..4e1c31744c1a0 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** @@ -655,7 +655,8 @@ private BagUserStateHandler createHandl } private static BeamFnApi.ProcessBundleRequest.CacheToken createCacheToken() { - ByteString token = ByteString.copyFrom(UUID.randomUUID().toString().getBytes(Charsets.UTF_8)); + ByteString token = + ByteString.copyFrom(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); return BeamFnApi.ProcessBundleRequest.CacheToken.newBuilder() .setUserState(BeamFnApi.ProcessBundleRequest.CacheToken.UserState.getDefaultInstance()) .setToken(token) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java index d6b48a936135e..4d19e87c3d11c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -33,7 +34,6 @@ import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; @@ -74,7 +74,7 @@ private void stageFiles(Map files) throws IOException { for (Map.Entry entry : files.entrySet()) { Files.write( Paths.get(stagingDir.toString(), entry.getKey()), - entry.getValue().getBytes(Charsets.UTF_8)); + entry.getValue().getBytes(StandardCharsets.UTF_8)); } } diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java index 533106869c62b..a36c1e8b2efbd 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java @@ -21,6 +21,7 @@ import com.google.auto.service.AutoService; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import org.apache.beam.runners.portability.PortableRunner; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; @@ -65,7 +65,8 @@ public PipelineResult run(Pipeline pipeline) { testOptions.setJobEndpoint( "localhost:" + new String( - Files.readAllBytes(Paths.get(localServicePortFilePath)), Charsets.UTF_8) + Files.readAllBytes(Paths.get(localServicePortFilePath)), + StandardCharsets.UTF_8) .trim()); } catch (IOException e) { throw new RuntimeException( diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml index 3c4cfdfbc6f58..5cee5d2f33e2a 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml @@ -119,6 +119,14 @@ page at http://checkstyle.sourceforge.net/config.html --> + + + + + + + + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java index 25efaeae2a0ef..2908171f5c02c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java @@ -19,6 +19,7 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; import java.util.Map; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; @@ -166,7 +166,7 @@ private static DoFn createDoFn(Level logLevel, String prefix, Schema r return new DoFn() { @ProcessElement public void processElement(@Element Row row, OutputReceiver out) { - String msg = prefix + new String(fn.apply(row), Charsets.UTF_8); + String msg = prefix + new String(fn.apply(row), StandardCharsets.UTF_8); // Looks like this is the best we can do. // https://stackoverflow.com/questions/2621701/setting-log-level-of-message-at-runtime-in-slf4j switch (logLevel) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java index fd49078350c48..558f912a6b1ff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.PrintStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -202,11 +202,11 @@ public abstract static class Builder { public Builder addExceptionStackTrace(Exception exception) throws IOException { ByteArrayOutputStream stream = new ByteArrayOutputStream(); - PrintStream printStream = new PrintStream(stream, false, Charsets.UTF_8.name()); + PrintStream printStream = new PrintStream(stream, false, StandardCharsets.UTF_8.name()); exception.printStackTrace(printStream); printStream.close(); - this.setExceptionStacktrace(new String(stream.toByteArray(), Charsets.UTF_8)); + this.setExceptionStacktrace(new String(stream.toByteArray(), StandardCharsets.UTF_8)); return this; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java index 7b9ac7e15c2e8..e318e82513ca4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java @@ -22,6 +22,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ExecutionException; import net.bytebuddy.ByteBuddy; import net.bytebuddy.description.modifier.FieldManifestation; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.DoFn.TimerId; import org.apache.beam.sdk.transforms.reflect.ByteBuddyDoFnInvokerFactory.DoFnMethodWithExtraParametersDelegation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CharMatcher; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache; @@ -191,7 +191,7 @@ public Constructor load(final OnTimerMethodSpecifier onTimerMethodSpecifier) "%s$%s$%s", OnTimerInvoker.class.getSimpleName(), CharMatcher.javaLetterOrDigit().retainFrom(timerId), - BaseEncoding.base64().omitPadding().encode(timerId.getBytes(Charsets.UTF_8))); + BaseEncoding.base64().omitPadding().encode(timerId.getBytes(StandardCharsets.UTF_8))); DynamicType.Builder builder = new ByteBuddy() @@ -241,7 +241,7 @@ public Constructor load(final OnTimerMethodSpecifier onTimerMethodSpecifier) "%s$%s$%s", OnTimerInvoker.class.getSimpleName(), CharMatcher.javaLetterOrDigit().retainFrom(timerId), - BaseEncoding.base64().omitPadding().encode(timerId.getBytes(Charsets.UTF_8))); + BaseEncoding.base64().omitPadding().encode(timerId.getBytes(StandardCharsets.UTF_8))); DynamicType.Builder builder = new ByteBuddy() diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java index 2f034626acd77..527a699568f40 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -28,7 +29,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardResourceHints; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ProtocolMessageEnum; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -179,7 +179,7 @@ public ResourceHint mergeWithOuter(ResourceHint outer) { @Override public byte[] toBytes() { - return String.valueOf(value).getBytes(Charsets.US_ASCII); + return String.valueOf(value).getBytes(StandardCharsets.US_ASCII); } } @@ -196,7 +196,7 @@ public static String parse(String s) { @Override public byte[] toBytes() { - return value.getBytes(Charsets.US_ASCII); + return value.getBytes(StandardCharsets.US_ASCII); } @Override @@ -254,7 +254,7 @@ public ResourceHint mergeWithOuter(ResourceHint outer) { @Override public byte[] toBytes() { - return String.valueOf(value).getBytes(Charsets.US_ASCII); + return String.valueOf(value).getBytes(StandardCharsets.US_ASCII); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java index bd8fdd84fb096..cb0845796fe9c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -32,10 +32,10 @@ public final class StructuralByteArrayTest { @Test public void testStructuralByteArray() throws Exception { assertEquals( - new StructuralByteArray("test string".getBytes(Charsets.UTF_8)), - new StructuralByteArray("test string".getBytes(Charsets.UTF_8))); + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8)), + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8))); assertFalse( - new StructuralByteArray("test string".getBytes(Charsets.UTF_8)) - .equals(new StructuralByteArray("diff string".getBytes(Charsets.UTF_8)))); + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8)) + .equals(new StructuralByteArray("diff string".getBytes(StandardCharsets.UTF_8)))); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java index 7fd54039b1dda..c4f83954e66cc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.sdk.io.WriteFiles.UNKNOWN_SHARDNUM; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets.UTF_8; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java index b87c9caa12441..90b0822d9dcaa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java @@ -31,6 +31,7 @@ import java.io.OutputStreamWriter; import java.io.Serializable; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.CopyOption; import java.nio.file.Files; import java.nio.file.Path; @@ -69,7 +70,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -368,10 +368,10 @@ public void testMatchWatchForNewFiles() throws IOException, InterruptedException public void testRead() throws IOException { final String path = tmpFolder.newFile("file").getAbsolutePath(); final String pathGZ = tmpFolder.newFile("file.gz").getAbsolutePath(); - Files.write(new File(path).toPath(), "Hello world".getBytes(Charsets.UTF_8)); + Files.write(new File(path).toPath(), "Hello world".getBytes(StandardCharsets.UTF_8)); try (Writer writer = new OutputStreamWriter( - new GZIPOutputStream(new FileOutputStream(pathGZ)), Charsets.UTF_8)) { + new GZIPOutputStream(new FileOutputStream(pathGZ)), StandardCharsets.UTF_8)) { writer.write("Hello world"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java index acde8c91431da..a38faf077e073 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java @@ -67,7 +67,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; @@ -212,7 +211,7 @@ public void testWriteTwo() throws Exception { @Category(NeedsRunner.class) public void testReadInvalidRecord() throws Exception { expectedException.expectMessage("Not a valid TFRecord. Fewer than 12 bytes."); - runTestRead("bar".getBytes(Charsets.UTF_8), new String[0]); + runTestRead("bar".getBytes(StandardCharsets.UTF_8), new String[0]); } @Test @@ -445,14 +444,14 @@ private static Iterable makeLines(int n, int minRecordSize) { static class ByteArrayToString extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - c.output(new String(c.element(), Charsets.UTF_8)); + c.output(new String(c.element(), StandardCharsets.UTF_8)); } } static class StringToByteArray extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - c.output(c.element().getBytes(Charsets.UTF_8)); + c.output(c.element().getBytes(StandardCharsets.UTF_8)); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 253308d1b93f0..8d9adbefd02bf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -51,6 +51,7 @@ import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -91,7 +92,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -642,7 +642,7 @@ private void runTestRead(String[] expected) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -865,7 +865,7 @@ public void testProgressEmptyFile() throws IOException { public void testProgressTextFile() throws IOException { String file = "line1\nline2\nline3"; try (BoundedSource.BoundedReader reader = - prepareSource(file.getBytes(Charsets.UTF_8)) + prepareSource(file.getBytes(StandardCharsets.UTF_8)) .createReader(PipelineOptionsFactory.create())) { // Check preconditions before starting assertEquals(0.0, reader.getFractionConsumed(), 1e-6); @@ -901,7 +901,7 @@ public void testProgressTextFile() throws IOException { @Test public void testProgressAfterSplitting() throws IOException { String file = "line1\nline2\nline3"; - BoundedSource source = prepareSource(file.getBytes(Charsets.UTF_8)); + BoundedSource source = prepareSource(file.getBytes(StandardCharsets.UTF_8)); BoundedSource remainder; // Create the remainder, verifying properties pre- and post-splitting. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java index 312605f3fcc5e..695ff4474d715 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java @@ -39,6 +39,7 @@ import java.io.OutputStream; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -73,7 +74,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Functions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; @@ -492,7 +492,7 @@ private static void assertOutputFiles( List expectedElements = new ArrayList<>(elems.length); for (String elem : elems) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); expectedElements.add(line); } @@ -509,7 +509,7 @@ private static void assertOutputFiles( private static List readLinesFromFile(File f) throws IOException { List currentFile = new ArrayList<>(); - try (BufferedReader reader = Files.newBufferedReader(f.toPath(), Charsets.UTF_8)) { + try (BufferedReader reader = Files.newBufferedReader(f.toPath(), StandardCharsets.UTF_8)) { while (true) { String line = reader.readLine(); if (line == null) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java index 17ca3ba85fd81..e52d4112e11e2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java @@ -20,8 +20,8 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.Writer; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Files; import org.junit.Assert; import org.junit.Rule; @@ -38,7 +38,7 @@ public class TextRowCountEstimatorTest { @Test public void testNonEmptyFiles() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 100; i++) { writer.write("123123123\n"); } @@ -47,7 +47,7 @@ public void testNonEmptyFiles() throws Exception { temporaryFolder.newFolder("testfolder"); temporaryFolder.newFolder("testfolder2"); file1 = temporaryFolder.newFile("testfolder/test2.txt"); - writer = Files.newWriter(file1, Charsets.UTF_8); + writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 50; i++) { writer.write("123123123\n"); } @@ -71,7 +71,7 @@ public void testEmptyFolder() throws Exception { @Test public void testEmptyFile() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 100; i++) { writer.write("\n"); } @@ -86,7 +86,7 @@ public void testEmptyFile() throws Exception { @Test(expected = TextRowCountEstimator.NoEstimationException.class) public void lotsOfNewLines() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 1000; i++) { writer.write("\n"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 0ab8efac7eb1a..cc174002bb464 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -35,6 +35,7 @@ import java.io.BufferedReader; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.text.DecimalFormat; import java.util.ArrayList; @@ -93,7 +94,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.ShardedKey; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -1035,7 +1035,8 @@ static void checkFileContents( List actual = Lists.newArrayList(); for (File outputFile : outputFiles) { List actualShard = Lists.newArrayList(); - try (BufferedReader reader = Files.newBufferedReader(outputFile.toPath(), Charsets.UTF_8)) { + try (BufferedReader reader = + Files.newBufferedReader(outputFile.toPath(), StandardCharsets.UTF_8)) { for (; ; ) { String line = reader.readLine(); if (line == null) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 2643fb556ff47..291bb52978808 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -59,6 +59,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.List; import java.util.Map; @@ -74,7 +75,6 @@ import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1727,7 +1727,7 @@ public void testWhenNoHelpIsRequested() { assertFalse( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertEquals("", output); } @@ -1739,7 +1739,7 @@ public void testDefaultHelpAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("Use --help= for detailed help.")); @@ -1753,7 +1753,7 @@ public void testSpecificHelpAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1769,7 +1769,7 @@ public void testSpecificHelpAsArgumentWithSimpleClassName() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1785,7 +1785,7 @@ public void testSpecificHelpAsArgumentWithClassNameSuffix() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1815,7 +1815,7 @@ public void testShortnameSpecificHelpHasMultipleMatches() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("Multiple matches found for NameConflict")); assertThat( output, @@ -1839,7 +1839,7 @@ public void testHelpWithOptionThatOutputsValidEnumTypes() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("")); } @@ -1851,7 +1851,7 @@ public void testHelpWithBadOptionNameAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("Unable to find option org.apache.beam.sdk.Pipeline")); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); @@ -1865,7 +1865,7 @@ public void testHelpWithHiddenMethodAndInterface() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); // A hidden interface. assertThat( output, not(containsString("org.apache.beam.sdk.options.DataflowPipelineDebugOptions"))); @@ -1877,7 +1877,7 @@ public void testHelpWithHiddenMethodAndInterface() { public void testProgrammaticPrintHelp() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PipelineOptionsFactory.printHelp(new PrintStream(baos)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); } @@ -1886,7 +1886,7 @@ public void testProgrammaticPrintHelp() { public void testProgrammaticPrintHelpForSpecificType() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PipelineOptionsFactory.printHelp(new PrintStream(baos), PipelineOptions.class); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index bd7a0da394ae5..3b22addbf5455 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertThrows; import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.HashMap; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -243,7 +243,8 @@ public static Iterable data() { .setUrn("pythonsdk:value") .setPayload( ByteString.copyFrom( - "some payload describing a python type", Charsets.UTF_8)) + "some payload describing a python type", + StandardCharsets.UTF_8)) .setRepresentation( SchemaApi.FieldType.newBuilder() .setAtomicType(SchemaApi.AtomicType.BYTES)) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 5298d29dad101..0c9e0065f5a64 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -29,12 +29,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Random; import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.collection.IsIterableContainingInOrder; import org.junit.Rule; @@ -165,7 +165,7 @@ public void testWritingByteWhenFinishedThrows() throws Exception { public void testWritingBytesWhenFinishedThrows() throws Exception { expectedException.expect(IOException.class); expectedException.expectMessage("Stream has been finished."); - testValues(toBytes("a")).write("b".getBytes(Charsets.UTF_8)); + testValues(toBytes("a")).write("b".getBytes(StandardCharsets.UTF_8)); } @Test @@ -203,7 +203,7 @@ public void testBehaviorWhenBufferPoolEmpty() throws Exception { private List toBytes(String... values) { ImmutableList.Builder builder = ImmutableList.builder(); for (String value : values) { - builder.add(value.getBytes(Charsets.UTF_8)); + builder.add(value.getBytes(StandardCharsets.UTF_8)); } return builder.build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java index e87f6a2b0d0a6..d26794274653a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java @@ -24,7 +24,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,7 +33,7 @@ @RunWith(JUnit4.class) public class ExposedByteArrayInputStreamTest { - private static final byte[] TEST_DATA = "Hello World!".getBytes(Charsets.UTF_8); + private static final byte[] TEST_DATA = "Hello World!".getBytes(StandardCharsets.UTF_8); private ByteArrayInputStream stream = new ByteArrayInputStream(TEST_DATA); @@ -74,6 +74,6 @@ public void testReadPartial() throws IOException { public void testReadAllAfterReadPartial() throws IOException { assertNotEquals(-1, exposedStream.read()); byte[] ret = exposedStream.readAll(); - assertArrayEquals("ello World!".getBytes(Charsets.UTF_8), ret); + assertArrayEquals("ello World!".getBytes(StandardCharsets.UTF_8), ret); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java index 7e1b213c85b25..a4a105a89ddc7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java @@ -25,7 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.UnsupportedEncodingException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -34,7 +34,7 @@ @RunWith(JUnit4.class) public class ExposedByteArrayOutputStreamTest { - private static final byte[] TEST_DATA = "Hello World!".getBytes(Charsets.UTF_8); + private static final byte[] TEST_DATA = "Hello World!".getBytes(StandardCharsets.UTF_8); private ExposedByteArrayOutputStream exposedStream = new ExposedByteArrayOutputStream(); private ByteArrayOutputStream stream = new ByteArrayOutputStream(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java index e15bd42dc3ce3..1f3ec0f427b4a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java @@ -24,12 +24,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; @@ -93,7 +93,7 @@ public void testDeserializationError() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("unable to deserialize a bogus string"); SerializableUtils.deserializeFromByteArray( - "this isn't legal".getBytes(Charsets.UTF_8), "a bogus string"); + "this isn't legal".getBytes(StandardCharsets.UTF_8), "a bogus string"); } /** A class that is not serializable by Java. */ diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java index 3bd87c2ae5c75..1c8d515d5c85e 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java @@ -33,6 +33,7 @@ import com.google.auto.value.AutoValue; import java.io.IOException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -60,7 +61,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -84,7 +84,7 @@ public class ExpansionServiceTest { private ExpansionService expansionService = new ExpansionService(); public static final List BYTE_LIST = ImmutableList.of("testing", "compound", "coders").stream() - .map(str -> str.getBytes(Charsets.UTF_8)) + .map(str -> str.getBytes(StandardCharsets.UTF_8)) .collect(Collectors.toList()); public static final Map BYTE_KV_LIST = ImmutableList.of("testing", "compound", "coders").stream() diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java index 67f386411d810..e56b95d7f8a6b 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.avro; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; import org.apache.avro.Schema; @@ -24,7 +25,6 @@ import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.CoderTranslator; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; /** Coder translator for AvroGenericCoder. */ public class AvroGenericCoderTranslator implements CoderTranslator { @@ -35,13 +35,13 @@ public List> getComponents(AvroGenericCoder from) { @Override public byte[] getPayload(AvroGenericCoder from) { - return from.getSchema().toString().getBytes(Charsets.UTF_8); + return from.getSchema().toString().getBytes(StandardCharsets.UTF_8); } @Override public AvroGenericCoder fromComponents( List> components, byte[] payload, TranslationContext context) { - Schema schema = new Schema.Parser().parse(new String(payload, Charsets.UTF_8)); + Schema schema = new Schema.Parser().parse(new String(payload, StandardCharsets.UTF_8)); return AvroGenericCoder.of(schema); } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java index 30a1a77872520..2a0bc36f6e9eb 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java @@ -36,6 +36,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -100,7 +101,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1436,7 +1436,7 @@ public void testMetadata() throws Exception { "longKey", 100L, "bytesKey", - "bytesValue".getBytes(Charsets.UTF_8)))); + "bytesValue".getBytes(StandardCharsets.UTF_8)))); writePipeline.run(); try (DataFileStream dataFileStream = @@ -1444,7 +1444,7 @@ public void testMetadata() throws Exception { assertEquals("stringValue", dataFileStream.getMetaString("stringKey")); assertEquals(100L, dataFileStream.getMetaLong("longKey")); assertArrayEquals( - "bytesValue".getBytes(Charsets.UTF_8), dataFileStream.getMeta("bytesKey")); + "bytesValue".getBytes(StandardCharsets.UTF_8), dataFileStream.getMeta("bytesKey")); } } diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index e4e5f35334456..c23a771f3cc8b 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -61,7 +62,6 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -499,7 +499,8 @@ public OutputT expand(InputT input) { requirementsFile.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), + StandardCharsets.UTF_8)) { for (String pkg : extraPackages) { fout.write(pkg); fout.write('\n'); diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java index 4392f23c46360..ab4d02ec838d0 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java @@ -24,13 +24,13 @@ import java.io.InputStreamReader; import java.net.ServerSocket; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeoutException; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; import org.slf4j.Logger; @@ -106,7 +106,8 @@ public AutoCloseable start() throws IOException, InterruptedException { new ProcessBuilder(bootstrapCommand).redirectError(ProcessBuilder.Redirect.INHERIT).start(); bootstrap.getOutputStream().close(); BufferedReader reader = - new BufferedReader(new InputStreamReader(bootstrap.getInputStream(), Charsets.UTF_8)); + new BufferedReader( + new InputStreamReader(bootstrap.getInputStream(), StandardCharsets.UTF_8)); String lastLine = reader.readLine(); String lastNonEmptyLine = lastLine; while (lastLine != null) { diff --git a/sdks/java/extensions/sql/jdbc/build.gradle b/sdks/java/extensions/sql/jdbc/build.gradle index 41fddce7116ab..c5d462e0f5cad 100644 --- a/sdks/java/extensions/sql/jdbc/build.gradle +++ b/sdks/java/extensions/sql/jdbc/build.gradle @@ -35,11 +35,11 @@ dependencies { implementation "jline:jline:2.14.6" permitUnusedDeclared "jline:jline:2.14.6" // BEAM-11761 implementation "sqlline:sqlline:1.4.0" - implementation library.java.vendored_guava_32_1_2_jre implementation library.java.vendored_calcite_1_28_0 permitUnusedDeclared library.java.vendored_calcite_1_28_0 testImplementation project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntimeMigration") testImplementation library.java.junit + testImplementation library.java.vendored_guava_32_1_2_jre // Depending on outputs so integrationTest can run with only test dependencies. // This enables us to test the JDBC jar being loaded on a custom classloader. integrationTest sourceSets.test.output diff --git a/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java b/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java index ac049608ebcbd..8c87343cd7c11 100644 --- a/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java +++ b/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java @@ -23,10 +23,10 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.checkerframework.checker.nullness.qual.Nullable; import sqlline.SqlLine; import sqlline.SqlLine.Status; @@ -68,11 +68,11 @@ static Status runSqlLine( SqlLine sqlLine = new SqlLine(); if (outputStream != null) { - sqlLine.setOutputStream(new PrintStream(outputStream, false, Charsets.UTF_8.name())); + sqlLine.setOutputStream(new PrintStream(outputStream, false, StandardCharsets.UTF_8.name())); } if (errorStream != null) { - sqlLine.setErrorStream(new PrintStream(errorStream, false, Charsets.UTF_8.name())); + sqlLine.setErrorStream(new PrintStream(errorStream, false, StandardCharsets.UTF_8.name())); } return sqlLine.begin(modifiedArgs, inputStream, true); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java index e5a46f877001a..e34106db1d936 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import java.io.File; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import org.apache.beam.sdk.extensions.sql.SqlTransform; import org.apache.beam.sdk.schemas.Schema; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -81,7 +81,7 @@ protected void after() {} public void testLegacyDefaultCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello,13\n\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + "hello,13\n\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = String.format( @@ -105,7 +105,7 @@ public void testLegacyDefaultCsv() throws Exception { public void testLegacyTdfCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello\t13\n\ngoodbye\t42\n".getBytes(Charsets.UTF_8)); + "hello\t13\n\ngoodbye\t42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -138,7 +138,7 @@ public void testLegacyTdfCsv() throws Exception { public void testExplicitCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello,13\n\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + "hello,13\n\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -165,7 +165,8 @@ public void testExplicitCsv() throws Exception { @Test public void testExplicitCsvExcel() throws Exception { Files.write( - tempFolder.newFile("test.csv").toPath(), "hello\n\ngoodbye\n".getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.csv").toPath(), + "hello\n\ngoodbye\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -191,7 +192,8 @@ public void testExplicitCsvExcel() throws Exception { public void testLines() throws Exception { // Data that looks like CSV but isn't parsed as it Files.write( - tempFolder.newFile("test.csv").toPath(), "hello,13\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.csv").toPath(), + "hello,13\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -210,7 +212,8 @@ public void testLines() throws Exception { @Test public void testJson() throws Exception { - Files.write(tempFolder.newFile("test.json").toPath(), JSON_TEXT.getBytes(Charsets.UTF_8)); + Files.write( + tempFolder.newFile("test.json").toPath(), JSON_TEXT.getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -229,7 +232,8 @@ public void testJson() throws Exception { public void testInvalidJson() throws Exception { File deadLetterFile = new File(tempFolder.getRoot(), "dead-letter-file"); Files.write( - tempFolder.newFile("test.json").toPath(), INVALID_JSON_TEXT.getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.json").toPath(), + INVALID_JSON_TEXT.getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java index c8c49a656e3be..09a6ced44d379 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java @@ -21,12 +21,12 @@ import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.data.BinaryStreamUtils; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.clickhouse.TableSchema.ColumnType; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.RowWithStorage; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Days; import org.joda.time.Instant; @@ -60,7 +60,7 @@ static void writeValue(ClickHouseOutputStream stream, ColumnType columnType, Obj byte[] bytes; if (value instanceof String) { - bytes = ((String) value).getBytes(Charsets.UTF_8); + bytes = ((String) value).getBytes(StandardCharsets.UTF_8); } else { bytes = ((byte[]) value); } diff --git a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java index 48904cae430f1..2cc89a2a0dc19 100644 --- a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java +++ b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java @@ -44,6 +44,7 @@ import java.io.OutputStream; import java.io.PrintStream; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -87,7 +88,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -524,7 +524,7 @@ public String createFileFromList(List input) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : input) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -548,7 +548,7 @@ public void multipleFilesTest() throws Exception { for (int lineNum = 0; lineNum < numLines; ++lineNum) { String elem = filename + " " + lineNum; byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -588,7 +588,7 @@ public void testWithHintMatchesManyFiles() throws IOException { for (int lineNum = 0; lineNum < 10 + num; ++lineNum) { String elem = filename + " " + lineNum; byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -817,7 +817,7 @@ private void runTestRead(String[] expected) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -840,7 +840,7 @@ private void runTestReadLineNumsAndFileName(String[] expected) throws Exception int lineNum = 0; for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); actualExpected.add(lineNum + " " + filePath + " " + line); lineNum++; @@ -1121,7 +1121,7 @@ public void testProgressEmptyFile() throws IOException { public void testProgressTextFile() throws IOException { String file = "line1\nline2\nline3"; try (BoundedSource.BoundedReader reader = - prepareSource(file.getBytes(Charsets.UTF_8)) + prepareSource(file.getBytes(StandardCharsets.UTF_8)) .createReader(PipelineOptionsFactory.create())) { // Check preconditions before starting assertEquals(0.0, reader.getFractionConsumed(), 1e-6); @@ -1157,7 +1157,7 @@ public void testProgressTextFile() throws IOException { @Test public void testProgressAfterSplitting() throws IOException { String file = "line1\nline2\nline3"; - BoundedSource source = prepareSource(file.getBytes(Charsets.UTF_8)); + BoundedSource source = prepareSource(file.getBytes(StandardCharsets.UTF_8)); BoundedSource remainder; // Create the remainder, verifying properties pre- and post-splitting. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index dd5a9abd5ac8e..98aade888a33d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -25,6 +25,7 @@ import com.google.protobuf.Timestamp; import java.io.IOException; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -47,7 +48,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Rule; @@ -170,7 +170,7 @@ public void testReadRaw() throws IOException { PCollectionRowTuple begin = PCollectionRowTuple.empty(p); Schema rawSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); - byte[] payload = "some payload".getBytes(Charsets.UTF_8); + byte[] payload = "some payload".getBytes(StandardCharsets.UTF_8); try (PubsubTestClientFactory clientFactory = clientFactory(ImmutableList.of(incomingMessageOf(payload, CLOCK.currentTimeMillis())))) { @@ -211,7 +211,7 @@ public void testReadAttributes() throws IOException { .addStringField("attr") .addMapField("attrMap", Schema.FieldType.STRING, Schema.FieldType.STRING) .build(); - byte[] payload = "some payload".getBytes(Charsets.UTF_8); + byte[] payload = "some payload".getBytes(StandardCharsets.UTF_8); String attr = "attr value"; try (PubsubTestClientFactory clientFactory = diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index 612b20393d789..6ee3d9d96ef68 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -57,7 +58,6 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -257,8 +257,8 @@ public synchronized ConsumerRecords poll(Duration timeout) { topicPartition.topic(), topicPartition.partition(), startOffset + i, - key.getBytes(Charsets.UTF_8), - value.getBytes(Charsets.UTF_8))); + key.getBytes(StandardCharsets.UTF_8), + value.getBytes(StandardCharsets.UTF_8))); } if (records.isEmpty()) { return ConsumerRecords.empty(); diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java index 4983d52a642f5..8071bad84d73b 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.tpcds; +import java.nio.charset.StandardCharsets; import java.util.Set; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; /** @@ -39,7 +39,7 @@ public class QueryReader { */ public static String readQuery(String queryFileName) throws Exception { String path = "queries/" + queryFileName + ".sql"; - return Resources.toString(Resources.getResource(path), Charsets.UTF_8); + return Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8); } /** diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index 1550a25b7c8f1..6efb7e7e06598 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -21,6 +21,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; import org.apache.commons.csv.CSVFormat; @@ -212,7 +212,7 @@ private static PCollection getTableCSV( private static org.apache.avro.Schema getAvroSchema(String tableName) throws IOException { String path = "schemas_avro/" + tableName + ".json"; return new org.apache.avro.Schema.Parser() - .parse(Resources.toString(Resources.getResource(path), Charsets.UTF_8)); + .parse(Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8)); } static org.apache.avro.Schema getProjectedSchema( diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java index 485fa83a4a8ee..97116e14cdcd5 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java @@ -20,11 +20,11 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.reflect.ClassPath; import org.json.simple.JSONArray; @@ -49,7 +49,7 @@ public class TableSchemaJSONLoader { @SuppressWarnings({"rawtypes", "DefaultCharset"}) public static String parseTableSchema(String tableName) throws Exception { String path = "schemas/" + tableName + ".json"; - String schema = Resources.toString(Resources.getResource(path), Charsets.UTF_8); + String schema = Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8); JSONObject jsonObject = (JSONObject) new JSONParser().parse(schema); JSONArray jsonArray = (JSONArray) jsonObject.get("schema"); diff --git a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java index a9ae5360a8598..b766d2b13a4bc 100644 --- a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java +++ b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java @@ -25,12 +25,12 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.UUID; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -77,7 +77,7 @@ public void testLauncherInstallsDependencies() throws IOException { try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("pypipackage1\n"); fout.write("pypipackage2\n"); } @@ -118,7 +118,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { dependency1.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(dependency1.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(dependency1.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("tempdata\n"); } @@ -128,7 +128,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { dependency2.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(dependency2.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(dependency2.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("tempdata\n"); } @@ -140,7 +140,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { requirementsFile.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write(dependency1.getAbsolutePath() + "\n"); fout.write(dependency2.getAbsolutePath() + "\n"); fout.write("pypipackage" + "\n"); From 99a23830037f58178d3fdf9db22f27b4de37dac4 Mon Sep 17 00:00:00 2001 From: Damon Date: Tue, 6 Aug 2024 09:48:47 -0700 Subject: [PATCH 16/78] Enable artifact staging during Prism Runner lifecycle (#32084) --- runners/prism/java/build.gradle | 1 + .../runners/prism/PrismArtifactStager.java | 173 ++++++++++++++++++ .../prism/PrismArtifactStagerTest.java | 143 +++++++++++++++ 3 files changed, 317 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java diff --git a/runners/prism/java/build.gradle b/runners/prism/java/build.gradle index 2b0635ca61255..96ab4e70a5792 100644 --- a/runners/prism/java/build.gradle +++ b/runners/prism/java/build.gradle @@ -30,6 +30,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:harness", configuration: "shadow") + implementation project(":runners:java-fn-execution") implementation project(":runners:portability:java") implementation library.java.joda_time diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java new file mode 100644 index 0000000000000..f1d99a213eea3 --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc; +import org.apache.beam.model.jobmanagement.v1.JobApi; +import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; +import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; +import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stages {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline} artifacts of prepared jobs. + */ +@AutoValue +abstract class PrismArtifactStager implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(PrismArtifactStager.class); + + /** + * Instantiate a {@link PrismArtifactStager} via call to {@link #of(String, String)}, assigning + * {@link Builder#setStagingEndpoint} using {@param prepareJobResponse} {@link + * JobApi.PrepareJobResponse#getArtifactStagingEndpoint} and {@link + * JobApi.PrepareJobResponse#getStagingSessionToken}. + */ + static PrismArtifactStager of(JobApi.PrepareJobResponse prepareJobResponse) { + return of( + prepareJobResponse.getArtifactStagingEndpoint().getUrl(), + prepareJobResponse.getStagingSessionToken()); + } + + /** + * Instantiates a {@link PrismArtifactStager} from the {@param stagingEndpoint} URL and {@param + * stagingSessionToken} to instantiate the {@link #getRetrievalService}, {@link + * #getManagedChannel}, and {@link #getStagingServiceStub} defaults. See the referenced getters + * for more details. + */ + static PrismArtifactStager of(String stagingEndpoint, String stagingSessionToken) { + return PrismArtifactStager.builder() + .setStagingEndpoint(stagingEndpoint) + .setStagingSessionToken(stagingSessionToken) + .build(); + } + + static Builder builder() { + return new AutoValue_PrismArtifactStager.Builder(); + } + + /** + * Stage the {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline} artifacts via {@link + * ArtifactStagingService#offer} supplying {@link #getRetrievalService}, {@link + * #getStagingServiceStub}, and {@link #getStagingSessionToken}. + */ + void stage() throws ExecutionException, InterruptedException { + LOG.info("staging artifacts at {}", getStagingEndpoint()); + ArtifactStagingService.offer( + getRetrievalService(), getStagingServiceStub(), getStagingSessionToken()); + } + + /** The URL of the {@link ArtifactStagingService}. */ + abstract String getStagingEndpoint(); + + /** + * Token associated with a staging session and acquired from a {@link + * JobServiceGrpc.JobServiceStub#prepare}'s {@link JobApi.PrepareJobResponse}. + */ + abstract String getStagingSessionToken(); + + /** + * The service that retrieves artifacts; defaults to instantiating from the default {@link + * ArtifactRetrievalService#ArtifactRetrievalService()} constructor. + */ + abstract ArtifactRetrievalService getRetrievalService(); + + /** + * Used to instantiate the {@link #getStagingServiceStub}. By default, instantiates using {@link + * ManagedChannelFactory#forDescriptor(Endpoints.ApiServiceDescriptor)}, where {@link + * Endpoints.ApiServiceDescriptor} is instantiated via {@link + * Endpoints.ApiServiceDescriptor.Builder#setUrl(String)} and the URL provided by {@link + * #getStagingEndpoint}. + */ + abstract ManagedChannel getManagedChannel(); + + /** + * Required by {@link ArtifactStagingService#offer}. By default, instantiates using {@link + * ArtifactStagingServiceGrpc#newStub} and {@link #getManagedChannel}. + */ + abstract ArtifactStagingServiceGrpc.ArtifactStagingServiceStub getStagingServiceStub(); + + @Override + public void close() { + LOG.info("shutting down {}", PrismArtifactStager.class); + getRetrievalService().close(); + getManagedChannel().shutdown(); + try { + getManagedChannel().awaitTermination(3000L, TimeUnit.MILLISECONDS); + } catch (InterruptedException ignored) { + } + } + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setStagingEndpoint(String stagingEndpoint); + + abstract Optional getStagingEndpoint(); + + abstract Builder setStagingSessionToken(String stagingSessionToken); + + abstract Builder setRetrievalService(ArtifactRetrievalService retrievalService); + + abstract Optional getRetrievalService(); + + abstract Builder setManagedChannel(ManagedChannel managedChannel); + + abstract Optional getManagedChannel(); + + abstract Builder setStagingServiceStub( + ArtifactStagingServiceGrpc.ArtifactStagingServiceStub stub); + + abstract Optional + getStagingServiceStub(); + + abstract PrismArtifactStager autoBuild(); + + final PrismArtifactStager build() { + + checkState(getStagingEndpoint().isPresent(), "missing staging endpoint"); + ManagedChannelFactory channelFactory = ManagedChannelFactory.createDefault(); + + if (!getManagedChannel().isPresent()) { + Endpoints.ApiServiceDescriptor descriptor = + Endpoints.ApiServiceDescriptor.newBuilder().setUrl(getStagingEndpoint().get()).build(); + setManagedChannel(channelFactory.forDescriptor(descriptor)); + } + + if (!getStagingServiceStub().isPresent()) { + setStagingServiceStub(ArtifactStagingServiceGrpc.newStub(getManagedChannel().get())); + } + + if (!getRetrievalService().isPresent()) { + setRetrievalService(new ArtifactRetrievalService()); + } + + return autoBuild(); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java new file mode 100644 index 0000000000000..d3ac8a72eafb9 --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService.EMBEDDED_ARTIFACT_URN; +import static org.junit.Assert.assertThrows; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; +import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.commons.io.output.ByteArrayOutputStream; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PrismArtifactStager}. */ +@RunWith(JUnit4.class) +public class PrismArtifactStagerTest { + + @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + + final ArtifactStagingService stagingService = + new ArtifactStagingService(new TestDestinationProvider()); + + @Test + public void givenValidArtifacts_stages() + throws IOException, ExecutionException, InterruptedException { + PrismArtifactStager underTest = prismArtifactStager(validArtifacts()); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + underTest.stage(); + assertThat(stagingService.getStagedArtifacts(underTest.getStagingSessionToken())).isNotEmpty(); + underTest.close(); + assertThat(underTest.getManagedChannel().isShutdown()).isTrue(); + } + + @Test + public void givenErrors_performsGracefulCleanup() throws IOException { + PrismArtifactStager underTest = prismArtifactStager(invalidArtifacts()); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + ExecutionException error = assertThrows(ExecutionException.class, underTest::stage); + assertThat(error.getMessage()).contains("Unexpected artifact type: invalid-type-urn"); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + underTest.close(); + assertThat(underTest.getManagedChannel().isShutdown()).isTrue(); + } + + private PrismArtifactStager prismArtifactStager( + Map> artifacts) throws IOException { + String serverName = InProcessServerBuilder.generateName(); + ArtifactRetrievalService retrievalService = new ArtifactRetrievalService(); + String stagingToken = "staging-token"; + stagingService.registerJob(stagingToken, artifacts); + + grpcCleanup.register( + InProcessServerBuilder.forName(serverName) + .directExecutor() + .addService(stagingService) + .addService(retrievalService) + .build() + .start()); + + ManagedChannel channel = + grpcCleanup.register(InProcessChannelBuilder.forName(serverName).build()); + + return PrismArtifactStager.builder() + .setStagingEndpoint("ignore") + .setStagingSessionToken(stagingToken) + .setManagedChannel(channel) + .build(); + } + + private Map> validArtifacts() { + return ImmutableMap.of( + "env1", + Collections.singletonList( + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(EMBEDDED_ARTIFACT_URN) + .setTypePayload( + RunnerApi.EmbeddedFilePayload.newBuilder() + .setData(ByteString.copyFromUtf8("type-payload")) + .build() + .toByteString()) + .setRoleUrn("role-urn") + .build())); + } + + private Map> invalidArtifacts() { + return ImmutableMap.of( + "env1", + Collections.singletonList( + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn("invalid-type-urn") + .setTypePayload( + RunnerApi.EmbeddedFilePayload.newBuilder() + .setData(ByteString.copyFromUtf8("type-payload")) + .build() + .toByteString()) + .setRoleUrn("role-urn") + .build())); + } + + private static class TestDestinationProvider + implements ArtifactStagingService.ArtifactDestinationProvider { + + @Override + public ArtifactStagingService.ArtifactDestination getDestination( + String stagingToken, String name) throws IOException { + return ArtifactStagingService.ArtifactDestination.create( + EMBEDDED_ARTIFACT_URN, ByteString.EMPTY, new ByteArrayOutputStream()); + } + + @Override + public void removeStagedArtifacts(String stagingToken) throws IOException {} + } +} From 741facf00993f24cca9418078ac62ff53c28e04e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 11:18:56 -0700 Subject: [PATCH 17/78] Bump github.com/docker/docker in /sdks (#32046) Bumps [github.com/docker/docker](https://github.com/docker/docker) from 25.0.5+incompatible to 25.0.6+incompatible. - [Release notes](https://github.com/docker/docker/releases) - [Commits](https://github.com/docker/docker/compare/v25.0.5...v25.0.6) --- updated-dependencies: - dependency-name: github.com/docker/docker dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6d42e02296c78..1716a6e2d22dc 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -137,7 +137,7 @@ require ( github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect github.com/containerd/containerd v1.7.11 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/docker/docker v25.0.5+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v25.0.6+incompatible // but required to resolve issue docker has with go1.20 github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 098f858488b72..a0b4738decc3d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -768,8 +768,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v25.0.5+incompatible h1:UmQydMduGkrD5nQde1mecF/YnSbTOaPeFIeP5C4W+DE= -github.com/docker/docker v25.0.5+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v25.0.6+incompatible h1:5cPwbwriIcsua2REJe8HqQV+6WlWc1byg2QSXzBxBGg= +github.com/docker/docker v25.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= From 5ab908b984d4144b5cbe584d7ed4ed7a4e226993 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 15:03:57 -0400 Subject: [PATCH 18/78] Add Lineage metrics for BigtableIO (#32068) * Add Lineage metrics for BigtableIO * add tests * simplify metrics query logics; exclude test actually already failing * Address comments, fix typo --- .../org/apache/beam/sdk/metrics/Lineage.java | 43 ++++++++++++++++--- .../io/google-cloud-platform/build.gradle | 4 ++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 14 ++++++ .../sdk/io/gcp/bigtable/BigtableService.java | 6 +++ .../io/gcp/bigtable/BigtableServiceImpl.java | 22 ++++++++++ .../io/gcp/bigquery/BigQueryIOReadTest.java | 23 ++-------- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 13 +----- .../sdk/io/gcp/bigtable/BigtableReadIT.java | 21 ++++++++- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 18 +++++++- 9 files changed, 123 insertions(+), 41 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java index 7890a9f74b941..8b69b0ef55236 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java @@ -17,17 +17,17 @@ */ package org.apache.beam.sdk.metrics; +import java.util.HashSet; +import java.util.Set; + /** * Standard collection of metrics used to record source and sinks information for lineage tracking. */ public class Lineage { - public static final String LINEAGE_NAMESPACE = "lineage"; - public static final String SOURCE_METRIC_NAME = "sources"; - public static final String SINK_METRIC_NAME = "sinks"; - - private static final StringSet SOURCES = Metrics.stringSet(LINEAGE_NAMESPACE, SOURCE_METRIC_NAME); - private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, SINK_METRIC_NAME); + private static final StringSet SOURCES = + Metrics.stringSet(LINEAGE_NAMESPACE, Type.SOURCE.toString()); + private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, Type.SINK.toString()); /** {@link StringSet} representing sources and optionally side inputs. */ public static StringSet getSources() { @@ -38,4 +38,35 @@ public static StringSet getSources() { public static StringSet getSinks() { return SINKS; } + + /** Query {@link StringSet} metrics from {@link MetricResults}. */ + public static Set query(MetricResults results, Type type) { + MetricsFilter filter = + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(LINEAGE_NAMESPACE, type.toString())) + .build(); + Set result = new HashSet<>(); + for (MetricResult metrics : results.queryMetrics(filter).getStringSets()) { + result.addAll(metrics.getCommitted().getStringSet()); + result.addAll(metrics.getAttempted().getStringSet()); + } + return result; + } + + /** Lineage metrics resource types. */ + public enum Type { + SOURCE("source"), + SINK("sink"); + + private final String name; + + Type(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + } } diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index e499bae6fc64f..23c56f13a94c7 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -218,6 +218,10 @@ task integrationTest(type: Test, dependsOn: processTestResources) { useJUnit { excludeCategories "org.apache.beam.sdk.testing.UsesKms" + filter { + // https://github.com/apache/beam/issues/32071 + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableReadIT.testE2EBigtableSegmentRead' + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index d78ae2cb6c578..6d20109e947ba 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1337,6 +1337,7 @@ private static class BigtableWriterFn private transient Set> badRecords = null; // Due to callback thread not supporting Beam metrics, Record pending metrics and report later. private transient long pendingThrottlingMsecs; + private transient boolean reportedLineage; // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1480,6 +1481,10 @@ public void finishBundle(FinishBundleContext c) throws Exception { throttlingMsecs.inc(excessTime); } } + if (!reportedLineage) { + bigtableWriter.reportLineage(); + reportedLineage = true; + } bigtableWriter = null; } @@ -1612,6 +1617,7 @@ public String toString() { private final BigtableConfig config; private final BigtableReadOptions readOptions; private @Nullable Long estimatedSizeBytes; + private transient boolean reportedLineage; private final BigtableServiceFactory.ConfigId configId; @@ -1989,6 +1995,13 @@ public List getRanges() { public ValueProvider getTableId() { return readOptions.getTableId(); } + + void reportLineageOnce(BigtableService.Reader reader) { + if (!reportedLineage) { + reader.reportLineage(); + reportedLineage = true; + } + } } private static class BigtableReader extends BoundedReader { @@ -2019,6 +2032,7 @@ true, makeByteKey(reader.getCurrentRow().getKey()))) || rangeTracker.markDone(); if (hasRecord) { ++recordsReturned; + source.reportLineageOnce(reader); } return hasRecord; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 261cc3ac081d8..50d8126999c4b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -57,6 +57,9 @@ CompletionStage writeRecord(KV * @throws IOException if there is an error closing the writer */ void close() throws IOException; + + /** Report Lineage metrics to runner. */ + default void reportLineage() {} } /** The interface of a class that reads from Cloud Bigtable. */ @@ -77,6 +80,9 @@ interface Reader { Row getCurrentRow() throws NoSuchElementException; void close(); + + /** Report Lineage metrics to runner. */ + default void reportLineage() {} } /** Returns a {@link Reader} that will read from the specified source. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index f06a4a1276864..6fdf67722bac2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -71,6 +71,7 @@ import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -212,6 +213,11 @@ public void close() { exhausted = true; } } + + @Override + public void reportLineage() { + Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } } @VisibleForTesting @@ -225,6 +231,9 @@ static class BigtableSegmentReaderImpl implements Reader { private final int refillSegmentWaterMark; private final long maxSegmentByteSize; private ServiceCallMetric serviceCallMetric; + private final String projectId; + private final String instanceId; + private final String tableId; private static class UpstreamResults { private final List rows; @@ -308,11 +317,19 @@ static BigtableSegmentReaderImpl create( // Asynchronously refill buffer when there is 10% of the elements are left this.refillSegmentWaterMark = Math.max(1, (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE)); + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; } @Override public void close() {} + @Override + public void reportLineage() { + Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } + @Override public boolean start() throws IOException { future = fetchNextSegment(); @@ -578,6 +595,11 @@ public void writeSingleRecord(KV> record) throws } } + @Override + public void reportLineage() { + Lineage.getSinks().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } + private ServiceCallMetric createServiceCallMetric() { // Populate metrics HashMap baseLabels = new HashMap<>(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java index 5c43666e79e5c..a8aca7570b33d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificRecordBase; @@ -61,9 +62,6 @@ import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; import org.apache.beam.sdk.io.gcp.testing.FakeJobService; import org.apache.beam.sdk.metrics.Lineage; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; @@ -351,18 +349,8 @@ private void checkTypedReadQueryObjectWithValidate( } private void checkLineageSourceMetric(PipelineResult pipelineResult, String tableName) { - MetricQueryResults lineageMetrics = - pipelineResult - .metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter( - MetricNameFilter.named( - Lineage.LINEAGE_NAMESPACE, Lineage.SOURCE_METRIC_NAME)) - .build()); - assertThat( - lineageMetrics.getStringSets().iterator().next().getCommitted().getStringSet(), - contains("bigquery:" + tableName.replace(':', '.'))); + Set result = Lineage.query(pipelineResult.metrics(), Lineage.Type.SOURCE); + assertThat(result, contains("bigquery:" + tableName.replace(':', '.'))); } @Before @@ -600,10 +588,7 @@ public void processElement(ProcessContext c) throws Exception { new MyData("b", 2L, bd1, bd2), new MyData("c", 3L, bd1, bd2))); PipelineResult result = p.run(); - // Skip when direct runner splits outside of a counters context. - if (useTemplateCompatibility) { - checkLineageSourceMetric(result, "non-executing-project:somedataset.sometable"); - } + checkLineageSourceMetric(result, "non-executing-project:somedataset.sometable"); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index bc90d4c8bae79..c5af8045bfe20 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -118,9 +118,6 @@ import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; import org.apache.beam.sdk.io.gcp.testing.FakeJobService; import org.apache.beam.sdk.metrics.Lineage; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.JavaFieldSchema; @@ -285,16 +282,8 @@ public void evaluate() throws Throwable { .withJobService(fakeJobService); private void checkLineageSinkMetric(PipelineResult pipelineResult, String tableName) { - MetricQueryResults lineageMetrics = - pipelineResult - .metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter( - MetricNameFilter.named(Lineage.LINEAGE_NAMESPACE, Lineage.SINK_METRIC_NAME)) - .build()); assertThat( - lineageMetrics.getStringSets().iterator().next().getCommitted().getStringSet(), + Lineage.query(pipelineResult.metrics(), Lineage.Type.SINK), hasItem("bigquery:" + tableName.replace(':', '.'))); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java index bc88858ebc33e..4ce9ad10b2c06 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; + import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; @@ -28,7 +31,9 @@ import java.util.Date; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -110,7 +115,8 @@ public void testE2EBigtableRead() { p.apply(BigtableIO.read().withBigtableOptions(bigtableOptionsBuilder).withTableId(tableId)) .apply(Count.globally()); PAssert.thatSingleton(count).isEqualTo(numRows); - p.run(); + PipelineResult r = p.run(); + checkLineageSourceMetric(r, tableId); } @Test @@ -138,6 +144,17 @@ public void testE2EBigtableSegmentRead() { .withMaxBufferElementCount(10)) .apply(Count.globally()); PAssert.thatSingleton(count).isEqualTo(numRows); - p.run(); + PipelineResult r = p.run(); + checkLineageSourceMetric(r, tableId); + } + + private void checkLineageSourceMetric(PipelineResult r, String tableId) { + // TODO(https://github.com/apache/beam/issues/32071) test malformed, + // when pipeline.run() is non-blocking, the metrics are not available by the time of query + if (options.getRunner().getName().contains("DirectRunner")) { + assertThat( + Lineage.query(r.metrics(), Lineage.Type.SOURCE), + hasItem(String.format("bigtable:%s.%s.%s", project, options.getInstanceId(), tableId))); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index bf9f7d991fa24..46bb3df836e56 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigtable; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import com.google.api.gax.rpc.ServerStream; @@ -39,8 +40,10 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.PAssert; @@ -142,7 +145,7 @@ public void processElement(ProcessContext c) { .withProjectId(project) .withInstanceId(options.getInstanceId()) .withTableId(tableId)); - p.run(); + PipelineResult r = p.run(); // Test number of column families and column family name equality Table table = getTable(tableId); @@ -154,6 +157,7 @@ public void processElement(ProcessContext c) { // Test table data equality List> tableData = getTableData(tableId); assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray())); + checkLineageSinkMetric(r, tableId); } @Test @@ -340,7 +344,7 @@ public void failureTest(int numRows, DoFn> tableData = getTableData(tableId); assertEquals(998, tableData.size()); + checkLineageSinkMetric(r, tableId); } @After @@ -412,4 +417,13 @@ private void deleteTable(String tableId) { tableAdminClient.deleteTable(tableId); } } + + private void checkLineageSinkMetric(PipelineResult r, String tableId) { + // Only check lineage metrics on direct runner until Dataflow runner v2 supported report back + if (options.getRunner().getName().contains("DirectRunner")) { + assertThat( + Lineage.query(r.metrics(), Lineage.Type.SINK), + hasItem(String.format("bigtable:%s.%s.%s", project, options.getInstanceId(), tableId))); + } + } } From 17283bb8294f22edfc4d00c49bf3d9a518a1551b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 15:35:16 -0400 Subject: [PATCH 19/78] Add Lineage metrics to PubsubIO (#32037) * Add Lineage metrics to PubsubIO * fix format and add test * make getDatacatalogname fail safe --- .../beam/sdk/io/gcp/pubsub/PubsubClient.java | 28 +++++++++++++++++++ .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 9 ++++++ .../io/gcp/pubsub/PubsubUnboundedSink.java | 13 +++++++++ .../io/gcp/pubsub/PubsubUnboundedSource.java | 14 ++++++++++ .../sdk/io/gcp/pubsub/PubsubClientTest.java | 2 ++ .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 3 ++ 6 files changed, 69 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java index 79a9bb7f07d64..f66ee6e1d8425 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java @@ -39,12 +39,15 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** An (abstract) helper class for talking to Pubsub via an underlying transport. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public abstract class PubsubClient implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(PubsubClient.class); private static final Map> schemaTypeToConversionFnMap = ImmutableMap.of( @@ -257,6 +260,10 @@ public String getFullPath() { return String.format("/subscriptions/%s/%s", projectId, subscriptionName); } + public String getDataCatalogName() { + return String.format("pubsub:subscription:%s.%s", projectId, subscriptionName); + } + @Override public boolean equals(@Nullable Object o) { if (this == o) { @@ -293,6 +300,7 @@ public static SubscriptionPath subscriptionPathFromName( /** Path representing a Pubsub topic. */ public static class TopicPath implements Serializable { + // Format: "projects//topics/" private final String path; TopicPath(String path) { @@ -310,6 +318,26 @@ public String getName() { return splits.get(3); } + /** + * Returns the data catalog name. Format "pubsub:topic:`project`.`topic`" This method is + * fail-safe. If topic path is malformed, it returns an empty string. + */ + public String getDataCatalogName() { + List splits = Splitter.on('/').splitToList(path); + if (splits.size() == 4) { + // well-formed path + return String.format("pubsub:topic:%s.%s", splits.get(1), splits.get(3)); + } else { + // Mal-formed path. It is either a test fixture or user error and will fail on publish. + // We do not throw exception instead return empty string here. + LOG.warn( + "Cannot get data catalog name for malformed topic path {}. Expected format: " + + "projects//topics/", + path); + return ""; + } + } + public String getFullPath() { List splits = Splitter.on('/').splitToList(path); checkState(splits.size() == 4, "Malformed topic path %s", path); 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 6233cf6690801..0fd4e9207d81a 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 @@ -49,6 +49,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; @@ -512,6 +513,10 @@ public String asPath() { } } + public String dataCatalogName() { + return String.format("pubsub:topic:%s.%s", project, topic); + } + @Override public String toString() { return asPath(); @@ -1617,6 +1622,10 @@ public void finishBundle() throws IOException { for (Map.Entry entry : output.entrySet()) { publish(entry.getKey(), entry.getValue().messages); } + // Report lineage for all topics seen + for (PubsubTopic topic : output.keySet()) { + Lineage.getSinks().add(topic.dataCatalogName()); + } output = null; pubsubClient.close(); pubsubClient = null; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index aa8e3a4114868..defea87e835a8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.PubsubClientFactory; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.metrics.SinkMetrics; import org.apache.beam.sdk.options.ValueProvider; @@ -69,6 +70,7 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -231,6 +233,9 @@ private static class WriterFn extends DoFn, Void> { /** Client on which to talk to Pubsub. Null until created by {@link #startBundle}. */ private transient @Nullable PubsubClient pubsubClient; + /** Last TopicPath that reported Lineage. */ + private transient @Nullable TopicPath reportedLineage; + private final Counter batchCounter = Metrics.counter(WriterFn.class, "batches"); private final Counter elementCounter = SinkMetrics.elementsWritten(); private final Counter byteCounter = SinkMetrics.bytesWritten(); @@ -290,6 +295,14 @@ private void publishBatch(List messages, int bytes) throws IOEx batchCounter.inc(); elementCounter.inc(messages.size()); byteCounter.inc(bytes); + // Report Lineage multiple once for same topic + if (!topicPath.equals(reportedLineage)) { + String name = topicPath.getDataCatalogName(); + if (!Strings.isNullOrEmpty(name)) { + Lineage.getSinks().add(topicPath.getDataCatalogName()); + } + reportedLineage = topicPath; + } } @StartBundle diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index b9a554d54ade7..b131b521c067e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -56,6 +56,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessages.DeserializeBytesIntoPubsubMessagePayloadOnly; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.SourceMetrics; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -1041,6 +1042,19 @@ public List split(int desiredNumSplits, PipelineOptions options) splitSource = new PubsubSource( outer, StaticValueProvider.of(outer.createRandomSubscription(options))); + TopicPath topic = outer.getTopic(); + if (topic != null) { + // is initial split on Read.fromTopic, report Lineage based on topic + Lineage.getSources().add(topic.getDataCatalogName()); + } + } else { + if (subscriptionPath.equals(outer.getSubscriptionProvider())) { + SubscriptionPath sub = subscriptionPath.get(); + if (sub != null) { + // is a split on Read.fromSubscription + Lineage.getSources().add(sub.getDataCatalogName()); + } + } } for (int i = 0; i < desiredNumSplits * SCALE_OUT; i++) { // Since the source is immutable and Pubsub automatically shards we simply diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java index 895ed35bfb120..fb007d1171db1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java @@ -171,6 +171,7 @@ public void subscriptionPathFromNameWellFormed() { SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something"); assertEquals("projects/test/subscriptions/something", path.getPath()); assertEquals("/subscriptions/test/something", path.getFullPath()); + assertEquals("pubsub:subscription:test.something", path.getDataCatalogName()); } @Test @@ -178,6 +179,7 @@ public void topicPathFromNameWellFormed() { TopicPath path = PubsubClient.topicPathFromName("test", "something"); assertEquals("projects/test/topics/something", path.getPath()); assertEquals("/topics/test/something", path.getFullPath()); + assertEquals("pubsub:topic:test.something", path.getDataCatalogName()); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index 3027db6aee9d5..74a98f0b8b438 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -237,6 +237,9 @@ public void testValueProviderTopic() { assertThat(pubsubRead.getTopicProvider(), not(nullValue())); assertThat(pubsubRead.getTopicProvider().isAccessible(), is(true)); assertThat(pubsubRead.getTopicProvider().get().asPath(), equalTo(provider.get())); + assertThat( + pubsubRead.getTopicProvider().get().dataCatalogName(), + equalTo("pubsub:topic:project.topic")); } @Test From e3e4454457762c85ca7c8068f0e9f2e20966dccc Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 6 Aug 2024 12:54:31 -0700 Subject: [PATCH 20/78] [#32085][prism] Fix session windowing. (#32086) --- .../runners/prism/internal/handlerunner.go | 24 ++++++++++++------- .../runners/portability/prism_runner_test.py | 22 +++++++++++++++++ 2 files changed, 37 insertions(+), 9 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go index a1eeeba02c4bb..eecebde3d693f 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go @@ -244,7 +244,7 @@ func (h *runner) ExecuteTransform(stageID, tid string, t *pipepb.PTransform, com kc := coders[kcID] ec := coders[ecID] - data = append(data, gbkBytes(ws, wc, kc, ec, inputData, coders, watermark)) + data = append(data, gbkBytes(ws, wc, kc, ec, inputData, coders)) if len(data[0]) == 0 { panic("no data for GBK") } @@ -290,7 +290,7 @@ func windowingStrategy(comps *pipepb.Components, tid string) *pipepb.WindowingSt } // gbkBytes re-encodes gbk inputs in a gbk result. -func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregate [][]byte, coders map[string]*pipepb.Coder, watermark mtime.Time) []byte { +func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregate [][]byte, coders map[string]*pipepb.Coder) []byte { // Pick how the timestamp of the aggregated output is computed. var outputTime func(typex.Window, mtime.Time, mtime.Time) mtime.Time switch ws.GetOutputTime() { @@ -333,9 +333,8 @@ func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregat kd := pullDecoder(kc, coders) vd := pullDecoder(vc, coders) - // Right, need to get the key coder, and the element coder. - // Cus I'll need to pull out anything the runner knows how to deal with. - // And repeat. + // Aggregate by windows and keys, using the window coder and KV coders. + // We need to extract and split the key bytes from the element bytes. for _, data := range toAggregate { // Parse out each element's data, and repeat. buf := bytes.NewBuffer(data) @@ -388,34 +387,41 @@ func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregat } // Use a decreasing sort (latest to earliest) so we can correct // the output timestamp to the new end of window immeadiately. - // TODO need to correct this if output time is different. sort.Slice(ordered, func(i, j int) bool { return ordered[i].MaxTimestamp() > ordered[j].MaxTimestamp() }) cur := ordered[0] sessionData := windows[cur] + delete(windows, cur) for _, iw := range ordered[1:] { - // If they overlap, then we merge the data. + // Check if the gap between windows is less than the gapSize. + // If not, this window is done, and we start a next window. if iw.End+gapSize < cur.Start { - // Start a new session. + // Store current data with the current window. windows[cur] = sessionData + // Use the incoming window instead, and clear it from the map. cur = iw sessionData = windows[iw] + delete(windows, cur) + // There's nothing to merge, since we've just started with this windowed data. continue } - // Extend the session + // Extend the session with the incoming window, and merge the the incoming window's data. cur.Start = iw.Start toMerge := windows[iw] delete(windows, iw) for k, kt := range toMerge { skt := sessionData[k] + // Ensure the output time matches the given function. + skt.time = outputTime(cur, kt.time, skt.time) skt.key = kt.key skt.w = cur skt.values = append(skt.values, kt.values...) sessionData[k] = skt } } + windows[cur] = sessionData } // Everything's aggregated! // Time to turn things into a windowed KV> diff --git a/sdks/python/apache_beam/runners/portability/prism_runner_test.py b/sdks/python/apache_beam/runners/portability/prism_runner_test.py index f1ccf66a22894..324fe5a17b545 100644 --- a/sdks/python/apache_beam/runners/portability/prism_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/prism_runner_test.py @@ -40,7 +40,9 @@ from apache_beam.runners.portability import portable_runner_test from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms import window from apache_beam.transforms.sql import SqlTransform +from apache_beam.utils import timestamp # Run as # @@ -178,6 +180,26 @@ def create_options(self): return options + # Slightly more robust session window test: + # Validates that an inner grouping doesn't duplicate data either. + # Copied also because the timestamp in fn_runner_test.py isn't being + # inferred correctly as seconds for some reason, but as micros. + # The belabored specification is validating the timestamp type works at least. + # See https://github.com/apache/beam/issues/32085 + def test_windowing(self): + with self.create_pipeline() as p: + res = ( + p + | beam.Create([1, 2, 100, 101, 102, 123]) + | beam.Map( + lambda t: window.TimestampedValue( + ('k', t), timestamp.Timestamp.of(t).micros)) + | beam.WindowInto(beam.transforms.window.Sessions(10)) + | beam.GroupByKey() + | beam.Map(lambda k_vs1: (k_vs1[0], sorted(k_vs1[1])))) + assert_that( + res, equal_to([('k', [1, 2]), ('k', [100, 101, 102]), ('k', [123])])) + # Can't read host files from within docker, read a "local" file there. def test_read(self): print('name:', __name__) From 0a42afa9f5c02e9d529e1c1f1b197472a44cc174 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 6 Aug 2024 16:10:08 -0700 Subject: [PATCH 21/78] [prism] Use non-deprecated docker types in environment. (#32092) * [prism] Use non-deprecated docker types in environment. * Include Go mod changes. * Update testcontainers-go * revert toolchain change * go mod tidy requirement, update minimum Go version. * Note Minimum Go version update. --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- CHANGES.md | 1 + sdks/go.mod | 20 +++--- sdks/go.sum | 70 ++++++++++++++----- .../runners/prism/internal/environments.go | 8 +-- 4 files changed, 69 insertions(+), 30 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7f12b53342602..129fa01f94a84 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Go SDK Minimum Go Version updated to 1.21 ([#32092](https://github.com/apache/beam/pull/32092)). ## Breaking Changes diff --git a/sdks/go.mod b/sdks/go.mod index 1716a6e2d22dc..7eb29b3cc77c3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -20,7 +20,7 @@ // directory. module github.com/apache/beam/sdks/v2 -go 1.20 +go 1.21 require ( cloud.google.com/go/bigquery v1.61.0 @@ -49,7 +49,7 @@ require ( github.com/nats-io/nats.go v1.35.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.1 - github.com/testcontainers/testcontainers-go v0.26.0 + github.com/testcontainers/testcontainers-go v0.32.0 github.com/tetratelabs/wazero v1.7.3 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c @@ -79,21 +79,23 @@ require ( dario.cat/mergo v1.0.0 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect - github.com/Microsoft/hcsshim v0.11.4 // indirect + github.com/Microsoft/hcsshim v0.11.5 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect + github.com/containerd/errdefs v0.1.0 // indirect github.com/containerd/log v0.1.0 // indirect - github.com/distribution/reference v0.5.0 // indirect + github.com/distribution/reference v0.6.0 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect + github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/user v0.1.0 // indirect github.com/nats-io/jwt/v2 v2.5.7 // indirect github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect - github.com/shirou/gopsutil/v3 v3.23.9 // indirect + github.com/shirou/gopsutil/v3 v3.23.12 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect @@ -114,7 +116,7 @@ require ( cloud.google.com/go/iam v1.1.8 // indirect cloud.google.com/go/longrunning v0.5.7 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect - github.com/Microsoft/go-winio v0.6.1 // indirect + github.com/Microsoft/go-winio v0.6.2 // indirect github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 // indirect github.com/apache/thrift v0.17.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect @@ -135,9 +137,9 @@ require ( github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect - github.com/containerd/containerd v1.7.11 // indirect + github.com/containerd/containerd v1.7.18 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/docker/docker v25.0.6+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v27.1.1+incompatible // but required to resolve issue docker has with go1.20 github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect @@ -165,7 +167,7 @@ require ( github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe // indirect github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect - github.com/opencontainers/image-spec v1.1.0-rc5 // indirect + github.com/opencontainers/image-spec v1.1.0 // indirect github.com/pierrec/lz4/v4 v4.1.18 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pkg/xattr v0.4.9 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index a0b4738decc3d..ce10d84dd044e 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -209,6 +209,7 @@ cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnR cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= cloud.google.com/go/datacatalog v1.20.1 h1:czcba5mxwRM5V//jSadyig0y+8aOHmN7gUl9GbHu59E= +cloud.google.com/go/datacatalog v1.20.1/go.mod h1:Jzc2CoHudhuZhpv78UBAjMEg3w7I9jHA11SbRshWUjk= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -346,6 +347,7 @@ cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8 cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= cloud.google.com/go/kms v1.18.0 h1:pqNdaVmZJFP+i8OVLocjfpdTWETTYa20FWOegSCdrRo= +cloud.google.com/go/kms v1.18.0/go.mod h1:DyRBeWD/pYBMeyiaXFa/DGNyxMDL3TslIKb8o/JkLkw= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -628,6 +630,7 @@ filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4 gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8= git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 h1:bvDV9vkmnHYOMsOr4WLk+Vo07yKIzd94sVoIqshQ4bU= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= github.com/Azure/azure-pipeline-go v0.2.3/go.mod h1:x841ezTBIMG6O3lAcl8ATHnsOPVl2bqk7S3ta6S6u4k= github.com/Azure/azure-storage-blob-go v0.14.0/go.mod h1:SMqIBi+SuiQH32bvyjngEewEeXoPfKMgWlBDaYf6fck= github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= @@ -643,10 +646,10 @@ github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 h1:oVLqHXhnYtUwM89y9T1fXGaK9wTkXHgNp8/ZNMQzUxE= github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0/go.mod h1:dppbR7CwXD4pgtV9t3wD1812RaLDcBjtblcDF5f1vI0= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= -github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= -github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Microsoft/hcsshim v0.11.4 h1:68vKo2VN8DE9AdN4tnkWnmdhqdbpUFM8OF3Airm7fz8= -github.com/Microsoft/hcsshim v0.11.4/go.mod h1:smjE4dvqPX9Zldna+t5FG3rnoHhaB7QYxPRqGcpAD9w= +github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= +github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU= +github.com/Microsoft/hcsshim v0.11.5 h1:haEcLNpj9Ka1gd3B3tAEs9CpE0c+1IhoL59w/exYU38= +github.com/Microsoft/hcsshim v0.11.5/go.mod h1:MV8xMfmECjl5HdO7U/3/hFVnkmSBjAjmA09d4bExKcU= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -754,8 +757,10 @@ github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 h1:DBmgJDC9dTfkVyGgipamEh2BpGYxScCH1TOF1LL1cXc= github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50/go.mod h1:5e1+Vvlzido69INQaVO6d87Qn543Xr6nooe9Kz7oBFM= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= -github.com/containerd/containerd v1.7.11 h1:lfGKw3eU35sjV0aG2eYZTiwFEY1pCzxdzicHP3SZILw= -github.com/containerd/containerd v1.7.11/go.mod h1:5UluHxHTX2rdvYuZ5OJTC5m/KJNs0Zs9wVoJm9zf5ZE= +github.com/containerd/containerd v1.7.18 h1:jqjZTQNfXGoEaZdW1WwPU0RqSn1Bm2Ay/KJPUuO8nao= +github.com/containerd/containerd v1.7.18/go.mod h1:IYEk9/IO6wAPUz2bCMVUbsfXjzw5UNP5fLz4PsUygQ4= +github.com/containerd/errdefs v0.1.0 h1:m0wCRBiu1WJT/Fr+iOoQHMQS/eP5myQ8lCv4Dz5ZURM= +github.com/containerd/errdefs v0.1.0/go.mod h1:YgWiiHtLmSeBrvpw+UfPijzbLaB77mEG1WwJTDETIV0= github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= @@ -763,13 +768,14 @@ github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHf github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= +github.com/creack/pty v1.1.18/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= -github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v25.0.6+incompatible h1:5cPwbwriIcsua2REJe8HqQV+6WlWc1byg2QSXzBxBGg= -github.com/docker/docker v25.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5QvfrDyIgxBk= +github.com/distribution/reference v0.6.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= +github.com/docker/docker v27.1.1+incompatible h1:hO/M4MtV36kzKldqnA37IWhebRA+LnqqcqDja6kVaKY= +github.com/docker/docker v27.1.1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -814,6 +820,7 @@ github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gorp/gorp v2.2.0+incompatible h1:xAUh4QgEeqPPhK3vxZN+bzrim1z5Av6q837gtjUlshc= +github.com/go-gorp/gorp v2.2.0+incompatible/go.mod h1:7IfkAQnO7jfT/9IQ3R9wL1dFhukN6aQxzKTHnkxzA/E= github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -881,6 +888,7 @@ github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= +github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/flatbuffers v2.0.8+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/flatbuffers v23.5.26+incompatible h1:M9dgRyhJemaM4Sw8+66GHBu8ioaQmyPLg1b8VwK5WJg= @@ -909,6 +917,7 @@ github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIG github.com/google/martian/v3 v3.2.1/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/martian/v3 v3.3.3 h1:DIhPTQrbPkgs2yJYdXU/eNACCG5DVQjySNRNlflZ9Fc= +github.com/google/martian/v3 v3.3.3/go.mod h1:iEPrYcgCF7jA9OtScMFQyAlZZ4YXTKEtJ1E6RWzmBA0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -967,6 +976,7 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFb github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= @@ -976,11 +986,16 @@ github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1: github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM= +github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a h1:bbPeKD0xmW/Y25WS6cokEszi5g+S0QxI/d45PkRi7Nk= +github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= github.com/jackc/pgx/v5 v5.3.1 h1:Fcr8QJ1ZeLi5zsPZqQeUZhNhxfkkKBOgJuYkJHoBOtU= +github.com/jackc/pgx/v5 v5.3.1/go.mod h1:t3JDKnCBlYIc0ewLF0Q7B8MXmoIaBOZj/ic7iHozM/8= github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jinzhu/inflection v1.0.0 h1:K317FqzuhWc8YvSVlFMCCUb36O/S9MCKRDI7QkRKD/E= +github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= +github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= @@ -990,6 +1005,7 @@ github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfC github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= @@ -1012,6 +1028,7 @@ github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORN github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -1036,8 +1053,13 @@ github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8Ie github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= +github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= +github.com/minio/minio-go/v7 v7.0.66/go.mod h1:DHAgmyQEGdW3Cif0UooKOyrT3Vxs82zNdV6tkKhRtbs= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= +github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= +github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= +github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5lXtc= @@ -1047,7 +1069,9 @@ github.com/moby/sys/user v0.1.0/go.mod h1:fKJhFOnsCN6xZ5gSfbM6zaHGgDJMrqt9/reuj4 github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6fbMZ9s0scYfZQ84/6SPL6zC8ACM2oIL0= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= @@ -1065,8 +1089,8 @@ github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OS github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= -github.com/opencontainers/image-spec v1.1.0-rc5 h1:Ygwkfw9bpDvs+c9E34SdgGOj41dX/cbdlwvlWt0pnFI= -github.com/opencontainers/image-spec v1.1.0-rc5/go.mod h1:X4pATf0uXsnn3g5aiGIsVnJBR4mxhKzfwmvK/B2NTm8= +github.com/opencontainers/image-spec v1.1.0 h1:8SG7/vwALn54lVB/0yZ/MMwhFrPYtpEHQb2IpWsCzug= +github.com/opencontainers/image-spec v1.1.0/go.mod h1:W4s4sFTMaBeK1BQLXbG4AdM2szdn85PY75RI83NrTrM= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= @@ -1099,6 +1123,7 @@ github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTE github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc= +github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk= @@ -1106,8 +1131,8 @@ github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63 h1:J6qvD6rbmOil46orKqJaRPG+zTpoGlBTUdyv8ki63L0= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63/go.mod h1:n+VKSARF5y/tS9XFSP7vWDfS+GUC5vs/YT7M5XDTUEM= -github.com/shirou/gopsutil/v3 v3.23.9 h1:ZI5bWVeu2ep4/DIxB4U9okeYJ7zp/QLTO4auRb/ty/E= -github.com/shirou/gopsutil/v3 v3.23.9/go.mod h1:x/NWSb71eMcjFIO0vhyGW5nZ7oSIgVjrCnADckb85GA= +github.com/shirou/gopsutil/v3 v3.23.12 h1:z90NtUkp3bMtmICZKpC4+WaknU1eXtp5vtbQ11DgpE4= +github.com/shirou/gopsutil/v3 v3.23.12/go.mod h1:1FrWgea594Jp7qmjHUUPlJDTPgcsb9mGnXDxavtikzM= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -1141,8 +1166,9 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/testcontainers/testcontainers-go v0.26.0 h1:uqcYdoOHBy1ca7gKODfBd9uTHVK3a7UL848z09MVZ0c= -github.com/testcontainers/testcontainers-go v0.26.0/go.mod h1:ICriE9bLX5CLxL9OFQ2N+2N+f+803LNJ1utJb1+Inx0= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/testcontainers/testcontainers-go v0.32.0 h1:ug1aK08L3gCHdhknlTTwWjPHPS+/alvLJU/DRxTD/ME= +github.com/testcontainers/testcontainers-go v0.32.0/go.mod h1:CRHrzHLQhlXUsa5gXjTOfqIEJcrK5+xMDmBr/WMI88E= github.com/tetratelabs/wazero v1.7.3 h1:PBH5KVahrt3S2AHgEjKu4u+LlDbbk+nsGE3KLucy6Rw= github.com/tetratelabs/wazero v1.7.3/go.mod h1:ytl6Zuh20R/eROuyDaGPkp82O9C/DJfXAwJfQ3X6/7Y= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= @@ -1198,7 +1224,9 @@ go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1: go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0 h1:Mne5On7VWdx7omSrSSZvM4Kw7cS7NQkOOmLcgscI51U= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0/go.mod h1:IPtUMKL4O3tH5y+iXVyAXqpAwMuzC1IrxVS81rummfE= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0 h1:IeMeyr1aBvBiPVYihXIaeIZba6b8E1bYp7lbdxK8CQg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0/go.mod h1:oVdCUtjq9MK9BlS7TtucsQwUcXcymNiEDjgDD2jMtZU= go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI= go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= @@ -1209,6 +1237,7 @@ go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqe go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= +go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -1482,7 +1511,7 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -1493,6 +1522,8 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1600,6 +1631,7 @@ gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0= gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= +gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= gonum.org/v1/plot v0.9.0/go.mod h1:3Pcqqmp6RHvJI72kgb8fThyUnav364FOsdDo2aGW5lY= @@ -1877,6 +1909,7 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= @@ -1892,8 +1925,11 @@ gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gorm.io/driver/postgres v1.5.2 h1:ytTDxxEv+MplXOfFe3Lzm7SjG09fcdb3Z/c056DTBx0= +gorm.io/driver/postgres v1.5.2/go.mod h1:fmpX0m2I1PKuR7mKZiEluwrP3hbs+ps7JIGMUBpCgl8= gorm.io/gorm v1.25.2 h1:gs1o6Vsa+oVKG/a9ElL3XgyGfghFfkKA2SInQaCyMho= +gorm.io/gorm v1.25.2/go.mod h1:L4uxeKpfBml98NYqVqwAdmV1a2nBtAec/cf3fpucW/k= gotest.tools/v3 v3.5.1 h1:EENdUnS3pdur5nybKYIh2Vfgc8IUNBjxDPSjtiJcOzU= +gotest.tools/v3 v3.5.1/go.mod h1:isy3WKz7GK6uNw/sbHzfKBLvlvXwUyV06n6brMxxopU= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/sdks/go/pkg/beam/runners/prism/internal/environments.go b/sdks/go/pkg/beam/runners/prism/internal/environments.go index 3a429920fb289..add7f769a702e 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/environments.go +++ b/sdks/go/pkg/beam/runners/prism/internal/environments.go @@ -32,8 +32,8 @@ import ( "google.golang.org/grpc/credentials/insecure" "google.golang.org/protobuf/proto" - dtyp "github.com/docker/docker/api/types" "github.com/docker/docker/api/types/container" + "github.com/docker/docker/api/types/image" "github.com/docker/docker/api/types/mount" dcli "github.com/docker/docker/client" "github.com/docker/docker/pkg/stdcopy" @@ -132,7 +132,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock } if _, _, err := cli.ImageInspectWithRaw(ctx, dp.GetContainerImage()); err != nil { // We don't have a local image, so we should pull it. - if rc, err := cli.ImagePull(ctx, dp.GetContainerImage(), dtyp.ImagePullOptions{}); err == nil { + if rc, err := cli.ImagePull(ctx, dp.GetContainerImage(), image.PullOptions{}); err == nil { // Copy the output, but discard it so we can wait until the image pull is finished. io.Copy(io.Discard, rc) rc.Close() @@ -164,7 +164,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock containerID := ccr.ID logger = logger.With("container", containerID) - if err := cli.ContainerStart(ctx, containerID, dtyp.ContainerStartOptions{}); err != nil { + if err := cli.ContainerStart(ctx, containerID, container.StartOptions{}); err != nil { cli.Close() return fmt.Errorf("unable to start container image %v with docker for env %v, err: %w", dp.GetContainerImage(), wk.Env, err) } @@ -189,7 +189,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock case resp := <-statusCh: logger.Info("docker container has self terminated", "status_code", resp.StatusCode) - rc, err := cli.ContainerLogs(ctx, containerID, dtyp.ContainerLogsOptions{Details: true, ShowStdout: true, ShowStderr: true}) + rc, err := cli.ContainerLogs(ctx, containerID, container.LogsOptions{Details: true, ShowStdout: true, ShowStderr: true}) if err != nil { logger.Error("docker container logs error", "error", err) } From 9b564ef925b83ca040c46d54314c600f5e65940c Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 6 Aug 2024 17:33:29 -0700 Subject: [PATCH 22/78] Exclude a not yet implemented pandas op from dataframe tests. (#32066) --- sdks/python/apache_beam/dataframe/pandas_doctests_test.py | 1 + sdks/python/setup.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py index a4bd0d0a81278..c7ea908a93365 100644 --- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py +++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py @@ -557,6 +557,7 @@ def test_series_tests(self): 'pandas.core.series.Series': ['ser.iloc[0] = 999'], }, not_implemented_ok={ + 'pandas.core.series.Series.case_when': ['*'], 'pandas.core.series.Series.transform': [ # str arg not supported. Tested with np.sum in # frames_test.py::DeferredFrameTest::test_groupby_transform_sum diff --git a/sdks/python/setup.py b/sdks/python/setup.py index c9b2d087d04ca..756c952b0101b 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -155,7 +155,7 @@ def cythonize(*args, **kwargs): # Exclude 1.5.0 and 1.5.1 because of # https://github.com/pandas-dev/pandas/issues/45725 dataframe_dependency = [ - 'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.2;python_version>="3.8"', + 'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.3;python_version>="3.8"', ] From eeddc6924c3230f6b502af67914918730a27efc9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 17:36:59 -0700 Subject: [PATCH 23/78] Bump google.golang.org/grpc from 1.64.0 to 1.65.0 in /sdks (#31824) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.64.0 to 1.65.0. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.64.0...v1.65.0) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 7 ++++--- sdks/go.sum | 13 ++++++++----- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 7eb29b3cc77c3..9cdad36010e43 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( golang.org/x/text v0.16.0 google.golang.org/api v0.187.0 google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d - google.golang.org/grpc v1.64.1 + google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.34.2 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 @@ -74,6 +74,7 @@ require ( ) require ( + cel.dev/expr v0.15.0 // indirect cloud.google.com/go/auth v0.6.1 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect dario.cat/mergo v1.0.0 // indirect @@ -135,8 +136,8 @@ require ( github.com/aws/aws-sdk-go-v2/service/sts v1.28.12 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b // indirect github.com/containerd/containerd v1.7.18 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/docker/docker v27.1.1+incompatible // but required to resolve issue docker has with go1.20 diff --git a/sdks/go.sum b/sdks/go.sum index ce10d84dd044e..935009cf1d83d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1,3 +1,5 @@ +cel.dev/expr v0.15.0 h1:O1jzfJCQBfL5BFoYktaxwIhuttaQPsVWerH9/EEKx0w= +cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= @@ -735,8 +737,9 @@ github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMr github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -754,8 +757,8 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 h1:DBmgJDC9dTfkVyGgipamEh2BpGYxScCH1TOF1LL1cXc= -github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50/go.mod h1:5e1+Vvlzido69INQaVO6d87Qn543Xr6nooe9Kz7oBFM= +github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b h1:ga8SEFjZ60pxLcmhnThWgvH2wg8376yUJmPhEH4H3kw= +github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/containerd/containerd v1.7.18 h1:jqjZTQNfXGoEaZdW1WwPU0RqSn1Bm2Ay/KJPUuO8nao= github.com/containerd/containerd v1.7.18/go.mod h1:IYEk9/IO6wAPUz2bCMVUbsfXjzw5UNP5fLz4PsUygQ4= @@ -1881,8 +1884,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.64.1 h1:LKtvyfbX3UGVPFcGqJ9ItpVWW6oN/2XqTxfAnwRRXiA= -google.golang.org/grpc v1.64.1/go.mod h1:hiQF4LFZelK2WKaP6W0L92zGHtiQdZxk8CrSdvyjeP0= +google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= +google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= From 99672af7fe12f72b562289d0c9449e4711b973c2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 17:38:40 -0700 Subject: [PATCH 24/78] Bump torch from 1.13.1 to 2.2.0 in /sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train (#31983) Bumps [torch](https://github.com/pytorch/pytorch) from 1.13.1 to 2.2.0. - [Release notes](https://github.com/pytorch/pytorch/releases) - [Changelog](https://github.com/pytorch/pytorch/blob/main/RELEASE.md) - [Commits](https://github.com/pytorch/pytorch/compare/v1.13.1...v2.2.0) --- updated-dependencies: - dependency-name: torch dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../ml-orchestration/kfp/components/train/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt index 2e65f0fba2468..ba1103dd1ef96 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt @@ -13,6 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -torch==1.13.1 +torch==2.2.0 numpy==1.22.4 Pillow==10.2.0 \ No newline at end of file From ebba3bb026b63f05d358a2cb5608cf61107ee504 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 22:23:26 -0700 Subject: [PATCH 25/78] Bump go.mongodb.org/mongo-driver from 1.13.1 to 1.16.0 in /sdks (#32097) Bumps [go.mongodb.org/mongo-driver](https://github.com/mongodb/mongo-go-driver) from 1.13.1 to 1.16.0. - [Release notes](https://github.com/mongodb/mongo-go-driver/releases) - [Commits](https://github.com/mongodb/mongo-go-driver/compare/v1.13.1...v1.16.0) --- updated-dependencies: - dependency-name: go.mongodb.org/mongo-driver dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 11 ++++------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9cdad36010e43..2d638fc1998ff 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/tetratelabs/wazero v1.7.3 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c - go.mongodb.org/mongo-driver v1.13.1 + go.mongodb.org/mongo-driver v1.16.0 golang.org/x/net v0.26.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.7.0 @@ -165,7 +165,7 @@ require ( github.com/moby/patternmatcher v0.6.0 // indirect github.com/moby/sys/sequential v0.5.0 // indirect github.com/moby/term v0.5.0 // indirect - github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe // indirect + github.com/montanaflynn/stats v0.7.1 // indirect github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 935009cf1d83d..138c484884d55 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1019,7 +1019,6 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= -github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= @@ -1075,8 +1074,8 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= -github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6fbMZ9s0scYfZQ84/6SPL6zC8ACM2oIL0= -github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= +github.com/montanaflynn/stats v0.7.1 h1:etflOAAHORrCC44V+aR6Ftzort912ZU+YLiSTuV8eaE= +github.com/montanaflynn/stats v0.7.1/go.mod h1:etXPPgVO6n31NxCd9KQUMvCM+ve0ruNzt6R8Bnaayow= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/nats-io/jwt/v2 v2.5.7 h1:j5lH1fUXCnJnY8SsQeB/a/z9Azgu2bYIDvtPVNdxe2c= @@ -1209,8 +1208,8 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD go.einride.tech/aip v0.67.1 h1:d/4TW92OxXBngkSOwWS2CH5rez869KpKMaN44mdxkFI= go.einride.tech/aip v0.67.1/go.mod h1:ZGX4/zKw8dcgzdLsrvpOOGxfxI2QSk12SlP7d6c0/XI= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.mongodb.org/mongo-driver v1.13.1 h1:YIc7HTYsKndGK4RFzJ3covLz1byri52x0IoMB0Pt/vk= -go.mongodb.org/mongo-driver v1.13.1/go.mod h1:wcDf1JBCXy2mOW0bWHwO/IOYqdca1MPCwDtFu/Z9+eo= +go.mongodb.org/mongo-driver v1.16.0 h1:tpRsfBJMROVHKpdGyc1BBEzzjDUWjItxbVSZ8Ls4BQ4= +go.mongodb.org/mongo-driver v1.16.0/go.mod h1:oB6AhJQvFQL4LEHyXi6aJzQJtBiTQHiAd83l0GdFaiw= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -1253,7 +1252,6 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1357,7 +1355,6 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210813160813-60bc85c4be6d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220325170049-de3da57026de/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= From 44a9942719e754fbd1967e475466f404b9f47e22 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 7 Aug 2024 08:19:28 +0200 Subject: [PATCH 26/78] Add warning + doc callout when encountering ri pickling errors (#32063) --- sdks/python/apache_beam/ml/inference/base.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 401b57fdb8079..29a568def07b9 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -1586,6 +1586,15 @@ def _run_inference(self, batch, inference_args): except BaseException as e: if self._metrics_collector: self._metrics_collector.failed_batches_counter.inc() + if (e is pickle.PickleError and + self._model_handler.share_model_across_processes()): + raise TypeError( + 'Pickling error encountered while running inference. ' + 'This may be caused by trying to send unpickleable ' + 'data to a model which is shared across processes. ' + 'For more information, see ' + 'https://beam.apache.org/documentation/ml/large-language-modeling/#pickling-errors' # pylint: disable=line-too-long + ) from e raise e predictions = list(result_generator) From 0d81c5993049bc72116c2871c9b50bbe4cfc43d5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 10:08:13 -0400 Subject: [PATCH 27/78] Bump golang.org/x/text from 0.16.0 to 0.17.0 in /sdks (#32098) Bumps [golang.org/x/text](https://github.com/golang/text) from 0.16.0 to 0.17.0. - [Release notes](https://github.com/golang/text/releases) - [Commits](https://github.com/golang/text/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/text dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 2d638fc1998ff..958a228c3546b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,9 +56,9 @@ require ( go.mongodb.org/mongo-driver v1.16.0 golang.org/x/net v0.26.0 golang.org/x/oauth2 v0.21.0 - golang.org/x/sync v0.7.0 + golang.org/x/sync v0.8.0 golang.org/x/sys v0.21.0 - golang.org/x/text v0.16.0 + golang.org/x/text v0.17.0 google.golang.org/api v0.187.0 google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d google.golang.org/grpc v1.65.0 diff --git a/sdks/go.sum b/sdks/go.sum index 138c484884d55..ef3c436f22e02 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1423,8 +1423,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1540,8 +1540,8 @@ golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From 81ad4fee378a586701dc0ff25bcc3c5cd7a9f3f6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 10:08:42 -0400 Subject: [PATCH 28/78] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#32096) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.17.18 to 1.17.27. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/credentials/v1.17.18...credentials/v1.17.27) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 23 +++++++++++------------ sdks/go.sum | 45 ++++++++++++++++++++++----------------------- 2 files changed, 33 insertions(+), 35 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 958a228c3546b..a5ad9f3b7f5c5 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.39.0 cloud.google.com/go/spanner v1.63.0 cloud.google.com/go/storage v1.43.0 - github.com/aws/aws-sdk-go-v2 v1.30.0 + github.com/aws/aws-sdk-go-v2 v1.30.3 github.com/aws/aws-sdk-go-v2/config v1.27.4 - github.com/aws/aws-sdk-go-v2/credentials v1.17.18 + github.com/aws/aws-sdk-go-v2/credentials v1.17.27 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 - github.com/aws/smithy-go v1.20.2 + github.com/aws/smithy-go v1.20.3 github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.8.1 @@ -122,18 +122,18 @@ require ( github.com/apache/thrift v0.17.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.5 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.20.11 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.28.12 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -158,7 +158,6 @@ require ( github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect - github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/compress v1.17.8 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/magiconair/properties v1.8.7 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index ef3c436f22e02..6afc175732ff3 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -677,39 +677,39 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.30.0 h1:6qAwtzlfcTtcL8NHtbDQAqgM5s6NDipQTkPxyH/6kAA= -github.com/aws/aws-sdk-go-v2 v1.30.0/go.mod h1:ffIFB97e2yNsv4aTSGkqtHnppsIJzw7G7BReUZ3jCXM= +github.com/aws/aws-sdk-go-v2 v1.30.3 h1:jUeBtG0Ih+ZIFH0F4UkmL9w3cSpaMv9tYYDbzILP8dY= +github.com/aws/aws-sdk-go-v2 v1.30.3/go.mod h1:nIQjQVp5sfpQcTc9mPSr1B0PaWK5ByX9MOoDadSN4lc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= github.com/aws/aws-sdk-go-v2/config v1.27.4 h1:AhfWb5ZwimdsYTgP7Od8E9L1u4sKmDW2ZVeLcf2O42M= github.com/aws/aws-sdk-go-v2/config v1.27.4/go.mod h1:zq2FFXK3A416kiukwpsd+rD4ny6JC7QSkp4QdN1Mp2g= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.17.18 h1:D/ALDWqK4JdY3OFgA2thcPO1c9aYTT5STS/CvnkqY1c= -github.com/aws/aws-sdk-go-v2/credentials v1.17.18/go.mod h1:JuitCWq+F5QGUrmMPsk945rop6bB57jdscu+Glozdnc= +github.com/aws/aws-sdk-go-v2/credentials v1.17.27 h1:2raNba6gr2IfA0eqqiP2XiQ0UVOpGPgDSi0I9iAP+UI= +github.com/aws/aws-sdk-go-v2/credentials v1.17.27/go.mod h1:gniiwbGahQByxan6YjQUMcW4Aov6bLC3m+evgcoN4r4= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.5 h1:dDgptDO9dxeFkXy+tEgVkzSClHZje/6JkPW5aZyEvrQ= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.5/go.mod h1:gjvE2KBUgUQhcv89jqxrIxH9GaKs1JbZzWejj/DaHGA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 h1:KreluoV8FZDEtI6Co2xuNk/UqI9iwMrOx/87PBNIKqw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvHE0Tjvn7kbxaUhl75CJi1sbfhMxkU= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9 h1:cy8ahBJuhtM8GTTSyOkfy6WVPV1IE+SS5/wfXUYuulw= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9/go.mod h1:CZBXGLaJnEZI6EVNcPd7a6B5IC5cA/GkRWtu9fp3S6Y= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9 h1:A4SYk07ef04+vxZToz9LWvAXl9LW0NClpPpMsi31cz0= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9/go.mod h1:5jJcHuwDagxN+ErjQ3PU3ocf6Ylc/p9x+BLO/+X4iXw= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 h1:SoNJ4RlFEQEbtDcCEt+QG56MY4fm4W8rYirAmq+/DdU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15/go.mod h1:U9ke74k1n2bf+RIgoX1SXFed1HLs51OgUSs+Ph0KJP8= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 h1:C6WHdGnTDIYETAm5iErQUiVNsclNx9qbJVPIt03B6bI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15/go.mod h1:ZQLZqhcu+JhSrA9/NXRm8SkDvsycE+JkV3WGY41e+IM= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 h1:Ji0DY1xUsUr3I8cHps0G+XM3WWU16lP6yG8qu1GAZAs= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2/go.mod h1:5CsjAbs3NlGQyZNFACh+zztPDI7fU6eW9QsxjfnuBKg= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 h1:dT3MqvGhSoaIhRseqw2I0yH81l7wiR2vjs57O51EAm8= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3/go.mod h1:GlAeCkHwugxdHaueRr4nhPuY+WW+gR8UjlcqzPr1SPI= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11 h1:o4T+fKxA3gTMcluBNZZXE9DNaMkJuUL1O3mffCUjoJo= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11/go.mod h1:84oZdJ+VjuJKs9v1UTC9NaodRZRseOXCTgku+vQJWR8= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 h1:HGErhhrxZlQ044RiM+WdoZxp0p+EGM62y3L6pwA4olE= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17/go.mod h1:RkZEx4l0EHYDJpWppMJ3nD9wZJAa8/0lq9aVC+r2UII= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -717,16 +717,16 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.20.11 h1:gEYM2GSpr4YNWc6hCd5nod4+d4kd9vWIAWrmGuLdlMw= -github.com/aws/aws-sdk-go-v2/service/sso v1.20.11/go.mod h1:gVvwPdPNYehHSP9Rs7q27U1EU+3Or2ZpXvzAYJNh63w= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5 h1:iXjh3uaH3vsVcnyZX7MqCoCfcyxIrVE9iOQruRaWPrQ= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5/go.mod h1:5ZXesEuy/QcO0WUnt+4sDkxhdXRHTu2yG0uCSH8B6os= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 h1:BXx0ZIxvrJdSgSvKTZ+yRBeSqqgPM89VPlulEcl37tM= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.4/go.mod h1:ooyCOXjvJEsUw7x+ZDHeISPMhtwI3ZCB7ggFMcFfWLU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 h1:yiwVzJW2ZxZTurVbYWA7QOrAaCYQR72t0wrSBfoesUE= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4/go.mod h1:0oxfLkpz3rQ/CHlx5hB7H69YUpFiI1tql6Q6Ne+1bCw= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.28.12 h1:M/1u4HBpwLuMtjlxuI2y6HoVLzF5e2mfxHCg7ZVMYmk= -github.com/aws/aws-sdk-go-v2/service/sts v1.28.12/go.mod h1:kcfd+eTdEi/40FIbLq4Hif3XMXnl5b/+t/KTfLt9xIk= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 h1:ZsDKRLXGWHk8WdtyYMoGNO7bTudrvuKpDKgMVRlepGE= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.3/go.mod h1:zwySh8fpFyXp9yOr/KVzxOl8SRqgf/IDw5aUt9UKFcQ= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.20.2 h1:tbp628ireGtzcHDDmLT/6ADHidqnwgF57XOXZe6tp4Q= -github.com/aws/smithy-go v1.20.2/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= +github.com/aws/smithy-go v1.20.3 h1:ryHwveWzPV5BIof6fyDvor6V3iUL7nTfiTKXHiW05nE= +github.com/aws/smithy-go v1.20.3/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -1003,7 +1003,6 @@ github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI= From 828717a71d638664ba12cad5c0c00193bb1cde35 Mon Sep 17 00:00:00 2001 From: Vatsal <36672090+imvtsl@users.noreply.github.com> Date: Wed, 7 Aug 2024 09:54:15 -0700 Subject: [PATCH 29/78] [#21515][Go SDK] Update go protobuf package to new version (#32045) --- CHANGES.md | 1 + sdks/go.mod | 2 +- sdks/go/cmd/beamctl/cmd/provision.go | 3 +-- sdks/go/container/boot_test.go | 2 +- sdks/go/container/tools/provision.go | 13 +++++++++---- sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go | 2 +- sdks/go/pkg/beam/artifact/gcsproxy/staging.go | 2 +- sdks/go/pkg/beam/artifact/materialize.go | 2 +- sdks/go/pkg/beam/artifact/materialize_test.go | 2 +- sdks/go/pkg/beam/coder.go | 12 ++++++------ sdks/go/pkg/beam/core/runtime/exec/translate.go | 2 +- sdks/go/pkg/beam/core/runtime/graphx/coder.go | 8 ++++---- .../pkg/beam/core/runtime/graphx/schema/schema.go | 2 +- .../beam/core/runtime/graphx/schema/schema_test.go | 3 +-- sdks/go/pkg/beam/core/runtime/graphx/translate.go | 8 ++++---- .../pkg/beam/core/runtime/graphx/translate_test.go | 10 +++++----- .../pkg/beam/core/runtime/harness/harness_test.go | 2 +- sdks/go/pkg/beam/core/runtime/harness/statemgr.go | 5 ++--- .../pkg/beam/core/runtime/pipelinex/clone_test.go | 2 +- sdks/go/pkg/beam/core/runtime/pipelinex/replace.go | 2 +- .../pkg/beam/core/runtime/pipelinex/replace_test.go | 2 +- sdks/go/pkg/beam/core/runtime/pipelinex/util.go | 2 +- .../go/pkg/beam/core/runtime/xlangx/resolve_test.go | 2 +- sdks/go/pkg/beam/core/util/protox/any.go | 6 +++--- sdks/go/pkg/beam/core/util/protox/any_test.go | 4 ++-- sdks/go/pkg/beam/core/util/protox/base64.go | 2 +- sdks/go/pkg/beam/core/util/protox/protox.go | 2 +- sdks/go/pkg/beam/create_test.go | 6 ++---- sdks/go/pkg/beam/provision/provision.go | 2 +- sdks/go/pkg/beam/runners/dataflow/dataflow.go | 3 +-- .../beam/runners/dataflow/dataflowlib/execute.go | 3 +-- sdks/go/pkg/beam/runners/universal/runnerlib/job.go | 3 +-- .../pkg/beam/runners/universal/runnerlib/stage.go | 2 +- sdks/go/pkg/beam/runners/universal/universal.go | 3 +-- .../go/pkg/beam/transforms/xlang/schema/external.go | 2 +- sdks/java/container/boot.go | 7 +++---- sdks/python/container/boot.go | 12 ++++++------ 37 files changed, 72 insertions(+), 76 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 129fa01f94a84..d082f03fd310e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,6 +69,7 @@ * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Go SDK Minimum Go Version updated to 1.21 ([#32092](https://github.com/apache/beam/pull/32092)). +* Updated Go protobuf package to new version (Go) ([#21515](https://github.com/apache/beam/issues/21515)). ## Breaking Changes diff --git a/sdks/go.mod b/sdks/go.mod index a5ad9f3b7f5c5..fb0b7f85f3dea 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -39,7 +39,6 @@ require ( github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.8.1 - github.com/golang/protobuf v1.5.4 // TODO(danoliveira): Fully replace this with google.golang.org/protobuf github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.6.0 github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 @@ -88,6 +87,7 @@ require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect + github.com/golang/protobuf v1.5.4 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect diff --git a/sdks/go/cmd/beamctl/cmd/provision.go b/sdks/go/cmd/beamctl/cmd/provision.go index cab82f7bf9db8..878c9a77da82f 100644 --- a/sdks/go/cmd/beamctl/cmd/provision.go +++ b/sdks/go/cmd/beamctl/cmd/provision.go @@ -17,7 +17,6 @@ package cmd import ( fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" - "github.com/golang/protobuf/proto" "github.com/spf13/cobra" ) @@ -53,6 +52,6 @@ func infoFn(cmd *cobra.Command, args []string) error { return err } - cmd.Print(proto.MarshalTextString(info.GetInfo())) + cmd.Print(info.GetInfo().String()) return nil } diff --git a/sdks/go/container/boot_test.go b/sdks/go/container/boot_test.go index e799e5d65b0cc..49c78047249e5 100644 --- a/sdks/go/container/boot_test.go +++ b/sdks/go/container/boot_test.go @@ -25,7 +25,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/artifact" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) func TestEnsureEndpointsSet_AllSet(t *testing.T) { diff --git a/sdks/go/container/tools/provision.go b/sdks/go/container/tools/provision.go index dab3383fc1710..6b370a5c2e663 100644 --- a/sdks/go/container/tools/provision.go +++ b/sdks/go/container/tools/provision.go @@ -29,8 +29,8 @@ import ( fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" - "github.com/golang/protobuf/jsonpb" - google_pb "github.com/golang/protobuf/ptypes/struct" + "google.golang.org/protobuf/encoding/protojson" + google_pb "google.golang.org/protobuf/types/known/structpb" ) // ProvisionInfo returns the runtime provisioning info for the worker. @@ -65,7 +65,8 @@ func OptionsToProto(v any) (*google_pb.Struct, error) { // JSONToProto converts JSON-encoded pipeline options to a proto struct. func JSONToProto(data string) (*google_pb.Struct, error) { var out google_pb.Struct - if err := jsonpb.UnmarshalString(string(data), &out); err != nil { + + if err := protojson.Unmarshal([]byte(data), &out); err != nil { return nil, err } return &out, nil @@ -85,5 +86,9 @@ func ProtoToJSON(opt *google_pb.Struct) (string, error) { if opt == nil { return "{}", nil } - return (&jsonpb.Marshaler{}).MarshalToString(opt) + bytes, err := protojson.Marshal(opt) + if err != nil { + return "", err + } + return string(bytes), err } diff --git a/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go b/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go index 15c2d9e2954a4..ceb8a319be982 100644 --- a/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go +++ b/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go @@ -22,8 +22,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/gcsx" - "github.com/golang/protobuf/proto" "golang.org/x/net/context" + "google.golang.org/protobuf/proto" ) // RetrievalServer is a artifact retrieval server backed by Google diff --git a/sdks/go/pkg/beam/artifact/gcsproxy/staging.go b/sdks/go/pkg/beam/artifact/gcsproxy/staging.go index a295084398078..9113e780f3391 100644 --- a/sdks/go/pkg/beam/artifact/gcsproxy/staging.go +++ b/sdks/go/pkg/beam/artifact/gcsproxy/staging.go @@ -28,8 +28,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/gcsx" - "github.com/golang/protobuf/proto" "golang.org/x/net/context" + "google.golang.org/protobuf/proto" ) // StagingServer is a artifact staging server backed by Google Cloud Storage diff --git a/sdks/go/pkg/beam/artifact/materialize.go b/sdks/go/pkg/beam/artifact/materialize.go index 866e0dd99b9fa..624e30efcd2b3 100644 --- a/sdks/go/pkg/beam/artifact/materialize.go +++ b/sdks/go/pkg/beam/artifact/materialize.go @@ -38,7 +38,7 @@ import ( pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/errorx" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // TODO(lostluck): 2018/05/28 Extract these from their enum descriptors in the pipeline_v1 proto diff --git a/sdks/go/pkg/beam/artifact/materialize_test.go b/sdks/go/pkg/beam/artifact/materialize_test.go index 35223c908b773..31890ed045cc8 100644 --- a/sdks/go/pkg/beam/artifact/materialize_test.go +++ b/sdks/go/pkg/beam/artifact/materialize_test.go @@ -29,9 +29,9 @@ import ( jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" - "github.com/golang/protobuf/proto" "google.golang.org/grpc" "google.golang.org/grpc/metadata" + "google.golang.org/protobuf/proto" ) // TestRetrieve tests that we can successfully retrieve fresh files. diff --git a/sdks/go/pkg/beam/coder.go b/sdks/go/pkg/beam/coder.go index 062bb337e8d81..b03b739ed7be4 100644 --- a/sdks/go/pkg/beam/coder.go +++ b/sdks/go/pkg/beam/coder.go @@ -30,8 +30,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/jsonx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - protov1 "github.com/golang/protobuf/proto" protov2 "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/protoadapt" "google.golang.org/protobuf/reflect/protoreflect" ) @@ -51,7 +51,7 @@ type jsonCoder interface { json.Unmarshaler } -var protoMessageType = reflect.TypeOf((*protov1.Message)(nil)).Elem() +var protoMessageType = reflect.TypeOf((*protoadapt.MessageV1)(nil)).Elem() var protoReflectMessageType = reflect.TypeOf((*protoreflect.ProtoMessage)(nil)).Elem() var jsonCoderType = reflect.TypeOf((*jsonCoder)(nil)).Elem() @@ -276,8 +276,8 @@ func protoEnc(in T) ([]byte, error) { switch it := in.(type) { case protoreflect.ProtoMessage: p = it - case protov1.Message: - p = protov1.MessageV2(it) + case protoadapt.MessageV1: + p = protoadapt.MessageV2Of(it) } b, err := protov2.MarshalOptions{Deterministic: true}.Marshal(p) if err != nil { @@ -293,8 +293,8 @@ func protoDec(t reflect.Type, in []byte) (T, error) { switch it := reflect.New(t.Elem()).Interface().(type) { case protoreflect.ProtoMessage: p = it - case protov1.Message: - p = protov1.MessageV2(it) + case protoadapt.MessageV1: + p = protoadapt.MessageV2Of(it) } err := protov2.UnmarshalOptions{}.Unmarshal(in, p) if err != nil { diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 72af9e80c4052..b74ede228fd97 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -33,7 +33,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // TODO(lostluck): 2018/05/28 Extract these from the canonical enums in beam_runner_api.proto diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder.go b/sdks/go/pkg/beam/core/runtime/graphx/coder.go index 87b3771e5756a..99ca5517d3d39 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder.go @@ -27,7 +27,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) const ( @@ -615,8 +615,8 @@ func (b *CoderMarshaller) internRowCoder(schema *pipepb.Schema) string { } func (b *CoderMarshaller) internCoder(coder *pipepb.Coder) string { - key := proto.MarshalTextString(coder) - if id, exists := b.coder2id[key]; exists { + key := coder.String() + if id, exists := b.coder2id[(key)]; exists { return id } @@ -626,7 +626,7 @@ func (b *CoderMarshaller) internCoder(coder *pipepb.Coder) string { } else { id = fmt.Sprintf("c%v@%v", len(b.coder2id), b.Namespace) } - b.coder2id[key] = id + b.coder2id[string(key)] = id b.coders[id] = coder return id } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go index fdd9355e1cb81..0d44e68285b55 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go @@ -37,8 +37,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" "github.com/google/uuid" + "google.golang.org/protobuf/proto" ) // Initialize registered schemas. For use by the beam package at beam.Init time. diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go index 37b3e79f8f504..367d70e81d174 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go @@ -24,7 +24,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" "github.com/google/go-cmp/cmp" "google.golang.org/protobuf/encoding/prototext" "google.golang.org/protobuf/testing/protocmp" @@ -806,7 +805,7 @@ func TestSchemaConversion(t *testing.T) { } if d := cmp.Diff(test.st, got, protocmp.Transform(), - protocmp.IgnoreFields(proto.MessageV2(&pipepb.Schema{}), "id"), + protocmp.IgnoreFields(&pipepb.Schema{}, "id"), ); d != "" { t.Errorf("diff (-want, +got): %v", d) } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index b05292546133a..65280ef6b9303 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -34,7 +34,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/options/resource" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/durationpb" ) @@ -1209,13 +1209,13 @@ func (m *marshaller) addWindowingStrategy(w *window.WindowingStrategy) (string, } func (m *marshaller) internWindowingStrategy(w *pipepb.WindowingStrategy) string { - key := proto.MarshalTextString(w) - if id, exists := m.windowing2id[key]; exists { + key := w.String() + if id, exists := m.windowing2id[(key)]; exists { return id } id := fmt.Sprintf("w%v", len(m.windowing2id)) - m.windowing2id[key] = id + m.windowing2id[string(key)] = id m.windowing[id] = w return id } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go index a331aedd585de..e18a5f97796b3 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go @@ -34,8 +34,8 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" "github.com/google/go-cmp/cmp" + "google.golang.org/protobuf/proto" ) func init() { @@ -181,13 +181,13 @@ func TestMarshal(t *testing.T) { } if got, want := len(p.GetComponents().GetTransforms()), test.transforms; got != want { - t.Errorf("got %d transforms, want %d : %v", got, want, proto.MarshalTextString(p)) + t.Errorf("got %d transforms, want %d : %v", got, want, p.String()) } if got, want := len(p.GetRootTransformIds()), test.roots; got != want { - t.Errorf("got %d roots, want %d : %v", got, want, proto.MarshalTextString(p)) + t.Errorf("got %d roots, want %d : %v", got, want, p.String()) } if got, want := p.GetRequirements(), test.requirements; !cmp.Equal(got, want, cmpopts.SortSlices(func(a, b string) bool { return a < b })) { - t.Errorf("incorrect requirements: got %v, want %v : %v", got, want, proto.MarshalTextString(p)) + t.Errorf("incorrect requirements: got %v, want %v : %v", got, want, p.String()) } }) } @@ -248,7 +248,7 @@ func TestMarshal_PTransformAnnotations(t *testing.T) { pts := p.GetComponents().GetTransforms() if got, want := len(pts), test.transforms; got != want { - t.Errorf("got %d transforms, want %d : %v", got, want, proto.MarshalTextString(p)) + t.Errorf("got %d transforms, want %d : %v", got, want, p.String()) } for _, pt := range pts { // Context annotations only apply to composites, and are not duplicated to leaves. diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go index 91dd3c591d5b3..8c25db613eba7 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go @@ -23,7 +23,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/exec" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // validDescriptor describes a valid pipeline with a source and a sink, but doesn't do anything else. diff --git a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go index 76d4e1f32c23a..061cfca011f55 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go @@ -28,7 +28,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" "github.com/apache/beam/sdks/v2/go/pkg/beam/log" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" - "github.com/golang/protobuf/proto" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) @@ -633,7 +632,7 @@ func (c *StateChannel) read(ctx context.Context) { if !ok { // This can happen if Send returns an error that write handles, but // the message was actually sent. - log.Errorf(ctx, "StateChannel[%v].read: no consumer for state response: %v", c.id, proto.MarshalTextString(msg)) + log.Errorf(ctx, "StateChannel[%v].read: no consumer for state response: %v", c.id, msg.String()) continue } @@ -641,7 +640,7 @@ func (c *StateChannel) read(ctx context.Context) { case ch <- msg: // ok default: - panic(fmt.Sprintf("StateChannel[%v].read: failed to consume state response: %v", c.id, proto.MarshalTextString(msg))) + panic(fmt.Sprintf("StateChannel[%v].read: failed to consume state response: %v", c.id, msg.String())) } } } diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go b/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go index 695830a483c07..b58a309837978 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go @@ -19,8 +19,8 @@ import ( "testing" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" "github.com/google/go-cmp/cmp" + "google.golang.org/protobuf/proto" ) func TestShallowClonePTransform(t *testing.T) { diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go b/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go index cfcce88675bed..9e527f2fd3220 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go @@ -28,7 +28,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // Update merges a pipeline with the given components, which may add, replace diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go b/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go index 79bfd43958aff..3024787e61631 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go @@ -20,8 +20,8 @@ import ( "testing" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" "github.com/google/go-cmp/cmp" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/testing/protocmp" ) diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go index 5fe9def9b2276..4735e7b77d206 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go @@ -19,7 +19,7 @@ import ( "sort" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // Bounded returns true iff all PCollections are bounded. diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go b/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go index 1f18b333541bd..eec13c451a135 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go @@ -20,7 +20,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) func createExternalEdge(typeUrn string, typePayload []byte) *graph.MultiEdge { diff --git a/sdks/go/pkg/beam/core/util/protox/any.go b/sdks/go/pkg/beam/core/util/protox/any.go index e539a8c19dec0..46bd08b1aff10 100644 --- a/sdks/go/pkg/beam/core/util/protox/any.go +++ b/sdks/go/pkg/beam/core/util/protox/any.go @@ -17,9 +17,9 @@ package protox import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/golang/protobuf/proto" - protobuf "github.com/golang/protobuf/ptypes/any" - protobufw "github.com/golang/protobuf/ptypes/wrappers" + "google.golang.org/protobuf/proto" + protobuf "google.golang.org/protobuf/types/known/anypb" + protobufw "google.golang.org/protobuf/types/known/wrapperspb" ) const ( diff --git a/sdks/go/pkg/beam/core/util/protox/any_test.go b/sdks/go/pkg/beam/core/util/protox/any_test.go index 1975bec405cb8..9eb7621db3510 100644 --- a/sdks/go/pkg/beam/core/util/protox/any_test.go +++ b/sdks/go/pkg/beam/core/util/protox/any_test.go @@ -19,8 +19,8 @@ import ( "bytes" "testing" - "github.com/golang/protobuf/proto" - protobufw "github.com/golang/protobuf/ptypes/wrappers" + "google.golang.org/protobuf/proto" + protobufw "google.golang.org/protobuf/types/known/wrapperspb" ) func TestProtoPackingInvertibility(t *testing.T) { diff --git a/sdks/go/pkg/beam/core/util/protox/base64.go b/sdks/go/pkg/beam/core/util/protox/base64.go index 7f0f5a4bdeea1..79ea8a025f7c9 100644 --- a/sdks/go/pkg/beam/core/util/protox/base64.go +++ b/sdks/go/pkg/beam/core/util/protox/base64.go @@ -19,7 +19,7 @@ import ( "encoding/base64" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) // MustEncodeBase64 encodes a proto wrapped in base64 and panics on failure. diff --git a/sdks/go/pkg/beam/core/util/protox/protox.go b/sdks/go/pkg/beam/core/util/protox/protox.go index 3555886eefc9a..892a2ba97d039 100644 --- a/sdks/go/pkg/beam/core/util/protox/protox.go +++ b/sdks/go/pkg/beam/core/util/protox/protox.go @@ -16,7 +16,7 @@ // Package protox contains utilities for working with protobufs. package protox -import "github.com/golang/protobuf/proto" +import "google.golang.org/protobuf/proto" // MustEncode encode the message and panics on failure. func MustEncode(msg proto.Message) []byte { diff --git a/sdks/go/pkg/beam/create_test.go b/sdks/go/pkg/beam/create_test.go index 785c3b33db621..e65fefc7f2d89 100644 --- a/sdks/go/pkg/beam/create_test.go +++ b/sdks/go/pkg/beam/create_test.go @@ -23,7 +23,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/protoadapt" ) func TestMain(m *testing.M) { @@ -157,7 +157,5 @@ func (t *testProto) Unmarshal(b []byte) error { // Ensure testProto is detected as a proto.Message and can be (un)marshalled by // the proto library. var ( - _ proto.Message = &testProto{} - _ proto.Marshaler = &testProto{} - _ proto.Unmarshaler = &testProto{} + _ protoadapt.MessageV1 = &testProto{} ) diff --git a/sdks/go/pkg/beam/provision/provision.go b/sdks/go/pkg/beam/provision/provision.go index 3c36973535e79..58a8f5ee82928 100644 --- a/sdks/go/pkg/beam/provision/provision.go +++ b/sdks/go/pkg/beam/provision/provision.go @@ -24,7 +24,7 @@ import ( "github.com/apache/beam/sdks/v2/go/container/tools" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" - google_pb "github.com/golang/protobuf/ptypes/struct" + google_pb "google.golang.org/protobuf/types/known/structpb" ) // Info returns the runtime provisioning info for the worker. diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go index ca701979497ae..73667fb8ee6ee 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go @@ -47,7 +47,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/dataflow/dataflowlib" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/gcsx" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/hooks/perf" - "github.com/golang/protobuf/proto" ) // TODO(herohde) 5/16/2017: the Dataflow flags should match the other SDKs. @@ -235,7 +234,7 @@ func Execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) if *dryRun { log.Info(ctx, "Dry-run: not submitting job!") - log.Info(ctx, proto.MarshalTextString(model)) + log.Info(ctx, model.String()) job, err := dataflowlib.Translate(ctx, model, opts, workerURL, modelURL) if err != nil { return nil, err diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go index 9a1641e314d12..806b8940ae994 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go @@ -30,7 +30,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/log" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/universal/runnerlib" - "github.com/golang/protobuf/proto" df "google.golang.org/api/dataflow/v1b3" "google.golang.org/api/googleapi" ) @@ -82,7 +81,7 @@ func Execute(ctx context.Context, raw *pipepb.Pipeline, opts *JobOptions, worker } // (2) Upload model to GCS - log.Info(ctx, proto.MarshalTextString(raw)) + log.Info(ctx, raw.String()) if err := StageModel(ctx, opts.Project, modelURL, protox.MustEncode(raw)); err != nil { return presult, err diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/job.go b/sdks/go/pkg/beam/runners/universal/runnerlib/job.go index 4e50661b3db8e..7d6a3027e47e6 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/job.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/job.go @@ -28,7 +28,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/log" jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" ) // JobOptions capture the various options for submitting jobs @@ -152,7 +151,7 @@ func WaitForCompletion(ctx context.Context, client jobpb.JobServiceClient, jobID } default: - return errors.Errorf("unexpected job update: %v", proto.MarshalTextString(msg)) + return errors.Errorf("unexpected job update: %v", msg.String()) } } } diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go b/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go index d5cc6aa7327a7..85d6fdc7e2ca4 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go @@ -29,8 +29,8 @@ import ( jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" - "github.com/golang/protobuf/proto" "google.golang.org/grpc" + "google.golang.org/protobuf/proto" ) // Stage stages the worker binary and any additional files to the given diff --git a/sdks/go/pkg/beam/runners/universal/universal.go b/sdks/go/pkg/beam/runners/universal/universal.go index 8af9e91e1e15e..c63175c58578f 100644 --- a/sdks/go/pkg/beam/runners/universal/universal.go +++ b/sdks/go/pkg/beam/runners/universal/universal.go @@ -32,7 +32,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/universal/extworker" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/universal/runnerlib" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/vet" - "github.com/golang/protobuf/proto" ) func init() { @@ -93,7 +92,7 @@ func Execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) return nil, errors.WithContextf(err, "generating model pipeline") } - log.Info(ctx, proto.MarshalTextString(pipeline)) + log.Info(ctx, pipeline.String()) opt := &runnerlib.JobOptions{ Name: jobopts.GetJobName(), diff --git a/sdks/go/pkg/beam/transforms/xlang/schema/external.go b/sdks/go/pkg/beam/transforms/xlang/schema/external.go index 75be90cbe7b3a..55a858b9cf9ef 100644 --- a/sdks/go/pkg/beam/transforms/xlang/schema/external.go +++ b/sdks/go/pkg/beam/transforms/xlang/schema/external.go @@ -20,7 +20,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/xlangx" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/proto" ) const schemaTransformURN = "beam:expansion:payload:schematransform:v1" diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go index 14e2e4311b458..c23e50dcf1b06 100644 --- a/sdks/java/container/boot.go +++ b/sdks/java/container/boot.go @@ -35,7 +35,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/util/execx" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/syscallx" - "github.com/golang/protobuf/proto" ) var ( @@ -126,12 +125,12 @@ func main() { if err := tools.MakePipelineOptionsFileAndEnvVar(options); err != nil { logger.Fatalf(ctx, "Failed to load pipeline options to worker: %v", err) } - os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint})) - os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pipepb.ApiServiceDescriptor{Url: *controlEndpoint})) + os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint}).String()) + os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *controlEndpoint}).String()) os.Setenv("RUNNER_CAPABILITIES", strings.Join(info.GetRunnerCapabilities(), " ")) if info.GetStatusEndpoint() != nil { - os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(info.GetStatusEndpoint())) + os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", info.GetStatusEndpoint().String()) } const jarsDir = "/opt/apache/beam/jars" diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go index 710041e0f0410..696604c64886d 100644 --- a/sdks/python/container/boot.go +++ b/sdks/python/container/boot.go @@ -41,8 +41,8 @@ import ( pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/execx" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" - "github.com/golang/protobuf/jsonpb" - "github.com/golang/protobuf/proto" + "google.golang.org/protobuf/encoding/protojson" + "google.golang.org/protobuf/proto" ) var ( @@ -217,12 +217,12 @@ func launchSDKProcess() error { os.Setenv("PIPELINE_OPTIONS", options) os.Setenv("SEMI_PERSISTENT_DIRECTORY", *semiPersistDir) - os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint})) - os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pipepb.ApiServiceDescriptor{Url: *controlEndpoint})) + os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint}).String()) + os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *controlEndpoint}).String()) os.Setenv("RUNNER_CAPABILITIES", strings.Join(info.GetRunnerCapabilities(), " ")) if info.GetStatusEndpoint() != nil { - os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(info.GetStatusEndpoint())) + os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", info.GetStatusEndpoint().String()) } if metadata := info.GetMetadata(); metadata != nil { @@ -441,7 +441,7 @@ func processArtifactsInSetupOnlyMode() { files := make([]string, len(infoJsons)) for i, info := range infoJsons { var artifactInformation pipepb.ArtifactInformation - if err := jsonpb.UnmarshalString(info, &artifactInformation); err != nil { + if err := protojson.Unmarshal([]byte(info), &artifactInformation); err != nil { log.Fatalf("Unable to unmarshal artifact information from json string %v", info) } From b54967eab41f51e9329833d5e2ac18ee522c151c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 7 Aug 2024 13:02:29 -0400 Subject: [PATCH 30/78] Fix Beam Schema to Iceberg Schema ID conversion logic (#32095) * fix iceberg schema ID logic * trigger integration tests --- .../IO_Iceberg_Integration_Tests.json | 2 +- .../beam/sdk/io/iceberg/IcebergUtils.java | 181 +++++++++--------- .../beam/sdk/io/iceberg/IcebergUtilsTest.java | 113 +++++------ 3 files changed, 148 insertions(+), 148 deletions(-) diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 3f63c0c9975f2..bbdc3a3910ef8 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 2 + "modification": 3 } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index a2f84e6475c9c..acd9b25a6a5e3 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -34,6 +34,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -115,113 +116,110 @@ private static Schema icebergStructTypeToBeamSchema(final Types.StructType struc } /** - * Represents an Object (in practice, either {@link Type} or {@link Types.NestedField}) along with - * the most recent (max) ID that has been used to build this object. + * Represents a {@link Type} and the most recent field ID used to build it. * *

Iceberg Schema fields are required to have unique IDs. This includes unique IDs for a {@link - * Types.ListType}'s collection type, a {@link Types.MapType}'s key type and value type, and - * nested {@link Types.StructType}s. When constructing any of these types, we use multiple unique - * ID's for the type's components. The {@code maxId} in this object represents the most recent ID - * used after building this type. This helps signal that the next field we construct should have - * an ID greater than this one. + * org.apache.iceberg.types.Type.NestedType}'s components (e.g. {@link Types.ListType}'s + * collection type, {@link Types.MapType}'s key type and value type, and {@link + * Types.StructType}'s nested fields). The {@code maxId} in this object represents the most recent + * ID used after building this type. This helps signal that the next {@link + * org.apache.iceberg.types.Type.NestedType} we construct should have an ID greater than this one. */ @VisibleForTesting - static class ObjectAndMaxId { + static class TypeAndMaxId { int maxId; - T object; + Type type; - ObjectAndMaxId(int id, T object) { + TypeAndMaxId(int id, Type object) { this.maxId = id; - this.object = object; + this.type = object; } } /** - * Given a Beam {@link Schema.FieldType} and an index, returns an Iceberg {@link Type} and the - * maximum index after building the Iceberg Type. This assumes the input index is already in use - * (usually by the parent {@link Types.NestedField}, and will start building the Iceberg type from - * index + 1. + * Takes a Beam {@link Schema.FieldType} and an index intended as a starting point for Iceberg + * {@link org.apache.iceberg.types.Type.NestedType}s. Returns an Iceberg {@link Type} and the + * maximum index after building that type. * - *

Returns this information in an {@link ObjectAndMaxId} instance. + *

Returns this information in an {@link TypeAndMaxId} object. */ @VisibleForTesting - static ObjectAndMaxId beamFieldTypeToIcebergFieldType( - int fieldId, Schema.FieldType beamType) { + static TypeAndMaxId beamFieldTypeToIcebergFieldType( + Schema.FieldType beamType, int nestedFieldId) { if (BEAM_TYPES_TO_ICEBERG_TYPES.containsKey(beamType.getTypeName())) { - return new ObjectAndMaxId<>(fieldId, BEAM_TYPES_TO_ICEBERG_TYPES.get(beamType.getTypeName())); + // we don't use nested field ID for primitive types. decrement it so the caller can use it for + // other types. + return new TypeAndMaxId( + --nestedFieldId, BEAM_TYPES_TO_ICEBERG_TYPES.get(beamType.getTypeName())); } else if (beamType.getTypeName().isCollectionType()) { // ARRAY or ITERABLE - // List ID needs to be unique from the NestedField that contains this ListType - int listId = fieldId + 1; Schema.FieldType beamCollectionType = Preconditions.checkArgumentNotNull(beamType.getCollectionElementType()); - ObjectAndMaxId listInfo = beamFieldTypeToIcebergFieldType(listId, beamCollectionType); - Type icebergCollectionType = listInfo.object; + // nestedFieldId is reserved for the list's collection type. + // we increment here because further nested fields should use unique ID's + TypeAndMaxId listInfo = + beamFieldTypeToIcebergFieldType(beamCollectionType, nestedFieldId + 1); + Type icebergCollectionType = listInfo.type; boolean elementTypeIsNullable = Preconditions.checkArgumentNotNull(beamType.getCollectionElementType()).getNullable(); Type listType = elementTypeIsNullable - ? Types.ListType.ofOptional(listId, icebergCollectionType) - : Types.ListType.ofRequired(listId, icebergCollectionType); + ? Types.ListType.ofOptional(nestedFieldId, icebergCollectionType) + : Types.ListType.ofRequired(nestedFieldId, icebergCollectionType); - return new ObjectAndMaxId<>(listInfo.maxId, listType); + return new TypeAndMaxId(listInfo.maxId, listType); } else if (beamType.getTypeName().isMapType()) { // MAP - // key and value IDs need to be unique from the NestedField that contains this MapType - int keyId = fieldId + 1; - int valueId = fieldId + 2; - int maxId = valueId; + // key and value IDs need to be unique + int keyId = nestedFieldId; + int valueId = keyId + 1; + // nested field IDs should be unique + nestedFieldId = valueId + 1; Schema.FieldType beamKeyType = Preconditions.checkArgumentNotNull(beamType.getMapKeyType()); - ObjectAndMaxId keyInfo = beamFieldTypeToIcebergFieldType(maxId, beamKeyType); - Type icebergKeyType = keyInfo.object; - maxId = keyInfo.maxId; + TypeAndMaxId keyInfo = beamFieldTypeToIcebergFieldType(beamKeyType, nestedFieldId); + Type icebergKeyType = keyInfo.type; + nestedFieldId = keyInfo.maxId + 1; Schema.FieldType beamValueType = Preconditions.checkArgumentNotNull(beamType.getMapValueType()); - ObjectAndMaxId valueInfo = beamFieldTypeToIcebergFieldType(maxId, beamValueType); - Type icebergValueType = valueInfo.object; - maxId = valueInfo.maxId; + TypeAndMaxId valueInfo = beamFieldTypeToIcebergFieldType(beamValueType, nestedFieldId); + Type icebergValueType = valueInfo.type; Type mapType = beamValueType.getNullable() ? Types.MapType.ofOptional(keyId, valueId, icebergKeyType, icebergValueType) : Types.MapType.ofRequired(keyId, valueId, icebergKeyType, icebergValueType); - return new ObjectAndMaxId<>(maxId, mapType); + return new TypeAndMaxId(valueInfo.maxId, mapType); } else if (beamType.getTypeName().isCompositeType()) { // ROW // Nested field IDs need to be unique from the field that contains this StructType - int maxFieldId = fieldId; - Schema nestedSchema = Preconditions.checkArgumentNotNull(beamType.getRowSchema()); List nestedFields = new ArrayList<>(nestedSchema.getFieldCount()); - for (Schema.Field field : nestedSchema.getFields()) { - ObjectAndMaxId converted = beamFieldToIcebergField(++maxFieldId, field); - Types.NestedField nestedField = converted.object; - nestedFields.add(nestedField); - maxFieldId = converted.maxId; + int icebergFieldId = nestedFieldId; + nestedFieldId = icebergFieldId + nestedSchema.getFieldCount(); + for (Schema.Field beamField : nestedSchema.getFields()) { + TypeAndMaxId typeAndMaxId = + beamFieldTypeToIcebergFieldType(beamField.getType(), nestedFieldId); + Types.NestedField icebergField = + Types.NestedField.of( + icebergFieldId++, + beamField.getType().getNullable(), + beamField.getName(), + typeAndMaxId.type); + + nestedFields.add(icebergField); + nestedFieldId = typeAndMaxId.maxId + 1; } Type structType = Types.StructType.of(nestedFields); - return new ObjectAndMaxId<>(maxFieldId, structType); + return new TypeAndMaxId(nestedFieldId - 1, structType); } - return new ObjectAndMaxId<>(fieldId, Types.StringType.get()); - } - - private static ObjectAndMaxId beamFieldToIcebergField( - int fieldId, final Schema.Field field) { - ObjectAndMaxId typeAndMaxId = beamFieldTypeToIcebergFieldType(fieldId, field.getType()); - Type icebergType = typeAndMaxId.object; - int id = typeAndMaxId.maxId; - - Types.NestedField icebergField = - Types.NestedField.of(fieldId, field.getType().getNullable(), field.getName(), icebergType); - - return new ObjectAndMaxId<>(id, icebergField); + return new TypeAndMaxId(nestedFieldId, Types.StringType.get()); } /** @@ -233,18 +231,23 @@ private static ObjectAndMaxId beamFieldToIcebergField( *

  • {@link Schema.TypeName.LOGICAL_TYPE} */ public static org.apache.iceberg.Schema beamSchemaToIcebergSchema(final Schema schema) { - Types.NestedField[] fields = new Types.NestedField[schema.getFieldCount()]; - int nextIcebergFieldId = 1; - for (int i = 0; i < schema.getFieldCount(); i++) { - Schema.Field beamField = schema.getField(i); - ObjectAndMaxId fieldAndMaxId = - beamFieldToIcebergField(nextIcebergFieldId, beamField); - Types.NestedField field = fieldAndMaxId.object; - fields[i] = field; - - nextIcebergFieldId = fieldAndMaxId.maxId + 1; + List fields = new ArrayList<>(schema.getFieldCount()); + int nestedFieldId = schema.getFieldCount() + 1; + int icebergFieldId = 1; + for (Schema.Field beamField : schema.getFields()) { + TypeAndMaxId typeAndMaxId = + beamFieldTypeToIcebergFieldType(beamField.getType(), nestedFieldId); + Types.NestedField icebergField = + Types.NestedField.of( + icebergFieldId++, + beamField.getType().getNullable(), + beamField.getName(), + typeAndMaxId.type); + + fields.add(icebergField); + nestedFieldId = typeAndMaxId.maxId + 1; } - return new org.apache.iceberg.Schema(fields); + return new org.apache.iceberg.Schema(fields.toArray(new Types.NestedField[fields.size()])); } /** Converts a Beam {@link Row} to an Iceberg {@link Record}. */ @@ -323,27 +326,21 @@ private static void copyFieldIntoRecord(Record rec, Types.NestedField field, Row public static Row icebergRecordToBeamRow(Schema schema, Record record) { Row.Builder rowBuilder = Row.withSchema(schema); for (Schema.Field field : schema.getFields()) { + boolean isNullable = field.getType().getNullable(); + @Nullable Object icebergValue = record.getField(field.getName()); + if (icebergValue == null) { + if (isNullable) { + rowBuilder.addValue(null); + continue; + } + throw new RuntimeException( + String.format("Received null value for required field '%s'.", field.getName())); + } switch (field.getType().getTypeName()) { case BYTE: - // I guess allow anything we can cast here - byte byteValue = (byte) record.getField(field.getName()); - rowBuilder.addValue(byteValue); - break; case INT16: - // I guess allow anything we can cast here - short shortValue = (short) record.getField(field.getName()); - rowBuilder.addValue(shortValue); - break; case INT32: - // I guess allow anything we can cast here - int intValue = (int) record.getField(field.getName()); - rowBuilder.addValue(intValue); - break; case INT64: - // I guess allow anything we can cast here - long longValue = (long) record.getField(field.getName()); - rowBuilder.addValue(longValue); - break; case DECIMAL: // Iceberg and Beam both use BigDecimal case FLOAT: // Iceberg and Beam both use float case DOUBLE: // Iceberg and Beam both use double @@ -352,29 +349,31 @@ public static Row icebergRecordToBeamRow(Schema schema, Record record) { case ARRAY: case ITERABLE: case MAP: - rowBuilder.addValue(record.getField(field.getName())); + rowBuilder.addValue(icebergValue); break; case DATETIME: // Iceberg uses a long for millis; Beam uses joda time DateTime - long millis = (long) record.getField(field.getName()); + long millis = (long) icebergValue; rowBuilder.addValue(new DateTime(millis, DateTimeZone.UTC)); break; case BYTES: // Iceberg uses ByteBuffer; Beam uses byte[] - rowBuilder.addValue(((ByteBuffer) record.getField(field.getName())).array()); + rowBuilder.addValue(((ByteBuffer) icebergValue).array()); break; case ROW: - Record nestedRecord = (Record) record.getField(field.getName()); + Record nestedRecord = (Record) icebergValue; Schema nestedSchema = checkArgumentNotNull( field.getType().getRowSchema(), "Corrupted schema: Row type did not have associated nested schema."); - Row nestedRow = icebergRecordToBeamRow(nestedSchema, nestedRecord); - rowBuilder.addValue(nestedRow); + rowBuilder.addValue(icebergRecordToBeamRow(nestedSchema, nestedRecord)); break; case LOGICAL_TYPE: throw new UnsupportedOperationException( "Cannot convert iceberg field to Beam logical type"); + default: + throw new UnsupportedOperationException( + "Unsupported Beam type: " + field.getType().getTypeName()); } } return rowBuilder.build(); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java index c4da0b22f4d95..a20d5b7c8f59a 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.IcebergUtils.ObjectAndMaxId; +import static org.apache.beam.sdk.io.iceberg.IcebergUtils.TypeAndMaxId; import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamFieldTypeToIcebergFieldType; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -316,11 +316,11 @@ private static class BeamFieldTypeTestCase { private void checkTypes(List testCases) { for (BeamFieldTypeTestCase testCase : testCases) { - ObjectAndMaxId ret = - beamFieldTypeToIcebergFieldType(testCase.icebergFieldId, testCase.beamType); + TypeAndMaxId ret = + beamFieldTypeToIcebergFieldType(testCase.beamType, testCase.icebergFieldId); assertEquals(testCase.expectedMaxId, ret.maxId); - checkEquals(testCase.expectedIcebergType, ret.object); + checkEquals(testCase.expectedIcebergType, ret.type); } } @@ -338,65 +338,65 @@ private void checkEquals(Type expected, Type actual) { @Test public void testPrimitiveBeamFieldTypeToIcebergFieldType() { + // primitive types don't use the nested field ID List primitives = Arrays.asList( - new BeamFieldTypeTestCase(1, Schema.FieldType.BOOLEAN, 1, Types.BooleanType.get()), - new BeamFieldTypeTestCase(3, Schema.FieldType.INT32, 3, Types.IntegerType.get()), - new BeamFieldTypeTestCase(6, Schema.FieldType.INT64, 6, Types.LongType.get()), - new BeamFieldTypeTestCase(10, Schema.FieldType.FLOAT, 10, Types.FloatType.get()), - new BeamFieldTypeTestCase(7, Schema.FieldType.DOUBLE, 7, Types.DoubleType.get()), - new BeamFieldTypeTestCase(11, Schema.FieldType.STRING, 11, Types.StringType.get()), - new BeamFieldTypeTestCase(15, Schema.FieldType.BYTES, 15, Types.BinaryType.get())); + new BeamFieldTypeTestCase(1, Schema.FieldType.BOOLEAN, 0, Types.BooleanType.get()), + new BeamFieldTypeTestCase(3, Schema.FieldType.INT32, 2, Types.IntegerType.get()), + new BeamFieldTypeTestCase(6, Schema.FieldType.INT64, 5, Types.LongType.get()), + new BeamFieldTypeTestCase(10, Schema.FieldType.FLOAT, 9, Types.FloatType.get()), + new BeamFieldTypeTestCase(7, Schema.FieldType.DOUBLE, 6, Types.DoubleType.get()), + new BeamFieldTypeTestCase(11, Schema.FieldType.STRING, 10, Types.StringType.get()), + new BeamFieldTypeTestCase(15, Schema.FieldType.BYTES, 14, Types.BinaryType.get())); checkTypes(primitives); } @Test public void testArrayBeamFieldTypeToIcebergFieldType() { - // Iceberg sets one field ID for the List type itself and another field ID for the collection - // type. + // Iceberg's ListType reserves one nested ID for its element type List listTypes = Arrays.asList( new BeamFieldTypeTestCase( 1, Schema.FieldType.array(Schema.FieldType.BOOLEAN), - 2, + 1, Types.ListType.ofRequired(1, Types.BooleanType.get())), new BeamFieldTypeTestCase( 3, Schema.FieldType.iterable(Schema.FieldType.INT32), - 4, + 3, Types.ListType.ofRequired(3, Types.IntegerType.get())), new BeamFieldTypeTestCase( 6, Schema.FieldType.array(Schema.FieldType.INT64), - 7, + 6, Types.ListType.ofRequired(6, Types.LongType.get())), new BeamFieldTypeTestCase( 10, Schema.FieldType.array(Schema.FieldType.FLOAT), - 11, + 10, Types.ListType.ofRequired(10, Types.FloatType.get())), new BeamFieldTypeTestCase( 7, Schema.FieldType.iterable(Schema.FieldType.DOUBLE), - 8, + 7, Types.ListType.ofRequired(7, Types.DoubleType.get())), new BeamFieldTypeTestCase( 11, Schema.FieldType.array(Schema.FieldType.STRING), - 12, + 11, Types.ListType.ofRequired(11, Types.StringType.get())), new BeamFieldTypeTestCase( 15, Schema.FieldType.iterable(Schema.FieldType.BYTES), - 16, + 15, Types.ListType.ofRequired(15, Types.BinaryType.get())), new BeamFieldTypeTestCase( 23, Schema.FieldType.array( Schema.FieldType.array(Schema.FieldType.iterable(Schema.FieldType.STRING))), - 26, + 25, Types.ListType.ofRequired( 23, Types.ListType.ofRequired( @@ -407,23 +407,23 @@ public void testArrayBeamFieldTypeToIcebergFieldType() { @Test public void testStructBeamFieldTypeToIcebergFieldType() { - // Iceberg sets one field ID for each nested type. + // Iceberg sets one unique field ID for each nested type. List listTypes = Arrays.asList( new BeamFieldTypeTestCase( 1, Schema.FieldType.row(Schema.builder().addStringField("str").build()), - 2, + 1, Types.StructType.of( - Types.NestedField.required(2, "str", Types.StringType.get()))), + Types.NestedField.required(1, "str", Types.StringType.get()))), new BeamFieldTypeTestCase( 3, Schema.FieldType.row(Schema.builder().addInt32Field("int").build()), - 4, + 3, Types.StructType.of( - Types.NestedField.required(4, "int", Types.IntegerType.get()))), + Types.NestedField.required(3, "int", Types.IntegerType.get()))), new BeamFieldTypeTestCase( - 0, + 1, Schema.FieldType.row(BEAM_SCHEMA_PRIMITIVE), 7, Types.StructType.of(ICEBERG_SCHEMA_PRIMITIVE.columns())), @@ -434,11 +434,11 @@ public void testStructBeamFieldTypeToIcebergFieldType() { .addArrayField("arr", Schema.FieldType.STRING) .addNullableStringField("str") .build()), - 18, + 17, Types.StructType.of( Types.NestedField.required( - 16, "arr", Types.ListType.ofRequired(17, Types.StringType.get())), - Types.NestedField.optional(18, "str", Types.StringType.get()))), + 15, "arr", Types.ListType.ofRequired(17, Types.StringType.get())), + Types.NestedField.optional(16, "str", Types.StringType.get()))), new BeamFieldTypeTestCase( 20, Schema.FieldType.row( @@ -452,10 +452,10 @@ public void testStructBeamFieldTypeToIcebergFieldType() { .addNullableRowField( "nullable_row", Schema.builder().addInt64Field("long").build()) .build()), - 25, + 24, Types.StructType.of( Types.NestedField.required( - 21, + 20, "row", Types.StructType.of( Types.NestedField.required( @@ -465,33 +465,34 @@ public void testStructBeamFieldTypeToIcebergFieldType() { Types.NestedField.required( 23, "str", Types.StringType.get()))))), Types.NestedField.optional( - 24, + 21, "nullable_row", Types.StructType.of( - Types.NestedField.required(25, "long", Types.LongType.get())))))); + Types.NestedField.required(24, "long", Types.LongType.get())))))); checkTypes(listTypes); } @Test public void testMapBeamFieldTypeToIcebergFieldType() { + // Iceberg's MapType reserves two nested IDs. one for its key type and one for its value type. List primitives = Arrays.asList( new BeamFieldTypeTestCase( 1, Schema.FieldType.map(Schema.FieldType.STRING, Schema.FieldType.INT32), - 3, - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.IntegerType.get())), + 2, + Types.MapType.ofRequired(1, 2, Types.StringType.get(), Types.IntegerType.get())), new BeamFieldTypeTestCase( 6, Schema.FieldType.map( Schema.FieldType.FLOAT, Schema.FieldType.array(Schema.FieldType.STRING)), - 9, + 8, Types.MapType.ofRequired( + 6, 7, - 8, Types.FloatType.get(), - Types.ListType.ofRequired(9, Types.StringType.get()))), + Types.ListType.ofRequired(8, Types.StringType.get()))), new BeamFieldTypeTestCase( 10, Schema.FieldType.map( @@ -499,30 +500,30 @@ public void testMapBeamFieldTypeToIcebergFieldType() { Schema.FieldType.map( Schema.FieldType.BOOLEAN, Schema.FieldType.map(Schema.FieldType.STRING, Schema.FieldType.INT32))), - 16, + 15, Types.MapType.ofRequired( + 10, 11, - 12, Types.StringType.get(), Types.MapType.ofRequired( + 12, 13, - 14, Types.BooleanType.get(), Types.MapType.ofRequired( - 15, 16, Types.StringType.get(), Types.IntegerType.get())))), + 14, 15, Types.StringType.get(), Types.IntegerType.get())))), new BeamFieldTypeTestCase( 15, Schema.FieldType.map( Schema.FieldType.row(Schema.builder().addStringField("str").build()), Schema.FieldType.row(Schema.builder().addInt32Field("int").build())), - 19, + 18, Types.MapType.ofRequired( + 15, 16, - 17, Types.StructType.of( - Types.NestedField.required(18, "str", Types.StringType.get())), + Types.NestedField.required(17, "str", Types.StringType.get())), Types.StructType.of( - Types.NestedField.required(19, "int", Types.IntegerType.get()))))); + Types.NestedField.required(18, "int", Types.IntegerType.get()))))); checkTypes(primitives); } @@ -574,9 +575,9 @@ public void testPrimitiveIcebergSchemaToBeamSchema() { .build(); static final org.apache.iceberg.Schema ICEBERG_SCHEMA_LIST = new org.apache.iceberg.Schema( - required(1, "arr_str", Types.ListType.ofRequired(2, Types.StringType.get())), - required(3, "arr_int", Types.ListType.ofRequired(4, Types.IntegerType.get())), - required(5, "arr_bool", Types.ListType.ofRequired(6, Types.BooleanType.get()))); + required(1, "arr_str", Types.ListType.ofRequired(4, Types.StringType.get())), + required(2, "arr_int", Types.ListType.ofRequired(5, Types.IntegerType.get())), + required(3, "arr_bool", Types.ListType.ofRequired(6, Types.BooleanType.get()))); @Test public void testArrayBeamSchemaToIcebergSchema() { @@ -607,9 +608,9 @@ public void testArrayIcebergSchemaToBeamSchema() { required( 1, "str_int", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.IntegerType.get())), + Types.MapType.ofRequired(3, 4, Types.StringType.get(), Types.IntegerType.get())), optional( - 4, + 2, "long_bool", Types.MapType.ofRequired(5, 6, Types.LongType.get(), Types.BooleanType.get()))); @@ -648,11 +649,11 @@ public void testMapIcebergSchemaToBeamSchema() { 1, "row", Types.StructType.of( - required(2, "str", Types.StringType.get()), - optional(3, "int", Types.IntegerType.get()), - required(4, "long", Types.LongType.get()))), + required(3, "str", Types.StringType.get()), + optional(4, "int", Types.IntegerType.get()), + required(5, "long", Types.LongType.get()))), optional( - 5, + 2, "nullable_row", Types.StructType.of( optional(6, "str", Types.StringType.get()), From 07e692b56fb19550c40eede5d39b951851eb8980 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 10:06:22 -0700 Subject: [PATCH 31/78] Bump github.com/nats-io/nats-server/v2 from 2.10.12 to 2.10.17 in /sdks (#31709) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.12 to 2.10.17. - [Release notes](https://github.com/nats-io/nats-server/releases) - [Changelog](https://github.com/nats-io/nats-server/blob/main/.goreleaser.yml) - [Commits](https://github.com/nats-io/nats-server/compare/v2.10.12...v2.10.17) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 14 +++++++------- sdks/go.sum | 32 ++++++++++++++++---------------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index fb0b7f85f3dea..654a456285431 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -44,8 +44,8 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 - github.com/nats-io/nats-server/v2 v2.10.16 - github.com/nats-io/nats.go v1.35.0 + github.com/nats-io/nats-server/v2 v2.10.18 + github.com/nats-io/nats.go v1.36.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.1 github.com/testcontainers/testcontainers-go v0.32.0 @@ -56,7 +56,7 @@ require ( golang.org/x/net v0.26.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.8.0 - golang.org/x/sys v0.21.0 + golang.org/x/sys v0.22.0 golang.org/x/text v0.17.0 google.golang.org/api v0.187.0 google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d @@ -89,10 +89,10 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/minio/highwayhash v1.0.2 // indirect + github.com/minio/highwayhash v1.0.3 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/user v0.1.0 // indirect - github.com/nats-io/jwt/v2 v2.5.7 // indirect + github.com/nats-io/jwt/v2 v2.5.8 // indirect github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect @@ -158,7 +158,7 @@ require ( github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect - github.com/klauspost/compress v1.17.8 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/moby/patternmatcher v0.6.0 // indirect @@ -181,7 +181,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.24.0 // indirect + golang.org/x/crypto v0.25.0 // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 6afc175732ff3..1c03d5afc89cd 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1019,8 +1019,8 @@ github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= -github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -1051,8 +1051,8 @@ github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/ github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= -github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= -github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= +github.com/minio/highwayhash v1.0.3 h1:kbnuUMoHYyVl7szWjSxJnxw11k2U709jqFPPmIUyD6Q= +github.com/minio/highwayhash v1.0.3/go.mod h1:GGYsuwP/fPD6Y9hMiXuapVvlIUEhFhMTh0rxU3ik1LQ= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= @@ -1077,12 +1077,12 @@ github.com/montanaflynn/stats v0.7.1 h1:etflOAAHORrCC44V+aR6Ftzort912ZU+YLiSTuV8 github.com/montanaflynn/stats v0.7.1/go.mod h1:etXPPgVO6n31NxCd9KQUMvCM+ve0ruNzt6R8Bnaayow= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= -github.com/nats-io/jwt/v2 v2.5.7 h1:j5lH1fUXCnJnY8SsQeB/a/z9Azgu2bYIDvtPVNdxe2c= -github.com/nats-io/jwt/v2 v2.5.7/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= -github.com/nats-io/nats-server/v2 v2.10.16 h1:2jXaiydp5oB/nAx/Ytf9fdCi9QN6ItIc9eehX8kwVV0= -github.com/nats-io/nats-server/v2 v2.10.16/go.mod h1:Pksi38H2+6xLe1vQx0/EA4bzetM0NqyIHcIbmgXSkIU= -github.com/nats-io/nats.go v1.35.0 h1:XFNqNM7v5B+MQMKqVGAyHwYhyKb48jrenXNxIU20ULk= -github.com/nats-io/nats.go v1.35.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= +github.com/nats-io/jwt/v2 v2.5.8 h1:uvdSzwWiEGWGXf+0Q+70qv6AQdvcvxrv9hPM0RiPamE= +github.com/nats-io/jwt/v2 v2.5.8/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= +github.com/nats-io/nats-server/v2 v2.10.18 h1:tRdZmBuWKVAFYtayqlBB2BuCHNGAQPvoQIXOKwU3WSM= +github.com/nats-io/nats-server/v2 v2.10.18/go.mod h1:97Qyg7YydD8blKlR8yBsUlPlWyZKjA7Bp5cl3MUE9K8= +github.com/nats-io/nats.go v1.36.0 h1:suEUPuWzTSse/XhESwqLxXGuj8vGRuPRoG7MoRN/qyU= +github.com/nats-io/nats.go v1.36.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= github.com/nats-io/nkeys v0.4.7 h1:RwNJbbIdYCoClSDNY7QVKZlyb/wfT6ugvFCiKy6vDvI= github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= @@ -1251,8 +1251,8 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= -golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= +golang.org/x/crypto v0.25.0 h1:ypSNr+bnYL2YhwoMt2zPxHFmbAN1KZs/njMG3hxUp30= +golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1425,7 +1425,6 @@ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1511,8 +1510,9 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= +golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= @@ -1521,8 +1521,8 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= -golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= -golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= +golang.org/x/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= +golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 656a296a82d00f1d17745bbe6a161e75104506e7 Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 7 Aug 2024 13:16:00 -0700 Subject: [PATCH 32/78] Enable Job management for the Prism runner (#32091) --- .../beam/runners/prism/PrismJobManager.java | 160 +++++++++++++ .../runners/prism/PrismJobManagerTest.java | 211 ++++++++++++++++++ 2 files changed, 371 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismJobManager.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismJobManagerTest.java diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismJobManager.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismJobManager.java new file mode 100644 index 0000000000000..e461e92c47496 --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismJobManager.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.io.Closeable; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.apache.beam.model.jobmanagement.v1.JobApi; +import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; +import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.joda.time.Duration; + +/** + * A wrapper for {@link JobServiceGrpc.JobServiceBlockingStub} that {@link #close}es when {@link + * StateListener#onStateChanged} is invoked with a {@link PipelineResult.State} that is {@link + * PipelineResult.State#isTerminal}. + */ +@AutoValue +abstract class PrismJobManager implements StateListener, Closeable { + + /** + * Instantiate a {@link PrismJobManager} with {@param options}, assigning {@link #getEndpoint} + * from {@link PortablePipelineOptions#getJobEndpoint} and {@link #getTimeout} from {@link + * PortablePipelineOptions#getJobServerTimeout}. Defaults the instantiations of {@link + * #getManagedChannel} and {@link #getBlockingStub}. See respective getters for more details. + */ + static PrismJobManager of(PortablePipelineOptions options) { + return builder() + .setEndpoint(options.getJobEndpoint()) + .setTimeout(Duration.standardSeconds(options.getJobServerTimeout())) + .build(); + } + + static Builder builder() { + return new AutoValue_PrismJobManager.Builder(); + } + + /** + * Executes {@link #getBlockingStub()}'s {@link JobServiceGrpc.JobServiceBlockingStub#prepare} + * method. + */ + JobApi.PrepareJobResponse prepare(JobApi.PrepareJobRequest request) { + return getBlockingStub().prepare(request); + } + + /** + * Executes {@link #getBlockingStub()}'s {@link JobServiceGrpc.JobServiceBlockingStub#run} method. + */ + JobApi.RunJobResponse run(JobApi.RunJobRequest request) { + return getBlockingStub().run(request); + } + + /** The {@link JobServiceGrpc} endpoint. */ + abstract String getEndpoint(); + + /** The {@link JobServiceGrpc} timeout. */ + abstract Duration getTimeout(); + + /** The {@link #getBlockingStub}'s channel. Defaulted from the {@link #getEndpoint()}. */ + abstract ManagedChannel getManagedChannel(); + + /** The wrapped service defaulted using the {@link #getManagedChannel}. */ + abstract JobServiceGrpc.JobServiceBlockingStub getBlockingStub(); + + /** Shuts down {@link #getManagedChannel}, if not {@link #isShutdown}. */ + @Override + public void close() { + if (isShutdown()) { + return; + } + getManagedChannel().shutdown(); + try { + getManagedChannel().awaitTermination(3000L, TimeUnit.MILLISECONDS); + } catch (InterruptedException ignored) { + } + } + + /** Queries whether {@link #getManagedChannel} {@link ManagedChannel#isShutdown}. */ + boolean isShutdown() { + return getManagedChannel().isShutdown(); + } + + /** + * Override of {@link StateListener#onStateChanged}. Invokes {@link #close} when {@link + * PipelineResult.State} {@link PipelineResult.State#isTerminal}. + */ + @Override + public void onStateChanged(PipelineResult.State state) { + if (state.isTerminal()) { + close(); + } + } + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setEndpoint(String endpoint); + + abstract Optional getEndpoint(); + + abstract Builder setTimeout(Duration timeout); + + abstract Optional getTimeout(); + + abstract Builder setManagedChannel(ManagedChannel managedChannel); + + abstract Optional getManagedChannel(); + + abstract Builder setBlockingStub(JobServiceGrpc.JobServiceBlockingStub blockingStub); + + abstract Optional getBlockingStub(); + + abstract PrismJobManager autoBuild(); + + final PrismJobManager build() { + + checkState(getEndpoint().isPresent(), "endpoint is not set"); + checkState(getTimeout().isPresent(), "timeout is not set"); + + if (!getManagedChannel().isPresent()) { + ManagedChannelFactory channelFactory = ManagedChannelFactory.createDefault(); + + setManagedChannel( + channelFactory.forDescriptor( + Endpoints.ApiServiceDescriptor.newBuilder().setUrl(getEndpoint().get()).build())); + } + + if (!getBlockingStub().isPresent()) { + setBlockingStub( + JobServiceGrpc.newBlockingStub(getManagedChannel().get()) + .withDeadlineAfter(getTimeout().get().getMillis(), TimeUnit.MILLISECONDS) + .withWaitForReady()); + } + + return autoBuild(); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismJobManagerTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismJobManagerTest.java new file mode 100644 index 0000000000000..1e38e4f8d12ed --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismJobManagerTest.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.prism; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; + +import java.io.IOException; +import java.util.Optional; +import org.apache.beam.model.jobmanagement.v1.JobApi; +import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.transforms.Impulse; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PrismJobManager}. */ +@RunWith(JUnit4.class) +public class PrismJobManagerTest { + @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + + @Rule public TestName testName = new TestName(); + + @Test + public void givenPrepareError_forwardsException_canGracefulShutdown() { + TestJobService service = + new TestJobService().withErrorResponse(new RuntimeException(testName.getMethodName())); + PrismJobManager underTest = prismJobManager(service); + assertThat(underTest.isShutdown()).isFalse(); + assertThrows( + RuntimeException.class, + () -> + underTest.prepare( + JobApi.PrepareJobRequest.newBuilder().setPipeline(pipelineOf()).build())); + assertThat(underTest.isShutdown()).isFalse(); + underTest.close(); + assertThat(underTest.isShutdown()).isTrue(); + } + + @Test + public void givenPrepareSuccess_forwardsResponse_canGracefulShutdown() { + TestJobService service = + new TestJobService() + .withPrepareJobResponse( + JobApi.PrepareJobResponse.newBuilder() + .setStagingSessionToken("token") + .setPreparationId("preparationId") + .setArtifactStagingEndpoint( + Endpoints.ApiServiceDescriptor.newBuilder() + .setUrl("localhost:1234") + .build()) + .build()); + PrismJobManager underTest = prismJobManager(service); + assertThat(underTest.isShutdown()).isFalse(); + JobApi.PrepareJobResponse response = + underTest.prepare(JobApi.PrepareJobRequest.newBuilder().setPipeline(pipelineOf()).build()); + assertThat(underTest.isShutdown()).isFalse(); + assertThat(response.getStagingSessionToken()).isEqualTo("token"); + assertThat(response.getPreparationId()).isEqualTo("preparationId"); + underTest.close(); + assertThat(underTest.isShutdown()).isTrue(); + } + + @Test + public void givenRunError_forwardsException_canGracefulShutdown() { + TestJobService service = + new TestJobService().withErrorResponse(new RuntimeException(testName.getMethodName())); + PrismJobManager underTest = prismJobManager(service); + assertThat(underTest.isShutdown()).isFalse(); + assertThrows( + RuntimeException.class, + () -> + underTest.run(JobApi.RunJobRequest.newBuilder().setPreparationId("prepareId").build())); + assertThat(underTest.isShutdown()).isFalse(); + underTest.close(); + assertThat(underTest.isShutdown()).isTrue(); + } + + @Test + public void givenRunSuccess_forwardsResponse_canGracefulShutdown() { + TestJobService service = + new TestJobService() + .withRunJobResponse(JobApi.RunJobResponse.newBuilder().setJobId("jobId").build()); + PrismJobManager underTest = prismJobManager(service); + assertThat(underTest.isShutdown()).isFalse(); + JobApi.RunJobResponse runJobResponse = + underTest.run(JobApi.RunJobRequest.newBuilder().setPreparationId("preparationId").build()); + assertThat(underTest.isShutdown()).isFalse(); + assertThat(runJobResponse.getJobId()).isEqualTo("jobId"); + underTest.close(); + assertThat(underTest.isShutdown()).isTrue(); + } + + @Test + public void givenTerminalState_closes() { + PrismJobManager underTest = prismJobManager(new TestJobService()); + assertThat(underTest.isShutdown()).isFalse(); + underTest.onStateChanged(PipelineResult.State.RUNNING); + assertThat(underTest.isShutdown()).isFalse(); + underTest.onStateChanged(PipelineResult.State.RUNNING); + assertThat(underTest.isShutdown()).isFalse(); + underTest.onStateChanged(PipelineResult.State.CANCELLED); + assertThat(underTest.isShutdown()).isTrue(); + + underTest.close(); + } + + private PrismJobManager prismJobManager(TestJobService service) { + String serverName = InProcessServerBuilder.generateName(); + try { + grpcCleanup.register( + InProcessServerBuilder.forName(serverName) + .directExecutor() + .addService(service) + .build() + .start()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + ManagedChannel channel = + grpcCleanup.register(InProcessChannelBuilder.forName(serverName).build()); + + return PrismJobManager.builder() + .setTimeout(Duration.millis(3000L)) + .setEndpoint("ignore") + .setManagedChannel(channel) + .build(); + } + + private static class TestJobService extends JobServiceGrpc.JobServiceImplBase { + + private Optional prepareJobResponse = Optional.empty(); + private Optional runJobResponse = Optional.empty(); + private Optional error = Optional.empty(); + + TestJobService withPrepareJobResponse(JobApi.PrepareJobResponse prepareJobResponse) { + this.prepareJobResponse = Optional.of(prepareJobResponse); + return this; + } + + TestJobService withRunJobResponse(JobApi.RunJobResponse runJobResponse) { + this.runJobResponse = Optional.of(runJobResponse); + return this; + } + + TestJobService withErrorResponse(RuntimeException error) { + this.error = Optional.of(error); + return this; + } + + @Override + public void prepare( + JobApi.PrepareJobRequest request, + StreamObserver responseObserver) { + if (prepareJobResponse.isPresent()) { + responseObserver.onNext(prepareJobResponse.get()); + responseObserver.onCompleted(); + } + if (error.isPresent()) { + responseObserver.onError(error.get()); + } + } + + @Override + public void run( + JobApi.RunJobRequest request, StreamObserver responseObserver) { + if (runJobResponse.isPresent()) { + responseObserver.onNext(runJobResponse.get()); + responseObserver.onCompleted(); + } + if (error.isPresent()) { + responseObserver.onError(error.get()); + } + } + } + + private static RunnerApi.Pipeline pipelineOf() { + Pipeline pipeline = Pipeline.create(); + pipeline.apply(Impulse.create()); + return PipelineTranslation.toProto(pipeline); + } +} From ea982127b60545164e0e280eb0d4140f35ae3156 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 7 Aug 2024 22:48:03 -0400 Subject: [PATCH 33/78] Override BQ load job location when necessary (#31986) --- .../apache_beam/io/gcp/bigquery_file_loads.py | 18 +++++++++++++++++- .../io/gcp/bigquery_file_loads_test.py | 10 ++++++++++ .../apache_beam/io/gcp/bigquery_tools.py | 8 ++++++-- 3 files changed, 33 insertions(+), 3 deletions(-) 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 e1a4af31f1c2e..3203c21a8e64a 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py @@ -765,10 +765,26 @@ def process( GlobalWindows.windowed_value((destination, job_reference))) def finish_bundle(self): + dataset_locations = {} + for windowed_value in self.pending_jobs: + table_ref = bigquery_tools.parse_table_reference(windowed_value.value[0]) + project_dataset = (table_ref.projectId, table_ref.datasetId) + job_ref = windowed_value.value[1] + # In some cases (e.g. when the load job op returns a 409 ALREADY_EXISTS), + # the returned job reference may not include a location. In such cases, + # we need to override with the dataset's location. + job_location = job_ref.location + if not job_location and project_dataset not in dataset_locations: + job_location = self.bq_wrapper.get_table_location( + table_ref.projectId, table_ref.datasetId, table_ref.tableId) + dataset_locations[project_dataset] = job_location + self.bq_wrapper.wait_for_bq_job( - job_ref, sleep_duration_sec=_SLEEP_DURATION_BETWEEN_POLLS) + job_ref, + sleep_duration_sec=_SLEEP_DURATION_BETWEEN_POLLS, + location=job_location) return self.pending_jobs 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 0605206714ed1..f27c7899f9f38 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 @@ -426,6 +426,7 @@ def test_records_traverse_transform_with_mocks(self): job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = bigquery_api.Job() result_job.jobReference = job_reference @@ -481,6 +482,7 @@ def test_load_job_id_used(self): job_reference = bigquery_api.JobReference() job_reference.projectId = 'loadJobProject' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = bigquery_api.Job() result_job.jobReference = job_reference @@ -515,6 +517,7 @@ def test_load_job_id_use_for_copy_job(self): job_reference = bigquery_api.JobReference() job_reference.projectId = 'loadJobProject' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = mock.Mock() result_job.jobReference = job_reference @@ -567,10 +570,12 @@ def test_wait_for_load_job_completion(self, sleep_mock): job_1.jobReference = bigquery_api.JobReference() job_1.jobReference.projectId = 'project1' job_1.jobReference.jobId = 'jobId1' + job_1.jobReference.location = 'US' job_2 = bigquery_api.Job() job_2.jobReference = bigquery_api.JobReference() job_2.jobReference.projectId = 'project1' job_2.jobReference.jobId = 'jobId2' + job_2.jobReference.location = 'US' job_1_waiting = mock.Mock() job_1_waiting.status.state = 'RUNNING' @@ -610,10 +615,12 @@ def test_one_load_job_failed_after_waiting(self, sleep_mock): job_1.jobReference = bigquery_api.JobReference() job_1.jobReference.projectId = 'project1' job_1.jobReference.jobId = 'jobId1' + job_1.jobReference.location = 'US' job_2 = bigquery_api.Job() job_2.jobReference = bigquery_api.JobReference() job_2.jobReference.projectId = 'project1' job_2.jobReference.jobId = 'jobId2' + job_2.jobReference.location = 'US' job_1_waiting = mock.Mock() job_1_waiting.status.state = 'RUNNING' @@ -650,6 +657,7 @@ def test_multiple_partition_files(self): job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = mock.Mock() result_job.jobReference = job_reference @@ -732,6 +740,7 @@ def test_multiple_partition_files_write_dispositions( job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = mock.Mock() result_job.jobReference = job_reference @@ -774,6 +783,7 @@ def test_triggering_frequency(self, is_streaming, with_auto_sharding): job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' + job_reference.location = 'US' result_job = bigquery_api.Job() result_job.jobReference = job_reference diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index a92f30ec35ce4..c7128e7899ecb 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -631,7 +631,8 @@ def _start_query_job( return self._start_job(request) - def wait_for_bq_job(self, job_reference, sleep_duration_sec=5, max_retries=0): + def wait_for_bq_job( + self, job_reference, sleep_duration_sec=5, max_retries=0, location=None): """Poll job until it is DONE. Args: @@ -639,6 +640,7 @@ def wait_for_bq_job(self, job_reference, sleep_duration_sec=5, max_retries=0): sleep_duration_sec: Specifies the delay in seconds between retries. max_retries: The total number of times to retry. If equals to 0, the function waits forever. + location: Fall back on this location if job_reference doesn't have one. Raises: `RuntimeError`: If the job is FAILED or the number of retries has been @@ -648,7 +650,9 @@ def wait_for_bq_job(self, job_reference, sleep_duration_sec=5, max_retries=0): while True: retry += 1 job = self.get_job( - job_reference.projectId, job_reference.jobId, job_reference.location) + job_reference.projectId, + job_reference.jobId, + job_reference.location or location) _LOGGER.info('Job %s status: %s', job.id, job.status.state) if job.status.state == 'DONE' and job.status.errorResult: raise RuntimeError( From adc3b2b4a5ffaeac50e73bce32027c85e8637cac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 22:37:13 -0700 Subject: [PATCH 34/78] Bump cloud.google.com/go/bigtable from 1.25.0 to 1.28.0 in /sdks (#32105) Bumps [cloud.google.com/go/bigtable](https://github.com/googleapis/google-cloud-go) from 1.25.0 to 1.28.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/documentai/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.25.0...pubsub/v1.28.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigtable dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 38 ++++++++++++------------ sdks/go.sum | 84 ++++++++++++++++++++++++++++------------------------- 2 files changed, 64 insertions(+), 58 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 654a456285431..5aeb14606c9d3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,12 +23,12 @@ module github.com/apache/beam/sdks/v2 go 1.21 require ( - cloud.google.com/go/bigquery v1.61.0 - cloud.google.com/go/bigtable v1.25.0 + cloud.google.com/go/bigquery v1.62.0 + cloud.google.com/go/bigtable v1.28.0 cloud.google.com/go/datastore v1.17.1 cloud.google.com/go/profiler v0.4.0 - cloud.google.com/go/pubsub v1.39.0 - cloud.google.com/go/spanner v1.63.0 + cloud.google.com/go/pubsub v1.40.0 + cloud.google.com/go/spanner v1.64.0 cloud.google.com/go/storage v1.43.0 github.com/aws/aws-sdk-go-v2 v1.30.3 github.com/aws/aws-sdk-go-v2/config v1.27.4 @@ -53,13 +53,13 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.16.0 - golang.org/x/net v0.26.0 + golang.org/x/net v0.27.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.22.0 golang.org/x/text v0.17.0 - google.golang.org/api v0.187.0 - google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d + google.golang.org/api v0.189.0 + google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.34.2 gopkg.in/yaml.v2 v2.4.0 @@ -74,8 +74,9 @@ require ( require ( cel.dev/expr v0.15.0 // indirect - cloud.google.com/go/auth v0.6.1 // indirect - cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect + cloud.google.com/go/auth v0.7.2 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.3 // indirect + cloud.google.com/go/monitoring v1.20.2 // indirect dario.cat/mergo v1.0.0 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect @@ -84,7 +85,7 @@ require ( github.com/containerd/errdefs v0.1.0 // indirect github.com/containerd/log v0.1.0 // indirect github.com/distribution/reference v0.6.0 // indirect - github.com/go-logr/logr v1.4.1 // indirect + github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/golang/protobuf v1.5.4 // indirect @@ -107,15 +108,16 @@ require ( go.opentelemetry.io/otel v1.24.0 // indirect go.opentelemetry.io/otel/metric v1.24.0 // indirect go.opentelemetry.io/otel/sdk v1.24.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.24.0 // indirect golang.org/x/time v0.5.0 // indirect ) require ( cloud.google.com/go v0.115.0 // indirect - cloud.google.com/go/compute/metadata v0.3.0 // indirect - cloud.google.com/go/iam v1.1.8 // indirect - cloud.google.com/go/longrunning v0.5.7 // indirect + cloud.google.com/go/compute/metadata v0.5.0 // indirect + cloud.google.com/go/iam v1.1.11 // indirect + cloud.google.com/go/longrunning v0.5.10 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.2 // indirect github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 // indirect @@ -154,7 +156,7 @@ require ( github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect - github.com/googleapis/gax-go/v2 v2.12.5 // indirect + github.com/googleapis/gax-go/v2 v2.13.0 // indirect github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect @@ -182,9 +184,9 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect golang.org/x/crypto v0.25.0 // indirect - golang.org/x/mod v0.17.0 // indirect - golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect + golang.org/x/mod v0.18.0 // indirect + golang.org/x/tools v0.22.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 1c03d5afc89cd..f0545e6c8e118 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -101,10 +101,10 @@ cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVo cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= -cloud.google.com/go/auth v0.6.1 h1:T0Zw1XM5c1GlpN2HYr2s+m3vr1p2wy+8VN+Z1FKxW38= -cloud.google.com/go/auth v0.6.1/go.mod h1:eFHG7zDzbXHKmjJddFG/rBlcGp6t25SwRUiEQSlO4x4= -cloud.google.com/go/auth/oauth2adapt v0.2.2 h1:+TTV8aXpjeChS9M+aTtN/TjdQnzJvmzKFt//oWu7HX4= -cloud.google.com/go/auth/oauth2adapt v0.2.2/go.mod h1:wcYjgpZI9+Yu7LyYBg4pqSiaRkfEK3GQcpb7C/uyF1Q= +cloud.google.com/go/auth v0.7.2 h1:uiha352VrCDMXg+yoBtaD0tUF4Kv9vrtrWPYXwutnDE= +cloud.google.com/go/auth v0.7.2/go.mod h1:VEc4p5NNxycWQTMQEDQF0bd6aTMb6VgYDXEwiJJQAbs= +cloud.google.com/go/auth/oauth2adapt v0.2.3 h1:MlxF+Pd3OmSudg/b1yZ5lJwoXCEaeedAguodky1PcKI= +cloud.google.com/go/auth/oauth2adapt v0.2.3/go.mod h1:tMQXOfZzFuNuUxOypHlQEXgdfX5cuhwU+ffUuXRJE8I= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= cloud.google.com/go/automl v1.6.0/go.mod h1:ugf8a6Fx+zP0D59WLhqgTDsQI9w07o64uf/Is3Nh5p8= cloud.google.com/go/automl v1.7.0/go.mod h1:RL9MYCCsJEOmt0Wf3z9uzG0a7adTT1fe+aObgSpkCt8= @@ -133,10 +133,10 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.61.0 h1:w2Goy9n6gh91LVi6B2Sc+HpBl8WbWhIyzdvVvrAuEIw= -cloud.google.com/go/bigquery v1.61.0/go.mod h1:PjZUje0IocbuTOdq4DBOJLNYB0WF3pAKBHzAYyxCwFo= -cloud.google.com/go/bigtable v1.25.0 h1:P3J0qFd2BUpvnamJOaTW9KkgqAiUXsFtFAW33sxj/hU= -cloud.google.com/go/bigtable v1.25.0/go.mod h1:NOwb5o8cw2LCEMP8SthXGxpZAjbQXc4Gb7V6A3TvsJc= +cloud.google.com/go/bigquery v1.62.0 h1:SYEA2f7fKqbSRRBHb7g0iHTtZvtPSPYdXfmqsjpsBwo= +cloud.google.com/go/bigquery v1.62.0/go.mod h1:5ee+ZkF1x/ntgCsFQJAQTM3QkAZOecfCmvxhkJsWRSA= +cloud.google.com/go/bigtable v1.28.0 h1:c0wc/wy+9Chj8BooqW/zgaeslXsA5YEYl84VBmvwp+4= +cloud.google.com/go/bigtable v1.28.0/go.mod h1:avmXcmxVbLJAo9moICRYMgDyTTPoV0MA0lHKnyqV4fQ= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= @@ -188,8 +188,8 @@ cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= -cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= +cloud.google.com/go/compute/metadata v0.5.0 h1:Zr0eK8JbFv6+Wi4ilXAR8FJ3wyNdpxHKJNPos6LTZOY= +cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= @@ -210,8 +210,8 @@ cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOX cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= -cloud.google.com/go/datacatalog v1.20.1 h1:czcba5mxwRM5V//jSadyig0y+8aOHmN7gUl9GbHu59E= -cloud.google.com/go/datacatalog v1.20.1/go.mod h1:Jzc2CoHudhuZhpv78UBAjMEg3w7I9jHA11SbRshWUjk= +cloud.google.com/go/datacatalog v1.20.4 h1:nUR7JBPZezl1+o+86N01VxAQQHY+It/D8tmNipcdVjI= +cloud.google.com/go/datacatalog v1.20.4/go.mod h1:71PDwywIYkNgSXdUU3H0mkTp3j15aahfYJ1CY3DogtU= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -327,8 +327,8 @@ cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGE cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= -cloud.google.com/go/iam v1.1.8 h1:r7umDwhj+BQyz0ScZMp4QrGXjSTI3ZINnpgU2nlB/K0= -cloud.google.com/go/iam v1.1.8/go.mod h1:GvE6lyMmfxXauzNq8NbgJbeVQNspG+tcdL/W8QO1+zE= +cloud.google.com/go/iam v1.1.11 h1:0mQ8UKSfdHLut6pH9FM3bI55KWR46ketn0PuXleDyxw= +cloud.google.com/go/iam v1.1.11/go.mod h1:biXoiLWYIKntto2joP+62sd9uW5EpkZmKIvfNcTWlnQ= cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= @@ -348,8 +348,8 @@ cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4 cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= -cloud.google.com/go/kms v1.18.0 h1:pqNdaVmZJFP+i8OVLocjfpdTWETTYa20FWOegSCdrRo= -cloud.google.com/go/kms v1.18.0/go.mod h1:DyRBeWD/pYBMeyiaXFa/DGNyxMDL3TslIKb8o/JkLkw= +cloud.google.com/go/kms v1.18.3 h1:8+Z2S4bQDSCdghB5ZA5dVDDJTLmnkRlowtFiXqMFd74= +cloud.google.com/go/kms v1.18.3/go.mod h1:y/Lcf6fyhbdn7MrG1VaDqXxM8rhOBc5rWcWAhcvZjQU= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -363,8 +363,8 @@ cloud.google.com/go/logging v1.7.0/go.mod h1:3xjP2CjkM3ZkO73aj4ASA5wRPGGCRrPIAeN cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= -cloud.google.com/go/longrunning v0.5.7 h1:WLbHekDbjK1fVFD3ibpFFVoyizlLRl73I7YKuAKilhU= -cloud.google.com/go/longrunning v0.5.7/go.mod h1:8GClkudohy1Fxm3owmBGid8W0pSgodEMwEAztp38Xng= +cloud.google.com/go/longrunning v0.5.10 h1:eB/BniENNRKhjz/xgiillrdcH3G74TGSl3BXinGlI7E= +cloud.google.com/go/longrunning v0.5.10/go.mod h1:tljz5guTr5oc/qhlUjBlk7UAIFMOGuPNxkNDZXlLics= cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= @@ -388,6 +388,8 @@ cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhI cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= +cloud.google.com/go/monitoring v1.20.2 h1:B/L+xrw9PYO7ywh37sgnjI/6dzEE+yQTAwfytDcpPto= +cloud.google.com/go/monitoring v1.20.2/go.mod h1:36rpg/7fdQ7NX5pG5x1FA7cXTVXusOp6Zg9r9e1+oek= cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= @@ -447,8 +449,8 @@ cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcd cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= -cloud.google.com/go/pubsub v1.39.0 h1:qt1+S6H+wwW8Q/YvDwM8lJnq+iIFgFEgaD/7h3lMsAI= -cloud.google.com/go/pubsub v1.39.0/go.mod h1:FrEnrSGU6L0Kh3iBaAbIUM8KMR7LqyEkMboVxGXCT+s= +cloud.google.com/go/pubsub v1.40.0 h1:0LdP+zj5XaPAGtWr2V6r88VXJlmtaB/+fde1q3TU8M0= +cloud.google.com/go/pubsub v1.40.0/go.mod h1:BVJI4sI2FyXp36KFKvFwcfDRDfR8MiLT8mMhmIhdAeA= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= @@ -538,8 +540,8 @@ cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+ cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= -cloud.google.com/go/spanner v1.63.0 h1:P6+BY70Wtol4MtryBgnXZVTZfsdySEvWfz0EpyLwHi4= -cloud.google.com/go/spanner v1.63.0/go.mod h1:iqDx7urZpgD7RekZ+CFvBRH6kVTW1ZSEb2HMDKOp5Cc= +cloud.google.com/go/spanner v1.64.0 h1:ltyPbHA/nRAtAhU/o742dXBCI1eNHPeaRY09Ja8B+hM= +cloud.google.com/go/spanner v1.64.0/go.mod h1:TOFx3pb2UwPsDGlE1gTehW+y6YlU4IFk+VdDHSGQS/M= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= @@ -827,8 +829,8 @@ github.com/go-gorp/gorp v2.2.0+incompatible/go.mod h1:7IfkAQnO7jfT/9IQ3R9wL1dFhu github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= -github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= @@ -966,8 +968,8 @@ github.com/googleapis/gax-go/v2 v2.5.1/go.mod h1:h6B0KMMFNtI2ddbGJn3T3ZbwkeT6yqE github.com/googleapis/gax-go/v2 v2.6.0/go.mod h1:1mjbznJAPHFpesgE5ucqfYEscaz5kMdcIDwU/6+DDoY= github.com/googleapis/gax-go/v2 v2.7.0/go.mod h1:TEop28CZZQ2y+c0VxMUmu1lV+fQx57QpBWsYpwqHJx8= github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= -github.com/googleapis/gax-go/v2 v2.12.5 h1:8gw9KZK8TiVKB6q3zHY3SBzLnrGp6HQjyfYBYGmXdxA= -github.com/googleapis/gax-go/v2 v2.12.5/go.mod h1:BUDKcWo+RaKq5SC9vVYL0wLADa3VcfswbOMMRmB9H3E= +github.com/googleapis/gax-go/v2 v2.13.0 h1:yitjD5f7jQHhyDsnhKEBU52NdvvdSeGzlAnDPT0hH1s= +github.com/googleapis/gax-go/v2 v2.13.0/go.mod h1:Z/fvTZXF8/uw7Xu5GuslPw+bplx6SS338j1Is2S+B7A= github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/gorilla/handlers v1.5.2 h1:cLTUSsNkgcwhgRqvCNmdbRWG0A3N4F+M2nWKdScwyEE= @@ -1232,6 +1234,8 @@ go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGX go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35HoYiQWGDFg= +go.opentelemetry.io/otel/sdk/metric v1.24.0 h1:yyMQrPzF+k88/DbH7o4FMAs80puqd+9osbiBrJrz/w8= +go.opentelemetry.io/otel/sdk/metric v1.24.0/go.mod h1:I6Y5FjH6rvEnTTAYQz3Mmv2kl6Ek5IIrmwTLqMrrOE0= go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= @@ -1312,8 +1316,8 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= -golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.18.0 h1:5+9lSbEzPSdWkH32vYPBwEpX8KwDbM52Ud9xBUvNlb0= +golang.org/x/mod v0.18.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1373,8 +1377,8 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= -golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= -golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= +golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= +golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1613,8 +1617,8 @@ golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= -golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= -golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= +golang.org/x/tools v0.22.0 h1:gqSGLZqv+AI9lIQzniJ0nZDRG5GBPsSi+DRNHWNz6yA= +golang.org/x/tools v0.22.0/go.mod h1:aCwcsjqvq7Yqt6TNyX7QMU2enbQ/Gt0bo6krSeEri+c= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1692,8 +1696,8 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.187.0 h1:Mxs7VATVC2v7CY+7Xwm4ndkX71hpElcvx0D1Ji/p1eo= -google.golang.org/api v0.187.0/go.mod h1:KIHlTc4x7N7gKKuVsdmfBXN13yEEWXWFURWY6SBp2gk= +google.golang.org/api v0.189.0 h1:equMo30LypAkdkLMBqfeIqtyAnlyig1JSZArl4XPwdI= +google.golang.org/api v0.189.0/go.mod h1:FLWGJKb0hb+pU2j+rJqwbnsF+ym+fQs73rbJ+KAUgy8= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1833,12 +1837,12 @@ google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= -google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d h1:PksQg4dV6Sem3/HkBX+Ltq8T0ke0PKIRBNBatoDTVls= -google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d/go.mod h1:s7iA721uChleev562UJO2OYB0PPT9CMFjV+Ce7VJH5M= -google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 h1:MuYw1wJzT+ZkybKfaOXKp5hJiZDn2iHaXRw0mRYdHSc= -google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4/go.mod h1:px9SlOOZBg1wM1zdnr8jEL4CNGUBZ+ZKYtNPApNQc4c= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d h1:k3zyW3BYYR30e8v3x0bTDdE9vpYFjZHK+HcyqkrppWk= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f h1:htT2I9bZvGm+110zq8bIErMX+WgBWxCzV3ChwbvnKnc= +google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f/go.mod h1:Sk3mLpoDFTAp6R4OvlcUgaG4ISTspKeFsIAXMn9Bm4Y= +google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade h1:WxZOF2yayUHpHSbUE6NMzumUzBxYc3YGwo0YHnbzsJY= +google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade h1:oCRSWfwGXQsqlVdErcyTt4A93Y8fo0/9D4b1gnI++qo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 1f09065ff325763464dfd618c4175a903f68301a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 8 Aug 2024 08:12:31 -0400 Subject: [PATCH 35/78] Fix classifier dropped in artifact pom.xml (#32100) --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index e603e49f842fe..ee116423e4b00 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1978,8 +1978,8 @@ class BeamModulePlugin implements Plugin { def dependencyNode = dependenciesNode.appendNode('dependency') def appendClassifier = { dep -> dep.artifacts.each { art -> - if (art.hasProperty('archiveClassifier')) { - dependencyNode.appendNode('archiveClassifier', art.archiveClassifier) + if (art.hasProperty('classifier')) { + dependencyNode.appendNode('classifier', art.classifier) } } } From 502c728dd23e8b93691c87bd3b597d017782b418 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 8 Aug 2024 10:29:28 -0400 Subject: [PATCH 36/78] Update cython requirement from <1 to <4 in /sdks/python (#32087) Updates the requirements on [cython](https://github.com/cython/cython) to permit the latest version. - [Release notes](https://github.com/cython/cython/releases) - [Changelog](https://github.com/cython/cython/blob/master/CHANGES.rst) - [Commits](https://github.com/cython/cython/compare/0.9.6.14...3.0.11) --- updated-dependencies: - dependency-name: cython dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/base_image_requirements_manual.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt index 5bc60c474b4fe..ab5dcf30aa50e 100644 --- a/sdks/python/container/base_image_requirements_manual.txt +++ b/sdks/python/container/base_image_requirements_manual.txt @@ -27,7 +27,7 @@ bs4 # Commonly used HTML processing tool. # Don't upgrade to Cython 3.x, until it's released, stable and we have consensus # to upgrade. Use 0.xx for now. -cython<1 +cython<4 # future is no longer a Beam dependency, but is an implicit dependency in # some versions of libraries that launch Beam pipelines, like tensorflow-transform. # Leaving 'future' in our containers for now prevent breaking tft users. From 529996241be1bd4482a529f88b742eeee9867daa Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:22:07 -0400 Subject: [PATCH 37/78] Bump go.mongodb.org/mongo-driver from 1.16.0 to 1.16.1 in /sdks (#32104) Bumps [go.mongodb.org/mongo-driver](https://github.com/mongodb/mongo-go-driver) from 1.16.0 to 1.16.1. - [Release notes](https://github.com/mongodb/mongo-go-driver/releases) - [Commits](https://github.com/mongodb/mongo-go-driver/compare/v1.16.0...v1.16.1) --- updated-dependencies: - dependency-name: go.mongodb.org/mongo-driver dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5aeb14606c9d3..397820f94cb77 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,7 +52,7 @@ require ( github.com/tetratelabs/wazero v1.7.3 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c - go.mongodb.org/mongo-driver v1.16.0 + go.mongodb.org/mongo-driver v1.16.1 golang.org/x/net v0.27.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.8.0 diff --git a/sdks/go.sum b/sdks/go.sum index f0545e6c8e118..05de032d212ab 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1209,8 +1209,8 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD go.einride.tech/aip v0.67.1 h1:d/4TW92OxXBngkSOwWS2CH5rez869KpKMaN44mdxkFI= go.einride.tech/aip v0.67.1/go.mod h1:ZGX4/zKw8dcgzdLsrvpOOGxfxI2QSk12SlP7d6c0/XI= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.mongodb.org/mongo-driver v1.16.0 h1:tpRsfBJMROVHKpdGyc1BBEzzjDUWjItxbVSZ8Ls4BQ4= -go.mongodb.org/mongo-driver v1.16.0/go.mod h1:oB6AhJQvFQL4LEHyXi6aJzQJtBiTQHiAd83l0GdFaiw= +go.mongodb.org/mongo-driver v1.16.1 h1:rIVLL3q0IHM39dvE+z2ulZLp9ENZKThVfuvN/IiN4l8= +go.mongodb.org/mongo-driver v1.16.1/go.mod h1:oB6AhJQvFQL4LEHyXi6aJzQJtBiTQHiAd83l0GdFaiw= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= From 16d95835df1b367892ca1e6895306af29d4b81c5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:22:25 -0400 Subject: [PATCH 38/78] Bump github.com/proullon/ramsql from 0.1.3 to 0.1.4 in /sdks (#32106) Bumps [github.com/proullon/ramsql](https://github.com/proullon/ramsql) from 0.1.3 to 0.1.4. - [Release notes](https://github.com/proullon/ramsql/releases) - [Commits](https://github.com/proullon/ramsql/compare/v0.1.3...v0.1.4) --- updated-dependencies: - dependency-name: github.com/proullon/ramsql dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 397820f94cb77..26c1d9d376902 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -46,7 +46,7 @@ require ( github.com/linkedin/goavro/v2 v2.13.0 github.com/nats-io/nats-server/v2 v2.10.18 github.com/nats-io/nats.go v1.36.0 - github.com/proullon/ramsql v0.1.3 + github.com/proullon/ramsql v0.1.4 github.com/spf13/cobra v1.8.1 github.com/testcontainers/testcontainers-go v0.32.0 github.com/tetratelabs/wazero v1.7.3 diff --git a/sdks/go.sum b/sdks/go.sum index 05de032d212ab..1f57de7507cd8 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1117,8 +1117,8 @@ github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:Om github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= -github.com/proullon/ramsql v0.1.3 h1:/LRcXJf4lEmhdb4tYcci473I2VynjcZSzh2hsjJ8rSk= -github.com/proullon/ramsql v0.1.3/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= +github.com/proullon/ramsql v0.1.4 h1:yTFRTn46gFH/kPbzCx+mGjuFlyTBUeDr3h2ldwxddl0= +github.com/proullon/ramsql v0.1.4/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= From 18849de0cc4e6aff82af6ed897fa7baec9deb084 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 8 Aug 2024 17:46:13 +0200 Subject: [PATCH 39/78] Revert "Update cython requirement from <1 to <4 in /sdks/python (#32087)" (#32110) This reverts commit 502c728dd23e8b93691c87bd3b597d017782b418. --- sdks/python/container/base_image_requirements_manual.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt index ab5dcf30aa50e..5bc60c474b4fe 100644 --- a/sdks/python/container/base_image_requirements_manual.txt +++ b/sdks/python/container/base_image_requirements_manual.txt @@ -27,7 +27,7 @@ bs4 # Commonly used HTML processing tool. # Don't upgrade to Cython 3.x, until it's released, stable and we have consensus # to upgrade. Use 0.xx for now. -cython<4 +cython<1 # future is no longer a Beam dependency, but is an implicit dependency in # some versions of libraries that launch Beam pipelines, like tensorflow-transform. # Leaving 'future' in our containers for now prevent breaking tft users. From a6de47572b927c8a1c3fdaf11f15b6d02473c3e8 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 8 Aug 2024 10:08:21 -0700 Subject: [PATCH 40/78] [Go SDK] s3 filesystem: Fix nillable content length, update deps. (#32111) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go.mod | 14 +++++++------- sdks/go.sum | 14 ++++++++++++++ sdks/go/pkg/beam/io/filesystem/s3/s3.go | 10 +++++++--- 3 files changed, 28 insertions(+), 10 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 26c1d9d376902..bd25b3beab5aa 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,10 +31,10 @@ require ( cloud.google.com/go/spanner v1.64.0 cloud.google.com/go/storage v1.43.0 github.com/aws/aws-sdk-go-v2 v1.30.3 - github.com/aws/aws-sdk-go-v2/config v1.27.4 + github.com/aws/aws-sdk-go-v2/config v1.27.27 github.com/aws/aws-sdk-go-v2/credentials v1.17.27 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 - github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.10 + github.com/aws/aws-sdk-go-v2/service/s3 v1.58.3 github.com/aws/smithy-go v1.20.3 github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 @@ -123,16 +123,16 @@ require ( github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 // indirect github.com/apache/thrift v0.17.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 1f57de7507cd8..a50c8ce9230c9 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -683,9 +683,13 @@ github.com/aws/aws-sdk-go-v2 v1.30.3 h1:jUeBtG0Ih+ZIFH0F4UkmL9w3cSpaMv9tYYDbzILP github.com/aws/aws-sdk-go-v2 v1.30.3/go.mod h1:nIQjQVp5sfpQcTc9mPSr1B0PaWK5ByX9MOoDadSN4lc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3 h1:tW1/Rkad38LA15X4UQtjXZXNKsCgkshC3EbmcUmghTg= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3/go.mod h1:UbnqO+zjqk3uIt9yCACHJ9IVNhyhOCnYk8yA19SAWrM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= github.com/aws/aws-sdk-go-v2/config v1.27.4 h1:AhfWb5ZwimdsYTgP7Od8E9L1u4sKmDW2ZVeLcf2O42M= github.com/aws/aws-sdk-go-v2/config v1.27.4/go.mod h1:zq2FFXK3A416kiukwpsd+rD4ny6JC7QSkp4QdN1Mp2g= +github.com/aws/aws-sdk-go-v2/config v1.27.27 h1:HdqgGt1OAP0HkEDDShEl0oSYa9ZZBSOmKpdpsDMdO90= +github.com/aws/aws-sdk-go-v2/config v1.27.27/go.mod h1:MVYamCg76dFNINkZFu4n4RjDixhVr51HLj4ErWzrVwg= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= github.com/aws/aws-sdk-go-v2/credentials v1.17.27 h1:2raNba6gr2IfA0eqqiP2XiQ0UVOpGPgDSi0I9iAP+UI= github.com/aws/aws-sdk-go-v2/credentials v1.17.27/go.mod h1:gniiwbGahQByxan6YjQUMcW4Aov6bLC3m+evgcoN4r4= @@ -695,6 +699,8 @@ github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvH github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.10 h1:zeN9UtUlA6FTx0vFSayxSX32HDw73Yb6Hh2izDSFxXY= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.10/go.mod h1:3HKuexPDcwLWPaqpW2UR/9n8N/u/3CKcGAzSs8p8u8g= github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 h1:SoNJ4RlFEQEbtDcCEt+QG56MY4fm4W8rYirAmq+/DdU= github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15/go.mod h1:U9ke74k1n2bf+RIgoX1SXFed1HLs51OgUSs+Ph0KJP8= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 h1:C6WHdGnTDIYETAm5iErQUiVNsclNx9qbJVPIt03B6bI= @@ -704,20 +710,28 @@ github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7 github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15 h1:Z5r7SycxmSllHYmaAZPpmN8GviDrSGhMS6bldqtXZPw= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15/go.mod h1:CetW7bDE00QoGEmPUoZuRog07SGVAUVW6LFpNP0YfIg= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 h1:dT3MqvGhSoaIhRseqw2I0yH81l7wiR2vjs57O51EAm8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3/go.mod h1:GlAeCkHwugxdHaueRr4nhPuY+WW+gR8UjlcqzPr1SPI= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17 h1:YPYe6ZmvUfDDDELqEKtAd6bo8zxhkm+XEFEzQisqUIE= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17/go.mod h1:oBtcnYua/CgzCWYN7NZ5j7PotFDaFSUjCYVTtfyn7vw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 h1:HGErhhrxZlQ044RiM+WdoZxp0p+EGM62y3L6pwA4olE= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17/go.mod h1:RkZEx4l0EHYDJpWppMJ3nD9wZJAa8/0lq9aVC+r2UII= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15 h1:246A4lSTXWJw/rmlQI+TT2OcqeDMKBdyjEQrafMaQdA= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15/go.mod h1:haVfg3761/WF7YPuJOER2MP0k4UAXyHaLclKXB6usDg= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= +github.com/aws/aws-sdk-go-v2/service/s3 v1.58.3 h1:hT8ZAZRIfqBqHbzKTII+CIiY8G2oC9OpLedkZ51DWl8= +github.com/aws/aws-sdk-go-v2/service/s3 v1.58.3/go.mod h1:Lcxzg5rojyVPU/0eFwLtcyTaek/6Mtic5B1gJo7e/zE= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 h1:BXx0ZIxvrJdSgSvKTZ+yRBeSqqgPM89VPlulEcl37tM= github.com/aws/aws-sdk-go-v2/service/sso v1.22.4/go.mod h1:ooyCOXjvJEsUw7x+ZDHeISPMhtwI3ZCB7ggFMcFfWLU= diff --git a/sdks/go/pkg/beam/io/filesystem/s3/s3.go b/sdks/go/pkg/beam/io/filesystem/s3/s3.go index 97a2c9aada14a..40fde0a300b28 100644 --- a/sdks/go/pkg/beam/io/filesystem/s3/s3.go +++ b/sdks/go/pkg/beam/io/filesystem/s3/s3.go @@ -149,7 +149,7 @@ func (f *fs) OpenWrite(ctx context.Context, filename string) (io.WriteCloser, er func (f *fs) Size(ctx context.Context, filename string) (int64, error) { bucket, key, err := parseURI(filename) if err != nil { - return -1, fmt.Errorf("error parsing S3 uri %s: %v", filename, err) + return -1, fmt.Errorf("error parsing S3 uri %s: %w", filename, err) } params := &s3.HeadObjectInput{ @@ -158,10 +158,14 @@ func (f *fs) Size(ctx context.Context, filename string) (int64, error) { } output, err := f.client.HeadObject(ctx, params) if err != nil { - return -1, fmt.Errorf("error getting metadata for object %s: %v", filename, err) + return -1, fmt.Errorf("error getting metadata for object %s: %w", filename, err) } - return output.ContentLength, err + if output.ContentLength != nil { + return *output.ContentLength, nil + } + + return -1, fmt.Errorf("content length for object %s was nil", filename) } // LastModified returns the time at which the file was last modified. From 679e9d799e5be7d906edd9e6d59aeefc5b755257 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 8 Aug 2024 09:55:23 -0700 Subject: [PATCH 41/78] Upgrade Beam to use Cython 3. Many files, including all of those that would be likely to have issues with string/bytes, were already setting language level to 3. Taking a pass through the compiled files, I did not see any incompatibilities. Tests seem to be fine as well. --- sdks/python/container/base_image_requirements_manual.txt | 4 +--- sdks/python/pyproject.toml | 2 +- sdks/python/setup.py | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt index 5bc60c474b4fe..2d99a55e564ea 100644 --- a/sdks/python/container/base_image_requirements_manual.txt +++ b/sdks/python/container/base_image_requirements_manual.txt @@ -25,9 +25,7 @@ # Consider constraining requirements of Beam itself when necessary. bs4 # Commonly used HTML processing tool. -# Don't upgrade to Cython 3.x, until it's released, stable and we have consensus -# to upgrade. Use 0.xx for now. -cython<1 +cython>=3,<4 # future is no longer a Beam dependency, but is an implicit dependency in # some versions of libraries that launch Beam pipelines, like tensorflow-transform. # Leaving 'future' in our containers for now prevent breaking tft users. diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index 32924a9297750..037e5a8aed6bb 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -28,7 +28,7 @@ requires = [ # Numpy headers "numpy>=1.14.3,<1.27", # Update setup.py as well. # having cython here will create wheels that are platform dependent. - "cython==0.29.36", + "cython>=3.0,<4", ## deps for generating external transform wrappers: # also update PyYaml bounds in sdks:python:generateExternalTransformsConfig 'pyyaml>=3.12,<7.0.0', diff --git a/sdks/python/setup.py b/sdks/python/setup.py index c9b2d087d04ca..5f631e3dfdab3 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -113,7 +113,7 @@ def get_version(): # `pipenv` package managers. pass -REQUIRED_CYTHON_VERSION = '0.28.1' +REQUIRED_CYTHON_VERSION = '3.0.0' try: _CYTHON_VERSION = distribution('cython').version if parse_version(_CYTHON_VERSION) < parse_version(REQUIRED_CYTHON_VERSION): From c825434965ee41f13b87ac37b99731c00c142bde Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 8 Aug 2024 10:19:52 -0700 Subject: [PATCH 42/78] Update base image requirements. Not running script due to pre-exising issues. --- sdks/python/container/py310/base_image_requirements.txt | 2 +- sdks/python/container/py311/base_image_requirements.txt | 2 +- sdks/python/container/py312/base_image_requirements.txt | 2 +- sdks/python/container/py38/base_image_requirements.txt | 2 +- sdks/python/container/py39/base_image_requirements.txt | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index e2806270e5443..35eea227888bf 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -36,7 +36,7 @@ cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 cryptography==42.0.5 -Cython==0.29.37 +Cython==3.0.10 Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 1cba006ad32cc..d6d523689fa77 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -35,7 +35,7 @@ cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 cryptography==42.0.5 -Cython==0.29.37 +Cython==3.0.10 Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 4a6147b573b85..b7a5f3687166b 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -35,7 +35,7 @@ cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 cryptography==42.0.7 -Cython==0.29.37 +Cython==3.0.10 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index f88dba103469b..c92761473362b 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -37,7 +37,7 @@ cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 cryptography==42.0.5 -Cython==0.29.37 +Cython==3.0.10 Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 39b888dd6ab75..bab94181499c4 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -36,7 +36,7 @@ cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 cryptography==42.0.5 -Cython==0.29.37 +Cython==3.0.10 Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 From 2b7e84239eab55347346859bbb78741684a5a6bf Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 8 Aug 2024 10:20:22 -0700 Subject: [PATCH 43/78] Remove now unneeded langauge level specifications. --- sdks/python/apache_beam/coders/coder_impl.py | 2 -- sdks/python/apache_beam/coders/stream.pyx | 2 -- sdks/python/apache_beam/metrics/cells.py | 2 -- sdks/python/apache_beam/metrics/execution.py | 2 -- sdks/python/apache_beam/metrics/monitoring_infos.py | 2 -- sdks/python/apache_beam/runners/common.py | 1 - sdks/python/apache_beam/runners/worker/logger.py | 2 -- sdks/python/apache_beam/runners/worker/opcounters.py | 2 -- sdks/python/apache_beam/runners/worker/operations.py | 2 -- sdks/python/apache_beam/runners/worker/statesampler_fast.pyx | 2 -- sdks/python/apache_beam/testing/fast_test_utils.pyx | 2 -- sdks/python/apache_beam/transforms/cy_combiners.py | 2 -- .../apache_beam/transforms/cy_dataflow_distribution_counter.pyx | 2 -- sdks/python/apache_beam/transforms/stats.py | 2 -- sdks/python/apache_beam/utils/counters.py | 1 - sdks/python/apache_beam/utils/windowed_value.py | 2 -- 16 files changed, 30 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index e44c2535156e5..ff5fb5bef7ac9 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """Coder implementations. The actual encode/decode implementations are split off from coders to diff --git a/sdks/python/apache_beam/coders/stream.pyx b/sdks/python/apache_beam/coders/stream.pyx index 8f941c151bde7..3977660f68b06 100644 --- a/sdks/python/apache_beam/coders/stream.pyx +++ b/sdks/python/apache_beam/coders/stream.pyx @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """Compiled version of the Stream objects used by CoderImpl. For internal use only; no backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py index 53b6fc8495920..3bfbfc6b2e773 100644 --- a/sdks/python/apache_beam/metrics/cells.py +++ b/sdks/python/apache_beam/metrics/cells.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """ This file contains metric cell classes. A metric cell is used to accumulate in-memory changes to a metric. It represents a specific metric in a single diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py index 4202f7996c7fd..570062371cae6 100644 --- a/sdks/python/apache_beam/metrics/execution.py +++ b/sdks/python/apache_beam/metrics/execution.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """ This module is for internal use only; no backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/metrics/monitoring_infos.py b/sdks/python/apache_beam/metrics/monitoring_infos.py index 7bc7cced280c1..0e638c9eb4fe7 100644 --- a/sdks/python/apache_beam/metrics/monitoring_infos.py +++ b/sdks/python/apache_beam/metrics/monitoring_infos.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - # pytype: skip-file import collections diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py index 40a3341e2b4c3..ba4dd98c7a937 100644 --- a/sdks/python/apache_beam/runners/common.py +++ b/sdks/python/apache_beam/runners/common.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -# cython: language_level=3 """Worker operations executor. diff --git a/sdks/python/apache_beam/runners/worker/logger.py b/sdks/python/apache_beam/runners/worker/logger.py index 1efebeb3c78c0..06e2508fb7d29 100644 --- a/sdks/python/apache_beam/runners/worker/logger.py +++ b/sdks/python/apache_beam/runners/worker/logger.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """Python worker logging.""" # pytype: skip-file diff --git a/sdks/python/apache_beam/runners/worker/opcounters.py b/sdks/python/apache_beam/runners/worker/opcounters.py index ba53cbcbce7f5..51ca4cf0545b7 100644 --- a/sdks/python/apache_beam/runners/worker/opcounters.py +++ b/sdks/python/apache_beam/runners/worker/opcounters.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """Counters collect the progress of the Worker for reporting to the service.""" # pytype: skip-file diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py index 00a652c49e669..58c807c28dbd4 100644 --- a/sdks/python/apache_beam/runners/worker/operations.py +++ b/sdks/python/apache_beam/runners/worker/operations.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """Worker operations executor.""" # pytype: skip-file diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx index 7c082b7a62269..d02d05c3af942 100644 --- a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx +++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """State sampler for tracking time spent in execution steps. The state sampler profiles the time spent in each step of a pipeline. diff --git a/sdks/python/apache_beam/testing/fast_test_utils.pyx b/sdks/python/apache_beam/testing/fast_test_utils.pyx index d78a5b773186f..d815bcfe8f966 100644 --- a/sdks/python/apache_beam/testing/fast_test_utils.pyx +++ b/sdks/python/apache_beam/testing/fast_test_utils.pyx @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - cimport libc.stdint cimport libc.stdlib cimport libc.string diff --git a/sdks/python/apache_beam/transforms/cy_combiners.py b/sdks/python/apache_beam/transforms/cy_combiners.py index 2267d02c1908c..b5cc7493a29a3 100644 --- a/sdks/python/apache_beam/transforms/cy_combiners.py +++ b/sdks/python/apache_beam/transforms/cy_combiners.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """A library of basic cythonized CombineFn subclasses. For internal use only; no backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx b/sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx index c1b32356ed1ff..c117ae32f4e5f 100644 --- a/sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx +++ b/sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx @@ -14,8 +14,6 @@ # limitations under the License. # -# cython: language_level=3 - """ For internal use only. No backwards compatibility guarantees.""" cimport cython diff --git a/sdks/python/apache_beam/transforms/stats.py b/sdks/python/apache_beam/transforms/stats.py index 2599760f8d5be..d389463e55a26 100644 --- a/sdks/python/apache_beam/transforms/stats.py +++ b/sdks/python/apache_beam/transforms/stats.py @@ -15,8 +15,6 @@ # limitations under the License. # -# cython: language_level=3 - """This module has all statistic related transforms. This ApproximateUnique class will be deprecated [1]. PLease look into using diff --git a/sdks/python/apache_beam/utils/counters.py b/sdks/python/apache_beam/utils/counters.py index 214fa433de1c4..57d73fa283ebf 100644 --- a/sdks/python/apache_beam/utils/counters.py +++ b/sdks/python/apache_beam/utils/counters.py @@ -17,7 +17,6 @@ # cython: profile=False # cython: overflowcheck=True -# cython: language_level=3 """Counters collect the progress of the Worker for reporting to the service. diff --git a/sdks/python/apache_beam/utils/windowed_value.py b/sdks/python/apache_beam/utils/windowed_value.py index fb15d3778a6a0..f6232ce2f6b0d 100644 --- a/sdks/python/apache_beam/utils/windowed_value.py +++ b/sdks/python/apache_beam/utils/windowed_value.py @@ -22,8 +22,6 @@ # editing this file as WindowedValues are created for every element for # every step in a Beam pipeline. -# cython: language_level=3 - # pytype: skip-file import collections From 1de0c4670eefc02a35c3ffa7d1e25bf7f69744ce Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 8 Aug 2024 15:11:15 -0700 Subject: [PATCH 44/78] Add no-except to time-critical function. --- sdks/python/apache_beam/runners/worker/statesampler_fast.pyx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx index d02d05c3af942..45700a0b0f81e 100644 --- a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx +++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx @@ -57,7 +57,7 @@ cdef extern from "crossplatform_time.h" nogil: long tv_nsec # nanoseconds int clock_gettime(int clock_id, timespec *result) -cdef inline int64_t get_nsec_time() nogil: +cdef inline int64_t get_nsec_time() noexcept nogil: """Get current time as microseconds since Unix epoch.""" cdef timespec current_time # First argument value of 0 corresponds to CLOCK_REALTIME. From 6bdf63a9d2deb00d6b33704c8a8416cf6783f4b6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 8 Aug 2024 22:44:50 -0700 Subject: [PATCH 45/78] Bump cloud.google.com/go/spanner from 1.64.0 to 1.66.0 in /sdks (#32126) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.64.0 to 1.66.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.64.0...spanner/v1.66.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 18 ++---------------- 2 files changed, 3 insertions(+), 17 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index bd25b3beab5aa..1f53410c5e042 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.17.1 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.40.0 - cloud.google.com/go/spanner v1.64.0 + cloud.google.com/go/spanner v1.66.0 cloud.google.com/go/storage v1.43.0 github.com/aws/aws-sdk-go-v2 v1.30.3 github.com/aws/aws-sdk-go-v2/config v1.27.27 diff --git a/sdks/go.sum b/sdks/go.sum index a50c8ce9230c9..8a5072a744094 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -540,8 +540,8 @@ cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+ cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= -cloud.google.com/go/spanner v1.64.0 h1:ltyPbHA/nRAtAhU/o742dXBCI1eNHPeaRY09Ja8B+hM= -cloud.google.com/go/spanner v1.64.0/go.mod h1:TOFx3pb2UwPsDGlE1gTehW+y6YlU4IFk+VdDHSGQS/M= +cloud.google.com/go/spanner v1.66.0 h1:PF1upR8n+DVUO9mUpCc1j5kyHn1Xfq0A53ZrnM0AmeU= +cloud.google.com/go/spanner v1.66.0/go.mod h1:gu+weqqrnoBsVlxOmMG5pzDZ2nkpqqJx4MsnmIacH5w= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= @@ -681,13 +681,9 @@ github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= github.com/aws/aws-sdk-go-v2 v1.30.3 h1:jUeBtG0Ih+ZIFH0F4UkmL9w3cSpaMv9tYYDbzILP8dY= github.com/aws/aws-sdk-go-v2 v1.30.3/go.mod h1:nIQjQVp5sfpQcTc9mPSr1B0PaWK5ByX9MOoDadSN4lc= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3 h1:tW1/Rkad38LA15X4UQtjXZXNKsCgkshC3EbmcUmghTg= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3/go.mod h1:UbnqO+zjqk3uIt9yCACHJ9IVNhyhOCnYk8yA19SAWrM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.27.4 h1:AhfWb5ZwimdsYTgP7Od8E9L1u4sKmDW2ZVeLcf2O42M= -github.com/aws/aws-sdk-go-v2/config v1.27.4/go.mod h1:zq2FFXK3A416kiukwpsd+rD4ny6JC7QSkp4QdN1Mp2g= github.com/aws/aws-sdk-go-v2/config v1.27.27 h1:HdqgGt1OAP0HkEDDShEl0oSYa9ZZBSOmKpdpsDMdO90= github.com/aws/aws-sdk-go-v2/config v1.27.27/go.mod h1:MVYamCg76dFNINkZFu4n4RjDixhVr51HLj4ErWzrVwg= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= @@ -697,8 +693,6 @@ github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDu github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 h1:KreluoV8FZDEtI6Co2xuNk/UqI9iwMrOx/87PBNIKqw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvHE0Tjvn7kbxaUhl75CJi1sbfhMxkU= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.10 h1:zeN9UtUlA6FTx0vFSayxSX32HDw73Yb6Hh2izDSFxXY= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.10/go.mod h1:3HKuexPDcwLWPaqpW2UR/9n8N/u/3CKcGAzSs8p8u8g= github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 h1:SoNJ4RlFEQEbtDcCEt+QG56MY4fm4W8rYirAmq+/DdU= @@ -708,28 +702,20 @@ github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15/go.mod h1:ZQLZqhcu+Jh github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15 h1:Z5r7SycxmSllHYmaAZPpmN8GviDrSGhMS6bldqtXZPw= github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15/go.mod h1:CetW7bDE00QoGEmPUoZuRog07SGVAUVW6LFpNP0YfIg= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 h1:dT3MqvGhSoaIhRseqw2I0yH81l7wiR2vjs57O51EAm8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3/go.mod h1:GlAeCkHwugxdHaueRr4nhPuY+WW+gR8UjlcqzPr1SPI= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17 h1:YPYe6ZmvUfDDDELqEKtAd6bo8zxhkm+XEFEzQisqUIE= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17/go.mod h1:oBtcnYua/CgzCWYN7NZ5j7PotFDaFSUjCYVTtfyn7vw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 h1:HGErhhrxZlQ044RiM+WdoZxp0p+EGM62y3L6pwA4olE= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17/go.mod h1:RkZEx4l0EHYDJpWppMJ3nD9wZJAa8/0lq9aVC+r2UII= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15 h1:246A4lSTXWJw/rmlQI+TT2OcqeDMKBdyjEQrafMaQdA= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15/go.mod h1:haVfg3761/WF7YPuJOER2MP0k4UAXyHaLclKXB6usDg= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= -github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/s3 v1.58.3 h1:hT8ZAZRIfqBqHbzKTII+CIiY8G2oC9OpLedkZ51DWl8= github.com/aws/aws-sdk-go-v2/service/s3 v1.58.3/go.mod h1:Lcxzg5rojyVPU/0eFwLtcyTaek/6Mtic5B1gJo7e/zE= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= From af6bf8a1423eef082a289e9a1cfcfe0bf2d8c0ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 8 Aug 2024 23:11:22 -0700 Subject: [PATCH 46/78] Bump golang.org/x/net from 0.27.0 to 0.28.0 in /sdks (#32128) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.27.0 to 0.28.0. - [Commits](https://github.com/golang/net/compare/v0.27.0...v0.28.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 1f53410c5e042..8849f9732afd3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,10 +53,10 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.16.1 - golang.org/x/net v0.27.0 + golang.org/x/net v0.28.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.8.0 - golang.org/x/sys v0.22.0 + golang.org/x/sys v0.23.0 golang.org/x/text v0.17.0 google.golang.org/api v0.189.0 google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f @@ -183,7 +183,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.25.0 // indirect + golang.org/x/crypto v0.26.0 // indirect golang.org/x/mod v0.18.0 // indirect golang.org/x/tools v0.22.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 8a5072a744094..90fb958b1df68 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1255,8 +1255,8 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.25.0 h1:ypSNr+bnYL2YhwoMt2zPxHFmbAN1KZs/njMG3hxUp30= -golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= +golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= +golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1377,8 +1377,8 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1515,8 +1515,8 @@ golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.23.0 h1:YfKFowiIMvtgl1UERQoTPPToxltDeZfbj4H7dVUCwmM= +golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= @@ -1525,8 +1525,8 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= -golang.org/x/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= -golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= +golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= +golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 27741fb6c9b6fc109a94bb91fa4a665473f50268 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 9 Aug 2024 05:11:41 -0700 Subject: [PATCH 47/78] Bump cloud.google.com/go/profiler from 0.4.0 to 0.4.1 in /sdks (#32125) Bumps [cloud.google.com/go/profiler](https://github.com/googleapis/google-cloud-go) from 0.4.0 to 0.4.1. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/v0.4.0...ai/v0.4.1) --- updated-dependencies: - dependency-name: cloud.google.com/go/profiler dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 8849f9732afd3..8c7a52ca951c7 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -26,7 +26,7 @@ require ( cloud.google.com/go/bigquery v1.62.0 cloud.google.com/go/bigtable v1.28.0 cloud.google.com/go/datastore v1.17.1 - cloud.google.com/go/profiler v0.4.0 + cloud.google.com/go/profiler v0.4.1 cloud.google.com/go/pubsub v1.40.0 cloud.google.com/go/spanner v1.66.0 cloud.google.com/go/storage v1.43.0 @@ -152,7 +152,7 @@ require ( github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/flatbuffers v23.5.26+incompatible // indirect - github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 // indirect + github.com/google/pprof v0.0.0-20240528025155-186aa0362fba // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 90fb958b1df68..e0039f28e43ba 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -439,8 +439,8 @@ cloud.google.com/go/privatecatalog v0.5.0/go.mod h1:XgosMUvvPyxDjAVNDYxJ7wBW8//h cloud.google.com/go/privatecatalog v0.6.0/go.mod h1:i/fbkZR0hLN29eEWiiwue8Pb+GforiEIBnV9yrRUOKI= cloud.google.com/go/privatecatalog v0.7.0/go.mod h1:2s5ssIFO69F5csTXcwBP7NPFTZvps26xGzvQ2PQaBYg= cloud.google.com/go/privatecatalog v0.8.0/go.mod h1:nQ6pfaegeDAq/Q5lrfCQzQLhubPiZhSaNhIgfJlnIXs= -cloud.google.com/go/profiler v0.4.0 h1:ZeRDZbsOBDyRG0OiK0Op1/XWZ3xeLwJc9zjkzczUxyY= -cloud.google.com/go/profiler v0.4.0/go.mod h1:RvPlm4dilIr3oJtAOeFQU9Lrt5RoySHSDj4pTd6TWeU= +cloud.google.com/go/profiler v0.4.1 h1:Q7+lOvikTGMJ/IAWocpYYGit4SIIoILmVZfEEWTORSY= +cloud.google.com/go/profiler v0.4.1/go.mod h1:LBrtEX6nbvhv1w/e5CPZmX9ajGG9BGLtGbv56Tg4SHs= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -938,8 +938,8 @@ github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 h1:hR7/MlvK23p6+lIw9SN1TigNLn9ZnF3W4SYRKq2gAHs= -github.com/google/pprof v0.0.0-20230602150820-91b7bce49751/go.mod h1:Jh3hGz2jkYak8qXPD19ryItVnUgpgeqzdkY/D0EaeuA= +github.com/google/pprof v0.0.0-20240528025155-186aa0362fba h1:ql1qNgCyOB7iAEk8JTNM+zJrgIbnyCKX/wdlyPufP5g= +github.com/google/pprof v0.0.0-20240528025155-186aa0362fba/go.mod h1:K1liHPHnj73Fdn/EKuT8nrFqBihUSKXoLYU0BuatOYo= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= From 6d96ae2580d61498cc6ee04c25777ebe5e79de32 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 9 Aug 2024 09:27:09 -0400 Subject: [PATCH 48/78] Fix Lineage name breaking change (#32122) --- .../src/main/java/org/apache/beam/sdk/metrics/Lineage.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java index 8b69b0ef55236..6166a562bf2df 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java @@ -55,8 +55,8 @@ public static Set query(MetricResults results, Type type) { /** Lineage metrics resource types. */ public enum Type { - SOURCE("source"), - SINK("sink"); + SOURCE("sources"), + SINK("sinks"); private final String name; From 01100a3b2fe9af9f30192f31cefe4e8d180dc782 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 9 Aug 2024 16:01:40 +0200 Subject: [PATCH 49/78] Generate python dependencies (#32132) --- .../license_scripts/dep_urls_py.yaml | 2 + .../py310/base_image_requirements.txt | 135 ++++++++--------- .../py311/base_image_requirements.txt | 133 ++++++++--------- .../py312/base_image_requirements.txt | 122 +++++++-------- .../py38/base_image_requirements.txt | 133 ++++++++--------- .../py39/base_image_requirements.txt | 139 +++++++++--------- 6 files changed, 336 insertions(+), 328 deletions(-) diff --git a/sdks/python/container/license_scripts/dep_urls_py.yaml b/sdks/python/container/license_scripts/dep_urls_py.yaml index 6fc5129e35c23..0fe830b7ab6ea 100644 --- a/sdks/python/container/license_scripts/dep_urls_py.yaml +++ b/sdks/python/container/license_scripts/dep_urls_py.yaml @@ -141,6 +141,8 @@ pip_dependencies: license: "https://raw.githubusercontent.com/jamescasbon/PyVCF/master/LICENSE" singledispatch: license: "file:///tmp/license_scripts/manual_licenses/singledispatch/LICENSE" + scikit-learn: + license: "https://raw.githubusercontent.com/scikit-learn/scikit-learn/master/COPYING" scipy: license: "https://raw.githubusercontent.com/scipy/scipy/master/LICENSE.txt" soupsieve: diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index e2806270e5443..2f736d0ebd26e 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -21,76 +21,77 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. -annotated-types==0.6.0 +annotated-types==0.7.0 async-timeout==4.0.3 -attrs==23.2.0 +attrs==24.2.0 beautifulsoup4==4.12.3 bs4==0.0.2 build==1.2.1 -cachetools==5.3.3 -certifi==2024.2.2 -cffi==1.16.0 +cachetools==5.4.0 +certifi==2024.7.4 +cffi==1.17.0 charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 -cryptography==42.0.5 +cryptography==43.0.0 Cython==0.29.37 -Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 -docker==7.0.0 +docker==7.1.0 docopt==0.6.2 docstring_parser==0.16 -exceptiongroup==1.2.0 +exceptiongroup==1.2.2 execnet==2.1.1 -fastavro==1.9.4 +fastavro==1.9.5 fasteners==0.19 -freezegun==1.4.0 +freezegun==1.5.1 future==1.0.0 -google-api-core==2.18.0 -google-api-python-client==2.126.0 +google-api-core==2.19.1 +google-api-python-client==2.140.0 google-apitools==0.5.31 -google-auth==2.29.0 +google-auth==2.33.0 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.47.0 -google-cloud-bigquery==3.20.1 -google-cloud-bigquery-storage==2.24.0 -google-cloud-bigtable==2.23.1 +google-cloud-aiplatform==1.61.0 +google-cloud-bigquery==3.25.0 +google-cloud-bigquery-storage==2.25.0 +google-cloud-bigtable==2.25.0 google-cloud-core==2.4.1 -google-cloud-datastore==2.19.0 -google-cloud-dlp==3.16.0 -google-cloud-language==2.13.3 +google-cloud-datastore==2.20.0 +google-cloud-dlp==3.21.0 +google-cloud-language==2.14.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.21.1 -google-cloud-pubsublite==1.10.0 -google-cloud-recommendations-ai==0.10.10 -google-cloud-resource-manager==1.12.3 -google-cloud-spanner==3.44.0 -google-cloud-storage==2.16.0 -google-cloud-videointelligence==2.13.3 -google-cloud-vision==3.7.2 +google-cloud-pubsub==2.23.0 +google-cloud-pubsublite==1.11.1 +google-cloud-recommendations-ai==0.10.12 +google-cloud-resource-manager==1.12.5 +google-cloud-spanner==3.48.0 +google-cloud-storage==2.18.2 +google-cloud-videointelligence==2.13.5 +google-cloud-vision==3.7.4 google-crc32c==1.5.0 -google-resumable-media==2.7.0 -googleapis-common-protos==1.63.0 +google-resumable-media==2.7.2 +googleapis-common-protos==1.63.2 greenlet==3.0.3 -grpc-google-iam-v1==0.13.0 +grpc-google-iam-v1==0.13.1 grpc-interceptor==0.15.4 -grpcio==1.62.2 -grpcio-status==1.62.2 +grpcio==1.65.4 +grpcio-status==1.62.3 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.100.1 +hypothesis==6.110.1 idna==3.7 iniconfig==2.0.0 -joblib==1.4.0 +Jinja2==3.0.3 +joblib==1.4.2 Js2Py==0.74 -jsonpickle==3.0.4 -jsonschema==4.21.1 +jsonpickle==3.2.2 +jsonschema==4.23.0 jsonschema-specifications==2023.12.1 +MarkupSafe==2.1.5 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 @@ -98,60 +99,60 @@ nose==1.3.7 numpy==1.26.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.10.1 +orjson==3.10.7 overrides==7.7.0 -packaging==24.0 +packaging==24.1 pandas==2.1.4 parameterized==0.9.0 -pluggy==1.4.0 -proto-plus==1.23.0 -protobuf==4.25.3 +pluggy==1.5.0 +proto-plus==1.24.0 +protobuf==4.25.4 psycopg2-binary==2.9.9 -pyarrow==14.0.2 +pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.0 pyasn1_modules==0.4.0 pycparser==2.22 -pydantic==2.7.0 -pydantic_core==2.18.1 +pydantic==2.8.2 +pydantic_core==2.20.1 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.3 -PyMySQL==1.1.0 +pymongo==4.8.0 +PyMySQL==1.1.1 pyparsing==3.1.2 -pyproject_hooks==1.0.0 +pyproject_hooks==1.1.0 pytest==7.4.4 pytest-timeout==2.3.1 -pytest-xdist==3.5.0 +pytest-xdist==3.6.1 python-dateutil==2.9.0.post0 -python-snappy==0.7.1 +python-snappy==0.7.2 pytz==2024.1 -PyYAML==6.0.1 -redis==5.0.3 -referencing==0.34.0 -regex==2024.4.16 +PyYAML==6.0.2 +redis==5.0.8 +referencing==0.35.1 +regex==2024.7.24 requests==2.31.0 requests-mock==1.12.1 -rpds-py==0.18.0 +rpds-py==0.20.0 rsa==4.9 -scikit-learn==1.4.2 -scipy==1.13.0 -shapely==2.0.4 +scikit-learn==1.5.1 +scipy==1.14.0 +shapely==2.0.5 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==2.0.29 -sqlparse==0.5.0 -tenacity==8.2.3 +SQLAlchemy==2.0.32 +sqlparse==0.5.1 +tenacity==8.5.0 testcontainers==3.7.1 -threadpoolctl==3.4.0 +threadpoolctl==3.5.0 tomli==2.0.1 -tqdm==4.66.2 -typing_extensions==4.11.0 +tqdm==4.66.5 +typing_extensions==4.12.2 tzdata==2024.1 tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.2.1 +urllib3==2.2.2 wrapt==1.16.0 -zstandard==0.22.0 +zstandard==0.23.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 1cba006ad32cc..712986882a06e 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -21,74 +21,75 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. -annotated-types==0.6.0 -attrs==23.2.0 +annotated-types==0.7.0 +attrs==24.2.0 beautifulsoup4==4.12.3 bs4==0.0.2 build==1.2.1 -cachetools==5.3.3 -certifi==2024.2.2 -cffi==1.16.0 +cachetools==5.4.0 +certifi==2024.7.4 +cffi==1.17.0 charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 -cryptography==42.0.5 +cryptography==43.0.0 Cython==0.29.37 -Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 -docker==7.0.0 +docker==7.1.0 docopt==0.6.2 docstring_parser==0.16 execnet==2.1.1 -fastavro==1.9.4 +fastavro==1.9.5 fasteners==0.19 -freezegun==1.4.0 +freezegun==1.5.1 future==1.0.0 -google-api-core==2.18.0 -google-api-python-client==2.126.0 +google-api-core==2.19.1 +google-api-python-client==2.140.0 google-apitools==0.5.31 -google-auth==2.29.0 +google-auth==2.33.0 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.47.0 -google-cloud-bigquery==3.20.1 -google-cloud-bigquery-storage==2.24.0 -google-cloud-bigtable==2.23.1 +google-cloud-aiplatform==1.61.0 +google-cloud-bigquery==3.25.0 +google-cloud-bigquery-storage==2.25.0 +google-cloud-bigtable==2.25.0 google-cloud-core==2.4.1 -google-cloud-datastore==2.19.0 -google-cloud-dlp==3.16.0 -google-cloud-language==2.13.3 +google-cloud-datastore==2.20.0 +google-cloud-dlp==3.21.0 +google-cloud-language==2.14.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.21.1 -google-cloud-pubsublite==1.10.0 -google-cloud-recommendations-ai==0.10.10 -google-cloud-resource-manager==1.12.3 -google-cloud-spanner==3.44.0 -google-cloud-storage==2.16.0 -google-cloud-videointelligence==2.13.3 -google-cloud-vision==3.7.2 +google-cloud-pubsub==2.23.0 +google-cloud-pubsublite==1.11.1 +google-cloud-recommendations-ai==0.10.12 +google-cloud-resource-manager==1.12.5 +google-cloud-spanner==3.48.0 +google-cloud-storage==2.18.2 +google-cloud-videointelligence==2.13.5 +google-cloud-vision==3.7.4 google-crc32c==1.5.0 -google-resumable-media==2.7.0 -googleapis-common-protos==1.63.0 +google-resumable-media==2.7.2 +googleapis-common-protos==1.63.2 greenlet==3.0.3 -grpc-google-iam-v1==0.13.0 +grpc-google-iam-v1==0.13.1 grpc-interceptor==0.15.4 -grpcio==1.62.2 -grpcio-status==1.62.2 +grpcio==1.65.4 +grpcio-status==1.62.3 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.100.1 +hypothesis==6.110.1 idna==3.7 iniconfig==2.0.0 -joblib==1.4.0 +Jinja2==3.0.3 +joblib==1.4.2 Js2Py==0.74 -jsonpickle==3.0.4 -jsonschema==4.21.1 +jsonpickle==3.2.2 +jsonschema==4.23.0 jsonschema-specifications==2023.12.1 +MarkupSafe==2.1.5 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 @@ -96,59 +97,59 @@ nose==1.3.7 numpy==1.26.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.10.1 +orjson==3.10.7 overrides==7.7.0 -packaging==24.0 +packaging==24.1 pandas==2.1.4 parameterized==0.9.0 -pluggy==1.4.0 -proto-plus==1.23.0 -protobuf==4.25.3 +pluggy==1.5.0 +proto-plus==1.24.0 +protobuf==4.25.4 psycopg2-binary==2.9.9 -pyarrow==14.0.2 +pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.0 pyasn1_modules==0.4.0 pycparser==2.22 -pydantic==2.7.0 -pydantic_core==2.18.1 +pydantic==2.8.2 +pydantic_core==2.20.1 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.3 -PyMySQL==1.1.0 +pymongo==4.8.0 +PyMySQL==1.1.1 pyparsing==3.1.2 -pyproject_hooks==1.0.0 +pyproject_hooks==1.1.0 pytest==7.4.4 pytest-timeout==2.3.1 -pytest-xdist==3.5.0 +pytest-xdist==3.6.1 python-dateutil==2.9.0.post0 -python-snappy==0.7.1 +python-snappy==0.7.2 pytz==2024.1 -PyYAML==6.0.1 -redis==5.0.3 -referencing==0.34.0 -regex==2024.4.16 +PyYAML==6.0.2 +redis==5.0.8 +referencing==0.35.1 +regex==2024.7.24 requests==2.31.0 requests-mock==1.12.1 -rpds-py==0.18.0 +rpds-py==0.20.0 rsa==4.9 -scikit-learn==1.4.2 -scipy==1.13.0 -shapely==2.0.4 +scikit-learn==1.5.1 +scipy==1.14.0 +shapely==2.0.5 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==2.0.29 -sqlparse==0.5.0 -tenacity==8.2.3 +SQLAlchemy==2.0.32 +sqlparse==0.5.1 +tenacity==8.5.0 testcontainers==3.7.1 -threadpoolctl==3.4.0 -tqdm==4.66.2 -typing_extensions==4.11.0 +threadpoolctl==3.5.0 +tqdm==4.66.5 +typing_extensions==4.12.2 tzdata==2024.1 tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.2.1 +urllib3==2.2.2 wrapt==1.16.0 -zstandard==0.22.0 +zstandard==0.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 4a6147b573b85..241d82913f2ec 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -21,72 +21,74 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. -annotated-types==0.6.0 -attrs==23.2.0 +annotated-types==0.7.0 +attrs==24.2.0 beautifulsoup4==4.12.3 bs4==0.0.2 build==1.2.1 -cachetools==5.3.3 -certifi==2024.2.2 -cffi==1.16.0 +cachetools==5.4.0 +certifi==2024.7.4 +cffi==1.17.0 charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 -cryptography==42.0.7 +cryptography==43.0.0 Cython==0.29.37 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 -docker==7.0.0 +docker==7.1.0 docopt==0.6.2 docstring_parser==0.16 execnet==2.1.1 -fastavro==1.9.4 +fastavro==1.9.5 fasteners==0.19 -freezegun==1.5.0 +freezegun==1.5.1 future==1.0.0 -google-api-core==2.19.0 -google-api-python-client==2.128.0 +google-api-core==2.19.1 +google-api-python-client==2.140.0 google-apitools==0.5.31 -google-auth==2.29.0 +google-auth==2.33.0 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.50.0 -google-cloud-bigquery==3.22.0 +google-cloud-aiplatform==1.61.0 +google-cloud-bigquery==3.25.0 google-cloud-bigquery-storage==2.25.0 -google-cloud-bigtable==2.23.1 +google-cloud-bigtable==2.25.0 google-cloud-core==2.4.1 -google-cloud-datastore==2.19.0 -google-cloud-dlp==3.16.0 -google-cloud-language==2.13.3 +google-cloud-datastore==2.20.0 +google-cloud-dlp==3.21.0 +google-cloud-language==2.14.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.21.1 -google-cloud-pubsublite==1.10.0 -google-cloud-recommendations-ai==0.10.10 -google-cloud-resource-manager==1.12.3 -google-cloud-spanner==3.46.0 -google-cloud-storage==2.16.0 -google-cloud-videointelligence==2.13.3 -google-cloud-vision==3.7.2 +google-cloud-pubsub==2.23.0 +google-cloud-pubsublite==1.11.1 +google-cloud-recommendations-ai==0.10.12 +google-cloud-resource-manager==1.12.5 +google-cloud-spanner==3.48.0 +google-cloud-storage==2.18.2 +google-cloud-videointelligence==2.13.5 +google-cloud-vision==3.7.4 google-crc32c==1.5.0 -google-resumable-media==2.7.0 -googleapis-common-protos==1.63.0 +google-resumable-media==2.7.2 +googleapis-common-protos==1.63.2 greenlet==3.0.3 -grpc-google-iam-v1==0.13.0 +grpc-google-iam-v1==0.13.1 grpc-interceptor==0.15.4 -grpcio==1.63.0 -grpcio-status==1.62.2 +grpcio==1.65.4 +grpcio-status==1.62.3 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.100.5 +hypothesis==6.110.1 idna==3.7 iniconfig==2.0.0 +Jinja2==3.0.3 joblib==1.4.2 -jsonpickle==3.0.4 -jsonschema==4.22.0 +jsonpickle==3.2.2 +jsonschema==4.23.0 jsonschema-specifications==2023.12.1 +MarkupSafe==2.1.5 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 @@ -94,59 +96,59 @@ nose==1.3.7 numpy==1.26.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.10.3 +orjson==3.10.7 overrides==7.7.0 -packaging==24.0 +packaging==24.1 pandas==2.1.4 parameterized==0.9.0 pluggy==1.5.0 -proto-plus==1.23.0 -protobuf==4.25.3 +proto-plus==1.24.0 +protobuf==4.25.4 psycopg2-binary==2.9.9 -pyarrow==14.0.2 +pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.0 pyasn1_modules==0.4.0 pycparser==2.22 -pydantic==2.7.1 -pydantic_core==2.18.2 +pydantic==2.8.2 +pydantic_core==2.20.1 pydot==1.4.2 PyHamcrest==2.1.0 -pymongo==4.7.1 -PyMySQL==1.1.0 +pymongo==4.8.0 +PyMySQL==1.1.1 pyparsing==3.1.2 pyproject_hooks==1.1.0 pytest==7.4.4 pytest-timeout==2.3.1 pytest-xdist==3.6.1 python-dateutil==2.9.0.post0 -python-snappy==0.7.1 +python-snappy==0.7.2 pytz==2024.1 -PyYAML==6.0.1 -redis==5.0.4 +PyYAML==6.0.2 +redis==5.0.8 referencing==0.35.1 -regex==2024.4.28 +regex==2024.7.24 requests==2.31.0 requests-mock==1.12.1 -rpds-py==0.18.1 +rpds-py==0.20.0 rsa==4.9 -scikit-learn==1.4.2 -scipy==1.13.0 -setuptools==69.5.1 -shapely==2.0.4 +scikit-learn==1.5.1 +scipy==1.14.0 +setuptools==72.1.0 +shapely==2.0.5 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==2.0.30 -sqlparse==0.5.0 -tenacity==8.3.0 +SQLAlchemy==2.0.32 +sqlparse==0.5.1 +tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.5.0 -tqdm==4.66.4 -typing_extensions==4.11.0 +tqdm==4.66.5 +typing_extensions==4.12.2 tzdata==2024.1 uritemplate==4.1.1 -urllib3==2.2.1 -wheel==0.43.0 +urllib3==2.2.2 +wheel==0.44.0 wrapt==1.16.0 -zstandard==0.22.0 +zstandard==0.23.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index f88dba103469b..0c605548f9548 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -21,79 +21,80 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. -annotated-types==0.6.0 +annotated-types==0.7.0 async-timeout==4.0.3 -attrs==23.2.0 +attrs==24.2.0 backports.zoneinfo==0.2.1 beautifulsoup4==4.12.3 bs4==0.0.2 build==1.2.1 -cachetools==5.3.3 +cachetools==5.4.0 certifi==2024.7.4 -cffi==1.16.0 +cffi==1.17.0 charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 -cryptography==42.0.5 +cryptography==43.0.0 Cython==0.29.37 -Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 -docker==7.0.0 +docker==7.1.0 docopt==0.6.2 docstring_parser==0.16 -exceptiongroup==1.2.0 +exceptiongroup==1.2.2 execnet==2.1.1 -fastavro==1.9.4 +fastavro==1.9.5 fasteners==0.19 -freezegun==1.4.0 +freezegun==1.5.1 future==1.0.0 -google-api-core==2.18.0 -google-api-python-client==2.126.0 +google-api-core==2.19.1 +google-api-python-client==2.140.0 google-apitools==0.5.31 -google-auth==2.29.0 +google-auth==2.33.0 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.47.0 -google-cloud-bigquery==3.20.1 -google-cloud-bigquery-storage==2.24.0 -google-cloud-bigtable==2.23.1 +google-cloud-aiplatform==1.61.0 +google-cloud-bigquery==3.25.0 +google-cloud-bigquery-storage==2.25.0 +google-cloud-bigtable==2.25.0 google-cloud-core==2.4.1 -google-cloud-datastore==2.19.0 -google-cloud-dlp==3.16.0 -google-cloud-language==2.13.3 +google-cloud-datastore==2.20.0 +google-cloud-dlp==3.21.0 +google-cloud-language==2.14.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.21.1 -google-cloud-pubsublite==1.10.0 -google-cloud-recommendations-ai==0.10.10 -google-cloud-resource-manager==1.12.3 -google-cloud-spanner==3.44.0 -google-cloud-storage==2.16.0 -google-cloud-videointelligence==2.13.3 -google-cloud-vision==3.7.2 +google-cloud-pubsub==2.23.0 +google-cloud-pubsublite==1.11.1 +google-cloud-recommendations-ai==0.10.12 +google-cloud-resource-manager==1.12.5 +google-cloud-spanner==3.48.0 +google-cloud-storage==2.18.2 +google-cloud-videointelligence==2.13.5 +google-cloud-vision==3.7.4 google-crc32c==1.5.0 -google-resumable-media==2.7.0 -googleapis-common-protos==1.63.0 +google-resumable-media==2.7.2 +googleapis-common-protos==1.63.2 greenlet==3.0.3 -grpc-google-iam-v1==0.13.0 +grpc-google-iam-v1==0.13.1 grpc-interceptor==0.15.4 -grpcio==1.62.2 -grpcio-status==1.62.2 +grpcio==1.65.4 +grpcio-status==1.62.3 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.100.1 +hypothesis==6.110.1 idna==3.7 -importlib_metadata==7.1.0 +importlib_metadata==8.2.0 importlib_resources==6.4.0 iniconfig==2.0.0 -joblib==1.4.0 +Jinja2==3.0.3 +joblib==1.4.2 Js2Py==0.74 -jsonpickle==3.0.4 -jsonschema==4.21.1 +jsonpickle==3.2.2 +jsonschema==4.23.0 jsonschema-specifications==2023.12.1 +MarkupSafe==2.1.5 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 @@ -101,62 +102,62 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.10.1 +orjson==3.10.7 overrides==7.7.0 -packaging==24.0 +packaging==24.1 pandas==2.0.3 parameterized==0.9.0 pkgutil_resolve_name==1.3.10 -pluggy==1.4.0 -proto-plus==1.23.0 -protobuf==4.25.3 +pluggy==1.5.0 +proto-plus==1.24.0 +protobuf==4.25.4 psycopg2-binary==2.9.9 -pyarrow==14.0.2 +pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.0 pyasn1_modules==0.4.0 pycparser==2.22 -pydantic==2.7.0 -pydantic_core==2.18.1 +pydantic==2.8.2 +pydantic_core==2.20.1 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.3 -PyMySQL==1.1.0 +pymongo==4.8.0 +PyMySQL==1.1.1 pyparsing==3.1.2 -pyproject_hooks==1.0.0 +pyproject_hooks==1.1.0 pytest==7.4.4 pytest-timeout==2.3.1 -pytest-xdist==3.5.0 +pytest-xdist==3.6.1 python-dateutil==2.9.0.post0 -python-snappy==0.7.1 +python-snappy==0.7.2 pytz==2024.1 -PyYAML==6.0.1 -redis==5.0.3 -referencing==0.34.0 -regex==2024.4.16 +PyYAML==6.0.2 +redis==5.0.8 +referencing==0.35.1 +regex==2024.7.24 requests==2.31.0 requests-mock==1.12.1 -rpds-py==0.18.0 +rpds-py==0.20.0 rsa==4.9 scikit-learn==1.3.2 scipy==1.10.1 -shapely==2.0.4 +shapely==2.0.5 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==2.0.29 -sqlparse==0.5.0 -tenacity==8.2.3 +SQLAlchemy==2.0.32 +sqlparse==0.5.1 +tenacity==8.5.0 testcontainers==3.7.1 -threadpoolctl==3.4.0 +threadpoolctl==3.5.0 tomli==2.0.1 -tqdm==4.66.3 -typing_extensions==4.11.0 +tqdm==4.66.5 +typing_extensions==4.12.2 tzdata==2024.1 tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.2.1 +urllib3==2.2.2 wrapt==1.16.0 -zipp==3.18.1 -zstandard==0.22.0 +zipp==3.19.2 +zstandard==0.23.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 39b888dd6ab75..52a7136ecffa7 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -21,77 +21,78 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. -annotated-types==0.6.0 +annotated-types==0.7.0 async-timeout==4.0.3 -attrs==23.2.0 +attrs==24.2.0 beautifulsoup4==4.12.3 bs4==0.0.2 build==1.2.1 -cachetools==5.3.3 -certifi==2024.2.2 -cffi==1.16.0 +cachetools==5.4.0 +certifi==2024.7.4 +cffi==1.17.0 charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 cramjam==2.8.3 crcmod==1.7 -cryptography==42.0.5 +cryptography==43.0.0 Cython==0.29.37 -Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.6.1 -docker==7.0.0 +docker==7.1.0 docopt==0.6.2 docstring_parser==0.16 -exceptiongroup==1.2.0 +exceptiongroup==1.2.2 execnet==2.1.1 -fastavro==1.9.4 +fastavro==1.9.5 fasteners==0.19 -freezegun==1.4.0 +freezegun==1.5.1 future==1.0.0 -google-api-core==2.18.0 -google-api-python-client==2.126.0 +google-api-core==2.19.1 +google-api-python-client==2.140.0 google-apitools==0.5.31 -google-auth==2.29.0 +google-auth==2.33.0 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.47.0 -google-cloud-bigquery==3.20.1 -google-cloud-bigquery-storage==2.24.0 -google-cloud-bigtable==2.23.1 +google-cloud-aiplatform==1.61.0 +google-cloud-bigquery==3.25.0 +google-cloud-bigquery-storage==2.25.0 +google-cloud-bigtable==2.25.0 google-cloud-core==2.4.1 -google-cloud-datastore==2.19.0 -google-cloud-dlp==3.16.0 -google-cloud-language==2.13.3 +google-cloud-datastore==2.20.0 +google-cloud-dlp==3.21.0 +google-cloud-language==2.14.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.21.1 -google-cloud-pubsublite==1.10.0 -google-cloud-recommendations-ai==0.10.10 -google-cloud-resource-manager==1.12.3 -google-cloud-spanner==3.44.0 -google-cloud-storage==2.16.0 -google-cloud-videointelligence==2.13.3 -google-cloud-vision==3.7.2 +google-cloud-pubsub==2.23.0 +google-cloud-pubsublite==1.11.1 +google-cloud-recommendations-ai==0.10.12 +google-cloud-resource-manager==1.12.5 +google-cloud-spanner==3.48.0 +google-cloud-storage==2.18.2 +google-cloud-videointelligence==2.13.5 +google-cloud-vision==3.7.4 google-crc32c==1.5.0 -google-resumable-media==2.7.0 -googleapis-common-protos==1.63.0 +google-resumable-media==2.7.2 +googleapis-common-protos==1.63.2 greenlet==3.0.3 -grpc-google-iam-v1==0.13.0 +grpc-google-iam-v1==0.13.1 grpc-interceptor==0.15.4 -grpcio==1.62.2 -grpcio-status==1.62.2 +grpcio==1.65.4 +grpcio-status==1.62.3 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.100.1 +hypothesis==6.110.1 idna==3.7 -importlib_metadata==7.1.0 +importlib_metadata==8.2.0 iniconfig==2.0.0 -joblib==1.4.0 +Jinja2==3.0.3 +joblib==1.4.2 Js2Py==0.74 -jsonpickle==3.0.4 -jsonschema==4.21.1 +jsonpickle==3.2.2 +jsonschema==4.23.0 jsonschema-specifications==2023.12.1 +MarkupSafe==2.1.5 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 @@ -99,61 +100,61 @@ nose==1.3.7 numpy==1.26.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.10.1 +orjson==3.10.7 overrides==7.7.0 -packaging==24.0 +packaging==24.1 pandas==2.1.4 parameterized==0.9.0 -pluggy==1.4.0 -proto-plus==1.23.0 -protobuf==4.25.3 +pluggy==1.5.0 +proto-plus==1.24.0 +protobuf==4.25.4 psycopg2-binary==2.9.9 -pyarrow==14.0.2 +pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.0 pyasn1_modules==0.4.0 pycparser==2.22 -pydantic==2.7.0 -pydantic_core==2.18.1 +pydantic==2.8.2 +pydantic_core==2.20.1 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.3 -PyMySQL==1.1.0 +pymongo==4.8.0 +PyMySQL==1.1.1 pyparsing==3.1.2 -pyproject_hooks==1.0.0 +pyproject_hooks==1.1.0 pytest==7.4.4 pytest-timeout==2.3.1 -pytest-xdist==3.5.0 +pytest-xdist==3.6.1 python-dateutil==2.9.0.post0 -python-snappy==0.7.1 +python-snappy==0.7.2 pytz==2024.1 -PyYAML==6.0.1 -redis==5.0.3 -referencing==0.34.0 -regex==2024.4.16 +PyYAML==6.0.2 +redis==5.0.8 +referencing==0.35.1 +regex==2024.7.24 requests==2.31.0 requests-mock==1.12.1 -rpds-py==0.18.0 +rpds-py==0.20.0 rsa==4.9 -scikit-learn==1.4.2 -scipy==1.13.0 -shapely==2.0.4 +scikit-learn==1.5.1 +scipy==1.13.1 +shapely==2.0.5 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==2.0.29 -sqlparse==0.5.0 -tenacity==8.2.3 +SQLAlchemy==2.0.32 +sqlparse==0.5.1 +tenacity==8.5.0 testcontainers==3.7.1 -threadpoolctl==3.4.0 +threadpoolctl==3.5.0 tomli==2.0.1 -tqdm==4.66.2 -typing_extensions==4.11.0 +tqdm==4.66.5 +typing_extensions==4.12.2 tzdata==2024.1 tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.2.1 +urllib3==2.2.2 wrapt==1.16.0 -zipp==3.18.1 -zstandard==0.22.0 +zipp==3.19.2 +zstandard==0.23.0 From 82c3b36af70f6d4fa90d69963da03284c0dd6d28 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 9 Aug 2024 10:53:17 -0400 Subject: [PATCH 50/78] Bump golang.org/x/oauth2 from 0.21.0 to 0.22.0 in /sdks (#32129) Bumps [golang.org/x/oauth2](https://github.com/golang/oauth2) from 0.21.0 to 0.22.0. - [Commits](https://github.com/golang/oauth2/compare/v0.21.0...v0.22.0) --- updated-dependencies: - dependency-name: golang.org/x/oauth2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 8c7a52ca951c7..9e32b2ba7b356 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -54,7 +54,7 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.16.1 golang.org/x/net v0.28.0 - golang.org/x/oauth2 v0.21.0 + golang.org/x/oauth2 v0.22.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.23.0 golang.org/x/text v0.17.0 diff --git a/sdks/go.sum b/sdks/go.sum index e0039f28e43ba..af1d7b4ba828f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1408,8 +1408,8 @@ golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= -golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= -golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.22.0 h1:BzDx2FehcG7jJwgWLELCdmLuxk2i+x9UDpSiss2u0ZA= +golang.org/x/oauth2 v0.22.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From 4b476832020c593bf79d6c06b370efb0cd3b03c5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 9 Aug 2024 10:54:06 -0400 Subject: [PATCH 51/78] Bump github.com/fsouza/fake-gcs-server from 1.47.7 to 1.49.2 in /sdks (#32124) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.47.7 to 1.49.2. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.47.7...v1.49.2) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 16 ++++------------ 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9e32b2ba7b356..624cc0ab1ce82 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -68,7 +68,7 @@ require ( require ( github.com/avast/retry-go/v4 v4.6.0 - github.com/fsouza/fake-gcs-server v1.47.7 + github.com/fsouza/fake-gcs-server v1.49.2 golang.org/x/exp v0.0.0-20231006140011-7918f672742d ) diff --git a/sdks/go.sum b/sdks/go.sum index af1d7b4ba828f..67686da8e408a 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -813,8 +813,8 @@ github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSw github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= -github.com/fsouza/fake-gcs-server v1.47.7 h1:56/U4rKY081TaNbq0gHWi7/71UxC2KROqcnrD9BRJhs= -github.com/fsouza/fake-gcs-server v1.47.7/go.mod h1:4vPUynN8/zZlxk5Jpy6LvvTTxItdTAObK4DYnp89Jys= +github.com/fsouza/fake-gcs-server v1.49.2 h1:fukDqzEQM50QkA0jAbl6cLqeDu3maQjwZBuys759TR4= +github.com/fsouza/fake-gcs-server v1.49.2/go.mod h1:17SYzJEXRcaAA5ATwwvgBkSIqIy7r1icnGM0y/y4foY= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g= github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks= @@ -1008,8 +1008,6 @@ github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHW github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI= -github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= -github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= @@ -1057,10 +1055,8 @@ github.com/minio/highwayhash v1.0.3 h1:kbnuUMoHYyVl7szWjSxJnxw11k2U709jqFPPmIUyD github.com/minio/highwayhash v1.0.3/go.mod h1:GGYsuwP/fPD6Y9hMiXuapVvlIUEhFhMTh0rxU3ik1LQ= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= -github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= -github.com/minio/minio-go/v7 v7.0.66/go.mod h1:DHAgmyQEGdW3Cif0UooKOyrT3Vxs82zNdV6tkKhRtbs= -github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= -github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= +github.com/minio/minio-go/v7 v7.0.71 h1:No9XfOKTYi6i0GnBj+WZwD8WP5GZfL7n7GOjRqCdAjA= +github.com/minio/minio-go/v7 v7.0.71/go.mod h1:4yBA8v80xGA30cfM3fz0DKYMXunWl/AV/6tWEs9ryzo= github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= @@ -1071,10 +1067,6 @@ github.com/moby/sys/user v0.1.0 h1:WmZ93f5Ux6het5iituh9x2zAG7NFY9Aqi49jjE1PaQg= github.com/moby/sys/user v0.1.0/go.mod h1:fKJhFOnsCN6xZ5gSfbM6zaHGgDJMrqt9/reuj4T7MmU= github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= -github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.7.1 h1:etflOAAHORrCC44V+aR6Ftzort912ZU+YLiSTuV8eaE= github.com/montanaflynn/stats v0.7.1/go.mod h1:etXPPgVO6n31NxCd9KQUMvCM+ve0ruNzt6R8Bnaayow= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= From cae9148ee1ddbd0c46769e9ecfd963c47d66fcca Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Fri, 9 Aug 2024 10:57:31 -0400 Subject: [PATCH 52/78] Bump up google-cloud-storage version to fix data corruption issue (#32135) --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 756c952b0101b..110ff6a89882f 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -440,7 +440,7 @@ def get_portability_package_data(): 'google-cloud-datastore>=2.0.0,<3', 'google-cloud-pubsub>=2.1.0,<3', 'google-cloud-pubsublite>=1.2.0,<2', - 'google-cloud-storage>=2.16.0,<3', + 'google-cloud-storage>=2.18.2,<3', # GCP packages required by tests 'google-cloud-bigquery>=2.0.0,<4', 'google-cloud-bigquery-storage>=2.6.3,<3', From 1c0cfa1ccbae2ff8bdca562a9d52757197cd4ddf Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Fri, 9 Aug 2024 19:57:48 +0200 Subject: [PATCH 53/78] Expose watermarkIdleDurationThreshold parameter to the user in SolaceIO (#32109) --- .../apache/beam/sdk/io/solace/SolaceIO.java | 28 +++++++++++++++++-- .../io/solace/read/UnboundedSolaceReader.java | 4 ++- .../io/solace/read/UnboundedSolaceSource.java | 9 ++++++ .../io/solace/read/WatermarkParameters.java | 26 +---------------- .../sdk/io/solace/read/WatermarkPolicy.java | 10 +++++-- .../beam/sdk/io/solace/SolaceIOTest.java | 3 +- 6 files changed, 48 insertions(+), 32 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index bb9f0c6ea689b..dcfdcc4fabb9c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -51,6 +51,7 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -393,7 +394,8 @@ public class SolaceIO { } }; private static final boolean DEFAULT_DEDUPLICATE_RECORDS = false; - + private static final Duration DEFAULT_WATERMARK_IDLE_DURATION_THRESHOLD = + Duration.standardSeconds(30); public static final int DEFAULT_WRITER_MAX_NUMBER_OF_WORKERS = 20; public static final int DEFAULT_WRITER_CLIENTS_PER_WORKER = 4; public static final Boolean DEFAULT_WRITER_PUBLISH_LATENCY_METRICS = false; @@ -440,7 +442,8 @@ public static Read read() { .setTypeDescriptor(TypeDescriptor.of(Solace.Record.class)) .setParseFn(SolaceRecordMapper::map) .setTimestampFn(SENDER_TIMESTAMP_FUNCTION) - .setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS)); + .setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS) + .setWatermarkIdleDurationThreshold(DEFAULT_WATERMARK_IDLE_DURATION_THRESHOLD)); } /** * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} to @@ -467,7 +470,8 @@ public static Read read( .setTypeDescriptor(typeDescriptor) .setParseFn(parseFn) .setTimestampFn(timestampFn) - .setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS)); + .setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS) + .setWatermarkIdleDurationThreshold(DEFAULT_WATERMARK_IDLE_DURATION_THRESHOLD)); } /** @@ -540,6 +544,19 @@ public Read withMaxNumConnections(Integer maxNumConnections) { return this; } + /** + * Optional. Denotes the duration for which the watermark can be idle. If there are no incoming + * messages for this ‘idle’ period of time, the watermark is set to a timestamp representing a + * time earlier than now by the ‘idle’ period of time (e.g. if the ‘idle’ period of time is set + * to 30 seconds, and there is no new data incoming for 30 seconds, the watermark will be set to + * max(currentWatermark, now() - 30 seconds). The default watermark idle duration threshold is + * {@link #DEFAULT_WATERMARK_IDLE_DURATION_THRESHOLD}. + */ + public Read withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { + configurationBuilder.setWatermarkIdleDurationThreshold(idleDurationThreshold); + return this; + } + /** * Optional, default: false. Set to deduplicate messages based on the {@link * BytesXMLMessage#getApplicationMessageId()} of the incoming {@link BytesXMLMessage}. If the @@ -652,6 +669,8 @@ abstract static class Configuration { abstract TypeDescriptor getTypeDescriptor(); + abstract Duration getWatermarkIdleDurationThreshold(); + public static Builder builder() { Builder builder = new org.apache.beam.sdk.io.solace.AutoValue_SolaceIO_Read_Configuration.Builder(); @@ -680,6 +699,8 @@ abstract Builder setParseFn( abstract Builder setTypeDescriptor(TypeDescriptor typeDescriptor); + abstract Builder setWatermarkIdleDurationThreshold(Duration idleDurationThreshold); + abstract Configuration build(); } } @@ -716,6 +737,7 @@ public PCollection expand(PBegin input) { configuration.getDeduplicateRecords(), coder, configuration.getTimestampFn(), + configuration.getWatermarkIdleDurationThreshold(), configuration.getParseFn()))); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index 0155345a23236..c18a9d110b2ad 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -62,7 +62,9 @@ class UnboundedSolaceReader extends UnboundedReader { public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { this.currentSource = currentSource; - this.watermarkPolicy = WatermarkPolicy.create(currentSource.getTimestampFn()); + this.watermarkPolicy = + WatermarkPolicy.create( + currentSource.getTimestampFn(), currentSource.getWatermarkIdleDurationThreshold()); this.sessionService = currentSource.getSessionServiceFactory().create(); this.sempClient = currentSource.getSempClientFactory().create(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java index 370159994941b..1cb17a49fbdba 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,6 +47,7 @@ public class UnboundedSolaceSource extends UnboundedSource timestampFn; + private final Duration watermarkIdleDurationThreshold; private final SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn; public Queue getQueue() { @@ -64,6 +66,10 @@ public SerializableFunction getTimestampFn() { return timestampFn; } + public Duration getWatermarkIdleDurationThreshold() { + return watermarkIdleDurationThreshold; + } + public SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> getParseFn() { return parseFn; } @@ -76,6 +82,7 @@ public UnboundedSolaceSource( boolean enableDeduplication, Coder coder, SerializableFunction timestampFn, + Duration watermarkIdleDurationThreshold, SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn) { this.queue = queue; this.sempClientFactory = sempClientFactory; @@ -84,6 +91,7 @@ public UnboundedSolaceSource( this.enableDeduplication = enableDeduplication; this.coder = coder; this.timestampFn = timestampFn; + this.watermarkIdleDurationThreshold = watermarkIdleDurationThreshold; this.parseFn = parseFn; } @@ -125,6 +133,7 @@ private List> getSolaceSources( enableDeduplication, coder, timestampFn, + watermarkIdleDurationThreshold, parseFn); sourceList.add(source); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java index f58cb1cc202d0..29b35d883f22f 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java @@ -21,7 +21,6 @@ import java.io.Serializable; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Duration; import org.joda.time.Instant; @@ -29,9 +28,6 @@ @AutoValue abstract class WatermarkParameters implements Serializable { - private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = - Duration.standardSeconds(30); - abstract Instant getCurrentWatermark(); abstract Instant getLastSavedWatermark(); @@ -48,8 +44,7 @@ static Builder builder() { return new AutoValue_WatermarkParameters.Builder() .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) .setLastSavedWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) - .setLastUpdateTime(Instant.now()) - .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); + .setLastUpdateTime(Instant.now()); } @AutoValue.Builder @@ -66,23 +61,4 @@ abstract static class Builder { abstract WatermarkParameters build(); } - - /** - * Create an instance of {@link WatermarkParameters} with a {@code SerializableFunction} to - * extract the event time. - */ - static WatermarkParameters create(SerializableFunction timestampFn) { - Preconditions.checkArgument(timestampFn != null, "timestampFn function is null"); - return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); - } - - /** - * Specify the watermark idle duration to consider before advancing the watermark. The default - * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. - */ - WatermarkParameters withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { - Preconditions.checkArgument( - idleDurationThreshold != null, "watermark idle duration threshold is null"); - return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); - } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java index 13d65639e3358..9d2ed24f3c06a 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java @@ -21,6 +21,7 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Ordering; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -39,8 +40,13 @@ class WatermarkPolicy implements Serializable { private WatermarkParameters watermarkParameters; - static WatermarkPolicy create(SerializableFunction timestampFunction) { - return new WatermarkPolicy(WatermarkParameters.create(timestampFunction)); + static WatermarkPolicy create( + SerializableFunction timestampFunction, Duration watermarkIdleDurationThreshold) { + return new WatermarkPolicy( + WatermarkParameters.builder() + .setTimestampFn(timestampFunction) + .setWatermarkIdleDurationThreshold(watermarkIdleDurationThreshold) + .build()); } private WatermarkPolicy(WatermarkParameters watermarkParameters) { diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index bd9d5d401b548..cc1fa1d667aaf 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -95,6 +95,7 @@ private static UnboundedSolaceSource getSource(Read spec, TestPi configuration.getDeduplicateRecords(), spec.inferCoder(pipeline, configuration.getTypeDescriptor()), configuration.getTimestampFn(), + configuration.getWatermarkIdleDurationThreshold(), configuration.getParseFn()); } @@ -527,7 +528,7 @@ public void testCheckpointMarkSafety() throws Exception { @Test public void testDefaultCoder() { Coder coder = - new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) + new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null, null) .getCheckpointMarkCoder(); CoderProperties.coderSerializable(coder); } From f73a6d1570acc2945db4d80206dd86d6054f2ac2 Mon Sep 17 00:00:00 2001 From: Timothy Itodo Date: Fri, 9 Aug 2024 14:06:02 -0500 Subject: [PATCH 54/78] Create Beam YAML Join documentation (#31494) --- .../en/documentation/sdks/yaml-join.md | 182 ++++++++++++++++++ .../partials/section-menu/en/sdks.html | 1 + 2 files changed, 183 insertions(+) create mode 100644 website/www/site/content/en/documentation/sdks/yaml-join.md diff --git a/website/www/site/content/en/documentation/sdks/yaml-join.md b/website/www/site/content/en/documentation/sdks/yaml-join.md new file mode 100644 index 0000000000000..d207926ff995b --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml-join.md @@ -0,0 +1,182 @@ +--- +type: languages +title: "Apache Beam YAML Join" +--- + + +# Beam YAML Join + +Beam YAML can join two or more inputs on specified columns. For example, the +following pipeline joins the First Input pcollection and Second Input +pcollection when col1 in First Input is equal to col2 in Second Input. + +``` +- type: Join + input: + input1: First Input + input2: Second Input + config: + equalities: + - input1: col1 + input2: col2 +``` + +When joining multiple inputs on one column that is named the same across all the +inputs, one can use the following shorthand syntax: + +``` +- type: Join + input: + input1: First Input + input2: Second Input + input3: Third Input + config: + equalities: col +``` + +## Join Types + +When using the Join transform, one can specify the type of join to perform on +the inputs. If no join type is specified, the inputs are all joined using an +inner join. The supported join types are: + +| Join Type | YAML Keyword | +| -------- | ------- | +| Inner Join | inner | +| Full Outer Join | left | +| Right Outer Join | right | + +The following example joins two inputs using an inner join on the specified +equalities: + +``` +- type: Join + input: + input1: First Input + input2: Second Input + config: + type: inner + equalities: + - input1: col1 + input2: col1 +``` + + +The following example joins two inputs using a left outer join on the specified +equalities. In this case, all rows from input1 will be kept because input1 is +the left input. Order of joins follows the sequence as specified in equalities. + +``` +- type: Join + input: + input1: First Input + input2: Second Input + config: + type: left + equalities: + - input1: col1 + input2: col1 +``` + +The following example joins three inputs using an full outer join on the +specified equalities: + +``` +- type: Join + input: + input1: First Input + input2: Second Input + input3: Third Input + config: + type: outer + equalities: + - input1: col1 + input2: col1 + - input2: col2 + input3: col2 +``` + +If you want a combination of join types, you can specify the inputs to be outer +joined. The following example joins input1 with input2 using a right outer join +since input2 is on the right side and will join input2 with input 3 using a left +outer join since input2 is on the left side. + +``` +- type: Join + input: + input1: First Input + input2: Second Input + input3: Third Input + config: + type: + outer: + - input2 + equalities: + - input1: col1 + input2: col1 + - input2: col2 + input3: col2 +``` + +## Fields +By default, the join transform includes all columns from all input tables. If +column names clash, it's best to rename them explicitly. Otherwise, the system +will deduplicate names by adding a numeric suffix + +To choose which columns to output, or to customize the output column names, use +the "fields" configuration. + +To specify which columns to output from an input, use the input reference as the +configuration key and a list of desired columns as the configuration value. The +following example outputs col1 from input1, col2 and col3 from input2, and all +the columns from input 3. If there is a name clash, it appends a numeric suffix +to avoid duplicate naming. + +``` +- type: Join + input: + input1: First Input + input2: Second Input + input3: Third Input + config: + equalities: col1 + fields: + input1: [col1] + input2: [col2, col3] +``` + +To rename a column in the output, create a mapping for the input with the key as +the new column name and the value as the original column name. The following +example maps col1 from input3 to the column name "renamed_col1": + +``` +- type: Join + input: + input1: First Input + input2: Second Input + input3: Third Input + config: + equalities: col1 + fields: + input1: [col1] + input2: [col2, col3] + input3: + renamed_col1: col1 +``` diff --git a/website/www/site/layouts/partials/section-menu/en/sdks.html b/website/www/site/layouts/partials/section-menu/en/sdks.html index fd7de314992b1..ea48eb6f40d9b 100644 --- a/website/www/site/layouts/partials/section-menu/en/sdks.html +++ b/website/www/site/layouts/partials/section-menu/en/sdks.html @@ -92,6 +92,7 @@
  • Yaml Aggregation
  • Error handling
  • Inlining Python
  • +
  • Yaml Join
  • YAML API reference External link. From ec152e283557a7b9ba273dac4b6fc6400786d2cf Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 9 Aug 2024 13:31:17 -0700 Subject: [PATCH 55/78] [#32139] Fail pipelines with Stateful SDFs. (#32140) * [#32139] Fail pipelines with Stateful SDFs. * rm debug print --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../runners/prism/internal/jobservices/management.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index 7676d958031c1..2b03eddff05d7 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -158,17 +158,26 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jo return nil, fmt.Errorf("unable to unmarshal ParDoPayload for %v - %q: %w", tid, t.GetUniqueName(), err) } + isStateful := false + // Validate all the state features for _, spec := range pardo.GetStateSpecs() { + isStateful = true check("StateSpec.Protocol.Urn", spec.GetProtocol().GetUrn(), urns.UserStateBag, urns.UserStateMultiMap) } // Validate all the timer features for _, spec := range pardo.GetTimerFamilySpecs() { + isStateful = true check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain(), pipepb.TimeDomain_EVENT_TIME, pipepb.TimeDomain_PROCESSING_TIME) } check("OnWindowExpirationTimerFamily", pardo.GetOnWindowExpirationTimerFamilySpec(), "") // Unsupported for now. + // Check for a stateful SDF and direct user to https://github.com/apache/beam/issues/32139 + if pardo.GetRestrictionCoderId() != "" && isStateful { + check("Splittable+Stateful DoFn", "See https://github.com/apache/beam/issues/32139 for information.", "") + } + case urns.TransformTestStream: var testStream pipepb.TestStreamPayload if err := proto.Unmarshal(t.GetSpec().GetPayload(), &testStream); err != nil { From 17298b5572e9b0d8aa8c4d0ca1e51c3f832c0067 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 9 Aug 2024 14:21:32 -0700 Subject: [PATCH 56/78] [#32115] Fix timer support, support timer clears. (#32119) --- .../prism/internal/engine/elementmanager.go | 22 ++-- .../runners/prism/internal/engine/timers.go | 115 +++++++++++------- .../runners/portability/prism_runner_test.py | 32 +++++ 3 files changed, 119 insertions(+), 50 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index bc8449c72b39a..c73db507c7920 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -869,14 +869,20 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag for tentativeKey, timers := range d.timers { keyToTimers := map[timerKey]element{} for _, t := range timers { - key, tag, elms := decodeTimer(inputInfo.KeyDec, true, t) - for _, e := range elms { - keyToTimers[timerKey{key: string(key), tag: tag, win: e.window}] = e - } - if len(elms) == 0 { - // TODO(lostluck): Determine best way to mark a timer cleared. - continue - } + // TODO: Call in a for:range loop when Beam's minimum Go version hits 1.23.0 + iter := decodeTimerIter(inputInfo.KeyDec, true, t) + iter(func(ret timerRet) bool { + for _, e := range ret.elms { + keyToTimers[timerKey{key: string(ret.keyBytes), tag: ret.tag, win: e.window}] = e + } + if len(ret.elms) == 0 { + for _, w := range ret.windows { + delete(keyToTimers, timerKey{key: string(ret.keyBytes), tag: ret.tag, win: w}) + } + } + // Indicate we'd like to continue iterating. + return true + }) } for _, elm := range keyToTimers { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go index 787d27858a0e5..9a3bd6f9682bc 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go @@ -31,53 +31,77 @@ import ( "google.golang.org/protobuf/encoding/protowire" ) -// DecodeTimer extracts timers to elements for insertion into their keyed queues. -// Returns the key bytes, tag, window exploded elements, and the hold timestamp. +type timerRet struct { + keyBytes []byte + tag string + elms []element + windows []typex.Window +} + +// decodeTimerIter extracts timers to elements for insertion into their keyed queues, +// through a go iterator function, to be called by the caller with their processing function. +// +// For each timer, a key, tag, windowed elements, and the window set are returned. +// // If the timer has been cleared, no elements will be returned. Any existing timers -// for the tag *must* be cleared from the pending queue. -func decodeTimer(keyDec func(io.Reader) []byte, usesGlobalWindow bool, raw []byte) ([]byte, string, []element) { - keyBytes := keyDec(bytes.NewBuffer(raw)) - - d := decoder{raw: raw, cursor: len(keyBytes)} - tag := string(d.Bytes()) - - var ws []typex.Window - numWin := d.Fixed32() - if usesGlobalWindow { - for i := 0; i < int(numWin); i++ { - ws = append(ws, window.GlobalWindow{}) - } - } else { - // Assume interval windows here, since we don't understand custom windows yet. - for i := 0; i < int(numWin); i++ { - ws = append(ws, d.IntervalWindow()) - } - } +// for the tag *must* be cleared from the pending queue. The windows associated with +// the clear are provided to be able to delete pending timers. +func decodeTimerIter(keyDec func(io.Reader) []byte, usesGlobalWindow bool, raw []byte) func(func(timerRet) bool) { + return func(yield func(timerRet) bool) { + for len(raw) > 0 { + keyBytes := keyDec(bytes.NewBuffer(raw)) + d := decoder{raw: raw, cursor: len(keyBytes)} + tag := string(d.Bytes()) + + var ws []typex.Window + numWin := d.Fixed32() + if usesGlobalWindow { + for i := 0; i < int(numWin); i++ { + ws = append(ws, window.GlobalWindow{}) + } + } else { + // Assume interval windows here, since we don't understand custom windows yet. + for i := 0; i < int(numWin); i++ { + ws = append(ws, d.IntervalWindow()) + } + } - clear := d.Bool() - hold := mtime.MaxTimestamp - if clear { - return keyBytes, tag, nil - } + clear := d.Bool() + hold := mtime.MaxTimestamp + if clear { + if !yield(timerRet{keyBytes, tag, nil, ws}) { + return // Halt iteration if yeild returns false. + } + // Otherwise continue handling the remaining bytes. + raw = d.UnusedBytes() + continue + } - firing := d.Timestamp() - hold = d.Timestamp() - pane := d.Pane() + firing := d.Timestamp() + hold = d.Timestamp() + pane := d.Pane() + + var elms []element + for _, w := range ws { + elms = append(elms, element{ + tag: tag, + elmBytes: nil, // indicates this is a timer. + keyBytes: keyBytes, + window: w, + timestamp: firing, + holdTimestamp: hold, + pane: pane, + sequence: len(elms), + }) + } - var ret []element - for _, w := range ws { - ret = append(ret, element{ - tag: tag, - elmBytes: nil, // indicates this is a timer. - keyBytes: keyBytes, - window: w, - timestamp: firing, - holdTimestamp: hold, - pane: pane, - sequence: len(ret), - }) + if !yield(timerRet{keyBytes, tag, elms, ws}) { + return // Halt iteration if yeild returns false. + } + // Otherwise continue handling the remaining bytes. + raw = d.UnusedBytes() + } } - return keyBytes, tag, ret } type decoder struct { @@ -140,6 +164,13 @@ func (d *decoder) Bytes() []byte { return b } +// UnusedBytes returns the remainder of bytes in the buffer that weren't yet used. +// Multiple timers can be provided in a single timers buffer, since multiple dynamic +// timer tags may be set. +func (d *decoder) UnusedBytes() []byte { + return d.raw[d.cursor:] +} + func (d *decoder) Bool() bool { if b := d.Byte(); b == 0 { return false diff --git a/sdks/python/apache_beam/runners/portability/prism_runner_test.py b/sdks/python/apache_beam/runners/portability/prism_runner_test.py index 324fe5a17b545..b179156877e46 100644 --- a/sdks/python/apache_beam/runners/portability/prism_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/prism_runner_test.py @@ -40,6 +40,7 @@ from apache_beam.runners.portability import portable_runner_test from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms import userstate from apache_beam.transforms import window from apache_beam.transforms.sql import SqlTransform from apache_beam.utils import timestamp @@ -200,6 +201,37 @@ def test_windowing(self): assert_that( res, equal_to([('k', [1, 2]), ('k', [100, 101, 102]), ('k', [123])])) + # The fn_runner_test.py version of this test doesn't execute the process + # method for some reason. Overridden here to validate that the cleared + # timer won't re-fire. + def test_pardo_timers_clear(self): + timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) + + class TimerDoFn(beam.DoFn): + def process(self, element, timer=beam.DoFn.TimerParam(timer_spec)): + unused_key, ts = element + timer.set(ts) + timer.set(2 * ts) + + @userstate.on_timer(timer_spec) + def process_timer( + self, + ts=beam.DoFn.TimestampParam, + timer=beam.DoFn.TimerParam(timer_spec)): + timer.set(timestamp.Timestamp(micros=2 * ts.micros)) + timer.clear() # Shouldn't fire again + yield 'fired' + + with self.create_pipeline() as p: + actual = ( + p + | beam.Create([('k1', 10), ('k2', 100)]) + | beam.ParDo(TimerDoFn()) + | beam.Map(lambda x, ts=beam.DoFn.TimestampParam: (x, ts))) + + expected = [('fired', ts) for ts in (20, 200)] + assert_that(actual, equal_to(expected)) + # Can't read host files from within docker, read a "local" file there. def test_read(self): print('name:', __name__) From b21a84a4cd607f58a3794e274a913ca48da2e42c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Fri, 9 Aug 2024 17:38:35 -0400 Subject: [PATCH 57/78] Managed Iceberg hive support and integration tests (#32052) * iceberg hive support and integration tests * split read and write tests; cleanup * add test documentation * extend new config_properties arg to translation tests * revert beam schema override * actually run hive ITs * trigger integration tests * cut down hive database source lines --- .../IO_Iceberg_Integration_Tests.json | 2 +- .../IO_Iceberg_Integration_Tests.yml | 2 +- sdks/java/io/iceberg/build.gradle | 5 + sdks/java/io/iceberg/hive/build.gradle | 80 +++++ sdks/java/io/iceberg/hive/exec/build.gradle | 58 ++++ .../io/iceberg/hive/IcebergHiveCatalogIT.java | 280 ++++++++++++++++++ .../testutils/HiveMetastoreExtension.java | 68 +++++ .../iceberg/hive/testutils/ScriptRunner.java | 203 +++++++++++++ .../hive/testutils/TestHiveMetastore.java | 273 +++++++++++++++++ .../resources/hive-schema-3.1.0.derby.sql | 267 +++++++++++++++++ .../sdk/io/iceberg/IcebergCatalogConfig.java | 37 ++- .../IcebergReadSchemaTransformProvider.java | 56 +--- .../IcebergWriteSchemaTransformProvider.java | 58 +--- .../iceberg/SchemaTransformConfiguration.java | 69 +++++ .../sdk/io/iceberg/IcebergIOReadTest.java | 16 +- .../sdk/io/iceberg/IcebergIOWriteTest.java | 40 ++- ...cebergReadSchemaTransformProviderTest.java | 4 +- ...IcebergSchemaTransformTranslationTest.java | 6 + ...ebergWriteSchemaTransformProviderTest.java | 5 +- .../beam/sdk/io/iceberg/ScanSourceTest.java | 33 ++- settings.gradle.kts | 4 + 21 files changed, 1423 insertions(+), 143 deletions(-) create mode 100644 sdks/java/io/iceberg/hive/build.gradle create mode 100644 sdks/java/io/iceberg/hive/exec/build.gradle create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java create mode 100644 sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index bbdc3a3910ef8..62ae7886c5731 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 4 } diff --git a/.github/workflows/IO_Iceberg_Integration_Tests.yml b/.github/workflows/IO_Iceberg_Integration_Tests.yml index 20d1f4bb60fd3..22b2b4f9287d5 100644 --- a/.github/workflows/IO_Iceberg_Integration_Tests.yml +++ b/.github/workflows/IO_Iceberg_Integration_Tests.yml @@ -75,4 +75,4 @@ jobs: - name: Run IcebergIO Integration Test uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:java:io:iceberg:integrationTest \ No newline at end of file + gradle-command: :sdks:java:io:iceberg:catalogTests \ No newline at end of file diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 7965cde86e7d9..3d653d6b276e9 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -115,6 +115,11 @@ task integrationTest(type: Test) { testClassesDirs = sourceSets.test.output.classesDirs } +tasks.register('catalogTests') { + dependsOn integrationTest + dependsOn ":sdks:java:io:iceberg:hive:integrationTest" +} + task loadTest(type: Test) { def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpTempLocation = project.findProperty('gcpTempLocation') ?: 'gs://temp-storage-for-end-to-end-tests/temp-lt' diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle new file mode 100644 index 0000000000000..b81867ec90ca3 --- /dev/null +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -0,0 +1,80 @@ +import groovy.json.JsonOutput + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.iceberg.hive', + exportJavadoc: false, + shadowClosure: {}, +) + +description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" +ext.summary = "Runtime dependencies needed for Hive catalog integration." + +def hive_version = "3.1.3" +def iceberg_version = "1.4.2" + +dependencies { + // dependencies needed to run with iceberg's hive catalog + runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + runtimeOnly library.java.bigdataoss_gcs_connector + runtimeOnly library.java.hadoop_client + + // ----- below dependencies are for testing and will not appear in the shaded jar ----- + // Beam IcebergIO dependencies + testImplementation project(path: ":sdks:java:core", configuration: "shadow") + testImplementation project(":sdks:java:managed") + testImplementation project(":sdks:java:io:iceberg") + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testRuntimeOnly library.java.snake_yaml + + // needed to set up the test environment + testImplementation "org.apache.iceberg:iceberg-common:$iceberg_version" + testImplementation "org.apache.iceberg:iceberg-core:$iceberg_version" + testImplementation "org.assertj:assertj-core:3.11.1" + testImplementation library.java.junit + + // needed to set up test Hive Metastore and run tests + testImplementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + testImplementation project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + testRuntimeOnly ("org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version") { + exclude group: "org.apache.hive", module: "hive-exec" + exclude group: "org.apache.parquet", module: "parquet-hadoop-bundle" + } + testImplementation "org.apache.iceberg:iceberg-parquet:$iceberg_version" + testImplementation "org.apache.parquet:parquet-column:1.12.0" +} + +task integrationTest(type: Test) { + group = "Verification" + def gcpTempLocation = project.findProperty('gcpTempLocation') ?: 'gs://temp-storage-for-end-to-end-tests/iceberg-hive-it' + systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ + "--tempLocation=${gcpTempLocation}", + ]) + + // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" + outputs.upToDateWhen { false } + + include '**/*IT.class' + + maxParallelForks 4 + classpath = sourceSets.test.runtimeClasspath + testClassesDirs = sourceSets.test.output.classesDirs +} \ No newline at end of file diff --git a/sdks/java/io/iceberg/hive/exec/build.gradle b/sdks/java/io/iceberg/hive/exec/build.gradle new file mode 100644 index 0000000000000..581f71ddedd1f --- /dev/null +++ b/sdks/java/io/iceberg/hive/exec/build.gradle @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +plugins { + id 'org.apache.beam.module' + id 'java' + id 'com.github.johnrengelman.shadow' +} + +dependencies { + implementation("org.apache.hive:hive-exec:3.1.3") + permitUnusedDeclared("org.apache.hive:hive-exec:3.1.3") +} + +configurations { + shadow +} + +artifacts { + shadow(archives(shadowJar) { + builtBy shadowJar + }) +} + +shadowJar { + zip64 true + + // need to shade "com.google.guava" to avoid Guava conflict + relocate 'com.google.protobuf', getJavaRelocatedPath('com.google.protobuf') + relocate 'shaded.parquet', getJavaRelocatedPath('shaded.parquet') + relocate 'org.apache.parquet', getJavaRelocatedPath('org.apache.parquet') + + version "3.1.3" + mergeServiceFiles() + + exclude 'LICENSE' + exclude( + 'org/xml/**', + 'javax/**', + 'com/sun/**' + ) +} +description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive :: Exec" +ext.summary = "A copy of the hive-exec dependency with some popular libraries relocated." diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java new file mode 100644 index 0000000000000..54a4998d37fba --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.iceberg.hive; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.hive.testutils.HiveMetastoreExtension; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.thrift.TException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +/** + * Read and write test for {@link Managed} {@link org.apache.beam.sdk.io.iceberg.IcebergIO} using + * {@link HiveCatalog}. + * + *

    Spins up a local Hive metastore to manage the Iceberg table. Warehouse path is set to a GCS + * bucket. + */ +public class IcebergHiveCatalogIT { + private static final Schema DOUBLY_NESTED_ROW_SCHEMA = + Schema.builder() + .addStringField("doubly_nested_str") + .addInt64Field("doubly_nested_float") + .build(); + + private static final Schema NESTED_ROW_SCHEMA = + Schema.builder() + .addStringField("nested_str") + .addInt32Field("nested_int") + .addFloatField("nested_float") + .addRowField("nested_row", DOUBLY_NESTED_ROW_SCHEMA) + .build(); + private static final Schema BEAM_SCHEMA = + Schema.builder() + .addStringField("str") + .addBooleanField("bool") + .addNullableInt32Field("nullable_int") + .addNullableInt64Field("nullable_long") + .addArrayField("arr_long", Schema.FieldType.INT64) + .addRowField("row", NESTED_ROW_SCHEMA) + .addNullableRowField("nullable_row", NESTED_ROW_SCHEMA) + .build(); + + private static final SimpleFunction ROW_FUNC = + new SimpleFunction() { + @Override + public Row apply(Long num) { + String strNum = Long.toString(num); + Row nestedRow = + Row.withSchema(NESTED_ROW_SCHEMA) + .addValue("nested_str_value_" + strNum) + .addValue(Integer.valueOf(strNum)) + .addValue(Float.valueOf(strNum + "." + strNum)) + .addValue( + Row.withSchema(DOUBLY_NESTED_ROW_SCHEMA) + .addValue("doubly_nested_str_value_" + strNum) + .addValue(num) + .build()) + .build(); + + return Row.withSchema(BEAM_SCHEMA) + .addValue("str_value_" + strNum) + .addValue(num % 2 == 0) + .addValue(Integer.valueOf(strNum)) + .addValue(num) + .addValue(LongStream.range(1, num % 10).boxed().collect(Collectors.toList())) + .addValue(nestedRow) + .addValue(num % 2 == 0 ? null : nestedRow) + .build(); + } + }; + + private static final org.apache.iceberg.Schema ICEBERG_SCHEMA = + IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA); + private static final SimpleFunction RECORD_FUNC = + new SimpleFunction() { + @Override + public Record apply(Row input) { + return IcebergUtils.beamRowToIcebergRecord(ICEBERG_SCHEMA, input); + } + }; + + private static HiveMetastoreExtension hiveMetastoreExtension; + + @Rule public TestPipeline writePipeline = TestPipeline.create(); + + @Rule public TestPipeline readPipeline = TestPipeline.create(); + + private static final String TEST_CATALOG = "test_catalog"; + private static final String TEST_TABLE = "test_table"; + private static HiveCatalog catalog; + private static final String TEST_DB = "test_db_" + System.nanoTime(); + + @BeforeClass + public static void setUp() throws TException { + String warehousePath = TestPipeline.testingPipelineOptions().getTempLocation(); + hiveMetastoreExtension = new HiveMetastoreExtension(warehousePath); + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), + TEST_CATALOG, + ImmutableMap.of( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, + String.valueOf(TimeUnit.SECONDS.toMillis(10))), + hiveMetastoreExtension.hiveConf()); + + String dbPath = hiveMetastoreExtension.metastore().getDatabasePath(TEST_DB); + Database db = new Database(TEST_DB, "description", dbPath, Maps.newHashMap()); + hiveMetastoreExtension.metastoreClient().createDatabase(db); + } + + @AfterClass + public static void cleanup() throws Exception { + hiveMetastoreExtension.cleanup(); + } + + private Map getManagedIcebergConfig(TableIdentifier table) { + String metastoreUri = hiveMetastoreExtension.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS); + + Map confProperties = + ImmutableMap.builder() + .put(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUri) + .build(); + + return ImmutableMap.builder() + .put("table", table.toString()) + .put("config_properties", confProperties) + .build(); + } + + @Test + public void testReadWithHiveCatalog() throws IOException { + TableIdentifier tableIdentifier = + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE + "_read_test")); + Table table = catalog.createTable(tableIdentifier, ICEBERG_SCHEMA); + + List expectedRows = + LongStream.range(1, 1000).boxed().map(ROW_FUNC::apply).collect(Collectors.toList()); + List records = + expectedRows.stream().map(RECORD_FUNC::apply).collect(Collectors.toList()); + + // write iceberg records with hive catalog + String filepath = table.location() + "/" + UUID.randomUUID(); + DataWriter writer = + Parquet.writeData(table.io().newOutputFile(filepath)) + .schema(ICEBERG_SCHEMA) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .withSpec(table.spec()) + .build(); + for (Record rec : records) { + writer.write(rec); + } + writer.close(); + AppendFiles appendFiles = table.newAppend(); + String manifestFilename = FileFormat.AVRO.addExtension(filepath + ".manifest"); + OutputFile outputFile = table.io().newOutputFile(manifestFilename); + ManifestWriter manifestWriter; + try (ManifestWriter openWriter = ManifestFiles.write(table.spec(), outputFile)) { + openWriter.add(writer.toDataFile()); + manifestWriter = openWriter; + } + appendFiles.appendManifest(manifestWriter.toManifestFile()); + appendFiles.commit(); + + // Run Managed Iceberg read + PCollection outputRows = + readPipeline + .apply( + Managed.read(Managed.ICEBERG).withConfig(getManagedIcebergConfig(tableIdentifier))) + .getSinglePCollection(); + PAssert.that(outputRows).containsInAnyOrder(expectedRows); + readPipeline.run().waitUntilFinish(); + } + + @Test + public void testWriteWithHiveCatalog() { + TableIdentifier tableIdentifier = + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE + "_write_test")); + catalog.createTable(tableIdentifier, IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA)); + + List inputRows = + LongStream.range(1, 1000).mapToObj(ROW_FUNC::apply).collect(Collectors.toList()); + List expectedRecords = + inputRows.stream().map(RECORD_FUNC::apply).collect(Collectors.toList()); + + // Run Managed Iceberg write + writePipeline + .apply(Create.of(inputRows)) + .setRowSchema(BEAM_SCHEMA) + .apply(Managed.write(Managed.ICEBERG).withConfig(getManagedIcebergConfig(tableIdentifier))); + writePipeline.run().waitUntilFinish(); + + // read back the records and check everything's there + Table table = catalog.loadTable(tableIdentifier); + TableScan tableScan = table.newScan().project(ICEBERG_SCHEMA); + List writtenRecords = new ArrayList<>(); + for (CombinedScanTask task : tableScan.planTasks()) { + InputFilesDecryptor decryptor = new InputFilesDecryptor(task, table.io(), table.encryption()); + for (FileScanTask fileTask : task.files()) { + InputFile inputFile = decryptor.getInputFile(fileTask); + CloseableIterable iterable = + Parquet.read(inputFile) + .split(fileTask.start(), fileTask.length()) + .project(ICEBERG_SCHEMA) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(ICEBERG_SCHEMA, fileSchema)) + .filter(fileTask.residual()) + .build(); + + for (Record rec : iterable) { + writtenRecords.add(rec); + } + } + } + assertThat(expectedRecords, containsInAnyOrder(writtenRecords.toArray())); + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java new file mode 100644 index 0000000000000..52de1b91a216a --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg.hive.testutils; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; + +/** + * A class that interacts with {@link TestHiveMetastore}. + * + *

    Trimmed down from Iceberg's + * integration testing util + */ +public class HiveMetastoreExtension { + private HiveMetaStoreClient metastoreClient; + private TestHiveMetastore metastore; + + public HiveMetastoreExtension(String warehousePath) throws MetaException { + metastore = new TestHiveMetastore(warehousePath); + HiveConf hiveConf = new HiveConf(TestHiveMetastore.class); + + metastore.start(hiveConf); + metastoreClient = new HiveMetaStoreClient(hiveConf); + } + + public void cleanup() throws Exception { + if (metastoreClient != null) { + metastoreClient.close(); + } + + if (metastore != null) { + metastore.reset(); + metastore.stop(); + } + + metastoreClient = null; + metastore = null; + } + + public HiveMetaStoreClient metastoreClient() { + return metastoreClient; + } + + public HiveConf hiveConf() { + return metastore.hiveConf(); + } + + public TestHiveMetastore metastore() { + return metastore; + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java new file mode 100644 index 0000000000000..adf941e00b4b6 --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.iceberg.hive.testutils; + +import java.io.IOException; +import java.io.LineNumberReader; +import java.io.PrintWriter; +import java.io.Reader; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; + +/** + * Tool to run database scripts. + * + *

    Copied over from Iceberg's + * integration testing + */ +@SuppressWarnings({"OperatorPrecedence", "DefaultCharset"}) +public class ScriptRunner { + + private static final String DEFAULT_DELIMITER = ";"; + + private final Connection connection; + + private final boolean stopOnError; + private final boolean autoCommit; + + private final PrintWriter logWriter = new PrintWriter(System.out); + private final PrintWriter errorLogWriter = new PrintWriter(System.err); + + /** Default constructor. */ + public ScriptRunner(Connection connection, boolean autoCommit, boolean stopOnError) { + this.connection = connection; + this.autoCommit = autoCommit; + this.stopOnError = stopOnError; + } + + /** + * Runs an SQL script (read in using the Reader parameter). + * + * @param reader - the source of the script + */ + public void runScript(Reader reader) throws IOException, SQLException { + try { + boolean originalAutoCommit = connection.getAutoCommit(); + try { + if (originalAutoCommit != this.autoCommit) { + connection.setAutoCommit(this.autoCommit); + } + runScript(connection, reader); + } finally { + connection.setAutoCommit(originalAutoCommit); + } + } catch (IOException | SQLException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException("Error running script. Cause: " + e, e); + } + } + + /** + * Runs an SQL script (read in using the Reader parameter) using the connection passed in. + * + * @param conn - the connection to use for the script + * @param reader - the source of the script + * @throws SQLException if any SQL errors occur + * @throws IOException if there is an error reading from the Reader + */ + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private void runScript(Connection conn, Reader reader) throws IOException, SQLException { + StringBuilder command = null; + try { + LineNumberReader lineReader = new LineNumberReader(reader); + String line; + while ((line = lineReader.readLine()) != null) { + if (command == null) { + command = new StringBuilder(); + } + String trimmedLine = line.trim(); + boolean fullLineDelimiter = false; + if (trimmedLine.startsWith("--")) { + println(trimmedLine); + } else if (trimmedLine.isEmpty() || trimmedLine.startsWith("//")) { + // Do nothing + } else if (!fullLineDelimiter && trimmedLine.endsWith(getDelimiter()) + || fullLineDelimiter && trimmedLine.equals(getDelimiter())) { + command.append(line, 0, line.lastIndexOf(getDelimiter())); + command.append(" "); + Statement statement = conn.createStatement(); + + println(command); + + boolean hasResults = false; + if (stopOnError) { + hasResults = statement.execute(command.toString()); + } else { + try { + statement.execute(command.toString()); + } catch (SQLException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + } + } + + if (autoCommit && !conn.getAutoCommit()) { + conn.commit(); + } + + ResultSet rs = statement.getResultSet(); + if (hasResults && rs != null) { + ResultSetMetaData md = rs.getMetaData(); + int cols = md.getColumnCount(); + for (int i = 0; i < cols; i++) { + String name = md.getColumnLabel(i); + print(name + "\t"); + } + println(""); + while (rs.next()) { + for (int i = 0; i < cols; i++) { + String value = rs.getString(i); + print(value + "\t"); + } + println(""); + } + } + + command = null; + try { + statement.close(); + } catch (Exception e) { + // Ignore to workaround a bug in Jakarta DBCP + } + Thread.yield(); + } else { + command.append(line); + command.append(" "); + } + } + if (!autoCommit) { + conn.commit(); + } + } catch (IOException | SQLException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + throw e; + } finally { + conn.rollback(); + flush(); + } + } + + private String getDelimiter() { + return DEFAULT_DELIMITER; + } + + private void print(Object obj) { + if (logWriter != null) { + System.out.print(obj); + } + } + + private void println(Object obj) { + if (logWriter != null) { + logWriter.println(obj); + } + } + + private void printlnError(Object obj) { + if (errorLogWriter != null) { + errorLogWriter.println(obj); + } + } + + private void flush() { + if (logWriter != null) { + logWriter.flush(); + } + if (errorLogWriter != null) { + errorLogWriter.flush(); + } + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java new file mode 100644 index 0000000000000..e3af43d58c65f --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg.hive.testutils; + +import static java.nio.file.Files.createTempDirectory; +import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; +import static java.nio.file.attribute.PosixFilePermissions.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.IHMSHandler; +import org.apache.hadoop.hive.metastore.RetryingHMSHandler; +import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.hive.HiveClientPool; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransportFactory; + +/** + * A Hive Metastore implementation for local testing. Not meant to be used directly. Use {@link + * HiveMetastoreExtension} instead. + * + *

    Copied over from Iceberg's + * integration testing util + */ +public class TestHiveMetastore { + + private static final String DEFAULT_DATABASE_NAME = "default"; + private static final int DEFAULT_POOL_SIZE = 5; + + // create the metastore handlers based on whether we're working with Hive2 or Hive3 dependencies + // we need to do this because there is a breaking API change between Hive2 and Hive3 + private static final DynConstructors.Ctor HMS_HANDLER_CTOR = + DynConstructors.builder() + .impl(HiveMetaStore.HMSHandler.class, String.class, Configuration.class) + .impl(HiveMetaStore.HMSHandler.class, String.class, HiveConf.class) + .build(); + + private static final DynMethods.StaticMethod GET_BASE_HMS_HANDLER = + DynMethods.builder("getProxy") + .impl(RetryingHMSHandler.class, Configuration.class, IHMSHandler.class, boolean.class) + .impl(RetryingHMSHandler.class, HiveConf.class, IHMSHandler.class, boolean.class) + .buildStatic(); + + // Hive3 introduces background metastore tasks (MetastoreTaskThread) for performing various + // cleanup duties. These + // threads are scheduled and executed in a static thread pool + // (org.apache.hadoop.hive.metastore.ThreadPool). + // This thread pool is shut down normally as part of the JVM shutdown hook, but since we're + // creating and tearing down + // multiple metastore instances within the same JVM, we have to call this cleanup method manually, + // otherwise + // threads from our previous test suite will be stuck in the pool with stale config, and keep on + // being scheduled. + // This can lead to issues, e.g. accidental Persistence Manager closure by + // ScheduledQueryExecutionsMaintTask. + private static final DynMethods.StaticMethod METASTORE_THREADS_SHUTDOWN = + DynMethods.builder("shutdown") + .impl("org.apache.hadoop.hive.metastore.ThreadPool") + .orNoop() + .buildStatic(); + + // It's tricky to clear all static fields in an HMS instance in order to switch derby root dir. + // Therefore, we reuse the same derby root between tests and remove it after JVM exits. + private static final File HIVE_LOCAL_DIR; + private static final String DERBY_PATH; + + static { + try { + HIVE_LOCAL_DIR = + createTempDirectory("hive", asFileAttribute(fromString("rwxrwxrwx"))).toFile(); + DERBY_PATH = HIVE_LOCAL_DIR + "/metastore_db"; + File derbyLogFile = new File(HIVE_LOCAL_DIR, "derby.log"); + System.setProperty("derby.stream.error.file", derbyLogFile.getAbsolutePath()); + setupMetastoreDB("jdbc:derby:" + DERBY_PATH + ";create=true"); + Runtime.getRuntime() + .addShutdownHook( + new Thread( + () -> { + Path localDirPath = new Path(HIVE_LOCAL_DIR.getAbsolutePath()); + FileSystem fs = Util.getFs(localDirPath, new Configuration()); + String errMsg = "Failed to delete " + localDirPath; + try { + assertThat(fs.delete(localDirPath, true)).as(errMsg).isTrue(); + } catch (IOException e) { + throw new RuntimeException(errMsg, e); + } + })); + } catch (Exception e) { + throw new RuntimeException("Failed to setup local dir for hive metastore", e); + } + } + + private HiveConf hiveConf; + private ExecutorService executorService; + private TServer server; + private HiveMetaStore.HMSHandler baseHandler; + private HiveClientPool clientPool; + private final String hiveWarehousePath; + + TestHiveMetastore(String hiveWarehousePath) { + this.hiveWarehousePath = hiveWarehousePath; + } + + /** + * Starts a TestHiveMetastore with the default connection pool size (5) with the provided + * HiveConf. + * + * @param conf The hive configuration to use + */ + public void start(HiveConf conf) { + start(conf, DEFAULT_POOL_SIZE); + } + + /** + * Starts a TestHiveMetastore with a provided connection pool size and HiveConf. + * + * @param conf The hive configuration to use + * @param poolSize The number of threads in the executor pool + */ + @SuppressWarnings("FutureReturnValueIgnored") + public void start(HiveConf conf, int poolSize) { + try { + TServerSocket socket = new TServerSocket(0); + int port = socket.getServerSocket().getLocalPort(); + initConf(conf, port); + + this.hiveConf = conf; + this.server = newThriftServer(socket, poolSize, hiveConf); + this.executorService = Executors.newSingleThreadExecutor(); + this.executorService.submit(() -> server.serve()); + this.clientPool = new HiveClientPool(1, hiveConf); + } catch (Exception e) { + throw new RuntimeException("Cannot start TestHiveMetastore", e); + } + } + + public void stop() throws Exception { + reset(); + if (clientPool != null) { + clientPool.close(); + } + if (server != null) { + server.stop(); + } + if (executorService != null) { + executorService.shutdown(); + } + if (baseHandler != null) { + baseHandler.shutdown(); + } + METASTORE_THREADS_SHUTDOWN.invoke(); + } + + public HiveConf hiveConf() { + return hiveConf; + } + + public String getDatabasePath(String dbName) { + return hiveWarehousePath + "/" + dbName + ".db"; + } + + public void reset() throws Exception { + if (clientPool != null) { + for (String dbName : clientPool.run(client -> client.getAllDatabases())) { + for (String tblName : clientPool.run(client -> client.getAllTables(dbName))) { + clientPool.run( + client -> { + client.dropTable(dbName, tblName, true, true, true); + return null; + }); + } + + if (!DEFAULT_DATABASE_NAME.equals(dbName)) { + // Drop cascade, functions dropped by cascade + clientPool.run( + client -> { + client.dropDatabase(dbName, true, true, true); + return null; + }); + } + } + } + + Path warehouseRoot = new Path(hiveWarehousePath); + FileSystem fs = Util.getFs(warehouseRoot, hiveConf); + for (FileStatus fileStatus : fs.listStatus(warehouseRoot)) { + if (!fileStatus.getPath().getName().equals("derby.log") + && !fileStatus.getPath().getName().equals("metastore_db")) { + fs.delete(fileStatus.getPath(), true); + } + } + } + + private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf conf) + throws Exception { + HiveConf serverConf = new HiveConf(conf); + serverConf.set( + HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + "jdbc:derby:" + DERBY_PATH + ";create=true"); + baseHandler = HMS_HANDLER_CTOR.newInstance("new db based metaserver", serverConf); + IHMSHandler handler = GET_BASE_HMS_HANDLER.invoke(serverConf, baseHandler, false); + + TThreadPoolServer.Args args = + new TThreadPoolServer.Args(socket) + .processor(new TSetIpAddressProcessor<>(handler)) + .transportFactory(new TTransportFactory()) + .protocolFactory(new TBinaryProtocol.Factory()) + .minWorkerThreads(poolSize) + .maxWorkerThreads(poolSize); + + return new TThreadPoolServer(args); + } + + private void initConf(HiveConf conf, int port) { + conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://localhost:" + port); + conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, hiveWarehousePath); + conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "false"); + conf.set(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES.varname, "false"); + conf.set("iceberg.hive.client-pool-size", "2"); + // Setting this to avoid thrift exception during running Iceberg tests outside Iceberg. + conf.set( + HiveConf.ConfVars.HIVE_IN_TEST.varname, HiveConf.ConfVars.HIVE_IN_TEST.getDefaultValue()); + } + + private static void setupMetastoreDB(String dbURL) throws SQLException, IOException { + Connection connection = DriverManager.getConnection(dbURL); + ScriptRunner scriptRunner = new ScriptRunner(connection, true, true); + + ClassLoader classLoader = ClassLoader.getSystemClassLoader(); + InputStream inputStream = classLoader.getResourceAsStream("hive-schema-3.1.0.derby.sql"); + try (Reader reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8)) { + scriptRunner.runScript(reader); + } + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql new file mode 100644 index 0000000000000..808c605857648 --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql @@ -0,0 +1,267 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- +-- This file was copied from Apache Hive, at: +-- https://github.com/apache/hive/blob/master/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-3.1.0.derby.sql +-- +-- This has been modified slightly for compatibility with older Hive versions. +-- +-- Timestamp: 2011-09-22 15:32:02.024 +-- Source database is: /home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Connection URL is: jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Specified schema is: APP +-- appendLogs: false + +-- ---------------------------------------------- +-- DDL Statements for functions +-- ---------------------------------------------- + +CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ; + +CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ; + +-- ---------------------------------------------- +-- DDL Statements for tables +-- ---------------------------------------------- +CREATE TABLE "APP"."DBS" ( + "DB_ID" BIGINT NOT NULL, + "DESC" VARCHAR(4000), + "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, + "NAME" VARCHAR(128), + "OWNER_NAME" VARCHAR(128), + "OWNER_TYPE" VARCHAR(10), + "CTLG_NAME" VARCHAR(256) +); + +CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); + +CREATE TABLE "APP"."ROLE_MAP" ("ROLE_GRANT_ID" BIGINT NOT NULL, "ADD_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "ROLE_ID" BIGINT); + +CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "USER_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128)); + +CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "OWNER_TYPE" VARCHAR(10), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N'); + +CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000), "IS_COMPRESSED" CHAR(1) NOT NULL, "LOCATION" VARCHAR(4000), "NUM_BUCKETS" INTEGER NOT NULL, "OUTPUT_FORMAT" VARCHAR(4000), "SERDE_ID" BIGINT, "CD_ID" BIGINT, "IS_STOREDASSUBDIRECTORIES" CHAR(1) NOT NULL); + +CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL); + +CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."BUCKETING_COLS" ("SD_ID" BIGINT NOT NULL, "BUCKET_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SD_PARAMS" ("SD_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."SKEWED_STRING_LIST_VALUES" ("STRING_LIST_ID" BIGINT NOT NULL, "STRING_LIST_VALUE" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_NAMES" ("SD_ID" BIGINT NOT NULL, "SKEWED_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_LIST_ID_KID" BIGINT NOT NULL, "LOCATION" VARCHAR(4000)); + +CREATE TABLE "APP"."SKEWED_VALUES" ("SD_ID_OID" BIGINT NOT NULL, "STRING_LIST_ID_EID" BIGINT NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255)); + +CREATE TABLE "APP"."CTLGS" ( + "CTLG_ID" BIGINT NOT NULL, + "NAME" VARCHAR(256) UNIQUE, + "DESC" VARCHAR(4000), + "LOCATION_URI" VARCHAR(4000) NOT NULL); + +-- ---------------------------------------------- +-- DML Statements +-- ---------------------------------------------- + +INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_VAL" FROM "APP"."SEQUENCE_TABLE" WHERE "SEQUENCE_NAME" = 'org.apache.hadoop.hive.metastore.model.MNotificationLog'); + +-- ---------------------------------------------- +-- DDL Statements for indexes +-- ---------------------------------------------- + + +CREATE UNIQUE INDEX "APP"."ROLEENTITYINDEX" ON "APP"."ROLES" ("ROLE_NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME"); + +CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."GLOBALPRIVILEGEINDEX" ON "APP"."GLOBAL_PRIVS" ("AUTHORIZER", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); + + +-- ---------------------------------------------- +-- DDL Statements for keys +-- ---------------------------------------------- + +-- primary/unique +ALTER TABLE "APP"."CDS" ADD CONSTRAINT "SQL110922153006460" PRIMARY KEY ("CD_ID"); + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); + +ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_PK" PRIMARY KEY ("SD_ID"); + +ALTER TABLE "APP"."SERDES" ADD CONSTRAINT "SERDES_PK" PRIMARY KEY ("SERDE_ID"); + +ALTER TABLE "APP"."ROLES" ADD CONSTRAINT "ROLES_PK" PRIMARY KEY ("ROLE_ID"); + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_PK" PRIMARY KEY ("SERDE_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID"); + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_PK" PRIMARY KEY ("DB_ID"); + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_PK" PRIMARY KEY ("ROLE_GRANT_ID"); + +ALTER TABLE "APP"."GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_PK" PRIMARY KEY ("USER_GRANT_ID"); + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "SQL110922153006740" PRIMARY KEY ("CD_ID", "COLUMN_NAME"); + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_PK" PRIMARY KEY ("TBL_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST" ADD CONSTRAINT "SKEWED_STRING_LIST_PK" PRIMARY KEY ("STRING_LIST_ID"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_PK" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_PK" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID"); + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_PK" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX"); + +ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); + +-- foreign + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK2" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_FK1" FOREIGN KEY ("ROLE_ID") REFERENCES "APP"."ROLES" ("ROLE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "COLUMNS_V2_FK1" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_FK1" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY ("SD_ID_OID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK2" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID"); + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +-- ---------------------------------------------- +-- DDL Statements for checks +-- ---------------------------------------------- + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); + +-- ---------------------------- +-- Transaction and Lock Tables +-- ---------------------------- +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint NOT NULL, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + HL_HEARTBEAT_COUNT integer, + HL_AGENT_INFO varchar(128), + HL_BLOCKEDBY_EXT_ID bigint, + HL_BLOCKEDBY_INT_ID bigint, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE AUX_TABLE ( + MT_KEY1 varchar(128) NOT NULL, + MT_KEY2 bigint NOT NULL, + MT_COMMENT varchar(255), + PRIMARY KEY(MT_KEY1, MT_KEY2) +); + +--1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK +--This is a good candidate for Index orgainzed table + +-- ----------------------------------------------------------------- +-- Record schema version. Should be the last step in the init script +-- ----------------------------------------------------------------- +INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '3.1.0', 'Hive release version 3.1.0'); \ No newline at end of file diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 2956d75a266e7..5307047354b8b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -19,19 +19,27 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; -import java.util.Properties; +import java.util.Map; +import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; +import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @AutoValue public abstract class IcebergCatalogConfig implements Serializable { @Pure + @Nullable public abstract String getCatalogName(); @Pure - public abstract Properties getProperties(); + @Nullable + public abstract Map getCatalogProperties(); + + @Pure + @Nullable + public abstract Map getConfigProperties(); @Pure public static Builder builder() { @@ -39,15 +47,32 @@ public static Builder builder() { } public org.apache.iceberg.catalog.Catalog catalog() { - return CatalogUtil.buildIcebergCatalog( - getCatalogName(), Maps.fromProperties(getProperties()), new Configuration()); + String catalogName = getCatalogName(); + if (catalogName == null) { + catalogName = "apache-beam-" + ReleaseInfo.getReleaseInfo().getVersion(); + } + Map catalogProps = getCatalogProperties(); + if (catalogProps == null) { + catalogProps = Maps.newHashMap(); + } + Map confProps = getConfigProperties(); + if (confProps == null) { + confProps = Maps.newHashMap(); + } + Configuration config = new Configuration(); + for (Map.Entry prop : confProps.entrySet()) { + config.set(prop.getKey(), prop.getValue()); + } + return CatalogUtil.buildIcebergCatalog(catalogName, catalogProps, config); } @AutoValue.Builder public abstract static class Builder { - public abstract Builder setCatalogName(String catalogName); + public abstract Builder setCatalogName(@Nullable String catalogName); + + public abstract Builder setCatalogProperties(@Nullable Map props); - public abstract Builder setProperties(Properties props); + public abstract Builder setConfigProperties(@Nullable Map props); public abstract IcebergCatalogConfig build(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index ef535353efd01..df7bda4560dd5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -18,18 +18,11 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.service.AutoService; -import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Config; import org.apache.beam.sdk.managed.ManagedTransformConstants; -import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -44,11 +37,12 @@ * org.apache.beam.sdk.values.Row}s. */ @AutoService(SchemaTransformProvider.class) -public class IcebergReadSchemaTransformProvider extends TypedSchemaTransformProvider { +public class IcebergReadSchemaTransformProvider + extends TypedSchemaTransformProvider { static final String OUTPUT_TAG = "output"; @Override - protected SchemaTransform from(Config configuration) { + protected SchemaTransform from(SchemaTransformConfiguration configuration) { return new IcebergReadSchemaTransform(configuration); } @@ -62,38 +56,10 @@ public String identifier() { return ManagedTransformConstants.ICEBERG_READ; } - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class Config { - public static Builder builder() { - return new AutoValue_IcebergReadSchemaTransformProvider_Config.Builder(); - } - - @SchemaFieldDescription("Identifier of the Iceberg table to write to.") - public abstract String getTable(); - - @SchemaFieldDescription("Name of the catalog containing the table.") - public abstract String getCatalogName(); - - @SchemaFieldDescription("Configuration properties used to set up the Iceberg catalog.") - public abstract Map getCatalogProperties(); - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); - - public abstract Builder setCatalogName(String catalogName); - - public abstract Builder setCatalogProperties(Map catalogProperties); - - public abstract Config build(); - } - } - static class IcebergReadSchemaTransform extends SchemaTransform { - private final Config configuration; + private final SchemaTransformConfiguration configuration; - IcebergReadSchemaTransform(Config configuration) { + IcebergReadSchemaTransform(SchemaTransformConfiguration configuration) { this.configuration = configuration; } @@ -102,7 +68,7 @@ Row getConfigurationRow() { // To stay consistent with our SchemaTransform configuration naming conventions, // we sort lexicographically and convert field names to snake_case return SchemaRegistry.createDefault() - .getToRowFunction(Config.class) + .getToRowFunction(SchemaTransformConfiguration.class) .apply(configuration) .sorted() .toSnakeCase(); @@ -113,19 +79,11 @@ Row getConfigurationRow() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - Properties properties = new Properties(); - properties.putAll(configuration.getCatalogProperties()); - - IcebergCatalogConfig.Builder catalogBuilder = - IcebergCatalogConfig.builder() - .setCatalogName(configuration.getCatalogName()) - .setProperties(properties); - PCollection output = input .getPipeline() .apply( - IcebergIO.readRows(catalogBuilder.build()) + IcebergIO.readRows(configuration.getIcebergCatalog()) .from(TableIdentifier.parse(configuration.getTable()))); return PCollectionRowTuple.of(OUTPUT_TAG, output); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index b3de7a88c541d..3f0f88946d9ca 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -18,19 +18,12 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.service.AutoService; -import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import org.apache.beam.sdk.managed.ManagedTransformConstants; -import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -48,7 +41,8 @@ * outputs a {@code PCollection} representing snapshots created in the process. */ @AutoService(SchemaTransformProvider.class) -public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformProvider { +public class IcebergWriteSchemaTransformProvider + extends TypedSchemaTransformProvider { static final String INPUT_TAG = "input"; static final String OUTPUT_TAG = "output"; @@ -64,7 +58,7 @@ public String description() { } @Override - protected SchemaTransform from(Config configuration) { + protected SchemaTransform from(SchemaTransformConfiguration configuration) { return new IcebergWriteSchemaTransform(configuration); } @@ -83,38 +77,10 @@ public String identifier() { return ManagedTransformConstants.ICEBERG_WRITE; } - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class Config { - public static Builder builder() { - return new AutoValue_IcebergWriteSchemaTransformProvider_Config.Builder(); - } - - @SchemaFieldDescription("Identifier of the Iceberg table to write to.") - public abstract String getTable(); - - @SchemaFieldDescription("Name of the catalog containing the table.") - public abstract String getCatalogName(); - - @SchemaFieldDescription("Configuration properties used to set up the Iceberg catalog.") - public abstract Map getCatalogProperties(); - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); - - public abstract Builder setCatalogName(String catalogName); - - public abstract Builder setCatalogProperties(Map catalogProperties); - - public abstract Config build(); - } - } - static class IcebergWriteSchemaTransform extends SchemaTransform { - private final Config configuration; + private final SchemaTransformConfiguration configuration; - IcebergWriteSchemaTransform(Config configuration) { + IcebergWriteSchemaTransform(SchemaTransformConfiguration configuration) { this.configuration = configuration; } @@ -123,7 +89,7 @@ Row getConfigurationRow() { // To stay consistent with our SchemaTransform configuration naming conventions, // we sort lexicographically and convert field names to snake_case return SchemaRegistry.createDefault() - .getToRowFunction(Config.class) + .getToRowFunction(SchemaTransformConfiguration.class) .apply(configuration) .sorted() .toSnakeCase(); @@ -136,19 +102,11 @@ Row getConfigurationRow() { public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection rows = input.get(INPUT_TAG); - Properties properties = new Properties(); - properties.putAll(configuration.getCatalogProperties()); - - IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder() - .setCatalogName(configuration.getCatalogName()) - .setProperties(properties) - .build(); - // TODO: support dynamic destinations IcebergWriteResult result = rows.apply( - IcebergIO.writeRows(catalog).to(TableIdentifier.parse(configuration.getTable()))); + IcebergIO.writeRows(configuration.getIcebergCatalog()) + .to(TableIdentifier.parse(configuration.getTable()))); PCollection snapshots = result diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java new file mode 100644 index 0000000000000..6e7a12aa15afc --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.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.io.iceberg; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.checkerframework.checker.nullness.qual.Nullable; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SchemaTransformConfiguration { + public static Builder builder() { + return new AutoValue_SchemaTransformConfiguration.Builder(); + } + + @SchemaFieldDescription("Identifier of the Iceberg table.") + public abstract String getTable(); + + @SchemaFieldDescription("Name of the catalog containing the table.") + @Nullable + public abstract String getCatalogName(); + + @SchemaFieldDescription("Properties used to set up the Iceberg catalog.") + @Nullable + public abstract Map getCatalogProperties(); + + @SchemaFieldDescription("Properties passed to the Hadoop Configuration.") + @Nullable + public abstract Map getConfigProperties(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String table); + + public abstract Builder setCatalogName(String catalogName); + + public abstract Builder setCatalogProperties(Map catalogProperties); + + public abstract Builder setConfigProperties(Map confProperties); + + public abstract SchemaTransformConfiguration build(); + } + + public IcebergCatalogConfig getIcebergCatalog() { + return IcebergCatalogConfig.builder() + .setCatalogName(getCatalogName()) + .setCatalogProperties(getCatalogProperties()) + .setConfigProperties(getConfigProperties()) + .build(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java index 3f31073b44486..fe4a07dedfdf9 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java @@ -21,7 +21,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import java.util.List; -import java.util.Properties; +import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -94,12 +95,17 @@ public void testSimpleScan() throws Exception { .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .collect(Collectors.toList()); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); PCollection output = testPipeline diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java index 02213c45e0756..2abe6b0934819 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java @@ -23,7 +23,6 @@ import java.io.Serializable; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.UUID; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; @@ -76,12 +75,17 @@ public void testSimpleAppend() throws Exception { // Create a table and add records to it. Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); testPipeline .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) @@ -110,12 +114,17 @@ public void testDynamicDestinationsWithoutSpillover() throws Exception { Table table2 = warehouse.createTable(table2Id, TestFixtures.SCHEMA); Table table3 = warehouse.createTable(table3Id, TestFixtures.SCHEMA); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); DynamicDestinations dynamicDestinations = new DynamicDestinations() { @@ -200,12 +209,17 @@ public void testDynamicDestinationsWithSpillover() throws Exception { elementsPerTable.computeIfAbsent(tableId, ignored -> Lists.newArrayList()).add(element); } - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); DynamicDestinations dynamicDestinations = new DynamicDestinations() { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java index effb5cc4838e9..0311c31da4058 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -101,8 +101,8 @@ public void testSimpleScan() throws Exception { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - IcebergReadSchemaTransformProvider.Config readConfig = - IcebergReadSchemaTransformProvider.Config.builder() + SchemaTransformConfiguration readConfig = + SchemaTransformConfiguration.builder() .setTable(identifier) .setCatalogName("name") .setCatalogProperties(properties) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java index 7863f7812a13f..86a5e0bcd432f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -71,6 +71,8 @@ public class IcebergSchemaTransformTranslationTest { .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .put("warehouse", "test_location") .build(); + private static final Map CONFIG_PROPERTIES = + ImmutableMap.builder().put("key", "value").put("key2", "value2").build(); @Test public void testReCreateWriteTransformFromRow() { @@ -79,6 +81,7 @@ public void testReCreateWriteTransformFromRow() { .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergWriteSchemaTransform writeTransform = (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); @@ -110,6 +113,7 @@ public void testWriteTransformProtoTranslation() .withFieldValue("table", "test_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergWriteSchemaTransform writeTransform = @@ -161,6 +165,7 @@ public void testReCreateReadTransformFromRow() { .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergReadSchemaTransform readTransform = @@ -192,6 +197,7 @@ public void testReadTransformProtoTranslation() .withFieldValue("table", identifier) .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", properties) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergReadSchemaTransform readTransform = diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index a2cd64e239567..6b555e7e14d0f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; import static org.hamcrest.MatcherAssert.assertThat; @@ -89,8 +88,8 @@ public void testSimpleAppend() { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - Config config = - Config.builder() + SchemaTransformConfiguration config = + SchemaTransformConfiguration.builder() .setTable(identifier) .setCatalogName("name") .setCatalogProperties(properties) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java index 007cb028c665d..38a15cb2aa98f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java @@ -20,13 +20,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import java.util.List; -import java.util.Properties; +import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -65,9 +66,11 @@ public void testUnstartedReaderReadsSamesItsSource() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -75,7 +78,7 @@ public void testUnstartedReaderReadsSamesItsSource() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) @@ -107,9 +110,11 @@ public void testInitialSplitting() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -117,7 +122,7 @@ public void testInitialSplitting() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) @@ -153,9 +158,11 @@ public void testDoubleInitialSplitting() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -163,7 +170,7 @@ public void testDoubleInitialSplitting() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) diff --git a/settings.gradle.kts b/settings.gradle.kts index 4d4b93908a02f..65a55885afa72 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -365,3 +365,7 @@ include("sdks:java:io:solace") findProject(":sdks:java:io:solace")?.name = "solace" include("sdks:java:extensions:combiners") findProject(":sdks:java:extensions:combiners")?.name = "combiners" +include("sdks:java:io:iceberg:hive") +findProject(":sdks:java:io:iceberg:hive")?.name = "hive" +include("sdks:java:io:iceberg:hive:exec") +findProject(":sdks:java:io:iceberg:hive:exec")?.name = "exec" From fc5a71db5caa95fd14988bfe475c240873216a2c Mon Sep 17 00:00:00 2001 From: Francis O'Hara Date: Fri, 9 Aug 2024 23:12:22 +0000 Subject: [PATCH 58/78] [CsvIO]: Implement CsvIOParse::withCustomRecordParsing method (#32142) * completed implementation without tests Co-authored-by: Lahari Guduru * intermediate stage Co-authored-by: Lahari Guduru * Implement CsvIOParse.withCustomRecordParsing Co-authored-by: Lahari Guduru --------- Co-authored-by: Lahari Guduru --- .../apache/beam/sdk/io/csv/CsvIOParse.java | 28 +++- .../sdk/io/csv/CsvIOParseConfiguration.java | 12 +- .../beam/sdk/io/csv/CsvIOParseTest.java | 127 +++++++++++++++++- 3 files changed, 158 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java index 0a27cdbc57eca..5981e81327652 100644 --- a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParse.java @@ -28,6 +28,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; +import org.checkerframework.checker.nullness.qual.NonNull; /** * {@link PTransform} for Parsing CSV Record Strings into {@link Schema}-mapped target types. {@link @@ -43,9 +44,30 @@ static CsvIOParse.Builder builder() { return new AutoValue_CsvIOParse.Builder<>(); } - // TODO(https://github.com/apache/beam/issues/31875): Implement in future PR. - public CsvIOParse withCustomRecordParsing( - Map> customProcessingMap) { + /** + * Configures custom cell parsing. + * + *

    Example

    + * + *
    {@code
    +   * CsvIO.parse().withCustomRecordParsing("listOfInts", cell-> {
    +   *
    +   *  List result = new ArrayList<>();
    +   *  for (String stringValue: Splitter.on(";").split(cell)) {
    +   *    result.add(Integer.parseInt(stringValue));
    +   *  }
    +   *
    +   * });
    +   * }
    + */ + public CsvIOParse withCustomRecordParsing( + String fieldName, SerializableFunction customRecordParsingFn) { + + Map> customProcessingMap = + getConfigBuilder().getOrCreateCustomProcessingMap(); + + customProcessingMap.put(fieldName, customRecordParsingFn::apply); + getConfigBuilder().setCustomProcessingMap(customProcessingMap); return this; } diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParseConfiguration.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParseConfiguration.java index dd9ef5b348686..2be871a9dc2dc 100644 --- a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParseConfiguration.java +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/CsvIOParseConfiguration.java @@ -60,18 +60,26 @@ abstract static class Builder implements Serializable { abstract Builder setCustomProcessingMap( Map> customProcessingMap); + abstract Optional>> getCustomProcessingMap(); + + final Map> getOrCreateCustomProcessingMap() { + if (!getCustomProcessingMap().isPresent()) { + setCustomProcessingMap(new HashMap<>()); + } + return getCustomProcessingMap().get(); + } + abstract Builder setCoder(Coder coder); abstract Builder setFromRowFn(SerializableFunction fromRowFn); - abstract Optional>> getCustomProcessingMap(); - abstract CsvIOParseConfiguration autoBuild(); final CsvIOParseConfiguration build() { if (!getCustomProcessingMap().isPresent()) { setCustomProcessingMap(new HashMap<>()); } + return autoBuild(); } } diff --git a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java index 05d6982004f45..a517cef3d51f5 100644 --- a/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java +++ b/sdks/java/io/csv/src/test/java/org/apache/beam/sdk/io/csv/CsvIOParseTest.java @@ -19,10 +19,17 @@ import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA; import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.NULLABLE_ALL_PRIMITIVE_DATA_TYPES_TYPE_DESCRIPTOR; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.TIME_CONTAINING_SCHEMA; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.TIME_CONTAINING_TYPE_DESCRIPTOR; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.TimeContaining; import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypes; import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypesFromRowFn; import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.nullableAllPrimitiveDataTypesToRowFn; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.timeContaining; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.timeContainingFromRowFn; +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.timeContainingToRowFn; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -38,17 +45,22 @@ import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.commons.csv.CSVFormat; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Tests for {@link CsvIOParse}. */ @RunWith(JUnit4.class) public class CsvIOParseTest { @@ -61,6 +73,12 @@ public class CsvIOParseTest { NULLABLE_ALL_PRIMITIVE_DATA_TYPES_TYPE_DESCRIPTOR, nullableAllPrimitiveDataTypesToRowFn(), nullableAllPrimitiveDataTypesFromRowFn()); + private static final Coder TIME_CONTAINING_CODER = + SchemaCoder.of( + TIME_CONTAINING_SCHEMA, + TIME_CONTAINING_TYPE_DESCRIPTOR, + timeContainingToRowFn(), + timeContainingFromRowFn()); private static final SerializableFunction ROW_ROW_SERIALIZABLE_FUNCTION = row -> row; @Rule public final TestPipeline pipeline = TestPipeline.create(); @@ -120,7 +138,7 @@ public void parseRows() { underTest( NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, csvFormat(), - emptyCustomProcessingMap(), + new HashMap<>(), ROW_ROW_SERIALIZABLE_FUNCTION, RowCoder.of(NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA))); PAssert.that(result.getOutput()).containsInAnyOrder(want); @@ -152,7 +170,7 @@ public void parsePOJOs() { underTest( NULLABLE_ALL_PRIMITIVE_DATA_TYPES_SCHEMA, csvFormat(), - emptyCustomProcessingMap(), + new HashMap<>(), nullableAllPrimitiveDataTypesFromRowFn(), NULLABLE_ALL_PRIMITIVE_DATA_TYPES_CODER)); PAssert.that(result.getOutput()).containsInAnyOrder(want); @@ -161,6 +179,98 @@ public void parsePOJOs() { pipeline.run(); } + @Test + public void givenSingleCustomParsingLambda_parsesPOJOs() { + PCollection records = + csvRecords( + pipeline, + "instant,instantList", + "2024-01-23T10:00:05.000Z,10-00-05-2024-01-23;12-59-59-2024-01-24"); + TimeContaining want = + timeContaining( + Instant.parse("2024-01-23T10:00:05.000Z"), + Arrays.asList( + Instant.parse("2024-01-23T10:00:05.000Z"), + Instant.parse("2024-01-24T12:59:59.000Z"))); + + CsvIOParse underTest = + underTest( + TIME_CONTAINING_SCHEMA, + CSVFormat.DEFAULT + .withHeader("instant", "instantList") + .withAllowDuplicateHeaderNames(false), + new HashMap<>(), + timeContainingFromRowFn(), + TIME_CONTAINING_CODER) + .withCustomRecordParsing("instantList", instantListParsingLambda()); + + CsvIOParseResult result = records.apply(underTest); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + @Test + public void givenMultipleCustomParsingLambdas_parsesPOJOs() { + PCollection records = + csvRecords( + pipeline, + "instant,instantList", + "2024-01-23@10:00:05,10-00-05-2024-01-23;12-59-59-2024-01-24"); + TimeContaining want = + timeContaining( + Instant.parse("2024-01-23T10:00:05.000Z"), + Arrays.asList( + Instant.parse("2024-01-23T10:00:05.000Z"), + Instant.parse("2024-01-24T12:59:59.000Z"))); + + CsvIOParse underTest = + underTest( + TIME_CONTAINING_SCHEMA, + CSVFormat.DEFAULT + .withHeader("instant", "instantList") + .withAllowDuplicateHeaderNames(false), + new HashMap<>(), + timeContainingFromRowFn(), + TIME_CONTAINING_CODER) + .withCustomRecordParsing( + "instant", + input -> + DateTimeFormat.forPattern("yyyy-MM-dd@HH:mm:ss") + .parseDateTime(input) + .toInstant()) + .withCustomRecordParsing("instantList", instantListParsingLambda()); + + CsvIOParseResult result = records.apply(underTest); + PAssert.that(result.getOutput()).containsInAnyOrder(want); + PAssert.that(result.getErrors()).empty(); + + pipeline.run(); + } + + @Test + public void givenCustomParsingError_emits() { + PCollection records = + csvRecords(pipeline, "instant,instantList", "2024-01-23T10:00:05.000Z,BAD CELL"); + CsvIOParse underTest = + underTest( + TIME_CONTAINING_SCHEMA, + CSVFormat.DEFAULT + .withHeader("instant", "instantList") + .withAllowDuplicateHeaderNames(false), + new HashMap<>(), + timeContainingFromRowFn(), + TIME_CONTAINING_CODER) + .withCustomRecordParsing("instantList", instantListParsingLambda()); + + CsvIOParseResult result = records.apply(underTest); + PAssert.that(result.getOutput()).empty(); + PAssert.thatSingleton(result.getErrors().apply(Count.globally())).isEqualTo(1L); + + pipeline.run(); + } + private static CSVFormat csvFormat() { return CSVFormat.DEFAULT .withAllowDuplicateHeaderNames(false) @@ -191,7 +301,16 @@ private static CsvIOParse underTest( return CsvIOParse.builder().setConfigBuilder(configBuilder).build(); } - private static Map> emptyCustomProcessingMap() { - return new HashMap<>(); + private static SerializableFunction> instantListParsingLambda() { + return input -> { + Iterable cells = Splitter.on(';').split(input); + ; + List output = new ArrayList<>(); + for (String cell : cells) { + output.add( + DateTimeFormat.forPattern("HH-mm-ss-yyyy-MM-dd").parseDateTime(cell).toInstant()); + } + return output; + }; } } From 488996913ff9b2edf83f855e7d0050075ac1b39d Mon Sep 17 00:00:00 2001 From: jonathan-lemos Date: Fri, 9 Aug 2024 21:21:44 -0400 Subject: [PATCH 59/78] Add support for setting an HTTP read timeout for BigQueryIO (#32118) Shamelessly stolen from https://github.com/apache/beam/pull/7097 --- .../apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java | 9 ++++++++- .../beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index ba76f483f774b..faa252e79b2d6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -39,7 +39,14 @@ public interface BigQueryOptions void setTempDatasetId(String value); @Description( - "Timeout for HTTP requests to BigQuery service in milliseconds. Set to 0 to disable.") + "Timeout for HTTP read requests to BigQuery service in milliseconds. Set to 0 to disable.") + @Default.Integer(80 * 1000) + Integer getHTTPReadTimeout(); + + void setHTTPReadTimeout(Integer timeout); + + @Description( + "Timeout for HTTP write requests to BigQuery service in milliseconds. Set to 0 to disable.") @Default.Integer(900 * 1000) Integer getHTTPWriteTimeout(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index b87b6a222a4d7..115875c59411c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1584,6 +1584,7 @@ private static Bigquery.Builder newBigQueryClient(BigQueryOptions options) { RetryHttpRequestInitializer httpRequestInitializer = new RetryHttpRequestInitializer(ImmutableList.of(404)); httpRequestInitializer.setCustomErrors(createBigQueryClientCustomErrors()); + httpRequestInitializer.setReadTimeout(options.getHTTPReadTimeout()); httpRequestInitializer.setWriteTimeout(options.getHTTPWriteTimeout()); ImmutableList.Builder initBuilder = ImmutableList.builder(); Credentials credential = options.getGcpCredential(); From 2f93d8bc19917f83d15f531bcbbfb7f36e21ff88 Mon Sep 17 00:00:00 2001 From: Hyeonho Kim Date: Sun, 11 Aug 2024 04:13:52 +0900 Subject: [PATCH 60/78] fix: cover bigquery datetime fraction 1 to 6 or absent (#32146) --- .../sdk/io/gcp/bigquery/BigQueryUtils.java | 4 +- .../io/gcp/bigquery/BigQueryUtilsTest.java | 103 +++++++++++++++++- 2 files changed, 100 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index 6b60b138b4fda..305abad5783aa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -175,7 +175,7 @@ public abstract static class Builder { /** * Native BigQuery formatter for it's timestamp format, depending on the milliseconds stored in - * the column, the milli second part will be 6, 3 or absent. Example {@code 2019-08-16 + * the column, the milli second part will be 6 to 1 or absent. Example {@code 2019-08-16 * 00:52:07[.123]|[.123456] UTC} */ private static final DateTimeFormatter BIGQUERY_TIMESTAMP_PARSER; @@ -202,7 +202,7 @@ public abstract static class Builder { .appendOptional( new DateTimeFormatterBuilder() .appendLiteral('.') - .appendFractionOfSecond(3, 6) + .appendFractionOfSecond(1, 6) .toParser()) .appendLiteral(" UTC") .toFormatter() diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java index 8a3ad16e190d4..e13e4a92a4dc4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java @@ -77,6 +77,10 @@ public class BigQueryUtilsTest { .addNullableField("timestamp_variant2", Schema.FieldType.DATETIME) .addNullableField("timestamp_variant3", Schema.FieldType.DATETIME) .addNullableField("timestamp_variant4", Schema.FieldType.DATETIME) + .addNullableField("timestamp_variant5", Schema.FieldType.DATETIME) + .addNullableField("timestamp_variant6", Schema.FieldType.DATETIME) + .addNullableField("timestamp_variant7", Schema.FieldType.DATETIME) + .addNullableField("timestamp_variant8", Schema.FieldType.DATETIME) .addNullableField("datetime", Schema.FieldType.logicalType(SqlTypes.DATETIME)) .addNullableField("datetime0ms", Schema.FieldType.logicalType(SqlTypes.DATETIME)) .addNullableField("datetime0s_ns", Schema.FieldType.logicalType(SqlTypes.DATETIME)) @@ -150,6 +154,22 @@ public class BigQueryUtilsTest { new TableFieldSchema() .setName("timestamp_variant4") .setType(StandardSQLTypeName.TIMESTAMP.toString()); + private static final TableFieldSchema TIMESTAMP_VARIANT5 = + new TableFieldSchema() + .setName("timestamp_variant5") + .setType(StandardSQLTypeName.TIMESTAMP.toString()); + private static final TableFieldSchema TIMESTAMP_VARIANT6 = + new TableFieldSchema() + .setName("timestamp_variant6") + .setType(StandardSQLTypeName.TIMESTAMP.toString()); + private static final TableFieldSchema TIMESTAMP_VARIANT7 = + new TableFieldSchema() + .setName("timestamp_variant7") + .setType(StandardSQLTypeName.TIMESTAMP.toString()); + private static final TableFieldSchema TIMESTAMP_VARIANT8 = + new TableFieldSchema() + .setName("timestamp_variant8") + .setType(StandardSQLTypeName.TIMESTAMP.toString()); private static final TableFieldSchema DATETIME = new TableFieldSchema().setName("datetime").setType(StandardSQLTypeName.DATETIME.toString()); @@ -240,6 +260,10 @@ public class BigQueryUtilsTest { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -271,6 +295,10 @@ public class BigQueryUtilsTest { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -312,6 +340,18 @@ public class BigQueryUtilsTest { .withZoneUTC() .parseDateTime("2019-08-18T15:52:07.123"), new DateTime(123456), + ISODateTimeFormat.dateHourMinuteSecondFraction() + .withZoneUTC() + .parseDateTime("2024-08-10T16:52:07.1"), + ISODateTimeFormat.dateHourMinuteSecondFraction() + .withZoneUTC() + .parseDateTime("2024-08-10T16:52:07.12"), + ISODateTimeFormat.dateHourMinuteSecondFraction() + .withZoneUTC() + .parseDateTime("2024-08-10T16:52:07.1234"), + ISODateTimeFormat.dateHourMinuteSecondFraction() + .withZoneUTC() + .parseDateTime("2024-08-10T16:52:07.12345"), LocalDateTime.parse("2020-11-02T12:34:56.789876"), LocalDateTime.parse("2020-11-02T12:34:56"), LocalDateTime.parse("2020-11-02T12:34:00.789876"), @@ -343,6 +383,11 @@ public class BigQueryUtilsTest { "timestamp_variant4", String.valueOf( new DateTime(123456L, ISOChronology.getInstanceUTC()).getMillis() / 1000.0D)) + .set("timestamp_variant5", "2024-08-10 16:52:07.1 UTC") + .set("timestamp_variant6", "2024-08-10 16:52:07.12 UTC") + // we'll loose precession, but it's something BigQuery can output! + .set("timestamp_variant7", "2024-08-10 16:52:07.1234 UTC") + .set("timestamp_variant8", "2024-08-10 16:52:07.12345 UTC") .set("datetime", "2020-11-02T12:34:56.789876") .set("datetime0ms", "2020-11-02T12:34:56") .set("datetime0s_ns", "2020-11-02T12:34:00.789876") @@ -364,7 +409,7 @@ public class BigQueryUtilsTest { Row.withSchema(FLAT_TYPE) .addValues( null, null, null, null, null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null) + null, null, null, null, null, null, null, null, null, null, null, null, null) .build(); private static final TableRow BQ_NULL_FLAT_ROW = @@ -376,6 +421,10 @@ public class BigQueryUtilsTest { .set("timestamp_variant2", null) .set("timestamp_variant3", null) .set("timestamp_variant4", null) + .set("timestamp_variant5", null) + .set("timestamp_variant6", null) + .set("timestamp_variant7", null) + .set("timestamp_variant8", null) .set("datetime", null) .set("datetime0ms", null) .set("datetime0s_ns", null) @@ -459,6 +508,10 @@ public class BigQueryUtilsTest { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -515,6 +568,10 @@ public void testToTableSchema_flat() { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -566,6 +623,10 @@ public void testToTableSchema_row() { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -603,6 +664,10 @@ public void testToTableSchema_array_row() { TIMESTAMP_VARIANT2, TIMESTAMP_VARIANT3, TIMESTAMP_VARIANT4, + TIMESTAMP_VARIANT5, + TIMESTAMP_VARIANT6, + TIMESTAMP_VARIANT7, + TIMESTAMP_VARIANT8, DATETIME, DATETIME_0MS, DATETIME_0S_NS, @@ -637,9 +702,17 @@ public void testToTableSchema_map() { public void testToTableRow_flat() { TableRow row = toTableRow().apply(FLAT_ROW); - assertThat(row.size(), equalTo(23)); + assertThat(row.size(), equalTo(27)); assertThat(row, hasEntry("id", "123")); assertThat(row, hasEntry("value", "123.456")); + assertThat(row, hasEntry("timestamp_variant1", "2019-08-16 13:52:07.000 UTC")); + assertThat(row, hasEntry("timestamp_variant2", "2019-08-17 14:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant3", "2019-08-18 15:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant4", "1970-01-01 00:02:03.456 UTC")); + assertThat(row, hasEntry("timestamp_variant5", "2024-08-10 16:52:07.100 UTC")); + assertThat(row, hasEntry("timestamp_variant6", "2024-08-10 16:52:07.120 UTC")); + assertThat(row, hasEntry("timestamp_variant7", "2024-08-10 16:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant8", "2024-08-10 16:52:07.123 UTC")); assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876")); assertThat(row, hasEntry("datetime0ms", "2020-11-02T12:34:56")); assertThat(row, hasEntry("datetime0s_ns", "2020-11-02T12:34:00.789876")); @@ -692,9 +765,17 @@ public void testToTableRow_row() { assertThat(row.size(), equalTo(1)); row = (TableRow) row.get("row"); - assertThat(row.size(), equalTo(23)); + assertThat(row.size(), equalTo(27)); assertThat(row, hasEntry("id", "123")); assertThat(row, hasEntry("value", "123.456")); + assertThat(row, hasEntry("timestamp_variant1", "2019-08-16 13:52:07.000 UTC")); + assertThat(row, hasEntry("timestamp_variant2", "2019-08-17 14:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant3", "2019-08-18 15:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant4", "1970-01-01 00:02:03.456 UTC")); + assertThat(row, hasEntry("timestamp_variant5", "2024-08-10 16:52:07.100 UTC")); + assertThat(row, hasEntry("timestamp_variant6", "2024-08-10 16:52:07.120 UTC")); + assertThat(row, hasEntry("timestamp_variant7", "2024-08-10 16:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant8", "2024-08-10 16:52:07.123 UTC")); assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876")); assertThat(row, hasEntry("datetime0ms", "2020-11-02T12:34:56")); assertThat(row, hasEntry("datetime0s_ns", "2020-11-02T12:34:00.789876")); @@ -720,9 +801,17 @@ public void testToTableRow_array_row() { assertThat(row.size(), equalTo(1)); row = ((List) row.get("rows")).get(0); - assertThat(row.size(), equalTo(23)); + assertThat(row.size(), equalTo(27)); assertThat(row, hasEntry("id", "123")); assertThat(row, hasEntry("value", "123.456")); + assertThat(row, hasEntry("timestamp_variant1", "2019-08-16 13:52:07.000 UTC")); + assertThat(row, hasEntry("timestamp_variant2", "2019-08-17 14:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant3", "2019-08-18 15:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant4", "1970-01-01 00:02:03.456 UTC")); + assertThat(row, hasEntry("timestamp_variant5", "2024-08-10 16:52:07.100 UTC")); + assertThat(row, hasEntry("timestamp_variant6", "2024-08-10 16:52:07.120 UTC")); + assertThat(row, hasEntry("timestamp_variant7", "2024-08-10 16:52:07.123 UTC")); + assertThat(row, hasEntry("timestamp_variant8", "2024-08-10 16:52:07.123 UTC")); assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876")); assertThat(row, hasEntry("datetime0ms", "2020-11-02T12:34:56")); assertThat(row, hasEntry("datetime0s_ns", "2020-11-02T12:34:00.789876")); @@ -746,7 +835,7 @@ public void testToTableRow_array_row() { public void testToTableRow_null_row() { TableRow row = toTableRow().apply(NULL_FLAT_ROW); - assertThat(row.size(), equalTo(23)); + assertThat(row.size(), equalTo(27)); assertThat(row, hasEntry("id", null)); assertThat(row, hasEntry("value", null)); assertThat(row, hasEntry("name", null)); @@ -754,6 +843,10 @@ public void testToTableRow_null_row() { assertThat(row, hasEntry("timestamp_variant2", null)); assertThat(row, hasEntry("timestamp_variant3", null)); assertThat(row, hasEntry("timestamp_variant4", null)); + assertThat(row, hasEntry("timestamp_variant5", null)); + assertThat(row, hasEntry("timestamp_variant6", null)); + assertThat(row, hasEntry("timestamp_variant7", null)); + assertThat(row, hasEntry("timestamp_variant8", null)); assertThat(row, hasEntry("datetime", null)); assertThat(row, hasEntry("datetime0ms", null)); assertThat(row, hasEntry("datetime0s_ns", null)); From 780eef98083fe56f81cc5c62dc8ff193993584f0 Mon Sep 17 00:00:00 2001 From: twosom <72733442+twosom@users.noreply.github.com> Date: Mon, 12 Aug 2024 22:35:35 +0900 Subject: [PATCH 61/78] Replace StateTag.StateBinder to top level StateBinder in SparkStateInternals (#31798) --- ...PostCommit_Java_ValidatesRunner_Spark.json | 3 +- ...idatesRunner_SparkStructuredStreaming.json | 3 +- ...mit_Java_ValidatesRunner_Spark_Java11.json | 3 +- .../apache/beam/runners/core/StateTag.java | 5 +- .../spark/stateful/SparkStateInternals.java | 108 +++++++++--------- 5 files changed, 62 insertions(+), 60 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json index b970762c83970..d59e273949da9 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json index b970762c83970..d59e273949da9 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json index b970762c83970..d59e273949da9 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test" } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java index 8c699ac311179..0106f95ed7486 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java @@ -69,8 +69,9 @@ public interface StateTag extends Serializable { /** * Visitor for binding a {@link StateSpec} and to the associated {@link State}. * - * @deprecated for migration only; runners should reference the top level {@link StateBinder} and - * move towards {@link StateSpec} rather than {@link StateTag}. + * @deprecated for migration only; runners should reference the top level {@link + * org.apache.beam.sdk.state.StateBinder} and move towards {@link StateSpec} rather than + * {@link StateTag}. */ @Deprecated public interface StateBinder { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java index 731cadb89f0c2..7ca0dc29e615d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java @@ -27,7 +27,6 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.StateTag.StateBinder; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; @@ -42,11 +41,13 @@ import org.apache.beam.sdk.state.ReadableStates; import org.apache.beam.sdk.state.SetState; import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateBinder; import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; +import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; @@ -96,45 +97,47 @@ public K getKey() { @Override public T state( StateNamespace namespace, StateTag address, StateContext c) { - return address.bind(new SparkStateBinder(namespace, c)); + return address.getSpec().bind(address.getId(), new SparkStateBinder(namespace, c)); } private class SparkStateBinder implements StateBinder { private final StateNamespace namespace; - private final StateContext c; + private final StateContext stateContext; - private SparkStateBinder(StateNamespace namespace, StateContext c) { + private SparkStateBinder(StateNamespace namespace, StateContext stateContext) { this.namespace = namespace; - this.c = c; + this.stateContext = stateContext; } @Override - public ValueState bindValue(StateTag> address, Coder coder) { - return new SparkValueState<>(namespace, address, coder); + public ValueState bindValue(String id, StateSpec> spec, Coder coder) { + return new SparkValueState<>(namespace, id, coder); } @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { - return new SparkBagState<>(namespace, address, elemCoder); + public BagState bindBag(String id, StateSpec> spec, Coder elemCoder) { + return new SparkBagState<>(namespace, id, elemCoder); } @Override - public SetState bindSet(StateTag> spec, Coder elemCoder) { + public SetState bindSet(String id, StateSpec> spec, Coder elemCoder) { throw new UnsupportedOperationException( String.format("%s is not supported", SetState.class.getSimpleName())); } @Override public MapState bindMap( - StateTag> address, + String id, + StateSpec> spec, Coder mapKeyCoder, Coder mapValueCoder) { - return new SparkMapState<>(namespace, address, MapCoder.of(mapKeyCoder, mapValueCoder)); + return new SparkMapState<>(namespace, id, MapCoder.of(mapKeyCoder, mapValueCoder)); } @Override public MultimapState bindMultimap( - StateTag> spec, + String id, + StateSpec> spec, Coder keyCoder, Coder valueCoder) { throw new UnsupportedOperationException( @@ -143,50 +146,51 @@ public MultimapState bindMultimap( @Override public OrderedListState bindOrderedList( - StateTag> spec, Coder elemCoder) { + String id, StateSpec> spec, Coder elemCoder) { throw new UnsupportedOperationException( String.format("%s is not supported", OrderedListState.class.getSimpleName())); } @Override - public CombiningState bindCombiningValue( - StateTag> address, + public CombiningState bindCombining( + String id, + StateSpec> spec, Coder accumCoder, CombineFn combineFn) { - return new SparkCombiningState<>(namespace, address, accumCoder, combineFn); + return new SparkCombiningState<>(namespace, id, accumCoder, combineFn); } @Override public - CombiningState bindCombiningValueWithContext( - StateTag> address, + CombiningState bindCombiningWithContext( + String id, + StateSpec> spec, Coder accumCoder, - CombineFnWithContext combineFn) { + CombineWithContext.CombineFnWithContext combineFn) { return new SparkCombiningState<>( - namespace, address, accumCoder, CombineFnUtil.bindContext(combineFn, c)); + namespace, id, accumCoder, CombineFnUtil.bindContext(combineFn, stateContext)); } @Override public WatermarkHoldState bindWatermark( - StateTag address, TimestampCombiner timestampCombiner) { - return new SparkWatermarkHoldState(namespace, address, timestampCombiner); + String id, StateSpec spec, TimestampCombiner timestampCombiner) { + return new SparkWatermarkHoldState(namespace, id, timestampCombiner); } } private class AbstractState { final StateNamespace namespace; - final StateTag address; + final String id; final Coder coder; - private AbstractState( - StateNamespace namespace, StateTag address, Coder coder) { + private AbstractState(StateNamespace namespace, String id, Coder coder) { this.namespace = namespace; - this.address = address; + this.id = id; this.coder = coder; } T readValue() { - byte[] buf = stateTable.get(namespace.stringKey(), address.getId()); + byte[] buf = stateTable.get(namespace.stringKey(), id); if (buf != null) { return CoderHelpers.fromByteArray(buf, coder); } @@ -194,12 +198,11 @@ T readValue() { } void writeValue(T input) { - stateTable.put( - namespace.stringKey(), address.getId(), CoderHelpers.toByteArray(input, coder)); + stateTable.put(namespace.stringKey(), id, CoderHelpers.toByteArray(input, coder)); } public void clear() { - stateTable.remove(namespace.stringKey(), address.getId()); + stateTable.remove(namespace.stringKey(), id); } @Override @@ -212,22 +215,21 @@ public boolean equals(@Nullable Object o) { } @SuppressWarnings("unchecked") AbstractState that = (AbstractState) o; - return namespace.equals(that.namespace) && address.equals(that.address); + return namespace.equals(that.namespace) && id.equals(that.id); } @Override public int hashCode() { int result = namespace.hashCode(); - result = 31 * result + address.hashCode(); + result = 31 * result + id.hashCode(); return result; } } private class SparkValueState extends AbstractState implements ValueState { - private SparkValueState( - StateNamespace namespace, StateTag> address, Coder coder) { - super(namespace, address, coder); + private SparkValueState(StateNamespace namespace, String id, Coder coder) { + super(namespace, id, coder); } @Override @@ -252,10 +254,8 @@ private class SparkWatermarkHoldState extends AbstractState private final TimestampCombiner timestampCombiner; SparkWatermarkHoldState( - StateNamespace namespace, - StateTag address, - TimestampCombiner timestampCombiner) { - super(namespace, address, InstantCoder.of()); + StateNamespace namespace, String id, TimestampCombiner timestampCombiner) { + super(namespace, id, InstantCoder.of()); this.timestampCombiner = timestampCombiner; } @@ -287,7 +287,7 @@ public ReadableState readLater() { @Override public Boolean read() { - return stateTable.get(namespace.stringKey(), address.getId()) == null; + return stateTable.get(namespace.stringKey(), id) == null; } }; } @@ -299,22 +299,22 @@ public TimestampCombiner getTimestampCombiner() { } @SuppressWarnings("TypeParameterShadowing") - private class SparkCombiningState extends AbstractState + private class SparkCombiningState extends AbstractState implements CombiningState { private final CombineFn combineFn; private SparkCombiningState( StateNamespace namespace, - StateTag> address, + String id, Coder coder, CombineFn combineFn) { - super(namespace, address, coder); + super(namespace, id, coder); this.combineFn = combineFn; } @Override - public SparkCombiningState readLater() { + public SparkCombiningState readLater() { return this; } @@ -348,7 +348,7 @@ public ReadableState readLater() { @Override public Boolean read() { - return stateTable.get(namespace.stringKey(), address.getId()) == null; + return stateTable.get(namespace.stringKey(), id) == null; } }; } @@ -369,10 +369,8 @@ private final class SparkMapState extends AbstractState> implements MapState { private SparkMapState( - StateNamespace namespace, - StateTag address, - Coder> coder) { - super(namespace, address, coder); + StateNamespace namespace, String id, Coder> coder) { + super(namespace, id, coder); } @Override @@ -490,7 +488,7 @@ public ReadableState isEmpty() { return new ReadableState() { @Override public Boolean read() { - return stateTable.get(namespace.stringKey(), address.getId()) == null; + return stateTable.get(namespace.stringKey(), id) == null; } @Override @@ -502,8 +500,8 @@ public ReadableState readLater() { } private final class SparkBagState extends AbstractState> implements BagState { - private SparkBagState(StateNamespace namespace, StateTag> address, Coder coder) { - super(namespace, address, ListCoder.of(coder)); + private SparkBagState(StateNamespace namespace, String id, Coder coder) { + super(namespace, id, ListCoder.of(coder)); } @Override @@ -537,7 +535,7 @@ public ReadableState readLater() { @Override public Boolean read() { - return stateTable.get(namespace.stringKey(), address.getId()) == null; + return stateTable.get(namespace.stringKey(), id) == null; } }; } From ab4ada4ff404951daa3903b5a96a051d396c4e7a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 12 Aug 2024 10:12:23 -0400 Subject: [PATCH 62/78] Skip most bigtableIO write error handling test on Dataflow runner (#32048) --- runners/google-cloud-dataflow-java/build.gradle | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 5d898bb57d86b..55f0074b9f314 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -573,6 +573,13 @@ task googleCloudPlatformLegacyWorkerIntegrationTest(type: Test, dependsOn: copyG testClassesDirs = files(project(":sdks:java:io:google-cloud-platform").sourceSets.test.output.classesDirs) useJUnit { excludeCategories "org.apache.beam.sdk.testing.UsesKms" + filter { + // Only needs to run on direct runner + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyMutationFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyRowFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithInvalidTimestampFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithOversizedQualifierFailures' + } } } @@ -623,7 +630,15 @@ task googleCloudPlatformRunnerV2IntegrationTest(type: Test) { maxParallelForks 4 classpath = configurations.googleCloudPlatformIntegrationTest testClassesDirs = files(project(":sdks:java:io:google-cloud-platform").sourceSets.test.output.classesDirs) - useJUnit { } + useJUnit { + filter { + // Only needs to run on direct runner + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyMutationFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyRowFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithInvalidTimestampFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithOversizedQualifierFailures' + } + } } task examplesJavaRunnerV2PreCommit(type: Test) { From 29de91383f3633b9a80c885a84726accc5fc6bf1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 10:16:52 -0400 Subject: [PATCH 63/78] Bump cloud.google.com/go/bigtable from 1.28.0 to 1.29.0 in /sdks (#32151) Bumps [cloud.google.com/go/bigtable](https://github.com/googleapis/google-cloud-go) from 1.28.0 to 1.29.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/documentai/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.28.0...pubsub/v1.29.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigtable dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 22 +++++++++++----------- sdks/go.sum | 52 ++++++++++++++++++++++++++-------------------------- 2 files changed, 37 insertions(+), 37 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 624cc0ab1ce82..16b39cdbe3a78 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -24,7 +24,7 @@ go 1.21 require ( cloud.google.com/go/bigquery v1.62.0 - cloud.google.com/go/bigtable v1.28.0 + cloud.google.com/go/bigtable v1.29.0 cloud.google.com/go/datastore v1.17.1 cloud.google.com/go/profiler v0.4.1 cloud.google.com/go/pubsub v1.40.0 @@ -58,8 +58,8 @@ require ( golang.org/x/sync v0.8.0 golang.org/x/sys v0.23.0 golang.org/x/text v0.17.0 - google.golang.org/api v0.189.0 - google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f + google.golang.org/api v0.191.0 + google.golang.org/genproto v0.0.0-20240730163845-b1a4ccb954bf google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.34.2 gopkg.in/yaml.v2 v2.4.0 @@ -74,9 +74,9 @@ require ( require ( cel.dev/expr v0.15.0 // indirect - cloud.google.com/go/auth v0.7.2 // indirect + cloud.google.com/go/auth v0.8.0 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.3 // indirect - cloud.google.com/go/monitoring v1.20.2 // indirect + cloud.google.com/go/monitoring v1.20.3 // indirect dario.cat/mergo v1.0.0 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect @@ -110,14 +110,14 @@ require ( go.opentelemetry.io/otel/sdk v1.24.0 // indirect go.opentelemetry.io/otel/sdk/metric v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.24.0 // indirect - golang.org/x/time v0.5.0 // indirect + golang.org/x/time v0.6.0 // indirect ) require ( cloud.google.com/go v0.115.0 // indirect cloud.google.com/go/compute/metadata v0.5.0 // indirect - cloud.google.com/go/iam v1.1.11 // indirect - cloud.google.com/go/longrunning v0.5.10 // indirect + cloud.google.com/go/iam v1.1.12 // indirect + cloud.google.com/go/longrunning v0.5.11 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.2 // indirect github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 // indirect @@ -154,7 +154,7 @@ require ( github.com/google/flatbuffers v23.5.26+incompatible // indirect github.com/google/pprof v0.0.0-20240528025155-186aa0362fba // indirect github.com/google/renameio/v2 v2.0.0 // indirect - github.com/google/s2a-go v0.1.7 // indirect + github.com/google/s2a-go v0.1.8 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect github.com/googleapis/gax-go/v2 v2.13.0 // indirect github.com/gorilla/handlers v1.5.2 // indirect @@ -187,6 +187,6 @@ require ( golang.org/x/mod v0.18.0 // indirect golang.org/x/tools v0.22.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 67686da8e408a..8b25c7c000d5d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -101,8 +101,8 @@ cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVo cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= -cloud.google.com/go/auth v0.7.2 h1:uiha352VrCDMXg+yoBtaD0tUF4Kv9vrtrWPYXwutnDE= -cloud.google.com/go/auth v0.7.2/go.mod h1:VEc4p5NNxycWQTMQEDQF0bd6aTMb6VgYDXEwiJJQAbs= +cloud.google.com/go/auth v0.8.0 h1:y8jUJLl/Fg+qNBWxP/Hox2ezJvjkrPb952PC1p0G6A4= +cloud.google.com/go/auth v0.8.0/go.mod h1:qGVp/Y3kDRSDZ5gFD/XPUfYQ9xW1iI7q8RIRoCyBbJc= cloud.google.com/go/auth/oauth2adapt v0.2.3 h1:MlxF+Pd3OmSudg/b1yZ5lJwoXCEaeedAguodky1PcKI= cloud.google.com/go/auth/oauth2adapt v0.2.3/go.mod h1:tMQXOfZzFuNuUxOypHlQEXgdfX5cuhwU+ffUuXRJE8I= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= @@ -135,8 +135,8 @@ cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9 cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= cloud.google.com/go/bigquery v1.62.0 h1:SYEA2f7fKqbSRRBHb7g0iHTtZvtPSPYdXfmqsjpsBwo= cloud.google.com/go/bigquery v1.62.0/go.mod h1:5ee+ZkF1x/ntgCsFQJAQTM3QkAZOecfCmvxhkJsWRSA= -cloud.google.com/go/bigtable v1.28.0 h1:c0wc/wy+9Chj8BooqW/zgaeslXsA5YEYl84VBmvwp+4= -cloud.google.com/go/bigtable v1.28.0/go.mod h1:avmXcmxVbLJAo9moICRYMgDyTTPoV0MA0lHKnyqV4fQ= +cloud.google.com/go/bigtable v1.29.0 h1:2CnFjKPwjpZMZdTi2RpppvxzD80zKzDYrLYEQw/NnAs= +cloud.google.com/go/bigtable v1.29.0/go.mod h1:5p909nNdWaNUcWs6KGZO8mI5HUovstlmrIi7+eA5PTQ= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= @@ -210,8 +210,8 @@ cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOX cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= -cloud.google.com/go/datacatalog v1.20.4 h1:nUR7JBPZezl1+o+86N01VxAQQHY+It/D8tmNipcdVjI= -cloud.google.com/go/datacatalog v1.20.4/go.mod h1:71PDwywIYkNgSXdUU3H0mkTp3j15aahfYJ1CY3DogtU= +cloud.google.com/go/datacatalog v1.20.5 h1:Cosg/L60myEbpP1HoNv77ykV7zWe7hqSwY4uUDmhx/I= +cloud.google.com/go/datacatalog v1.20.5/go.mod h1:DB0QWF9nelpsbB0eR/tA0xbHZZMvpoFD1XFy3Qv/McI= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -327,8 +327,8 @@ cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGE cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= -cloud.google.com/go/iam v1.1.11 h1:0mQ8UKSfdHLut6pH9FM3bI55KWR46ketn0PuXleDyxw= -cloud.google.com/go/iam v1.1.11/go.mod h1:biXoiLWYIKntto2joP+62sd9uW5EpkZmKIvfNcTWlnQ= +cloud.google.com/go/iam v1.1.12 h1:JixGLimRrNGcxvJEQ8+clfLxPlbeZA6MuRJ+qJNQ5Xw= +cloud.google.com/go/iam v1.1.12/go.mod h1:9LDX8J7dN5YRyzVHxwQzrQs9opFFqn0Mxs9nAeB+Hhg= cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= @@ -348,8 +348,8 @@ cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4 cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= -cloud.google.com/go/kms v1.18.3 h1:8+Z2S4bQDSCdghB5ZA5dVDDJTLmnkRlowtFiXqMFd74= -cloud.google.com/go/kms v1.18.3/go.mod h1:y/Lcf6fyhbdn7MrG1VaDqXxM8rhOBc5rWcWAhcvZjQU= +cloud.google.com/go/kms v1.18.4 h1:dYN3OCsQ6wJLLtOnI8DGUwQ5shMusXsWCCC+s09ATsk= +cloud.google.com/go/kms v1.18.4/go.mod h1:SG1bgQ3UWW6/KdPo9uuJnzELXY5YTTMJtDYvajiQ22g= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -363,8 +363,8 @@ cloud.google.com/go/logging v1.7.0/go.mod h1:3xjP2CjkM3ZkO73aj4ASA5wRPGGCRrPIAeN cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= -cloud.google.com/go/longrunning v0.5.10 h1:eB/BniENNRKhjz/xgiillrdcH3G74TGSl3BXinGlI7E= -cloud.google.com/go/longrunning v0.5.10/go.mod h1:tljz5guTr5oc/qhlUjBlk7UAIFMOGuPNxkNDZXlLics= +cloud.google.com/go/longrunning v0.5.11 h1:Havn1kGjz3whCfoD8dxMLP73Ph5w+ODyZB9RUsDxtGk= +cloud.google.com/go/longrunning v0.5.11/go.mod h1:rDn7//lmlfWV1Dx6IB4RatCPenTwwmqXuiP0/RgoEO4= cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= @@ -388,8 +388,8 @@ cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhI cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= -cloud.google.com/go/monitoring v1.20.2 h1:B/L+xrw9PYO7ywh37sgnjI/6dzEE+yQTAwfytDcpPto= -cloud.google.com/go/monitoring v1.20.2/go.mod h1:36rpg/7fdQ7NX5pG5x1FA7cXTVXusOp6Zg9r9e1+oek= +cloud.google.com/go/monitoring v1.20.3 h1:v/7MXFxYrhXLEZ9sSfwXdlTLLB/xrU7xTyYjY5acynQ= +cloud.google.com/go/monitoring v1.20.3/go.mod h1:GPIVIdNznIdGqEjtRKQWTLcUeRnPjZW85szouimiczU= cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= @@ -943,8 +943,8 @@ github.com/google/pprof v0.0.0-20240528025155-186aa0362fba/go.mod h1:K1liHPHnj73 github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= -github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= -github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= +github.com/google/s2a-go v0.1.8 h1:zZDs9gcbt9ZPLV0ndSyQk6Kacx2g/X+SKYovpnz3SMM= +github.com/google/s2a-go v0.1.8/go.mod h1:6iNWHTpQ+nfNRN5E00MSdfDwVesa8hhS32PhPO8deJA= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -1543,8 +1543,8 @@ golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= -golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= +golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1688,8 +1688,8 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.189.0 h1:equMo30LypAkdkLMBqfeIqtyAnlyig1JSZArl4XPwdI= -google.golang.org/api v0.189.0/go.mod h1:FLWGJKb0hb+pU2j+rJqwbnsF+ym+fQs73rbJ+KAUgy8= +google.golang.org/api v0.191.0 h1:cJcF09Z+4HAB2t5qTQM1ZtfL/PemsLFkcFG67qq2afk= +google.golang.org/api v0.191.0/go.mod h1:tD5dsFGxFza0hnQveGfVk9QQYKcfp+VzgRqyXFxE0+E= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1829,12 +1829,12 @@ google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= -google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f h1:htT2I9bZvGm+110zq8bIErMX+WgBWxCzV3ChwbvnKnc= -google.golang.org/genproto v0.0.0-20240725223205-93522f1f2a9f/go.mod h1:Sk3mLpoDFTAp6R4OvlcUgaG4ISTspKeFsIAXMn9Bm4Y= -google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade h1:WxZOF2yayUHpHSbUE6NMzumUzBxYc3YGwo0YHnbzsJY= -google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade h1:oCRSWfwGXQsqlVdErcyTt4A93Y8fo0/9D4b1gnI++qo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto v0.0.0-20240730163845-b1a4ccb954bf h1:OqdXDEakZCVtDiZTjcxfwbHPCT11ycCEsTKesBVKvyY= +google.golang.org/genproto v0.0.0-20240730163845-b1a4ccb954bf/go.mod h1:mCr1K1c8kX+1iSBREvU3Juo11CB+QOEWxbRS01wWl5M= +google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f h1:b1Ln/PG8orm0SsBbHZWke8dDp2lrCD4jSmfglFpTZbk= +google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f/go.mod h1:AHT0dDg3SoMOgZGnZk29b5xTbPHMoEC8qthmBLJCpys= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf h1:liao9UHurZLtiEwBgT9LMOnKYsHze6eA6w1KQCMVN2Q= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From f4e43148118be30f7002bd00b5a8df44bd201a24 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 10:17:20 -0400 Subject: [PATCH 64/78] Bump cloud.google.com/go/pubsub from 1.40.0 to 1.41.0 in /sdks (#32149) Bumps [cloud.google.com/go/pubsub](https://github.com/googleapis/google-cloud-go) from 1.40.0 to 1.41.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.40.0...pubsub/v1.41.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/pubsub dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 16b39cdbe3a78..5d179736baf09 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -27,7 +27,7 @@ require ( cloud.google.com/go/bigtable v1.29.0 cloud.google.com/go/datastore v1.17.1 cloud.google.com/go/profiler v0.4.1 - cloud.google.com/go/pubsub v1.40.0 + cloud.google.com/go/pubsub v1.41.0 cloud.google.com/go/spanner v1.66.0 cloud.google.com/go/storage v1.43.0 github.com/aws/aws-sdk-go-v2 v1.30.3 diff --git a/sdks/go.sum b/sdks/go.sum index 8b25c7c000d5d..2b43c6d62bc92 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -449,8 +449,8 @@ cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcd cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= -cloud.google.com/go/pubsub v1.40.0 h1:0LdP+zj5XaPAGtWr2V6r88VXJlmtaB/+fde1q3TU8M0= -cloud.google.com/go/pubsub v1.40.0/go.mod h1:BVJI4sI2FyXp36KFKvFwcfDRDfR8MiLT8mMhmIhdAeA= +cloud.google.com/go/pubsub v1.41.0 h1:ZPaM/CvTO6T+1tQOs/jJ4OEMpjtel0PTLV7j1JK+ZrI= +cloud.google.com/go/pubsub v1.41.0/go.mod h1:g+YzC6w/3N91tzG66e2BZtp7WrpBBMXVa3Y9zVoOGpk= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= From 8b6f37b28746bcd8e6c8be4382ef72072cfdb959 Mon Sep 17 00:00:00 2001 From: jonathan-lemos Date: Mon, 12 Aug 2024 10:17:56 -0400 Subject: [PATCH 65/78] Fix broken Beam Quest URL in README.md (#32145) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 0d564024f87df..8a9bb4e988d58 100644 --- a/README.md +++ b/README.md @@ -109,7 +109,7 @@ Here are some resources actively maintained by the Beam community to help you ge A comprehensive, interactive learning experience covering Beam concepts in depth. - Beam Quest + Beam Quest A certification granted by Google Cloud, certifying proficiency in Beam. From 2ce0ee34470519a1da61a2ac241997232891f48a Mon Sep 17 00:00:00 2001 From: Ayush Pandey <32012449+itsayushpandey@users.noreply.github.com> Date: Mon, 12 Aug 2024 08:35:32 -0600 Subject: [PATCH 66/78] Added insertion and enrichment pipeline (#31657) * Adding insertion and enrichment pipeline * Enhanced Data Schema * Added Apache Licensed to the notebook * Adding Chunking Strategy * removed unused imports * Modified insertion logic in redis for incorporating chunking strategy * refacted redis code * code review changes * Added chunking code in notebook * Added code review changes * Code review changes: using chunking strategy as enum * Added Code Review Changes * Code review changes * Added code review changes * Added Code Review Changes * Code review changes --- .../rag_usecase/beam_rag_notebook.ipynb | 1795 +++++++++++++++++ .../beam-ml/rag_usecase/chunks_generation.py | 129 ++ .../beam-ml/rag_usecase/redis_connector.py | 349 ++++ .../beam-ml/rag_usecase/redis_enrichment.py | 110 + 4 files changed, 2383 insertions(+) create mode 100644 examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb create mode 100644 examples/notebooks/beam-ml/rag_usecase/chunks_generation.py create mode 100644 examples/notebooks/beam-ml/rag_usecase/redis_connector.py create mode 100644 examples/notebooks/beam-ml/rag_usecase/redis_enrichment.py diff --git a/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb b/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb new file mode 100644 index 0000000000000..e271074af5551 --- /dev/null +++ b/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb @@ -0,0 +1,1795 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": 1, + "id": "af127b51-1c7e-4e56-9759-aee40d9df194", + "metadata": {}, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "code", + "execution_count": 9, + "id": "160b9fee-00e9-4dd1-b1db-3d050e1bc710", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Requirement already satisfied: pandas==1.4.4 in /usr/local/lib/python3.10/site-packages (1.4.4)\n", + "Requirement already satisfied: python-dateutil>=2.8.1 in /usr/local/lib/python3.10/site-packages (from pandas==1.4.4) (2.8.2)\n", + "Requirement already satisfied: pytz>=2020.1 in /usr/local/lib/python3.10/site-packages (from pandas==1.4.4) (2022.2.1)\n", + "Requirement already satisfied: numpy>=1.21.0 in /usr/local/lib/python3.10/site-packages (from pandas==1.4.4) (1.24.4)\n", + "Requirement already satisfied: six>=1.5 in /usr/local/lib/python3.10/site-packages (from python-dateutil>=2.8.1->pandas==1.4.4) (1.16.0)\n", + "\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m A new release of pip is available: \u001b[0m\u001b[31;49m24.1.1\u001b[0m\u001b[39;49m -> \u001b[0m\u001b[32;49m24.2\u001b[0m\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m To update, run: \u001b[0m\u001b[32;49m/usr/local/opt/python@3.10/bin/python3.10 -m pip install --upgrade pip\u001b[0m\n", + "Requirement already satisfied: numpy==1.24.4 in /usr/local/lib/python3.10/site-packages (1.24.4)\n", + "\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m A new release of pip is available: \u001b[0m\u001b[31;49m24.1.1\u001b[0m\u001b[39;49m -> \u001b[0m\u001b[32;49m24.2\u001b[0m\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m To update, run: \u001b[0m\u001b[32;49m/usr/local/opt/python@3.10/bin/python3.10 -m pip install --upgrade pip\u001b[0m\n", + "Requirement already satisfied: apache_beam==2.56.0 in /usr/local/lib/python3.10/site-packages (2.56.0)\n", + "Requirement already satisfied: crcmod<2.0,>=1.7 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.7)\n", + "Requirement already satisfied: orjson<4,>=3.9.7 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (3.10.0)\n", + "Requirement already satisfied: dill<0.3.2,>=0.3.1.1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.3.1.1)\n", + "Requirement already satisfied: cloudpickle~=2.2.1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2.2.1)\n", + "Requirement already satisfied: fastavro<2,>=0.23.6 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.9.4)\n", + "Requirement already satisfied: fasteners<1.0,>=0.3 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.19)\n", + "Requirement already satisfied: grpcio!=1.48.0,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<2,>=1.33.1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.53.0)\n", + "Requirement already satisfied: hdfs<3.0.0,>=2.1.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2.7.3)\n", + "Requirement already satisfied: httplib2<0.23.0,>=0.8 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.22.0)\n", + "Requirement already satisfied: jsonschema<5.0.0,>=4.0.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (4.19.0)\n", + "Requirement already satisfied: jsonpickle<4.0.0,>=3.0.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (3.0.2)\n", + "Requirement already satisfied: numpy<1.27.0,>=1.14.3 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.24.4)\n", + "Requirement already satisfied: objsize<0.8.0,>=0.6.1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.7.0)\n", + "Requirement already satisfied: packaging>=22.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (23.2)\n", + "Requirement already satisfied: pymongo<5.0.0,>=3.8.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (4.7.2)\n", + "Requirement already satisfied: proto-plus<2,>=1.7.1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.23.0)\n", + "Requirement already satisfied: protobuf!=4.0.*,!=4.21.*,!=4.22.0,!=4.23.*,!=4.24.*,<4.26.0,>=3.20.3 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (3.20.3)\n", + "Requirement already satisfied: pydot<2,>=1.2.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (1.4.2)\n", + "Requirement already satisfied: python-dateutil<3,>=2.8.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2.8.2)\n", + "Requirement already satisfied: pytz>=2018.3 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2022.2.1)\n", + "Requirement already satisfied: redis<6,>=5.0.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (5.0.1)\n", + "Requirement already satisfied: regex>=2020.6.8 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2023.3.23)\n", + "Requirement already satisfied: requests<3.0.0,>=2.24.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (2.31.0)\n", + "Requirement already satisfied: typing-extensions>=3.7.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (4.10.0)\n", + "Requirement already satisfied: zstandard<1,>=0.18.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.22.0)\n", + "Requirement already satisfied: pyarrow<15.0.0,>=3.0.0 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (10.0.1)\n", + "Requirement already satisfied: pyarrow-hotfix<1 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.5)\n", + "Requirement already satisfied: js2py<1,>=0.74 in /usr/local/lib/python3.10/site-packages (from apache_beam==2.56.0) (0.74)\n", + "Requirement already satisfied: docopt in /usr/local/lib/python3.10/site-packages (from hdfs<3.0.0,>=2.1.0->apache_beam==2.56.0) (0.6.2)\n", + "Requirement already satisfied: six>=1.9.0 in /usr/local/lib/python3.10/site-packages (from hdfs<3.0.0,>=2.1.0->apache_beam==2.56.0) (1.16.0)\n", + "Requirement already satisfied: pyparsing!=3.0.0,!=3.0.1,!=3.0.2,!=3.0.3,<4,>=2.4.2 in /usr/local/lib/python3.10/site-packages (from httplib2<0.23.0,>=0.8->apache_beam==2.56.0) (3.0.9)\n", + "Requirement already satisfied: tzlocal>=1.2 in /usr/local/lib/python3.10/site-packages (from js2py<1,>=0.74->apache_beam==2.56.0) (5.2)\n", + "Requirement already satisfied: pyjsparser>=2.5.1 in /usr/local/lib/python3.10/site-packages (from js2py<1,>=0.74->apache_beam==2.56.0) (2.7.1)\n", + "Requirement already satisfied: attrs>=22.2.0 in /usr/local/lib/python3.10/site-packages (from jsonschema<5.0.0,>=4.0.0->apache_beam==2.56.0) (23.1.0)\n", + "Requirement already satisfied: jsonschema-specifications>=2023.03.6 in /usr/local/lib/python3.10/site-packages (from jsonschema<5.0.0,>=4.0.0->apache_beam==2.56.0) (2023.7.1)\n", + "Requirement already satisfied: referencing>=0.28.4 in /usr/local/lib/python3.10/site-packages (from jsonschema<5.0.0,>=4.0.0->apache_beam==2.56.0) (0.30.2)\n", + "Requirement already satisfied: rpds-py>=0.7.1 in /usr/local/lib/python3.10/site-packages (from jsonschema<5.0.0,>=4.0.0->apache_beam==2.56.0) (0.10.0)\n", + "Requirement already satisfied: dnspython<3.0.0,>=1.16.0 in /usr/local/lib/python3.10/site-packages (from pymongo<5.0.0,>=3.8.0->apache_beam==2.56.0) (2.6.1)\n", + "Requirement already satisfied: async-timeout>=4.0.2 in /usr/local/lib/python3.10/site-packages (from redis<6,>=5.0.0->apache_beam==2.56.0) (4.0.3)\n", + "Requirement already satisfied: charset-normalizer<4,>=2 in /usr/local/lib/python3.10/site-packages (from requests<3.0.0,>=2.24.0->apache_beam==2.56.0) (3.2.0)\n", + "Requirement already satisfied: idna<4,>=2.5 in /usr/local/lib/python3.10/site-packages (from requests<3.0.0,>=2.24.0->apache_beam==2.56.0) (3.4)\n", + "Requirement already satisfied: urllib3<3,>=1.21.1 in /usr/local/lib/python3.10/site-packages (from requests<3.0.0,>=2.24.0->apache_beam==2.56.0) (2.0.4)\n", + "Requirement already satisfied: certifi>=2017.4.17 in /usr/local/lib/python3.10/site-packages (from requests<3.0.0,>=2.24.0->apache_beam==2.56.0) (2023.7.22)\n", + "\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m A new release of pip is available: \u001b[0m\u001b[31;49m24.1.1\u001b[0m\u001b[39;49m -> \u001b[0m\u001b[32;49m24.2\u001b[0m\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m To update, run: \u001b[0m\u001b[32;49m/usr/local/opt/python@3.10/bin/python3.10 -m pip install --upgrade pip\u001b[0m\n", + "Requirement already satisfied: redis==5.0.1 in /usr/local/lib/python3.10/site-packages (5.0.1)\n", + "Requirement already satisfied: async-timeout>=4.0.2 in /usr/local/lib/python3.10/site-packages (from redis==5.0.1) (4.0.3)\n", + "\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m A new release of pip is available: \u001b[0m\u001b[31;49m24.1.1\u001b[0m\u001b[39;49m -> \u001b[0m\u001b[32;49m24.2\u001b[0m\n", + "\u001b[1m[\u001b[0m\u001b[34;49mnotice\u001b[0m\u001b[1;39;49m]\u001b[0m\u001b[39;49m To update, run: \u001b[0m\u001b[32;49m/usr/local/opt/python@3.10/bin/python3.10 -m pip install --upgrade pip\u001b[0m\n" + ] + } + ], + "source": [ + "#installing dependencies\n", + "!pip install pandas==1.4.4\n", + "!pip install numpy==1.24.4\n", + "!pip install apache_beam==2.56.0\n", + "!pip install redis==5.0.1\n", + "!pip install langchain==0.1.14 #used for chunking" + ] + }, + { + "cell_type": "code", + "execution_count": 1, + "id": "bb8f59b0-254f-4b8e-a3dc-9015f35ef798", + "metadata": {}, + "outputs": [], + "source": [ + "#Imports Required for the notebook\n", + "import pandas as pd\n", + "import numpy as np\n", + "import apache_beam as beam\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "from apache_beam.transforms.enrichment import Enrichment\n", + "from apache_beam.ml.transforms.embeddings.huggingface import SentenceTransformerEmbeddings\n", + "import tempfile\n", + "import redis\n", + "import redis_connector\n", + "import redis_enrichment\n", + "from redis_connector import *\n", + "from redis_enrichment import *\n", + "from redis.commands.search.indexDefinition import (IndexDefinition,IndexType)\n", + "from redis.commands.search.query import Query\n", + "from redis.commands.search.field import (TextField,VectorField)\n", + "from chunks_generation import *" + ] + }, + { + "cell_type": "code", + "execution_count": 2, + "id": "3d274890-4e6b-4a3d-b682-9fc6e21e5cca", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "'2.56.0'" + ] + }, + "execution_count": 2, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "#To check beam version installed \n", + "beam.__version__" + ] + }, + { + "cell_type": "code", + "execution_count": 3, + "id": "c62d2ac3-36f5-42f2-8560-2e72421a1ff9", + "metadata": {}, + "outputs": [], + "source": [ + "import json\n", + "\n", + "with open('hf_small_wikipedia.json', 'r') as j:\n", + " contents = json.loads(j.read())" + ] + }, + { + "cell_type": "markdown", + "id": "19c1c652-b9df-4f7e-bcb5-7ee2d290e091", + "metadata": {}, + "source": [ + "# For now Reading json data locally" + ] + }, + { + "cell_type": "code", + "execution_count": 2, + "id": "fe0d6dc7-1809-44c9-9a36-b0781ec6731a", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "[{'id': '12', 'url': 'https://en.wikipedia.org/wiki/Anarchism', 'title': 'Anarchism', 'text': 'Anarchism is a political philosophy and movement that is skeptical of all ... \\nSocial theories\\nSocialism'}]\n" + ] + } + ], + "source": [ + "import json\n", + "\n", + "with open('hf_small_wikipedia.json', 'r') as j:\n", + " contents = json.loads(j.read())\n", + "\n", + "\n", + "print(contents[:1])\n" + ] + }, + { + "cell_type": "code", + "execution_count": 4, + "id": "947974b9-0218-4cb0-bd5a-1d57fd37c2f9", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "list" + ] + }, + "execution_count": 4, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "type(contents)" + ] + }, + { + "cell_type": "markdown", + "id": "aa06d33f-ed94-4bea-8b33-04c947a99034", + "metadata": {}, + "source": [ + "# Create Redis Client for connecting to Redis Vector Database" + ] + }, + { + "cell_type": "code", + "execution_count": 5, + "id": "43342378-18cd-4fd3-849c-2c6f8dc9a5ee", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "True" + ] + }, + "execution_count": 5, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "REDIS_HOST = \"localhost\"\n", + "REDIS_PORT = 6379\n", + "REDIS_PASSWORD = \"\" # default for passwordless Redis\n", + "\n", + "# Connect to Redis\n", + "redis_client = redis.Redis(\n", + " host=REDIS_HOST,\n", + " port=REDIS_PORT,\n", + " password=REDIS_PASSWORD\n", + ")\n", + "redis_client.ping()" + ] + }, + { + "cell_type": "markdown", + "id": "72cd4ad1-6577-453a-a2f7-947ae3149993", + "metadata": {}, + "source": [ + "# Creating a Search Index\n", + "Below cells will show how to specify and create a search index in Redis vector DB. Below are the following steps:\n", + "\n", + "1) Set some constants for defining our index like the distance metric and the index name\n", + "2) Define the index schema with RediSearch fields\n", + "3) Create the index" + ] + }, + { + "cell_type": "code", + "execution_count": 6, + "id": "0a1b7250-6bb1-4f29-81fd-9934e7a457cc", + "metadata": {}, + "outputs": [], + "source": [ + "#Constants\n", + "EMBEDDING_MODEL = 'all-MiniLM-L6-v2' # Embedding model name to be use with ML Transform\n", + "VECTOR_DIM = 384 # length of the vector for above embedding model\n", + "VECTOR_NUMBER = 2 # initial number of vectors\n", + "INDEX_NAME = \"embeddings-index\" # name of the search index \n", + "PREFIX = \"doc\" # prefix for the document keys \n", + "DISTANCE_METRIC = \"COSINE\" # distance metric for the vectors (ex. COSINE, IP, L2)" + ] + }, + { + "cell_type": "code", + "execution_count": 7, + "id": "78ef7e80-680e-424d-b018-be3bd71008ba", + "metadata": {}, + "outputs": [], + "source": [ + "# Define RediSearch fields for each of the columns in the dataset\n", + "url = TextField(name=\"url\")\n", + "title = TextField(name=\"title\")\n", + "title_embedding = VectorField(\"title_vector\",\n", + " \"FLAT\", {\n", + " \"TYPE\": \"FLOAT32\",\n", + " \"DIM\": VECTOR_DIM,\n", + " \"DISTANCE_METRIC\": DISTANCE_METRIC,\n", + " \"INITIAL_CAP\": VECTOR_NUMBER,\n", + " }\n", + ")\n", + "\n", + "text = TextField(name=\"text\")\n", + "text_embedding = VectorField(\"text_vector\",\n", + " \"FLAT\", {\n", + " \"TYPE\": \"FLOAT32\",\n", + " \"DIM\": VECTOR_DIM,\n", + " \"DISTANCE_METRIC\": DISTANCE_METRIC,\n", + " \"INITIAL_CAP\": VECTOR_NUMBER,\n", + " }\n", + ")\n", + "fields = [url, title, title_embedding, text, text_embedding]\n" + ] + }, + { + "cell_type": "code", + "execution_count": 8, + "id": "120eabcf-a87a-4fdf-ba29-3117dec9d858", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Index already exists\n" + ] + } + ], + "source": [ + "# Check if index exists\n", + "try:\n", + " redis_client.ft(INDEX_NAME).info()\n", + " print(\"Index already exists\")\n", + "except:\n", + " # Create RediSearch Index\n", + " redis_client.ft(INDEX_NAME).create_index(\n", + " fields = fields,\n", + " definition = IndexDefinition(prefix=[PREFIX], index_type=IndexType.HASH)\n", + ")" + ] + }, + { + "cell_type": "markdown", + "id": "f93ea35f-47d3-43b3-ab32-5dba16612337", + "metadata": {}, + "source": [ + "# Creating Knowledge Base in Redis Vector Database\n", + "After creating a search index, we can load documents into it. We will use the same documents we used in the previous cell." + ] + }, + { + "cell_type": "code", + "execution_count": 9, + "id": "5736710f-b16d-405e-a1fe-f504e753b024", + "metadata": { + "scrolled": true + }, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "INFO:root:Missing pipeline option (runner). Executing pipeline using the default runner: DirectRunner.\n", + "WARNING:apache_beam.runners.interactive.interactive_environment:Dependencies required for Interactive Beam PCollection visualization are not available, please use: `pip install apache-beam[interactive]` to install necessary dependencies to enable all data visualization features.\n" + ] + }, + { + "data": { + "application/javascript": [ + "\n", + " if (typeof window.interactive_beam_jquery == 'undefined') {\n", + " var jqueryScript = document.createElement('script');\n", + " jqueryScript.src = 'https://code.jquery.com/jquery-3.4.1.slim.min.js';\n", + " jqueryScript.type = 'text/javascript';\n", + " jqueryScript.onload = function() {\n", + " var datatableScript = document.createElement('script');\n", + " datatableScript.src = 'https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js';\n", + " datatableScript.type = 'text/javascript';\n", + " datatableScript.onload = function() {\n", + " window.interactive_beam_jquery = jQuery.noConflict(true);\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }\n", + " document.head.appendChild(datatableScript);\n", + " };\n", + " document.head.appendChild(jqueryScript);\n", + " } else {\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "INFO:apache_beam.runners.worker.statecache:Creating state cache with size 104857600\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "2024-08-09 13:01:57.330902: I tensorflow/core/platform/cpu_feature_guard.cc:182] This TensorFlow binary is optimized to use available CPU instructions in performance-critical operations.\n", + "To enable the following instructions: AVX2 FMA, in other operations, rebuild TensorFlow with the appropriate compiler flags.\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n" + ] + }, + { + "data": { + "application/vnd.jupyter.widget-view+json": { + "model_id": "7042b4db72ae4741ad73040ec6888413", + "version_major": 2, + "version_minor": 0 + }, + "text/plain": [ + "Batches: 0%| | 0/1 [00:00> beam.Create(contents) \n", + " | \"Generate text chunks\" >> ChunksGeneration(chunk_size = 500, chunk_overlap = 0, chunking_strategy = ChunkingStrategy.SPLIT_BY_TOKENS)\n", + " | \"Insert document in Redis\" >> InsertDocInRedis(host='127.0.0.1',port=6379, batch_size=10)\n", + " | \"Generate Embeddings\" >> MLTransform(write_artifact_location=artifact_location).with_transform(generate_embedding_fn) \n", + " | \"Insert Embedding in Redis\" >> InsertEmbeddingInRedis(host='127.0.0.1',port=6379, batch_size=10,embedded_columns=['title','text'])\n", + " )" + ] + }, + { + "cell_type": "markdown", + "id": "6206f83e-d3b6-4869-a7f2-2f662d3968f0", + "metadata": {}, + "source": [ + "## Pipeline Steps:\n", + "\n", + "Now that we have ingested the documents in Redis, we will create a embeddings transform, which is used for storing the text and its embedding in redis vector db\n" + ] + }, + { + "cell_type": "markdown", + "id": "f71e0cad-c062-4c12-9ba6-17010758f6db", + "metadata": {}, + "source": [ + "# Running Search Queries/ Perform Enrichment" + ] + }, + { + "cell_type": "markdown", + "id": "42697871-a5be-48cc-b961-799d69fc750b", + "metadata": {}, + "source": [ + "## Pipeline Steps:\n", + "Create a search transform, which emits the document Id, vector score along with the matching text from knowledge base\n" + ] + }, + { + "cell_type": "code", + "execution_count": 10, + "id": "bc447d60-0588-4c6d-8a5c-b3f97e12461e", + "metadata": { + "scrolled": true + }, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "INFO:root:Missing pipeline option (runner). Executing pipeline using the default runner: DirectRunner.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "WARNING:root:This output type hint will be ignored and not used for type-checking purposes. Typically, output type hints for a PTransform are single (or nested) types wrapped by a PCollection, PDone, or None. Got: Union[Tuple[apache_beam.pvalue.PCollection[~MLTransformOutputT], apache_beam.pvalue.PCollection[apache_beam.pvalue.Row]], apache_beam.pvalue.PCollection[~MLTransformOutputT]] instead.\n", + "INFO:apache_beam.runners.worker.statecache:Creating state cache with size 104857600\n", + "INFO:sentence_transformers.SentenceTransformer:Load pretrained SentenceTransformer: all-MiniLM-L6-v2\n", + "INFO:sentence_transformers.SentenceTransformer:Use pytorch device_name: mps\n" + ] + }, + { + "data": { + "application/vnd.jupyter.widget-view+json": { + "model_id": "0230c48548aa4229aaecabdea860a5f3", + "version_major": 2, + "version_minor": 0 + }, + "text/plain": [ + "Batches: 0%| | 0/1 [00:00> beam.Create(data)\n", + " | \"Generate Embedding\" >> MLTransform(write_artifact_location=artifact_location).with_transform(generate_embedding_fn)\n", + " | \"Enrich W/ Redis\" >> Enrichment(redis_handler)\n", + " | \"Print\" >> beam.Map(print)\n", + " )" + ] + }, + { + "cell_type": "markdown", + "id": "9126f5a9-179e-4059-b868-0838b0944902", + "metadata": {}, + "source": [ + "# Conclusion\n", + "\n", + "Here we have demonstrated how we can implement Ingestion and Enrichment pipeline using redis vector DB by using ML Transfrom's SentenceTransformerEmbeddings for generating the embeddings of the text chunks." + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.13" + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} diff --git a/examples/notebooks/beam-ml/rag_usecase/chunks_generation.py b/examples/notebooks/beam-ml/rag_usecase/chunks_generation.py new file mode 100644 index 0000000000000..1dd85d12f633c --- /dev/null +++ b/examples/notebooks/beam-ml/rag_usecase/chunks_generation.py @@ -0,0 +1,129 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import absolute_import + +import apache_beam as beam +from langchain.text_splitter import CharacterTextSplitter, RecursiveCharacterTextSplitter +from langchain_text_splitters import SentenceTransformersTokenTextSplitter + +from apache_beam.transforms import DoFn +from apache_beam.transforms import PTransform +from enum import Enum + + +__all__ = [ + 'ChunksGeneration', + 'ChunkingStrategy' +] + +class ChunkingStrategy(Enum): + SPLIT_BY_CHARACTER = 0 + RECURSIVE_SPLIT_BY_CHARACTER = 1 + SPLIT_BY_TOKENS = 2 + + +class ChunksGeneration(PTransform): + """ChunkingStrategy is a ``PTransform`` that takes a ``PCollection`` of + key, value tuple or 2-element array and generates different chunks for documents. + """ + + def __init__( + self, + chunk_size: int, + chunk_overlap: int, + chunking_strategy: ChunkingStrategy + ): + """ + + Args: + chunk_size : Chunk size is the maximum number of characters that a chunk can contain + chunk_overlap : the number of characters that should overlap between two adjacent chunks + chunking_strategy : Defines the way to split text + + Returns: + :class:`~apache_beam.transforms.ptransform.PTransform` + + """ + + self.chunk_size = chunk_size + self.chunk_overlap = chunk_overlap + self.chunking_strategy = chunking_strategy + + def expand(self, pcoll): + return pcoll \ + | "Generate text chunks" >> beam.ParDo(_GenerateChunksFn(self.chunk_size, + self.chunk_overlap, + self.chunking_strategy)) + + +class _GenerateChunksFn(DoFn): + """Abstract class that takes in ptransform + and generate chunks. + """ + + def __init__( + self, + chunk_size: int, + chunk_overlap: int, + chunking_strategy: ChunkingStrategy + ): + + self.chunk_size = chunk_size + self.chunk_overlap = chunk_overlap + self.chunking_strategy = chunking_strategy + + def process(self, element, *args, **kwargs): + + # For recursive split by character + if self.chunking_strategy == ChunkingStrategy.RECURSIVE_SPLIT_BY_CHARACTER: + text_splitter = RecursiveCharacterTextSplitter( + chunk_size=self.chunk_size, + chunk_overlap=self.chunk_overlap, + length_function=len, + is_separator_regex=False, + ) + + # For split by character + elif self.chunking_strategy == ChunkingStrategy.SPLIT_BY_CHARACTER: + text_splitter = CharacterTextSplitter( + chunk_size=self.chunk_size, + chunk_overlap=self.chunk_overlap, + length_function=len, + is_separator_regex=False, + ) + + # For split by tokens + elif self.chunking_strategy == ChunkingStrategy.SPLIT_BY_TOKENS: + text_splitter = SentenceTransformersTokenTextSplitter( + chunk_overlap=self.chunk_overlap, + model_name='all-MiniLM-L6-v2' + ) + + else: + raise ValueError(f"Invalid chunking strategy: {self.chunking_strategy}") + + texts = text_splitter.split_text(element['text'])[:] + + element_copy = element.copy() + del element_copy['text'] + for i, section in enumerate(texts): + element_copy['text'] = section + element_copy['section_id'] = i + 1 + yield element_copy + + diff --git a/examples/notebooks/beam-ml/rag_usecase/redis_connector.py b/examples/notebooks/beam-ml/rag_usecase/redis_connector.py new file mode 100644 index 0000000000000..039e5bee95e97 --- /dev/null +++ b/examples/notebooks/beam-ml/rag_usecase/redis_connector.py @@ -0,0 +1,349 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 + +import apache_beam as beam +import numpy as np + +from apache_beam.transforms import DoFn +from apache_beam.transforms import PTransform +from apache_beam.transforms import Reshuffle + + +import redis +from typing import Optional + +# Set the logging level to reduce verbose information +import logging + +logging.root.setLevel(logging.INFO) +logger = logging.getLogger(__name__) + +__all__ = ['InsertDocInRedis', 'InsertEmbeddingInRedis'] + + + +"""This module implements IO classes to read write documents in Redis. + + +Insert Doc in Redis: +----------------- +:class:`InsertDocInRedis` is a ``PTransform`` that writes key and values to a +configured sink, and the write is conducted through a redis pipeline. + +The ptransform works by getting the first and second elements from the input, +this means that inputs like `[k,v]` or `(k,v)` are valid. + +Example usage:: + + pipeline | InsertDocInRedis(host='localhost', + port=6379, + batch_size=100) +""" + + +class InsertDocInRedis(PTransform): + """InsertDocInRedis is a ``PTransform`` that writes a ``PCollection`` of + key, value tuple or 2-element array into a redis server. + """ + + def __init__(self, + host: str, + port: int, + command: Optional[str] = None, + batch_size: int = 100 + ): + + """ + + Args: + host (str): The redis host + port (int): The redis port + command (str): command to be executed with redis client + batch_size(int): Number of key, values pairs to write at once + + Returns: + :class:`~apache_beam.transforms.ptransform.PTransform` + + """ + + self._host = host + self._port = port + self._command = command + self._batch_size = batch_size + + def expand(self, pcoll): + return pcoll \ + | "Reshuffle for Redis Insert" >> Reshuffle() \ + | "Insert document into Redis" >> beam.ParDo(_InsertDocRedisFn(self._host, + self._port, + self._command, + self._batch_size) + ) + + +class _InsertDocRedisFn(DoFn): + """Abstract class that takes in redis + credentials to connect to redis DB + """ + + def __init__(self, + host: str, + port: int, + command: Optional[str] = None, + batch_size: int = 100 + ): + self.host = host + self.port = port + self.command = command + self.batch_size = batch_size + + self.batch_counter = 0 + self.batch = list() + + self.text_col = None + + def finish_bundle(self): + self._flush() + + def process(self, element, *args, **kwargs): + self.batch.append(element) + self.batch_counter += 1 + if self.batch_counter >= self.batch_size: + self._flush() + yield element + + def _flush(self): + if self.batch_counter == 0: + return + + with _InsertDocRedisSink(self.host, self.port) as sink: + + if not self.command: + sink.write(self.batch) + + else: + sink.execute_command(self.command, self.batch) + + self.batch_counter = 0 + self.batch = list() + + +class _InsertDocRedisSink(object): + """Class where we create redis client + and write insertion logic in redis + """ + + def __init__(self, + host: str, + port: int + ): + self.host = host + self.port = port + self.client = None + + def _create_client(self): + if self.client is None: + self.client = redis.Redis(host=self.host, + port=self.port) + + def write(self, elements): + self._create_client() + with self.client.pipeline() as pipe: + logger.info(f'Inserting documents in Redis. Total docs: {len(elements)}') + for element in elements: + doc_key = f"doc_{str(element['id'])}_section_{str(element['section_id'])}" + for k, v in element.items(): + logger.debug(f'Inserting doc_key={doc_key}, key={k}, value={v}') + pipe.hset(name=doc_key, key=k, value=v) + + pipe.execute() + logger.info(f'Inserting documents complete.') + + + def execute_command(self, command, elements): + self._create_client() + with self.client.pipeline() as pipe: + for element in elements: + k, v = element + pipe.execute_command(command, k, v) + pipe.execute() + + def __enter__(self): + self._create_client() + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + if self.client is not None: + self.client.close() + + +"""This module implements IO classes to read write text Embeddings in Redis. + + +Insert Embedding in Redis : +----------------- +:class:`InsertEmbeddingInRedis` is a ``PTransform`` that writes key and values to a +configured sink, and the write is conducted through a redis pipeline. + +The ptransform works by getting the first and second elements from the input, +this means that inputs like `[k,v]` or `(k,v)` are valid. + +Example usage:: + + pipeline | InsertEmbeddingInRedis(host='localhost', + port=6379, + batch_size=100) +""" + + +class InsertEmbeddingInRedis(PTransform): + """WriteToRedis is a ``PTransform`` that writes a ``PCollection`` of + key, value tuple or 2-element array into a redis server. + """ + + def __init__(self, + host: str, + port: int, + command: Optional[str] = None, + batch_size: int = 100, + embedded_columns: list = [] + ): + + """ + + Args: + host (str): The redis host + port (int): The redis port + command (str): command to be executed with redis client + batch_size (int): Number of key, values pairs to write at once + embedded_columns (list): list of column whose embedding needs to be generated + + Returns: + :class:`~apache_beam.transforms.ptransform.PTransform` + + """ + + self._host = host + self._port = port + self._command = command + self._batch_size = batch_size + self.embedded_columns = embedded_columns + + def expand(self, pcoll): + return pcoll \ + | "Reshuffle for Embedding in Redis Insert" >> Reshuffle() \ + | "Write `Embeddings` to Redis" >> beam.ParDo(_WriteEmbeddingInRedisFn(self._host, + self._port, + self._command, + self._batch_size, + self.embedded_columns)) + + +class _WriteEmbeddingInRedisFn(DoFn): + """Abstract class that takes in redis credentials + to connect to redis DB + """ + + def __init__(self, + host: str, + port: int, + command: Optional[str] = None, + batch_size: int = 100, + embedded_columns: list = [] + ): + self.host = host + self.port = port + self.command = command + self.batch_size = batch_size + self.embedded_columns = embedded_columns + + self.batch_counter = 0 + self.batch = list() + + def finish_bundle(self): + self._flush() + + def process(self, element, *args, **kwargs): + self.batch.append(element) + self.batch_counter += 1 + if self.batch_counter >= self.batch_size: + self._flush() + + def _flush(self): + if self.batch_counter == 0: + return + + with _InsertEmbeddingInRedisSink(self.host, self.port, self.embedded_columns) as sink: + + if not self.command: + sink.write(self.batch) + + else: + sink.execute_command(self.command, self.batch) + + self.batch_counter = 0 + self.batch = list() + + +class _InsertEmbeddingInRedisSink(object): + """Class where we create redis client + and write text embedding in redis DB + """ + + def __init__(self, + host: str, + port: int, + embedded_columns: list = [] + ): + self.host = host + self.port = port + self.client = None + self.embedded_columns = embedded_columns + + def _create_client(self): + if self.client is None: + self.client = redis.Redis(host=self.host, + port=self.port) + + def write(self, elements): + self._create_client() + with self.client.pipeline() as pipe: + for element in elements: + doc_key = f"doc_{str(element['id'])}_section_{str(element['section_id'])}" + for k, v in element.items(): + if k in self.embedded_columns: + v = np.array(v, dtype=np.float32).tobytes() + pipe.hset(name=doc_key, key=f'{k}_vector', value=v) + pipe.execute() + + def execute_command(self, command, elements): + self._create_client() + with self.client.pipeline() as pipe: + for element in elements: + k, v = element + pipe.execute_command(command, k, v) + pipe.execute() + + def __enter__(self): + self._create_client() + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + if self.client is not None: + self.client.close() \ No newline at end of file diff --git a/examples/notebooks/beam-ml/rag_usecase/redis_enrichment.py b/examples/notebooks/beam-ml/rag_usecase/redis_enrichment.py new file mode 100644 index 0000000000000..df00ede790df8 --- /dev/null +++ b/examples/notebooks/beam-ml/rag_usecase/redis_enrichment.py @@ -0,0 +1,110 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +"""This module implements enrichment classes to implement semantic search on Redis Vector DB. + + +Redis :Enrichment Handler +----------------- +:class:`RedisEnrichmentHandler` is a ``EnrichmentSourceHandler`` that performs enrichment/search +by fetching the similar text to the user query/prompt from the knowledge base (redis vector DB) and returns +the similar text along with its embeddings as Beam.Row Object. + +Example usage:: + redis_handler = RedisEnrichmentHandler(redis_host='127.0.0.1', redis_port=6379) + + pipeline | Enrichment(redis_handler) + +No backward compatibility guarantees. Everything in this module is experimental. +""" + +import numpy as np +import redis +from redis.commands.search.query import Query + +import apache_beam as beam +from apache_beam.transforms.enrichment import EnrichmentSourceHandler + +__all__ = [ + 'RedisEnrichmentHandler', +] + + +class RedisEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): + """A handler for :class:`apache_beam.transforms.enrichment.Enrichment` + transform to interact with redis vector DB. + + Args: + redis_host (str): Redis Host to connect to redis DB + redis_port (int): Redis Port to connect to redis DB + index_name (str): Index Name created for searching in Redis DB + vector_field (str): vector field to compute similarity score in vector DB + return_fields (list): returns list of similar text and its embeddings + hybrid_fields (str): fields to be selected + k (int): Value of K in KNN algorithm for searching in redis + """ + + def __init__( + self, + redis_host: str, + redis_port: int, + index_name: str = "embeddings-index", + vector_field: str = "text_vector", + return_fields: list = ["id", "title", "url", "text"], + hybrid_fields: str = "*", + k: int = 2, + ): + self.redis_host = redis_host + self.redis_port = redis_port + self.index_name = index_name + self.vector_field = vector_field + self.return_fields = return_fields + self.hybrid_fields = hybrid_fields + self.k = k + self.client = None + + def __enter__(self): + """connect to the redis DB using redis client.""" + self.client = redis.Redis(host=self.redis_host, port=self.redis_port) + + def __call__(self, request: beam.Row, *args, **kwargs): + """ + Reads a row from the redis Vector DB and returns + a `Tuple` of request and response. + + Args: + request: the input `beam.Row` to enrich. + """ + + # read embedding vector for user query + + embedded_query = request['text'] + + # Prepare the Query + base_query = f'{self.hybrid_fields}=>[KNN {self.k} @{self.vector_field} $vector AS vector_score]' + query = ( + Query(base_query) + .return_fields(*self.return_fields) + .paging(0, self.k) + .dialect(2) + ) + + params_dict = {"vector": np.array(embedded_query).astype(dtype=np.float32).tobytes()} + + # perform vector search + results = self.client.ft(self.index_name).search(query, params_dict) + + return beam.Row(text=embedded_query), beam.Row(docs=results.docs) \ No newline at end of file From 9aaf7e41dec8b9d3effdc7cb6c887f8df8e62a64 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Mon, 12 Aug 2024 11:00:54 -0400 Subject: [PATCH 67/78] fix link (#32156) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8a9bb4e988d58..8a6db16220683 100644 --- a/README.md +++ b/README.md @@ -109,7 +109,7 @@ Here are some resources actively maintained by the Beam community to help you ge A comprehensive, interactive learning experience covering Beam concepts in depth. - Beam Quest + Beam Quest A certification granted by Google Cloud, certifying proficiency in Beam. From edf4c4f5f19ef6ffd25493262261c713ba045980 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 10:20:55 -0700 Subject: [PATCH 68/78] Bump golang.org/x/sys from 0.23.0 to 0.24.0 in /sdks (#32150) Bumps [golang.org/x/sys](https://github.com/golang/sys) from 0.23.0 to 0.24.0. - [Commits](https://github.com/golang/sys/compare/v0.23.0...v0.24.0) --- updated-dependencies: - dependency-name: golang.org/x/sys dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5d179736baf09..7c3ade37490d1 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,7 +56,7 @@ require ( golang.org/x/net v0.28.0 golang.org/x/oauth2 v0.22.0 golang.org/x/sync v0.8.0 - golang.org/x/sys v0.23.0 + golang.org/x/sys v0.24.0 golang.org/x/text v0.17.0 google.golang.org/api v0.191.0 google.golang.org/genproto v0.0.0-20240730163845-b1a4ccb954bf diff --git a/sdks/go.sum b/sdks/go.sum index 2b43c6d62bc92..5b9bfaef8a48d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1507,8 +1507,8 @@ golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.23.0 h1:YfKFowiIMvtgl1UERQoTPPToxltDeZfbj4H7dVUCwmM= -golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= From df10609129093bebdd857d4d9e6daaf68d3cbfbb Mon Sep 17 00:00:00 2001 From: Damon Date: Mon, 12 Aug 2024 14:52:26 -0700 Subject: [PATCH 69/78] Update names.py with container image tag (#32160) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index 40147e9926dc6..3d51bd21e705b 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20240613' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20240809' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From 74668038c023a451bd84075c2ea33ee71c67336d Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Tue, 13 Aug 2024 01:52:54 +0400 Subject: [PATCH 70/78] Support withFormatRecordOnFailureFunction() for BigQuery STORAGE_WRITE_API and STORAGE_API_AT_LEAST_ONCE methods (#31659) * Support withFormatRecordOnFailureFunction() for BigQuery STORAGE_WRITE_API and STORAGE_API_AT_LEAST_ONCE methods * Update CHANGES.md --- CHANGES.md | 1 + .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 18 +- .../io/gcp/bigquery/SplittingIterable.java | 18 +- .../bigquery/StorageApiConvertMessages.java | 8 +- .../StorageApiDynamicDestinations.java | 2 +- .../StorageApiDynamicDestinationsBeamRow.java | 16 +- ...geApiDynamicDestinationsGenericRecord.java | 20 +- .../StorageApiDynamicDestinationsProto.java | 31 +- ...StorageApiDynamicDestinationsTableRow.java | 16 +- .../gcp/bigquery/StorageApiWritePayload.java | 22 +- .../StorageApiWriteUnshardedRecords.java | 65 ++-- .../StorageApiWritesShardedRecords.java | 26 +- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 336 +++++++++++++++++- 13 files changed, 522 insertions(+), 57 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index d082f03fd310e..950abc694488e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,6 +69,7 @@ * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Go SDK Minimum Go Version updated to 1.21 ([#32092](https://github.com/apache/beam/pull/32092)). +* [BigQueryIO] Added support for withFormatRecordOnFailureFunction() for STORAGE_WRITE_API and STORAGE_API_AT_LEAST_ONCE methods (Java) ([#31354](https://github.com/apache/beam/issues/31354)). * Updated Go protobuf package to new version (Go) ([#21515](https://github.com/apache/beam/issues/21515)). ## Breaking Changes diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 1238271c791ea..2a16bf31a6cba 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2711,9 +2711,14 @@ public Write withFormatFunction(SerializableFunction formatFunct } /** - * If an insert failure occurs, this function is applied to the originally supplied row T. The - * resulting {@link TableRow} will be accessed via {@link - * WriteResult#getFailedInsertsWithErr()}. + * If an insert failure occurs, this function is applied to the originally supplied T element. + * + *

    For {@link Method#STREAMING_INSERTS} method, the resulting {@link TableRow} will be + * accessed via {@link WriteResult#getFailedInsertsWithErr()}. + * + *

    For {@link Method#STORAGE_WRITE_API} and {@link Method#STORAGE_API_AT_LEAST_ONCE} methods, + * the resulting {@link TableRow} will be accessed via {@link + * WriteResult#getFailedStorageApiInserts()}. */ public Write withFormatRecordOnFailureFunction( SerializableFunction formatFunction) { @@ -3773,6 +3778,7 @@ private WriteResult continueExpandTyped( dynamicDestinations, elementSchema, elementToRowFunction, + getFormatRecordOnFailureFunction(), getRowMutationInformationFn() != null); } else if (getWriteProtosClass() != null && getDirectWriteProtos()) { // We could support both of these by falling back to @@ -3795,7 +3801,9 @@ private WriteResult continueExpandTyped( storageApiDynamicDestinations = (StorageApiDynamicDestinations) new StorageApiDynamicDestinationsProto( - dynamicDestinations, getWriteProtosClass()); + dynamicDestinations, + getWriteProtosClass(), + getFormatRecordOnFailureFunction()); } else if (getAvroRowWriterFactory() != null) { // we can configure the avro to storage write api proto converter for this // assuming the format function returns an Avro GenericRecord @@ -3818,6 +3826,7 @@ private WriteResult continueExpandTyped( dynamicDestinations, avroSchemaFactory, recordWriterFactory.getToAvroFn(), + getFormatRecordOnFailureFunction(), getRowMutationInformationFn() != null); } else { RowWriterFactory.TableRowWriterFactory tableRowWriterFactory = @@ -3827,6 +3836,7 @@ private WriteResult continueExpandTyped( new StorageApiDynamicDestinationsTableRow<>( dynamicDestinations, tableRowWriterFactory.getToRowFn(), + getFormatRecordOnFailureFunction(), getRowMutationInformationFn() != null, getCreateDisposition(), getIgnoreUnknownValues(), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java index b8eeb2522cf2b..e40824eab08b6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java @@ -21,6 +21,7 @@ import com.google.auto.value.AutoValue; import com.google.cloud.bigquery.storage.v1.ProtoRows; import com.google.protobuf.ByteString; +import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -42,6 +43,8 @@ abstract static class Value { abstract ProtoRows getProtoRows(); abstract List getTimestamps(); + + abstract List<@Nullable TableRow> getFailsafeTableRows(); } interface ConvertUnknownFields { @@ -96,11 +99,18 @@ public Value next() { } List timestamps = Lists.newArrayList(); + List<@Nullable TableRow> failsafeRows = Lists.newArrayList(); ProtoRows.Builder inserts = ProtoRows.newBuilder(); long bytesSize = 0; while (underlyingIterator.hasNext()) { StorageApiWritePayload payload = underlyingIterator.next(); ByteString byteString = ByteString.copyFrom(payload.getPayload()); + @Nullable TableRow failsafeTableRow = null; + try { + failsafeTableRow = payload.getFailsafeTableRow(); + } catch (IOException e) { + // Do nothing, table row will be generated later from row bytes + } if (autoUpdateSchema) { try { @Nullable TableRow unknownFields = payload.getUnknownFields(); @@ -116,7 +126,10 @@ public Value next() { // This generally implies that ignoreUnknownValues=false and there were still // unknown values here. // Reconstitute the TableRow and send it to the failed-rows consumer. - TableRow tableRow = protoToTableRow.apply(byteString); + TableRow tableRow = + failsafeTableRow != null + ? failsafeTableRow + : protoToTableRow.apply(byteString); // TODO(24926, reuvenlax): We need to merge the unknown fields in! Currently we // only execute this // codepath when ignoreUnknownFields==true, so we should never hit this codepath. @@ -142,12 +155,13 @@ public Value next() { timestamp = elementsTimestamp; } timestamps.add(timestamp); + failsafeRows.add(failsafeTableRow); bytesSize += byteString.size(); if (bytesSize > splitSize) { break; } } - return new AutoValue_SplittingIterable_Value(inserts.build(), timestamps); + return new AutoValue_SplittingIterable_Value(inserts.build(), timestamps, failsafeRows); } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java index aefdb79c535c8..0c6f82b9df813 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java @@ -179,15 +179,17 @@ public void processElement( .withTimestamp(timestamp); o.get(successfulWritesTag).output(KV.of(element.getKey(), payload)); } catch (TableRowToStorageApiProto.SchemaConversionException conversionException) { - TableRow tableRow; + TableRow failsafeTableRow; try { - tableRow = messageConverter.toTableRow(element.getValue()); + failsafeTableRow = messageConverter.toFailsafeTableRow(element.getValue()); } catch (Exception e) { badRecordRouter.route(o, element, elementCoder, e, "Unable to convert value to TableRow"); return; } o.get(failedWritesTag) - .output(new BigQueryStorageApiInsertError(tableRow, conversionException.toString())); + .output( + new BigQueryStorageApiInsertError( + failsafeTableRow, conversionException.toString())); } catch (Exception e) { badRecordRouter.route( o, element, elementCoder, e, "Unable to convert value to StorageWriteApiPayload"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java index 8ec4d52e3b90f..87667ef2cb171 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java @@ -34,7 +34,7 @@ public interface MessageConverter { StorageApiWritePayload toMessage( T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception; - TableRow toTableRow(T element); + TableRow toFailsafeTableRow(T element); } StorageApiDynamicDestinations(DynamicDestinations inner) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java index 70ecb06d5b8d7..fd5fe27f0c7c6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java @@ -35,6 +35,7 @@ class StorageApiDynamicDestinationsBeamRow { private final TableSchema tableSchema; private final SerializableFunction toRow; + private final @Nullable SerializableFunction formatRecordOnFailureFunction; private final boolean usesCdc; @@ -42,10 +43,12 @@ class StorageApiDynamicDestinationsBeamRow inner, Schema schema, SerializableFunction toRow, + @Nullable SerializableFunction formatRecordOnFailureFunction, boolean usesCdc) { super(inner); this.tableSchema = BeamRowToStorageApiProto.protoTableSchemaFromBeamSchema(schema); this.toRow = toRow; + this.formatRecordOnFailureFunction = formatRecordOnFailureFunction; this.usesCdc = usesCdc; } @@ -96,12 +99,19 @@ public StorageApiWritePayload toMessage( Message msg = BeamRowToStorageApiProto.messageFromBeamRow( descriptorToUse, toRow.apply(element), changeType, changeSequenceNum); - return StorageApiWritePayload.of(msg.toByteArray(), null); + return StorageApiWritePayload.of( + msg.toByteArray(), + null, + formatRecordOnFailureFunction != null ? toFailsafeTableRow(element) : null); } @Override - public TableRow toTableRow(T element) { - return BigQueryUtils.toTableRow(toRow.apply(element)); + public TableRow toFailsafeTableRow(T element) { + if (formatRecordOnFailureFunction != null) { + return formatRecordOnFailureFunction.apply(element); + } else { + return BigQueryUtils.toTableRow(toRow.apply(element)); + } } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java index c96bb4ce75236..a387495863a26 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java @@ -36,16 +36,21 @@ class StorageApiDynamicDestinationsGenericRecord, GenericRecord> toGenericRecord; private final SerializableFunction<@Nullable TableSchema, Schema> schemaFactory; + private final @javax.annotation.Nullable SerializableFunction + formatRecordOnFailureFunction; + private boolean usesCdc; StorageApiDynamicDestinationsGenericRecord( DynamicDestinations inner, SerializableFunction<@Nullable TableSchema, Schema> schemaFactory, SerializableFunction, GenericRecord> toGenericRecord, + @Nullable SerializableFunction formatRecordOnFailureFunction, boolean usesCdc) { super(inner); this.toGenericRecord = toGenericRecord; this.schemaFactory = schemaFactory; + this.formatRecordOnFailureFunction = formatRecordOnFailureFunction; this.usesCdc = usesCdc; } @@ -96,13 +101,20 @@ public StorageApiWritePayload toMessage( toGenericRecord.apply(new AvroWriteRequest<>(element, avroSchema)), changeType, changeSequenceNum); - return StorageApiWritePayload.of(msg.toByteArray(), null); + return StorageApiWritePayload.of( + msg.toByteArray(), + null, + formatRecordOnFailureFunction != null ? toFailsafeTableRow(element) : null); } @Override - public TableRow toTableRow(T element) { - return BigQueryUtils.convertGenericRecordToTableRow( - toGenericRecord.apply(new AvroWriteRequest<>(element, avroSchema)), bqTableSchema); + public TableRow toFailsafeTableRow(T element) { + if (formatRecordOnFailureFunction != null) { + return formatRecordOnFailureFunction.apply(element); + } else { + return BigQueryUtils.convertGenericRecordToTableRow( + toGenericRecord.apply(new AvroWriteRequest<>(element, avroSchema)), bqTableSchema); + } } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java index 57dbdc9d1e770..d7359f99b96df 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java @@ -22,23 +22,29 @@ import com.google.cloud.bigquery.storage.v1.TableSchema; import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; import java.lang.reflect.InvocationTargetException; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.Preconditions; import org.checkerframework.checker.nullness.qual.NonNull; /** Storage API DynamicDestinations used when the input is a compiled protocol buffer. */ class StorageApiDynamicDestinationsProto extends StorageApiDynamicDestinations { - DescriptorProtos.DescriptorProto descriptorProto; + private final DescriptorProtos.DescriptorProto descriptorProto; + private final @Nullable SerializableFunction formatRecordOnFailureFunction; @SuppressWarnings({"unchecked", "nullness"}) StorageApiDynamicDestinationsProto( - DynamicDestinations inner, Class protoClass) { + DynamicDestinations inner, + Class protoClass, + @Nullable SerializableFunction formatRecordOnFailureFunction) { super(inner); try { + this.formatRecordOnFailureFunction = formatRecordOnFailureFunction; this.descriptorProto = fixNestedTypes( (Descriptors.Descriptor) @@ -84,12 +90,27 @@ public StorageApiWritePayload toMessage( // we can forward // the through directly. This means that we don't currently support ignoreUnknownValues or // autoUpdateSchema. - return StorageApiWritePayload.of(element.toByteArray(), null); + return StorageApiWritePayload.of( + element.toByteArray(), + null, + formatRecordOnFailureFunction != null ? toFailsafeTableRow(element) : null); } @Override - public TableRow toTableRow(T element) { - throw new RuntimeException("Not implemented!"); + public TableRow toFailsafeTableRow(T element) { + if (formatRecordOnFailureFunction != null) { + return formatRecordOnFailureFunction.apply(element); + } else { + try { + return TableRowToStorageApiProto.tableRowFromMessage( + DynamicMessage.parseFrom( + TableRowToStorageApiProto.wrapDescriptorProto(descriptorProto), + element.toByteArray()), + true); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } }; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java index 264dac34473ed..08588cfc78500 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java @@ -36,6 +36,7 @@ public class StorageApiDynamicDestinationsTableRow extends StorageApiDynamicDestinations { private final SerializableFunction formatFunction; + private final @Nullable SerializableFunction formatRecordOnFailureFunction; private final boolean usesCdc; private final CreateDisposition createDisposition; @@ -51,12 +52,14 @@ public class StorageApiDynamicDestinationsTableRow inner, SerializableFunction formatFunction, + @Nullable SerializableFunction formatRecordOnFailureFunction, boolean usesCdc, CreateDisposition createDisposition, boolean ignoreUnknownValues, boolean autoSchemaUpdates) { super(inner); this.formatFunction = formatFunction; + this.formatRecordOnFailureFunction = formatRecordOnFailureFunction; this.usesCdc = usesCdc; this.createDisposition = createDisposition; this.ignoreUnknownValues = ignoreUnknownValues; @@ -151,8 +154,12 @@ public DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) } @Override - public TableRow toTableRow(T element) { - return formatFunction.apply(element); + public TableRow toFailsafeTableRow(T element) { + if (formatRecordOnFailureFunction != null) { + return formatRecordOnFailureFunction.apply(element); + } else { + return formatFunction.apply(element); + } } @Override @@ -183,7 +190,10 @@ public StorageApiWritePayload toMessage( unknownFields, changeType, changeSequenceNum); - return StorageApiWritePayload.of(msg.toByteArray(), unknownFields); + return StorageApiWritePayload.of( + msg.toByteArray(), + unknownFields, + formatRecordOnFailureFunction != null ? toFailsafeTableRow(element) : null); } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java index 5b6f27949870b..f0fce11b2d32b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java @@ -39,12 +39,17 @@ public abstract class StorageApiWritePayload { public abstract @Nullable Instant getTimestamp(); + @SuppressWarnings("mutable") + public abstract @Nullable byte[] getFailsafeTableRowPayload(); + @AutoValue.Builder public abstract static class Builder { public abstract Builder setPayload(byte[] value); public abstract Builder setUnknownFieldsPayload(@Nullable byte[] value); + public abstract Builder setFailsafeTableRowPayload(@Nullable byte[] value); + public abstract Builder setTimestamp(@Nullable Instant value); public abstract StorageApiWritePayload build(); @@ -53,15 +58,22 @@ public abstract static class Builder { public abstract Builder toBuilder(); @SuppressWarnings("nullness") - static StorageApiWritePayload of(byte[] payload, @Nullable TableRow unknownFields) + static StorageApiWritePayload of( + byte[] payload, @Nullable TableRow unknownFields, @Nullable TableRow failsafeTableRow) throws IOException { @Nullable byte[] unknownFieldsPayload = null; if (unknownFields != null) { unknownFieldsPayload = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), unknownFields); } + @Nullable byte[] failsafeTableRowPayload = null; + if (failsafeTableRow != null) { + failsafeTableRowPayload = + CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), failsafeTableRow); + } return new AutoValue_StorageApiWritePayload.Builder() .setPayload(payload) .setUnknownFieldsPayload(unknownFieldsPayload) + .setFailsafeTableRowPayload(failsafeTableRowPayload) .setTimestamp(null) .build(); } @@ -77,4 +89,12 @@ public StorageApiWritePayload withTimestamp(Instant instant) { } return CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), fields); } + + public @Memoized @Nullable TableRow getFailsafeTableRow() throws IOException { + @Nullable byte[] failsafeTableRowPayload = getFailsafeTableRowPayload(); + if (failsafeTableRowPayload == null) { + return null; + } + return CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), failsafeTableRowPayload); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 8a902ec6d264e..369bb2d78634c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import com.google.api.core.ApiFuture; @@ -257,15 +258,20 @@ static class AppendRowsContext extends RetryManager.Operation.Context timestamps; + List<@Nullable TableRow> failsafeTableRows; int failureCount; public AppendRowsContext( - long offset, ProtoRows protoRows, List timestamps) { + long offset, + ProtoRows protoRows, + List timestamps, + List<@Nullable TableRow> failsafeTableRows) { this.offset = offset; this.protoRows = protoRows; this.timestamps = timestamps; this.failureCount = 0; + this.failsafeTableRows = failsafeTableRows; } } @@ -278,6 +284,7 @@ class DestinationState { private long currentOffset = 0; private List pendingMessages; private List pendingTimestamps; + private List<@Nullable TableRow> pendingFailsafeTableRows; private transient @Nullable WriteStreamService maybeWriteStreamService; private final Counter recordsAppended = Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); @@ -319,6 +326,7 @@ public DestinationState( this.shortTableUrn = shortTableUrn; this.pendingMessages = Lists.newArrayList(); this.pendingTimestamps = Lists.newArrayList(); + this.pendingFailsafeTableRows = Lists.newArrayList(); this.maybeWriteStreamService = writeStreamService; this.useDefaultStream = useDefaultStream; this.initialTableSchema = messageConverter.getTableSchema(); @@ -553,6 +561,7 @@ void addMessage( throws Exception { maybeTickleCache(); ByteString payloadBytes = ByteString.copyFrom(payload.getPayload()); + @Nullable TableRow failsafeTableRow = payload.getFailsafeTableRow(); if (autoUpdateSchema) { if (appendClientInfo == null) { appendClientInfo = getAppendClientInfo(true, null); @@ -565,7 +574,10 @@ void addMessage( Preconditions.checkStateNotNull(appendClientInfo) .encodeUnknownFields(unknownFields, ignoreUnknownValues)); } catch (TableRowToStorageApiProto.SchemaConversionException e) { - TableRow tableRow = appendClientInfo.toTableRow(payloadBytes); + @Nullable TableRow tableRow = payload.getFailsafeTableRow(); + if (tableRow == null) { + tableRow = checkNotNull(appendClientInfo).toTableRow(payloadBytes); + } // TODO(24926, reuvenlax): We need to merge the unknown fields in! Currently we only // execute this // codepath when ignoreUnknownFields==true, so we should never hit this codepath. @@ -583,6 +595,8 @@ void addMessage( } } pendingMessages.add(payloadBytes); + pendingFailsafeTableRows.add(failsafeTableRow); + org.joda.time.Instant timestamp = payload.getTimestamp(); pendingTimestamps.add(timestamp != null ? timestamp : elementTs); } @@ -601,7 +615,9 @@ long flush( pendingMessages.clear(); final ProtoRows inserts = insertsBuilder.build(); List insertTimestamps = pendingTimestamps; + List<@Nullable TableRow> failsafeTableRows = pendingFailsafeTableRows; pendingTimestamps = Lists.newArrayList(); + pendingFailsafeTableRows = Lists.newArrayList(); // Handle the case where the request is too large. if (inserts.getSerializedSize() >= maxRequestSize) { @@ -616,15 +632,18 @@ long flush( maxRequestSize); } for (int i = 0; i < inserts.getSerializedRowsCount(); ++i) { - ByteString rowBytes = inserts.getSerializedRows(i); + @Nullable TableRow failedRow = failsafeTableRows.get(i); + if (failedRow == null) { + ByteString rowBytes = inserts.getSerializedRows(i); + failedRow = + TableRowToStorageApiProto.tableRowFromMessage( + DynamicMessage.parseFrom( + TableRowToStorageApiProto.wrapDescriptorProto( + getAppendClientInfo(true, null).getDescriptor()), + rowBytes), + true); + } org.joda.time.Instant timestamp = insertTimestamps.get(i); - TableRow failedRow = - TableRowToStorageApiProto.tableRowFromMessage( - DynamicMessage.parseFrom( - TableRowToStorageApiProto.wrapDescriptorProto( - getAppendClientInfo(true, null).getDescriptor()), - rowBytes), - true); failedRowsReceiver.outputWithTimestamp( new BigQueryStorageApiInsertError( failedRow, "Row payload too large. Maximum size " + maxRequestSize), @@ -647,7 +666,7 @@ long flush( this.currentOffset += inserts.getSerializedRowsCount(); } AppendRowsContext appendRowsContext = - new AppendRowsContext(offset, inserts, insertTimestamps); + new AppendRowsContext(offset, inserts, insertTimestamps, failsafeTableRows); retryManager.addOperation( c -> { @@ -692,18 +711,22 @@ long flush( Set failedRowIndices = error.getRowIndexToErrorMessage().keySet(); for (int failedIndex : failedRowIndices) { // Convert the message to a TableRow and send it to the failedRows collection. - ByteString protoBytes = failedContext.protoRows.getSerializedRows(failedIndex); - org.joda.time.Instant timestamp = failedContext.timestamps.get(failedIndex); BigQueryStorageApiInsertError element = null; + org.joda.time.Instant timestamp = failedContext.timestamps.get(failedIndex); try { - TableRow failedRow = - TableRowToStorageApiProto.tableRowFromMessage( - DynamicMessage.parseFrom( - TableRowToStorageApiProto.wrapDescriptorProto( - Preconditions.checkStateNotNull(appendClientInfo) - .getDescriptor()), - protoBytes), - true); + TableRow failedRow = failedContext.failsafeTableRows.get(failedIndex); + if (failedRow == null) { + ByteString protoBytes = + failedContext.protoRows.getSerializedRows(failedIndex); + failedRow = + TableRowToStorageApiProto.tableRowFromMessage( + DynamicMessage.parseFrom( + TableRowToStorageApiProto.wrapDescriptorProto( + Preconditions.checkStateNotNull(appendClientInfo) + .getDescriptor()), + protoBytes), + true); + } element = new BigQueryStorageApiInsertError( failedRow, error.getRowIndexToErrorMessage().get(failedIndex)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index a7da19a75f850..f3f512110b50f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -149,12 +149,17 @@ class AppendRowsContext extends RetryManager.Operation.Context timestamps; + List<@Nullable TableRow> failsafeTableRows; AppendRowsContext( - ShardedKey key, ProtoRows protoRows, List timestamps) { + ShardedKey key, + ProtoRows protoRows, + List timestamps, + List<@Nullable TableRow> failsafeTableRows) { this.key = key; this.protoRows = protoRows; this.timestamps = timestamps; + this.failsafeTableRows = failsafeTableRows; } @Override @@ -685,8 +690,11 @@ public void process( Set failedRowIndices = error.getRowIndexToErrorMessage().keySet(); for (int failedIndex : failedRowIndices) { // Convert the message to a TableRow and send it to the failedRows collection. - ByteString protoBytes = failedContext.protoRows.getSerializedRows(failedIndex); - TableRow failedRow = appendClientInfo.get().toTableRow(protoBytes); + TableRow failedRow = failedContext.failsafeTableRows.get(failedIndex); + if (failedRow == null) { + ByteString protoBytes = failedContext.protoRows.getSerializedRows(failedIndex); + failedRow = appendClientInfo.get().toTableRow(protoBytes); + } org.joda.time.Instant timestamp = failedContext.timestamps.get(failedIndex); o.get(failedRowsTag) .outputWithTimestamp( @@ -851,9 +859,12 @@ public void process( + ". This is unexpected. All rows in the request will be sent to the failed-rows PCollection."); } for (int i = 0; i < splitValue.getProtoRows().getSerializedRowsCount(); ++i) { - ByteString rowBytes = splitValue.getProtoRows().getSerializedRows(i); org.joda.time.Instant timestamp = splitValue.getTimestamps().get(i); - TableRow failedRow = appendClientInfo.get().toTableRow(rowBytes); + TableRow failedRow = splitValue.getFailsafeTableRows().get(i); + if (failedRow == null) { + ByteString rowBytes = splitValue.getProtoRows().getSerializedRows(i); + failedRow = appendClientInfo.get().toTableRow(rowBytes); + } o.get(failedRowsTag) .outputWithTimestamp( new BigQueryStorageApiInsertError( @@ -872,7 +883,10 @@ public void process( // RetryManager AppendRowsContext context = new AppendRowsContext( - element.getKey(), splitValue.getProtoRows(), splitValue.getTimestamps()); + element.getKey(), + splitValue.getProtoRows(), + splitValue.getTimestamps(), + splitValue.getFailsafeTableRows()); contexts.add(context); retryManager.addOperation(runOperation, onError, onSuccess, context); recordsAppended.inc(splitValue.getProtoRows().getSerializedRowsCount()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index c5af8045bfe20..2736ed7beb881 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -83,6 +83,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.LongStream; import java.util.stream.StreamSupport; import org.apache.avro.Schema.Field; @@ -3065,7 +3066,312 @@ public void testStreamingInsertsExtendedErrorRetrieval() throws Exception { } @Test - public void testStorageApiErrors() throws Exception { + public void testStorageApiErrorsWriteProto() throws Exception { + assumeTrue(useStorageApi); + final Method method = + useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API; + + final int failFrom = 10; + + Function getPrimitive = + (Integer i) -> + Proto3SchemaMessages.Primitive.newBuilder() + .setPrimitiveDouble(i) + .setPrimitiveFloat(i) + .setPrimitiveInt32(i) + .setPrimitiveInt64(i) + .setPrimitiveUint32(i) + .setPrimitiveUint64(i) + .setPrimitiveSint32(i) + .setPrimitiveSint64(i) + .setPrimitiveFixed32(i) + .setPrimitiveFixed64(i) + .setPrimitiveBool(true) + .setPrimitiveString(Integer.toString(i)) + .setPrimitiveBytes( + ByteString.copyFrom(Integer.toString(i).getBytes(StandardCharsets.UTF_8))) + .build(); + List goodRows = + IntStream.range(1, 20).mapToObj(getPrimitive::apply).collect(Collectors.toList()); + + Function getPrimitiveRow = + (Integer i) -> + new TableRow() + .set("primitive_double", Double.valueOf(i)) + .set("primitive_float", Float.valueOf(i).doubleValue()) + .set("primitive_int32", i.intValue()) + .set("primitive_int64", i.toString()) + .set("primitive_uint32", i.toString()) + .set("primitive_uint64", i.toString()) + .set("primitive_sint32", i.toString()) + .set("primitive_sint64", i.toString()) + .set("primitive_fixed32", i.toString()) + .set("primitive_fixed64", i.toString()) + .set("primitive_bool", true) + .set("primitive_string", i.toString()) + .set( + "primitive_bytes", + BaseEncoding.base64() + .encode( + ByteString.copyFrom(i.toString().getBytes(StandardCharsets.UTF_8)) + .toByteArray())); + + Function shouldFailRow = + (Function & Serializable) + tr -> + tr.containsKey("primitive_int32") + && (Integer) tr.get("primitive_int32") >= failFrom; + fakeDatasetService.setShouldFailRow(shouldFailRow); + + SerializableFunction formatRecordOnFailureFunction = + input -> { + TableRow failedTableRow = new TableRow().set("testFailureFunctionField", "testValue"); + failedTableRow.set("originalValue", input.getPrimitiveFixed32()); + return failedTableRow; + }; + + WriteResult result = + p.apply(Create.of(goodRows)) + .apply( + BigQueryIO.writeProtos(Proto3SchemaMessages.Primitive.class) + .to("project-id:dataset-id.table") + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withMethod(method) + .withoutValidation() + .withFormatRecordOnFailureFunction(formatRecordOnFailureFunction) + .withPropagateSuccessfulStorageApiWrites(true) + .withTestServices(fakeBqServices)); + + PCollection deadRows = + result + .getFailedStorageApiInserts() + .apply( + MapElements.into(TypeDescriptor.of(TableRow.class)) + .via(BigQueryStorageApiInsertError::getRow)); + + List expectedFailedRows = + goodRows.stream() + .filter(primitive -> primitive.getPrimitiveFixed32() >= failFrom) + .map(formatRecordOnFailureFunction::apply) + .collect(Collectors.toList()); + PAssert.that(deadRows).containsInAnyOrder(expectedFailedRows); + p.run(); + + // Round trip through the coder to make sure the types match our expected types. + assertThat( + fakeDatasetService.getAllRows("project-id", "dataset-id", "table").stream() + .map( + tr -> { + try { + byte[] bytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), tr); + return CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), bytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()), + containsInAnyOrder( + Iterables.toArray( + Iterables.filter( + goodRows.stream() + .map(primitive -> getPrimitiveRow.apply(primitive.getPrimitiveFixed32())) + .collect(Collectors.toList()), + r -> !shouldFailRow.apply(r)), + TableRow.class))); + } + + @Test + public void testStorageApiErrorsWriteBeamRow() throws Exception { + assumeTrue(useStorageApi); + final Method method = + useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API; + + final int failFrom = 10; + final String shouldFailName = "failme"; + + List goodRows = + Lists.newArrayList( + new SchemaPojo("a", 1), + new SchemaPojo("b", 2), + new SchemaPojo("c", 10), + new SchemaPojo("d", 11), + new SchemaPojo(shouldFailName, 1)); + + String nameField = "name"; + String numberField = "number"; + Function shouldFailRow = + (Function & Serializable) + tr -> + shouldFailName.equals(tr.get(nameField)) + || (Integer.valueOf((String) tr.get(numberField)) >= failFrom); + fakeDatasetService.setShouldFailRow(shouldFailRow); + + SerializableFunction formatRecordOnFailureFunction = + input -> { + TableRow failedTableRow = new TableRow().set("testFailureFunctionField", "testValue"); + failedTableRow.set("originalName", input.name); + failedTableRow.set("originalNumber", input.number); + return failedTableRow; + }; + + WriteResult result = + p.apply(Create.of(goodRows)) + .apply( + BigQueryIO.write() + .to("project-id:dataset-id.table") + .withMethod(method) + .useBeamSchema() + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()) + .withPropagateSuccessfulStorageApiWrites(true) + .withTestServices(fakeBqServices) + .withFormatRecordOnFailureFunction(formatRecordOnFailureFunction) + .withoutValidation()); + + PCollection deadRows = + result + .getFailedStorageApiInserts() + .apply( + MapElements.into(TypeDescriptor.of(TableRow.class)) + .via(BigQueryStorageApiInsertError::getRow)); + PCollection successfulRows = result.getSuccessfulStorageApiInserts(); + + List expectedFailedRows = + goodRows.stream() + .filter(pojo -> shouldFailName.equals(pojo.name) || pojo.number >= failFrom) + .map(formatRecordOnFailureFunction::apply) + .collect(Collectors.toList()); + PAssert.that(deadRows).containsInAnyOrder(expectedFailedRows); + PAssert.that(successfulRows) + .containsInAnyOrder( + Iterables.toArray( + Iterables.filter( + goodRows.stream() + .map( + pojo -> { + TableRow tableRow = new TableRow(); + tableRow.set(nameField, pojo.name); + tableRow.set(numberField, String.valueOf(pojo.number)); + return tableRow; + }) + .collect(Collectors.toList()), + r -> !shouldFailRow.apply(r)), + TableRow.class)); + p.run(); + + assertThat( + fakeDatasetService.getAllRows("project-id", "dataset-id", "table"), + containsInAnyOrder( + Iterables.toArray( + Iterables.filter( + goodRows.stream() + .map( + pojo -> { + TableRow tableRow = new TableRow(); + tableRow.set(nameField, pojo.name); + tableRow.set(numberField, String.valueOf(pojo.number)); + return tableRow; + }) + .collect(Collectors.toList()), + r -> !shouldFailRow.apply(r)), + TableRow.class))); + } + + @Test + public void testStorageApiErrorsWriteGenericRecord() throws Exception { + assumeTrue(useStorageApi); + final Method method = + useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API; + + final long failFrom = 10L; + List goodRows = LongStream.range(0, 20).boxed().collect(Collectors.toList()); + + String fieldName = "number"; + Function shouldFailRow = + (Function & Serializable) + tr -> (Long.valueOf((String) tr.get(fieldName))) >= failFrom; + fakeDatasetService.setShouldFailRow(shouldFailRow); + + SerializableFunction formatRecordOnFailureFunction = + input -> { + TableRow failedTableRow = new TableRow().set("testFailureFunctionField", "testValue"); + failedTableRow.set("originalElement", input); + return failedTableRow; + }; + + WriteResult result = + p.apply(Create.of(goodRows)) + .apply( + BigQueryIO.write() + .to("project-id:dataset-id.table") + .withMethod(method) + .withAvroFormatFunction( + (SerializableFunction, GenericRecord>) + input -> + new GenericRecordBuilder(avroSchema) + .set(fieldName, input.getElement()) + .build()) + .withSchema( + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName(fieldName).setType("INTEGER")))) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()) + .withPropagateSuccessfulStorageApiWrites(true) + .withTestServices(fakeBqServices) + .withFormatRecordOnFailureFunction(formatRecordOnFailureFunction) + .withoutValidation()); + + PCollection deadRows = + result + .getFailedStorageApiInserts() + .apply( + MapElements.into(TypeDescriptor.of(TableRow.class)) + .via(BigQueryStorageApiInsertError::getRow)); + PCollection successfulRows = result.getSuccessfulStorageApiInserts(); + + List expectedFailedRows = + goodRows.stream() + .filter(l -> l >= failFrom) + .map(formatRecordOnFailureFunction::apply) + .collect(Collectors.toList()); + PAssert.that(deadRows).containsInAnyOrder(expectedFailedRows); + PAssert.that(successfulRows) + .containsInAnyOrder( + Iterables.toArray( + Iterables.filter( + goodRows.stream() + .map( + l -> { + TableRow tableRow = new TableRow(); + tableRow.set(fieldName, String.valueOf(l)); + return tableRow; + }) + .collect(Collectors.toList()), + r -> !shouldFailRow.apply(r)), + TableRow.class)); + p.run(); + + assertThat( + fakeDatasetService.getAllRows("project-id", "dataset-id", "table"), + containsInAnyOrder( + Iterables.toArray( + Iterables.filter( + goodRows.stream() + .map( + l -> { + TableRow tableRow = new TableRow(); + tableRow.set(fieldName, String.valueOf(l)); + return tableRow; + }) + .collect(Collectors.toList()), + r -> !shouldFailRow.apply(r)), + TableRow.class))); + } + + @Test + public void testStorageApiErrorsWriteTableRows() throws Exception { assumeTrue(useStorageApi); final Method method = useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API; @@ -3132,6 +3438,22 @@ public void testStorageApiErrors() throws Exception { tr -> tr.containsKey("name") && tr.get("name").equals(failValue); fakeDatasetService.setShouldFailRow(shouldFailRow); + SerializableFunction formatRecordOnFailureFunction = + input -> { + TableRow failedTableRow = new TableRow().set("testFailureFunctionField", "testValue"); + if (input != null) { + Object name = input.get("name"); + if (name != null) { + failedTableRow.set("name", name); + } + Object number = input.get("number"); + if (number != null) { + failedTableRow.set("number", number); + } + } + return failedTableRow; + }; + WriteResult result = p.apply(Create.of(Iterables.concat(goodRows, badRows))) .apply( @@ -3143,6 +3465,7 @@ public void testStorageApiErrors() throws Exception { .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()) .withPropagateSuccessfulStorageApiWrites(true) .withTestServices(fakeBqServices) + .withFormatRecordOnFailureFunction(formatRecordOnFailureFunction) .withoutValidation()); PCollection deadRows = @@ -3153,9 +3476,14 @@ public void testStorageApiErrors() throws Exception { .via(BigQueryStorageApiInsertError::getRow)); PCollection successfulRows = result.getSuccessfulStorageApiInserts(); - PAssert.that(deadRows) - .containsInAnyOrder( - Iterables.concat(badRows, Iterables.filter(goodRows, shouldFailRow::apply))); + List expectedFailedRows = + badRows.stream().map(formatRecordOnFailureFunction::apply).collect(Collectors.toList()); + expectedFailedRows.addAll( + goodRows.stream() + .filter(shouldFailRow::apply) + .map(formatRecordOnFailureFunction::apply) + .collect(Collectors.toList())); + PAssert.that(deadRows).containsInAnyOrder(expectedFailedRows); PAssert.that(successfulRows) .containsInAnyOrder( Iterables.toArray( From b0f2683cda15c1b308bd583bca99be992fdee79b Mon Sep 17 00:00:00 2001 From: Kiruphasankaran Nataraj Date: Tue, 13 Aug 2024 20:01:46 +0530 Subject: [PATCH 71/78] GitHub issue #30257 Adds a static comparing method to the SerializableComparator interface --- .../transforms/SerializableComparator.java | 21 ++++++- .../SerializableComparatorTest.java | 63 +++++++++++++++++++ 2 files changed, 83 insertions(+), 1 deletion(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SerializableComparatorTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java index c66fbb7d7497b..16304633c993b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java @@ -19,10 +19,29 @@ import java.io.Serializable; import java.util.Comparator; +import java.util.Objects; +import java.util.function.Function; /** * A {@code Comparator} that is also {@code Serializable}. * * @param type of values being compared */ -public interface SerializableComparator extends Comparator, Serializable {} +public interface SerializableComparator extends Comparator, Serializable { + /** + * Analogous to {@link Comparator#comparing(Function)}, except that it takes in a {@link + * SerializableFunction} as the key extractor and returns a {@link SerializableComparator}. + * + * @param keyExtractor the function used to extract the {@link java.lang.Comparable} sort key + * @return A {@link SerializableComparator} that compares by an extracted key + * @param the type of element to be compared + * @param the type of the {@code Comparable} sort key + * @see Comparator#comparing(Function) + */ + static > SerializableComparator comparing( + SerializableFunction keyExtractor) { + Objects.requireNonNull(keyExtractor); + return (SerializableComparator) + (c1, c2) -> keyExtractor.apply(c1).compareTo(keyExtractor.apply(c2)); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SerializableComparatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SerializableComparatorTest.java new file mode 100644 index 0000000000000..09583ec44f28c --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SerializableComparatorTest.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.transforms; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.beam.sdk.util.SerializableUtils; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link SerializableComparator}. */ +@RunWith(JUnit4.class) +public class SerializableComparatorTest { + + /** + * Tests if the {@link SerializableComparator} returned by {@link + * SerializableComparator#comparing(SerializableFunction)} using {@link + * SerializableUtils#ensureSerializable(Serializable)}. + */ + @Test + public void testSerializable() { + SerializableFunction fn = Integer::parseInt; + + SerializableComparator cmp = SerializableComparator.comparing(fn); + SerializableUtils.ensureSerializable(cmp); + } + + /** + * Tests if {@link SerializableComparator#comparing(Function)} throws a {@link + * java.lang.NullPointerException} if null is passed to it. + */ + @Test(expected = NullPointerException.class) + public void testIfNPEThrownForNullFunction() { + SerializableComparator.comparing(null); + } + + /** Tests the basic comparison function of the {@link SerializableComparator} returned. */ + @Test + public void testBasicComparison() { + SerializableFunction fn = Integer::parseInt; + SerializableComparator cmp = SerializableComparator.comparing(fn); + + Assert.assertTrue(cmp.compare("1", "10") < 0); + Assert.assertTrue(cmp.compare("9", "6") > 0); + } +} From b2d26b6b5f376db079679d620a812af25c4a90f8 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 13 Aug 2024 17:43:30 +0200 Subject: [PATCH 72/78] Fix upload_graph on v2 (#32165) * Fix upload_graph on v2 * compliation nits * compliation nits * remove streaming test change, update CHANGES * mutability fix * Test fix * Remove upload_graph from it --- CHANGES.md | 10 ++++++++++ .../beam/runners/dataflow/DataflowRunner.java | 12 +++++++++++- .../beam/runners/dataflow/DataflowRunnerTest.java | 13 +++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 950abc694488e..fce3aa26a72b7 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -83,6 +83,7 @@ ## Bugfixes * Fixed incorrect service account impersonation flow for Python pipelines using BigQuery IOs ([#32030](https://github.com/apache/beam/issues/32030)). +* Auto-disable broken and meaningless `upload_graph` feature when using Dataflow Runner V2 ([#32159](https://github.com/apache/beam/issues/32159)). ## Security Fixes * Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). @@ -116,6 +117,10 @@ * [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) * Fixed a logging issue where Python worker dependency installation logs sometimes were not emitted in a timely manner ([#31977](https://github.com/apache/beam/pull/31977)) +## Known Issues + +* Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). + # [2.57.0] - 2024-06-26 ## Highlights @@ -167,6 +172,10 @@ jackson-2.15 has known breaking changes. An important one is it imposed a buffer limit for parser. If your custom PTransform/DoFn are affected, refer to [#31580](https://github.com/apache/beam/pull/31580) for mitigation. +## Known Issues + +* Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). + # [2.56.0] - 2024-05-01 ## Highlights @@ -202,6 +211,7 @@ * The beam interactive runner does not correctly run on flink ([#31168](https://github.com/apache/beam/issues/31168)). * When using the Flink runner from Python, 1.17 is not supported and 1.12/13 do not work correctly. Support for 1.17 will be added in 2.57.0, and the ability to choose 1.12/13 will be cleaned up and fully removed in 2.57.0 as well ([#31168](https://github.com/apache/beam/issues/31168)). +* Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). # [2.55.1] - 2024-04-08 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 708c63413268a..abe7d0d364d3f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -1385,7 +1385,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { byte[] jobGraphBytes = DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8); int jobGraphByteSize = jobGraphBytes.length; if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES - && !hasExperiment(options, "upload_graph")) { + && !hasExperiment(options, "upload_graph") + && !useUnifiedWorker(options)) { List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); options.setExperiments( ImmutableList.builder().addAll(experiments).add("upload_graph").build()); @@ -1396,6 +1397,15 @@ public DataflowPipelineJob run(Pipeline pipeline) { CREATE_JOB_REQUEST_LIMIT_BYTES); } + if (hasExperiment(options, "upload_graph") && useUnifiedWorker(options)) { + ArrayList experiments = new ArrayList<>(options.getExperiments()); + while (experiments.remove("upload_graph")) {} + options.setExperiments(experiments); + LOG.warn( + "The upload_graph experiment was specified, but it does not apply " + + "to runner v2 jobs. Option has been automatically removed."); + } + // Upload the job to GCS and remove the graph object from the API call. The graph // will be downloaded from GCS by the service. if (hasExperiment(options, "upload_graph")) { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index cf1066e41d25e..37c20c61ad8e5 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -838,6 +838,19 @@ public void testUploadGraph() throws IOException { .startsWith("gs://valid-bucket/temp/staging/dataflow_graph")); } + @Test + public void testUploadGraphV2IsNoOp() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setExperiments(Arrays.asList("upload_graph", "use_runner_v2")); + Pipeline p = buildDataflowPipeline(options); + p.run(); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); + assertValidJob(jobCaptor.getValue()); + assertNull(jobCaptor.getValue().getStepsLocation()); + } + /** Test for automatically using upload_graph when the job graph is too large (>10MB). */ @Test public void testUploadGraphWithAutoUpload() throws IOException { From ab81e1fc5e9f10a955bb56ca21675004af4ba180 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 13 Aug 2024 14:51:13 -0400 Subject: [PATCH 73/78] Added a data corruption known issue to CHANGES.md and release blogs. (#32166) * Added a data corruption known issue to CHANGES.md and release blogs. * Update issue link * Update issue link in release blogs --- CHANGES.md | 7 +++++++ website/www/site/content/en/blog/beam-2.53.0.md | 1 + website/www/site/content/en/blog/beam-2.54.0.md | 1 + website/www/site/content/en/blog/beam-2.55.0.md | 3 +++ website/www/site/content/en/blog/beam-2.56.0.md | 6 ++++++ website/www/site/content/en/blog/beam-2.57.0.md | 4 ++++ website/www/site/content/en/blog/beam-2.58.0.md | 6 ++++++ 7 files changed, 28 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index fce3aa26a72b7..cf2478e02358f 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -84,6 +84,7 @@ * Fixed incorrect service account impersonation flow for Python pipelines using BigQuery IOs ([#32030](https://github.com/apache/beam/issues/32030)). * Auto-disable broken and meaningless `upload_graph` feature when using Dataflow Runner V2 ([#32159](https://github.com/apache/beam/issues/32159)). +* (Python) Upgraded google-cloud-storage to version 2.18.2 to fix a data corruption issue ([#32135](https://github.com/apache/beam/pull/32135)). ## Security Fixes * Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). @@ -120,6 +121,7 @@ ## Known Issues * Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.57.0] - 2024-06-26 @@ -175,6 +177,7 @@ ## Known Issues * Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.56.0] - 2024-05-01 @@ -212,6 +215,7 @@ * The beam interactive runner does not correctly run on flink ([#31168](https://github.com/apache/beam/issues/31168)). * When using the Flink runner from Python, 1.17 is not supported and 1.12/13 do not work correctly. Support for 1.17 will be added in 2.57.0, and the ability to choose 1.12/13 will be cleaned up and fully removed in 2.57.0 as well ([#31168](https://github.com/apache/beam/issues/31168)). * Large Dataflow graphs using runner v2, or pipelines explicitly enabling the `upload_graph` experiment, will fail at construction time ([#32159](https://github.com/apache/beam/issues/32159)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.55.1] - 2024-04-08 @@ -266,6 +270,7 @@ * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). * WriteToJson broken in languages other than Java (X-lang) ([#30776](https://github.com/apache/beam/issues/30776)). * Python pipelines might occasionally become stuck due to a regression in grpcio ([#30867](https://github.com/apache/beam/issues/30867)). The issue manifests frequently with Bigtable IO connector, but might also affect other GCP connectors. Fixed in 2.56.0. +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.54.0] - 2024-02-14 @@ -307,6 +312,7 @@ * Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. ([#30360](https://github.com/apache/beam/issues/30360)). * Python pipelines that run with 2.53.0-2.54.0 SDKs and perform file operations on GCS might be affected by excess HTTP requests. This could lead to a performance regression or a permission issue. ([#28398](https://github.com/apache/beam/issues/28398)) * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.53.0] - 2024-01-04 @@ -351,6 +357,7 @@ * Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. ([#30360](https://github.com/apache/beam/issues/30360)). * Python pipelines that run with 2.53.0-2.54.0 SDKs and perform file operations on GCS might be affected by excess HTTP requests. This could lead to a performance regression or a permission issue. ([#28398](https://github.com/apache/beam/issues/28398)) * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. # [2.52.0] - 2023-11-17 diff --git a/website/www/site/content/en/blog/beam-2.53.0.md b/website/www/site/content/en/blog/beam-2.53.0.md index 9a15e86ef3dc8..39f851d1563b0 100644 --- a/website/www/site/content/en/blog/beam-2.53.0.md +++ b/website/www/site/content/en/blog/beam-2.53.0.md @@ -68,6 +68,7 @@ For more information on changes in 2.53.0, check out the [detailed release notes * Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. ([#30360](https://github.com/apache/beam/issues/30360)). * Python pipelines that run with 2.53.0-2.54.0 SDKs and perform file operations on GCS might be affected by excess HTTP requests. This could lead to a performance regression or a permission issue. ([#28398](https://github.com/apache/beam/issues/28398)) * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md diff --git a/website/www/site/content/en/blog/beam-2.54.0.md b/website/www/site/content/en/blog/beam-2.54.0.md index a3d649ec6f3f3..ecef90fe84603 100644 --- a/website/www/site/content/en/blog/beam-2.54.0.md +++ b/website/www/site/content/en/blog/beam-2.54.0.md @@ -64,6 +64,7 @@ For more information on changes in 2.54.0, check out the [detailed release notes * Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. ([#30360](https://github.com/apache/beam/issues/30360)). * Python pipelines that run with 2.53.0-2.54.0 SDKs and perform file operations on GCS might be affected by excess HTTP requests. This could lead to a performance regression or a permission issue. ([#28398](https://github.com/apache/beam/issues/28398)) * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md diff --git a/website/www/site/content/en/blog/beam-2.55.0.md b/website/www/site/content/en/blog/beam-2.55.0.md index 6314dfa928279..2ef05fd781b49 100644 --- a/website/www/site/content/en/blog/beam-2.55.0.md +++ b/website/www/site/content/en/blog/beam-2.55.0.md @@ -74,6 +74,9 @@ For more information on changes in 2.55.0, check out the [detailed release notes ## Known Issues * In Python pipelines, when shutting down inactive bundle processors, shutdown logic can overaggressively hold the lock, blocking acceptance of new work. Symptoms of this issue include slowness or stuckness in long-running jobs. Fixed in 2.56.0 ([#30679](https://github.com/apache/beam/pull/30679)). +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. + +For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md ## List of Contributors diff --git a/website/www/site/content/en/blog/beam-2.56.0.md b/website/www/site/content/en/blog/beam-2.56.0.md index 8107b22920f19..3a441536d7027 100644 --- a/website/www/site/content/en/blog/beam-2.56.0.md +++ b/website/www/site/content/en/blog/beam-2.56.0.md @@ -54,6 +54,12 @@ For more information on changes in 2.56.0, check out the [detailed release notes * Fixed locking issue when shutting down inactive bundle processors. Symptoms of this issue include slowness or stuckness in long-running jobs (Python) ([#30679](https://github.com/apache/beam/pull/30679)). * Fixed logging issue that caused silecing the pip output when installing of dependencies provided in `--requirements_file` (Python). +## Known Issues + +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. + +For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md + ## List of Contributors According to git shortlog, the following people contributed to the 2.56.0 release. Thank you to all contributors! diff --git a/website/www/site/content/en/blog/beam-2.57.0.md b/website/www/site/content/en/blog/beam-2.57.0.md index 62f2896bba0c0..b583b4ee3c516 100644 --- a/website/www/site/content/en/blog/beam-2.57.0.md +++ b/website/www/site/content/en/blog/beam-2.57.0.md @@ -76,6 +76,10 @@ For more information on changes in 2.57.0, check out the [detailed release notes jackson-2.15 has known breaking changes. An important one is it imposed a buffer limit for parser. If your custom PTransform/DoFn are affected, refer to [#31580](https://github.com/apache/beam/pull/31580) for mitigation. +## Known Issues + +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. + For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md ## List of Contributors diff --git a/website/www/site/content/en/blog/beam-2.58.0.md b/website/www/site/content/en/blog/beam-2.58.0.md index 603403cd7fdbc..c5d858091fff8 100644 --- a/website/www/site/content/en/blog/beam-2.58.0.md +++ b/website/www/site/content/en/blog/beam-2.58.0.md @@ -49,6 +49,12 @@ For more information about changes in 2.58.0, check out the [detailed release no * [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) +## Known Issues + +* Python pipelines that run with 2.53.0-2.58.0 SDKs and read data from GCS might be affected by a data corruption issue ([#32169](https://github.com/apache/beam/issues/32169)). The issue will be fixed in 2.59.0 ([#32135](https://github.com/apache/beam/pull/32135)). To work around this, update the google-cloud-storage package to version 2.18.2 or newer. + +For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md + ## List of Contributors According to git shortlog, the following people contributed to the 2.58.0 release. Thank you to all contributors! From 8ff7f0d75e45aa31bcc56d2bcd38ef49125295aa Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Aug 2024 23:12:52 -0700 Subject: [PATCH 74/78] Bump github.com/docker/docker in /sdks (#32176) Bumps [github.com/docker/docker](https://github.com/docker/docker) from 27.1.1+incompatible to 27.1.2+incompatible. - [Release notes](https://github.com/docker/docker/releases) - [Commits](https://github.com/docker/docker/compare/v27.1.1...v27.1.2) --- updated-dependencies: - dependency-name: github.com/docker/docker dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 3 ++- sdks/go.sum | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 7c3ade37490d1..53aa6f23bfaa2 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -93,6 +93,7 @@ require ( github.com/minio/highwayhash v1.0.3 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/user v0.1.0 // indirect + github.com/moby/sys/userns v0.1.0 // indirect github.com/nats-io/jwt/v2 v2.5.8 // indirect github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect @@ -142,7 +143,7 @@ require ( github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b // indirect github.com/containerd/containerd v1.7.18 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/docker/docker v27.1.1+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v27.1.2+incompatible // but required to resolve issue docker has with go1.20 github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/go-control-plane v0.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 5b9bfaef8a48d..f49007e6b7d9a 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -779,8 +779,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5QvfrDyIgxBk= github.com/distribution/reference v0.6.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v27.1.1+incompatible h1:hO/M4MtV36kzKldqnA37IWhebRA+LnqqcqDja6kVaKY= -github.com/docker/docker v27.1.1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v27.1.2+incompatible h1:AhGzR1xaQIy53qCkxARaFluI00WPGtXn0AJuoQsVYTY= +github.com/docker/docker v27.1.2+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -1065,6 +1065,8 @@ github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5 github.com/moby/sys/sequential v0.5.0/go.mod h1:tH2cOOs5V9MlPiXcQzRC+eEyab644PWKGRYaaV5ZZlo= github.com/moby/sys/user v0.1.0 h1:WmZ93f5Ux6het5iituh9x2zAG7NFY9Aqi49jjE1PaQg= github.com/moby/sys/user v0.1.0/go.mod h1:fKJhFOnsCN6xZ5gSfbM6zaHGgDJMrqt9/reuj4T7MmU= +github.com/moby/sys/userns v0.1.0 h1:tVLXkFOxVu9A64/yh59slHVv9ahO9UIev4JZusOLG/g= +github.com/moby/sys/userns v0.1.0/go.mod h1:IHUYgu/kao6N8YZlp9Cf444ySSvCmDlmzUcYfDHOl28= github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= github.com/montanaflynn/stats v0.7.1 h1:etflOAAHORrCC44V+aR6Ftzort912ZU+YLiSTuV8eaE= From c23e60383bce1628245636d4fc557b70843de342 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 14 Aug 2024 05:34:26 -0400 Subject: [PATCH 75/78] Add Lineage metrics to Python BigQueryIO (#32116) * Add Lineage metrics to Python BigQueryIO * Introduce metric.Lineage StringSet wrapper Reflect Java SDK #32090 * Direct Read * Export Read * ReadAllFromBigQuery * FILE_LOAD Write * fix lint; add tests * Consistent metrics name * Update sdks/python/apache_beam/metrics/metric.py Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- sdks/python/apache_beam/io/gcp/bigquery.py | 11 +++ .../apache_beam/io/gcp/bigquery_file_loads.py | 12 +++ .../io/gcp/bigquery_file_loads_test.py | 10 +++ .../io/gcp/bigquery_read_internal.py | 7 ++ .../io/gcp/bigquery_schema_tools_test.py | 18 ++-- .../apache_beam/io/gcp/bigquery_test.py | 43 ++++++++++ sdks/python/apache_beam/metrics/metric.py | 83 ++++++++++++++++++- .../python/apache_beam/metrics/metric_test.py | 22 +++++ 8 files changed, 192 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index e1c509d0e490c..b897df2d32ab3 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -400,6 +400,7 @@ def chain_after(result): from apache_beam.io.iobase import SourceBundle from apache_beam.io.textio import _TextSource as TextSource from apache_beam.metrics import Metrics +from apache_beam.metrics.metric import Lineage from apache_beam.options import value_provider as vp from apache_beam.options.pipeline_options import DebugOptions from apache_beam.options.pipeline_options import GoogleCloudOptions @@ -809,6 +810,11 @@ def split(self, desired_bundle_size, start_position=None, stop_position=None): self.table_reference.get(), project=self._get_project()) elif not self.table_reference.projectId: self.table_reference.projectId = self._get_project() + Lineage.sources().add( + 'bigquery', + self.table_reference.projectId, + self.table_reference.datasetId, + self.table_reference.tableId) schema, metadata_list = self._export_files(bq) self.export_result = _BigQueryExportResult( @@ -1157,6 +1163,11 @@ def split(self, desired_bundle_size, start_position=None, stop_position=None): self.table_reference.projectId, self.table_reference.datasetId, self.table_reference.tableId) + Lineage.sources().add( + "bigquery", + self.table_reference.projectId, + self.table_reference.datasetId, + self.table_reference.tableId) if self.use_native_datetime: requested_session.data_format = bq_storage.types.DataFormat.ARROW 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 3203c21a8e64a..a7311ad6d0637 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py @@ -40,6 +40,7 @@ from apache_beam.io import filesystems as fs from apache_beam.io.gcp import bigquery_tools from apache_beam.io.gcp.bigquery_io_metadata import create_bigquery_io_metadata +from apache_beam.metrics.metric import Lineage from apache_beam.options import value_provider as vp from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.transforms import trigger @@ -564,6 +565,11 @@ def process_one(self, element, job_name_prefix): write_disposition = self.write_disposition wait_for_job = True self._observed_tables.add(copy_to_reference.tableId) + Lineage.sinks().add( + 'bigquery', + copy_to_reference.projectId, + copy_to_reference.datasetId, + copy_to_reference.tableId) else: wait_for_job = False write_disposition = 'WRITE_APPEND' @@ -735,6 +741,12 @@ def process( yield pvalue.TaggedOutput( TriggerLoadJobs.TEMP_TABLES, bigquery_tools.get_hashable_destination(table_reference)) + else: + Lineage.sinks().add( + 'bigquery', + table_reference.projectId, + table_reference.datasetId, + table_reference.tableId) _LOGGER.info( 'Triggering job %s to load data to BigQuery table %s.' 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 f27c7899f9f38..e4c0e34d9c1f7 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 @@ -42,6 +42,7 @@ from apache_beam.io.gcp.internal.clients import bigquery as bigquery_api from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultMatcher from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultStreamingMatcher +from apache_beam.metrics.metric import Lineage from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import StandardOptions from apache_beam.runners.dataflow.test_dataflow_runner import TestDataflowRunner @@ -510,6 +511,9 @@ def test_load_job_id_used(self): | "GetJobs" >> beam.Map(lambda x: x[1]) assert_that(jobs, equal_to([job_reference]), label='CheckJobProjectIds') + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SINK), + set(["bigquery:project1.dataset1.table1"])) def test_load_job_id_use_for_copy_job(self): destination = 'project1:dataset1.table1' @@ -563,6 +567,9 @@ def test_load_job_id_use_for_copy_job(self): job_reference ]), label='CheckCopyJobProjectIds') + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SINK), + set(["bigquery:project1.dataset1.table1"])) @mock.patch('time.sleep') def test_wait_for_load_job_completion(self, sleep_mock): @@ -725,6 +732,9 @@ def test_multiple_partition_files(self): copy_jobs | "CountCopyJobs" >> combiners.Count.Globally(), equal_to([6]), label='CheckCopyJobCount') + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SINK), + set(["bigquery:project1.dataset1.table1"])) @parameterized.expand([ param(write_disposition=BigQueryDisposition.WRITE_TRUNCATE), diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py index f3881ed261ae3..f038b48e04d53 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py @@ -44,6 +44,7 @@ from apache_beam.io.gcp.bigquery_io_metadata import create_bigquery_io_metadata from apache_beam.io.iobase import BoundedSource from apache_beam.io.textio import _TextSource +from apache_beam.metrics.metric import Lineage from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.value_provider import ValueProvider @@ -261,6 +262,12 @@ def process(self, for metadata in metadata_list: yield self._create_source(metadata.path, schema) + Lineage.sources().add( + 'bigquery', + table_reference.projectId, + table_reference.datasetId, + table_reference.tableId) + if element.query is not None: self.bq._delete_table( table_reference.projectId, diff --git a/sdks/python/apache_beam/io/gcp/bigquery_schema_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_schema_tools_test.py index 72697e29c4d56..7ae49dff205d7 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_schema_tools_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_schema_tools_test.py @@ -136,12 +136,11 @@ def test_bad_schema_public_api_export(self, get_table): with self.assertRaisesRegex(ValueError, "Encountered an unsupported type: 'DOUBLE'"): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table="dataset.sample_table", method="EXPORT", project="project", output_type='BEAM_ROW') - pipeline @mock.patch.object(BigQueryWrapper, 'get_table') def test_bad_schema_public_api_direct_read(self, get_table): @@ -159,21 +158,19 @@ def test_bad_schema_public_api_direct_read(self, get_table): with self.assertRaisesRegex(ValueError, "Encountered an unsupported type: 'DOUBLE'"): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table="dataset.sample_table", method="DIRECT_READ", project="project", output_type='BEAM_ROW') - pipeline def test_unsupported_value_provider(self): with self.assertRaisesRegex(TypeError, 'ReadFromBigQuery: table must be of type string' '; got ValueProvider instead'): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table=value_provider.ValueProvider(), output_type='BEAM_ROW') - pipeline def test_unsupported_callable(self): def filterTable(table): @@ -185,9 +182,8 @@ def filterTable(table): 'ReadFromBigQuery: table must be of type string' '; got a callable instead'): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table=res, output_type='BEAM_ROW') - pipeline def test_unsupported_query_export(self): with self.assertRaisesRegex( @@ -195,12 +191,11 @@ def test_unsupported_query_export(self): "Both a query and an output type of 'BEAM_ROW' were specified. " "'BEAM_ROW' is not currently supported with queries."): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table="project:dataset.sample_table", method="EXPORT", query='SELECT name FROM dataset.sample_table', output_type='BEAM_ROW') - pipeline def test_unsupported_query_direct_read(self): with self.assertRaisesRegex( @@ -208,12 +203,11 @@ def test_unsupported_query_direct_read(self): "Both a query and an output type of 'BEAM_ROW' were specified. " "'BEAM_ROW' is not currently supported with queries."): p = apache_beam.Pipeline() - pipeline = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( + _ = p | apache_beam.io.gcp.bigquery.ReadFromBigQuery( table="project:dataset.sample_table", method="DIRECT_READ", query='SELECT name FROM dataset.sample_table', output_type='BEAM_ROW') - pipeline if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index e53204a5ebc6f..c263b636b57ad 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -50,6 +50,7 @@ from apache_beam.io.gcp.bigquery import TableRowJsonCoder from apache_beam.io.gcp.bigquery import WriteToBigQuery from apache_beam.io.gcp.bigquery import _StreamToBigQuery +from apache_beam.io.gcp.bigquery_read_internal import _BigQueryReadSplit from apache_beam.io.gcp.bigquery_read_internal import _JsonToDictCoder from apache_beam.io.gcp.bigquery_read_internal import bigquery_export_destination_uri from apache_beam.io.gcp.bigquery_tools import JSON_COMPLIANCE_ERROR @@ -61,6 +62,7 @@ from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultMatcher from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultStreamingMatcher from apache_beam.io.gcp.tests.bigquery_matcher import BigQueryTableMatcher +from apache_beam.metrics.metric import Lineage from apache_beam.options import value_provider from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import StandardOptions @@ -85,9 +87,11 @@ from apitools.base.py.exceptions import HttpError from apitools.base.py.exceptions import HttpForbiddenError from google.cloud import bigquery as gcp_bigquery + from google.cloud import bigquery_storage_v1 as bq_storage from google.api_core import exceptions except ImportError: gcp_bigquery = None + bq_storage = None HttpError = None HttpForbiddenError = None exceptions = None @@ -460,6 +464,8 @@ def test_create_temp_dataset_exception(self, exception_type, error_message): self.assertIn(error_message, exc.exception.args[0]) @parameterized.expand([ + # read without exception + param(responses=[], expected_retries=0), # first attempt returns a Http 500 blank error and retries # second attempt returns a Http 408 blank error and retries, # third attempt passes @@ -540,6 +546,9 @@ def store_callback(unused_request): # metadata (numBytes), and once to retrieve the table's schema # Any additional calls are retries self.assertEqual(expected_retries, mock_get_table.call_count - 2) + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SOURCE), + set(["bigquery:project.dataset.table"])) @parameterized.expand([ # first attempt returns a Http 429 with transient reason and retries @@ -719,6 +728,40 @@ def test_read_export_exception(self, exception_type, error_message): mock_query_job.assert_called() self.assertIn(error_message, exc.exception.args[0]) + def test_read_direct_lineage(self): + with mock.patch.object(bigquery_tools.BigQueryWrapper, + '_bigquery_client'),\ + mock.patch.object(bq_storage.BigQueryReadClient, + 'create_read_session'),\ + beam.Pipeline() as p: + + _ = p | ReadFromBigQuery( + method=ReadFromBigQuery.Method.DIRECT_READ, + table='project:dataset.table') + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SOURCE), + set(["bigquery:project.dataset.table"])) + + def test_read_all_lineage(self): + with mock.patch.object(_BigQueryReadSplit, '_export_files') as export, \ + beam.Pipeline() as p: + + export.return_value = (None, []) + + _ = ( + p + | beam.Create([ + beam.io.ReadFromBigQueryRequest(table='project1:dataset1.table1'), + beam.io.ReadFromBigQueryRequest(table='project2:dataset2.table2') + ]) + | beam.io.ReadAllFromBigQuery(gcs_location='gs://bucket/tmp')) + self.assertSetEqual( + Lineage.query(p.result.metrics(), Lineage.SOURCE), + set([ + 'bigquery:project1.dataset1.table1', + 'bigquery:project2.dataset2.table2' + ])) + @unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') class TestBigQuerySink(unittest.TestCase): diff --git a/sdks/python/apache_beam/metrics/metric.py b/sdks/python/apache_beam/metrics/metric.py index 77cafb8bd64b7..6b8e4754a79ce 100644 --- a/sdks/python/apache_beam/metrics/metric.py +++ b/sdks/python/apache_beam/metrics/metric.py @@ -28,6 +28,7 @@ # mypy: disallow-untyped-defs import logging +import re from typing import TYPE_CHECKING from typing import Dict from typing import FrozenSet @@ -39,6 +40,7 @@ from typing import Union from apache_beam.metrics import cells +from apache_beam.metrics.execution import MetricResult from apache_beam.metrics.execution import MetricUpdater from apache_beam.metrics.metricbase import Counter from apache_beam.metrics.metricbase import Distribution @@ -50,7 +52,7 @@ from apache_beam.metrics.execution import MetricKey from apache_beam.metrics.metricbase import Metric -__all__ = ['Metrics', 'MetricsFilter'] +__all__ = ['Metrics', 'MetricsFilter', 'Lineage'] _LOGGER = logging.getLogger(__name__) @@ -223,7 +225,7 @@ def matches( def query( self, filter: Optional['MetricsFilter'] = None - ) -> Dict[str, List['MetricResults']]: + ) -> Dict[str, List['MetricResult']]: """Queries the runner for existing user metrics that match the filter. It should return a dictionary, with lists of each kind of metric, and @@ -305,3 +307,80 @@ def with_steps(self, steps: Iterable[str]) -> 'MetricsFilter': self._steps.update(steps) return self + + +class Lineage: + """Standard collection of metrics used to record source and sinks information + for lineage tracking.""" + + LINEAGE_NAMESPACE = "lineage" + SOURCE = "sources" + SINK = "sinks" + + _METRICS = { + SOURCE: Metrics.string_set(LINEAGE_NAMESPACE, SOURCE), + SINK: Metrics.string_set(LINEAGE_NAMESPACE, SINK) + } + + def __init__(self, label: str) -> None: + """Create a Lineage with valid label (:data:`~Lineage.SOURCE` or + :data:`~Lineage.SINK`) + """ + self.metric = Lineage._METRICS[label] + + @classmethod + def sources(cls) -> 'Lineage': + return cls(Lineage.SOURCE) + + @classmethod + def sinks(cls) -> 'Lineage': + return cls(Lineage.SINK) + + _RESERVED_CHARS = re.compile(r'[:\s.]') + + @staticmethod + def wrap_segment(segment: str) -> str: + """Wrap segment to valid segment name. + + Specifically, If there are reserved chars (colon, whitespace, dot), escape + with backtick. If the segment is already wrapped, return the original. + """ + if segment.startswith("`") and segment.endswith("`"): return segment + if Lineage._RESERVED_CHARS.search(segment): + return "`" + segment + "`" + return segment + + @staticmethod + def get_fq_name( + system: str, *segments: str, route: Optional[str] = None) -> str: + """Assemble fully qualified name + (`FQN `_). + Format: + + - `system:segment1.segment2` + - `system:routine:segment1.segment2` + - `system:`segment1.with.dots:clons`.segment2` + + This helper method is for internal and testing usage only. + """ + segs = '.'.join(map(Lineage.wrap_segment, segments)) + if route: + return ':'.join((system, route, segs)) + return ':'.join((system, segs)) + + def add( + self, system: str, *segments: str, route: Optional[str] = None) -> None: + self.metric.add(self.get_fq_name(system, *segments, route=route)) + + @staticmethod + def query(results: MetricResults, label: str) -> Set[str]: + if not label in Lineage._METRICS: + raise ValueError("Label {} does not exist for Lineage", label) + response = results.query( + MetricsFilter().with_namespace(Lineage.LINEAGE_NAMESPACE).with_name( + label))[MetricResults.STRINGSETS] + result = set() + for metric in response: + result.update(metric.committed) + result.update(metric.attempted) + return result diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py index e3701228feecd..3a8da021101e5 100644 --- a/sdks/python/apache_beam/metrics/metric_test.py +++ b/sdks/python/apache_beam/metrics/metric_test.py @@ -28,6 +28,7 @@ from apache_beam.metrics.execution import MetricKey from apache_beam.metrics.execution import MetricsContainer from apache_beam.metrics.execution import MetricsEnvironment +from apache_beam.metrics.metric import Lineage from apache_beam.metrics.metric import MetricResults from apache_beam.metrics.metric import Metrics from apache_beam.metrics.metric import MetricsFilter @@ -248,5 +249,26 @@ def test_create_counter_distribution(self): sampler.stop() +class LineageTest(unittest.TestCase): + def test_fq_name(self): + test_cases = { + "apache-beam": "apache-beam", + "`apache-beam`": "`apache-beam`", + "apache.beam": "`apache.beam`", + "apache:beam": "`apache:beam`", + "apache beam": "`apache beam`", + "`apache beam`": "`apache beam`", + "apache\tbeam": "`apache\tbeam`", + "apache\nbeam": "`apache\nbeam`" + } + for k, v in test_cases.items(): + self.assertEqual("apache:" + v, Lineage.get_fq_name("apache", k)) + self.assertEqual( + "apache:beam:" + v, Lineage.get_fq_name("apache", k, route="beam")) + self.assertEqual( + "apache:beam:" + v + '.' + v, + Lineage.get_fq_name("apache", k, k, route="beam")) + + if __name__ == '__main__': unittest.main() From 8fbad48568833d60a5244d00f4b4b943d82bac0b Mon Sep 17 00:00:00 2001 From: scwhittle Date: Wed, 14 Aug 2024 15:26:12 +0200 Subject: [PATCH 76/78] Change FnApiDoFnRunner to skip trySplit checkpoint requests if not draining and nothing has yet been claimed by the tracker. (#32044) --- .../beam/fn/harness/FnApiDoFnRunner.java | 57 ++- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 465 ++++++++++++++++-- 2 files changed, 485 insertions(+), 37 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index f85622ab89fee..c39722c90d89e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -118,6 +119,7 @@ import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Durations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -327,6 +329,11 @@ static class Factory currentTracker; + /** + * If non-null, set to true after currentTracker has had a tryClaim issued on it. Used to ignore + * checkpoint split requests if no progress was made. + */ + private @Nullable AtomicBoolean currentTrackerClaimed; /** * Only valid during {@link #processTimer} and {@link #processOnWindowExpiration}, null otherwise. @@ -877,12 +884,18 @@ private void processElementForSplitRestriction( currentElement = elem.withValue(elem.getValue().getKey()); currentRestriction = elem.getValue().getValue().getKey(); currentWatermarkEstimatorState = elem.getValue().getValue().getValue(); + currentTrackerClaimed = new AtomicBoolean(false); currentTracker = RestrictionTrackers.observe( doFnInvoker.invokeNewTracker(processContext), new ClaimObserver() { + private final AtomicBoolean claimed = + Preconditions.checkNotNull(currentTrackerClaimed); + @Override - public void onClaimed(PositionT position) {} + public void onClaimed(PositionT position) { + claimed.lazySet(true); + } @Override public void onClaimFailed(PositionT position) {} @@ -894,6 +907,7 @@ public void onClaimFailed(PositionT position) {} currentRestriction = null; currentWatermarkEstimatorState = null; currentTracker = null; + currentTrackerClaimed = null; } this.stateAccessor.finalizeState(); @@ -909,12 +923,18 @@ private void processElementForWindowObservingSplitRestriction( (Iterator) elem.getWindows().iterator(); while (windowIterator.hasNext()) { currentWindow = windowIterator.next(); + currentTrackerClaimed = new AtomicBoolean(false); currentTracker = RestrictionTrackers.observe( doFnInvoker.invokeNewTracker(processContext), new ClaimObserver() { + private final AtomicBoolean claimed = + Preconditions.checkNotNull(currentTrackerClaimed); + @Override - public void onClaimed(PositionT position) {} + public void onClaimed(PositionT position) { + claimed.lazySet(true); + } @Override public void onClaimFailed(PositionT position) {} @@ -927,6 +947,7 @@ public void onClaimFailed(PositionT position) {} currentWatermarkEstimatorState = null; currentWindow = null; currentTracker = null; + currentTrackerClaimed = null; } this.stateAccessor.finalizeState(); @@ -937,6 +958,8 @@ private void processElementForTruncateRestriction( currentElement = elem.withValue(elem.getValue().getKey().getKey()); currentRestriction = elem.getValue().getKey().getValue().getKey(); currentWatermarkEstimatorState = elem.getValue().getKey().getValue().getValue(); + // For truncation, we don't set currentTrackerClaimed so that we enable checkpointing even if no + // progress is made. currentTracker = RestrictionTrackers.observe( doFnInvoker.invokeNewTracker(processContext), @@ -989,6 +1012,8 @@ private void processElementForWindowObservingTruncateRestriction( currentRestriction = elem.getValue().getKey().getValue().getKey(); currentWatermarkEstimatorState = elem.getValue().getKey().getValue().getValue(); currentWindow = currentWindows.get(windowCurrentIndex); + // We leave currentTrackerClaimed unset as we want to split regardless of if tryClaim is + // called. currentTracker = RestrictionTrackers.observe( doFnInvoker.invokeNewTracker(processContext), @@ -1081,12 +1106,18 @@ private void processElementForWindowObservingSizedElementAndRestriction( currentRestriction = elem.getValue().getKey().getValue().getKey(); currentWatermarkEstimatorState = elem.getValue().getKey().getValue().getValue(); currentWindow = currentWindows.get(windowCurrentIndex); + currentTrackerClaimed = new AtomicBoolean(false); currentTracker = RestrictionTrackers.observe( doFnInvoker.invokeNewTracker(processContext), new ClaimObserver() { + private final AtomicBoolean claimed = + Preconditions.checkNotNull(currentTrackerClaimed); + @Override - public void onClaimed(PositionT position) {} + public void onClaimed(PositionT position) { + claimed.lazySet(true); + } @Override public void onClaimFailed(PositionT position) {} @@ -1107,7 +1138,7 @@ public void onClaimFailed(PositionT position) {} // Attempt to checkpoint the current restriction. HandlesSplits.SplitResult splitResult = - trySplitForElementAndRestriction(0, continuation.resumeDelay()); + trySplitForElementAndRestriction(0, continuation.resumeDelay(), false); /** * After the user has chosen to resume processing later, either the restriction is already @@ -1132,7 +1163,7 @@ private abstract class SplittableFnDataReceiver implements HandlesSplits, FnDataReceiver { @Override public HandlesSplits.SplitResult trySplit(double fractionOfRemainder) { - return trySplitForElementAndRestriction(fractionOfRemainder, Duration.ZERO); + return trySplitForElementAndRestriction(fractionOfRemainder, Duration.ZERO, true); } @Override @@ -1278,6 +1309,13 @@ private HandlesSplits.SplitResult trySplitForWindowObservingTruncateRestriction( if (currentWindow == null) { return null; } + // We are requesting a checkpoint but have not yet progressed on the restriction, skip + // request. + if (fractionOfRemainder == 0 + && currentTrackerClaimed != null + && !currentTrackerClaimed.get()) { + return null; + } SplitResultsWithStopIndex splitResult = computeSplitForProcessOrTruncate( @@ -1620,7 +1658,7 @@ static HandlesSplits.SplitResult constructSplitResult } private HandlesSplits.SplitResult trySplitForElementAndRestriction( - double fractionOfRemainder, Duration resumeDelay) { + double fractionOfRemainder, Duration resumeDelay, boolean requireClaimForCheckpoint) { KV watermarkAndState; WindowedSplitResult windowedSplitResult = null; synchronized (splitLock) { @@ -1628,6 +1666,13 @@ private HandlesSplits.SplitResult trySplitForElementAndRestriction( if (currentTracker == null) { return null; } + // The tracker has not yet been claimed meaning that a checkpoint won't meaningfully advance. + if (fractionOfRemainder == 0 + && requireClaimForCheckpoint + && currentTrackerClaimed != null + && !currentTrackerClaimed.get()) { + return null; + } // Make sure to get the output watermark before we split to ensure that the lower bound // applies to the residual. watermarkAndState = currentWatermarkEstimator.getWatermarkAndState(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 11f25ab0116ef..f4d555dabcc18 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -24,6 +24,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; @@ -53,6 +54,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import org.apache.beam.fn.harness.FnApiDoFnRunner.SplitResultsWithStopIndex; import org.apache.beam.fn.harness.FnApiDoFnRunner.WindowedSplitResult; @@ -151,6 +153,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; +import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Rule; @@ -1370,41 +1373,83 @@ public void testRegistration() { *

  • splitting thread: {@link * NonWindowObservingTestSplittableDoFn#waitForSplitElementToBeProcessed()} *
  • process element thread: {@link - * NonWindowObservingTestSplittableDoFn#enableAndWaitForTrySplitToHappen()} + * NonWindowObservingTestSplittableDoFn#splitElementProcessed()} *
  • splitting thread: perform try split - *
  • splitting thread: {@link - * NonWindowObservingTestSplittableDoFn#releaseWaitingProcessElementThread()} + *
  • splitting thread: {@link NonWindowObservingTestSplittableDoFn#trySplitPerformed()} * + *
  • process element thread: {@link + * NonWindowObservingTestSplittableDoFn#waitForTrySplitPerformed()} * */ static class NonWindowObservingTestSplittableDoFn extends DoFn { - private static final ConcurrentMap> - DOFN_INSTANCE_TO_LOCK = new ConcurrentHashMap<>(); + private static final ConcurrentMap DOFN_INSTANCE_TO_LATCHES = + new ConcurrentHashMap<>(); private static final long SPLIT_ELEMENT = 3; private static final long CHECKPOINT_UPPER_BOUND = 8; - private KV getLatches() { - return DOFN_INSTANCE_TO_LOCK.computeIfAbsent( - this.uuid, (uuid) -> KV.of(new CountDownLatch(1), new CountDownLatch(1))); + static class Latches { + public Latches() {} + + CountDownLatch blockProcessLatch = new CountDownLatch(0); + CountDownLatch processEnteredLatch = new CountDownLatch(1); + CountDownLatch splitElementProcessedLatch = new CountDownLatch(1); + CountDownLatch trySplitPerformedLatch = new CountDownLatch(1); + AtomicBoolean abortProcessing = new AtomicBoolean(); + } + + private Latches getLatches() { + return DOFN_INSTANCE_TO_LATCHES.computeIfAbsent(this.uuid, (uuid) -> new Latches()); + } + + public void splitElementProcessed() { + getLatches().splitElementProcessedLatch.countDown(); } - public void enableAndWaitForTrySplitToHappen() throws Exception { - KV latches = getLatches(); - latches.getKey().countDown(); - if (!latches.getValue().await(30, TimeUnit.SECONDS)) { + public void waitForSplitElementToBeProcessed() throws InterruptedException { + if (!getLatches().splitElementProcessedLatch.await(30, TimeUnit.SECONDS)) { fail("Failed to wait for trySplit to occur."); } } - public void waitForSplitElementToBeProcessed() throws Exception { - KV latches = getLatches(); - if (!latches.getKey().await(30, TimeUnit.SECONDS)) { - fail("Failed to wait for split element to be processed."); + public void trySplitPerformed() { + getLatches().trySplitPerformedLatch.countDown(); + } + + public void waitForTrySplitPerformed() throws InterruptedException { + if (!getLatches().trySplitPerformedLatch.await(30, TimeUnit.SECONDS)) { + fail("Failed to wait for trySplit to occur."); } } - public void releaseWaitingProcessElementThread() { - KV latches = getLatches(); - latches.getValue().countDown(); + // Must be called before process is invoked. Will prevent process from doing anything until + // unblockProcess is + // called. + public void setupBlockProcess() { + getLatches().blockProcessLatch = new CountDownLatch(1); + } + + public void enterProcessAndBlockIfEnabled() throws InterruptedException { + getLatches().processEnteredLatch.countDown(); + if (!getLatches().blockProcessLatch.await(30, TimeUnit.SECONDS)) { + fail("Failed to wait for unblockProcess to occur."); + } + } + + public void waitForProcessEntered() throws InterruptedException { + if (!getLatches().processEnteredLatch.await(5, TimeUnit.SECONDS)) { + fail("Failed to wait for process to begin."); + } + } + + public void unblockProcess() throws InterruptedException { + getLatches().blockProcessLatch.countDown(); + } + + public void setAbortProcessing() { + getLatches().abortProcessing.set(true); + } + + public boolean shouldAbortProcessing() { + return getLatches().abortProcessing.get(); } private final String uuid; @@ -1421,13 +1466,14 @@ public ProcessContinuation processElement( throws Exception { long checkpointUpperBound = CHECKPOINT_UPPER_BOUND; long position = tracker.currentRestriction().getFrom(); - boolean claimStatus; - while (true) { + boolean claimStatus = true; + while (!shouldAbortProcessing()) { claimStatus = tracker.tryClaim(position); if (!claimStatus) { break; } else if (position == SPLIT_ELEMENT) { - enableAndWaitForTrySplitToHappen(); + splitElementProcessed(); + waitForTrySplitPerformed(); } context.outputWithTimestamp( context.element() + ":" + position, @@ -1511,15 +1557,17 @@ public ProcessContinuation processElement( RestrictionTracker tracker, ManualWatermarkEstimator watermarkEstimator) throws Exception { + enterProcessAndBlockIfEnabled(); long checkpointUpperBound = Long.parseLong(context.sideInput(singletonSideInput)); long position = tracker.currentRestriction().getFrom(); - boolean claimStatus; - while (true) { + boolean claimStatus = true; + while (!shouldAbortProcessing()) { claimStatus = tracker.tryClaim(position); if (!claimStatus) { break; } else if (position == NonWindowObservingTestSplittableDoFn.SPLIT_ELEMENT) { - enableAndWaitForTrySplitToHappen(); + splitElementProcessed(); + waitForTrySplitPerformed(); } context.outputWithTimestamp( context.element() + ":" + position, @@ -1549,7 +1597,8 @@ public TruncateResult truncateRestriction(@Restriction OffsetRange throws Exception { // Waiting for split when we are on the second window. if (splitAtTruncate && processedWindowCount == PROCESSED_WINDOW) { - enableAndWaitForTrySplitToHappen(); + splitElementProcessed(); + waitForTrySplitPerformed(); } processedWindowCount += 1; return TruncateResult.of(new OffsetRange(range.getFrom(), range.getTo() / 2)); @@ -1755,7 +1804,217 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { return ((HandlesSplits) mainInput).trySplit(0); } finally { - doFn.releaseWaitingProcessElementThread(); + doFn.trySplitPerformed(); + } + }); + + // Check that before processing an element we don't report progress + assertNoReportedProgress(context.getBundleProgressReporters()); + mainInput.accept( + valueInGlobalWindow( + KV.of( + KV.of("7", KV.of(new OffsetRange(0, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), + 2.0))); + HandlesSplits.SplitResult trySplitResult = trySplitFuture.get(); + + // Check that after processing an element we don't report progress + assertNoReportedProgress(context.getBundleProgressReporters()); + + // Since the SPLIT_ELEMENT is 3 we will process 0, 1, 2, 3 then be split. + // We expect that the watermark advances to MIN + 2 since the manual watermark estimator + // has yet to be invoked for the split element and that the primary represents [0, 4) with + // the original watermark while the residual represents [4, 5) with the new MIN + 2 + // watermark. + assertThat( + mainOutputValues, + contains( + timestampedValueInGlobalWindow( + "7:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0))), + timestampedValueInGlobalWindow( + "7:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1))), + timestampedValueInGlobalWindow( + "7:2", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2))), + timestampedValueInGlobalWindow( + "7:3", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(3))))); + + BundleApplication primaryRoot = Iterables.getOnlyElement(trySplitResult.getPrimaryRoots()); + DelayedBundleApplication residualRoot = + Iterables.getOnlyElement(trySplitResult.getResidualRoots()); + assertEquals(ParDoTranslation.getMainInputName(pTransform), primaryRoot.getInputId()); + assertEquals(TEST_TRANSFORM_ID, primaryRoot.getTransformId()); + assertEquals( + ParDoTranslation.getMainInputName(pTransform), + residualRoot.getApplication().getInputId()); + assertEquals(TEST_TRANSFORM_ID, residualRoot.getApplication().getTransformId()); + assertEquals( + valueInGlobalWindow( + KV.of( + KV.of("7", KV.of(new OffsetRange(0, 4), GlobalWindow.TIMESTAMP_MIN_VALUE)), + 4.0)), + inputCoder.decode(primaryRoot.getElement().newInput())); + assertEquals( + valueInGlobalWindow( + KV.of( + KV.of( + "7", + KV.of( + new OffsetRange(4, 5), + GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)))), + 1.0)), + inputCoder.decode(residualRoot.getApplication().getElement().newInput())); + Instant expectedOutputWatermark = GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)); + assertEquals( + ImmutableMap.of( + "output", + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() + .setSeconds(expectedOutputWatermark.getMillis() / 1000) + .setNanos((int) (expectedOutputWatermark.getMillis() % 1000) * 1000000) + .build()), + residualRoot.getApplication().getOutputWatermarksMap()); + // We expect 0 resume delay. + assertEquals( + residualRoot.getRequestedTimeDelay().getDefaultInstanceForType(), + residualRoot.getRequestedTimeDelay()); + // We don't expect the outputs to goto the SDK initiated checkpointing listener. + assertTrue(splitListener.getPrimaryRoots().isEmpty()); + assertTrue(splitListener.getResidualRoots().isEmpty()); + mainOutputValues.clear(); + executorService.shutdown(); + } + + Iterables.getOnlyElement(context.getFinishBundleFunctions()).run(); + assertThat(mainOutputValues, empty()); + + Iterables.getOnlyElement(context.getTearDownFunctions()).run(); + assertThat(mainOutputValues, empty()); + + // Assert that state data did not change + assertEquals( + new FakeBeamFnStateClient(StringUtf8Coder.of(), stateData).getData(), + fakeClient.getData()); + } + + @Test + public void testProcessElementForSizedElementAndRestrictionSplitBeforeTryClaim() + throws Exception { + Pipeline p = Pipeline.create(); + addExperiment(p.getOptions().as(ExperimentalOptions.class), "beam_fn_api"); + // TODO(BEAM-10097): Remove experiment once all portable runners support this view type + addExperiment(p.getOptions().as(ExperimentalOptions.class), "use_runner_v2"); + PCollection valuePCollection = p.apply(Create.of("unused")); + PCollectionView singletonSideInputView = valuePCollection.apply(View.asSingleton()); + WindowObservingTestSplittableDoFn doFn = + new WindowObservingTestSplittableDoFn(singletonSideInputView); + valuePCollection.apply( + TEST_TRANSFORM_ID, ParDo.of(doFn).withSideInputs(singletonSideInputView)); + + RunnerApi.Pipeline pProto = + ProtoOverrides.updateTransform( + PTransformTranslation.PAR_DO_TRANSFORM_URN, + PipelineTranslation.toProto(p, SdkComponents.create(p.getOptions()), true), + SplittableParDoExpander.createSizedReplacement()); + String expandedTransformId = + Iterables.find( + pProto.getComponents().getTransformsMap().entrySet(), + entry -> + entry + .getValue() + .getSpec() + .getUrn() + .equals( + PTransformTranslation + .SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN) + && entry.getValue().getUniqueName().contains(TEST_TRANSFORM_ID)) + .getKey(); + RunnerApi.PTransform pTransform = + pProto.getComponents().getTransformsOrThrow(expandedTransformId); + String inputPCollectionId = + pTransform.getInputsOrThrow(ParDoTranslation.getMainInputName(pTransform)); + RunnerApi.PCollection inputPCollection = + pProto.getComponents().getPcollectionsOrThrow(inputPCollectionId); + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents(pProto.getComponents()); + Coder inputCoder = + WindowedValue.getFullCoder( + CoderTranslation.fromProto( + pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), + rehydratedComponents, + TranslationContext.DEFAULT), + (Coder) + CoderTranslation.fromProto( + pProto + .getComponents() + .getCodersOrThrow( + pProto + .getComponents() + .getWindowingStrategiesOrThrow( + inputPCollection.getWindowingStrategyId()) + .getWindowCoderId()), + rehydratedComponents, + TranslationContext.DEFAULT)); + String outputPCollectionId = pTransform.getOutputsOrThrow("output"); + + ImmutableMap> stateData = + ImmutableMap.of( + iterableSideInputKey( + singletonSideInputView.getTagInternal().getId(), ByteString.EMPTY), + asList("8")); + + FakeBeamFnStateClient fakeClient = new FakeBeamFnStateClient(StringUtf8Coder.of(), stateData); + + BundleSplitListener.InMemory splitListener = BundleSplitListener.InMemory.create(); + + PTransformRunnerFactoryTestContext context = + PTransformRunnerFactoryTestContext.builder(TEST_TRANSFORM_ID, pTransform) + .beamFnStateClient(fakeClient) + .processBundleInstructionId("57") + .pCollections(pProto.getComponentsOrBuilder().getPcollectionsMap()) + .coders(pProto.getComponents().getCodersMap()) + .windowingStrategies(pProto.getComponents().getWindowingStrategiesMap()) + .splitListener(splitListener) + .build(); + List> mainOutputValues = new ArrayList<>(); + context.addPCollectionConsumer( + outputPCollectionId, + (FnDataReceiver) (FnDataReceiver>) mainOutputValues::add); + + new FnApiDoFnRunner.Factory<>().createRunnerForPTransform(context); + + Iterables.getOnlyElement(context.getStartBundleFunctions()).run(); + mainOutputValues.clear(); + + assertThat( + context.getPCollectionConsumers().keySet(), + containsInAnyOrder(inputPCollectionId, outputPCollectionId)); + + FnDataReceiver> mainInput = + context.getPCollectionConsumer(inputPCollectionId); + assertThat(mainInput, instanceOf(HandlesSplits.class)); + + doFn.setupBlockProcess(); + { + // Setup and launch the trySplit thread. + ExecutorService executorService = Executors.newSingleThreadExecutor(); + Future trySplitFuture = + executorService.submit( + () -> { + try { + // Verify that a split before anything is claimed is ignored. + doFn.waitForProcessEntered(); + Assert.assertNull(((HandlesSplits) mainInput).trySplit(0)); + doFn.unblockProcess(); + + doFn.waitForSplitElementToBeProcessed(); + // Currently processing "3" out of range [0, 5) elements. + assertEquals(0.6, ((HandlesSplits) mainInput).getProgress(), 0.01); + + // Check that during progressing of an element we report progress + assertReportedProgressEquals( + context.getShortIdMap(), context.getBundleProgressReporters(), 3.0, 2.0); + + return ((HandlesSplits) mainInput).trySplit(0); + } finally { + doFn.trySplitPerformed(); } }); @@ -1845,6 +2104,149 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { fakeClient.getData()); } + @Test + public void testProcessElementForSizedElementAndRestrictionNoTryClaim() throws Exception { + Pipeline p = Pipeline.create(); + addExperiment(p.getOptions().as(ExperimentalOptions.class), "beam_fn_api"); + // TODO(BEAM-10097): Remove experiment once all portable runners support this view type + addExperiment(p.getOptions().as(ExperimentalOptions.class), "use_runner_v2"); + PCollection valuePCollection = p.apply(Create.of("unused")); + PCollectionView singletonSideInputView = valuePCollection.apply(View.asSingleton()); + WindowObservingTestSplittableDoFn doFn = + new WindowObservingTestSplittableDoFn(singletonSideInputView); + doFn.setAbortProcessing(); + valuePCollection.apply( + TEST_TRANSFORM_ID, ParDo.of(doFn).withSideInputs(singletonSideInputView)); + + RunnerApi.Pipeline pProto = + ProtoOverrides.updateTransform( + PTransformTranslation.PAR_DO_TRANSFORM_URN, + PipelineTranslation.toProto(p, SdkComponents.create(p.getOptions()), true), + SplittableParDoExpander.createSizedReplacement()); + String expandedTransformId = + Iterables.find( + pProto.getComponents().getTransformsMap().entrySet(), + entry -> + entry + .getValue() + .getSpec() + .getUrn() + .equals( + PTransformTranslation + .SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN) + && entry.getValue().getUniqueName().contains(TEST_TRANSFORM_ID)) + .getKey(); + RunnerApi.PTransform pTransform = + pProto.getComponents().getTransformsOrThrow(expandedTransformId); + String inputPCollectionId = + pTransform.getInputsOrThrow(ParDoTranslation.getMainInputName(pTransform)); + RunnerApi.PCollection inputPCollection = + pProto.getComponents().getPcollectionsOrThrow(inputPCollectionId); + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents(pProto.getComponents()); + Coder inputCoder = + WindowedValue.getFullCoder( + CoderTranslation.fromProto( + pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), + rehydratedComponents, + TranslationContext.DEFAULT), + (Coder) + CoderTranslation.fromProto( + pProto + .getComponents() + .getCodersOrThrow( + pProto + .getComponents() + .getWindowingStrategiesOrThrow( + inputPCollection.getWindowingStrategyId()) + .getWindowCoderId()), + rehydratedComponents, + TranslationContext.DEFAULT)); + String outputPCollectionId = pTransform.getOutputsOrThrow("output"); + + ImmutableMap> stateData = + ImmutableMap.of( + iterableSideInputKey( + singletonSideInputView.getTagInternal().getId(), ByteString.EMPTY), + asList("8")); + + FakeBeamFnStateClient fakeClient = new FakeBeamFnStateClient(StringUtf8Coder.of(), stateData); + + BundleSplitListener.InMemory splitListener = BundleSplitListener.InMemory.create(); + + PTransformRunnerFactoryTestContext context = + PTransformRunnerFactoryTestContext.builder(TEST_TRANSFORM_ID, pTransform) + .beamFnStateClient(fakeClient) + .processBundleInstructionId("57") + .pCollections(pProto.getComponentsOrBuilder().getPcollectionsMap()) + .coders(pProto.getComponents().getCodersMap()) + .windowingStrategies(pProto.getComponents().getWindowingStrategiesMap()) + .splitListener(splitListener) + .build(); + List> mainOutputValues = new ArrayList<>(); + context.addPCollectionConsumer( + outputPCollectionId, + (FnDataReceiver) (FnDataReceiver>) mainOutputValues::add); + + new FnApiDoFnRunner.Factory<>().createRunnerForPTransform(context); + + Iterables.getOnlyElement(context.getStartBundleFunctions()).run(); + mainOutputValues.clear(); + + assertThat( + context.getPCollectionConsumers().keySet(), + containsInAnyOrder(inputPCollectionId, outputPCollectionId)); + + FnDataReceiver> mainInput = + context.getPCollectionConsumer(inputPCollectionId); + assertThat(mainInput, instanceOf(HandlesSplits.class)); + + { + // Check that before processing an element we don't report progress + assertNoReportedProgress(context.getBundleProgressReporters()); + mainInput.accept( + valueInGlobalWindow( + KV.of( + KV.of("5", KV.of(new OffsetRange(5, 10), GlobalWindow.TIMESTAMP_MIN_VALUE)), + 5.0))); + // Check that after processing an element we don't report progress + assertNoReportedProgress(context.getBundleProgressReporters()); + + // Since we set abort processing above, we expect the input restriction to be output with a + // resume + // delay. + BundleApplication primaryRoot = Iterables.getOnlyElement(splitListener.getPrimaryRoots()); + DelayedBundleApplication residualRoot = + Iterables.getOnlyElement(splitListener.getResidualRoots()); + assertEquals(ParDoTranslation.getMainInputName(pTransform), primaryRoot.getInputId()); + assertEquals(TEST_TRANSFORM_ID, primaryRoot.getTransformId()); + assertEquals( + ParDoTranslation.getMainInputName(pTransform), + residualRoot.getApplication().getInputId()); + assertEquals(TEST_TRANSFORM_ID, residualRoot.getApplication().getTransformId()); + assertEquals( + valueInGlobalWindow( + KV.of( + KV.of("5", KV.of(new OffsetRange(5, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), + 0.0)), + inputCoder.decode(primaryRoot.getElement().newInput())); + assertEquals( + valueInGlobalWindow( + KV.of( + KV.of("5", KV.of(new OffsetRange(5, 10), GlobalWindow.TIMESTAMP_MIN_VALUE)), + 5.0)), + inputCoder.decode(residualRoot.getApplication().getElement().newInput())); + assertThat(residualRoot.getApplication().getOutputWatermarksMap(), anEmptyMap()); + assertEquals( + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.newBuilder() + .setSeconds(54) + .setNanos(321000000) + .build(), + residualRoot.getRequestedTimeDelay()); + splitListener.clear(); + } + } + private static final MonitoringInfo WORK_COMPLETED_MI = MonitoringInfo.newBuilder() .setUrn(MonitoringInfoConstants.Urns.WORK_COMPLETED) @@ -2187,7 +2589,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc return ((HandlesSplits) mainInput).trySplit(0); } finally { - doFn.releaseWaitingProcessElementThread(); + doFn.trySplitPerformed(); } }); @@ -3143,10 +3545,11 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser () -> { try { doFn.waitForSplitElementToBeProcessed(); - - return ((HandlesSplits) mainInput).trySplit(0); + HandlesSplits.SplitResult result = ((HandlesSplits) mainInput).trySplit(0); + Assert.assertNotNull(result); + return result; } finally { - doFn.releaseWaitingProcessElementThread(); + doFn.trySplitPerformed(); } }); From dbd719ba1448c13bc97237d1b701e2978c0e29d4 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 14 Aug 2024 09:50:17 -0400 Subject: [PATCH 77/78] [WIP] Gemma Sentiment and Summarization Example Notebook (#32172) --- .../gemma_2_sentiment_and_summarization.ipynb | 625 ++++++++++++++++++ 1 file changed, 625 insertions(+) create mode 100644 examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb diff --git a/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb b/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb new file mode 100644 index 0000000000000..b45d9d7aea959 --- /dev/null +++ b/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb @@ -0,0 +1,625 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": null, + "id": "BrKf6TQ98qIJ", + "metadata": { + "id": "BrKf6TQ98qIJ" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "markdown", + "id": "hHg4SoUr8qIK", + "metadata": { + "id": "hHg4SoUr8qIK" + }, + "source": [ + "# Use Gemma to gauge sentiment and summarize conversations\n", + "\n", + "\n", + " \n", + " \n", + "
    \n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
    " + ] + }, + { + "cell_type": "markdown", + "id": "yOs5SCyPdYNi", + "metadata": { + "id": "yOs5SCyPdYNi" + }, + "source": [ + "Gemma is a family of lightweight, state-of-the art open models built from research and technology used to create the Gemini models. You can use Gemma models in your Apache Beam inference pipelines.\n", + "\n", + "Because large language models (LLMs) like Gemma are versatile, you can integrate them into business processes. The example in this notebook demonstrates how to use Gemma to gauge the sentiment of a conversation, summarize that conversation's content, and draft a reply for a difficult conversation. The system allows a person to review the reply before it's sent to customers. For more information, see the blog post [Gemma for Streaming ML with Dataflow](https://developers.googleblog.com/en/gemma-for-streaming-ml-with-dataflow).\n", + "\n", + "A requirement of this work is that customers who express a negative sentiment receive a reply in near real-time. As a result, the workflow needs to use a streaming data pipeline with an LLM that has minimal latency.\n", + "\n", + "## Use case\n", + "\n", + "An example use case is a bustling food chain grappling with analyzing and storing a high volume of customer support requests. Customer interactions include both chats generated by automated chatbots and nuanced conversations that require the attention of live support staff.\n", + "\n", + "### Requirements\n", + "\n", + "To address both types of interactions, the workflow has the following requirements.\n", + "\n", + "- It needs to efficiently manage and store chat data by summarizing positive interactions for easy reference and future analysis.\n", + "\n", + "- It must use real-time issue detection and resolution.\n", + "\n", + "- Sentiment analysis must identify dissatisfied customers and generate tailored responses to address their concerns.\n", + "\n", + "### Workflow\n", + "\n", + "To meet these requirements, the pipeline processes completed chat messages in near real time. First, the pipeline uses Gemma to monitor the sentiment of the customer chats. All chats are then summarized, with positive or neutral sentiment chats sent directly to a data platform, BigQuery, by using the available Dataflow I/Os.\n", + "\n", + "For chats that have a negative sentiment, the Gemma model crafts a contextually appropriate response for the customer. This response is sent to a human for review so that they can refine the message before it reaches the customer.\n", + "\n", + "This example addresses important complexities inherent in using an LLM within a pipeline. For example, processing the responses in code is challenging because of the non-deterministic nature of the text. In this example, the workflow requires the LLM to generate JSON responses, which is not the default format. The worklow needs to parse and validate the response, a process similar to processing data from sources that don't always have correctly structured data.\n", + "\n", + "This workflow allows businesses to respond to customers faster and to provide personalized responses when needed.\n", + "\n", + "- The automation of positive chat summarization allows support staff to focus on more complex interactions.\n", + "- The scalability of the system makes it possible to adapt to increasing chat volumes without compromising response quality.\n", + "\n", + "You can also use the in-depth analysis of chat data to drive data-driven decision-making." + ] + }, + { + "cell_type": "markdown", + "id": "tGZfleinj3xM", + "metadata": { + "id": "tGZfleinj3xM" + }, + "source": [ + "## The data processing pipeline" + ] + }, + { + "cell_type": "markdown", + "id": "G-VpT7-FjcSu", + "metadata": { + "id": "G-VpT7-FjcSu" + }, + "source": [ + "![Screenshot 2024-08-08 at 11.15.41.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAACVwAAAVECAYAAAAlH0OMAAABXmlDQ1BJQ0MgUHJvZmlsZQAAKJF1kEFLAlEQx/+WpahEQYcOBXZZKExEjU4dzCQChdWKskO0rpsKqz3ebkS3PkT0EaJvIEGHOlTXICjwVETQoU7BXkq2eW6lFs0w/H/8mXlvGKAnoDCmuwFUaybPLcwF1/LrQc8zvJQ+TCOsqAZLyHKaWvCt3WHdwiX0Zkq81cifX0izT1Im03/lzY69/O3vCl9RM1TSD6q4yrgJuCLE8q7JBO8TD3NaivhAcMnhY8EFh09bPcu5JPE18aBaVorE98ShQodf6uCqvqN+7SC2D2i1lSXSEapRzCOFNGUQMmKIUsaQReqfmXhrJoltMOyBo4ISyjBpOkEOgw6NeBE1qAgjRBxFhCoubv37hm2PDwEzr/TVeNvb0IATuo9/su1Jm8DAI3D5wBSu/FzWZbmNrVjUYX8d6Du07bdVwDMBNO9s+71u280joLcBnFmfhiBimBzeyIwAAACKZVhJZk1NACoAAAAIAAQBGgAFAAAAAQAAAD4BGwAFAAAAAQAAAEYBKAADAAAAAQACAACHaQAEAAAAAQAAAE4AAAAAAAAAkAAAAAEAAACQAAAAAQADkoYABwAAABIAAAB4oAIABAAAAAEAAAlcoAMABAAAAAEAAAVEAAAAAEFTQ0lJAAAAU2NyZWVuc2hvdDy2X1MAAAAJcEhZcwAAFiUAABYlAUlSJPAAAAHYaVRYdFhNTDpjb20uYWRvYmUueG1wAAAAAAA8eDp4bXBtZXRhIHhtbG5zOng9ImFkb2JlOm5zOm1ldGEvIiB4OnhtcHRrPSJYTVAgQ29yZSA2LjAuMCI+CiAgIDxyZGY6UkRGIHhtbG5zOnJkZj0iaHR0cDovL3d3dy53My5vcmcvMTk5OS8wMi8yMi1yZGYtc3ludGF4LW5zIyI+CiAgICAgIDxyZGY6RGVzY3JpcHRpb24gcmRmOmFib3V0PSIiCiAgICAgICAgICAgIHhtbG5zOmV4aWY9Imh0dHA6Ly9ucy5hZG9iZS5jb20vZXhpZi8xLjAvIj4KICAgICAgICAgPGV4aWY6UGl4ZWxZRGltZW5zaW9uPjEzNDg8L2V4aWY6UGl4ZWxZRGltZW5zaW9uPgogICAgICAgICA8ZXhpZjpQaXhlbFhEaW1lbnNpb24+MjM5NjwvZXhpZjpQaXhlbFhEaW1lbnNpb24+CiAgICAgICAgIDxleGlmOlVzZXJDb21tZW50PlNjcmVlbnNob3Q8L2V4aWY6VXNlckNvbW1lbnQ+CiAgICAgIDwvcmRmOkRlc2NyaXB0aW9uPgogICA8L3JkZjpSREY+CjwveDp4bXBtZXRhPgq6WWIXAAAAHGlET1QAAAACAAAAAAAAAqIAAAAoAAACogAAAqIAAcYyo/fplgAAQABJREFUeAHs3Qe4XFW5P+DvtJyEklASeu8lIL1X6V5AELiKXkWuCioK2BsoKNarXjs27BSvgAiKgiCIgBBASIDQISEJJQkloSSn//ca/zOcMzlJ5iRz5kx51/NwZ+89e9Ze612b+zzEX77V1Je10AgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIEBgqQJNAldLNXIDAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEcgICV14EAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIlCggcFUilNsIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAgcOUdIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAQIkCAlclQrmNAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECAlfeAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECJQoIHBVIpTbCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgIHDlHSBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgECJAgJXJUK5jQABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgJX3gECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAiUKCBwVSKU2wgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQICBw5R0gQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAiQICVyVCuY0AAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQICV94BAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIlCggcFUilNsIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAgcOUdIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAQIkCAlclQrmNAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECAlfeAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECJQoIHBVIpTbCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgIHDlHSBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgECJAgJXJUK5jQABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgJX3gECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAiUKCBwVSKU2wgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQICBw5R0gQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAiQICVyVCuY0AAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQICV94BAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIlCggcFUilNsIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAgcOUdIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAQIkCAlclQrmNAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECAlfeAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECJQoIHBVIpTbCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgIHDlHSBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgECJAgJXJUK5jQABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgJX3gECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAiUKCBwVSKU2wgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQICBw5R0gQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAiQICVyVCuY0AAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQICV94BAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIlCggcFUilNsIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAgcOUdIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAQIkCAlclQrmNAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECAlfeAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECJQoIHBVIpTbCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgIHDlHSBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgECJAgJXJUK5jQABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgJX3gECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAiUKCBwVSKU2wgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQICBw5R0gQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAiQICVyVCuY0AAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQICV94BAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIlCggcFUilNsIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAgcOUdIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAQIkCAlclQrmNAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECAlfeAQIECBAgMMwCM2bMiHnz5g3zU3RPgAABAgQIECBAgAABAgQIECBAgAABAgQIECBA4DWBDTfcMFZeeeXXLjgiQKBsAgJXZaPUEQECBAgQWFTg6aefjve9732LfuEKAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAgWEU2HvvveNjH/vYMD5B1wQaV0DgqnHX3swJECBAoAIC9913X5x11lkVeJJHECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIEHhNYOLEiXHeeee9dsERAQJlExC4KhuljggQIECAwKIC/QNXn/3c5xa9wRUCBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECZRS49He/i6lTp4bAVRlRdUWgSEDgqgjEKQECBAgQKKdAceBqm222LWf3+iJAgAABAgQIECBAgAABAgQIECBAgAABAgQIECAwQODz554jcDVAxAmB8gsIXJXfVI8ECBAgQKAgIHBVoHBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQAQGBqwoge0TDCwhcNfwrAIAAAQIEhlNA4Go4dfVNgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQLCBwVSzinED5BQSuym+qRwIECBAgUBAQuCpQOCBAgAABAgQIECBAgAABAgQIECBAgAABAgQIEKiAgMBVBZA9ouEFBK4a/hUAQIAAAQLDKSBwNZy6+iZAgAABAgQIECBAgAABAgQIECBAgAABAgQIECgWELgqFnFOoPwCAlflN9UjAQIECBAoCAhcFSgcECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIVEBA4KoCyB7R8AICVw3/CgAgQIAAgeEUELgaTl19EyBAgAABAgQIECBAgAABAgQIECBAgAABAgQIFAsIXBWLOCdQfgGBq/Kb6pEAAQIECBQEBK4KFA4IECBAgAABAgQIECBAgAABAgQIECBAgAABAgQqICBwVQFkj2h4AYGrhn8FABAgQIDAcAoIXA2nrr4JECBAgAABAgQIECBAgAABAgQIECBAgAABAgSKBQSuikWcEyi/gMBV+U31SIAAAQIECgICVwUKBwQIECBAgAABAgQIECBAgAABAgQIECBAgAABAhUQELiqALJHNLyAwFXDvwIACBAgQGA4BQSuhlNX3wQIECBAgAABAgQIECBAgAABAgQIECBAgAABAsUCAlfFIs4JlF9A4Kr8pnokQIAAAQIFAYGrAoUDAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBCggIXFUA2SMaXkDgquFfAQAECBAgMJwCAlfDqatvAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBYgGBq2IR5wTKLyBwVX5TPRIgQIAAgYKAwFWBwgEBAgQIECBAgAABAgQIECBAgAABAgQIECBAgEAFBASuKoDsEQ0vIHDV8K8AAAIECBAYTgGBq+HU1TcBAgQIECBAgAABAgQIECBAgAABAgQIECBAgECxgMBVsYhzAuUXELgqv6keCRAgQIBAQUDgqkDhgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAoAICAlcVQPaIhhcQuGr4VwAAAQIECAyngMDVcOrqmwABAgQIECBAgAABAgQIECBAgAABAgQIECBAoFhA4KpYxDmB8gsIXJXfVI8ECBAgQKAgIHBVoHBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQAQGBqwoge0TDCwhcNfwrAIAAAQIEhlNA4Go4dfVNgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQLCBwVSzinED5BQSuym+qRwIECBAgUBAQuCpQOCBAgAABAgQIECBAgAABAgQIECBAgAABAgQIEKiAgMBVBZA9ouEFBK4a/hUAQIAAAQLDKSBwNZy6+iZAgAABAgQIECBAgAABAgQIECBAgAABAgQIECgWELgqFnFOoPwCAlflN9UjAQIECBAoCAhcFSgcECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIVEBA4KoCyB7R8AICVw3/CgAgQIAAgeEUELgaTl19EyBAgAABAgQIECBAgAABAgQIECBAgAABAgQIFAsIXBWLOCdQfgGBq/Kb6pEAAQIECBQEBK4KFA4IECBAgAABAgQIECBAgAABAgQIECBAgAABAgQqICBwVQFkj2h4AYGrhn8FABAgQIDAcAoIXA2nrr4JECBAgAABAgQIECBAgAABAgQIECBAgAABAgSKBQSuikWcEyi/gMBV+U31SIAAAQIECgICVwUKBwQIECBAgAABAgQIECBAgAABAgQIECBAgAABAhUQELiqALJHNLyAwFXDvwIACBAgQGA4BQSuhlNX3wQIECBAgAABAgQIECBAgAABAgQIECBAgAABAsUCAlfFIs4JlF9A4Kr8pnokQIAAAQIFAYGrAoUDAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBCggIXFUA2SMaXkDgquFfAQAECBAgMJwCAlfDqatvAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBYgGBq2IR5wTKLyBwVX5TPRIgQIAAgYKAwFWBwgEBAgQIECBAgAABAgQIECBAgAABAgQIECBAgEAFBASuKoDsEQ0vIHDV8K8AAAIECBAYTgGBq+HU1TcBAgQIECBAgAABAgQIECBAgAABAgQIECBAgECxgMBVsYhzAuUXELgqv6keCRAgQIBAQUDgqkDhgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAoAICAlcVQPaIhhcQuGr4VwAAAQIECAyngMDVcOrqmwABAgQIECBAgAABAgQIECBAgAABAgQIECBAoFhA4KpYxDmB8gsIXJXfVI8ECBAgQKAgIHBVoHBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQAQGBqwoge0TDCwhcNfwrAIAAAQIEhlNA4Go4dfVNgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQLCBwVSzinED5BQSuym+qRwIECBAgUBAQuCpQOCBAgAABAgQIECBAgAABAgQIECBAgAABAgQIEKiAgMBVBZA9ouEFBK4a/hUAQIAAAQLDKSBwNZy6+iZAgAABAgQIECBAgAABAgQIECBAgAABAgQIECgWELgqFnFOoPwCAlflN9UjAQIECBAoCAhcFSgcECBAgAABAgQIECBAgAABAgQIECBAgAABAgQIVEBA4KoCyB7R8AICVw3/CgAgQIAAgeEUELgaTl19EyBAgAABAgQIECBAgAABAgQIECBAgAABAgQIFAsIXBWLOCdQfgGBq/Kb6pEAAQIECBQEBK4KFA4IECBAgAABAgQIECBAgAABAgQIECBAgAABAgQqICBwVQFkj2h4AYGrhn8FABAgQIDAcAoIXA2nrr4JECBAgAABAgQIECBAgAABAgQIECBAgAABAgSKBQSuikWcEyi/gMBV+U31SIAAAQIECgICVwUKBwQIECBAgAABAgQIECBAgAABAgQIECBAgAABAhUQELiqALJHNLyAwFXDvwIACBAgQGA4BQSuhlNX3wQIECBAgAABAgQIECBAgAABAgQIECBAgAABAsUCAlfFIs4JlF9A4Kr8pnokQIAAAQIFAYGrAoUDAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBCggIXFUA2SMaXkDgquFfAQAECBAgMJwCAlfDqatvAgQIECBAgAABAgQIECBAgAABAgQIECBAgACBYgGBq2IR5wTKLyBwVX5TPRIgQIAAgYKAwFWBwgEBAgQIECBAgAABAgQIECBAgAABAgQIECBAgEAFBASuKoDsEQ0vIHDV8K8AAAIECBAYTgGBq+HU1TcBAgQIECBAgAABAgQIECBAgAABAgQIECBAgECxgMBVsYhzAuUXELgqv6keCRAgQIBAQUDgqkDhgAABAgQIECBAgAABAgQIECBAgAABAgQIECBAoAICAlcVQPaIhhcQuGr4VwAAAQIECAyngMDVcOrqmwABAgQIECBAgAABAgQIECBAgAABAgQIECBAoFhA4KpYxDmB8gsIXJXfVI8ECBAgQKAgIHBVoHBAgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQAQGBqwoge0TDCwhcNfwrAIAAAQIEhlNA4Go4dfVNgAABAgQIECBAgAABAgQIECBAgAABAgQIECBQLCBwVSzinED5BQSuym+qRwIECBAgUBAQuCpQOCBAgAABAgQIECBAgAABAgQIECBAgAABAgQIEKiAgMBVBZA9ouEFBK4a/hUAQIAAAQLDKSBwNZy6+iZAgAABAgQIECBAgAABAgQIECBAgAABAgQIECgWELgqFnFOoPwCAlflN9UjAQIECBAoCAhcFSgcEBiyQF/2i6Yh/8oPCBAgQIAAAQIECBAgQIAAAQIECBAgQIBAYwsIXDX2+pt9ZQQErirj7CkECBAg0KACAlcNuvCmvVwCC7v64rop3bEg+zxyp7YYM0rsarlA/ZgAAQIECBAgQIAAAQIECBAgQIAAAQIEGkpA4KqhlttkR0hA4GqE4D2WAAECBBpDQOCqMdbZLMsj0JEFrO6f2Rt/uKMjnp2X6ltFjB0T8Za92mPb9Vujva08z9HLQIH5r/bFnY92xcKuiL22aotVVhRwGyjkjAABAgQIECBAgAABAgQIECBAgAABArUlIHBVW+tltLUpIHBVm+tm1AQIECBQIwICVzWyUIY54gIPzOqJi/7REc+93Be9/85aFcbUnOV/NhjfHKccPDpWW0kYqABThoNJj3bH+X9eEM+82Bt9mftaqzTH+48YE7tt3lqG3nVBgAABAgQIECBAgAABAgQIECBAgAABAiMhIHA1Euqe2WgCAleNtuLmS4AAAQIVFRC4qii3h9WgwMNP98Tf7uuKe5/siZ7eJU+gpTniwG3b4qCJbbGq4NWSsZbwbW/mPGV6d1x+e2dMejgra1XUmrJM275bt8XxWWWxzdduiXSuESBAgAABAgQIECBAgAABAgQIECBAgEDtCAhc1c5aGWntCghc1e7aGTkBAgQI1ICAwFUNLJIhVlwgVVJ6aUFfXHRzR9wzvWfIz29va4qTD2iPieu3RGvLkH/e0D+Yl20f+MNrFsSNWcgtrcOSWrJ9676j4017jorRmblGgAABAgQIECBAgAABAgQIECBAgAABArUhIHBVG+tklLUtIHBV2+tn9AQIECBQ5QICV1W+QIZXcYFXOvriyjs7467He+LlhUtJ/CxhdKnq0uZrtcSxu7XFxmtIXS2BKvdVClpdclNH3Di1M17Itm0cSttgQkscn4WuDtp+VKQqYxoBAgQIECBAgAABAgQIECBAgAABAgQIVLeAwFV1r4/R1YeAwFV9rKNZECBAgECVCghcVenCGFbFBeZngZ+7nuiO30/qjM7u8j7+4O3a4sBsm8HVbTO4COyrWcBt0iNd8bPrO2L2vKXs2bjIr1+7kOpb7b5lW5yUVRbbKAu42WbwNRtHBAgQIECAAAECBAgQIECAAAECBAgQqDYBgatqWxHjqUcBgat6XFVzIkCAAIGqERC4qpqlMJARFLjloe646q7OSFWWlraN3bIOc+UxTXH0zm2x79Zty9pF3f3u7se74ztXL4hnX+iN3qEVtVqsRdpa8KDt2+KUQ0dH2tpRI0CAAAECBAgQIECAAAECBAgQIECAAIHqExC4qr41MaL6ExC4qr81NSMCBAgQqCIBgasqWgxDqahAV0/ElOndcf193fH4s9lJhdoWa7fEETu2xZbrtERzg+aBHpjZE5f/syP++XBXdA8T/frjs20G92qPA7ZtFbyq0LvtMQQIECBAgAABAgQIECBAgAABAgQIEChVQOCqVCn3EVh2AYGrZbfzSwIECBAgsFQBgaulErmhzgRSJaW0dd0vbuyIaXOWfQu75WXZc4vWePNeoyJVZGqElgpYpW0bf3FDR/z5ro6KTXmr9Vrjk8eOiTVXabbNYMXUPYgAAQIECBAgQIAAAQIECBAgQIAAAQJLFhC4WrKPbwmUQ0DgqhyK+iBAgAABAosRELhaDIzLdSkw87neuDLbOvDBWT3R2T3yU1x1xaasAlNbHDSxNVpb6jd4lapYXXjTwrhuclfMmV/5kNuK7U2x7zZt8Y4D22O1lZpHfuGNgAABAgQIECBAgAABAgQIECBAgAABAg0uIHDV4C+A6VdEQOCqIsweQoAAAQKNKiBw1agr31jzTiGffzzQHTfc3xVpK8FqaxtOaIkT9hwVG01ojtY6ygMt7OqLOx7pjh9duzDmjkDQqnidx49tjvccOjp226w1xoyq34Bb8bydEyBAgAABAgQIECBAgAABAgQIECBAoNoEBK6qbUWMpx4FBK7qcVXNiQABAgSqRkDgqmqWwkCGQaAnK6b0hzs64x8PdseCzrSpXfW21paI7dZviZMOaK+LbQYfe6Ynvvr7BTHruZ5I61AtrTnLWW24Rkt89I1jYtO1MnSNAAECBAgQIECAAAECBAgQIECAAAECBCouIHBVcXIPbEABgasGXHRTJkCAAIHKCQhcVc7akyov0JdlrB6f3RPXZFvZ3ftkT6Tzam8rj2mKQ7dvi723ao0VarAKUwpa/f72zrjhvq7o6ale8NFtmfMObXHsHu2x9qp1VFas2l9w4yNAgAABAgQIECBAgAABAgQIECBAgEAmIHDlNSAw/AICV8Nv7AkECBAg0MACAlcNvPgNNvV7pvXExbd0xLxXqzcE1H9JJoxtivdn29+tOa45mqs8D5REF2YVxC7+R0f8X2ZcS23cik3xpf9aKTZds8qRawnVWAkQIECAAAECBAgQIECAAAECBAgQILAUAYGrpQD5mkAZBASuyoCoCwIECBAgsDgBgavFybhejwIpbPXPh7vjz3d3Rkd39c+wJcsAvW7D1njT7qNi/MrZXnhV2Lq6++L3k7ri6js74pkXq2jvwKVYNWWcu23eGm/ZZ3RstV5LVKfuUibhawIECBAgQIAAAQJ1JjB9+vR4+OGHY86cufHSS/NjlVVWidVXHx+bbbZZbLTRhnU2W9MZCYEXXnghJk+enHvHnn/++VhxxRVy79j6668X22yzTfYXnhrvL+PMnj07Lrvssnj11QVx+OGHxbbbbjvsS9PR0RGXXnpZzJgxI3bZZZc4+OCDhv2ZHkCAAAEC1ScgcFV9a2JE9ScgcFV/a2pGBAgQIFBFAgJXVbQYhlIxgVnP98blkzrjgZk90VsDBa/S9nfH7NoWu2zaGiuNro5oUEdXX9w/ozd+cu2CeCLbtrFWWgpabbxmS/znXu2x37Zt0VwdnLXCZ5wECBAgQIAAAQIEyi6wcOHC+MlPfhp//OMf44knnlhs/5tsskm84Q1HxCmnnBKjR49e7H2+qH2BQw89LJ555pnCRD7xiU/E29721sL5shxcccUVcckll8Q990yO3t7B/7LQ+PHj48ADD4zTTnt/rLPOOsvymJr7zcyZM+PYY4+NefPm58be0tIS55//gzjggAOGbS7d3d3x5je/Je69997CM0499ZT4yEc+Ujh3QIAAAQKNISBw1RjrbJYjKyBwNbL+nk6AAAECdS4gcFXnC2x6ixXoyf58dfrcLDB03cJ44ZXqT12loND4lZvj5P3bY5O1RvZv3M58rje+fsWCeOTp7kiOtdJSWO19h4+JvbdujRRi0wgQIECAAAECBAgQGFmBSZMmxVlnnRXTpk0veSApePX1r389Jk4c/io8JQ/KjWUTeOihh+Koo44e0N/1118X66+//oBrpZ7MmjUrzj77s3HzzTeX+pNYeeWV45xzPpeN46iSf1OrN5577rlx4YUXDRj+dtttl1W8unTAtXKe/PnPf4kzzjhjQJepsthdd92ZVRxbccB1JwQIECBQ3wICV/W9vmZXHQICV9WxDkZBgAABAnUqIHBVpwtrWiULvLSgL/7+QHfccF9XvNJR/cGrNLFdN2uNw7Zvi/VWr2zwavqcnrgyqwz218md0VkDWzLmX4KVxzTF4TuOiqN2HRVrjKusWX4MPgkQIECAAAECBAgQeE3g5Zdfjq997X/it7/9bfT1Df2/w1pbW7PAxunxnve8pyG3gHtNsv6OfvCDH8S3vvXtwsTSdpJXX/2nwnmpB6mK1a9//Zv45je/GQsWLCj1ZwPuO/LII3PBq7Fjxw64Xk8nZ555Zub75wFTSuG2FHIbrnbRRRdnrucs0v3f/35jrL322otcd4EAAQIE6ldA4Kp+19bMqkdA4Kp61sJICBAgQKAOBQSu6nBRTWmZBFLw6oIbOuLhp7NtBmugalNrS8Qxu4yKfbdpjfbW4a3W9GpnX/zpzs749Y0dWdBq6P9jyDItSBl+1JYZ7bhJW3z46DGxyorDa1SG4eqCAAECBAgQIECAQEMI9PT05LYTmzJlymLnu8IKK8R6660Xc+bMiRdeeGHQ+9LWZz/96U9i7733HvR7F2tT4PjjT4j+78aybjX35S9/OX7+818sFiGF9jbaaKNI4b/Zs2cvdpvBD3zgtDj99NMX20+tf3HllVfFRz/60QHTeMc73pFVnvvMgGvlPJk+fXoceeRR0dHRUeh2yy23jKuuurJw7oAAAQIEGkNA4Kox1tksR1ZA4Gpk/T2dAAECBOpcQOCqzhfY9IYk0NUT8egzPXHpbZ0x6/kaSF1ls1t3teZ4w45tsdPGrZG2HSxn6848/nRXZ1x1R0ekbQRrqe2QebzjgNGxxTotkcJpGgECBAgQIECAAAEC1SHwve99P77zne8sMpgNNtggF2zZfvvtY8MNN8j+++bf/4HzyiuvxF/+8pf45S9/FQ8++GDudyls9T//87UstHHkIv24ULsCKWC3zz77Dqh6dskll8ROO+04pEndfvukSKGh4uppabu6FKDac889Y/PNN4+2trZcv11dXXH77bfHr3716/j73/9e+N3b3va2+Oxnzy68i0MaRA3dfP7558cFF/wsF4A64ogj4txzz4kxY8YM6wxuvPHG+PznvxBPPfVU7LjjjvHVr34l0v8P0AgQIECgsQQErhprvc12ZAQErkbG3VMJECBAoEEEBK4aZKFNc0gCHVkVp2vu6Y6bH+yK+Vnlq1pou2zSGm/YqS3WXqV5uYNXvdmUp0zrjl/euDAemJGlrmqkpf89ZsMJLXH8nu3x+mzLxeYyB9BqhMEwCRAgQIAAAQIECFStwH333R8nnHBCpCpX+ZbCUyeddFKceeYZMXr06PzlRT5TeCZtD5eCIenz8MMPW+QeF2pb4He/+1185jNnFSax+uqrxy233DykbSNTxaqjjjoqZs16qtBPOth///2zgM+5S92y7pprromPf/wTWRW2/4xPf/rTA/qo95P072X697GSbSSeWcn5eRYBAgQILFlA4GrJPr4lUA4BgatyKOqDAAECBAgsRkDgajEwLje8QIpZvZyFrS65pSPueuK1/zGgmmHas7+cu//WbfHG3UZFyzKGjZ55oTe+fPmCeOyZ7kgVrmqltTRHnHbE6Dhwu1ExZtQyTr5WJmucBAgQIECAAAECBGpQIG0fdswxx8Zjjz02YPRnn31WvP3tbx9wbUknqSLOOuuss6RbBv3uX/+6O1cha+bMmbkKRuuss3ZsvPHGuWpHSwqZpKDX/PkvDehz3LixufP03a233hoPPfRQVqnn6ZgwYXxWjWmn2HXXXQfcn07S1ojXXXddzJw5K1dJaLPNNoutttoyJk6cuMi9+QsLFiyIzs6u/GkWSGuP9vb23PnTTz8dt912Wzz++OO57fA23HDDOOyww2LcuHGF+/MHkyZNismTp8Szzz6bjXFCbLHF5rHDDjvEqquumr9liZ9pzR544IF48skn4/nnX8hCS2vltnxMFYm23nrrJf52KF++973vi7/97W+Fnxx33HHx5S9/qXBeysGnPvXpuOyyywbcmly++91Fq6oNuKnfybK+Y9OmTYu77rorW+OZua0K11prrVh33fVi3333iVRda0ntpZdeytbxtb/wtdJKKxbCT1OnTo177pkc06Y9ESuttFK2flvGwQcfFGlbxP6ts7Mz87sh9+9Yet9SpbjNN98iqyC1Q+G96X9/Oh7s/V5xxRUG9D3YPfl/B1Ifab533nln9o48mM13ndz4tt9+u0hbgy6upWBcT89rlbTb20ctMXC5uH5cJ0CAAIHaFhC4qu31M/raEBC4qo11MkoCBAgQqFEBgasaXTjDrphA+vO/+7IqT3+5pzOemP3aHwZWbADL8KC0zeBhr2uLHbNt9dpK/Mupz87ri9/f1hHXTe6Mlxe+9oe8y/D4iv4khasO33FUHLvHqFhjXJa60ggQIECAAAECBAgQqEqB3//+9/GJT3xywNj22GP3bKvAXw7rlm1/+cs1kbZMS4GhwVoKb73zne/Mqmy9Y9BxpJDSvvvuN+CnDz74QNx9993Z1mufL2xz2P+GnXfeOX76058UQjbf//4PcmNIgZjilsJA5513XhaU+neIq//3qdpTqvqUb6ed9v543/veF9/4xjfjwgsvzMJYA/tL28B94QtfiKOPPir3kxkzZmT3vz8efvjhfBeFzxTMSmGmgw8+uHCt+CAFun7845/EzTffXPxV4TwFtz72sY8OGjIr3FTCwcKFC2O33XaP9Jlv3//+9+KQQw7Jny71M1W1OvDAAwfct9pqq8XVV/8p0udwtfvvvz/bJvO7kbbJS+Gk4pbCVm9605viQx86MxeYKv4+nR9yyKExffr0wle/+tUvs8DUhrn1vO666wvX8wdrrrlmVu3tglx4Ll1Lz/7kJz+VBeKez99S+EwBu//93//NbaNYuPj/D+bOnRt77bX3gMs//vGP4oADDihcW9y/Ayn0d+qpp2bv1yOFe/MH6d+rb37zG7kAYv5a/8/DDz8iFxbMX3vXu96V/f+Hj+dPfRIgQIBAgwgIXDXIQpvmiAoIXI0ov4cTIECAQL0LCFzV+wqbX7kE0h+Z/nVKZxa86o5XOxb9A9RyPaec/Wy9brY1xwHtMW5M02K3GVzY1Rd/vacrfnjtwuxvl9bGvJJRCpJttV5rfOb4FWKVFVW0Kud7oy8CBAgQIECAAAECwyFw2mkfiL/+9a+FrlN1nnSeKuIMR0tblX31q1+NX/zilyV1n4JPX/vaVyOFlvq3wcImF110YbzrXe+OVIFqcW2//faLH/3oh/H1r38jF4xZ3H3peqp2ddVVVxYqGuXvLQ5cvfe9782FVK699tr8LYt8trW15Z635pprxDvecVKuqtUiN/W7kEIxRx55ZL8r/z78zne+E9/73vcXub64Cx/4wGlx+umnL+7rpV5Pla1Shat8S5W8Jk26fZH1yH8/2OevfvWrLLz2xQFffeMbX89tMTjgYhlPrrrqqmzrwc/kqpYtrdvNN988fvjD82P99ddf5NbiwNX//u8349vf/nZW1eq1EFbxj1Ko6Xe/+7+s+tU9ccYZZ0Z3d3fxLYXztF3nH/94VaSqZP3bsgaurrji9/Ge95wSc+bM6d/dgONUOe78838wILyVv0HgKi/hkwABAo0tIHDV2Otv9pURELiqjLOnECBAgECDCghcNejCm/YyCaQ40px5vfHXe7vi5ge7s7+5ukzdVPRHK7Q3xe6btcZRO4+KFf6980Th+X+d3BWX3toRM+b2RL+dCwrfV+vBFlmQ7OTXj47tNmiN1hIreFXrXIyLAAECBAgQIECAQKMI7LDDjvHqq68WprvVVlvFlVf+oXBe7oNzzjknLrro4iF1m6oQfeUrXx7wm8ECVykstqRwS76DTTbZZEAVn/z1wT7PO+8L8Z//+Z8DvioOXJX63LFjx8aoUaMihWmW1tZdd9249tprIgW18i1V5Ephn8FaCtGkMFtxa25uzrwvyioa7Vj8VUnnxXNNgbVUJWwo7ZRTTs1Veur/mzvuuGPQ6mH971nW46uv/nOceeaZQ/p5eif+8IcrFtnirzhwVepapypYaUu/wdakeGBHHXVUVh3t6wMuL2vgqtTxbbrpprmgV/G2nQJXA5bBCQECBBpWQOCqYZfexCsoIHBVQWyPIkCAAIHGExC4arw1N+PyCDwwqyeuvLN2thmcMLYpTtijPTZbqzkef7Y3fnXjwpgybfF/+7U8SuXrpSkrYrXe6s1xzG7t8R+7jCpfx3oiQIAAAQIECBAgQGDYBebNm7/IlnPHHntsVoHqK4t9dqqc8/LLLy/2++IvUuWefKgjbcH2/ve/f8AtW265ZZx44omx//7757Z1u//++7JQ0XdyWwP2v/EnP/lx7p78tcECV/nvVl999dyWfK3Z3wS5/vq/xTPPPJP/apHPtIXfgQceEGPHjsvuvS7S9nf9W6p+9Kc//bH/pSgOIeW/TOGmnXbaKRduStu53ZhtJ7e41pT9x9TEiRNj9913z6olPZEbZ/G2d6ma1eGHH1boItmffvoZcdddd+Wu7brrrvG2t70tdtll50hzfuqpp+I3v7kwUkWp/kGfLbbYIheuKXRU4kEazz777JNVS3otIHbuuefk1qvELnK3HXXU0fHQQw8VfpKqp91www2F8+KD+fPnx3PPPVd8ebHna6yxRmGbyLR+Rx99dLz00kuF+1ddddV45ztPyr0/G2+8cTz22GPx61//JtJ2mv3be97z7mwbxo/1v7TIloL5L1NlqhQ+W2+99XIVv+677/78V4t8ppDdHnvskW0zuEV276SYMmXKgHvSe3PLLTfn1jD/xbIGrvK/T8/ca6+9su0KN4t77703brvt9vxXhc9vfetb8YY3HFE4TwcCVwM4nBAgQKBhBQSuGnbpTbyCAgJXFcT2KAIECBBoPAGBq8ZbczMun0Bnlle6JwstXXhzZ3RkW/NVexuV/YXldVZtjrse7o6XFvRW+3AL48v+AnWcfMDoeEMWtEoVuzQCBAgQIECAAAECBGpLIAVPjjjiDQMG/elPfzoXThlwsd/JmWd+KK6++up+V5Z8mLY422abbbJKxH1x6KGHxfTpr23FlsIvv/3tJbHKKqsM6KS3tzc+8IEPxnXXXVe4vu+++wzYAnBxgau0BeG3v/2tSCGW1FI4LAV+Zs2aVegrf3DggQfGD37w/UIg7JVXXslt49c/dJUqTE2ZMrlwT/rtYIGrdN8ll1wc2223Xb77LNTz6/jCF84rnOcPUhWiX//6V7HzzjvnL+VCUsXb7p1xxhlx2mkDA2qpgteXv/yVXOWrj33sowPGle/shz/8YXzzm/+bP822sm/Ktre7e0jbAKYfp2DQ8cefUOgnHdx0099jrbXWGnBtaSd77rnXgADVwQcflLn/YLE/u/jii+Nznztnsd8Xf/GpT30qTj75nbnLn/nMZ7Lt/C4t3JK2okzW22+/feFa/uDnP/9FZvla5bSVV145/vnPW3NVyPL3FFe4StfT1oOXX355oUJXel9TRa2//OWa/M8Kn2l7wcsvvyxWW2213LX070EKzV1zzcB703aYu+yyS+F3yxO4Sv++/eY3v84FGPMdDrat46mnnhIf+chH8rfkPgWuBnA4IUCAQMMKCFw17NKbeAUFBK4qiO1RBAgQINB4AgJXjbfmZlx+gdnz+uK6ezvj5oe6I/vzz6pr7W1NscGE5thgfEs8OacnHn6qJ+a/2hcvvdIT3YvuBFE140/hqtdv1xbH7t4e62bVrTQCBAgQIECAAAECBGpT4NFHH80q3PzHgMGfddZn4h3veMeAa/1Phhq4+vGPfxQHHHBArsJO/35T1avrrvtrpK3zBmupwtHee++T/bfcv/9jLt1/6623RKpWlNpggasUBPrjH6/KqlWNHdDlV77y1fjZz3424Fq696qrrsxCM+MGXP/Sl74Uv/jFLwdcu+GGvw0Y52CBq0984uPxrne9a8DvUgWxVNmof7WpdMPHP/6xePe73z3g3hdffDFSMKn/vUurNjagg34n8+bNyyqX7dbvSuQCQhMnbjvg2tJOvvWtbw8IRm277bZZVajLl/azRb7fY4894/nnny9cP+SQQ+L73/9e4bz4YKiBq+Se/FNgbq+99o4FCxYUuvyf//mfeOMbjy6cFx8ceeRR8fDDDxcupwDewQcfXDgvDlylIF8KCb7uda8r3JMOUsWuU09974Br6Z29+OKLYocddhhwfbBKb2nLzLR1Zr4ta+Bq3LixccUVVwx4X1OfKeiVwpWPP/54/hHZ+eGLbFEpcFXgcUCAAIGGFhC4aujlN/kKCQhcVQjaYwgQIECgMQUErhpz3c16eARmz+uNH13XEU+90Jv9IePwPGMovbZkGaXx2VaCEzdoi9aWf1eGeuyZ7nj06X+nrHqy/z1hbjbmhR3VMd783FqzcW+yVmt87Jgxsf54Qau8i08CBAgQIECAAAECtSowWDDnuOOOy6r+fGmxUxpq4OpnP7sgty3dYKGnE044frHPSV9cffWfcyGa/E0XXvibwhaIgwWu3vKWt8TnP39u/vbCZ9qC78QT31o4TwdvfeuJcc455wy4lk5uu+22LHB20oDrKWSUwkb5Nljg6pZbbokJE8bnbyl8nnTSO7OqSf8snKeDVEUpbQFY3N761rfFnXfeWbj8+te/Pn74w/ML5/mDVOXq+uuvz20Vd++998UjjzySq3iVqiilbev23nvvOPvsswcEj775zW/kqnfl+yjls3grwA9+8IPxwQ9+oJSfDrinONSUtuH729+uH3BP/5OhBq5OPfXUrFLThwcNPR166KGFSlT9n5E/vuOOO7MtHaflT7PqU6dn1dVOK5wXB65SVbZrrvlL4fv8QUdHR+y22+4DzDfddNP4858XrQaXAmHp3vSbfPvUpz6ZVek6OX8ayxq4OuaYN8bXvva1Qj/9D4r/3U1BsP/7v9/2v8WWggM0nBAgQKBxBQSuGnftzbxyAgJXlbP2JAIECBBoQAGBqwZcdFMeVoEFnX0x5cneuOy2jpi/YGRSVylaNWFccxZaaolxKwwMLD36dHc89szAslYd2ZhfeCkLXmWfI922WKc1TjqwPV63UWsWEhvp0Xg+AQIECBAgQIAAAQLlEnjd63YYEBJJ25GlbQAX1376059moaC7Bv06bd83adKkAd9deeUfYquttsqCOqcvso3agBtLOOkfGhoscHX22WfF29/+9kV6mjlzZrz+9QcNuJ7CVil0VdxS+CZtfdi/LS1wlapk3XHHwHnnf//xj38iV3Eof56CVilwNVj78Ic/klXo+mPhq8ECV3PmzM0sPxj/+te/CveVcvC1r301jjnmmFJuzd2TtlVMWy72b8UO/b9b0vF73nNK/P3vfx9wy1133RlpC7/B2k03/SMuuuiiwb7KXbvpppsihc7yLW0jeNJJ78i20ftNFrj7Qv7yMn2++c1vzraB/Hzht8WBqxTg+t73vlv4vv/BAQccGE899VTh0hvecER861vfKpz3P0jV2+bMmVO4VK7AVd6i0HG/g89+9nPZtpeXFK6kbRYvvfR3hfN0oMLVAA4nBAgQaFgBgauGXXoTr6CAwFUFsT2KAAECBBpPQOCq8dbcjCsj8HIWtvr9HZ1x52Pd0fHan88O68ObsqTVymOactWh1swCV4O1R57qjsefHRi4SvelilwvZdsMzh+hbQbXWa05/mOX9jhql7YY1ZoiYxoBAgQIECBAgAABAvUk8L73vS+rlvS3wpTa2tpy1YfWXHPNwrVSDy688KI499xzC7c3Zf8xdPvtt8Uqq6ySqzCVKk0tT/viF78Y+apYgwWufvKTH8f++++/yCMGC1z94hc/z7af22uRe5clcLXddtvFZZddukhf6UJx4GqnnXbKQi8XD3rv0gJXU6dOjVNOOTVmz5496O+XdHGogavi8FLagvGmmwaGppb0vP7f/fznv8iqpn25/6XcVnZpS7uhtsG2wUzbE6ZtCou3QBxq3+n+tP1g2oYw34oDV2kryLQl5GCtOHCV/t360IfOHOzW3HaZwxG4SoHI/fbbd9BnClwNyuIiAQIECAwiIHA1CIpLBMosIHBVZlDdESBAgACB/gICV/01HBMor0BvFmKan4WYzv/rwpg+J9u/bxhbqga19fptsVYWtGoePGuVe/rDWeDqiUECV/mh9WbDfOHl3njpleEdb/55bVm46sR92uPYPUbFmFGCVnkXnwQIECBAgAABAgTqTeDSSy+LT3/60wOmte+++8QFF1ww4FopJ8cff0JMmTKlcOvEidvG5Zdfnjv/wAc+GNdee23hu3Qw1O3pDj/88Nh8881zfQwWuMpvXzjgIdnJYIGrX//617H77rsV3xpPPPFEHHbYwCBQcWWn4i0Fd9xxx/jtb1+rHNS/0+LA1W677ZZVYvp1/1sKxx/60IfjT3/6U+G8uMJVsW9LS0scf/zxscceu0faTjBtQ/fYY49n29j9OTePQkfZwVADVyef/N+RtknMtxNPPDEL052TPx3S54wZM+Kggw4e8JsJEyZkW0b+Kdvub9yA60s7SWGon/zkp4XbkkEK9Y0dOzbSmn7hC+cVvksHaXvAFPwrtaXw3AEHHFC4vThwld++sHBDv4PiwFV6v1M1ssFa2vYxVSvLt3JVuFrcvwPpOcWBq8GCgipc5VfEJwECBBpbQOCqsdff7CsjIHBVGWdPIUCAAIEGFRC4atCFN+2KCnRmFa4mPdod10zujDnzy7ttX6oGtf745lhvfEuMblv6H+4+NKs7ps1etMJVMUjaZnBeFrx6taO8480/p6WlKQ55XVu8cddRsfGaWVpMI0CAAAECBAgQIECgrgUWLFiQVfU5JlJlp/4tbauWtlcrtX3/+z/IVS3qf//pp5+eC7yka1/60pfiF7/4ZeHrNdZYI26++R+F86EeNFrgatq06dlWh4cOYPqv//qvLERz9oBr6WTevHmx2267ZxWTX/vvxqEErtLWkLvvvkd0dXUV+v7pT3+SVU7ar3A+1IOPfvSjceWVVw342ZFHHhlpm8hS22233RYpCNbT89p/O6ew2a9+9atcF9ddd328//3vH9BdCvlttNGGA64N5UTg6l3xiU98fChk7iVAgACBOhAQuKqDRTSFqhcQuKr6JTJAAgQIEKhlAYGrWl49Y681gZ7sz6Avubkjbs/CVymEtTytOctWrb5yc2y7QWu0lxC0yj/rwSxwNb2EwFX+/pcX9sUL83qjJ5XrKkNryapvbbpWS3zqTWNi7dUErcpAqgsCBAgQIECAAAECNSMwefLkeMtbThwQZGltbc22rzslC7C8L0aNGrXYufRm5Xgvvvji+PznvzAg4LPuuutm1Zr+GCussELutzfeeGNuO7z+HX3rW9+KN7zhiP6XCsfd3d1ZVaavxb777pf9s0/hev6g0QJXN930j0jb2fVvf/jDFbH11lv3v5Q7Hix4NJTA1Z///Jc444wzCv2OGTMm7rhj0hLfg8LNizmYP39+HHnkUfHMM88MuOOwww7LQmOfjQkTxg+4XnwyadKkOO20D+TCZPnv0jt66aW/i2222SZ36cUXX4x99tk3Ojs787fEW996YpxzzjmF8+KDX/7yVzF6dHu86U1virSdZnETuBK4Kn4nnBMgQKARBASuGmGVzXGkBQSuRnoFPJ8AAQIE6lpA4Kqul9fkqlAg5ZZmPdcbV9zRGffPfO1vyw5lqClotdnaLTFuheZsy4Kh/DLigZnd8eScoT23J9td8OVXe7KKV32xPLmrVInrPYeMie03zKpx2T5waAvnbgIECBAgQIAAAQJ1IvDtb387UpWq4rbJJpvEhz50Zmy//fax9tprF76eN29+TJ58T1ah6H9j6tSphevpIG3hlqoO9d+yL1UlOvDA1w8I3KTAzA9+8P0BW7il36d7zzjjzMIWhOnZp532/uz3B6avc63RAld33nlnFh56W376uc8USDvmmDcOuJaCTW9723/FQw89NOD6UAJXH//4x+OKK/5Q+P0hhxySvRvfK5wv68E///nPeOc7Tx4QzEt9jRs3Nhfw2nPPPbNqVBtF2iYwtYULF8ajjz4aP/rRj+Oaa67JXev/f9J2fcXbUn74wx+JP/7xj/1vi/e+973x4Q9/aMC1dPLzn/88vvzlr+Sup3f7lFPeEyeccMKAYJnAlcDVIi+OCwQIEGgAAYGrBlhkUxxxAYGrEV8CAyBAgACBehYQuKrn1TW3ahZIOy788+HuuHZyVzyTVZAqpa08pik2WqMl1l61ZchBq3z/yxK4yv+2szurdpVtibiwszf7g+v81aV/rrlKc7xh51Fx3J7t0ZpVuNIIECBAgAABAgQIEGhcgRRySmGT++67f7EI48aNiw022CCefvrpmDt37mLve/vb3x5nn33WIt///ve/z7Yn++SA6ylcM3HixEhbw40fPyFSJaNJk27PKhnNH3DfwQcflIWzXguENVrgKlVv2muvvSNV/sq3jTfeOD760Y9kVZ32iVRpLIWyUmjunnvuyfx4suAAAEAASURBVN9S+Cw1cJXegz333CvS8/Lty1/+Uhx33HH50+X6/OIXvxipqtTi2ujRo2PTTTfNVbJ66qmncvMa7N5tt902V90qH87K35MCWscdd3ykrTL7t2S1xx57xJZbbpm94/dGCn/NmvVU/1ti3XXXiT/84Q8xduzYwnWBK4GrwsvggAABAg0kIHDVQIttqiMmIHA1YvQeTIAAAQKNICBw1QirbI7VLNDR1RfXTOmKq//Vtdhhpm34tlq3NbcFXzpenjZ1RnfMmDu0Clf9n5dyVh3ZUOe+0B3dJXRz/F7t8dZ922NMe1MMsRhX/8c6JkCAAAECBAgQIECgjgRSyOkrX/lKXHbZZcs8qxSM+sY3vhFpG7rBWgpcpeDVUNo666QgzBVZJaRxhZ81WuAqTby48lQeo729PRfESmGpxbVSA1fFlbSam5vj1ltvidVWW21xXQ/pehpjqiz1ne98N1fBakg//v83b7HF5vHd7343UohqsHb55ZfHJz/5qcG+Wuy1VG3twgt/EzvuuOOAewSuBK4GvBBOCBAg0CACAlcNstCmOaICAlcjyu/hBAgQIFDvAgJX9b7C5lcrAtOzbf7+ck9X3D3ttT+4bs4SSutPaIkNsn9WKNMWfPdngauZyxG4yntmf6k5Xnq1N+a/0htpy8H+ra2lKfbbtjXetEd7bLrWv7do6P+9YwIECBAgQIAAAQIECCSBW2+9NatQ9dmYMWNGySApFHX22WfHQQe9fom/SRWavvrVry6xylH/DlIlo7TtYP/tDNP3jRi4SlWb0jZ6N930j/5EixyffPLJcfHFFw8INJUauPrqV78WF1xwQaHPnXbaMS655JLCebkOnnzyyTjrrLPitttuL7nLFOL7wAc+EP/93ycXth1c3I9TqO9znztngMHi7l111VWzANd3YrfddlvkFoErgatFXgoXCBAg0AACAlcNsMimOOICAlcjvgQGQIAAAQL1LCBwVc+ra261KHDvjJ648B8d2ZaBTbHDxq0xqrW8daHue7I7Zj33WqhreY16evti7ov/3mYwBcTWGNccnzpuhdhiHUGr5bX1ewIECBAgQIAAAQKNIJDCPeef/8O46qqrsq3XZi12yquvvnq86U3H5oIwi6tqNdiPb7jhhvjpTy+IO+64Y7CvY4011oi3vOUt8a53/feg1bIaMXCVoFJg7ayzzs6tS1fXwIrM48ePj9NP/2DObYcddoxXX321YFtq4Oqwww6PJ554ovC7j3zkI3HqqacUzst9cOmll8VFF10Y998/Nfr6Uu3mRVt6rw488ID42Mc+ntv2b9E7Br/yyCOPZGG98+Paa6+NYqv0i9Tv0UcfHe9973sX26/AlcDV4G+XqwQIEKhvAYGr+l5fs6sOAYGr6lgHoyBAgACBOhUQuKrThTWtmhV4/pW+mPRoT7zSGZECTOVu5Q5cpfGlP6tOWyMeseOoOHa3UVlIrNyj1h8BAgQIECBAgAABAo0gkIIrDz74YMydOzfStoNjx66c22Ju6623ji222CL3F1OW1WHatGkxdeoD8dRTs6KjozOrZLVWFn5ZL3beeadI27xpgwt0dnbGfffdH/fcc3c0N7fEJptsHLvuuuug4bTBe6iuq7Nnz46777475syZE88991yMHj0m945ttNGGscMOO0RbW9syD/j555/P+r4n9469+OKLMWHChFzFtLR94NixY5e5Xz8kQIAAAQL1KiBwVa8ra17VJCBwVU2rYSwECBAgUHcCAld1t6QmVKMCLy3si8ee7Y1pc/pi8L9rW56J3Tu9O556vnwVrvqPKgWtDtm+Lfbdui1WWWEY0mL9H+aYAAECBAgQIECAAAECBAgQIECAAAECBGpWQOCqZpfOwGtIQOCqhhbLUAkQIECg9gQErmpvzYy4vgRSdaj7Z/bG43N6o6d3+Od27/SuLHA1vA9asT3izXu3x66btobY1fCvqScQIECAAAECBAgQIECAAAECBAgQIECg1gQErmptxYy3FgUErmpx1YyZAAECBGpGQOCqZpbKQOtMoCsrMvXk3N54bHZfvNIxnDWtBsJNyQJXTw9z4Co9sbk5Ytv1WuLwHdpi0zVbBg7CGQECBAgQIECAAAECBAgQIECAAAECBAg0tIDAVUMvv8lXSEDgqkLQHkOAAAECjSkgcNWY627WIyfQm2WrXny1N+58vLJBq/yMp0zrjqdfGJ4tBfPP6P/ZnJW4Oni7tjgsC16t2K7eVX8bxwQIECBAgAABAgQIECBAgAABAgQIEGhUAYGrRl15866kgMBVJbU9iwABAgQaTkDgquGW3IRHUOC5l/vigVm9MTf7TFsJjkSb/ER3PPNi5QJX+TlOGNsUh24/KvbesjVX/Sp/3ScBAgQIECBAgAABAgQIECBAgAABAgQINJ6AwFXjrbkZV15A4Kry5p5IgAABAg0kIHDVQIttqiMmMH9BXzw+uzemz+2LVOFqJNs9WeDq2REIXKU5p/pW26zfEkftPCo2mpDtOagRIECAAAECBAgQIECAAAECBAgQIECAQEMKCFw15LKbdIUFBK4qDO5xBAgQINBYAgJXjbXeZltZgZ7eiPuyilbT5/RGOq6Gds8TXVngamQH09YSsfMmrXHiPu3R3loNKsZAgAABAgQIECBAgAABAgQIECBAgAABApUUELiqpLZnNaqAwFWjrrx5EyBAgEBFBASuKsLsIQ0m0J3lmaZlIasnsqpWL3dU1+TveTwLXM0b2cBVXmTNVZrjkO3aYo/NW6M1C2FpBAgQIECAAAECBAgQIECAAAECBAgQINAYAgJXjbHOZjmyAgJXI+vv6QQIECBQ5wICV3W+wKZXUYG0XeDcl3rjX9P6YkHnCO8duJiZ350FrmZXSeAqP8S0veBJ+7fHWqs257YdzF/3SYAAAQIECBAgQIAAAQIECBAgQIAAAQL1KSBwVZ/ralbVJSBwVV3rYTQECBAgUGcCAld1tqCmM2ICz7/cF/fN7I0XXumLFLyq1nb3Y1ngan51VLjqb7TCqKZ43UYtcdzuo2Kl0U39v3JMgAABAgQIECBAgAABAgQIECBAgAABAnUm8Plzz42pU++PiRMnxnnnnVdnszMdAtUhIHBVHetgFAQIECBQpwICV3W6sKZVMYH5C/risWzrwOlz+qKKc1YFj39lgas5VRi4yg9wtZWa4phdR8UOG7XGqNb8VZ8ECBAgQIAAAQIECBAgQIAAAQIECBAgUE8CAlf1tJrmUq0CAlfVujLGRYAAAQJ1ISBwVRfLaBIjJDD5ySxoNbc3eqqvYNRiRe7KAldzqzhwlQbenBW4mjCuOd57cHusnW0zqBEgQIAAAQIECBAgQIAAAQIECBAgQIBAfQkIXNXXeppNdQoIXFXnuhgVAQIECNSJgMBVnSykaVRMoLunL6bNzapaPdsbr3ZW7LFle9Bdj2aBq5dqIyE2JttmcM/NW+OQ17XFqivaZrBsL4GOCBAgQIAAAQIECBAgQIAAAQIECBAgMMICAlcjvAAe3xACAlcNscwmSYAAAQIjJSBwNVLynltrAqmK1Yuv9sUdj/XEgq5aG/1r470zC1w9VyOBq/yoV8iCV2/bd1Rsv2FrtLXkr/okQIAAAQIECBAgQIAAAQIECBAgQIAAgVoVELiq1ZUz7loSELiqpdUyVgIECBCoOQGBq5pbMgMeAYHnXu6LqbN6I3329Y3AAMr4yDuywNXzNRa4StNvyXYW3GB8cxy/R3tsuqZtBsv4SuiKAAECBAgQIECAAAECBAgQIECAAAECFRcQuKo4uQc2oIDAVQMuuikTIECAQOUEBK4qZ+1JtSWQclUvLeiLR5/tiyfn9kaN56wK+JMe6YoXXq6NLQULg+53kCpcHbBtWxyY/bPaSrYZ7EfjkAABAgQIECBAgAABAgQIECBAgAABAjUjIHBVM0tloDUsIHBVw4tn6AQIECBQ/QICV9W/RkZYeYG0feB9M3ti+ty+SMf11Go9cJVfi9FtESfsMSr22rItmuSu8iw+CRAgQIAAAQIECBAgQIAAAQIECBAgUBMCAlc1sUwGWeMCAlc1voCGT4AAAQLVLSBwVd3rY3SVFejqiZiWhayemN0Tr3RU9tmVelq9BK6SV3MWtNpynZY4fIe23GelDD2HAAECBAgQIECAAAECBAgQIECAAAECBJZPQOBq+fz8mkApAgJXpSi5hwABAgQILKOAwNUywvlZXQn0ZvsFzn2pL+6e3hOv1ljQqqU5hlSF6/aHu+LFV0or25UKR9XCVoqpwtV+W7XGkTuPipXHKHdVV/9ymgwBAgQIECBAgAABAgQIECBAgAABAnUpIHBVl8tqUlUmIHBVZQtiOAQIECBQXwICV/W1nmYzdIEUtLp/Zm+8+GpfpOBVrbQUtNp8reYY1Rox5cnSAlRpbrc93BnzXiltopuv3ZLr//4ZWemvGmirrdQUB01siwO2bYvkoxEgQIAAAQIECBAgQIAAAQIECBAgQIBAdQoIXFXnuhhVfQkIXNXXepoNAQIECFSZgMBVlS2I4VRMYN6Cvnjs2b6Y8VxvTQWt0jZ6a63SHFut0xTjsmpOT73QF7c/Vnog6raHssBVFi4rpe2+eWucfEB73Ppwd1wzuSuefbH0YFcp/Q/XPVus3RxHZ9WuNssCYxoBAgQIECBAgAABAgQIECBAgAABAgQIVJ+AwFX1rYkR1Z+AwFX9rakZESBAgEAVCQhcVdFiGEpFBHqyzNCUGb3xZApa1UZ+qOCy0uim2HPzllhxVETaRi+1WVngatIQAlf/zAJX80sMXO21ZWu8Y7/23HM6uyOuvLMzrru3K3de7f8nVbjafoOWOGn/9hg9yjaD1b5exkeAAAECBAgQIECAAAECBAgQIECAQGMJCFw11nqb7cgICFyNjLunEiBAgECDCAhcNchCm2akoNXjs3vj8Tl98WpHaRWeqoVtxfam2GKtplhv9eZoLdoqb+bzfXHH46VXuLo1C1y9VGLgap+tWuO/9v134Cpv8fizvfHnuzvjvpk90VcDjOPHNsXrsy0G99u6LVoVvMovo08CBAgQIECAAAECBAgQIECAAAECBAiMqIDA1Yjye3iDCAhcNchCmyYBAgQIjIyAwNXIuHtq5QRS0Oq5l/viziyU1JFVaaql1pYFhDaa0BzbrttcqGhVPP4hB64ezAJX2XaKpbT9t2mNE/ceGLjK/+62R7rjsts74+Wsr9J6y/9yZD7XWbU53nnAqFh/9ZbFWo7MyDyVAAECBAgQIECAAAECBAgQIECAAAECjScgcNV4a27GlRcQuKq8uScSIECAQAMJCFw10GI34FSfe6kv7p/VGy+80he9tZAK6rdGG45vzlW1StsILqnNyLZGvPOJ0vdGvHUIgavXT2yN/9xz8MBVGtO8rFLW9dkWg3/N/qmFalej2yK2Xb8l3rxXe4wds2TXJZn7jgABAgQIECBAgAABAgQIECBAgAABAgSWT+Dz554TU6dOjYkTJ8Z55523fJ35NQECgwoIXA3K4iIBAgQIECiPgMBVeRz1Uj0CKfgzP6u69NizfTE9CyPVUmvOMkCrr9QUW63THONXLi0Q9OTc3rhrWunzvOWBrCrVwtLSZwdv1xbH7zFqqYRPzO6JK+7oikee6Yne0oey1H6H64ZxKzTF0Tu3xS6btkZ7W2nOwzUW/RIgQIAAAQIECBAgQIAAAQIECBAgQKARBQSuGnHVzbnSAgJXlRb3PAIECBBoKAGBq4Za7rqfbApbTZ7RG9OzEFItBH/6L0hLc8Tum7bEGmObhrTl3VADVzdngatXSgxcHfq6tnjTbksPXKV5pK0b75nWHT/9W0dNVLtqynJWq6zYFO8/dHS2zWCGrxEgQIAAAQIECBAgQIAAAQIECBAgQIBAxQQEripG7UENLCBw1cCLb+oECBAgMPwCAlfDb+wJwy/Q2R0xbU5vPDGnL17tLK160/CPqrQnpG3uNl2jOTZeoynaWoZebSmFy/41hApXQwlcHbFDW7xx19ICV/nZvpht33jtlK646YGu6O7JX63ez1Thatcs6HZ4NtfxKwteVe9KGRkBAgQIECBAgAABAgQIECBAgAABAvUkIHBVT6tpLtUqIHBVrStjXAQIECBQFwICV3WxjA07iVRV6fks4POvJ3prLmjVmmV71hjXHDtu1ByjWpZ9Cadlgau7hxK4mppVuOooLZT2Hzu1xVE7Dy1wlZ/JrOd742dZtaun59VGtbFRrRFv2as922awJUa1Dj34lp+3TwIECBAgQIAAAQIECBAgQIAAAQIECBBYuoDA1dKN3EFgeQUErpZX0O8JECBAgMASBASuloDjq6oWmD2/Lx6Y1RsvvNpXE1vY9cdcc1xTbL1OS7alXcTyRntSVa97ppdeSuofWeDq1RIDV0fv0hZv2HHZAldpvh1dWRhuWk9cfHNHpCpk1d6asxDcuqs2x/F7jIots/XRCBAgQIAAAQIECBAgQIAAAQIECBAgQGB4BASuhsdVrwT6Cwhc9ddwTIAAAQIEyiwgcFVmUN0Nu8C8LGD16LO9MfP5vugtrVDTsI+plAekYNW4FZpi87WaYr3Vyrd13eOz+2Lyk0MIXN2fBa5K3Hbx2Gw7wcOyrfaWt72ysC8uvb0zq0TWnYWwlre34f99S7Y8+27VGgdt1xYTxpZvrYZ/5J5AgAABAgQIECBAgAABAgQIECBAgACB2hAQuKqNdTLK2hYQuKrt9TN6AgQIEKhyAYGrKl8gwysIdGfbB07OKjnVWtAqTaAlS1ttv0FLbDC+KZqXt6RVQeTfB49lgaspQwhc3ZQFrhaUGLg6bvdRccj2yx+4SiPty8Jxz77YG9/9y8J47uXaSMq1ZdsMptDZgdu2RVOZ161oGZ0SIECAAAECBAgQIECAAAECBAgQIECgoQQErhpquU12hAQErkYI3mMJECBAoDEEBK4aY51reZYpaPV4VtHqibm92VZ4tTWTtmxXuo0mNMfG2T8rtg/P2B99ti/unVF6hau/39cZC7Ot/kppJ2Rb66UqT+Vsnd19cfOD3XH13V3xclb5qtpbClpttmZLHJ5V+tp2fdsMVvt6GR8BAgQIECBAgAABAgQIECBAgAABArUhIHBVG+tklLUtIHBV2+tn9AQIECBQ5QICV1W+QA08vJ4saDV7fl/cPb0324au+oM5/ZcqVbFabaWm2HHDllhpdP9vyn/8yDO9cd/MDKvENpTA1Vv2ao8Dts3KPA1D68wyYj//28Js7D3R1T0MDxiGLvfYojWO2WVUrLKiclfDwKtLAgQIECBAgAABAgQIECBAgAABAgQaSEDgqoEW21RHTEDgasToPZgAAQIEGkFA4KoRVrn25jj3pb5ciGjeq33RW1tZq1h1hYiJ2faBq2WhnHJvHzjYSj78dG/cP6v0wNWNWYWrUgNsb91nVOy3dXkrXPWfQwrVPf5sT1x8S2c89ULpc+jfR6WPU9hqv23a4rBsq8WW5ko/3fMIECBAgAABAgQIECBAgAABAgQIECBQHwICV/WxjmZR3QICV9W9PkZHgAABAjUuIHBV4wtYR8NPuaoUsEpb5M14rjbCN/35VxrdlNs6cOMJTRUN4jyUBa6mDlPg6u37tsfeWw1Phav+dr3Zcl8zuStunNqVewf6f1etxxuv0RxHZ9Wutlq3JdS7qtZVMi4CBAgQIECAAAECBAgQIECAAAECBKpVQOCqWlfGuOpJQOCqnlbTXAgQIECg6gQErqpuSRpyQClsdc+03ngyC1rVWkWrpixts8VazbHVOs0VqWhV/II8+FRvPJD9U2q74d6O6CxxC7937t8eaRu9SrVUeetnN3TE5OnZfoM10JqzCldbrdMS7359e6zQLnZVA0tmiAQIECBAgAABAgQIECBAgAABAgQIVImAwFWVLIRh1LWAwFVdL6/JESBAgMBICwhcjfQKNPbzu7JczeOze+OJ2X2xIAvb1FrbcPWm2HTNlhiXbSM4Ui2FrVLoqtT2tyxw1VVi4Oq/D2yP3TarXOAqzaEvew3undETf/xXZzw5p/R5lTr/4bhvtZWaYv9sm8HXT2yLtpbheII+CRAgQIAAAQIECBAgQIAAAQIECBAgUF8CAlf1tZ5mU50CAlfVuS5GRYAAAQJ1IiBwVScLWaPTmDqrJx56uraCVqmi1bgxTbHjRs2xygojX9Xo/mw7wYezbQVLbX+bkgWuSiwg9e6D2mOXTSobuOo/jz9loavr7+uOVztq4x2ZMLYp3nlAe2y8RsuIVDvrb+eYAAECBAgQIECAAIHqF7j0d7+LqVPvr/6BGmFZBbbZZts4/oQTytqnzggQIECAQC0KCFzV4qoZc60JCFzV2ooZLwECBAjUlIDAVU0tV90N9v4scPVwDQWuxrRFvG7DlpiwclO0Vkklo/tnZoGrZ0oPXF2fBa66SwxcnXpIexYsG7nAVap2Neel3rjyzq6487ESy3KN8L8l7dk7smW2zeBb926PVVYc+UDeCHN4PAECBAgQIECAAAECixFIYatLL/3dYr51ud4Fjj/+BKGrel9k8yNAgACBpQoIXC2VyA0ElltA4Gq5CXVAgAABAgQWLyBwtXgb3wy/wH1ZWOiRIYSFhn9Egz8hhWg2Gt8cW6zdHK3Ng98zUlfvm5EZPlta4CrViUoVrkoNXL3v0NG5gNlIza3/c+9+oieuvrszZjxX2lz7/3Ykjlca3RT/te+o2GEEA2sjMW/PJECAAAECBAgQIECgNIH+gautt9mmtB+5q+YFHpg6NTcHgauaX0oTIECAAIEyCAhclQFRFwSWIiBwtRQgXxMgQIAAgeURELhaHj2/XV6Be7Ow0KMlhoWW91nL+vt1V22OnTauvqBVfj5DMcwFriZngasSM0unHTY6ttugSkp5ZRPuycZ93ZSu+P0dnfnpV/Xn/tu0xolZpSuNAAECBAgQIECAAAECxQL9A1cXXHBB9KUSv1rdC7z73e/OzVHgqu6X2gQJECBAoAQBgasSkNxCYDkFBK6WE9DPCRAgQIDAkgQErpak47vhFpiSBa4eq8LAVdoIbs1xTbH5Ws0xPts+sJpbV7Y94JIqVvVmf2bf1dMXLy349x/et2QVuhZ2LTqjru6+XKCp/zcbZFW9UqWmamszsypXf5ncGXc91hPV/D9JCFxV25tjPAQIECBAgAABAgSqR0DgqnrWopIjEbiqpLZnESBAgEC1CwhcVfsKGV89CAhc1cMqmgMBAgQIVK2AwFXVLk1DDGzyk73x+OwSyy1VQKQpyxaNbmuK7ddvjnVWrb6gUQUIauoR9z7ZHb+9tTOee7kv+9vg1Td0gavqWxMjIkCAAAECBAgQIFAtAgJX1bISlR2HwFVlvT2NAAECBKpbQOCqutfH6OpDQOCqPtbRLAgQIECgSgUErqp0YRpkWPdM740n5lRH4Kot2zlvm3WbY/3VmqOttUEWoA6m+fLCvrjlwe644s7OqgtdCVzVwQtmCgQIECBAgAABAgSGSUDgaphgq7xbgasqXyDDI0CAAIGKCghcVZTbwxpUQOCqQRfetAkQIECgMgICV5Vx9pTBBe7OAlfTRjhwNSoLWq2dVbPaZt2WrLrV4ON0tfoFnnmxNy6f1Bn3z+hZZGvEkRq9wNVIyXsuAQIECBAgQIAAgeoXELiq/jUajhEKXA2Hqj4JECBAoFYFBK5qdeWMu5YEBK5qabWMlQABAgRqTkDgquaWrK4G/K9pvTF97shVuFp1xabYbdOWWGFUXbE27GTStoIPP9MT51+zMBZ2jTyDwNXIr4ERECBAgAABAgQIEKhWAYGral2Z4R2XwNXw+uqdAAECBGpLQOCqttbLaGtTQOCqNtfNqAkQIECgRgQErmpkoep0mHc90RtPPlf5wNVqKzXFFms1x1qrNEVTndo28rReWtAXN9zfFddO6fp/7N13nJ1lnf//9ylTM5Mp6b2TDklISKGEQDBSQ9YAIkIwuoq6CquufsvPr35ZXdf9qisgrtgWFCFAkJIEpIWQBFIJKaRMeu+ZyUwmU0/53deBwRBmknNmTrnL634Yz8w5933d1/X8HP+Y4/t8LoXCmZMgcJU5e+6MAAIIIIAAAggggIDdBQhc2b1CqZkfgavUuDIqAggggIAzBQhcObNuzNpZAgSunFUvZosAAggg4DABAlcOK5jLprt6V1j7TlhtidJ0mC0DB1pBqwGd/fKTtEqTeuZuU3E6oj8sbNCuo5nZZpDAVeZqz50RQAABBBBAAAEEELC7AIEru1coNfMjcJUaV0ZFAAEEEHCmAIErZ9aNWTtLgMCVs+rFbBFAAAEEHCZA4MphBXPZdFftDGt/eeoDVwG/rI5WPvXtFJAJXXF4R6AhJG3eH9afl9Spui696yZwlV5v7oYAAggggAACCCCAgJMECFw5qVrJmyuBq+RZMhICCCCAgPMFCFw5v4aswP4CBK7sXyNmiAACCCDgYAECVw4ungumvnJHWAcqUhe4Ml2suhb5NKS7X0X53mtpFbZ2azxdH1VNvVTXGI1tr2e0jUt+ji8WPivM9SkYcMGb6TxLqG2Iat67jXqnrNGyOM/JSXqZwFWSIBkGAQQQQAABBBBAAAEXChC4cmFR41gSgas4kDgFAQQQQMAzAgSuPFNqFppBAQJXGcTn1ggggAAC7hcgcOX+Gtt5hSu2h3XwZOoCV1cPD6gwzyevRa1M0GrH0ai2H4moIWT5fvCfT7wVfBaM+de/o1/DevplOoG5+TDvtOq6qB54qU77T1hIKT4IXKUYmOERQAABBBBAAAEEEHCwAIErBxevDVMncNUGPC5FAAEEEHCdAIEr15WUBdlQgMCVDYvClBBAAAEE3CNA4Mo9tXTiSpZbgatDKQxcTb84GOvm5ESb1sw5FJa2Hg5r74moahsSGyHL6nLVs9SnC7r5lZ/t7oia2Wbw3Z0hq+NVg8qrUxf4I3CV2HuQsxFAAAEEEEAAAQQQ8JIAgSsvVfvvayVw9XcLfkIAAQQQQIDAFe8BBFIvQOAq9cbcAQEEEEDAwwIErjxcfBssfZkVuDqcwsDVzVbgynRw8sJRVRvVCmuLxuq6tq02N8unCQMDKs7/oPtV20az99WmE9j9z9boSIregwSu7F1/ZocAAggggAACCCCAQCYFCFxlUj9z9yZwlTl77owAAgggYD8BAlf2qwkzcp8AgSv31ZQVIYAAAgjYSIDAlY2K4cGpvLMtrCOVqeswNGNs0BOqx09FtHJnRPWNyVmu2VpwZC+/+nVy+R6DFte/PVervcdTs70ggavkvB8ZBQEEEEAAAQQQQAABNwoQuHJjVc+/JgJX5zfiDAQQQAAB7wgQuPJOrVlp5gQIXGXOnjsjgAACCHhAgMCVB4ps4yUSuGp7cY6diurtrWFFk5xb81udwUb18at3R7/c3CTsR8/Wan85gau2vxMZAQEEEEAAAQQQQAABBBIRSGbgqqGhQadOnVJ9fb0aGxuVk5Oj/Px8tW/fPpEppfzcSCSi2tpaVVdXq66uTsFgMDbXdu3aKS8vL+X3T+QGxtTM05ian7Ozs2OmBQUFCgQCiQz1sXMJXH2Mg18QQAABBDwuQODK428Alp8WAQJXaWHmJggggAACXhUgcOXVyttj3SkNXFkpoRnWloJuPhrD0qsbwmoIJTlt9SFa0GpwdeWwoApz3at4/9xaHawgcOXeCrMyBBBAAAEEEEAAAQTsKZCMwFVNTY1ee+01bS/bpBzfKRXn1is70KiK2hw1RHIVLOiuiy++WJdccknGETZv3qylS5equuKA2mXVqDinXvXhgE435KiqoZ169umvSZMmqW/fvhmdqwmCLVmyRBs3rFUgVKF22Q0qzGmImUZ9Oar1ddS4ceN04YUXqrCwMOG5ErhKmIwLEEAAAQRcLEDgysXFZWm2ESBwZZtSMBEEEEAAATcKELhyY1WdsybTmeloVWrCQj4rcHWziwNXYSsjtHx76vya3kUmbHXFkKCyXZpd++EztTp8ksBVU715RAABBBBAAAEEEEAAgfQItCVwdeLECb3+ynzt3LBIEwY06vILGhVspunSnuN+vb09S8ci/XXNdTN1weAhsa5S6VmhFAqFtG1rmRY897gKogd19fAGDer8yb+/zJeJlm0PasXOLA0Ze73GT5qszp07p2uasfucPHlSq1Ys1TsLX9DoXnW6fHCjSvI/+XnF/nK/Vu8OauepHrp86s26aNSYWPereCdL4CpeKc5DAAEEEPCCAIErL1SZNWZagMBVpivA/RFAAAEEXC1A4MrV5bX14swWeG9vC+tYigJXZku86S4OXB3/cCvByCc//0163Yd292uI9c+Nxw+ertWRyk9+4J+MtU62uoPdfmlOMoZiDAQQQAABBBBAAAEEEHCZQGsCV2ZLvhUrVmj98pc0quNujenboKw4/lTbZ4WE3t6Rr8YOV+r6669XaWlpyjVNKGzBggXKOvmOruhfqR4l5/+7q65R2nwwqLf399RFE2/QxIkTUx4QM6abNm3SGy/P1djOOzTWMs2J4wtH5aeNaa4O6WJde8N09erVKy5TAldxMXESAggggIBHBAhceaTQLDOjAgSuMsrPzRFAAAEE3C5A4MrtFbbv+ghcta02CzeFVVmThrSVNc2CXJ+uGdHM16XbtgRbXP39OTU6ZoXXUnEQuEqFKmMigAACCCCAAAIIIOAOgdYErt56a5E2LPmz/mlKdasQ3rU6M71zdKS+8vVvJdSVKdGb1dfX64+/+40mlK7U6D6hRC+Pnf/bJYUadtksXXbZ5a26Pt6LNm3aqBeefFjfvPqEtdVhvFf9/bxjVT49vmGoZn/lPhUVFf39hRZ+InDVAgxPI4AAAgh4UoDAlSfLzqLTLEDgKs3g3A4BBBBAwFsCBK68VW87rTYWuLK2FExV2CVgdbi6yaUdro5XR7Vki7XnQhqPSYP86lIUx1en0zinZNzqfz9ZoxOWZyoOAlepUGVMBBBAAAEEEEAAAQTcIZBo4GrLli1atuAB3TK6QgU5rfsbxvwdvmxHUBvrL9esu2enJHTV0NCgp+b8Rb0b39AV1rZ8Putv89Yc1fU+PfNeqcZM/Youuuii1gxx3mv27NmjF598SLeNOqDOhefvwNXSgBv2BfT20Yv0ubu/qvbt27d0Wux5Alfn5OFFBBBAAAGPCRC48ljBWW5GBAhcZYSdmyKAAAIIeEWAwJVXKm2/daY8cGVlg24aE8c+APajOe+MNuwLa/uR1n3Aft7BWzihJN+nK4e5r8vV/3yiRhWnU2NJ4KqFNxNPI4AAAggggAACCCCAgBIJXNXW1uonP/qhvnHZHnVq37a/X8JWruil9TnKG3yXpk6dmvRKLFmyRFvf+YNmTapVsI3f2amo8emhxT31j1/7trp165bUuZouXA888IDuGPqeupdG1MpcWGxOEaski7dlaVtosmbPnq1AoOW/nQlcJbWMDIYAAggg4HABAlcOLyDTd4QAgStHlIlJIoAAAgg4VYDAlVMr5/x5m8DVUqvD1fEUbedmPti90aWBq7ctt6NVbfuQPdF3kPlW8nUXBZXtsgzb9/5Sk7KtGQlcJfou43wEEEAAAQQQQAABBLwjEG/gKmr98TzvxefV/vhfdeWQhqQB/W7VQH3mrm+rtLQ0aWNWVFRo7p/+U7cN36r2ecn5m3XTwSxtDUzXTTd/Rn5/GxNcZ6x01coVqlj/W31qaOu2ZzxjqI9+/MPyrrrqH+5Tv379Pnru7B8IXJ0twu8IIIAAAl4WIHDl5eqz9nQJELhKlzT3QQABBBDwpACBK0+W3RaLTnngyvpC6Y2jXZYO+rByr78f0qm69JbRBK6mWB2uivLa8r3f9M45nrt99/EaVdUm5/8IOPt+BK7OFuF3BBBAAAEEEEAAAQQQaBKIN3BlOjHN/e9/08yhZcoJJu9vl5W7shUc8nWNGTuhaUptfly/fr2OrfqFrh5S2+axmgaoD/n05Hvd9bmv/XtSt0B89JGf6dreq9WljR3DmuZpHt8/ENTOLCscNuPWM5/+2M8Erj7GwS8IIIAAAh4XIHDl8TcAy0+LAIGrtDBzEwQQQAABrwoQuPJq5TO/btNyf2lZWCeqk/eB8Zmrygr4dMPoltv4n3mu035+aV1I9Y3pnbWJWV02OKCOhe4KXH3nTzWqrk/Ne5DAVXrfo9wNAQQQQAABBBBAAAEnCcQbuNq+fZvem/+v+szYevmT+OfY/gq/3jh0iWb94z8njW3uE7/T2HYL1bejtW9hEo8F67I1ZNoPNGDAwKSMakJsTz18r24fW6GsJH5sUHHaCoet76uvfvsn8plvLTVzELhqBoWnEEAAAQQ8K0DgyrOlZ+FpFCBwlUZsboUAAggg4D0BAlfeq7ldVmw6XO0rj1jdhZqfUUOo+efDVlLLXHv2YQJc5l/TEbQ+ib5kQPK2G2ga1w6P898LqTGc/plcekFAnds3/6Fx+meTnDt+ywpc1RC4Sg4moyCAAAIIIIAAAggggEDcAvEGrp776zPqU/e8xvRp4Y/kuO/48RNN56hfvFqg+/6/h5SXl/fxF1v5289++DV9/coTystq5QAtXLbpYEDLq6Zo9hf/sYUzEnt68eLFqt38O00bkbwtGs0MzGcVj76Tr0/f+W/q1q1bs5MicNUsC08igAACCHhUgMCVRwvPstMqQOAqrdzcDAEEEEDAawIErrxWcdbrBoGX1lodrpL7Wft5WUzMygSuOrkscHXfozWqazwjqXdeifhPoMNV/FaciQACCCCAAAIIIICA1wTiDVz96sGf69M939XALsn/1s0ji3L1qc/9UP369Wszf0VFhX77s2/pe9fVtHmsswc4WuXXb1YM1P/54b+e/VKrfn/iiSc01LdAo3sn/w/reWuz1XnsP2n8+PHNzo3AVbMsPIkAAggg4FEBAlceLTzLTqsAgau0cnMzBBBAAAGvCRC48lrFWa8bBF7dENLp+vSuxOyGMHlIQCXt3NXh6pV1jaquiyrSzI4X9VYQK9TM8yHr/+eoD30ypBW2zq1t+PvzY/oFdc2FSf5qd3rLzt0QQAABBBBAAAEEEEAgRQLxBq5+8qPv68vjtqtDQTN/nLRxbs+szlHfSd/QuHHj2jiStHXrVr3z3I9192V1bR7r7AGq63364fxO+tkvHjr7pVb9/stf/lI391tpbX2Y/BDbm1uyFO5zt6ZOndrs3AhcNcvCkwgggAACHhUgcOXRwrPstAoQuEorNzdDAAEEEPCaAIErr1Wc9bpBYPGWsE5U/z3Yk441mcDVtJEB5WW7K3CVDjvugQACCCCAAAIIIIAAAgicLRBv4Opff/i/9KVxu9StOPmBqz8vy9Xwq+7VmDFjzp5ewr9v375dr8/5V90zJfmBq4oan/7vvFL98sFfJzyv5i548MEHdX2v5RrQOfmBq9c2WYGrXp/Xpz/96eZuLQJXzbLwJAIIIICARwUIXHm08Cw7rQIErtLKzc0QQAABBLwmQODKaxVnvW4QeG93RLuPJ//D9nPZmKCVCVyZ4BUHAggggAACCCCAAAIIIIBA2wTiDVw98usHdU335erfKfnhoF8vzNX1d92vPn36tG0x1tWVlZX6/c/u1bem1Sb978ZDJ/16dO0Q/c///YM2z9MM8Kc//UnDAq9obN/kbyn49KocDZv6XY0YMaLZuRK4apaFJxFAAAEEPCpA4MqjhWfZaRUgcJVWbm6GAAIIIOA1AQJXXqs463WDwP7yiFbtTG/gakQvvwZ18buBjzUggAACCCCAAAIIIIAAAhkXiDdw9dyzz6h7zfMa3z+54aBGa7ifvpyv7/zwv5Sbm9tmj3A4rJ/f/w19fXK52uUktyPzun0BvXt6imZ/8R/bPE8zwBtvvKHIzkd1zbCGpIzXNEjEWvYji3J161d/oQ4dOjQ9/bFHAlcf4+AXBBBAAAGPCxC48vgbgOWnRYDAVVqYuQkCCCCAgFcFCFx5tfKs28kC5kPcBe+FFEpT5ior4NMNowNOJmPuCCCAAAIIIIAAAggggICtBOINXB0/fkyLn/yOZoyuS2rnKNM16uW9YzT7q/+SNJcnHvsvTSxeon4dk9uNa/66HPWb/D0NHz48KXOtqanR3N/cpzvGVSqQxO8Vnaj267myC/Slb/7fFudJ4KpFGl5AAAEEEPCgAIErDxadJaddgMBV2sm5IQIIIICAlwQIXHmp2qzVTQK7jkW1dk9yP8RuyadPR7/G9E3ip9At3YjnEUAAAQQQQAABBBBAAAGPCMQbuKqvr9cTv/+Jbh++RblZyesctXR7ngou/IZGjb44aeLvrVmjyrUP6MrBdUkbs67Rp6fW99bnvvpjZWVlJW3cOY89pGu6vq0OBckzXb8vqENFt2vatTe0OE8CVy3S8AICCCCAgAcFCFx5sOgsOe0CBK7STs4NEUAAAQS8JEDgykvVZq1uEmiwtn9YvCWsU3XJ+3C4OR/zbd8pwwIqzPU19zLPIYAAAggggAACCCCAAAIItEIg3sBVNBrVggXz1bHiaU3on7wt8P5rxSDddve3VFpa2orZN39JeXm5Hv3tf+qeCVuVn6Rs1M5jAa2LfEY3z/gHq8NX8v4uXbVypQ6u+o2mjzrd/GJa8ezvl3fW1f/wLfXr16/FqwlctUjDCwgggAACHhQgcOXBorPktAsQuEo7OTdEAAEEEPCSAIErL1WbtbpN4PipqJZuDcv6/D0lh/ks+2Krs1WvDnS3SgkwgyKAAAIIIIAAAggggIBnBeINXBmgU6dO6eEH/kN3jt6uHsVt21s+ZDVK/uu7Oeo06guaMmVK0v1XrFihjYse0V0TTyvYxp3pj53y61dLeujr935XnTt3TupcT58+rT/87hFd1X21RvRoW/foiPU3+YJ12arvMl0333yzgsFgi3MlcNUiDS8ggAACCHhQgMCVB4vOktMuQOAq7eTcEAEEEEDASwIErrxUbeesNWR9flxdG1Vto9QYisp8eGnCP1kBn3Ksb8kW5Hzw6JwVpW6m2w5H9P7+tn3g3tLszFaCo3r75Sdv1RIRzyOAAAIIIIAAAggggAACrRJIJHBlbrBt2za9+swv9PlxFSrKb923bsxVb23J0h7/ZN1+xyxlZ2e3au7nuigcDmvOk4+re91runJIY+xv+XOd39Jrp+p8emxZga7+zLc1dOjQlk5r0/NHjhzRY//1r7p7wgl1Lmz939UbDgS17PjF+vzd9yg/P/+ccyJwdU4eXkQAAQQQ8JgAgSuPFZzlZkSAwFVG2LkpAggggIBXBAhceaXSzlhno/Wl0k1WeGjX8cg5uzZZ2St1KvJpRM+AivKcsbZUzvL9/WFtO9y6D9xbmlevUr/G9idp1ZIPzyOAAAIIIIAAAggggAACbRFINHBl7rVmzRq9t/C/NePCYyq2Qld+88dxnIfZln7dviytqx6nu7/09XN2YYpzyBZPa2xs1F/+9AeNyFmq4T1DysuK/+9V08H5ZK1P8zeWaNCEWZowYUKL90nGC5s3b9bSl36rGSMOqbRdYqbmM4ydx/36244LNOsf/1nFxcXnnRKBq/MScQICCCCAgIcECFx5qNgsNWMCBK4yRs+NEUAAAQS8IEDgygtVtv8aq+ukLQcjOnQyItPdKt7DfLhc2s6nQd386moFsLx87DwatTpdhRVOwK85L9PNalAXvwZ19VsdxZo7w3vPmQ/8axqiqjMd16wP1M37LtvaIaKd1WkNI++9H1gxAggggAACCCCAAALJEGhN4Mp0j1q7dq3WLH1eQwp36fLB1h8pcRxHT/n08oZ85fe+WtdMuz6uYFAcw57zlMrKSr3y8jzVHlisTw05pW5xboW4cmdQqw510yVTbtPo0aNTGgwzC4hEIiorK9OKRc+pizbrUyMaFIjju0cmFPa3DXmqLRyv626cEdvy0Gdac5/nIHB1HiBeRgABBBDwlACBK0+Vm8VmSIDAVYbguS0CCCCAgDcECFx5o852XaXZKvBARVRrdoVj2wa2ZZ4DOvs0tEfA0wGYyhrLcndEVdZ2jMY2kcN8Lpyf7dPIXn7rg/Dzf0icyNhOPNd8+9s47j4e1aGK5oOARqmkQOrdwa/O7X0xvzg+X3ciB3NGAAEEEEAAAQQQQACBJAu0JnDVNIXa2lo98cQTOrRjpW4YeVqDu4YUDERj2/eZv1PM34PmT8IT1X69vSNX7x/tpjvv/qIGDBjQNETaHnfu3Kk//vGPGlJ6UFddUKuOhWaeH3SSMnM0X3AJRXzadDCoN7bkq7jnWN11113Kzc1N2xzNjUzwasGCBVr21su68cJTurBno/VFm7+bmnma7zdV1vi1dEeOlu0s0vR/uFWXXnppQvMkcJUQFycjgAACCLhcgMCVywvM8mwhQODKFmVgEggggAACbhUgcOXWytp/XebDyvX7wtp9LPFwUEurKy3waeJAv/WhqHcDQ6ZDWHl1VJsPhFV+uiWpjz9fmOvTkO5+dbFCQ1lW5yavH9uPRLXjaFi1DR98+B+Ph+l0VdrOrwt7+1RgeXIggAACCCCAAAIIIIAAAucSaEvgqmnc/fv3a8OGDao8tkvhuhPyh6p0uqpcVXV+lXTqodySgRo6bIT69u2blq5WTfM6+7Gurk7btm3T9rKNqi7fK3+4SjUnD8gXzFdu+y5WC+FSdeg6QEOHDlWvXr0UCGSu3fLx48dlPis7sn+rjh0oU6SuQgV5fuUVdZOC7eTL665+AwZr0KBBKi0tPXup5/2dwNV5iTgBAQQQQMBDAgSuPFRslpoxAQJXGaPnxggggAACXhAgcOWFKttvjebbtlsOhrX1UDT2rdtkzrBDoQldebvTVZNnVa10rCqqSqtTU521JV7YcjeH6cKUa4XSCvOkTpZXibUtI52ZpJNWh7D1eyPWt8A/hPqAK6H/NlsNDrG2uOzbyR/XNhQJDc7JCCCAAAIIIIAAAggg4BqBZASuDEbU+jZTKBSS2W7Q/HvllVe0aNGi2FZ81113nSZPnqysrCxbuJkuUo2NjVq+fHlsG7/y8nLdeeedse347DJHA1VVVaVnn302tn2j3+/Xvffeqy5dusj8HAwG2xQII3Bli7cik0AAAQQQsIkAgSubFIJpuFqAwJWry8viEEAAAQQyLUDgKtMV8Ob9dx6NaJ0VbEnV0am9dNkFtGpKla8bxz1SGdWybeGkBQC7l/g0fkDmvpXtxhqxJgQQQAABBBBAAAEE3CSQrMDV2Sb19fV6+umn1aFDB5mtBysqKtS9e/dYR6Y+ffooOzv77EvS+vuRI0f04osv6vbbb9eWLVu0Y8cOzZw5s00hpmQs4OTJk9qzZ482bdqkU6dOqUePHtq1a5cuv/xyXXTRRcm4RWwMAldJo2QgBBBAAAEXCBC4ckERWYLtBQhc2b5ETBABBBBAwMkCBK6cXD1nzr2mXlq0OaT6UGrnP7rPB12GUnsXRneDwMGTUa3ZFVZjOLmr6Vbs07j+ATpdJZeV0RBAAAEEEEAAAQQQcIVAqgJXBqe6ulpz585Vt27dNH78eJmQ03vvvRcLFI0dO1ZXXnllRgJONTU1+tWvfqUbbrghtn2g6Xa1YMGC2FxuuummjNT12LFjevnll3X48GGNGTNGgwcPjoXS5syZoyuuuCIWtjKdrZJ1ELhKliTjIIAAAgi4QYDAlRuqyBrsLkDgyu4VYn4IIIAAAo4WIHDl6PI5bvJmK8ElZWGVt2HLtngXnW99aXfqCKvVf/I+F4331pznIAGzfeDiLUlOWp2x/n7W1oIX9vbLb23jyIEAAggggAACCCCAAAIINAmkMnDVdI/f//73MgEr06HJZ+0jb8JFZrvBbdu2ffR8SUlJWrpeme0OTYjJhMCuuuqqpinKdOQyIaxp06ZpxIgRHz2fqh/MFoynT5+OBawWL14cMzHBqokTJ8ZuaeY5b948GRezHWOyDwJXyRZlPAQQQAABJwsQuHJy9Zi7UwQIXDmlUswTAQQQQMCRAgSuHFk2x066/HRUS6xwiwlepeMY0i2goT1IuqTD2qn3WLgprMqa1L4hx/QNqE9H3odOfY8wbwQQQAABBBBAAAEEUiGQjsCV2U7wz3/+syZNmhQLWDWtw2yfV1ZWFut8VV5eruHDh8c6ThUUFDSdkvTHVatWxbbou/XWWz8x9tGjRzV//nzdcsstKiws/MTryXpi3759Wr9+vQ4ePKhevXqpf//+GjBgwEfdvhoaGvTCCy/EAmjXXnttSoJoBK6SVU3GQQABBBBwgwCBKzdUkTXYXYDAld0rxPwQQAABBBwtQODK0eVz3OTX74tox5FI2uadZ3W5umpYUNnBtN2SGzlIYOP+iLYeTv37sSDXvA/N1oKErhz09mCqCCCAAAIIIIAAAgikVCAdgSuzgNraWv3yl7/UbbfdFgsYnb0o0+3JdHQyna/Mtn5Tp05VVlZWrCPW2ee29ncT/HrwwQd17733qri4uNlhVq9erddff13f+973knpv09HKBK1MoGvXrl2aMWNGrJuV6fh15hGJRPT888/LdLgywa9UHQSuUiXLuAgggAACThQgcOXEqjFnpwkQuHJaxZgvAggggICjBAhcOapcjp/sgrVhNYRS203oTCSzneDlgwMqaffxD1LPPIefvSlQ0yAt3BhSY+p2E/wY7JDufg21/nEggAACCCCAAAIIIIAAAkYgXYErc68DBw7opZde0jXXXKO+ffuapz5xVFVVyXSh2r59uzp16qQ+ffrEOl/l5lrfIGnDYcZ99NFHZTpGDRo0qMWRQqFQLPiVl5cXm2cgEGjx3HheOHHihNatW6fjx4/LdK4yXbzMloUmTHb2YUJWr776qurq6mKhs7be++zxz/ydwNWZGvyMAAIIIOB1AQJXXn8HsP50CBC4Socy90AAAQQQ8KwAgSvPlj7tC6+wthNctDlN6ZYzVje6T0B9OxG4OoOEHy2BPcejWrM7fe/HHOsz/anD6bbGmw8BBBBAAAEEEEAAAQQ+EEhn4Mp0eTp06JD+8Ic/6I477mi201VTXUzoyGw5uHjx4lhHqJEjR+q6665rejnhx6eeekq9e/eOdZU638Um+GTCWVdcccU5w1nnGscErObOnatTp05p/Pjxuuiii9SuXTsFgy23vl6wYIHMtobGJjvbapWdwoPAVQpxGRoBBBBAwHECBK4cVzIm7EABAlcOLBpTRgABBBBwjgCBK+fUyukz3XU0orV7U79929lOfTv6NbovnYXOdvH6729tCau8On3d1sxugpda3dY6FhD+8/p7j/UjgAACCCCAAAIIIGAE0hm4ahI3oSsTLpo+fXqsi1XT8y09mvDVm2++qbKyMvXq1UsXXnihunbtqsLCwpYu+eh5s0Xf2rVrtWnTpth2hs11lvro5DN+2L9/v5555hnddddd6tChwxmvNP+juY/pZmW2DVyzZo1qampi4a5x48Y1f8FZz5ouWKazV1MQ6qyXk/5r031mzrxFM1O4dWHSJ86ACCCAAAIIpECAwFUKUBkSgbMECFydBcKvCCCAAAIIJFOAwFUyNRnrXALv749o2+H0B646Ffp0mRV04UCgSaDe2tby5XVhWV/yTusxomdAg7oSuEorOjdDAAEEEEAAAQQQQMCmApkIXJlOVyYANW/ePH3ta19T+/bt49Ix3aL27NkT63i1c+dODR06NBZqOlfwypz33HPP6Ytf/KKKi4vjuk/TSRs2bIh12Jo1a5YKCgqanv7E4+7du7Vs2bLYtoGmk1W/fv3UvXt3xbsloAmEvf3227FAWMeOHT8xfiqeIHCVClXGRAABBBBwqgCBK6dWjnk7SYDAlZOqxVwRQAABBBwnQODKcSVz7ITf2xPR7mPpD1wV50tThrW8dYBjQZl4qwWOVEb0zrb0vxe7l/g1fgDd1lplUHo3AABAAElEQVRdOC5EAAEEEEAAAQQQQMBFApkIXDXxvfvuu1q5cmWsi5TZbi+Rw3SQWrRokZYuXarLLrsstm2fCV6ZDlY+3wdfMKmtrY1tDWg6aZkAVGuOV155JRacmjp16keXm25WpuuW6Wa1ZMkSHThwQLfeequGDBny0b0/Ovk8P5jg2fz583XPPffEHTw7z5BxvUzgKi4mTkIAAQQQ8IgAgSuPFJplZlSAwFVG+bk5AggggIDbBQhcub3C9lnfems7wR3WtoLpPkoL/Jo8hJBLut3tfL8dR6Navzec9im2z5OuHk74L+3w3BABBBBAAAEEEEAAARsKZDJwZTjeeuutWGBpxowZysuz/lhJ8Kiurta2bdtktuTz+/3q06ePRo4cGQsvzZ07N7b14OTJkxMOQjVN4/Tp03rqqadktgYcNmyYDh8+rNWrV6uysjLW9cp0tOrdu3cs6NV0TbyPu3bt0uuvv66bb745rq0V4x03nvMIXMWjxDkIIIAAAl4RIHDllUqzzkwKELjKpD73RgABBBBwvQCBK9eX2DYLLDsU0aYD6Q9cdS/xWV2F2FLQNm8EG0yk7KD1XrT+pfvIy/bp0xfyXky3O/dDAAEEEEAAAQQQQMCOApkOXJntBRcuXCjzudBXvvIV5ebmtorJdJ0yXa9Mt6j169fHAlfdunWLdc9q6njVqoGtiyoqKvSjH/1IHTp0iHW7uvrqq2WCVsFgsNVBrv379+vHP/6x7r///rSHrYwDgavWvhu4DgEEEEDAjQIErtxYVdZkNwECV3arCPNBAAEEEHCVAIErV5XT1os5eDKqFdvT31VoSHe/hlr/OBBoEshU+C/fClxNI3DVVAYeEUAAAQQQQAABBBDwtECmA1dN+K+99ppMNynT7amtx9atW/W3v/1NJSUlysnJ0aBBg9SvXz8VFxfHPbQJcB08eFA7duyQ6UTVtI3g7NmzWx0Ka7q5CXA999xzuvTSSzV48OCmp9P6SOAqrdzcDAEEEEDA5gIErmxeIKbnCgECV64oI4tAAAEEELCrAIEru1bGffNqCEkvrQvJ+hJt2g6fT5o0KKDO7a0fOBD4UGDXsYjW7kl/h6uifOmqYWwpyBsRAQQQQAABBBBAAAEEJLsErhoaGvT000+rsLBQ06dPb3Vpqqqq9Oijj+rGG29Uz549deTIEW3atEnvvvtubLvBG264Idb96lw32Llzp+bNm6fGxkZdeeWV6t+/v4qKimRCYfX19W2an9kC8be//a2mTJmi0aNHn2saKX2NwFVKeRkcAQQQQMBhAgSuHFYwputIAQJXjiwbk0YAAQQQcIoAgSunVMod81y6NaxjVelLXGVZu7dNGRZQuxwCV+54ByVnFcdORbW0LP3d1nqV+jS2P1sKJqeKjIIAAggggAACCCCAgLMF7BK4Moom4PTHP/5RQ4YM0eWXXy6/P7Eu0XV1dXr88cdjXaPM9WcepnvW8uXLtWrVqliA6pJLLlHnzp2Vl5cX61518uRJ7d69W2vWrJEJRV177bWxcc7cjrC2tjYWlho7dqwmTZqU8HaC5h5PPfVULGhl7p/Jg8BVJvW5NwIIIICA3QQIXNmtIszHjQIErtxYVdaEAAIIIGAbAQJXtimFJyay/0RUq3alL+hS2k6aPJSOQp54cyWwyLDV3GreGqvbWgLXJOPUi/r41b9TYv/HRTLuyxgIIIAAAggggAACCCBgPwE7Ba6MTk1NjV544YVYd6qzQ1Pn01u8eHFsW8JPfepTCgSa/5KJGf/AgQMqKyvT0aNH1b17d5kt/kKhkPr27asBAwbEnmvpXqaD1jPPPBMLZJlr4z1MB68nnnhCw4cP17hx4+K9LGXnEbhKGS0DI4AAAgg4UIDAlQOLxpQdJ0DgynElY8IIIIAAAk4SIHDlpGo5f651jVG9sTEss71gqg+/1dTqSitsZbZx40DgbIF3tkZ0pCp92wqaL4hfMTigknZ0Wzu7FvyOAAIIIIAAAggggIAXBewWuDI1MJ2kHnzwwdh2fqYT1Jldplqq0f79+/X888/r7rvvVkFBQUunxbpZmfCT+RzKBLTMUVlZqREjRmjixInq0qWLsrKyWrzevGC2KFy2bJnuuuuu855rzjdhrsceeywW5DKds+xwELiyQxWYAwIIIICAXQQIXNmlEszDzQIErtxcXdaGAAIIIJBxAQJXGS+B5yaw+1hEa/dGFE1xe6HeHfy6uB/dhDz3BotzwQfKI1q103ofxnl+W0/Ly/bp6uEBmW0uORBAAAEEEEAAAQQQQAABOwauTFVMJykToLr44otjXaHOVakjR47oD3/4g+644w716dOn2VMjkYj27NmjtWvXxrYMLCoqioWsTFer+vp6bdy4UVu3blU4HFaPHj00atQolZaWtjjWK6+8EuuMNXPmTGVnZzd7nnkyan3oMH/+/Fho7NOf/rSCQXt0vyZw1WLJeAEBBBBAwIMCBK48WHSWnHYBAldpJ+eGCCCAAAJeEiBw5aVq22OtJuCyYntEh06mrruQ6W41dURA7XLoJmSPqttvFnWNsrqthdLSbc2sfpS1nWA/thO03xuBGSGAAAIIIIAAAgggkCEBuwauDMfhw4f1xz/+UbNnz1bXrl2bFWpsbNRf/vIXTZ48ObYlYHPdsLZv3665c+fGulGZDlNm20ATkjr7XBO2Mt2v3nnnHa1cuVKdOnXSrbfeqvbt23/i3ubcp556SgMHDpTpwtXSMW/evNg2h5/5zGfi6obV0jjJfp7AVbJFGQ8BBBBAwMkCBK6cXD3m7hQBAldOqRTzRAABBBBwpACBK0eWzfGTrq6Pymzpdtp6TPZhOgiNHxBQp/aErZJt67bxth+JaMO+1AX/mryK832aMozWVk0ePCKAAAIIIIAAAggggIBk58CVqc/evXtjYSnTSap3796fKNmiRYti3bBuvPHGjwJUZgu/AwcOaOfOnbGuVSZcddlll8WCVn6zz3ochwlerV69Orb1YH5+vkaOHBnbEtCEsJqOioqK2NzMvc8OhJkg2MKFC3Xo0CF99rOfVW5ubtNltngkcGWLMjAJBBBAAAGbCBC4skkhmIarBQhcubq8LA4BBBBAINMCBK4yXQHv3t90GHpzU1h1jckLXZmI1ai+fvXtGN8Hud7VZ+VNAkvLwjp2KnnvwaZxz3ycMDCgbsUEAM804WcEEEAAAQQQQAABBLwuYPfAlamP2QrwpZde0u23367i4uKPSrZhwwYtWLBA9957r/Ly8mLbAZrnTIeqrKwsTZo0Sd26dVNJSclHYayPLo7zB7Pd4IkTJ7Rp0yatX78+Froy3bTMuGa7wB07duhPf/qTvvvd76qgoOCjUd966y2ZrQ5nzJhhq85WTRMkcNUkwSMCCCCAAAISgSveBQikXoDAVeqNuQMCCCCAgIcFCFx5uPg2WHpVbVTr9kZ0PAmBl5ygNKZvQF0Jttigss6ZgnkPLi6LqDGU/NCViVgN7eHXBd38Im7lnPcEM0UAAQQQQAABBBBAIB0CTghcGYeysjI9/vjjsXBVx44ddfTo0dhWgrNmzYoFnzZu3BgLRZkOVjfccEMsGJVsv9ra2tg83njjDfXv319DhgxRr169Yp2wysvLNX36dJn7b968WWYrwa9+9avNbkeY7Hm1ZjwCV61R4xoEEEAAAbcKELhya2VZl50ECFzZqRrMBQEEEEDAdQIErlxXUsctKBSW3re2ddt/0oReEp++30qymO0Dx1idrXKziLWcKWh96Vcha8c8Y2y+AeyzeHzWf5ltFwM0AfuIyoSu3t5quq199FSbfzDvxN5Wp7XRfaywFW/LNnsyAAIIIIAAAggggAACbhNwSuDKuG/ZskVmC8E777xTTz/9dKyrVV1dXawkffv2jQWgOnfuHAs9pbJOZrtB03XLdL0yXaxM1y2zdeDEiRNjASszR7MFogmG2fUgcGXXyjAvBBBAAIFMCBC4yoQ69/SaAIErr1Wc9SKAAAIIpFWAwFVaubnZOQRO10vbDke053hEkTibDXUs9Glkr4Da58r6YPccg3vsJRMc2mU5HjkZVb31c9jyjAWuLAcT/gkGfGqfZwWCOvisre6AM2+Pypqolm2LqDYJW1wa48FWV6sLuvoJtnnsf3ssFwEEEEAAAQQQQACBeAWcELgyf0eGw2FFIhG99tprWr58uQ4fPqxhw4bp2muv1aBBgxQIWN/oycBh5rVw4cLYvCoqKtS5S2fNumuWTADMzMl0vDJfOLLbQeDKbhVhPggggAACmRQgcJVJfe7tFQECV16pNOtEAAEEEMiIAIGrjLBz03MImLDQkcqIKk5LNQ1RNVhdr8JWlybTkSlo/cvL9qkoX+psdbVqn2e/D0/PsbSUvmQyamZrxt3HojpkdQszZvEc+dlS307+WPjK2Hr5qG2wuq3tj+ig5Wf9/wmtOgqs8J8JW/UqpbNVqwC5CAEEEEAAAQQQQAABjwjYOXBlOkmZrla7du1URXWFwgopNy/H+pcbC19VnzqtUJ31XFaeenbvpVGjRqmkpCTllTPBr3379llbB27S0RNHFQ2av4QjsXll52TLzKu2ulY51rwK8wo1duzY2NaDdgpeEbhK+duEGyCAAAIIOEiAwJWDisVUHStA4MqxpWPiCCCAAAJOECBw5YQqMUcEzi1gtg5cszuivSdamRKyhjfbDE4Y6FfHQjpelVdHtXJnRHVW4M98fB/PYQKBA7sENKS7T2abSw4EEEAAAQQQQAABBBBA4FwCdg1c1dbW6oknnlB9tFaXT7tMRcWFysrO+th2gabzVWNDoxrqG7R21QaVvbdNt916mwYMGHCuJbf5tVdffVXvrn1Xl14zQb369lBufm6si9WZgapQKBSb1+GDR/X2K8s0qP8Fmj59epvvnawBCFwlS5JxEEAAAQTcIEDgyg1VZA12FyBwZfcKMT8EEEAAAUcLELhydPmYPALWVnjSu7tDsce2cpig0MAufg3vSejKbGtZcTqqE1b46oTVOayyNirTAavpCFoBtYIcn0rbSR2tbmul7fxW97WmV3lEAAEEEEAAAQQQQAABBM4tYLfAlQlRvf/+Bs1/eb7GXjZaIy4e/rGQ1blWc+JYuf727GsaNXy0Jk6cqLw8aw/7JB4HDhzQCy++oKIuBbriU5fJdLOK5zCdul594Q0FQ9m64fob1KFDh3guS+k5BK5SysvgCCCAAAIOEyBw5bCCMV1HChC4cmTZmDQCCCCAgFMECFw5pVLME4FPClTXRbV0a/hjQaBPnpX4M306+HRRn0BsG8fEr3b3FfWNUQUCvtj2lu5eKatDAAEEEEAAAQQQQACBVArYLXC1ceNGvb1yidXVapJKO5UmvPSa07V6b/laBRtzdNONNyV8fUsXnD59Wo/9+VGNvXKMevXrEXcIrGk8sw3hlvVbtWl1mb40+0vKyclpeikjjwSuMsLOTRFAAAEEbCpA4MqmhWFarhIgcOWqcrIYBBBAAAG7CRC4sltFmA8C8QnUNUpLtoRUXR/f+YmeNbS739oej05XibpxPgIIIIAAAggggAACCCAQj4CdAlfHjh3Tz3/xM935T59Tx86t7wJlthl84cn5mjxhioYPHx7b7i8ei5bOMWGp3//h9+o/so9GWh232nK8s3C5ju0t1xfu/oKysrLaMlSbriVw1SY+LkYAAQQQcJkAgSuXFZTl2FKAwJUty8KkEEAAAQTcIkDgyi2VZB1eEjDb3a3aGdbBCuuHFB1me8FLBvjVrZjQVYqIGRYBBBBAAAEEEEAAAQQ8LGCXwFVdXZ3mPjtXfUf20qChA9pckcqKKj31+2f1xbu/qO7du7dpvCVLlqhszyZd+5lpCgaDbRorHA7rlede17D+IzR+/Pg2jdWWiwlctUWPaxFAAAEE3CZA4MptFWU9dhQgcGXHqjAnBBBAAAHXCBC4ck0pWYiHBHYfj+q93eGUr7idtdPClUODym7b59opnyc3QAABBBBAAAEEEEAAAQScJmCXwNW+ffv0ylt/0423XZs0wvff26zqQzW66aabWt3l6uTJk1Z3q9/pzm/enrR5VZ08pTdfXKIvfeFLCW9NmKxJELhKliTjIIAAAgi4QYDAlRuqyBrsLkDgyu4VYn4IIIAAAo4WIHDl6PIxeY8KvLkprJM1qetu1cTqs7pcje4TUJ+O1g8cCCCAAAIIIIAAAggggAACSROwS+DqxXkvqEOfYg1MQnerJhyzteBrz72p22d+Trm5uU1PJ/S4ceNGbd2/RZOuSl43qqjVLvr1+W/q8rGT1adPn4Tmk6yTCVwlS5JxEEAAAQTcIEDgyg1VZA12FyBwZfcKMT8EEEAAAUcLELhydPmYvAcFyk9H9NbmSNpWnpPl07SRAQXYWTBt5twIAQQQQAABBBBAAAEE3C9gh8BVfX29fv7Az3X7l2cqv11+0tCj0ahee3Ghrpo4VT169GjVuE/MeUIXjOmvXv16tur6li7atG6L9m8+pDvvvLOlU1L6PIGrlPIyOAIIIICAwwQIXDmsYEzXkQIErhxZNiaNAAIIIOAUAQJXTqkU80TgA4FFm8OqOJ367lZnek8YGFC3YrpcnWnCzwgggAACCCCAAAIIIIBAWwTsELjasWOHXvzb8/rcl29ry1KavXb96vdVfaRWM26e0ezr53vy4d/8StNmXq3CosLznZrQ6yfLK/XQj/5LDz7wYEa2FSRwlVC5OBkBBBBAwOUCBK5cXmCWZwsBAle2KAOTQAABBBBwqwCBK7dWlnW5UaCuMaqX14XTvrSBXf0a2ZMWV2mH54YIIIAAAggggAACCCDgWgE7BK5Wr16tjTs26NP/cE3SnXdu3aXNK7dp9hdmt2rsf/t//6Y77rnN2pIwp1XXt3RRJBLRj7/zU/303/9D7dq1a+m0lD1P4CpltAyMAAIIIOBAAQJXDiwaU3acAIErx5WMCSOAAAIIOEmAwJWTqsVcvS5QXh3VW1vSH7jqWuTXxEEErrz+/mP9CCCAAAIIIIAAAgggkDwBOwSuXn75ZTVk1WrC5EuSt7APRzq496Bef+4t/ct3/iXhsWtra/XLX/1Cs++blfC18Vzw8+8/oH++91vq1q1bPKcn9RwCV0nlZDAEEEAAAYcLELhyeAGZviMECFw5okxMEgEEEEDAqQIErpxaOefPOxyR9ldEFImmZqu6zoU+tUvuF2Ezjn7oZFTLt6c/cFXSzqcrhwYyvn4mgAACCCCAAAIIIIAAAgi4RcAOgas333xTR6oOasp1k5POumfHXq15a73u+co9CY8dCoX0H//5U9319TuUlRVM+PrzXWA6XP3r/T9SUVHR+U5N+usErpJOyoAIIIAAAg4WIHDl4OIxdccIELhyTKmYKAIIIICAEwUIXDmxau6Yc12j9NqGkEJW8CoVx7j+fvUsdVdXpn0nolq9K/2Bq8Jcn6aOcFfgKmq96ZaVhVRvvQ99VuavMM96opnsX3tr7eb1s4+g33qymecD1nP+D992XYr9CrrrLXg2A78jgAACCCCAAAIIIIBAKwXsELjatGmTXl/ymm6b/ZlWrqLlyza+t1nle09q5sxbWj7pHK/84oGf68Y7rlP7osJznJX4SzWna/Tv3/u5HnnkkcQvTsIVBK6SgMgQCCCAAAKuESBw5ZpSshAbCxC4snFxmBoCCCCAgPMFCFw5v4ZOXUGqA1eXDPCrR4m70i77TkSswFWKEmrneCMVWKGja1wWuIpYjP97To0qTpvoVeJHoIW3ViyH9WEQ6wcz89ShsIUTE78lVyCAAAIIIIAAAggggICLBOwQuKqsrNSDv35AX/jmncrKzkqabjQa1dLXl2lgjws06qJRrRr30T/9t8ZMvkhdundu1fUtXbR35z69+cJiffe732vplJQ+T+AqpbwMjgACCCDgMAECVw4rGNN1pACBK0eWjUkjgAACCDhFgMCVUyrlvnmmOnA1fkBA3UuaaUHkYMoDFVGt3JH+DldF+T5dNcxdHa5MZ7XvtyFwFc/b6Ce356ukwF3vwXjWzTkIIIAAAggggAACCCBwfgE7BK7MLB/782O68NJh6t6r2/knHecZZkvAvz37um6dfpsKC1vXoWrZ8mWqVqUuvHhEnHeN77TFry5Vrw59NWHChPguSPJZBK6SDMpwCCCAAAKOFiBw5ejyMXmHCBC4ckihmCYCCCCAgDMFCFw5s25umHWqA1cTBgbUrdhdYZfjp6JaUpb+wFXn9j5deoG7AleNFuP3n6rRyVZ2uIrnf4P/8fl8tc9z13swnnVzDgIIIIAAAggggAACCJxfwC6Bq+XLl+t47RGNu+zi8086zjOOHz2hdUs26taZtyoYDMZ51cdPO3DggP628CVdd+s0a9v25HQOrq+r18vPvKovzfqyAoHM/I1L4OrjdeY3BBBAAAFvCxC48nb9WX16BAhcpceZuyCAAAIIeFSAwJVHC2+DZac6cDVxUEBdi9wVdqmpl17ZEEp79fp18mlUn8x8GJ2qxTZajN9/OrWBq5/dmS+zHSMHAggggAACCCCAAAIIIHC2gF0CV6dOndITT/1FUz8zRQUF7c6eZsK/m+5WT/72GX1qyjRdeOGFCV/fdIEZZ85Tc9Spb6lGXdL6cZrGM49rlq9VqDKqG66/QT5fZv5WI3B1ZkX4GQEEEEDA6wIErrz+DmD96RAgcJUOZe6BAAIIIOBZAQJXni19xhdO4CrxEkStS15dH1JNQ+LXtuWKsf381pYLyflGcVvmkcxrG6zA1f8xHa5qjGpqjl/Myld+dmY+xE/NihgVAQQQQAABBBBAAAEEkiVgl8CVWc+KFSu0cMkbuvufPi9/oPV/+0WjUb23Yp1O7j+l22+/vc1UJgz2k3//N93xtc+qY+cObRqv/Hi55jzyrP7H9/6H8vLy2jRWWy4mcNUWPa5FAAEEEHCbAIErt1WU9dhRgMCVHavCnBBAAAEEXCNA4Mo1pXTcQlIduJpkbYHXxdoKz23H/vKoVu1M37aCfovw+lFBBd3V4Er1jVH9n6drVZnCwNUv785Xbpb73oNu+98U60EAAQQQQAABBBBAIBMCdgpcmW5SL7zwghqD9briU5cq0Mo/ADet26KdG3br9ts+p/z8/KSwrt+wXivWLNdVN05WQWHrOnAdO3JCr/71dV17zXUaOnRoUubV2kEIXLVWjusQQAABBNwoQODKjVVlTXYTIHBlt4owHwQQQAABVwkQuHJVOR21mFQHri61AledXRi4ktWQ6dX3wzpdn7rOTGe+kQZ382tYj9Z/w/nMsez0c50VuPpBigNXD81upyyXBdXsVEPmggACCCCAAAIIIICAkwXsFLgyjiZ0NW/+PFWcPqFrbr5K2dnZCfGuf3eD3n1rnb70xS+pY8eOCV17rpNjXbPee0/Pz3tOn//qZ1VUUnSu0z/xWuXJKj33p3m6btp1Gjli5CdeT/cTBK7SLc79EEAAAQTsLEDgys7VYW5uESBw5ZZKsg4EEEAAAVsKELiyZVk8MalUB64uswJXndwYuLLeHZsORFR2KJLy90lOljRlWFB51qPbjtoGK3D1TK2qUtjh6uEvtVOABldue+uwHgQQQAABBBBAAAEEkiJgt8CVWVR9fb2WL1+u9RvX6cLxw9Wzbw8VtC9ocb319Q06duiYNq0tU6AxqOuuu07FxcUtnt+WF7Zs2aLFS95Sp94dNPSiwbHgld/f/JeDIpGIThw9oS0btunYnuO66qqrNXjw4LbcPmnXErhKGiUDIYAAAgi4QIDAlQuKyBJsL0DgyvYlYoIIIIAAAk4WIHDl5Oo5e+6pDlxdPjigjoXuTLs0WjsKLt4SVlVtartcjejp16CuzX+A7ex3n1RjBa5+aHW4SqXhb/6xddtdON2W+SOAAAIIIIAAAggggMD5BewYuGqa9YEDB7Ru/bpY8Kpz904aOKS/ikqL1KlrR52qqlb50XIdPnBEm60tBLt17q5JEy/VwIEDFQiktsXvqVOn9O6772rLts1qCNdr6KghKrY6Xpl5mb+OTciq4liF1q1+X9n+bE24ZKIuvPBC5ebmNi0t448ErjJeAiaAAAIIIGAjAQJXNioGU3GtAIEr15aWhSGAAAII2EGAwJUdquDNOaQ6cHWFFbjq4NLAlXnHVNVJS7aE1BBKzfunR6lfl/R3Z9jKiJktGf+v6XCVotCa38r6/drqcMWBAAIIIIAAAggggAACCDQnYOfAVdN8zTaDJuBkPjuqrKzUkSNHZLb4M92ievbsqfHjx6eso1XTHJp7NB2sdu7cGevGdfz4cZWVlalbt24qKSlR7969NXr06Nhjc9dm+jkCV5muAPdHAAEEELCTAIErO1WDubhVgMCVWyvLuhBAAAEEbCFA4MoWZfDkJFIduJo8NKDSdu7scNX0hjlSGdWy7WHrA++mZ5Lz2C7Hpystv+xgcsaz4yjVdVbgam6tTqUocBW0vtj9q9kEruxYe+aEAAIIIIAAAggggIAdBJwQuDrb6dixY3ruued00003qWvXrme/nPbfTfhryZIlOnr0aGxO2dnZaZ9DojckcJWoGOcjgAACCLhZgMCVm6vL2uwiQODKLpVgHggggAACrhQgcOXKsjpiUQSuklOmgxVRrdkdUWM4OamrrkU+jesfkAkMufk4ZQWu7k9h4ConS3rgbgJXbn4PsTYEEEAAAQQQQAABBNoi4MTAlVnvxo0btXTpUn35y1+Wz5fZLzmZANiTTz6pr3zlK8rJyWlLOdJ2LYGrtFFzIwQQQAABBwgQuHJAkZii4wUIXDm+hCwAAQQQQMDOAgSu7Fwdd88t1YGr8QMD6lbsU2Y//k1PDUMRadGmsEyIqLWH2QJvQBe/hvf0e8LMdLaKBa7aYHYu6zyrS9h/3pV/rlN4DQEEEEAAAQQQQAABBDws4NTAVTgc1ksvvSSzrd/111+vYDAzrZGbwlbTpk2LbXHolLcSgSunVIp5IoAAAgikQ4DAVTqUuYfXBQhcef0dwPoRQAABBFIqQOAqpbwMfg6BVAeuAn6pi9WtaXiPgApyzzERl7zUEJZMt6uyQ2HV1Ce2qO4lPg3q6ldJvhVQ80JCzeKpqrECV8/WymwtmIqjMNen/3cngatU2DImAggggAACCCCAAAJuEHBq4MrYNzY26rHHHtPYsWM1atSotJcjFArpL3/5S+z+w4YNy3inrUQACFwlosW5CCCAAAJuFyBw5fYKsz47CBC4skMVmAMCCCCAgGsFCFy5trS2X1iqA1dNACZ4NcLq2tSrg19ZLt8mz6w5auWH9pVHtOd4VHUNUn0oKtMByzxvDuORFfApx/oScom1413/zgEVeSwXVNsQ1do9YT2xtF6NoQ9ckv3fRVZ47ad3eAw22YiMhwACCCCAAAIIIICAiwWcHLgyZamurtbjjz+uW2+9VaWlpWmt1DvvvKODBw9q5syZab1vMm5G4CoZioyBAAIIIOAWAQJXbqkk67CzAIErO1eHuSGAAAIIOF6AwJXjS+jYBaQrcGWATNemvGxpTN+AOhV6o4WTyVeFrK5X5l/ESlvFAlfW0s3WgQELJGiFz0z4ymvHrqNh/Xlxgw6fjFguqVt9STuffvI5AlepE2ZkBBBAAAEEEEAAAQScLeD0wFXU+iNz06ZNmj9/vr75zW8qLy8vLQXZvHmzXnvtNd15550qKSlJyz2TeRMCV8nUZCwEEEAAAacLELhyegWZvxMECFw5oUrMEQEEEEDAsQIErhxbOsdPPJ2BqyYsEzbq1cGnC6zt8wqsLd84vCFgclV7j0f0xoZGrdyeopZWZ1F2sIJ9P/4sgauzWPgVAQQQQAABBBBAAAEEPhRweuCqqZDz5s1TQ0ODZsyYIb8/td/qqays1EMPPRQLeLVv375pCo56JHDlqHIxWQQQQACBFAsQuEoxMMMjYAkQuOJtgAACCCCAQAoFCFylEJehzymQicBV04RMx6uLevnVp5M/1vGp6Xke3SfQYOWr5r/boNetsFUqO1qdLde5yKf7byVwdbYLvyOAAAIIIIAAAggggMAHAm4JXJnVzJkzR4MGDdLFF1+c0vI+++yz6t27t8aNG5fS+6RycAJXqdRlbAQQQAABpwkQuHJaxZivEwUIXDmxaswZAQQQQMAxAgSuHFMq1000k4Erg2lCV8V5Pg3t4VcXKxzD4S4BE7RatLFRizc36vipFO4d2AJbt2K/fnBLerbUaGEKPI0AAggggAACCCCAAAI2FnBT4KqiokIPPPCA7rvvPhUXF6dE3XTSKi8v1+c+9zllZWWl5B7pGJTAVTqUuQcCCCCAgFMECFw5pVLM08kCBK6cXD3mjgACCCBgewECV7YvkWsnmOnAVROsiVr16fTBNoPtcgheNbk49TEUlnYeiWjuivrYNoKZWkePUr++/xkCV5ny574IIIAAAggggAACCNhdwE2BK2O9ZcsWLVy4UHfccYeKioqSxh+NRlVWVqbnn38+FujKzc1N2tiZGIjAVSbUuScCCCCAgF0FCFzZtTLMy00CBK7cVE3WggACCCBgOwECV7YriWcmZJfAVRN4dlC6oKtfA61/xK6aVJz1eLQyqj8trteuo2GFI5mde++Ofv2vGQSuMlsF7o4AAggggAACCCCAgH0F3Ba4MtJLly7V4cOHNXPmzKTB19TU6PHHH9ctt9yikpKSpI2bqYEIXGVKnvsigAACCNhRgMCVHavCnNwmQODKbRVlPQgggAACthIgcGWrcnhqMnYLXDXhdyjwaUh3vzq3J3bVZGL3x2NVEb21KaSlZSHVNaR/+8DmfPp28ut/3EzgqjkbnkMAAQQQQAABBBBAAAHJjYGruro6zZkzR2PGjNHIkSPl87Xt7+rGxkY99thjGjFihCZMmOCKtw2BK1eUkUUggAACCCRJgMBVkiAZBoFzCBC4OgcOLyGAAAIIINBWAQJXbRXk+tYK2DVw1bSeniU+XdQnINP5isOeAo3W9oHLt4X0zLJ6NYTsNcf+Xfz67k0EruxVFWaDAAIIIIAAAggggIB9BNwYuDK6J0+e1K9//WvdeuutGjhwYKvBQ6GQXnnlFZ06dSrW3SoQCLR6LDtdSODKTtVgLggggAACmRYgcJXpCnB/LwgQuPJClVkjAggggEDGBAhcZYze8ze2e+DKFCg/WzLBmYHWv7Z9Lzf55TYBo/V7w2qwQkdNRzQald/6BnHWxz6H/qDjU1bAp7zsT64iGIgqL+uTz3cu8ivgbxrZXo/WMrVqZ0ivrWvU/vKIzO92OwZ2Deg7N+babVrMBwEEEEAAAQQQQAABBGwi4NbAleE9fvy45s6dq1mzZikvr3VfRNm2bZvefvtt3X333TapWHKmQeAqOY6MggACCCDgDgECV+6oI6uwtwCBK3vXh9khgAACCDhcgMCVwwvo4Ok7IXDVxFuSL43sHZDZbtAux6m6qN7cFFY4Et+MdhwOa/uh+NpA+a1l/uT2fBW1s896zSpNrmrv8bDmrW7U+/vOSJrFR5DWswb3COifryNwlVZ0boYAAggggAACCCCAgIME3By4Ml8GWrx4sQ4dOqSZM2cqGEysdXR5eXlsa8Ibb7xRvXr1clBVzz9VAlfnN+IMBBBAAAHvCBC48k6tWWnmBAhcZc6eOyOAAAIIeECAwJUHimzTJZqg0KodYR2qtGF7ombMTLenLlbXpzF9/Wd1kGrm5DQ8darWClxtTiBwdcgKXB2OP3D173fkq32efQJXNfVRPb6kXhv3h1XfmAbgNt5iWM+Avnktgas2MnI5AggggAACCCCAAAKuFXBz4MoULRKJ6JFHHtGwYcM0efLkuOtYV1enhx9+WFOmTNGYMWPivs4pJxK4ckqlmCcCCCCAQDoECFylQ5l7eF2AwJXX3wGsHwEEEEAgpQIErlLKy+BxCBysiKrsUEQna5wRvMq3tuUb1NWnvp381vZ9cSwwRadUmcCV1eEqEifb9oMh7TgSX1cos66fWoGrQhsEruoarW9Gbwpp4fuNjnmPmJKP6BXQP32awFWK3v4MiwACCCCAAAIIIICA4wXcHrgyBaqvr9ef//xnTZ06VX379o2rZi+//LL8fr+mTZsW1/lOO4nAldMqxnwRQAABBFIpQOAqlbqMjcAHAgSueCcggAACCCCQQgECVynEZeiEBLYcjGjr4UjcW+QlNHgKTm6X49OEgQErlCRlIndlAlcLrcCVtVNDXMc2K3C1M4HA1X98Pl8FuZlY2QfLMR3Qth4M6zev1ak+vsZccTmk4ySfxTa2f0BfvIrAVTq8uQcCCCCAAAIIIIAAAk4U8ELgytRl+/btMiGqL3/5y8rJyTlnqbZs2aJVq1bplltuUW6uO/+eInB1zrcALyKAAAIIeEyAwJXHCs5yMyJA4Coj7NwUAQQQQMArAgSuvFJpZ6zTbJO39XBUe09YaRsHHFkBqXuxT8OtbkY5wfROuPLDDlfxBq62Hghp19H4O1z9vzvzZUJlmThMJ64XVzdqx+GwQs54K3zE1Lm9XzeNzdKI3kHlZn30ND8ggAACCCCAAAIIIIAAAh8T8Ergyix60aJF2rVrl+644w5lZ2d/zKHpl0OHDum3v/2tZs+erV69ejU97bpHAleuKykLQgABBBBogwCBqzbgcSkCcQoQuIoTitMQQAABBBBojQCBq9aocU2qBQ5XRrVpf0RVddG4Ozilek7nGj8n6NPI3v5Y+CrgP9eZyXut0tqC0WwpGGeDKyUauPr5Xe2U1/zn4MlbxBkjmXUcr4rq9Q0NWrw55Ii6nzF9FeX7NH5QUNeOyk6r25lz4GcEEEAAAQQQQAABBBBwjoCXAlfhsPWlmhdfVIcOHXTFFVd8okh1dXV6/PHHdc0116hPnz6feN1NTxC4clM1WQsCCCCAQFsFCFy1VZDrETi/AIGr8xtxBgIIIIAAAq0WIHDVajouTLGA6Wy053hUG/bGHypK8ZTOObzZRq7A2h1h3ICgiqxtBlN9mMCV2VIw3mPL/pD2HIvvfL+1ll/MapfWDk0vrG7QWxtDqmmIN0IW78pTe56p+xVDs3TDxVmxLRgz0xMstWtkdAQQQAABBBBAAAEEEEi+gJcCV0avqqpK//3f/61Zs2apuLj4I9Co1bZ5wYIFse0GTeDK7QeBK7dXmPUhgAACCCQiQOAqES3ORaB1AgSuWufGVQgggAACCMQlQOAqLiZOyqBAdX1UZQcj2l8eVcQBWZygtc1g345+DeziT2mno5MfdriKtzSJBq7+8+52Kd8msTEkrd4Z0ivrGnX4pLP2DjRBq1F9App6Ybb6W7UmaBXvO5HzEEAAAQQQQAABBBBAwAh4LXBl1my2FTSdrO655x516tTJPKUVK1bIfDZ12223qaCgIPacm/+LwJWbq8vaEEAAAQQSFSBwlagY5yOQuACBq8TNuAIBBBBAAIG4BQhcxU3FiRkWqDgd1fIdYdU3KO5t9DI55aC1teDF/QLqWuyT6RiV7CPRwNWmfY3adzy+UJOZ7y/vzle2tVViKg4TnKuojurXr9bpQHl8c0rFPFozpgladSz060tX5ahPpzTtH9maiXINAggggAACCCCAAAII2FrAi4Er081q27ZtsZDV7bffrmPHjsUCWN/85jdjHa5sXbAkTY7AVZIgGQYBBBBAwBUCBK5cUUYWYXMBAlc2LxDTQwABBBBwtgCBK2fXz2uzb7R2xNt3IqKNByIKxbc7XkaJTHCptMCn4T0DKm2X3KmctAJoZktBEwCK59i4L6T9x+NDM/N+4Av5ygrEOXg8E/jwnGNVUb2wqkHr9oRk6umkozjfp5svydZFVmervOzk2zjJgrkigAACCCCAAAIIIIBA2wS8GLgyYpFIRC+++KJKSkq0e/dujR8/XkOGDGkbpoOuJnDloGIxVQQQQACBlAsQuEo5MTdAQASueBMggAACCCCQQgECVynEZeiUCdRbW9Gt2xPWkcqoQg5okGQCTIO6+tW/s1+5WclhSTRw9f7ekA6ciC/hZOb70Ox2CiSxgVOltQXiim0hzV/ToAarfk46iqyg1fiBQd00LlumcxkHAggggAACCCCAAAIIINBWAa8FrkzQqqysTDt37tSOHTu0fft2BYNBXXTRRerWrZtGjx790TaDbbW18/UEruxcHeaGAAIIIJBuAQJX6Rbnfl4UIHDlxaqzZgQQQACBtAkQuEobNTdKsoC1E4HMtnorrW0Ga6xtBp1w5ASlEb0C6t2h7d2REg1cbbACVwfjDVxZoaJfWYErE7xq62GVKRa0en5lQ6xebR0v3ddPGhzU9LHZap/ni7ubWLrnyP0QQAABBBBAAAEEEEDAeQJeCVzV1NRozZo1sW0Es7KyNGnSJBUXF8dCVtXV1TL/tmzZovfeey/23NVXX60ePXrInOvGg8CVG6vKmhBAAAEEWitA4Kq1clyHQPwCBK7it+JMBBBAAAEEEhYgcJUwGRfYTCBiJXp2Ho1o25GI6hwQvDJbAHYt8umCbn5rm8HWJ5oSDlxZW/gdLI+zw5UVuHrYClzFu11hc2+JsNV5bNP+sF7f0Kiyg/Hdt7lxMvGc6ew1sndAU0dma6DVmYwDAQQQQAABBBBAAAEEEEi2gBcCVyZotW7dOvXv318XXHCBunTpIr+/+b+xGhsbtW/fPq1cuVKhUEjTp09XYWFhstkzPh6Bq4yXgAkggAACCNhIgMCVjYrBVFwrQODKtaVlYQgggAACdhAgcGWHKjCHZAiYgM+qXREdORmRCWHZ/TBRKxPmGWwFr7ICic+24nRUb24Kxx2KWre7UYcr4tt/0QSOHv5iu8Qn9eEVlTURPbm0QWutbR+ddJiAmelk9YUrczSkRyuK4qTFMlcEEEAAAQQQQAABBBDIqICbA1dRqyX1m2++qcWLF+tb3/qW2rdvn5D1kiVL9Nprr+mee+5R9+7dE7rW7icTuLJ7hZgfAggggEA6BQhcpVObe3lVgMCVVyvPuhFAAAEE0iJA4CotzNwkTQJmm8Hj1VG9b3VWOnk6TTdt420Kc30a1M2nPh2a/5ZvS8ObwNVCK3AV77Z/iQSusq2tDx/8QuKBq1O1Uc1/t0Hv7gqrus4BqbczcEsLfLrx4iyN7hdUblbrO4+dMSQ/IoAAAggggAACCCCAAAItCrg5cGW6Wpl/M2fOVH5+fosG53ph165dev3113XLLbfEtiA817lOeo3AlZOqxVwRQAABBFItQOAq1cKMj4BE4Ip3AQIIIIAAAikUIHCVQlyGzqiA2WJwh/Wv1gHbDBqorsU+DevuV1F+fGGfRANXa3c1xrp/xVOUvCzpP++OP3BV1xDV+r0RPbeyXmZeTjpMR6uL+wessFW28nPis3fS+pgrAggggAACCCCAAAII2FPArYGrHTt26Mknn9TXvvY1lZaWthrfdMlavXq13njjDd13333Kzc1t9Vh2upDAlZ2qwVwQQAABBDItQOAq0xXg/l4QIHDlhSqzRgQQQACBjAkQuMoYPTdOg0B9o2LdrvaecEYIKGg1uerd0acRPQMy2/qd60g0cPXezkYdrYxvS8ECK4T0s8/H9y3ksoNhPb64PtZZzHQYc8phtg+cMCiom8dmq6gdQSun1I15IoAAAggggAACCCDgFgE3Bq7q6uo0Z84cjR8/XkOHDk1KqZ577jl17dpVEydOTMp4mR6EwFWmK8D9EUAAAQTsJEDgyk7VYC5uFSBw5dbKsi4EEEAAAVsIELiyRRmYRIoFjlZFtfVQRMdOOSMRVGQFni6wthnsWdpy6irRwNUaK3B1LM7Alemy9dM7zh242nMsrNc2NGrt7rBC4RQXMInDm6DVyN4BXT0ySxd0Dcj8zoEAAggggAACCCCAAAIIpFvAjYGrvXv3xrYSvPHGG5PGWVVVpb/+9a/6/Oc/r2AwmLRxMzUQgatMyXNfBBBAAAE7ChC4smNVmJPbBAhcua2irAcBBBBAwFYCBK5sVQ4mk2KBXcciVseriGMCQqUF0rh+gWa3uks0cPWuFbg6HmfgqkOhXz/+bF6z1ThdH9VLaxq0cGNITuto1aHApzuvyNbg7s7/kL7Z4vAkAggggAACCCCAAAIIOEbAjYGrBQsWqHfv3ho5cmTS6hCJRDR37lxNmTJFnTp1Stq4mRqIwFWm5LkvAggggIAdBQhc2bEqzMltAgSu3FZR1oMAAgggYCsBAle2KgeTSYNATUNUO45Y/45GHBEYyrGyQb06+jW0m1/BwN+BEg1crd7RqBNV8W0p2Lm9T/ff9vEOVyZc9eq6Ri3ZYgW3HNIprEnLdOyaMS5LI/sE1S6HllZNLjwigAACCCCAAAIIIIBA5gTcGLh66KGHYp2oSkpKkgq7YsUKnTx5UtOmTUvquJkYjMBVJtS5JwIIIICAXQUIXNm1MszLTQIErtxUTdaCAAIIIGA7AQJXtisJE0qTQHl1VBusblcmuOSETk0FudZWeD396lLkj22Dl2jgatX2RpWfii9w1a3Epx/M/CBwZbYL3HIwrLnLG3T4ZHzXp6mE571NQa5P4wcGdf3FWcrPJmh1XjBOQAABBBBAAAEEEEAAgbQJuC1wVVtbqwcffFDf+973km64f/9+zZkzR9/5zneSPna6ByRwlW5x7ocAAgggYGcBAld2rg5zc4sAgSu3VJJ1IIAAAgjYUoDAlS3LwqTSJBC28kOHTkb17q6wIlYHJ7sffiszVJzv17gBPtU3Sgs3hWWei+dIJHDVo4Nf3/+HPJ2wQmm/e71O+09Y2zA6K2tlGQV026ScWEcrX5xG8ThyDgIIIIAAAggggAACCCCQDAG3Ba727dun+fPn66tf/WoyeD42RlVVlX784x/rpz/96ceed+IvBK6cWDXmjAACCCCQKgECV6mSZVwE/i5A4OrvFvyEAAIIIIBA0gUIXCWdlAEdKFDXGNXWQ1HtMcEiq6OT3Q+zzWAna9u/fSeisW5X8cx35bZGVVTHl5rqanXRGtYroLfLQlawywFJtA8BTLBqhDXvqSOzdEG3QNw28fhxDgIIIIAAAggggAACCCCQTAG3Ba7Wrl2rjRs36o477kgmU2ys+vp6ffvb39avfvWrpI+d7gEJXKVbnPshgAACCNhZgMCVnavD3NwiQODKLZVkHQgggAACthQgcGXLsjCpDAmcqpOWbwvpdIMcsc1gIkyJBK4SGdcO55qgVaG1feCsK3M0vGfADlNiDggggAACCCCAAAIIIIDAOQXcFrg6cuSInn76aX3jG98457pb8+Lhw4f161//Wvfff39rLrfVNQSubFUOJoMAAgggkGEBAlcZLgC394QAgStPlJlFIoAAAghkSoDAVabkua9dBcw2gwcrotqwP+Ko7k7n81yxtUEnTzunW9X51tP0eoEVtJo+LlsX9w8oP5u9A5tceEQAAQQQQAABBBBAAAF7C7gtcNXQ0KCf/exn+pd/+RdlZWUlFX/Hjh2aN2+e7rvvvqSOm4nBCFxlQp17IoAAAgjYVYDAlV0rw7zcJEDgyk3VZC0IIIAAArYTIHBlu5IwIZsINFpbC76/LxwLXzU4YJvB87EttwJXlS4KXBXm+TSmX0A3j8tRXvb5Vs/rCCCAAAIIIIAAAggggIC9BNwWuDK6Dz/8sGbMmKHu3bsnFXvhwoUKBoO64oorkjpuJgYjcJUJde6JAAIIIGBXAQJXdq0M83KTAIErN1WTtSCAAAII2E6AwJXtSsKEbCRg+kFVW9sMrtoRVmWts7tDLStrUFWNs9fQ9NYY1TegWyfmqKSdT2Y7QQ4EEEAAAQQQQAABBBBAwGkCbgxcLVq06P9n7z7gpKrO/48/07awlGWRJiAIKlKNigrYY0k0xhRBjcYUU9RfmjEaTfsZW5qaGE0zMU3//mKLLUoSe8UGKhIpAtJ7XZZtU//nuctMZnZny8zembl37ue8ssy9d245532GlbBfniMDBgyQQw891LbpiEQicv/998vpp58utbW1tt23VDcicFUqeZ6LAAIIIOBEAQJXTpwV+lRuAgSuym1GGQ8CCCCAgKMECFw5ajrojIMFVm9LyLJNcWlocWdoae6SsDS4ODQW8IscPCIgJ08JyQTzSkMAAQQQQAABBBBAAAEE3CxQjoGrNWvWyAsvvCCf/vSnbZua+vp6efDBB+Uzn/mMBALu//+CBK5s+2hwIwQQQACBMhAgcFUGk8gQHC9A4MrxU0QHEUAAAQTcLEDgys2zR9+LLRCLi7yzJi5rd8RFt93UXl4cNtW63BcW0wJWtaaS1bkzK2WqqWxFQSs3feroKwIIIIAAAggggAACCHQmUI6Bq1gsZlWjGjNmjEyfPr2zoff4uFa3+sMf/iDHHHOMTJ06tcfXOflEAldOnh36hgACCCBQbAECV8UW53leFCBw5cVZZ8wIIIAAAkUTIHBVNGoeVEYCOxsTsnhDXDbXuyfA9NLiiDS2uCslNqCPT848vEJ0CcGaKqJWZfRbiKEggAACCCCAAAIIIOB5gXIMXOmkNjQ0yG233Sbnn3++jB49Ou95jsfj8thjj0ljY6Occ8454vebssdl0AhclcEkMgQEEEAAAdsECFzZRsmNEOhUgMBVpzS8gQACCCCAQO8FCFz13pA7eFdg9daEvLc57orKUS8tCktjqzsCYjWVPjlsbEA+asJW/asJWnn3dxgjRwABBBBAAAEEEECgfAXKNXClM7Zx40Z5+OGHZdasWTJ48OCcJ1HDVvPmzZOFCxfKF77whZyvd/IFBK6cPDv0DQEEEECg2AIErootzvO8KEDgyouzzpgRQAABBIomQOCqaNQ8qEwFIjGR9zbGZdmmuDg5zvSiCVw1uSBwdeiYoJx7dFvQykfWqkx/1zAsBBBAAAEEEEAAAQQQKOfAVSKRkGXLlsnjjz8uRxxxhMyYMUMCgUCPJn379u3y1FNPSTQaldNOO03q6up6dJ1bTiJw5ZaZop8IIIAAAsUQIHBVDGWe4XUBAlde/wQwfgQQQACBggoQuCooLzf3kIAuM7jUBK82mWUGzd8tO6698G5YmsMO7NheqYkjA3LS5JBMMK9+glaO+/zQIQQQQAABBBBAAAEEELBXoJwDV+lSv/3tb6WiokLOPPNMKzylSwP62v3rGg1oxWIxWbx4sTz00ENy6qmnyvTp09NvUzbbBK7KZioZCAIIIICADQIErmxA5BYIdCNA4KobIN5GAAEEEECgNwIErnqjx7UIdBRYvzMhb6+OSTja8b1SHnn+P2FpiTgrcKV/x15b45NzZ1bKIaN79q+dS2nIsxFAAAEEEEAAAQQQQAABuwS8ErjSSlWLFi2ylgfUoJWGrwYOHCh9+/aV5uZmaWpqSn3169dPpk2bJiNHjuwQyrLLvdT3IXBV6hng+QgggAACThIgcOWk2aAv5SpA4KpcZ5ZxIYAAAgg4QoDAlSOmgU6UmUCrCVut2hKXpWaZwVjcGYN7zgSuWh0UuKoMiXzyqEo5bP+A9KuipJUzPiX0AgEEEEAAAQQQQAABBIol4JXAVdJTK1hpwKq1tVV27twp9fX1otWudMnA/v37S3V1tVRVVSVPL9tXAldlO7UMDAEEEEAgDwECV3mgcQkCOQoQuMoRjNMRQAABBBDIRYDAVS5anItAbgK7m0UWro3J9j1meYQSB6+eXdjqiKpbNZU+OWxsUD55ZEiqKwha5faJ4mwEEEAAAQQQQAABBBAoFwGvBa7KZd56Ow4CV70V5HoEEEAAgXISIHBVTrPJWJwqQODKqTNDvxBAAAEEykKAwFVZTCODcLBAwqzit2V3QuatLO0yg04IXE0aFZBPH1sptX18ZnkIB08aXUMAAQQQQAABBBBAAAEECixA4KrAwA69PYErh04M3UIAAQQQKIkAgauSsPNQjwkQuPLYhDNcBBBAAIHiChC4Kq43T/OuQNgsM7jCW8y1PQAAQABJREFULDP4vvnS7WK3Z95plUis2E8VK1g1ft+AnDwlJJNN4IqGAAIIIIAAAggggAACCCAgQuDKm58CAlfenHdGjQACCCCQXYDAVXYXjiJgpwCBKzs1uRcCCCCAAALtBAhctQNhF4ECCzS1JuS19xOyuykucVP9qljtaRO4ihYxcKUVrPpW+eTsGRVyxLhgsYbJcxBAAAEEEEAAAQQQQAABVwgQuHLFNNneSQJXtpNyQwQQQAABFwsQuHLx5NF11wgQuHLNVNFRBBBAAAE3ChC4cuOs0We3C+gyg5vNMoML18ZkT0txRvPUgrDEipTwqjFBqzMOC1lBKw1d0RBAAAEEEEAAAQQQQAABBDIFCFxlenhlj8CVV2aacSKAAAII9ESAwFVPlDgHgd4JELjqnR9XI4AAAggg0KUAgasueXgTgYIKxOIiizfEZe32uLRECvooeWpBqwlcFfYZNZU+mbJfUD5+ZEhq+xC0Kqw2d0cAAQQQQAABBBBAAAE3CxC4cvPs5d93Alf523ElAggggED5CRC4Kr85ZUTOEyBw5bw5oUcIIIAAAmUkQOCqjCaTobhWoDmckPkrY7K1oXBDePLt1oIuYThpVEDOO6ZC6vr6hahV4eaROyOAAAIIIIAAAggggEB5CBC4Ko95zHUUBK5yFeN8BBBAAIFyFiBwVc6zy9icIkDgyikzQT8QQAABBMpSgMBVWU4rg3KZQGskIa8uj8rC1XEZOdgviYT9kaUnTOBKlzK0uw0wlazON0GryaOC4vfbfXfuhwACCCCAAAIIIIAAAgiUpwCBq/Kc1+5GReCqOyHeRwABBBDwkgCBKy/NNmMtlQCBq1LJ81wEEEAAAU8IELjyxDQzSIcK6BJ/68xygn99vlU27Gxb7y8UEJk5PiSVFaZSlI25qyfeMoErGx2iMZGdDXFpbI7LUQcF5WunV8ugfvb22cbucisEEEAAAQQQQAABBBBAwFECBK4cNR1F6wyBq6JR8yAEEEAAARcIELhywSTRRdcLELhy/RQyAAQQQAABJwsQuHLy7NC3chZYuy0uD70RlmUbYxIx4aX2bZ/+fpk2LmgO+3odlNLKVlrhyo4WN/fapUGrlrjE0vo9sMYnHzq0Qj5zYpX4bQyK2dFn7oEAAggggAACCCCAAAIIOE2AwJXTZqQ4/SFwVRxnnoIAAggg4A4BAlfumCd66W4BAlfunj96jwACCCDgcAECVw6fILpXdgLbGhLywuKIPPduRMLR7od38IiA7DsoKBWm8lW+Fao0JPVkLwNXGtpqbElYYatorPOeDB/ol4s+VG2FxYKmzzQEEEAAAQQQQAABBBBAAIGOAgSuOpp44QiBKy/MMmNEAAEEEOipAIGrnkpxHgL5CxC4yt+OKxFAAAEEEOhWgMBVt0ScgIAtArp84L8XRKyv1kjngaVsD6swha6OODAk/av9ouGpXFvMXPTUgnCul6XObwknZOsurWjVs4cH/SLjRwTlB+f0kdo+lLtKQbKBAAIIIIAAAggggAACCOwVIHDlzY8CgStvzjujRgABBBDILkDgKrsLRxGwU4DAlZ2a3AsBBBBAAIF2AgSu2oGwi4DNAloZ6tVlUXnmPxFZu92krnrRhgzwywHDAzKgT27BK61I9fQ7uQeuNBi2uzEhTWb5QB1Hrq1vlU8+ekSFzJ5ZKX0qCV7l6sf5CCCAAAIIIIAAAgggUL4CBK7Kd267GhmBq650eA8BBBBAwGsCBK68NuOMtxQCBK5Koc4zEUAAAQQ8I0DgyjNTzUCLLBA32ar1O+Pyx2daZNOuPNJKXfT3IBO6Gj00IH5fz0JM4WhCnl3Y88CV9n1XQ1x2N/UuIJYcQrUJW/1gdh+ZMjooIZYZTLLwigACCCCAAAIIIIAAAh4WIHDlzckncOXNeWfUCCCAAALZBQhcZXfhKAJ2ChC4slOTeyGAAAIIINBOgMBVOxB2EbBBYJsJKz3wSlgWb4hJa8SGG2a5RXWFTw40watRgwMSjWU5Ie2Q9uG5/7SmHcm+qVWs6k1Fqz1NsW7vmf0OnR8NBXwmcBWQSz5cLaP2MWsO0hBAAAEEEEAAAQQQQAABDwukB66uuuoqT0ls2rRJ9uzZIxUVFbLffvt5auw/+clPrPHOmjVbZs2e7amxM1gEEEAAAQTaCxC4ai/CPgL2CxC4st+UOyKAAAIIIJASIHCVomADgV4L7NiTkJeXRuXfC8K2B5Y669ygfj45aN+gDKjxd7rsX4spbvX8u50HruImaNXSmpCdJigWMdWwCtkCJmt17rFV8pHDQ1LXl+BVIa25NwIIIIAAAggggAACCDhXID1w5dxe0rNCCRC4KpQs90UAAQQQcJMAgSs3zRZ9dasAgSu3zhz9RgABBBBwhQCBK1dME510gcDz70bk4XkRaQknpLCRpY4YurLgyEEBOXT/kLRmCUw1mTDVi4uyLykYMdWxtu6MSThSvF5rf2vMMoOXfrRajpkQ6jggjiCAAAIIIIAAAggggAACZS6waNG7cu0115T5KBleZwIErjqT4TgCCCCAgJcECFx5abYZa6kECFyVSp7nIoAAAgh4QoDAlSemmUEWSCAWF3lrZVSeWhiRVVvNTolbZcgn44YFZMyQgGjfkq2pNW4CV5lrG4ZNMGu3qci1p8WcWLysVbJL1qsGr448ICjnHFMlE0cFMt5jBwEEEEAAAQQQQAABBBDwgoAGr7zW7r/vPlm8eLEMHjxYLvmf//Ha8K3xTpw4yZPjZtAIIIAAAgikCxC4StdgG4HCCBC4Kowrd0UAAQQQQMASIHDFBwGB3AV0Cb6djQn5y3OtsmyjKRHlsNa/2iczx1dIwgSatDWaUNVLi9sCV3GTr2poisuuPfFOlyBsu6q4v541o1I+dWyl9K3a2+niPp6nIYAAAggggAACCCCAAAIIFEngFz+/WV577TUZOXKk3HTzz4v0VB6DAAIIIIAAAk4TIHDltBmhP+UoQOCqHGeVMSGAAAIIOEaAwJVjpoKOuERgh6kK9dDrYXlnddQs3+fsTu9b57eWGdxjlhR8bmFYdpuQmIatorESlbTqhmvYQL/MMsGrM6ZVdHMmbyOAAAIIIIAAAggggAACCLhVgMCVW2eOfiOAAAIIIGCvAIErez25GwLZBAhcZVPhGAIIIIAAAjYJELiyCZLblL1AfVNCXl8elTlvRaQ57MzAUrZJqK7wyWmHheTR18KyfIPDE2J7B3DImKB89sQqOXhEQPz+bKPiGAIIIIAAAggggAACCCCAgFsFCFy5deboNwIIIIAAAvYKELiy15O7IZBNgMBVNhWOIYAAAgggYJMAgSubILlN2QokTLbq9RVReeDVsDQ0uydopRMypL9PvvDBShk1KGCFxP7yTKs8Nr/VUUsJdvbBCQVEjptUIZd/rFp8rDLYGRPHEUAAAQQQQAABBBBAAAHXCRC4ct2U0WEEEEAAAQQKIkDgqiCs3BSBDAECVxkc7CCAAAIIIGCvAIErez25W/kIxE22aqFZNvCp/0Rl+caYuClqNWSAX06eHJQjDgiKVrhKbzqm+15ulfnvRyUeT3/HmduD+5tlBmdWWlW6KoKZY3Fmj+kVAggggAACCCCAAAIIIIBAVwIErrrS4T0EEEAAAQS8I0DgyjtzzUhLJ0DgqnT2PBkBBBBAwAMCBK48MMkMMScBDVZt2x2Xe+eG5T9rYzldW+qTNVx1zMFBOXNaSEKBzsNJWrXrqXci8qenW2TnHhekrgzsmCEB+c5ZfWTkIL8EWGaw1B81no8AAggggAACCCCAAAII5C1A4CpvOi5EAAEEEECgrAQIXJXVdDIYhwoQuHLoxNAtBBBAAIHyECBwVR7zyCjsEahvSsiDr4dl4ZqYNLW6qaaVyLETgnLylJBZRtDf4yX4tjfEZc78iDzwSqu0Rpw/3qqQT2aaQNklp1VLv6rOA2X2fBq4CwIIIIAAAggggAACCCCAQCEECFwVQpV7IoAAAggg4D4BAlfumzN67D4BAlfumzN6jAACCCDgIgECVy6aLLpaMIHGloS8vjwqc96OSEOz84NHSQi/yRyN3zcgZxweknFDA8nDOb8uWR+TvzzbIgtXRSXmgoJXfUzY6osnVclxk0LSl+BVzvPNBQgggAACCCCAAAIIIIBAKQUIXJVSn2cjgAACCCDgHAECV86ZC3pSvgIErsp3bhkZAggggIADBAhcOWAS6EJJBd42IaO7XghLUzghutSeW1r/ap9cdEqljBkcsGWJPQ1azX8/Kjc/3Cz1Tc5PXflM2Gz4QL/879k1ZrlB1hh0y+eWfiKAAAIIIIAAAggggAACBK74DCCAAAIIIICAChC44nOAQOEFCFwV3pgnIIAAAgh4WIDAlYcn3+NDX7QuJk+9E5ElG2MSd36+KDVbtX18cuohIZlxUEiqK1KHbdvY0ZCQR+eF5bE3WmWPqfzl9BYwWauTplbIucdUyr51BK+cPl/0DwEEEEAAAQQQQAABBBAgcMVnAAEEEEAAAQRUgMAVnwMECi9A4KrwxjwBAQQQQMDDAgSuPDz5Hhy6VrDa2ZiQB15tlTdXxlwlUBXyyRHjgnLesRViijsVvK3fEZebTLWr9za4Y5lBDV595bRqOXGKBtGKIVTwKeABCCCAAAIIIIAAAggggEBZChC4KstpZVAIIIAAAgjkLEDgKmcyLkAgZwECVzmTcQECCCCAAAI9FyBw1XMrznS3QCQq8vfXW2XeipgrKjcltTU6dMQBQfnwB0IyvNYvupResVrYmL29Miq3P9Ei67c7P6CmNmOHBuTzJ1XJNBNOoyGAAAIIIIAAAggggAACCDhPgMCV8+aEHiGAAAIIIFAKAQJXpVDnmV4TIHDltRlnvAgggAACRRUgcFVUbh5WAoHG1oS8ZUJDD70RkUYXLJGXJNKKTfsPCchHDgvJhBGB5OGSvDY0J+T/XmiVZxaGpb7J+csMKtLJZpnBs4+ulFGDTUitJGo8FAEEEEAAAQQQQAABBBBAIJsAgatsKhxDAAEEEEDAewIErrw354y4+AIEropvzhMRQAABBDwkQODKQ5PtwaEu3xSTPz3bai0jqMsJuqVVV4hceGKVTBwZEA1eOaVtb4jLzY80y5vvm9JXLmiVZhnGc0zo6rzjKl3QW7qIAAIIIIAAAggggAACCHhDgMCVN+aZUSKAAAIIINCdAIGr7oR4H4HeCxC46r0hd0AAAQQQQKBTAQJXndLwhksFNFilQasnF0Zk4eqYuChnJQNrfHLCpJAcc3BIahyaEYqalQVfWRqRe19uleUbnb/MoH6MDxgesIJXM42rkwJsLv0tRrcRQAABBBBAAAEEEEAAgV4JELjqFR8XI4AAAgggUDYCBK7KZioZiIMFCFw5eHLoGgIIIICA+wUIXLl/DhlBpsBus/zdb59skbXb4qLhIDe0ypDI5JFBOe/YChO0cscCeJFoQu58vlXmzA+7ZqnG6QeF5OtnVFvBNp87mN3w8aWPCCCAAAIIIIAAAggggEBOAgSucuLiZAQQQAABBMpWgMBV2U4tA3OQAIErB00GXUEAAQQQKD8BAlflN6eMSCQWF1m0Lib3vhKWbbvNjkObhn4O2z8gpx9aISPqHLR2YA5e67bH5e4XWuX5/4Ql7oJyYv2qffIh433B8ZWiSw7SEEAAAQQQQAABBBBAAAEEiitA4Kq43jwNAQQQQAABpwoQuHLqzNCvchIgcFVOs8lYEEAAAQQcJ0DgynFTQodsFAhHRR5/MyyvLotKfZOz0kBjh/jlQ4eEZMrooPhdnvvRoNXLiyNyz0utssIs5+jkphXEjp8cMoGrKhnY1+XwToambwgggAACCCCAAAIIIIBAJwIErjqB4TACCCCAAAIeEyBw5bEJZ7glESBwVRJ2HooAAggg4BUBAldemWlvj1PDVn96plWWbix9GEgDP589sVImjQhIwJ1FrTr9MEUMry4x+MenWiRslhx0WjtsXEi++8lqqanyCUsKOm126A8CCCCAAAIIIIAAAgh4RYDAlVdmmnEigAACCCDQtQCBq659eBcBOwQIXNmhyD0QQAABBBDoRIDAVScwHC5LgbdXxeRfb4dl1dbiLzNYUyly0pQKmTk+KLV9yruy0qotcXnw1VZ5ZmFYoqXPuFlVxGbNqJRpBwTLLuRWlr9RGRQCCCCAAAIIIIAAAgiUtQCBq7KeXgaHAAIIIIBAjwUIXPWYihMRyFuAwFXedFyIAAIIIIBA9wIErro34ozyEoiZrNUcs8zgs4ui0tRa+CpMFUGfTNkvIJ8+tkKqK8o7aNX+k7JgZUx+9c8mWbctLoWXzny6VrCq6+uXz51YJad8IJT5JnsIIIAAAggggAACCCCAAAIlEyBwVTJ6HowAAggggICjBAhcOWo66EyZChC4KtOJZVgIIIAAAs4QIHDljHmgF8UX2LI7If801a5efS8qiQKlgSaNDMjHj6yQfQf6PVtZqaE5Ia8sjcgfnmwR3S5GCwXFBK2q5YNTQzKwxlsht2L48gwEEEAAAQQQQAABBBBAoDcCBK56o8e1CCCAAAIIlI8AgavymUtG4lwBAlfOnRt6hgACCCBQBgIErspgEhlCrwQWronJ46bi1RpThSluQx5IKyvtt49fTp1aIYeNDQhxn7bp2bAjLnc+1ypzl0QkHLUBOsus11T5ZPr4kHz+xErZp78/yxkcQgABBBBAAAEEEEAAAQQQKLUAgatSzwDPRwABBBBAwBkCBK6cMQ/0orwFCFyV9/wyOgQQQACBEgsQuCrxBPB4RwhEYyIvL43Kfa+0ii45mG/TsNVnj6+UaeOCEiTv04FRA226vOC19zXKuu29gO5wZ5GxQwPy/bP7yPBav+g80BBAAAEEEEAAAQQQQAABBJwpQODKmfNCrxBAAAEEECi2AIGrYovzPC8KELjy4qwzZgQQQACBogkQuCoaNQ9ygcCW+rg8tTAiry6LmipMPe9wdYXICZNCcuLEkPTvQ9qnO7k9LQl5xjjf93KrbNudf/BKg1UHjwjIWTMq5eiDQwStuoPnfQQQQAABBBBAAAEEEEDAAQIErhwwCXQBAQQQQAABBwgQuHLAJNCFshcgcFX2U8wAEUAAAQRKKUDgqpT6PNupAis2meXvXmiRrbsTXS4zWBEUGTcsIJ83Va0IWuU+mw3NCbltTrO8aqqL5brMYG2NT84+pko+dkSFBKgmljs+VyCAAAIIIIAAAggggAACJRIgcFUieB6LAAIIIICAwwQIXDlsQuhOWQoQuCrLaWVQCCCAAAJOESBw5ZSZoB9OE2iNJOTNlTH5+2th0YpM7duBJmj18SMrZP8hfvFT1Ko9T4/3dQnHFZti8rt/t8iitd2XFauqMEGrmZXyoUMrZFA/4HsMzYkIIIAAAggggAACCCCAgEMECFw5ZCLoBgIIIIAAAiUWIHBV4gng8Z4QIHDliWlmkAgggAACpRIgcFUqeZ7rFoFdjQl5+I1WWbA6Li0mhDWizi8nTQ7J9AODLGFn4yTqEo6PzQvLI6+3yuZdHZcZ1KDVEQcE5dPHV8nowZS0spGeWyGAAAIIIIAAAggggAACRRUgcFVUbh6GAAIIIICAYwUIXDl2auhYGQkQuCqjyWQoCCCAAALOEyBw5bw5oUfOE9D6Vlvq47J+e1ymjg5KMOC8PpZLj1oiIr831a7++VarJPYWFhtrqol996xqE3YLEHIrl4lmHAgggAACCCCAAAIIIOBZAQJXnp16Bo4AAggggECGAIGrDA52ECiIAIGrgrByUwQQQAABBNoECFzxSUAAAacJxE3Q6j+ro/Lqe1E5aN+AzBgflMoQywc6bZ7oDwIIIIAAAggggAACCCCQjwCBq3zUuAYBBBBAAIHyEyBwVX5zyoicJ0DgynlzQo8QQAABBMpIgMBVGU0mQ0EAAQQQQAABBBBAAAEEEEAAAQQcLkDgyuETRPcQQAABBBAokgCBqyJB8xhPCxC48vT0M3gEEEAAgUILELgqtDD3RwABBBBAAAEEEEAAAQQQQAABBBBIChC4SkrwigACCCCAgLcFCFx5e/4ZfXEECFwVx5mnIIAAAgh4VIDAlUcnnmEjgAACCCCAAAIIIIAAAggggAACJRAgcFUCdB6JAAIIIICAAwUIXDlwUuhS2QkQuCq7KWVACCCAAAJOEiBw5aTZoC8IIIAAAggggAACCCCAAAIIIIBAeQsQuCrv+WV0CCCAAAII9FSAwFVPpTgPgfwFCFzlb8eVCCCAAAIIdCtA4KpbIk5AAAEEEEAAAQQQQAABBBBAAAEEELBJgMCVTZDcBgEEEEAAAZcLELhy+QTSfVcIELhyxTTRSQQQQAABtwoQuHLrzNFvBBBAAAEEEEAAAQQQQAABBBBAwH0CBK7cN2f0GAEEEEAAgUIIELgqhCr3RCBTgMBVpgd7CCCAAAII2CpA4MpWTm6GAAIIIIAAAggggAACCCCAAAIIINCFAIGrLnB4CwEEEEAAAQ8JELjy0GQz1JIJELgqGT0PRgABBBDwggCBKy/MMmNEAAEEEEAAAQQQQAABBBBAAAEEnCFA4MoZ80AvEEAAAQQQKLUAgatSzwDP94IAgSsvzDJjRAABBBAomQCBq5LR82AEEEAAAQQQQAABBBBAAAEEEEDAcwIErjw35QwYAQQQQACBrAIErrKycBABWwUIXNnKyc0QQAABBBDIFCBwlenBHgIIIIAAAggggAACCCCAAAIIIIBA4QQIXBXOljsjgAACCCDgJgECV26aLfrqVgECV26dOfqNAAIIIOAKAQJXrpgmOokAAggggAACCCCAAAIIIIAAAgiUhQCBq7KYRgaBAAIIIIBArwUIXPWakBsg0K0AgatuiTgBAQQQQACB/AUIXOVvx5UIIIAAAggggAACCCCAAAIIIIAAArkJELjKzYuzEUAAAQQQKFcBAlflOrOMy0kCBK6cNBv0BQEEEECg7AQIXJXdlDIgBBBAAAEEEEAAAQQQQAABBBBAwLECBK4cOzV0DAEEEEAAgaIKELgqKjcP86gAgSuPTjzDRgABBBAojgCBq+I48xQEEEAAAQQQQAABBBBAAAEEEEAAARECV3wKEEAAAQQQQEAFCFzxOUCg8AIErgpvzBMQQAABBDwsQODKw5PP0BFAAAEEEEAAAQQQQAABBBBAAIEiCxC4KjI4j0MAAQQQQMChAgSuHDoxdKusBAhcldV0MhgEEEAAAacJELhy2ozQHwQQQAABBBBAAAEEEEAAAQQQQKB8BQhcle/cMjIEEEAAAQRyESBwlYsW5yKQnwCBq/zcuAoBBBBAAIEeCRC46hETJyGAAAIIIIAAAggggAACCCCAAAII2CBA4MoGRG6BAAIIIIBAGQgQuCqDSWQIjhcgcOX4KaKDCCCAAAJuFiBw5ebZo+8IIIAAAggggAACCCCAAAIIIICAuwQIXLlrvugtAggggAAChRIgcFUoWe6LwH8FCFz914ItBBBAAAEEbBcgcGU7KTdEAAEEEEAAAQQQQAABBBBAAIFeCCxa9K4seneR6GuyLVq0KLnJKwIIOFBg4sSJVq8mTpyU6t2s2bNT22wUTiDb90x9Gt83C2fOnRGwQ6D9982JkyZK+vdQO57h9HsQuHL6DNG/chAgcFUOs8gYEEAAAQQcK0DgyrFTQ8cQQAABBBBAAAEEEEAAAQQQ8IRAMlj1wP33ExDwxIwzSC8JaKBAAwReDBIUcp71+ybfMwspzL0RKJ3ArFltYVUvhFYJXJXuc8aTvSNA4Mo7c81IEUAAAQRKIEDgqgToPBIBBBBAAAEEEEAAAQQQQAABBKwKVl0FBsaPH5+h1H4/4012EECgpAJLly5NPT99O3XQbHgpRJA+bju39XvmAw/c3+kt079Ppm93egFvIIBAyQSS3yuTr9k6ot83yzl4ReAq26xzDAF7BQhc2evJ3RBAAAEEEMgQIHCVwcEOAggggAACCCCAAAIIIIAAAggUWKCzyizJcMCZZ54pye0Cd4XbI4BAgQQ0QJD+lf4YDRBQ8SpdpPvtrr5v6vdMbXzf7N6RMxBwssCjjz5qdS/5mt7Xcg1eEbhKn2W2ESiMAIGrwrhyVwQQQAABBCwBAld8EBBAAAEEEEAAAQQQQAABBBBAoFgC2aqzaEiAkFWxZoDnIFAaAQ0QtA8RlGuAwE7hbEErvmfaKcy9EHCmQPL7ZfI12cv/vfpqa5nW5L7bXwlcuX0G6b8bBAhcuWGW6CMCCCCAgGsFCFy5duroOAIIIIAAAggggAACCCCAAAKuEmgftiI04Krpo7MI2CLQPng1ceJE+d+rf2jLvcvtJhq2uvaaa1LD4ntmioINBDwj0P57pg68nMKqBK4881FmoCUUIHBVQnwejQACCCBQ/gIErsp/jhkhAggggAACCCCAAAIIIIAAAqUWSP5ATftBaKDUs8HzESi9wI033mgtOZjsSblVbUmOK9/X9gFVrQKYXDow33tyHQIIuFegffCqXEJXyT8fTp48Wa6//nr3ThA9R8DBAgSuHDw5dA0BBBBAwP0CBK7cP4eMAAEEEEAAAQQQQAABBBBAAAGnCmSr0HLFFVc4tbv0CwEEiijQPkBA6KoNn7BVET+EPAoBFwm0/55ZDqErAlcu+gDSVdcKELhy7dTRcQQQQAABNwgQuHLDLNFHBBBAAAEEEEAAAQQQQAABBNwpcO45Z6c6ToWWFAUbCCCwV6B9gMDroav2YSsNqGpVQBoCCCCgAkuXLhX9vqmv2tweuiJwZU2jp35JmNE2tSZkT0tCGs2rboejCYnEEhKLt1EE/CKhoE8qzVefCp/UVPqkb5VPqs2rz1Na9gyWwJU9jtwFAQQQQACBrAIErrKycBABBBBAAAEEEEAAAQQQQAABBHopkPwhmt6GsFUvMbkcgTIWSA9dTZw4Uf736h+W8Wg7H1p62EpDVlQD7NyKdxDwukD6sqxuDl0l/6zIkoLl/YnWkFXC/BI1oapITKS+KSG7GuPS0JyQhpa4tEQS0hoVE7jSM0WCfhO2ColUhXzSr8ov/at9UltjXvv4JBTwiQayfCZ5RfjK4ur2FwJX3RJxAgIIIIAAAvkLELjK344rEUAAAQQQQAABBBBAAAEEEEAgu0B6cICwVXYjjiKAwH8FvB66ar/8KpWt/vvZYAsBBLILfPGLX0y94dbqgASuUlNY1htauUqrWO3YE5ct9XFT3SpuVbfS8JWGsPR982JCWW2BK59JU2moymSrrIBVKCimypXfqnI1tDYgtSZ4VWGqX+k5tO4FCFx1b8QZCCCAAAII5C1A4CpvOi5EAAEEEEAAAQQQQAABBBBAAIEsAulhK6q0ZAHiEAIIZBVID125uWJL1sF1c5DlV7sB4m0EEOggoMsKaqWrZHNj6IrAVXL2yvNV81MapGoOJ6yqVlvqY7JxZ8yqaKUBrFyaBqx0ecHhAwMyZEDAqnpVbfZNMSyr2lUu9/LauQSuvDbjjBcBBBBAoKgCBK6Kys3DEEAAAQQQQAABBBBAAAEEECh7gfTgwB133FH242WACCBgn0D6MlluDA/kI5EeUqUiYD6CXIOAdwXSg6puXJKVwFV5f3a1cpWGrbY1xGXttpjUN8et/bhZOnDv6oE9BtBgVcD8oiErXV5w1KCA1PX1W/tUuuqakcBV1z68iwACCCCAQK8ECFz1io+LEUAAAQQQQAABBBBAAAEEEEAgTYDgQBoGmwggkLNAesUWN4YHch6wuSAZUiVslY8e1yCAgJuDqgSuyvfzq4GqFhO22ro7Lpu1stUuU9nK7OcatGovpMGrPmZ5Qa10NXSAX/bp55eqkI8qV+2h0vYJXKVhsIkAAggggIDdAgSu7BblfggggAACCCCAAAIIIIAAAgh4V4DggHfnnpEjYJdAesWWcq9ylR5SveKKK0SXYaUhgAACuQi4OahK4CqXmXbPubpYYCwmsqsxLss2x0zoSsNWcdGKV3a0YMAnVabS1VCztOABQ4PW8oLBgB13Ls97ELgqz3llVAgggAACDhEgcOWQiaAbCCCAAAIIIIAAAggggAACCLhcgOCAyyeQ7iPgIIEvfvGLVm/KucpV+vdMqls56MNHVxBwoYBbg6oErlz4YetBlzVY1dCs1a1isnJrTHY1xSUeS4gGsexoPlPlym9KXQ0ySwqOGRy0qlz1q/aJVr+idRQgcNXRhCMIIIAAAgjYJkDgyjZKboQAAggggAACCCCAAAIIIICApwWobuXp6WfwCNgq4NbwQC4I6YGrO+64I5dLORcBBBDoIODGoCqBqw7TWBYHwtGEbNgZl407Y7LFhK6aWu2KWmXy9K3yW8sKDq8NWEsMUuUq0ye5R+AqKcErAggggAACBRAgcFUAVG6JAAIIIIAAAggggAACCCCAgMcE0oMDLIvlsclnuAgUSMCN4YFcKAip5qLFuQgg0J3AjTfeKLq8oLZ77r2vu9Md8T6BK0dMg+2daI4kZPmmqGzYEZOGloRETACrEK0y6JO+prLVqEFBGTc0IBVmn9ZRgMBVRxOOIIAAAgggYJsAgSvbKLkRAggggAACCCCAAAIIIIAAAp4VSA9cUanFsx8DBo6ArQLJ8EA5Liu4aNG7cu0111heLCdo68eGmyHgWQENW+n3TW3/e/XVMnHiJMdbELhy/BTl3EGNVjWailb/WRORdSZwFTNLCcYLk7eylhAMBnwyenBAJo8KSWXIJ0SuOk4ZgauOJhxBAAEEEEDANgECV7ZRciMEEEAAAQQQQAABBBBAAAEEPCtApRbPTj0DR6BgAm4MD/QUg5BqT6U4DwEEchFwW2VAAle5zK47ztXlBHc3J2Tx+ohZVjAmCRO20q9CNJ9JV+mXVriaMCIoNVU+qTABLFqmAIGrTA/2EEAAAQQQsFWAwJWtnNwMAQQQQAABBBBAAAEEEEAAAc8JpAcHqNTiuelnwAgUTKCcA1fJkMH48eNFl2GlIYAAAnYIuK0yYPJ74eTJk+X666+3g4B7lFBAc1VNprrVzsa4vLchKpvrY0Xpzb51ATloeFAG9PFLnwoCV+3RCVy1F2EfAQQQQAABGwUIXNmIya0QQAABBBBAAAEEEEAAAQQQ8KBAeuCK5QQ9+AFgyAgUUMBt1Vp6SkFVwJ5KcR4CCOQi8Oijj4p+abvn3vtyubQk5xK4Kgl7wR6qlawaWhKyY09clm+KytbdxQlcDa8NyLhhQRlY45e+psoVLVOAwFWmB3sIIIAAAgjYKkDgylZOboYAAggggAACCCCAAAIIIICA5wQIXHluyhkwAkUTcFu1lp7CELjqqRTnIYBALgJuqwxI4CqX2XX+uRq40uUEtzfE5f0txQtcDTOBq7FD2wJX/asJXLX/pBC4ai/CPgIIIIAAAjYKELiyEZNbIYAAAggggAACCCCAAAIIIOBBAQJXHpx0hoxAkQTKMXC1aNG7cu0111iCLMNapA8Sj0HAIwIErjwy0Q4dplXhSgNXpsLVis3FDVyN2xu46kfgqsOng8BVBxIOIIAAAgggYJ8AgSv7LLkTAggggAACCCCAAAIIIIAAAl4USFYnGD9+vFxxxRVeJGDMCCBQIIFyD1zp90z93klDAAEE7BAgcGWHIvfIV8AUuJJGs6Tgzsa4LNsYlc31xVlScN+BATlweEgG9PFJTSUVrtrPH4Gr9iLsI4AAAgggYKMAgSsbMbkVAggggAACCCCAAAIIIIAAAh4UIHDlwUlnyAgUSeDRRx8V/dJ2z733FemphX1MelVAAleFtebuCHhR4Itf/KI17FmzZsus2bMdTZD8M+TkyZPl+uuvd3Rf6Vz3Ahq4ao0kpL4pIYvXRWTjruIErkYOCsiEESHpV+WTyhCBq/YzReCqvQj7CCCAAAII2ChA4MpGTG6FAAIIIIAAAggggAACCCCAgAcFkj8so8KVByefISNQYIFyD1zdcccdBRbk9ggg4DUBAldem3FnjTdqMlYNLXFZtC4qG3bGJBZPiC41WIjmN9mqgPll1D5tgas+FT4JBgrxJHffk8CVu+eP3iOAAAIIOFyAwJXDJ4juIYAAAggggAACCCCAAAIIIOBwAQJXDp8guoeAiwUIXLl48ug6AgiURIDAVUnYeWiaQFNrQhatj8r6HVGr4lUsnvamjZtBv5iKVn7ZzwpcBalu1YktgatOYDiMAAIIIICAHQIEruxQ5B4IIIAAAggggAACCCCAAAIIeFeAwJV3556RI1BoAQJXhRbm/gggUG4CBK7KbUbdNx5dVnD1tphsNBWuduyJS4vZL0TTilZ1ff2yb13AhK6CEqK6VVZmAldZWTiIAAIIIICAPQIEruxx5C4IIIAAAggggAACCCCAAAIIeFWAwJVXZ55xI1B4AQJXhTfmCQggUF4CBK7Kaz7dOJpoLCHbG+KyuT4ua7fHZHez/SWuzGqCMqDGVLcaFJAhAwJW8CpgKl7ROgoQuOpowhEEEEAAAQRsEyBwZRslN0IAAQQQQAABBBBAAAEEEEDAkwIErgo77bFYTOrr66Wurq6wD+LuCDhQgMCVAyeFLiGAgKMFCFw5eno80bm4KWjVEk5Y1a3e3xK1XrXqlR63o/l1KcGgTwb3D8jYIQGpNcGrqpBPfJrConUQIHDVgYQDCCCAAAII2CdA4Mo+S+6EAAIIIIAAAggggAACCCCAgBcFCFzZN+vhcFjmzJkjTzzxhKxdu1bWrVsnGzduFA1d9enTR8aMGSP777+/9frhD39Ypk+fbt/DuRMCDhQgcOXASaFLCCDgaAECV46eHs90TrNVDS0JWWUCV1tMpav6prhETOWrRC9DVz4TqqowYasBffwyrDYg+w8OSJ9KklZdfbAIXHWlw3sIIIAAAgj0UoDAVS8BuRwBBBBAAAEEEEAAAQQQQAABjwsQuOr9B2Dp0qVy1113yf333y87duzo8Q2PPfZYufLKK+Woo47q8TWc2LnAhg0b5Omnn5YLLrig85N4p6gCBK6Kys3DEECgDAQIXJXBJJbJEMJRsYJWW3fHZf2OqBXACptKV/lmrjRspZWt+lf7ZURdwFS48kt/E7wKBcoErEDDIHBVIFhuiwACCCCAgAoQuOJzgAACCCCAAAIIIIAAAggggAACvREgcJW/XsL8M/+f//zn8rOf/Uzi8XjeNzr++OPl9ttvl0GDBuV9Dy9fGIlE5He/+53cdNNNopZ33nmnlzkcNXYCV46aDjqDAAIuECBw5YJJ8kgXtZpVzPzxdmdjXNbtiMnOPXETuoqLBrGiptpVLi0U8JnKViL9TNiqrq9fRprAlVa5CpjlBTWIRetcgMBV5za8gwACCCCAQK8FCFz1mpAbIIAAAggggAACCCCAAAIIIOBpAQJX+U2/VrK6+OKL5dlnn+30BgMHDpTRo0dLbW2ttcSgLjOoyw5ma1OmTJGHH35Y+vfvn+1tjnUisHjxYtEfTr/33nvWGaeddhqBq06sSnGYwFUp1HkmAgi4WYDAlZtnr/z6rqGrcDQhTeGE7DCBK612pQGsXU3mjZ6sL2jCVD6TqKrt45OBNX4Z0j9gXn1mGcG2ylaErbr/zBC46t6IMxBAAAEEEMhbgMBV3nRciAACCCCAAAIIIIAAAggggAACRoDAVe4fg3Xr1skZZ5wh69ev73Dx9OnT5Rvf+Ia1TGC/fv0y3tcqWBoQuuWWW0SDKO2rYh155JHywAMPSHV1dcZ17HQu8Je//EWuuOKK1AkErlIUjtggcOWIaaATCCDgIgECVy6aLI90VWtZabaqoTlhha027IyZJQZjZj8uTa0Jq0pVwO9LVapqq4yVsKpj9TUVrWpNJavhA/0yrDYgdSZ0VVPVdi6FrXr2ASJw1TMnzkIAAQQQQCAvAQJXebFxEQIIIIAAAggggAACCCCAAAII7BUgcJXbR0FDUp/4xCdk7ty5GRdOmDBBbrzxRitolfFGJztLliyR8847z6p8lX7KqaeeKnfffXf6Iba7ECBw1QWOA94icOWASaALCCDgKgECV66aLk91VpcX1KUEV26JybJNUVm2ISLrtsek2lSrqgz5JBgwHCaYFTXntZqKWE3huIwbGpKDR4Zk7JCAjBjkl6AJZukygrSeCxC46rkVZyKAAAIIIJCzAIGrnMm4AAEEEEAAAQQQQAABBBBAAAEE0gQIXKVh9GDz1ltvleuuuy7jTF0OUCtT1dXVZRzvbmfFihVy+umniy5PmN7mzJkjRxxxRPohtjsRIHDVCYxDDhO4cshE0A0EEHCNAIEr10xVrztqMvxi8ktS3xSX3WaJvmYTUGqJJEwFVCu3ZIJJPqkIilSZMFN/UyVKK0VpqKnUgaX3N0dlyfqovLE8LEvWRU0f2/rp3xuk0mBWJGpCV2Ysh44NyfSDKmXcsICMHKSJrNI1DYJFzPKI6q3VuppNKCwSM3MQT4hW21JvDY5VV/hkgFn2sL+pzqXW5nBJG4GrkvLzcAQQQACBchcgcFXuM8z4EEAAAQQQQAABBBBAAAEEECisAIGrnvsuXLhQPvShD0kkEkldNHHiRHnkkUektrY2dSyXjXnz5lmhq4Suv7K36XKFf/7zn5O7vHYhQOCqCxwHvEXgygGTQBcQQMBVAgSuXDVdvepsWENJJgC0YlPMfEVly+6Y7NwTt4JK+sfCChP+GdDHJ/v088vYoUE5cHhI+lT6TACrV4/t9cXvb47JeyZw9czCFplvQleaVmqfSbL+VGt+OXpipZx8SJXpf+kDVxpm06DVso1RWbUlKlsb2oJXYXNcw2LqPaivX4b0D8gBw4Oyv6nKpQGsUGlzYkLgqtcfWW6AAAIIIIBA5wIErjq34R0EEEAAAQQQQAABBBBAAAEEEOhegMBV90bJMy666CJ58MEHk7vWqwZKZsyYkXEs153zzz9fnnjiidRlfvNTn1dffVX233//1DE2sgsQuMru4pSjBK6cMhP0AwEE3CJA4MotM5V/P1tMZaXdLQnZsCMma82SfJt2xmVLfUz2mGO6DF/UVF3SpfkCJuij1ZZqTMhqsAkBDa1tCy2NrPObIJZfaqrax5zy71MuV640gaulJnD19Dst8sYyE7jqoh1rAlenHFrawJW67mqMW8sf6hKIm+vjss2E2xpbE1ZFsZjx9hnKYMBnBdr6VvllWK35GhiQUaYql273r26rftXFUAv2FoGrgtFyYwQQQAABBEQIXPEpQAABBBBAAAEEEEAAAQQQQACB3ggQuOqZXlNTkxx88MHS3NycuuD444+3lhJMHchzY+7cufKxj30sdfXAgQPl6quvFg1itW/PPPOMxPQnQ6bpedOmTbO2N2zYIL/73e9k/vz51v5BBx0kJ510klU9SwNcXTVd0vCpp56SF198UTZt2iTbtm2TyspKGTx4sDXmU045RQ4//HDzg7/8/4m/9nnBggXy8ssvW3+ftXPnTmspRa0WptXBBgwYIOPHj5eZM2daX/r8ztratWtlyZIl1ttPP/20/PGPf0ydqh6XXXZZar9v3749CsQVwyDVKQ9tELjy0GQzVAQQsEWAwJUtjI69iVZ+2mEqK60zYasFqyIyb0VYNICl1a66akHzR7mQWbpv6uiQHDImZFVfGmYCWBoUKnZzU+BKK4VpsG3llpi8tTIi766NmGUEE2YZwa7VtLJVX7Ok4OFmScQp+4XMcohmSccaf4dKXl3fxZ53CVzZ48hdEEAAAQQQyCpA4CorCwcRQAABBBBAAAEEEEAAAQQQQKCHAgSuegb1wAMPyCWXXJJx8sMPPyxHH310xrF8d6699loZPXq0HHXUUVbwyNfJT9D222+/VOhLn619eOutt2TWrFmye/fuDo8fO3asPP7447LPPvt0eE+DVT/72c/krrvukmi065/0jRs3Tn7wgx/IRz7ykQ736epAS0uLaBWq2267TbZs2dLVqan3NOj1la98RS6++OKsIa877rhDvvOd76TO72pDQ3IaJOusFcOgs2d74TiBKy/MMmNEAAE7BQhc2anprHuFzfKBe0xVJV1CUMM/a7dFrWpLURMAincTAPKbYJXm5/fpF5DhA/3ygf0r5OB9g9aSg1WmClYxm1sCV01aSawpIYtMyEq9tYrYdhN2M9yStpJ3VrrA3oCbhtr22ycgh+4fkjFmiUGtflXsJQYJXGWdIg4igAACCCBgjwCBK3scuQsCCCCAAAIIIIAAAggggAACXhUgcNWzmT/33HNFqyklW01NjSxfvlyCwWDyUFFe2weufvvb38pxxx0nu3btyvr8MWPGyBtvvNHhPa009bnPfa7T6zpcsPfA7Nmz5ZZbbpGKiorOTkkdX7NmjRUEW7lyZepYLhsnn3yy/PWvf+3wLLsCV8UwyGW85XgugatynFXGhAAChRQgcFVI3dLeu8Esbbd+b2WrlxaHrSUEE90lf7J0ucJUujrqwAoTAqqQ0YP9UtfXVF4qYubKDYErZd1qwlWrt8bkzRUReXVZa7chqyzUxtUnA43v0QdXmEpXQRlRF7CWeMx2bqGOEbgqlCz3RQABBBBAwAgQuOJjgAACCCCAAAIIIIAAAggggAACvREgcNUzvfSgk17xwQ9+UO69996eXWzjWen90ApXuhTfnDlzOn3ClVdeKZdffnnG+08++aQVtgqHwxnHq6urrSUEJ06cKPX19VblrPXr12ecozsnnniiNfbOqnDpORs3bhRdinDz5s26m2r6jClTpsiwYcOkrq7Oen/16tWyYsUKaW1tTZ2X3NDKWBp2S29333233HDDDdYhraDV0NCQeluDYGqSbAceeKA88sgjyd3UazEMUg/z8AaBKw9PPkNHAIG8BAhc5cXm6ItM9kfi5hdd2u61ZRFZuj4ia7bFpDWi7+TetPrSvib4M25oUGaMr5CxQ4ISNKs+Fyt05fTAlVrr6ttLN0Tl1ffCsnJrVDZsj0l+2iLVpoLYfoODMnFkUI40QbchA/yiFceKlXEjcJX77xGuQAABBBBAoMcCBK56TMWJCCCAAAIIIIAAAggggAACCCCQRYDAVRaUdod0qT5dUi+9fe9735NLL700/VBRttMDV1plq7Gx0Xqu36wzo8v9feADH7COzZ8/31pKb968eTJq1KhU35qammT69OlWICp5MBQKyde//nW57LLLOlSTWrt2rXz1q1+VuXPnJk+3Xn/1q1/JOeeck3Esfeeiiy6SBx98MHVI+3fVVVdZQa+BAwemjic3dLnBm2++2VreMBKJJA9bAbAXXnjBqjCQOpi2ocsVXnHFFakjp512mtx5552p/WwbxTLI9myvHSNw5bUZZ7wIINBbAQJXvRV03vUxs1xgiwlXLdsYlScXtFpVl5pa41YIK5/eatCnMuSToWa5u1OmVsrk/UJSU+Ur2lJ3Tg9chc0q2Y2mmthbqyLy1IIW2b4nnne4TedHA27VFX45cHhQTjmk0lpasMr4a+iqGI3AVTGUeQYCCCCAgGcFCFx5duoZOAIIIIAAAggggAACCCCAAAK2CBC46p7xvffeE60mld50Kb9Zs2alHyrKdnrgKvlAXdbw4YcflqOOOip5yHpdt26djBw5MuPYj370I/nFL36ROta/f3/rWq061VmLRqPywx/+UG6//fbUKUOHDpXXXntNNPTVvi1evNha5jD9+HXXXScXX3xx+qGs2xrkuuaaazLe07G190+ekE/gqhgGyf55/ZXAldc/AYwfAQRyFSBwlauY88/XSlabdsVlyfqovLi41dqOxRJ5V1zSEWsIqLbGbypcVZql7kIyss5vha6KoeH0wNXu5oSsMxWtFpjAlVa4amgx4TYTesu3aeWwgElXjdonIMdOqJCD9g3KUFPlKhQoTuKKwFW+M8d1CCCAAAII9ECAwFUPkDgFAQQQQAABBBBAAAEEEEAAAQQ6FSBw1SlN6o3nn3++Q7jqvvvus5bWS51UpI1sgatvfetbVvWo7rqwZs0amTlzZsbSfdmW7OvsPrNnz5bnnnsu9fY3vvEN+f73v5/aT27cdNNN8tOf/jS5KxMmTBCtUtWTpuGuyZMny/bt21On33jjjVZlrNSBtI1cA1fFMkjroqc3CVx5evoZPAII5CFA4CoPNIdf0tiakOWmutW76yIyf0VEtjf0Iv2TNta+pqrVIWMqZNKokFnuLmAFsNLeLtim0wNX24zv4nVR+c+aiLyzOiLN4XwXE8wk1Ipi08apdUjGDQuIVrkqRiNwVQxlnoEAAggg4FkBAleenXoGjgACCCCAAAIIIIAAAggggIAtAgSuume89957rWX10s98+umnZerUqemHirLdPnClFaxef/110WUBu2taYervf/976rQTTzxRNDjW0/bOO+/ISSedlDq9oqJCFi5cKHV1daljunHDDTfIv//9b1m9erXo8n09rW6VvIkujahjSjYNdWm4K1vLNXBVLINsffXiMQJXXpx1xowAAr0RIHDVGz1nXru7OS4LTfDnP2ujssh81TfZE7jqU+mzlrnTANDhY0OyT39T9qoIzemBq407Y/LmyohlvXxTtFfLCaZzDurnt8JWuoTjlP2Cov7FaASuiqHMMxBAAAEEPCtA4MqzU8/AEUAAAQQQQAABBBBAAAEEELBFgMBV94waSvrKV76SceJTTz0lhxxySMaxYuy0D1ydeuqpcvfdd/fo0YcddpisXbs2de7NN98sn/nMZ1L7Pdk49NBDRZcqTLZ//OMfMn369ORuh9etW7dKv379pKqqqsN7nR246KKL5MEHH0y9feWVV8rll1+e2k/fyDVwVQqD9P56bZvAlddmnPEigEBvBQhc9VbQeddrwOrN9zVwFZFlG6KiS97Z0aorfDJmSNCEgIJy1IEVMsQsc1eM5vTA1fodMXltWcQs4RiRVVtiEo7a4z3QLOGoywnqEo6HmoBbDYGrYnzceAYCCCCAAAKFFSBwVVhf7o4AAggggAACCCCAAAIIIIBAuQsQuOp+hnUZPV1OL705ZUlBrdikFaS6a+FwWEaNGiXx+H+rKugyf7rcXy7ts5/9rMyZMyd1yS9/+Us577zzUvu92dC+acWsq666SubNm5e6VVdLJuYSuHKDQWrQZbJB4KpMJpJhIIBA0QQIXBWNumgP2mUCV7qUoC5xpxWXGmwKXOmSdvsNDsoEE7iaOb5ChhK4suZ03faYvPJeW+Bq7Tb7Ale1JnB1wDATuBodlGljK6TGLOlYjEaFq2Io8wwEEEAAAc8KELjy7NQzcAQQQAABBBBAAAEEEEAAAQRsESBw1T3j4sWL5bjjjss48be//a3MmjUr41gxdtpXuPrpT38qF154YbePfu+99+Too49Onefz+UQrR/n9uVVDeOaZZ+TVV19N3UeX+tMl/3JtO3bskBUrVqS+1FjvW19f3+FWl112mXznO9/pcFwP5BK4cppB1gGV2UECV2U2oQwHAQQKLkDgquDERX/AbhO4WrA6agWulqy3b0lBrXA1zgSAJo4KyRHjQjKYJQWtud2wMy7zlkdksalwtcIE3OyqcFXX1y8TRgRFlxQ8ZEyIJQWL/juJByKAAAIIIFAAAQJXBUDllggggAACCCCAAAIIIIAAAgh4SIDAVfeTreGg8ePHZ5x4ww03yJe//OWMY8XYaR+4uv/+++WEE07o9tH/+te/5IILLuj2vFxPOOOMM+TPf/5zl5c1NDTIQw89ZFWtWrp0qaxcuVJ27tzZ5TXpb9oVuCqlQfp4vLRN4MpLs81YEUDADgECV3YoOusee1oSsnhdRN5dG5UFqyKys/G/1UZ701Nd0m6CCVtNMl9TTdWlQSYQVIzm9CUFN9fH5Z1VUePdVuWqJWLPkoIaaJs6OiSTjffBpqqYBt6K0ahwVQxlnoEAAggg4FkBAleenXoGjgACCCCAAAIIIIAAAggggIAtAgSuumdMJBLWcnytra2pk3UZPV1OrwGMkk8AAEAASURBVNitfeDqpZde6hAGy9an3/zmN3L11Vdne6tXxyZNmiS65GK2tnHjRvn9738vd955p+zevTvbKR2OBQIBqaqqksbGxtR7dgWuSmGQGoRHNwhceXTiGTYCCOQtQOAqbzrHXqiBnw074lYA6KXFYdlcH7OlrwP6+OWogypkiqm4NGZIQPoVaYk7pweudjUlZNWWtnDba8vC0tRqT+Bq5KCAHDuhUsbvG5R96/xSESRwZcsHmZsggAACCCBQSgECV6XU59kIIIAAAggggAACCCCAAAIIuF+AwFXP5lADVk8++WTq5BEjRsjbb7+d2u/thganHnvsMTnmmGNk5syZUldXl/WW7QNXc+fOlQMPPDDruekHb731VrnuuutShyoqKqS2tja1n+/GmDFj5PHHH+9wuVay0upXu3bt6vBe+gENWI0dO1amTJkiJ510kpx88snW8oEPPvhg6rRvfetbctVVV6X20zdyWVKw2Abp/fTqNoErr84840YAgXwFCFzlK+fc66ImX9VgqlzpcoJPLGiRddtjEo0lxOT5825BU8xqUP+AfHBKhUw1gStd7q4yVJwAkNMDV83hhOwwVcTeej8izyxslXqzpGOsF0XFzCrcEgr4ZOzQoJxySKWMM6/9qn0SKE5BMaHCVd6/TbgQAQQQQACB7gUIXHVvxBkIIIAAAggggAACCCCAAAIIINC5AIGrzm3S39EqTRr8SW+vvPKKHHDAAemH8t7+0pe+JA8//LB1vc/8ZOfggw+WRx55RAYOHJhxz3wDVxrm+vznP5+6ly6RqCGvQrQNGzbI6aefLuvXr8+4fSgUkmnTplnhKq2MNXHiRGucWtEqvX3hC18QDeokm12Bq2IaJPvu9VcCV17/BDB+BBDIVYDAVa5izj8/boJVkajI2u1Rmbs0LMs2RmXTzrhETOgqn+Y3QZ/BJmw1enBAjj64Qg4cFrTCVsUKADk9cBU14apWU1Vs8bqovLQkLOtNwG3r7vyrimmQbXitX8aPCMnM8RUyfGDABLBENIhVjEbgqhjKPAMBBBBAwLMCBK48O/UMHAEEEEAAAQQQQAABBBBAAAFbBAhc9Yxx8+bNVlBIlxdMtu9973ty6aWXJnfzft2zZ49MmDBBWlpaUvc44ogjZM6cOan95Ea+gavFixfLcccdl7yNaGWplStXSnV1deqYXRsf/vCHZf78+Rm3O/fcc+X666+XAQMGZBzPtjNr1ix5/vnnU29985vflO9+97up/fSNXCpcFdMgvY9e3iZw5eXZZ+wIIJCPAIGrfNTccc32hrgsNlWuFq2NmOUFo9LYEhcNB+XSNFSlS9lp+OfgEUGZOtosb2cCQMVsycDVM++0yLzlYSt4pOEj/QcD2vTPyvrHZf06ZmKlnPyBKlMdKiC6JF8x25ptMXlnddRUFovIexuixjoh8Ry9g6bLunTjpFFBmTiyzby2pkilrfZiEbgq5qeGZyGAAAIIeE6AwJXnppwBI4AAAggggAACCCCAAAIIIGCrAIGrnnN+6EMfkjfffDN1gYaHXn/99U6X/0ud2M3GPffcI1/72tcyzvr1r38tZ599dsYx3ck3cNXa2mpdG0/7SdM///lPq+JUh4f04sCCBQusZQHTb/HVr35Vrr766vRDXW6feOKJon/nlWwaatNwW7aWS+CqWAbZ+unVYwSuvDrzjBsBBPIVIHCVr5zzr9OqS7uaErJoXUReNpWXttTHpcEsd/ffKH/3Y+hT6RMN/MwcXymHjA7JoH4+0WPFbMnA1fP/abGW7QsGRYJmyb2A32fG0hZqikYTpoKXyAxTgeuDU0sTuNpjlnHc0ZCQ+e+HZe57raL7za091zbDMUsH+mVEXUBmmnGM3zdowlc+K/BWTG8CV8XU5lkIIIAAAp4TIHDluSlnwAgggAACCCCAAAIIIIAAAgjYKkDgquec//jHP+TCCy/MuECX6fvZz36WcSyXna1bt1oBJV2GL9l0GcGFCxdKZWVl8lDqNd/Ald7gsMMOk7Vr16bulU+Frm9/+9uifR01apT1pZW49CvZ7rjjDvnOd76T3LVeNaSm5/ek1dfXW8sMRqNm7Z297etf/7r84Ac/SO5mvP71r3+Vyy+/PHXstNNOE13+sbNWDIPOnu3F4wSuvDjrjBkBBHojQOCqN3rOvlYrPukqgqu3xmTBqois2da2tGCDqXTVZIJA+n62pnGqqgqf1FT5ZMiAgOxb55dDx5ilBIcHJWiKLekSg8VoGqCKmCCV9n/FpqgsXB2xXrUKlBW4MqErHUTMVJGKmoHqH+UmjArJIfuHZMwQ7XdAKjSYZc4vRkRM/41B1PR5salu9dbKsGzaFZet9TFpNNatYY2GdWxWv8wvNSbE1t+ErXT5wNGm7x8YE7KCV2pdjL6n94zAVboG2wgggAACCNgsQODKZlBuhwACCCCAAAIIIIAAAggggIDHBAhc5Tbhn/zkJ+XFF19MXaRL82k1qrPOOit1rKcb4XBYPv7xj8sbb7yRcckll1wi1157bcax5E5vAldaRUuraSVb37595ZVXXpFhw4YlD3X5+u6774pWn0pfVvG2224TXS4w2W644Qa55ZZbkrvWvTU81tN21113yWWXXZZxelcef/vb30QDWcl26qmnyt13353c7fBaDIMOD/XwAQJXHp58ho4AAnkJELjKi801F2nIp8WEfRpMtaVVW6JmqTsTXtoclTUmxBQ3YaX2oStrqT4T8RluwkoaWtIqSwcMC5owkAlgaWWrtv8VfPzaL+3zrsa4rN8RM19Rq2KUBsXalhPUrrRFkTTKpOfrV18TEhtgKnLpsofDagMy0Gxrv/euPljQfqu1pqr2mD7ubo7Le+tjstSEr1Yb982mupi19GG7Hmi//OaX/c0SiOOGBuUg473fPgFT6conlaHkCNtdVOBdAlcFBub2CCCAAALeFiBw5e35Z/QIIIAAAggggAACCCCAAAII9FaAwFVugkuWLLFCR+kVmHzmBzNa1emb3/xmj2/W3NxsVWa67777Mq4ZPny4PPnkkzJ06NCM48md3gSuNm3aJNOnT5fGxsbk7azA1x/+8IfUfmcbsVhMPvWpT8mzzz6bOmXQoEHyzjvvSEVFRepY+wCU35QCWLFihWi4q7u2fv16Of7440WrXKW3L3zhC/KTn/wk/VBq++GHH5YvfelLqf0ZM2aIhnw6a8Uw6OzZXjxO4MqLs967MT/22GMZS7dWVVWJVtbLp2nFvfTqgXqPAw44QM4777ycb9fU1CQ33XRTxnUamJ06dWrGsXx2duzYIb/61a9Sl2qQt7NlVFMnsVG2AgSuynZqMwamYaStu9vCS+u2x2TDzrg0myCWLjsYi7cFlnSJvoqgT6pNdashA8zSdoMCMnJQW3hJo02FDi2ZbliVqprDJrRkqnDtbk5IvVkCUUNX+mre7hAQyxjk3j4GTFWoAX38Umu+9FWX6dMgVlVIJGQqXhV6HFb4y/RFnddtbzNX+5ZwXMKmCpd6ax/Uu8qEqtQ7aa3LCQ7q598bJWs/uuLsE7gqjjNPQQABBBDwqACBK49OPMNGAAEEEEAAAQQQQAABBBBAwCYBAle5Q15//fXyy1/+ssOFWv1JqzGdcMIJ5gc3bf/Kv/1J+q/pNWSllaA2btyY8baGkh5//HGZOHFixvH0nd4ErvQ++kP9a665Jv2WcsEFF8h1110nNTU1GceTO9pnrSKVXh1L3/vWt74lV111VfI061X/rkod0ltXSwImz9Mgmy7PuHz58uSh1Os555yTEUZIvWE2nnjiCTn//PNTh/r16ycvvPCCjBw5MnWs/UahDdo/z8v7BK68PPv5jV2XBNXvLent7bfflhEjRqQf6nZbQ60HHnigtLa2ZpxbV1cnixcvNktwmQRADu25556T2bNnZ1wxd+5c6xkZB/PYWbVqVcbSrKFQqENQrLPb6n9PjjrqKBk9enRnp3DcZQIErlw2Yb3obnLpvZaISJMJW21v0CBTW+hKQ0IVQZH+JqA0qK+pCmUCStUm3x40oSBdwq8YTZcQbDRVrbaZcNK6nTFpMFWirECY6Zwu15dL09CVBpq0StQAE7jSQFNyXLosYjGaLi8YMUsdtgXIErJjT7xtTGaZxID5c3uF6dvAGp8VsKo225V7vbXvpWwErkqpz7MRQAABBMpegMBV2U8xA0QAAQQQQAABBBBAAAEEEECgoAIErvLj1YDSrbfemvVi/SH/SSedZP0AfMyYMaLVsJYtW2Z9vfnmm7J06dIO1wWDQfm///u/DmGl9if2NnAViUTkuOOO6xBs0n5+97vflcMOO0z0GRoY04ouzz//vNx4443SflnAQw89VB566KEOIS29v1aZWr16darreq+LL75YLr30UtGwQ3p7//335f/9v/8nWolGAxLZ2sknnyxaOStbU8tjjjkm4y2tDqaVsmpra80PBOPy4x//OOP9QhtkPMzjOwSuPP4ByGP47cNHeovbb79ddDnXXNrTTz+dsdxp+rX/+te/5PDDD08/1O22fh/5+c9/njpPA2AaBLOjtR9zTwJXGhrTyl+vvvqqVXlw8uTJdnSFezhAgMCVAyahyF3Q4JV+abhJg1dREwrSFjDVnzT4o2GrkAlZFSv4o5WttNqWBqy2NbQFk3aaqlbJ6lv58vh0TFpFylSQqjVLC2rgah9TPUqrXWlVKfPHxaK0qLGOmMpWja1a5cpU8dpb4UqrbWk/+polD9U6x1xuwfpO4KpgtNwYAQQQQAABEQJXfAoQQAABBBBAAAEEEEAAAQQQQKA3AgSu8tf7/e9/L9///vfNciptPxjL906VlZXWUlXnnntut7fobeBKH7BgwQIriLBt27asz+vfv79UV1fL5s2bs75/0EEHyT/+8Y8O4ankyfPnz5czzjjDCpolj+mrLj04xgS7tPqULmuoSwiuW7cu/RSrWszll18uF110Uer44MGDrcCXLrPVvmmgasKECaJLcmVrWsVmzZo1osbprdAG6c/y8jaBKy/Pfv5j1zCU/r5Nti90saxo8pz2r7rMqwY5szUNKl1xxRXZ3ur02Ec/+lEr3JQ8QSvr3XLLLcndXr3mGrjSKou//vWvU99jdalXAle9mgJHXUzgylHT4cnOaOBrm6m2taU+LmvNMnwNJgjW2z/rtofUMP6APj4ZZSpdDRnQVu2qWIGy9n1x+j6BK6fPEP1DAAEEEHC1AIErV08fnUcAAQQQQAABBBBAAAEEEECg5AIErno3BU8++aT84he/kDfeeCPnG2mASJfL0x/+93S5LDsCV9rRtWvXyoUXXphzhRatjqVL8g0fPrzL8f7pT3+SH/zgBxIOm9IBPWhqoQEGrRzWp08fmTJlimzatCl1pS6b1X6pwuSbWuFFl/pqaWlJHsp41aXAJk2alHFMdwpt0OGBHjxA4MqDk27DkC+77DK56667UnfSMJGGinJpusyeVtDTpiHS3bt3py7XQJdWuepp02UJx44dm/H9TMNcH/vYx3p6iy7PyzVwpcvObt26NXVPAlcpirLYIHBVFtPo2kG0RBKyxwSs1u+IWYGr3abKVdgsuVeIVmWqd/UzywsOHWCWGKwLSI2pLKVLDtIyBQhcZXqwhwACCCCAgK0CBK5s5eRmCCCAAAIIIIAAAggggAACCHhOgMCVPVOuFZP0B/APPvhgxg/ls929pqbGCg9pBRatFpVLsytwpc/UagVaqeo3v/mNvPXWW9bye531RZfo04owGmLoadMKNddee6088sgjnV7Sr18/Of300+Wb3/ymjBs3LnXe1772NbnnnntS+xqo0n521jRwoJ4rVqzocEpXwYhCG3TojMcOELjy2ITbNFz9npEMnegttVLd8uXLRb9f9KTpkqbTpk1LnXrBBRfIE088karap/fTJfnaL3GauqDdxty5czPCVXr9kiVLZODAge3OzG9Xg6l6v2TTyi8aOu2sEbjqTKY8jic/+7NmzZZZ5r99Tm7JP0NqKFIrr9HcLaAFW3XpwO1mGcFVW6OyfU/c9spW7YX0+92Q/n4ZPThoLTFYW0PgqoOR+cNqYSJv7Z/EPgIIIIAAAh4UIHDlwUlnyAgggAACCCCAAAIIIIAAAgjYKJD8Ydn48eNzXmLJxm6Uza30B+cbNmywlsvTJfN0u6GhQUaPHm0FijRUNGzYMMeNV5fke+WVV2Tjxo2yfft2CYVCVp+132PMMoD77LNP3n3esmWLrFy50vrSIITeW++nnzmtNBMMBvO+d/sLtaKNVsZqbm62qnBp5bABAwa0Py3rfiENsj7QAwcJXHlgkgswRP0epEuFpv+Iuasqd+27oBX2rrzyytTh3/3ud/LMM8+I3iPZbr/9dvnkJz+Z3O3y9eabb5af/OQnqXMOO+ww+fe//53aL/YGgatiixf3eQSuCu+tyxlrYFID8LQ2AU30ROMiq03Qap1ZRrC+KS5N4eLEfGoq/XuXFwya4FVA/CZzZXJYtL0CVLjio4AAAggggEABBQhcFRCXWyOAAAIIIIAAAggggAACCCDgAQECVx6YZIaIQIkEShW4eumll2TmzJlWZSS7h/7A/ffLAw/cb91Wq6fRCiOgS4jq330n2+WXX54Rokoez/b66U9/OiMQtXDhQnnxxRflf/7nf1Kn63KuujxqT9pZZ50lL7zwQupUXfJQK+qVqhG4KpV8cZ5L4KrwzhqI/8pXviIzZsyQo48+2vrSSktebuGoSIsJWC3dGJE122ISjSUkXpy8lflvtUjQpKz2HxKUg4YHpTLok5B9WXzXTyuBK9dPIQNAAAEEEHCyAIErJ88OfUMAAQQQQAABBBBAAAEEEEDA+QIErpw/R/QQAbcKlCpwddutt8r8+fNkhgldzZwxU6ZMnWobIYEr2yi7vNEPf/hD+fWvf50659hjj7WWbE0d6GRDqwweeOCB0tTUZJ1xwAEHWNX7tm3bJhpUSlbNGjx4sLz77rumikrXIYtIJGJVJ9SqecmmS7FOnz49uVv0VwJXRScv6gMJXBWeW6t5XnLJJakHVVVVpYJXWsHOi61tKcG4rDXVrbbUx0SzVsVax06/Det34uG1ARk5KCCD+mnFK5PColkCBK74ICCAAAIIIFBAAQJXBcTl1ggggAACCCCAAAIIIIAAAgh4QIDAlQcmmSEiUCKBUgauXn75pdSohwwdagWvNICly2T2phG46o1ez6999tln5eyzz05d0KdPH1mxYkW3S5BqJav0pQI1vPLjH//Yus8HP/hB0WpXyfb000/L1G7CeG+88YacfvrpyUukb9++smzZsg79eOutt0RDXcl2yimnWJsa/PrLX/4iOp5du3ZZ4a1p06bJpz71qdRyZhrm0qpsyeY35V5OOumk5K71+txzz4mGv7RpUKS+vt7a1l90fOmf60MOOUSGDBmSej/bRjweF+3zk08+KUuWLBFd+rWxsdFa7lWXvT3mmGPk5JNPFg2m0YorQOCq8N6bN2+Wiy66KOuD6urqrPCVVknUpU290tbviFlhq+0NcdndbNYWLEGrNSErDVuN2idgha9K0AVHPpLAlSOnhU4hgAACCJSLAIGrcplJxoEAAggggAACCCCAAAIIIIBAaQQIXJXGnaci4AUBpwSu0q21+pFV+cr8ML22dmD6Wz3aJnDVI6Zen6QhJK1OpRWrkk3DQR/4wAeSu1lfr7nmmoylAv/2t79ZwSE9+brrrpNbTfWzZPve974nl156aXI366uer9cl22mnnSZ33nlncjf1qgGqp556KrW/detW2bFjh3ziE5+QRYsWpY4nN2pra+VPf/qTaOWuVatWyRFHHJF8S0KhkGzYsCG1rxsHHXSQ7Ny5M+NYZzt6349+9KOdvS1z5syxxrR8+fJOz9E3AoGA6PKMV155JcGrLqXsfZPAlb2e2e6mvz+/9KUvZXsr49jIkSNT4av0UGPGSWWys3xTVJZtjEpzJCGt5qsUrSrkk+oKv4zfN2iWFwyUoguOfCaBK0dOC51CAAEEECgXAQJX5TKTjAMBBBBAAAEEEEAAAQQQQACB0ggQuCqNO09FwAsCTgxcpbsfbioN6ZKDGsDSqkI9aQSueqJkzzkf//jH5eWXX07d7IYbbpAvf/nLqf1sG8cdd5wsXrzYequiokI0VFRdXW3taxUpDUAl21FHHSWPPfZYcjfr67nnnitaCSvZfvrTn8qFF16Y3E29tg9caQWdj3zkIzJv3rzUOekbupTh/PnzZdSoUUULXGl47atf/ao89NBD6V3pdnvgwIFy1113iXrRCi9A4KrwxlqNLunc06dp6FGrXh199NFlFUCMmWJWsXhC3jNhq6Uboma7bb+nLnaeF/D7JGhyVhNHhGTc0KAJfYqYQ55vBK48/xEAAAEEEECgkAIErgqpy70RQAABBBBAAAEEEEAAAQQQKH8BAlflP8eMEIFSCTg9cJV0qaqqtn6QrsGrKVOmJA9nfSVwlZWlIAd/8YtfyI9+9KPUvc8880z54x//mNpvv7Fx48aMJQI1fPX3v/89dZoGjrTCmS7zpy0YDMrSpUulf//+qXPSN2KxmHV+Q0ND6vDrr78u+++/f2o/udE+cKXhMK2g1VnTJfuSwaeeVLjSoIcuSaht+/btoksCJpsugabVqJLtlltukVNPPTW5a71Go1E566yzZO7cuRnHdUer+EyaNEl0KcF3331X3nnnnYzKYnpOZWWl3HPPPdZSg7pPK5xAMgg0a9ZsmTV7duEeZMOdk3+GnDx5slx//fU23LE4t9Dqc9mCkz19ulbaS4avampqenqZI89rjSakxVS00upWWuUqUZriVikbDViN37ctcKUVr0LB1Fue3SBw5dmpZ+AIIIAAAsUQIHBVDGWegUD5CCTM/2Nav369vL9ihehfFjU2NpqvPdLS2ioB8y85g8GQ+QuUCqkwf4nSx/zrv+rqPta/AuzTp+21pm9f6devn+j/kUz/i5zyEWIkCCCAAAIIIIAAAgh4TyD5w7Lx48fLFVdc4T0ARowAAgUTcEvgKh1g6NChbUsOmspX+40enf6WtU3gqgNJwQ5oBagPf/jDqftrIGjhwoWp/fYbd999d8YSgVdffbVV0Sn9vPPPP1+eeOKJ1KGult9bsGBBajlCvWC//fazqlKlLk7baB+40r87079303bkkUfKCSecYC0VqMsL6pJ+N910k2j1LG09CVxZJ+79ZeLEiaJLoiXbs88+Kxp46ar9/ve/7xAA0/vcdtttGSE1vUckEpH7779fvv3tb0ur+TvDZNMlHl944QVrHMljvNovQODKftP2d9Tw4uc+97n2h/PanzFjRmrZwZ5WSszrQQW6qDmckIaWhLy/JSorN0cL9JT/z955wDlRbm38sJulLWVh6XXpXQEF7HLBiqKiCHYpn1fBgg0VxY6KYgELFmxYkGLBqyJYAAsqKjakSu9tKUtn23eegRkns8kk2U2yyeY5vxsz87Z55/9G72byvM8JflgYWjWt5ZGMGh6pULaUQHSV6EHBVaJ/Anj/JEACJEACESVAwVVE8XJwEigxBHbs2C7fffudLF68SPbs2RO2+8LDozvvGha28TgQCZAACZAACZAACZAACZBA9AlQcBV95rwiCSQKgXgUXNnXphlSSB1JOZiWlmZUUXBlJxTZYzhMIY1XVlaWdSGIsCB88hVwrPnkk0+sKoiDWrVqZZ3j4NVXX5Vhw/59lgUBFhyhfMVLL70k9957r1V19dVXG0Ipq8B24BRcmVXDhw+XIUOGmKfGOxyq8EytbNmyxnmkBVdw84Hoa9euXdY8rrjiCnniiSdcxVNwuoIoZe3atVa/hx56SAYNGmSd8yD8BKIpuMLmXLzgmIZXoON8tDvSHsfPP/+cLNeNvRDjXX/99dY4zrGc5/brhKsulHEgJPzmm2/CunhIXWq6XnXs2DGsY0dysD0qttq5L09Wb82VNduKX3CFe22kYqv61ZIlrXySpJah4IqCq0j+G8CxSYAESIAEEp4ABVcJ/xEgABJwJYAvmj/MmSOzZs00dqe5Ni5EJR56XXHlVYXoyS4kQAIkQAIkQAIkQAIkQAKxQoCCq1hZCc6DBEoeAbvgCinLohXr1q3zEpeE47rHHnusOl+dKBvUOfyDD943hoR4hxFZAldddZV8/vnn1kXGjh0rF/tIswZxFpwaTVGRPzesFStWSJcuXazxateubaTQswpsB85rv/nmm3LOOefYWvx76EtwBfHF1KlTpVQpd8FApAVXt99+u4wfP96aLJ7nzZw500gTaBX6OUAKRfs9I/3i3LlzpVq1an56sLioBEzBVbVq1SU9vaohcDKETkfETubxv6IliKYOC6by8vwcQ1iF/vr6t9/h46LOl/0LEkDKQaQ0xQupS2M5svbny/Y9ebI2M0fWZebGxFQbVFPBVXqyVK2QZLhcxcSkinESFFwVI3xemgRIgARIoOQToOCq5K9xUe5w2rTPZPmyZcYQeOiA3UzOwB/8VapUsYrxAADp47DLqoKmj6ug6eMyMhpJ/fr1JR4tca0bS8ADrPnbb40XPEiKVLRt10769OkbqeE5LgmQAAmQAAmQAAmQAAmQQBQIUHAVBci8BAkkKAG74KqkIEhJSbE2tVFwFflVdTpS+XOZcgqDkK4P6fJ8Bdxn7K5NvpywIEqBgGvHjh3GEMnJybJ06VKB4MhX+BJc/fjjj4bzj6/29rJICq7++ecfOemkkwyhjXlNCNggIAw24IY1Y8YMq/ngwYPlwQcftM55EF4CpuAqvKMmzmj4DQO/ceDdfDnPQQNpBcMZdevWtURWEHLGS+xSd6ttWXmyfnuurN8RG4IriK3qVk2WahWTpWI5d8FqvHAuyjwpuCoKPfYlARIgARIggQAEKLgKACjBqydOfE8WLlgQFgqwxG3atJlgN2Kr1q0D7swKy0U5SJEITJ36kfymNuuRDAquIkmXY5MACZAACZAACZAACZBAdAhQcBUdzrwKCSQiAbvgqkXLllFDsHHDBq80dOG4cPsOHYx0UZs2bpIPP/zAGJKCq3CQdR8DgiE4RZmBFIEQSDnjsccek6efftoqfuWVV6RXr17Wuf3A6fj0wAMPGOnQ7G0WLVpkiCfMsk6dOsm0adPM0wLvTsEVNrKuXLmyQDtfBZEUXE2cOFFuvPFG67I1a9YU/KYQSrz33nty0003WV1OPvlk/XfgQ+ucB+ElYAquqlevLjV0vZJKQThUSkpBQBTkcSltb4iNtL1xbPQLfGxcA32NfoevW/D437E/+uhDWa+uf9isjfScdmGT/dgpgipsXTDjBLMa+/btk8suuyyYpq5typQpY/x3Av9OHHXUUa5tY7USDleZu/PU3UodrlR0FQsBh6t6KrhKr0iHK6wHBVex8KnkHEiABEiABEosAQquSuzShuXGwim4sk+olu7QOP2006WZ2k8zYpPA77/9JvjCG+mg4CrShDk+CZAACZAACZAACZAACUSeAAVXkWfMK5BAohKwC64mTpocNQzPPfuszJnzfZGv17hxY00jeIIKfk7U1F7pxnjvT5ki778/xTim4KrIiIMaAEKGjRs3Gm0h1IAIq3Llyl59Tz/9dPnjjz+MMogyIJiqWrWqVxvz5LPPPpN+/fqZp9K9e3eBMMker732mtx1111W0R133CFDhw61zp0HTsFVO3WFR9q+YCKSgqtHH31UnnnmGWsa5557rrzxxhvWeTAHC3RDb9euXa2mcPIxWVuFPAgbAVNw1bv3xdLbR/rMsF0oDAOZf0O2bdtWRowYEYYRozPEgQMHBC54hQ0zZSCEVnA9jOfYfSBfdmhKwTXbco20grFwL/XT/xVc0eGKgqtY+ExyDiRAAiRAAiWYAAVXJXhxw3BrkRJcmVNr1KiRXKRf/PxZaZvtYvU9Oztbpkye5Dq9Tp06x6Ww7IXnn5PNmze73pu9Eg+rKmr6SDA5ePCgl824vZ3zmIIrJxGekwAJkAAJkAAJkAAJkED8ETB/LEPqJLcfk+PvzjhjEiCB4iYQj4Kr9PRqhqMShFYQXDmDgisnkcifw6HJLoiC49Jpp51mXTgzM1PgfIU0gAikDLSnwLMaHjnIysoy0gXm5OQYJampqbJs2TLxeDxW04EDBwo+v2bA3QouV/7CKbg677zzBKKtYCKSgiuIdz7++GNrGkgveMopp1jnwRzgeeGoUaOspniOiJSMcPdhhJ8ABVfhZ+oc8dChQ9KnTx9nset5vKYMdL0prdx7MF+y9uXLqq05xitQ+2jU16ycLHjVqZIkVSokReOSMX0NOlzF9PJwciRAAiRAAvFOgIKreF/ByM4/0oIrzB4incsuv0LwhSPeAsKiR0Y87Drtc3v2lM6du7i2ibXKNWtWy6vjxgWcFqzNu+vDqdq16wgsqu27cfClE3xGPfG46zgUXLniYSUJkAAJkAAJkAAJkAAJxAUBCq7iYpk4SRKISwLxIrjCM5HDTlYnSPv2HVxZU3DliicilVPUVWzw4MHW2DfffLPcc8891rmz/rbbbvNyp7Ia2g7g9DR37lyr5PPPP5djjz3WOm/durVs3brVOIeb1pIlSyQ5Odmqdx44BVfOOTrb288jKbjq1q2bzJ8/3365sBwjrSNEbozwE6DgKvxMnSNCRHhxEO5hEBXCxQoixXhNGei8d/M8R7MHHszOl22aTnDjjlzZvjdPdupL9ZTFFoZkVv+hCTSldHIpaVg9WWpXTZLK5ZOkbOlSokXFOr/iAkPBVXGR53VJgARIgAQSggAFVwmxzIW+yWgIrjA5PJSC0xUeRMRTlFTB1efTPpMff/zRdSmQFvJyFco57dednca+8Lxs2rTJWWydU3BloeABCZAACZAACZAACZAACcQtAQqu4nbpOHESiHkCsS64Qlqo4zVd4AnqZmXfiOYGloIrNzqRqcOzKaToM+P444/3cp8aNGiQpnl836wWpAzs3Lmzde7r4KmnnpKRI0daVRBwQSSFgNsVrmFGMGn4nIKrxx9/XAYMGGAO4foeScFVRkaG7N271/X6hakcP3689OjRozBd2ScAAQquAgAKQ3Vubq5cdNFFfkfC/zeYQqtg/7/B72AxWrF7f75s3ZUnC9dlyx8rs6Vi+VKSXilJkopR1JSnYiu8/lmXKxu350rjWsnSoo5H2jZMkTpVk6UcRFcJaHhFwVWM/kvEaZEACZAACZQMAhRclYx1jNRdREtwhfljh9fA/7tG6tWrF6nbCfu4JVVwNWHCu7J40SJXXkNuvkXS09Nd26CSgquAiNiABEiABEiABEiABEiABOKeAAVXcb+EvAESiFkCsSi4atSosZUysFq1aiGzo+AqZGRh6QDxw+LFi42xypYtKytWrDBEckgjCKclpBVEBONGhXa//fabnHnmmTg0omvXrgKnLMTbb78tt956q3GMf0CcddVVV1nnvg6cgiuk4OvXr5+vpgXKIiW4ApsGDRrIgQMHrGtWqVIlaHGh1cnHAe6PgisfYMJQRMFVGCAGGAL/bvTq1curVUlNGWjeJDKuZqur1d4DeYddrbbnydptubJsU478syFb6lZLlka1POokJVLGUzw2V3Dc2ndI5C8VgK3cmCs105Kkvs6reV2P4XZVq0qypGuKwTR9eVR4VZxuXCbXaLxTcBUNyrwGCZAACZBAwhKg4Cphlz6oGw9GcHXNf/+rwpt/Hy7hy8b+fftkz949sn79ehXuLBakqEN5oKhUqZIMGny9pKamBmoaE/UlVXD14otjZeOGDX4ZIw3k0Dvu9Ftvr6Dgyk6DxyRAAiRAAiRAAiRAAiRQMglQcFUy15V3RQKxQCBWBFfYdHY4ZeCJ0rhx4yKhoeCqSPgK3fnuu++WcePGWf1nzpxpuF799ddf0r17d6s8GDcqNM7LyzOEWtu3bzf6litXzhBxeTweuf7662Xy5MnWmBBn1a9f3zr3dRCLgivME6nQFtk2Zr722mty3nnn+boFlsUIAQquorMQEFyVLl3acrIqaSkDnRRz80T2HMg3RFbzlmfLYnW2Wrk5R3bvz5ND2SINayZL49oeSa9YSippCr/iCKQ03JaVJ8vW58iarbni0RyCpT0i5cskSb30ZGmXkSJt6nukTYOUhHK7ouCqOD6NvCYJkAAJkEDCEKDgKmGWulA3GozgasiQmyU9wG6+jRs3ymeffqLCqzUB54GHVlf366+7C4pnF0TACdoalFTB1ROPj5Q9e/bY7tT7MJQ0gBRcebPjGQmQAAmQAAmQAAmQAAmURAIUXJXEVeU9kUBsEChOwdXcuT+pk9WJhtCqQ4cOYQNCwVXYUIY00IwZM+SKK66w+piuUy+++KLcd999VvnTTz8tV155pXXudnDNNdfI1KlTrSbfffedtGzZUjp27Chr1641yvGsc+7cuVYbfwexKrjq37+/fPrpp9a0hwwZIsOHD7fOeRB7BCi4is6afP3114YgsaSmDARF1ZVKjr627sqVTTvzZF1mriFkWqGuVht35Eqmipuycw9vtK+ublJwkGqkwqtamr4v2r/uYBbr1HFr1eZc2XxkbrgH/MyUpHkO0zTlYT11u2pY3SMZNT1SV+dYt2qSVE5Nkorloj1bzCx6QcFV9FjzSiRAAiRAAglIgIKrBFz0EG45XIIrXBIOV5/873/y66+/BJzBJZdeJq1btw7YrrgblFTB1aOPjPCyCndy7tSps/QMcicbBVdOejwnARIgARIgARIgARIggZJHgIKrkremvCMSiBUCxSW4+mb2bENoBfeScAcFV+EmGtx4u3fvlubNm0tOTo7RAeKrZ555xhBXTZ8+3Rrk999/l3r16lnnbgfvvfee3HTTTVaTl156SY477jhp3769VTZw4EAZOXKkde7vIFYFV4888oiMHj3amvZ//vMfL/cuq4IHMUOAgquYWYq4n0iOphDcf1Dk95WH9JUt81dly1oVXeXk5Eue/t5jT2pStnQpSS1bSo5qlCJN63gMoVO0ZEwQW2EuS9bmyF86x/0H8wXpBe2BuUB8VU7driqqA1d7nedxzUtLk1rJRtpBe9uSdkzBVUlbUd4PCZAACZBATBGg4CqmliPmJhNOwRVuDqKr99+fIvPVqtstatWuLYMGDQ6Ly9X+/fslMzNTtusrNy9XKqRWkNQKFaRy5cpFTl1YHIIr2JXv3LlDMrdlSlZWlpQtW9a4nwp6T1WrVtXdGkW36403wRWYYH2RxhLOXAcOHJByZcuFnYvbZ5Z1JEACJEACJEACJEACJJDIBCi4SuTV572TQGQJFJfgKpJ3RcFVJOm6j92jRw/55ZfDm0HbtGkjs2bNkhYtWsiOHTuMjhBkzZkzx30QWy1c/e1pxG688UbjHM5XZrzzzjty5plnmqd+36MpuMK9b9myxZoLOLRt29Y6tx9MnDhRcF9mpKWlyR9//BHSc9X58+cLUjo2aNDASK2I9Irnn3++4HkmI/wEKLgKP9NEG9FMzbda0/Kt3pJrpBFcr0KrLep0lbUfQitvsRX4eJJF0/eVkmZ1PZpe0CNpqaWMtH3RYLdXBVa79ubLio05smxDjuSo6xZSIPqKw2kGS0lNdeSqq2kGG1THS+es77WrJGn6QaQhjJZUzNcMw19GwVX4mXJEEiABEiABErAIUHBloeCBDwLhFlzhEjt37pQxo5+R3FzdHuESl112ubRs1cqlhe+qzZs2yQ8//iBb9aEBhFYQXPmL2nXqGA9V2rRuIzVr1fLXzCr/Qq3HlyxdYp0fVGEPRE+BonqNGl5NIPrqP2CAV5m/E+y6+/XXX+UfvW5m5nZDbAWBka+A+KpZs+bSomULadWqtQRjZ/z5tM9k2fLlXsOBXaBw3hPal05JkWuvG+TVNVIOV/j8LF60SBYtXqRslrquc/ny5aWZPjBr2bKVYeuenKzf/oKIOXO+l99++81vy9bKuPtpp/mtNytmqr307NmzzFPrvUOHjtLrwgutc38HP/30kz4M/NlftTRt0kTO7nGO33pWkAAJkAAJkAAJkAAJkECkCVBwFWnCHJ8EEpcABVeJu/aRuPPHH39cnnzySWNoj8cj06ZNkzPOOMO61LXXXisjRoywzoM5OPnkk2Xx4sVGU7g/QcAFpysEns0t1edWwQiLoim4QorMdevWGXPEP7766is5+uijrXP7wcqVKzW15gmWMxjqBg8eLA8++KC9meuxMy0hhFcQvoVj46jrhRO0koKrxFp4ODvBy8l8N07sCFQ7BPkQ3J3wjn8Y77Y2Rn/9yQHZASFWWqNCq6UbsmXe8mz5Y2WOOkblFXCMsnW3DjVzn9TRtH11VMhUX9+rVkwSlEUyMPdtu/KMOW/QlIIbtrv/7uScS800zNUjXZqnSNuGKVKj8uEUgx6deCnd226fPq4FvngPlje4F3dQcFXcK8DrkwAJkAAJlGgCFFyV6OUt8s1FQnCFSU396CMVssxznV/jxo2lX//gREkYaOvWrTJr5kxZsOBvY4eF6+COylL6V297fdBwxhlnuu7Owg7Ev/7609E79NNKlSrJ7UPvcO0IQREYwcI+GFGXc7DKldNUhNMjYGrGie9NkIULFzq7F+q8TJkycs/we736RkJwtUiFVjPU6n379kyvawVzkp6eLmeceZYK0gKL+fCwbMK77/gdFoI9OLEFimeefsraKWlviwd7w+6+J6AwDnMwH9zZ+5vHZ53dw3jwZZ7znQRIgARIgARIgARIgASiTSCWBVf4W3ry5MmFQoIfglNTU6VixYrGq1q1aoIfqeEuzCABEogOAQquosM5Ua6CTW09e/a0bvfiiy+WKfq8z4xJkyZJt27dzNOg3u+77z558cUXjbY1dNNlw4YNLRet448/XvAZDiaiKbiCgOqff/6xpvWRPqs96aSTrHPngf0eUYdnWjP1OWwwz9d++OEHueCCC7ye1z788MNy3XXXOS/D8zARoOAqTCDjYJgcFUlBILVPHZ72HcqXbM2Ymm2k+zssCkpWsQ8cnVI8mkpPU/7BvSkF5479yIe0z659+QIXq2Ub1dVqa46s25ajjlZ5si0rT6+hYqw8Q27kSgXiIqQVTEtNkiaaVrB21eQj13TtVuhK7EvPPiIQW6zpBHftzZO9BwLP035BMEktmyS1qqhQTOeboU5XjWomS2N16aqsLl1wuzJFU9nKAazA+oC+DinvXL2+/s8QZiH5CPiWTlHWmpEYzHGerOXFGRRcFSd9XpsESIAESKDEE6DgqsQvcZFuMFKCKwhlnh0zRvw5NWHScCG6+57hAcUoSB/32aefGkIoWNkWJeAQ1advX2natJnPYaIluIKg6HPdYYfUgUUNPPjo0/cSg6evseJJcHXo0CGZPGmisTPQ172EUgbHqz59+gpEYv4Cn63HHn3E64GQvS0+o8Pvvc8vW7TN3LZNxowZbe/mdXzllVcZ7ltehY6TJ0c94Sq6Gzz4ekEaTgYJkAAJkAAJkAAJkAAJFBeBWBZcffbZZ9KvX7+wocGGnWbNmsnZZ58tQ4YMMYRYYRucA5EACRQgQMFVASQsKAIBPFvCf8P37dtnjAIHquzsbOMYz4iWLVsmeD4YSiAdX58+fawuECPBsR6BNHq33HKLVed2EE3B1Wnq2P7nn/9uKg3k7LV792457rjjvNIQ1tGNiM8995yccsopfm8Lvz0gdaB9MymEzH/99ZdgQyojMgQouIoM11gZFb+AIF0ehFYQ/ezX/4Rl7c+T3Zrq70C2ioD0hXq0U62PIf4powKgCiqEqlROU+YdEQKZIqD9OgbS8W3YoRkl1uXI7yuyDeHV9t25EoTGygsLhEkQKFVWwVXrBilSX8VLEGDhWhBsFfHnG+tauA4cqHCXEF2t0rSHf6/KNkRj+1V8hmv5SyloDeI4wN/4EF41qpEsLet5pH2jFHXqSjLEYxBNQUwFVhC3Ia0i0hiCPwRupuAK9wnWZVVoVbGc8tbxILoqk3JYiIUxMOtoBwVX0SbO65EACZAACSQUAQquEmq5Q77ZSAmuMJHJkyfJ3/Pnu84pkBgFD0fGj39TNm7Y4DpOKJWlS5c2nLXq1atXoFs0BFe/zZsnH3881a/Ip8Ckgiho27adXKwPfvClwRnxIrjau3evvPP2W7J+/XrnLRT6vLaKlK686mpXW/cXXxzr+vlCCsW6dev6nQN2Tk777FO/9cfpTsceLukA9+zZI088PtJv/3Llysldw+72ubZ+O7GCBEiABEiABEiABEiABMJMIJEEV3Z0cDJ56qmn5KyzzrIX85gESCCMBCi4CiNMDmUQcAqbTCxIB1gYR8SDBw9KkyZNBO/O+OKLLwxnRGe5r3PnvEaNGhW0YHjVqlXSqVMna1gIyTa4PC8dOHBgAectOFy1adNG8CzqoosuEqRKtAfYXH/99fYi43kUxrrwwgsNl32IqbAhdu3atTJu3Dh58803BRsazYBzJNIt9urVyyziewQIUHAVAagxMiQESxBB7VYXp20qiIJQCmIrOC3B6QlCIwiQTGETfg6AyCdJD+BqBberSioEqqjCq4oqhEK7fzbmyHJ9rdycK5t25sr2PXmaQvCwcAuirWADvz0kqwiqQXUVLak7VMcmKlhSxyiIwPYcyJOd+w47ZQU7nls7Q9RVHmIoddPS950659WaBhGCsUVrDwuvdiuXUAN8IEyDQ1c1TS2YoeKr5nXhdKWut2WSjPvYo+zBGq5feeCtkAzeyhq/vkB0laQcUlR4VkZf4I15pldMlgrqplUcoisKrkL9JLA9CZAACZAACYRAgIKrEGAlYNNICq5+VDvpzz+f5koV9tZIl+Yr8OX99ddek9WrV/mqLlJZ+fLl5fobbiywUznSgqvly5fJW+PHGw8minQDPjqfrLvNTj/9jAI18SC4wq7Al196UTZv3lxg/kUtqF69ukA0BaGdr0DqwjlzvvdVZZT1PO98rwdazoYQiS1dutRZbJ2na0qSIUNuts6dB+iLMfxFq9at5dJLL/NXzXISIAESIAESIAESIAESiAqBRBVcAS6+S3z44YfSpUuXqLDmRUgg0QhQcJVoKx75+0X6P6TIc8ZDDz0kgwYNchYHdQ6HKzhd2QPpZ+FiD5FRMBFNwRXEUHDf8hc333yz3HPPPV7VeBYLt653333Xq9w8wX3C9Wqbur3bRVZmPd6feOIJ6d+/v72IxxEgQMFVBKDGwJAQ9iBl4N6DeZKpAqPNmu5vl4qY4GwVSuYPCIcgKqqg7ktwyfptWbYsXJcta7bkqKAoFInVYSiHhUWioqRkqVIhSZrWTpZmtT3qEpWiIqMkQxC2Q1P9Ze7O01R8eXJQHbngPhVMikI7dgiZIIiC2CpVhUsYu9qR1yEdE0KxRSq4+ntNtmzYruIxdezarWkSkf4vVHctCMeQFrBRrWRp3zjFuA5EavuUPcYLNiC8grANAq6aKuCqWuFwisXSKn7zsTc+2GFDbkfBVcjI2IEESIAESIAEgidAwVXwrBKxZSQFVwsXLhSIfdyiQYOG8n/XXOOzyU8//ijTpn3ms85ZWKVKVUlPT1fBziaBBXYwcZLu4jrjjDO9mkZScAVL8+eeHSO7du3yuqavEzzQhztTbm6ubNy40Xj31c5ehp1tt90+VCAms0c8CK7gEAWnqEjFMcceq/bmF/gcfumSJfLOO2/7rEPhMcdo3wt894VQDCkJTWt6f4PccuttUqVKFZ/Vs2fPkplff+2zDoU9zjnXsHT324AVJEACJEACJEACJEACJBAFAvEmuJoyZYrUqlUrIBn8eIUfjuG0C7eOTz75xPjx3NkR3ze/++47wYYOBgmQQHgJUHAVXp4cTQTPJE899dQCKPDf8ZYtWxYoD6bAl4jrAn1eBGFTsBFNwRWeKQ4YMECfrfreDNujRw/NKjDe59THjh0rjz76qE9HL58dtDAtLU2GDRtmXNNfG5aHjwAFV+FjGSsjwUEpR0VKO1W4tDbzsAsVXKOy1WUJoqlQAsIlCIqQcxBirfmaim+1puTLUvHWIU2PF2pATFQjLUnaNigtR2WkSE09rlYJjlCHXZ5ydX771eUK14IoaiuEV+qghfR8wQrF4J5VXlPzQSiGsavoNSFkQro+CL4Oi9E0zZ8KxuD+BccuOHctWJ1tpBuEy1YonCDsQmrAhupw1aYhHK5wrSRDuBVKqkLD8coQiYkhcINIrEG6x0g36NF1iJboioKrUD/VbE8CJEACJEACIRCg4CoEWAnYNJKCK9hav6Tp2tzCn/sP/hB/6slRkpWV5dZd2nfoIGerQxbSrpmxc+cO+VoFLH/+8YdZ5PO9bNmycvvQO7ycj3C9gzYb7AMHD8i4V17x2d8s7Ny5S4GdzknJyYYAzGyD919//UX+9/HH9qICxxBL9enbVxo1amylkIOYZ9XKlTJlymS/u8fMgbp3P01O7drVPDXenfeEwueee9arjfOkYcMMOe+885zFxpyqOX5gGPvC87Jp06YCbc2Ctu3aSZ8+fc3TAu8rVqyQN994vUC5swDrlZGRIRnKBj+cbNy4weACS3Vflu7O/ldccaU0b9HCWWz0hWgqD17MPgLCt0GDve3UzWZwLBuv1umB4tye50nnzp19Npsw4V1ZrLsh/QWc2GrWrOmvmuUkQAIkQAIkQAIkQAIkEBUC8Sa4+v3338VXGvlAsPBddOrUqXLvvfcWcOC97bbb5K677go0BOtJgARCJEDBVYjA2DwgAfy3vLU6hkNQawacmf7880/zNOT3xYsXF0jBN2bMGLnssuBdyaMpuMINYvPnrbfeavz/mvPZWfPmzdXxfY5fDtgAOnr0aKPv9u3b/bbDBkO4hkEAVLFiRb/tWBFeAhRchZdnLIwGZyuk5NuSlSfr1b0J6fKQ1g5CrMIGUuJBoLRqc45sUBHXVh37gCGCch8Rpn0pKiKC4xMERHXTVURU3SPN63gMZ6sKmkIPQih7QCx2UEVPSH+4Q9MgGoIrFV3hHvAy0/LhHQE9GMRIehnD1QqiKgiuUlVkZYit9BpI1wfxmK+AwxWEaUvXH0mVqG5XcNiCYA2iMjduuG55FYtVq5Qs9arpvanoCucQYRU2MCaYVdb0h/U0zSJEY0iF6FGnq2gEBVfRoMxrkAAJkAAJJCwBCq4SdumDuvFICq727t0rj498zHUeEEoNu9vbvhodVqxYriKcN1z7durUWXr6EAWhE8QzE959xzXVG9r1vvhiOeqoo3HoM/Aw4pERD/usMwvP7dlTxTSBU0u89uo4TY+42uxW4B2uVoOvv16qVk0vUIeClSq6evut8QJXJX+BXddDbr7FX7VV/ugjI1zFW25srUGOHBRVcPXG66/rva1wDut1XqNGDRkw8P8KuHeh0Z49ezT15KteD9G8Oh85wY8t/732Ol9V8srLL8m6det81iWreO6e4feKx+MpUD99+ufyg8vDKbMDdk9edvkV5qnX+5OjnvArLExNTZU77rzLEt95deQJCZAACZAACZAACZAACUSRQKIIrkykP//8s7rknu/1/auapgvHj/X+0pWbfflOAiQQGgEKrkLjxdYkECoBZAPAhkcI0PCsqW7duoazvq9nXc6x8Yx1/vz5smDBAsnMzDSeYWEjZEZGhjRs2FAaNGjA/190QovCOQVXUYAcxUtAHASB0gpN+bdxJ8RWh0VKbqKhYKaH/hBdQby1UQVKi9fmqAOVip9M1ZOfQSB+gtiqiabb69iktCGyylBRUjkVJUEEBUEWBFP2gI4K1zNTCUJAhtceTdG3F8IrnKvwCsIsdNVH7oZAKUXfKxxJgZiij99xDncuXAMiJsdlrEvC+QvCKnDbvFP/O6UuXgs01SBSDkK4lgexmtX63wOMhzSA1TT9X6v6HsO5q6KKu3BNXK8ogf5Ih1hJx6tTJVka1/AYLlpFHTeYOVFwFQwltiFlZH/MAABAAElEQVQBEiABEiCBQhKg4KqQ4BKkWyQFV0D48EMPuqZbg1Xs/Q88qH/kem9V+FHTCX4eIJ3gtdcNMh4Q+FuqzZs3ywvPP+ev2ijv1r27dO36H79twim4CiRygvsSXJjcIlB6QIiD7r3v/gI8nWMGmku0BFebdLfc2LEvOKfndQ4XtIEqtqpQoYJXuf0ELl4QXbntuEP7/7vmv8aDIHtfHH/xxQz5Xm3l/YW/z9rz6hS2ZcsWf92s8jJlyshdw+7WL5L6jdEWgUSJbdq0kb6XXGrrwUMSIAESIAESIAESIAESKB4CiSa4AmW4XL300ktewKdPn65px4/xKuMJCZBA0QhQcFU0fuxNAiSQeAQouCo5aw6RElLvwZlpuQqutqkLFYREATRRQQPA+BgvU8ddom5Qm3fkyR4VYDnT5uHnmbKaYq+6uj7VqqIuTdU80rC6ioZqeYwUgnC6Uk1S0IFkErgGUv0d1OsfFmKh7LAMCgKnw6kPD18X6f1wHso1MBmMu0ddvNZuyzXSJq5ShuvU+QouYTs1vSHSKNoDjlMVVExWR12omtdNlioVkoyUheESReEe4HRVMw2Cq2TD8aqsOneFgM4+3aCPKbgKGhUbkgAJkAAJkEDoBCi4Cp1ZIvWItOBq9DPPqAgm0xXpnXcNM3ZX2Rv9rTunlixZYi/yOk7Sv1wvuKCXq/MPdmCNePghrx3JXoPoSceOx8gFvXo5i63zcAmucnNzZepHH1nj+jqAuKZlq1a+qqyy2bNnyUxNl+gWt90+VCpXruzWRGJFcPXpJ/8T7Bx3i4FqSY4Uh4ECLllwy3KL9u07yIUXXVSgyT///GO4hxWoOFIAJzWI0Oyxa9cuI+2lvcztuP+AAUaqSHubf5Yulbfffste5HUcrHuaVyeekAAJkAAJkAAJkAAJkEAECCSi4GrGjBm6KcbbqfYVTTnfy+U7ZATQc0gSKPEEKLgq8UvMGyQBEggzAQquwgy0GIeDYAip8LbsOpwiD+5MkQik+luzRYVIKkbakKmiLhVC2aO0Cp7SUpOkQ6PScnSjFGnXUN2fKicbKfEggiqsIMm4iu1S5qGXAAnj2ycT4jHGzFdsEHdlqTvYQnXy+nHJIVm2IVtWajpFe0D8VEdTJNatlqyCMo+RwtBeH67jqirkqp+ebAjYqqlYrbD8gp0PBVfBkmI7EiABEiABEigEAQquCgEtgbpEWnD17LNjZNvWra5Eh95xp1SsWNG1TWEr4Z4EFyV/0aRJE7m6X39/1RIuwZXfC4RYsXjRIpkw4V3XXv93zTXq4tTQtU2sCK4CfT6qVKkqN99yi6uwzrzRfN2u88zTT8vOnTvMogLvEKJBkOaMQ4cOGSI0iPR8RUfdwQ6Bnz3mzftVPp461V7kenzSySfLGWec6dXmm9mz5euvv/Iqs5/ceNMQqV69ur2IxyRAAiRAAiRAAiRAAiRQLAQSUXC1Zs2aAm5WDzzwgFyvqeCDDbjwfvXVV/KdOupu2rTJSOcEB1z8nY/U46effrpxDacbbrDjY3PPl19+KQsXLjTS0CMVPZx0kW6+atWqusmoo5x66qnGteAwHSiQNgrzNAN9zRSKW/W7/fvvvy+//PKL0Qbp7pFOKiMjQ0477TTp0qWL2a3Q70hZhRc2xWATFtYA3+OQzhHOYt3VpbqFukMHG2vXrpXFixdbzTt06GCMZRYs0u/YH374oXG9jfrsoGzZsnLUUUdZr+bNmwf1fdQcz3zHvLEuq1atMl6YR/ny5Y01Qfqtk/X74YknniiVKlUyu4T8fuDAAfn+++/1O+XXsnr1asH64DstWGFdunXrZqx9uXLlQh472h0ouIo2cV6PBEgg3glQcBXvK/jv/JHyb/W2HMORabs6MsHtKhIBMdKO3fmyZmuOLNW0e0jFh78NIQyqXilJGqobE1711dmqrro/1UhLkvKaQjBUx6lIzD3YMSFeg5vWtl15slaFZWu35ur94nWYL+65tKYtbF4vReqpe1fVCqWMFIDBjh9Ku1RlV6VCstSrqmxV2BVpjhRchbI6bEsCJEACJEACIRKg4CpEYAnWPNKCqzFjRkvmtm2uVO+48y7XdHGunQNUvjruFeMBrb9mcE6Cg5K/iDXB1fLly2T8m2/6m65R3q//AGncuLFrm1gQXAVKp4cb+M9/usl/9CFxsDFz5tcye9Ys1+a33nabpKVVKdBmnO5UX7t2TYFyFNSqXVsGD/b+UcXfvzsej8enq1rNWrX0h5kbvMZ/T8VzeMDvK5BCEf9uMEiABEiABEiABEiABEggFggkouDq999/100TZ3jhf/XVV+X888/3KvN1sk2/Bz/xxBPqaPu2z+8H9j7YCIT0heecc4692PUYm0beeustTdE+Vr/HrHVti8p27drJsGHDDIGXW2P8gPvxxx9bTSDkgnhr9OjRMmrUKNd76dSpkwwdOlS/x/3H6h/swfLly+Wee+4xxEOB+hx//PEGL1wvULzxxhtyxx13WM0mTJhgMIAw7cYbb5S5c+dadb4Ojj76aHnkkUeCFpNBLAZWn3zyiUAM5xYQ3g1QJ+QhQ4YYjN3a2uuw9vgcPqOO4jt37rRXFTiGWO0W3cR0jW7MMoVzBRrFQAEFVzGwCJwCCZBAXBGg4Cqulst1shAILVIB1GpNiQfXqZxw5RJ0XBX7jLPV7Gm1ulz9vvyQ7FfxUbJmEWlSK1la1PVIp6Yp0qq+R8qpA1SKJ7BI3zF8TJ4iPeNGTaE4d+kh+W15tmxVFzGkTjyqcYrUVfepFBVfRUoI5dG0gmWUY6OaydK6boqRLjGSkCi4iiRdjk0CJEACJJDwBCi4SviPgCsAf6IRe6chQ26WdN0hWZh4fORjxu5at7733f+AQKQSasDRCA8Xs7J2qRPVITl08KB+KTmk/rH/jjR1qnsav1gTXOHBKR7M79+/37ifg4cOSh62ZhyJZcuWyd9/zzdPfb7Hi+AKu5Xffedtn/dgFg78v2uMnbnmeaB37Op97dVxrs369OkrbfXHBmd89dWX8u033ziLjXPsNr9n+L3W5xS7hkc+9qhgR7EzkKLSX/pIp5vbU0+OEqQm9BWYI+bKIAESIAESIAESIAESIIFYIJCIgqvnn39eHnzwQS/8c+bMEbgeuQXa9OvXL6AYxjnGxRdfbIh1Aglj8H0EP7RC1BNqwJ0LLl3+wim4guMVRFTTp0/318WrHE4JI0aMkP/+979e5f5OsBEHwrRx48ZJdna2v2Y+yyEkgojMzbnLl+AKg+E+9+3b53NcZyGeV2B+5557rrPK63zevHnqjHyBz++JXg0dJ7V1gw9ctpo2beqoKXi6YcMGueSSS/xu3CnY43BJ27Zt5b333pNauhEoFoOCq1hcFc6JBEgglglQcBXLqxP83HL0sT8crRauy5Y1KrjKVbGV/uQRkcC4EF1t2ZknyzbmGOkDM2p4jLR3dTXFXs20ZHVkKqUpBOPL1coNFtjC1WqT3vPG7bmyQtML7tibJ1U0xZ/p3qV/ukYkIORK1n80Utew1uqoVVrFV6pvi1hQcBUxtByYBEiABEiABETFGX/L8OHDDRT33X+/tG7dhlhIwCIQScEVBFEPPnC/YWlvXdBxgB2dELIEG0ht8Lc+8F2xYrls2bJFIFAqShS34AoOWkuWLDbSP2zUB6cQkIFbUSJeBFe/6cPoQIK4W269VZBWMNjYsWOHphV8yrV5jx7nyHG6I9oZgdzD/nvtdVKvXj2j25o1q+VVfeDuDKSfuPOuYbrL/AXZqp9PZ/TqdaF00JQeiEAOXz3PO1+C2bHtvAbPSYAESIAESIAESIAESCASBBJNcLV7924j5RvSzJkBIRQ2ebhtGEIaOYitnN9VkdYNKQRbt25tbLqAe9b69evNoa13uENNmjTJVUQEN6yXXnrJ6oODKlWqSPv27aVmzZqGgzTmjVR6cI5yxmOPPWYIjpzlOHcKriAy+vTTT62maWlpAoepzp07y+bNmw2HqL/++quAmxOEULfq9zm3ACOIh5Bu0RmpqamGKxfcm7FZB8IvJ1P0ufTSS+XZZ591drfOnYIrzGvMmDFeYiusJ66DdzgQ+/pOjvXDHOAY5SvgmHX22WdLZmamVzXSEyKFINYFAi+4kf36668FRFl16tQxWOI7pb9AX7ig2T+TaIsNQhBr4bOFZyz4bCElI4R59sD32S+++CIm09ZTcGVfKR6TAAmQQGACFFwFZhQPLQ6qo9UeFQQtUsEVUuBFI/YeyJftmloQQqD2jVKkmqYTrJIaQSVQNG4qwDXwa8shdRJbskHTDGr6RgixsnOL9htMgEta1RmaThCCq3KlJWLpC3ExCq4s5DwgARIgARIggfAToOAq/ExL0oiRFFzB/Qep69wCYhqIagLFP0uX6oPBGcYD3UBtQ6kvLsEVHKxmzZopv/7yi2tKhlDuxWwbL4Kr7/WhOtbULe69735JSUlxa+JVl5OTI/ghyC266g8Y3bp1L9AEu6nxefWX9uHcnucZPyqg49dffyXfzJ5dYIw2bdpI30sulS9mzJDvvy/4o0Hbtupa1fewaxUegL/91vgCY5gFRXGWM8fgOwmQAAmQAAmQAAmQAAmEi0AiCa4gjLnssssETlX2QAq6++67z17kdYx+xx13nJcgBt9nbrrpJkN85HSugoDmhhtukB9++MFrHDhr9T3yvcGrQk+caQ6TNC/Kww8/LFdeeaVAFOSMGfrdBA5VdpEOUgTCjQmiJmc4BVf2+t69extiJed9/KLfa6+++mrZunWr1Rzz+uqrrwzRlFXoOBg8eLBMmTLFqxTCpOeee05OPfVUTfny749/+L6GVIe4lz179nj1mThxonTvXvA7Hho5BVcY0xQiNWvWzHDjOumkk6x0e9u3bzfWHc5mENfZA5yvu+46e5F1DOEX7teMY489Vp588knBd0RngNOjjz4q77zzjlcVPlv4jPkLMJ42bZpXNQRxcAirXr26VzkYIRUiUg/aI5BAzd42mscUXEWTNq9FAiRQEghQcFUSVlEM96Ws/fmydGO2rFcHpmgEUgZWKpcstdOSjLR6ZfXRO9yXSnrkKN5Nu/LU7SpXNu3Ild0HvIXpkbr/BtU80ry2RyqULWWka4zUdSi4ihRZjksCJEACJEACSoCCK34M3AhEUnC1TR8iPvvsGLfLG45BcA7yF9jB+sH7U0K2y/c3nrO8OARXS3SX8YcffmCkDXTOJxzn8SK4+lJ31n733bd+bzlU9zNzIIimfKX6M+u7dDlOzvGTCgLpCJ0P1c1+HTseI0gXiHj5pRd97ka/8MKLpH2HDrJq1Up5/bXXzK7WO34AgQMWHvJ/881s+dr2QN5qpAcVK1YUpB9kkAAJkAAJkAAJkAAJkECsEEgEwRVSuM+cOdMQFcFR2R4NGzbUTRXfi5sDEUQ0zzzzjNWtUqVK6uo71VV0hE0jD2iKv5dfftnqB9HR3LlzfQqiIEZ66KGHrLZ33XWX3Hbbbda5rwM4UMEZyf49CUIkXyny/Amu7rjjDkPs5Gt8lMGtCykRsbHEjC5duni5Y5nleIfI7Pzzz7cXycknnyyvvPKKVKtWzavcfgLHrv79+3s9I8jIyDDWBt8hneEUXJn1ENRBEOVvgw8+C3Csggu1GY0aNTLWxZnCEJt24DBlCsGqVq0qP/30k+E6Zvb19Y60ix999JFV1apVK/n2W9/fkb/55huB4M0ecPaCgMotIGiD05i59pg7XK7ghhZLQcFVLK0G50ICJBAPBCi4iodVCjzHPeo2tWtfnvyzKUc2qggoGlFN0+k1qpEi6fqeVr6UuqpG46rFfw0Yf+5UcdvWrFxZeSS1YDRmVS89WZrVSpGK5UpJapnIwabgKhqryWuQAAmQAAkkLAEKrhJ26YO68UgKruapTf7HH091nccxuuvz/PMv8NkGDwThAIRdv5GKaAuu/vzzD/noww+tHbWRuK94EVzNmDFd5ugPFv4C4qRhd9/jr9pv+cjHHvVKD+FsiNQXcKvyFTO//lpmz57lq0pq1aolg6+/wUgF+MTjIwukmcCD6zvuvMv4UQQ7pjEP86G2fcBr9KF6/foN5L33JsiihQvtVdbxUUcfrQ/TL7bOeUACJEACJEACJEACJEACxU0g3gRX+LvfTRwFnkgdB1eqrKwsw53JLq6x8y5fvrxMmDDBSDFoL7cfr1mzRk444QRB2ngzII5CyrxgAmKl2TYX3SFDhsjw4cMLdHU6KaGPLxclZ8drr71WN/58aBXjh1qkFnSGL8EVXLs++eQTZ9MC53B4cgqAIPjp2rVrgbY9e/Y0RElmBVIVItWev5R9Zju8Q3QFVyqI1cy4/fbb5c47C25a8SW4qlGjhnFtbHRxC6RShLjLHhBSNWnSxF5UwHWsR48eMn78eK82vk5wH2BrBr5TIqUhHMjsgftEqkmkiDQDnyt8voKJ0aNHG25XZlukrv/ss89c01aabaP1TsFVtEjzOiRAAiWFAAVXJWMldx8RXC2LouAKQqtGNTwC4VVa+aSEEVzlQnCl4ratWXmyakuO7NgbHYcrCK6aquCqEgVXJeNfWt4FCZAACZBAYhKg4Cox1z3Yu46k4GrSxPdkwYIFrlPp2/cSadO2rc8276uz1V9//umzzl6Ih6R169Uzdo+mpVURjyfZqv7kf/+zjn0dRFNwhR3SL459wW/KOnN+eMhav359qaoPWatqysXUCv+meViuu2wX+hHpmP3jRXD17bffyFdffmlOu8A7ONx3/wOSnPzvehZo5CiA0OnBB+4vIIayNzv11K7S/bTT7EXW8cqVK+SN11+3zu0HcKUafu99slA/0/hsOqOefgbtbm2TJ01Sh8H5zmZipjR8SndT79r1725pe8PzL7hAjjnmWHsRj0mABEiABEiABEiABEigWAnEm+AqXLAg3HrhhRckIyPDdUikmvvggw+sNhDITJ482ToPdAAXKntaPKTtmz9/vsAtyR6nnHKKl7vT008/baQTtLfxdfzHH38YwiDcB5ya8P3F4/EUaOpLcAUXJgicgokzzjjDuI7ZduDAgTJy5Ejz1HhfuXKlla7drLj33nuN1IvmeaB3uHq99dZbVjPwWrp0aQFXMF+Cq2CFcBDjQVxlF3bBsezEE0+0rosDpzCrTp06At74ThsokPIPjl5YF7wgPHPG22+/bbhUmeVoDwc0OKgFE/v37xekOLS7tr377ruCtYqVoOAqVlaC8yABEogXAhRcxctKuc9z78F8MVMKbohSSsHqlZKkcU2PpFdIUhFQ4giu4HC1a3+ebNudJyvU4Wr7nugIruofSSlYkSkF3f9lYC0JkAAJkAAJxDIBCq5ieXWKf26RElxB+PL4yMdc0+bh4eNdw+4WOBk5Y/nyZTL+zTedxV7n2K18qu6URYo4Xw+K0fjVca8Idhr7i2gKrgLNBXNs0qSpnHHmmVK7dm2fUw6GS7wIrn799Rf538cf+7xPs/C224cGtcPZbI+d6U+OesI89fl+1tk9jJ3nvirxIB0pCe0P1O3tIKiaqzua4VTmjG7duhtiKrP8T33A/sEH75un1jt+2Lj8iiuNfz+sQsfBzbfcWuCHFUcTnpIACZAACZAACZAACZBAVAkkmuCqZcuWcvnllwvSvmHzRaDo2LGjlzvzU089JVdddVWgbl71HTQ9+bp166wyuErZHZBQceWVV8r06dOtNkg/+Pzzz/t0kbIahXDgFFwdf/zxAjFMsIE5DxgwwGqOzUS//fabdY4DCKXsaRAhIEIbX88GvDraTjZt2qSbVI6RQ4cOWaVw2Dpa3YLt4Utwhedk4BZMONcETmenn366V1cI47p16+ZVBqcvOIilpv67gcqrQQgnN954o0ycONHqceGFF3qloLQqXA5uuOEGmaSbgswIJhWl2TYa7xRcRYMyr0ECJFCSCFBwVTJW80B2viCt4KL12bIuMzopBWtUTpbmtT1SJTXJSHGnP9EkROTli0Dgtm13rvyzMUcyVXgVjWhY3SOt6qZIec18XcYTOdhMKRiN1eQ1SIAESIAEEpYABVcJu/RB3XikBFe//TZPpuouWLeoW7euXHvdIJ9NJk2aKAv0Iai/gFjrqquvNgRK/tqgPJDIKVqCq63qbvXcc8+6TVUyMhpJf30w7bYLtiQJruDUNVHT6rkFPh/4nAQb69evl5dfetG1+UUX9Zaj27f32+b1116TVatW+qw/59xzZfasWUZaQWeD6wYNFuxkNmPv3r3iL/Ug5uDLJQt9kUIDQjMGCZAACZAACZAACZAACcQSgUQQXJ111lma8v58gYsU0s4FGxD9QFiEjUdmfPvtt9KqVSvzNKj3q/U77rRp06y2Y8aMkcsuu8w6xwFEMxDP2APfIeFWhFR2EP4gHXphwym4Qpo+pOsLNpxp8tBv3rx50qBBA2sIZ3pDCJggZAo1Tj75ZK80ey+99JJcdNFFXsM4BVdwwoKoze17t30Ap6PYm7ox7JxzzrE3Mda9S5cu+j1ylVc5vstiPrg/OEz52yjm1cnHCa73888/WzUQcpk/tFuFAQ5eeeUVueeee6xWffr0MZzbrIJiPqDgqpgXgJcnARKIOwLm/w/07n2x9Na0xLEc5t+QbTXLxYgRI2J5qlGfW45qrPYdOiy4WrMtV3JVFaQZryMS+uei/v0jUjst2RAAVSpfKqICoIjcRBEGBddDufmGs9Xi9TmyJStX8vVP9wjhliRlnaT/aFQjWVrXS5HSKrbyBN7DUeg7pOCq0OjYkQRIgARIgAQCE6DgKjCjRG4RCcEVHjaPGf2M7N692xXt+edr2jR96OgMuAs99ugjkp2d7ayyzjt2PEYu6NXLOvd3ECuCK4h0Zs782t80jfLb9CF25coF0wfYO5UkwdXOnTvkad317RY9zztfOnXq5NbEq+6XX36WQGkkb7zxJqnu8uPJrFkzZdbMmV7jmic19YeLzbqT2hkVKlSQoXfcWeCh/bhXXvba5W72q6G7qbds3myeer23b99BLnT8SODVgCckQAIkQAIkQAIkQAIkUAwEzB/LWrRoIUOHxtYGgc8++0z69evnRQUuUPYNEaiEIArfVzdu3KgbIN6XKVOmyIEDB6x+SNF26623yqBBg4JytTI7IpWdPc0cxDwQKgXjjGWOgfeZ+j3kJ3XUNWPIkCEyfPhw89R4x/yRrhDX9BcQeiE94WmaSh0pEVNSUvw1LVDuFFwFm37PHOjgwYNGukLzHO9YH8zDDAjDfvnlF/NUfKUdtCpdDuBA9sUXX1gtIAwDd3s4BVdIp2gXL9nb+jo+Ux2o7Q5dr2sK+p49exZois/S4MGDC5SbBRUrVjSEfFgXvJyfTbOdr3es57Zt26wqiPAyMjKs82AOIISzO1xBAPb5558H0zUqbSi4igpmXoQESKAEEaDgquQs5sGcfFm4LkdWb82VQ3oM0VUkAgKg5ORSUq9qsrStnyKpmuJOixIqQHbXvjz5e222bNqRJzkRFLh5lHWKiqwaq+CqjQqukiMotsIiUnCVUB9l3iwJkAAJkEC0CVBwFW3i8XW9SAiupn8+TX744QdXEBAW3XzLLfpHfnKBdniQ+OyY0QXK7QUXXniRtNd0C4Fi1BOPuwq/ouVwNWXyZJk//y+/001LqyK33nab33qzoiQJrnBPTz35pOzatdO8vQLv9es3kGs0hUewMU537a5du8Zv8/Lly8uddw0rIIyyd8Cu5Ndfe9VeFPDYnwDwm9mz5euvvwrY396gl6aH6NCho72IxyRAAiRAAiRAAiRAAiRQ7ATiTXD1+++/FxD+OCFu3brVcApyfn/t2rWrkfYu2BR3EHch1V+441x12IVgyBlr1qwR1EE4FiiwOQT3gzR0cMEqU0bzmbiEU3D1saaBP+GEE1x6FKxCOsbMzEyrAveA+ZqBNIXLli0zT+WBBx6Q66+/3joP9mDYsGHy6qv/fneDO5n9HOM4BVdwrPrggw+CvYQEK7jCgI888oiMHu3+LMO8MERUcK7q3bu3Onc3MYsLvGdlZbnWF+gQZEHVqlVlyZIlQbaOfDMKriLPmFcgARIoWQQouCo565mtLlertubIuu25snNvnhxQx6tIBByWKpYrJXWqJEuTmh4pVzrR5FaHqSKF4/LNObJpZ67s3p8v2ep6FYkoX6aUpKUmS730ZGlUPdlwvIrEdcwxKbgySfCdBEiABEiABCJAgIKrCEAtQUOGW3D1xYwZ8v333wUkdG7P87x2uNo7rFy5Ut54/TV7UYHj/gMGCnamBor77h3u2iQcgquzzjpbTjjxRNfruKWpQ8eMjEYyYOBA1zFQGYyAp1//AdK4cWPXsR59ZITXTm5n406dOkvP885zFvs8H/vC87LJh+uT2bhtu3bSp09f89Tr/YMP3pc///jDq8x5MuTmWyQ9Pd1ZXOA8U4V6YwII9Vq3bi2XXOqdksM5UDAOa84+GBNjOwM/gLw49gVnses5hHcQ4DFIgARIgARIgARIgARIIJYIlETBFfjCMQrp1ebMmeOFGy5S77zzjiAFXaAYO3as3H///YGahVzfpk0bma2bOHwFhDgPP/ywvPfeewJXqWACDl5IR4f79RdOwRXcnZAuMZQAOzyLMuNJ3WiDdIlmOAVZr2la9/OC/P5pjoF3J/eOHTvKDH0mYQ+n4CrU9IWhCK5wXbiU3Xvvva4OZPb54Ri8XnzxRZ/fe//880/DqczZJxznEL0hpX0sBAVXsbAKnAMJkEA8EaDgKp5Wy32uuZrWbktWnmzelSsbVHQFB6ZIRKoKgGpWTjZSCtaqkmSkuIvEdWJ9zAPZ+bJR3a0guALz/RESuFVJTZI66iYG5jUqJRnpHCPJhoKrSNLl2CRAAiRAAglPgIKrhP8IuAIIl+AKD3u/VCv/P/90F89gMtWqVZPB198gHo/H59xWrVqpDkPugqsz1Nb/pJNO9tnfLNy6ZYs899yz5qnP90CCK4hv8MOCWxxzzLFy/gUXuDUxBGQQkvkL7DIedvc9AVNOjHzsUdm3b5+/YYzyeBJcrVixQt5843XX+0HKEgiafLmhmR1zc3Nl4nsTAu7QvfzyK6SF7rYOFG/qDuwVK5YHambUY153Dbvb707xQC5r9osE63Rm78NjEiABEiABEiABEiABEogGgZIquAK7Xbt2CYQ4zu9svTSN/Svqohsonn32WUP8ZLaDSCstLc08LfQ70sYhHZ9bYO5Tp04VOFHNnTvXEJC5tUfdf9VF+MEHH/T5nTwcgisIn9auXWtN4+WXXzYctsyC4447TpDizoznn39e+vb1vUnHbOPrfeTIkfKULU39qaeeaqSKtLeNtuDKvPa8efNksjpdI+XhunXrzGK/7/Xq1ZM333xTjj76aK82GOess87yKqtRo4bXeWFPvvnmG+P5TGH7h7MfBVfhpMmxSIAEEoEABVclZ5Xz1WBp78F82aHuVivUeWnbbk11p65L4cosWEqNrJI1n2Ba+SRppOntqldKlgqaTjDSKe5idYVyckXgcgWx1Up1FoPLVa7y1mUISyB1I9IJQmjVWJ3EKpcvJXC70uKIBgVXEcXLwUmABEiABBKdAAVXif4JcL//wgquIHDJytol27dvF6RqWKC7V1EWKPDg+b/XXiduDwgh3npy1BOuQ9WuU0cGDvw/v7uNkRbuvQnvyv79+13HadCgofzfNde4tnlkxMOuO4arVa8ugwYNlpSUFL/jTJ36kfymD0rd4uKL+0i7o47y2QTCL4zxl+5uDRRX9+sfMOVArDhc4V4COWShTUtNt9C37yU+RVf43E2eNFEWLVqEpn4jXYV+N900xDWdoNk5GCcxs23jxk2kX//+5mmB948/nirzfv21QLmvgg76w0SvXhf6qmIZCZAACZAACZAACZAACRQrgZIsuALYP9R5t0ePHpKdne3F2enO5FV55OTTTz+V/rbvBNg08v333/tqGtEybM7BdWfNmqWpzb8uICCzX9xX+j3UOwVXEMIgBWCwkZeXJ3Xr1hV8hzXjo48+0g1TJ5mnctlll8mXX35pnd95551y++23W+fBHtxwww0yadIkqznSJkLcZY/iElzZ5/DPP/9Ya4L0lQcOHLBXW8fYiDVt2jQ5yvZcYMeOHdK8eXOrDQ4WL17s0w3Lq1GcnVBwFWcLxumSAAkUOwEKrop9CcI6AYir9qoIaNmRVHcQYOXkFF0EBJFPkiqAIPiB0KpprWRDeJWUJBEXAIUVUBgHg7BK/1yVzD15smxTjmSqwG3/oTyjrKiiK4jbUlRsBUGbPXUjyiMdFFxFmjDHJwESIAESSGgCFFwl9PIHvPlgBFcYxG4zjweoe/bskXxsvwgxLtbUBe3a+RYVmUNhXIickNrBLerXb2DskoWQBoEHuhs2rDd2yn6rOzWDEYBBuAWxlFsgTR3S1bkFUix07txFqmrqu1L6F/Re5VNXd6hWqFDB6DZHH3rPmDHdbQjD3ercnj314erRlpBs166dsmb1GmOn8po1q137m5XBuDjFkuBq/l9/yZQpk83p+32vXbu2PmhuIQ0zGkqtWrVl8+bNslqFdUuWLpGNGzb47WdWwIUMbmTBBFi/Om5cME3lrLN7yAknnOC37WIVgk1Q8V8wcdFFveXo9u2Daco2JEACJEACJEACJEACJBBVAiVdcAWYzz33nDz00ENeXMuWLWuIg5AGz19g88cpp5xiVcMFF25Z5cqVs8qK4wAbkb766isjNeKCBQsKTOHbb7+VVrq5xR5OwRVS3fXu3dvexPUYadXtgiE0RrpGu2ho+PDhXsKoSy+9VOASFmogDeGPP/5odYNz1yOPPGKd4yAWBFf2CSH9I0RXEBh98MEHBTaJ9dRnAq+/7u0CDQEfNruZAecspCEsSUHBVUlaTd4LCZBANAhQcBUNytG7Bn5lyVaB1fa9+bJFnZfWa2pBuDAV1XkJLlblSv+b2i69Qikpm6JuS1EQAEWPXuhXws9a+zSVIMRWm3flyYYdOXJQUw0ivWNhA0zhJFapXJLUS4eTWJIgraAnOTriNgquCrty7EcCJEACJEACQRCg4CoISAncJFjBVTgQdeveXbp2De6h4LRpn8lPtgenbtcvX768ka4BApxgRFb2saqrO9WN6nrkFhPefcfYQerWxlfdgIEDJSOjkVEFgdrTTz3ptcvXVx+UJekWk+qaImC/7k6G21eo0UdTMbRt2861WywJrjBRuJEFcqhyvaEAlU2bNpUrr7o6KHcrDIXP0WOPPhJQ9Ie2Q4bcLKboD+fOgHAQqSDtO7ydbczz24feIZUqVTJP+U4CJEACJEACJEACJEACMUMgEQRX+B5wzjnnCNK42aO9boqYPn26T8ddtIOIpkGDBroz/t9faT7//HM59tjgNnzYrxWp47Fjx8r999/vNfzDDz8s1113nVeZU3B1zz33yM033+zVxu3kl19+MZzCzDYQrMHhCe9mQFAEVysz4H4FF6xQo0OHDl7p+iDkGjLE+/t9rAmu7PeIZxhwGrOnV6xYsaKmt19hb2bwBFczfN2nWRev7xRcxevKcd4kQALFRYCCq+IiH7nrwuUqW1Pb7VDR1dptubJdHZj2HjycXjBUIRAcrDwq/ilXupSmtEuSBtWSJb1ikpTxJG4qQefKgSlEVluy8mRt5uHUgvvUWSxXFwJrEUpA2JaibFPLJEl6hcO8Kyn30hBbRUncRsFVKCvGtiRAAiRAAiQQIgEKrkIElmDNoyG48ng8ckGvXoZzU7B4d+7cKaOfedrrgXWwfUNpB8v+e4bf69rlr7/+lPenTHFt46uy/4CB0qjRYcEV6kNJLedrvGDLAjkuYZxYE1wh9cULzz8nu3fvDvY2g24HQd4NN95kuY0F2/Gt8W/KsmXLXJunq6PZkJtvcW2DymDGqlo1XW6+JfBYAS/GBiRAAiRAAiRAAiRAAiQQAQKJILgCNoiDunbtWmDzBZyvBg0a5JdsR00PvnbtWqs+VKESOt5xxx3q2rxB4KCMV6dOnYyXOSi+n3yjbs54xzzhTgXRVLABVyQ8IzLjmmuukUcffdQ8Nd6dgqvjjjtOPvnkE682bifDhg2TV1991Wpy9tlny1tvvWWd42D27Nly8cUXW2WlS5eWn376ybhnqzDAAZyt4HBlD7h5HX300faiiDtcwaEbYqGlS5caa4J1GT9+vCHA85qInxMI86666iqvWoyRlpZmld14440yceJE6xyfzykhPqN455139JnExwZjiAObNGkicNOKlaDgKlZWgvMgARKIFwIUXMXLSgU/T2h84LwEERDcrbap+9LGnbmye3+eQAgUSpRVoRXEP7XSdGO3phKsVO6ws5VqsKImAAplvsXRFqwhrNqvTldZyniLOl1tUt44P6BrEGxAUFVeeUNgVTsNwrZkTSkoUloFWOAdraDgKlqkeR0SIAESIIGEJEDBVUIue9A3HWnBFdx6Lrn0Mqmn6fVCDThcwekq0nHnXcMkNTXV72XgUPTkqCfkwIEDftv4qug/YIAKrhpbVRAVjXvlZcnMzLTKInHQuXNnOben94Nn53ViTXCF+W3ZssUQJhXG1ct5f+Y5UjrC2QrpCEON7zS9xpdffuHa7fjjj5eze5zj2gaVwXyWj9F0h0h7yCABEiABEiABEiABEiCBWCSQKIIrsB89enSB1HTYyPG9poqHEMpXOEUx+C4CUVCtWrV8NS9QhpR/EERBwGMGUhxecskl5qnhsnXllVda51WrVjWcguGSHEz069dPPvvs3+/YcK6CMMweTsEV6uA+BReqQLFp0ybD1QuOX2Y47wHlO3bsMNrZv/uFmlYQYiGItMyAiMjpTIa6aDhcdevWTebPn29OxRCxQcwWTCxcuFBOPfVUqynWcs2aNYLNYWZAbIXPlz3effddOeOMM+xFfo/xTAOCQDhqmYHPFdYmVoKCq1hZCc6DBEggXghQcBUvKxX6PCECgvvSrn15slXdlyAG2r0/33C/ys5FnTowaT3+ZMRfjRD8JOsLfw6m6AFeFcqWkor6qqZiqyqppTStHZ2t/K0EWOeosxgcxcAbYjc4ix1S1ihHPdYEvKGfAm+wRvrAFHWwgrAKrOEkBnFbRRW3ebQ+Ws5W5n1RcGWS4DsJkAAJkAAJRIAABVcRgFqChoyU4AoPo086+WTp0uU4SUlJKTSx6dM/lx/mzClUf1jxn3HmmfLB+++79u914YXSoUNH1zaLFy2SCZr2LpTo13+ANG7c2KvL9u3bDdHV3r17vcqDPTm1a1dZs3q1rFy50m+XtLQqcuttt/mtR0UsCq4wr127dhmiq61bt+K0SIEfH666up/gvTCB3ekQyLnF1fqDRZMmTd2aGHU7dmyXZ55+2rVdb93hfdRR3ruxXTuwkgRIgARIgARIgARIgASiSCCRBFfZ2dnSvXv3AmnPIYiaPHmyT+oQG8ENyv5d7wLdUDFu3Dif7e2FSGUIwdGsWbOsYrjp/vXXXwL3JzMgnGnZsqWXMzBSBdrdosy2zndcA+5PdtHN888/L301Jb09fAmusNEEgphA4XS3gkvTr7/+KpUrVy7Q9eWXXxakxjMjOTnZcO9q0aKFWeT3HU5W4GUPCJLuu+8+e5FxHA3B1ZgxY2TEiBHWteE89sUXX3ilUbQqHQfO9Iq+hGNIVXmmPtv4448/rN5oN0efldhTNVqVjoOnnnpKRo4c6VX69ddf6/fPo7zKivOEgqvipM9rkwAJxCMBCq7icdWCnzPEPYYQSJU+e1UABBFQloqudh/IM9yXDmYfFgNhRKSzg+injP7JWFFdrSqVSzJEPxBdQWgF8Q+UQhALMQoSUNSGmMoUXkHcttvgDfEVeGuqx5x8yVHhFRyrwLtsivLWF1jDPQysy5c5ImwD62KATcFVwbVlCQmQAAmQAAmEjQAFV2FDWSIHCrfgCrt327RpK8fpA1n7jsyiwPvj99/l88+nyf79+4MepqU+4DznnHONNHIjH3tU7DtsnYPgge7lV/y7S9hZb57PmfO9fPXll4IH1cFEv/79VXDVpEBTpM3738dTZcmSJQXq/BVAwHbmWWcZwrDvv/tOH97O8NfUKL9u0GCpU6eO3zaxKrjChPHjxg8/zBE4TOEHhVADAr8TTzzJEPzZf5wIdRw81H7s0Uf8fnYw9rC77xH8MBBMPPvsGNnmIiQbesedApEggwRIgARIgARIgARIgARikUAiCa7AH25JPXr0KJDm3k3gBAHTgw8+6LV8cKRC2j9/rspwtLrpppu8UsZhgNt0E81dd93lNRZOBg8e7JVOrly5coZrVbt27Qq0tRfcfffdXuIvOHBBwOMUQ/kSXGGcC3WjEoRFvgQ++O6Ee8T92+Np3XRid+Sy1+Xk5BiOXosXL7aKITJ78cUXjXKr0HEwadIkGTp0qNfzATgaz9Y0hb4220RDcLVq1Sqv1I+YMkRw4OHmPoZ+p59+uuzcudO6S3/rjs8j0jPaHdDgbo3PY8OGDa3+zoMJEybIkCFDvIpDTRPp1TlCJxRcRQgshyUBEiixBCi4KrFLW+DGDqnYBwKrfZrqDunustV1yXS6gsVVkqqAPHBa0hdSCZYvnaSCIDEEQQUGi3IBhGN7NR0iUiJi7kiXCGETwhQulVOhEsRKqfqKhUA6QYis9qvL1X49zs5RcRtcrlT8dtjd6l9nq3IGbxVfedRdTEVvxRkUXBUnfV6bBEiABEigxBOg4KrEL3GRbrAwgis8MMRDXTygTU2tIBUrVZRGGY2kWfPmEROMQKT04w8/6A7fP8WedsB583hAe+aZZwkEV2ZMfG+CwKbfX3g8Hhl+732uD0LNvti1/MWM6bJixYoCD97NNniHCAcpBRs08P/gc6GmjJg371dZvny537EgHmrbtp0htoLoCoEdyS8872793737aQI3LH8Ry4Irc8579uyRX375WZboA/iNGzd6PVg229jfa6rYr2WLltJJHzojlWU44u2335J/li71OVSr1q11V/VlPut8Fc6YPl13IH/vq0qqVasmNw252WcdC0mABEiABEiABEiABEggFggkmuAKzJ0iJZRB1INUgb7EPdg8csopp8iyZcvQ1IqMjAxjLKR1gzNRKf21Binnv/nmGxk1apRXOjp06tChg5HGz5dICwIlCG/wfckMbAaBsAk/vkJ8Y7pMHzhwwBCOQfj0rW5oscctt9xizMlehmN/givUtW3bVuBideyxxxr3j+/mEAJBJGV350JbiIE+/fRT415x7iu+081EEHLZA88brrvuOkPsBhEZvgfjOhCHTZkypYAwDe0//PBD3XRzon0Y6zgagitcDKK59957z7ouDuBGdueddxpzq1KlilEHwRS+00MIBW52sRW+x/7yyy8+P1voDOEU+tkDz2VwDXzumjVrZqw9Pod4FgiB3LRp07y+S+NzC3FTME5i9utE+piCq0gT5vgkQAIljQAFVyVtRf3fj2qWjNyBeIeACWG+Hz7Tf6rWB3IfCILwbp7jsDgDqQ837siVjTs1Xd+uXNm577B4DHMso6KwKhVKGWn4aldJllppScb8i3O+uLbFWw8M5lbhkZmZjPXU5I334g4Krop7BXh9EiABEiCBEk2AgqsSvbwJd3N4OIlUb5mZ24yHrnC9qlo1XWrWrGm8fO22jQQkPLheoUKpnZoCb68+6M7JzZHy5cpLOX0YXKNGDalXr55AyBVMIOXEak0TmKVjZe3OMh6Q1qxZS2rqOFVVQIaH8YkeENyt03Xfs3eP8cPCAV33smXLSWqFVKmgor86desK0lUwSIAESIAESIAESIAESIAEIkMgEQVX+K4GIc/69eu9oPbu3dsQy3gVHjn5888/5ZJLLpFt27b5qjY2h2ADkz21n71hc93I9Mknn/gV3aAt0tVBYAVnKWdg8w8cn/BdOTMz01ltnJ922mny9ttv+/zO6hRcYT5LfWxCwTVwD77mgNSFEAbhu3GgeOyxxwSCMF+Be8F3fX8bcCA0e/zxx+WKK67w1d0oi5bgCu7MEI/NnTvX51wgpsJ3Vmzi8uXkjHuBe9dJJ53ksz8Kt2/fbny2flcXcF+BMcAcvHw5c0PAB3EahH+xFhRcxdqKcD4kQAKxToCCq1hfocSdX44mBzmorlwQWq3PzJXNu1RslZWnqRE1PZ8ms4BbFAKuXHCIStV0fDUqJUnNtGSpl65/+1VOMlIkop4RPAEKroJnxZYkQAIkQAIkEDIBCq5CRsYOJEACJEACJEACJEACJEACJEACJEACNgKJKLjC7UPcdPnll9tIHD6EOKZbt24FylGATUID1PEYrkyhBFyKkIYOYqZAMX78eHnggQe8nK4C9cFmnr59+8oTTzxhuFb7au8UXOEa2FB04403eqXx89UXZWeddZa8/PLLhjOVvzbO8okTJxrOWXbXLmcb53lGRoa89tprctRRRzmrvM6jJbjCRSFwu+qqq+Tnn3/2mkOgE/CF6Ow///lPoKaCzV9IOzh58uSAbe0NmjRpYlzjhBNOsBfHzDEFVzGzFJwICZBAnBCg4CpOFirBpglHqH0H8mXX/nz5feUh+XV5tmTuzpOsfQU3CdjRpKUmSbWKydK5WYq0a5gilcuVEqQa5DZ0OyX3Ywqu3PmwlgRIgARIgASKRICCqyLhY2cSIAESIAESIAESIAESIAESIAESSHgCiSq4wsI7RUgoq1+/vnz//fd+hUVwZ4ZT1dixYwWORL6coDAO4tRTT5WhQ4dKly5dDhcE+U+koxs3bpzx2rFjh99ecILuqinnkfauU6dOftuhwnmvEFxdf/31RtpDpLSbP3++z/4Qi91www1BiYZ8DQD3ptGjRxtpA/25g6HfMcccY4iaevXq5Vc0Zh8/moIr87r4XEBAhZSJbgERFNzSwBeuZ6EExHxPPfWUkcrx4MGDfrs2bdpUbr31VsN9C45hsRoUXMXqynBeJEACsUqAgqtYXZnEnZfpbLVsU478tSpb1qm7FVIJHjiUL4fU8cotyqSowErdrpBasH61ZDlaRVcZNZI17WAp8SS59WSdSYCCK5ME30mABEiABEggAgQouIoAVA5JAiRAAiRAAiRAAiRAAiRAAiRAAglEIJYFV7G+DBAT/fjjj0aqN7ggpaSkSMOGDY0XnJqqVatWpFtAijqkqV+3bp2R/hApEHGN6tWrG+5Uxx13XNCCHn+CK3OCCxYskHnz5hn3UqFCBWncuLG0atVKcB/hCAjVIOrCs6ytW7caDl61atUyBG4QD+F68RKY/5o1a4w1wdpAFIe1xrq0a9dOmjVrVuRbgdjqp59+klWrVhkpB3EOMSA+X1iTOnXqSFJS7P9SScFVkT8KHIAESCDBCFBwlWALHge3u/dgvuFmBVerWX8flH0H8yTX3diqwF0l658scLvq2raMtM9IUderJEOIVaAhCwoQoOCqABIWkAAJkAAJkED4CFBwFT6WHIkESIAESIAESIAESIAESIAESIAEEpEABVeJseqBBFeJQYF3GW0CFFxFmzivRwIkEO8EKLiK9xUsefOHo9XvK3Nk8fps+WdjjmTn5ovq6EOKUppDsJy6WjWrkyKt6nqkfaMUqZUW+8LxkG4yQo0puIoQWA5LAiRAAiRAAiBAwRU/ByRAAiRAAiRAAiRAAiRAAiRAAiRAAkUhQMFVUejFT18KruJnrUrSTCm4KkmryXshARKIBgEKrqJBmdcIhkCeulgdUnHV4vU5Mmv+IVmzLUd27cuTvBDFVua1DrtcJUujmsnSTZ2uGut7aU8pSVIxFsM/AQqu/LNhDQmQAAmQAAkUmQAFV0VGyAFIgARIgARIgARIgARIgARIgARIIKEJUHCVGMtPwVVirHOs3SUFV7G2IpwPCZBArBOg4CrWVyhx5ncgO1+27c6Xv9dky7cLD8rWrDzJyVF3q0IigMtVSnIpqV0lWU5pXUZa1fNoasFShuiqkEMmRDcKrhJimXmTJEACJEACxUWAgqviIs/rkgAJkAAJkAAJkAAJkAAJkAAJkEDJIEDBVclYx0B3QcFVIEKsjwSBkii4WrhwgTz04IMGrqFDh0qLFi0igY5jkgAJJCgBCq4SdOFj8LZ378+XlVtyZb4Krn7+55DhbhWOaaZXTJLOzUpL2/op0qhGspQvQ4srN64UXLnRYR0JkAAJkAAJFJEABVdFBMjuJEACJEACJEACJEACJEACJEACJJDgBCi4SowPAAVXibHOsXaXFFzF2opwPiRAArFMYMmSJTJq1Chjir17Xyy9L744lqer4tMHZOHChdK2bVsZMWJETM+VkwudwPY9eYbYasGaHFm4Llv2HCist5X3tSuXT5LW9T2G4KpdA49U0nOGfwIUXPlnwxoSIAESIAESKDIBCq6KjJADkAAJkAAJkAAJkAAJkAAJkAAJkEBCEzB/LINLC9xaGCWTAAVXJXNdY/2uIByAgKB169Zy3/0PxPp0g5ofHa6CwsRGJEAChSBgF1zdd//9+t/ONoUYJXpdzL8hKbiKHvNoXmmbphD8dcUhWbg2R5ZtypF9B8MjuKpYrpQ0raWCqwYpckzjFElLpeDKbV0puHKjwzoSIAESIAESKCIBCq6KCJDdSYAESIAESIAESIAESIAESIAESCDBCbw/ZYq8//4Ug8Krr76a4DRK7u1TcFVy1zaW76wkCq7A+5K+fQzs5513nuDFIAESIIFwELC7AlJwFQ6iHKMoBLbsypO5yw4LrlZtyZH9h8IjuKpQtpQ0rOGRdiq46tI0RapUoODKbZ0ouHKjwzoSIAESIAESKCIBCq6KCJDdSYAESIAESIAESIAESIAESIAESCDBCVBwlRgfAAquEmOdY+0u8blDxENqrFDYUXAVCi22JQESCJaAXXA1cdLkYLsVWzs6XBUb+qhceCscrpZnGw5Xyzdnh9HhKkmaHHG4OtZwuCoVlfuJ14tQcBWvK8d5kwAJkAAJxAUBCq7iYpk4SRIgARIgARIgARIgARIgARIgARKIWQJMjxWzSxPWif3xxx+yceNGa8yWLVtKo0aNrHMekEC4CdhTY5U0wZUpMmAq1nB/ajgeCSQ2AQquEnv9Y+3uM3fnyZ+rcmTB2mxZvD5b9oYppWCl8knSsm6KtK3vkaMzUqRyeQqu3Naegis3OqwjARIgARIggSISoOCqiADZnQRIgARIgARIgARIgARIgARIgARIwEqPRfEAPwwkQALhIhBvwoFQ7tsUXKEPU7GGQo5tSYAE3AjEmyug+d/Ctm3byogRI9xujXVxSCBrf74s35Qr89dky7zlh2TXvryw3EV6xSQ5tklpaVM/RZ2ukiW1DAVXbmApuHKjwzoSIAESIAESKCIBCq6KCJDdSYAESIAESIAESIAESIAESIAESIAExPzBDCgoHuAHggRIIBwERo0aJXC5QsRDaqxQ7pnOgKHQYlsSIIFgCNhFqvHiCmj+/UjBVTArHH9t9h/Kl8278uTvNTny7cKDkrk7V3KLoLmCrCopSaRWWrKc0rqMtK7nkRqVk6RMCgVXbp8OCq7c6LCOBEiABEiABIpIgIKrIgJkdxIgARIgARIgARIgARIgARIgARIgAaF4gB8CEiCBcBOIN6eWUO//kr59jC50BgyVHNuTAAn4ImAXXMWLSJWCK18rWXLKIK6C6Grx+hyZveCgrMvMlSx1ucrLL9w9JqvYCukEM6p7pGvb0tK0lkfKqtgK5Qz/BCi48s+GNSRAAiRAAiRQZAIUXBUZIQcgARIgARIgARIgARIgARIgARIggYQnYBdcUTyQ8B8HAiCBIhOwCwfuu/9+ad26TZHHjLUBTKEB5jV06FDBfzsZJEACJFBYAvEoUjX/O0iHq8Kuenz0W7MtV+b+ky1LN2TL6q25kp2bL/khiq5KqYlVGU8pyajhkZZ1PdK5aYrUqZocHwCKeETGHwAAQABJREFUeZYUXBXzAvDyJEACJEACJZsABVcle315dyRAAiRAAiRAAiRAAiRAAiRAAiQQLQLmj2a4HsUD0aLO65BAySRgCgdwd/Hi1BLqSlCoGioxticBEvBHwC5SjZd0grgX829HCq78rWzJKM/any+bduTKbyuzZc7ig7LvYH7IqQU9R9ytTmxZWo7OSDHSClYoy1SCwXxCKLgKhhLbkAAJkAAJkEAhCVBwVUhw7EYCJEACJEACJEACJEACJEACJEACJOBFgOIBLxw8IQESKCSBeBUOFOZ2TbEB+lKoWhiC7EMCJAACpkg1nsRWmLf530AKrkCj5EZOrsiB7HxZtC5bfl6WLZt35sr2PXlyMFsMtyu3Oy+trlZwtkqvlCR1qiRJ52alpVltj5TRVIIQYTECE6DgKjAjtiABEiABEiCBQhOg4KrQ6NiRBEiABEiABEiABEiABEiABEiABEjAQcD84QzFFA844PCUBEggKALxKhwI6uYcjShUdQDhKQmQQMgE4lmkav7dSMFVyMseVx2QPjBPX7v25cnWrDxZsCZH5q/JNkRXKHOLtNQkqVYxSY5qmCKt6nmkmgqvKpVLkiQ1t0KaQUZgAhRcBWbEFiRAAiRAAiRQaAIUXBUaHTuSAAmQAAmQAAmQAAmQAAmQAAmQAAn4IHBJ3z5GaYsWLQzRlY8mLCIBEiABnwRGjRolS5YsMerizanF5w0FUWgKDtD0vPPOM15BdGMTEiABEpB4Flth+cz//lFwlRgf5uzcfMPVas22XFm5OVc278qVbbvz5MAhLc/Jl7wj2qtkda6Cs1W50qWkugqsaqUlS6OayVIvPdlwu/IkJwavcN0lBVfhIslxSIAESIAESMAHAQqufEBhEQmQAAmQAAmQAAmQAAmQAAmQAAmQQKEJvD9lirz//hSjP0VXhcbIjiSQcATiXThQlAUzhaoYg+6ARSHJviSQOAQgToVI1YyJkyabh3HzTsFV3CxVWCb6/+zdCbgcRaEv8MoVBJRIIoiCl1WvwZMIRL3K4gK4AIosmoRF3MUNQVzuuwqak0DYURHEFZBFICTBlR0BrxpkUQMXEtkX2QQCQZTlBoE3NdBjz3LOmTlnlp7uX39fnO7q6uqqX83lO+/N/6sqLXIVQul//lla7urJf4bw14efLv97oBS6+tujT4flpdBVXLTq+aXtAuPKVmuWVrZ66YRn/63wvOe2EIwrW7WlN8VpROCqOHNtpAQIECDQAwGBqx6geyUBAgQIECBAgAABAgQIEMi5QDp0ZcWWnE+24RFog0A6bDUwMBBmDs5qQ6v900R6a8HYa6Gr/pk7PSXQK4Fk+9X4/n5dEbBIgatjj/tOuPbaa0f8ujxvhdK2eauvEdZc8yVh49e8Jmy5xeZhhVJZL4+7774nzD7ooLoufP2oI8P4VVetK2+mIIav/v74M+GR0r9/PP50eLy0ytU/SytcxTDV80orWL2gtLrV+JX/Lfxs/knh6kVXVTW53bbbhvfusnNVWdYvPv/FL4VHH320qpuf3Guv8LrXvbaqrBMXAledUNUmAQIECBB4TkDgyleBAAECBAgQIECAAAECBAgQ6IRA8iNabFvoqhPC2iSQD4F02CqOqB9XaWnHTKSDqlYHbIeoNgjkVyAv268mfysWYUvBz3/pv8LChZe1/KVcbbXVwnt33il87GMfLa38tGLLz7fjgZtvuSXssecH65q64LxzwsQJE+rK21XwdGmPwXe9Z6fw0EMPVTV53LHfCv/5+tdXlWX94u3bbh8eeeSRqm4eevBB4W3bbFNV1okLgatOqGqTAAECBAg8JyBw5atAgAABAgQIECBAgAABAgQIdEogvU2W0FWnlLVLoH8FasNWMwcHw8DA5P4d0Bh7ng5dxaasdDVGUI8TyJlA3EYw/nczfsajX1e2SqZF4CqRGPnzFRtuGA4+6MCw4YYbjFy5zTV6Fbi67rrF4aN7faJqNKuWVtS6sBT06vWqX1WdauJC4KoJJFUIECBAgEA/Cghc9eOs6TMBAgQIECBAgAABAgQIEOgPgdptsoSu+mPe9JJANwSErRor14au/HezsZNSAkUTiCGruLJVcvR72CqOQ+Aqmc3mPtdYY/Vw0gnHl7YbXLO5B9pUq1eBq+9+7/vhRyefUjWKd7z9beXgWVVhH1wIXPXBJOkiAQIECBAYjYDA1WjUPEOAAAECBAgQIECAAAECBAi0IpD8oBafiVtlxX8xROAgQKB4ArUrtAwMDIRp06cXemWr2m+B0FWtiGsCxRaoDajmIWwVZzT5+9CWgs1/vzcq/Q190onHh3/7t39r/qEx1uxV4Gr3938g3HLrrVW9P3DWYNhu23dWlfXDhcBVP8ySPhIgQIAAgVEICFyNAs0jBAgQIECAAAECBAgQIECAQMsCAgQtk3mAQO4EakMDMWw1c3BW7sbZjgHV/jcztmm1q3bIaoNA/wjUBlRjz/MStopjKXrg6pWveEXYY/fdIkXlePLJJ0tbRt4YFl1zdbjtttsr5emTww89JGy91VvTRR0970Xg6p577g07v29a1bie97znlbcTHD9+fFV5P1wIXPXDLOkjAQIECBAYhYDA1SjQPEKAAAECBAgQIECAAAECBAiMSkCAYFRsHiLQ9wK1Qas4oDyFBjo1QbXbsibvEbxKJHwSyKdAo6BVHlcDLHrgavPNNgvf+ubXh/wSH3/ij8IPfnh83f1NN90k/OC736kr71RBLwJXZ86bH77+zaOrhvTaqVPD977z7aqyfrkQuOqXmdJPAgQIECDQooDAVYtgqhMgQIAAAQIECBAgQIAAAQJjFmgUvEq2Gkw+x/wSDRAg0DOBGBaIRwxaJedJZ/IYGkjG1qnPRv/NjO9KtmZNPjv1fu0SINB5gUYhq+SteQ2oClwNH7iK8z/rwIPCueedn3wVyp8rrrhi+O2vL+natoKtBK7iCl3/+Mc/wsSJE6v63OrF3vt8Llz1hz9UPbbfvvvUrQhWVaF08dRTT4VHHnkkTJgwIYwbN672ds+umw1cPf3002HZsmVlv3ZtGznumdLRs5F7MQECBAgQyLmAwFXOJ9jwCBAgQIAAAQIECBAgQIBARgVigCAeCxY8+1nbzRi8So70eVLmkwCB7AikQ1Xp83QPBa3SGqM7Hyp4lbSW/Lcy+UzKfRIgkC2B9H8n0+e1vcxr0CoZp8DVyIGrSy79dfjy/gckZJXPX/7sp+GlL12zfH3Z7y8P119/feVePFl//fXDNltvVVVWe3H2OeeG+++/v6o4riIVV9BKH0MFrn5z6cVh5ZVXDnH7v+NPPDFce+114c677goxNLTSSiuFtddeK2z1lreUVrR8X3jJGmukmxz2PAa23rn9u8M///nPqnpnzZsb1llnnaqy5aWA19lnnxOW/PnP4cabbgq33nJriGXx/WuttVZ4+cvXDu9429vCdtu+c8iA2uOPPxHOmDu3qt14scsuO4eJpeDWUEcc9/kXXFB1O4a8PvLhD1WVxYtGgaujv/H1sMXmm4XHHnssnHzqj8Nll/0+3H7HHeH//u//Qtw+Mc7v1E03DbvtOiNMetWr6tpstkDgqlkp9QgQIECAwCgEBK5GgeYRAgQIECBAgAABAgQIECBAoG0CccusJYuXhPLnkiVta1dDBAj0XiCGrOIxbfr0MDAwufcdykkPRgqs5mSYhkGgkAJFCqcKXI0cuLrhxhvDBz70kbr/Wzj91JPDK1/5ynL54UceFc76yU+r6sSw1WGHHFxVVnvx8U98KvzvtddWFX9yr4+Hj320+n2NAlcvetGLwq8uOC/86KSTwwmlrQ9jyGmoY4UVVgi7l0JDe3/m00OGntLPXvSri8MBX5uZLgrrr7demDf39Kqyv/zlzrD/V79WDlpV3WhwEQNon/7kJ8LWW7217m5cUWrbd+1QV37GaaeGV2y4YV15UnDFlVeFfT63X3JZ/oyrUl2+8LdVZfGiUeDq9B+fEu77633hoEMODQ899FDdM+mCLbfYPMwenBmie6uHwFWrYuoTIECAAIEWBASuWsBSlQABAgQIECBAgAABAgQIEOi4QBIkiC+KIaxnPwWxyhD+h0BGBZJgVexeDFYNTH42aCVk1fkJS0Kr8U3+m9l5b28g0C6BRv/dLNp/MwWuRg5cLTjrJ+GIo75e97W7+MLzw/jx48vlvQhcxbDX9tttG4799nF1fRuqYPPNNguHzDkwvPCFLxyqSrl85qzZpZWjLqyq84E93x/22fszlbILL/pVOOTQw8Jjjz9eKWvmJIbQalf+6lXg6sjDDysHy5YvX95M18O/v/zl4RtHHRFieKyVQ+CqFS11CRAgQIBAiwICVy2CqU6AAAECBAgQIECAAAECBAgQIDBqgfSWaHPPnDfqdjxIgAABAgQI9LeAwNXwgatHH300fOoznw1xlav0MX7VVcPFF/1rK7teBK7S/Wnl/K2lLQaPPPzQIR956qmnytsJ/v3vf6+q84PvfTdsusnG5bIHHnggvHf6ruWt96oqNXERw16n/OiEqq0JexW4aqK7dVXWetnLwmml1c1WLX0Hmj0ErpqVUo8AAQIECIxCQOBqFGgeIUCAAAECBAgQIECAAAECBAgQGJWAwNWo2DxEgAABAgRyJyBw1Thw9cwzz4Tbb78jzCltNXftddfVzftOO74nHPCVL1fKsxK4WmmllcJLX7pmiFv9DXd8df+vhB3fU7+FX3zmT4sWlUNm6ecnTFgtnH/O2ZXtCH94/AnhhyecmK5SvrfzTjuG17/udWHdddcJN954U1h42WXh4ksuraq3zr//e9h3n71DDH4lR1YCV3E7wnXXXTfcdddd4Z///GfSvbrPuLJY3F6w2UPgqlkp9QgQIECAwCgEBK5GgeYRAgQIECBAgAABAgQIECBAgACBUQkIXI2KzUMECBAgQCB3AkUPXMUJ3XKLzavmNQZtrr/hxvC3v/2tqjx9Mfe0H4cNN9ygUtTrwNXb37ZN2GXnncImm2wSnr/iiuHBBx8MV171h3D8iT8Kd95ZH7560YteFM49+xflupVBPHdy9DHHhtPPmFtV/K7ttw+zZn61UvaVA75aF6T6wPv3CPt8du9KneTk6G8dE06fe2Z4/etfF3bfddeydww2pY9eB642mjQpfOiDe4b/fP3rQ7SJ2yQu+tOicOb8BeHyK65Id7VyfvKJJ4RXv3qjyvVwJwJXw+m4R4AAAQIExiggcDVGQI8TIECAAAECBAgQIECAAAECBAg0LSBw1TSVigQIECBAINcCAletT28MDX1+v32rHuxl4Oo9O7w7xBWrxo0bV9WneBG3/otbIt5ZWrGp9jj04DnhbdtsXVsc3lfaKrC2/mGHHBy22XqrSt0v739AuOTSX1eu48kO735XmPnVA6rKkovYXlzZaqijl4GrSa96VfjOt48J48ePr+teDN/Fsf7mt7+ruzd92vvCf33xC3XljQoErhqpKCNAgAABAm0SELhqE6RmCBAgQIAAAQIECBAgQIAAAQIERhQQuBqRSAUCBAgQIFAIAYGr1qZ5i803C9846sjK1nrJ070KXL3lzW8KRxx2aF1/kn7Fz/vvvz/s8YEPhUceeSRdXNrS783hyMMPqyq7/Y47wozd9qgqiytmXXj+ueEFL3hBpbzRloLxZlxp6/177B4mDwxU6jZz0qvA1Vove1k45aQTw2qrrTZkN5988smw9z77hquv+d+qOvGZC8791zaLVTdrLgSuakBcEiBAgACBdgoIXLVTU1sECBAgQIAAAQIECBAgQIAAAQLDCQhcDafjHgECBAgQKI6AwFVzcx23wPvgnu8Pn9jr42GFFVaoe6hXgavDDz0kbL3VW+v6U1vwlQO+VtoC8JKq4rXXXiv87KwFVWWn/vi0cOxx36kq2+yNbwzHHP2NqrL77rs/7DJteogrQDU61l13nTB1003L/974hv8Mq6++eqNqlbJeBa5mTJ8WvvSFz1f6MdTJgrN+Eo446ut1t3/xs5+El730pXXltQUCV7UirgkQIECAQBsFBK7aiKkpAgQIECBAgAABAgQIECBAgACBYQUEroblcZMAAQIECBRGQOBq+KmOqzrFbfemvfe94dWv3mjIyr0IXMUtBC8495wwYcLQqzMlHZ6/4Kxw5NerQ1Nx5arf/s+lVVsRfuJTn65byen/femLYdr73ps0Vfn8n9/8Juz/1ZkhrgA13BHDaq977WvD9tttG961/XYNV+PqVeCq2cDarbfeFnZ7/551wzzhh98Pr5kypa68tkDgqlbENQECBAgQaKOAwFUbMTVFgAABAgQIECBAgAABAgQIECAwrIDA1bA8bhIgQIAAgcIICFyF8IX9Plc133EFqzXWWD2sueaa4RUbbhhWWmmlqvuNLnoRuFp/vfXCvLmnN+pOXdmNN90U9vzgh+vKLyptFZhsp/fww38L2717h/D0009X1fvlz34aXvrSNavKkouFl/0+/PdX9g/Lly9Piob93GTjjcOBswdD3MovffQqcHX+Ob8ML37xi9NdaXj+zDPPhLe/c7vw93/8o+r+YYccHLbZequqskYXAleNVJQRIECAAIE2CQhctQlSMwQIECBAgAABAgQIECBAgAABAiMKCFyNSKQCAQIECBAohEDRA1ebb7ZZ+NY367eKa3XyRxu42mmX94V7//rXqtd9srRt4cc++pGqsptvuSXssecHq8rGr7pquOiC8xquGFVVsXSxcOFl4fNf+q/a4vC70gpXz3/+88vl5553Xph14JyqOq/6j/8IPz7lpKqy2osHli4NP//FL8PPfv6LcP/999ferrueOHFiOPP006pW5hpt4Orsc84NB845uOodcUWtyxf+tqosXrx92+3DI488UlV+ykknho0mTaoqa3Tx6KOPhreVAle1YbT43YnfoZEOgauRhNwnQIAAAQJjEBC4GgOeRwkQIECAAAECBAgQIECAAAECBFoSELhqiUtlAgQIECCQWwGBq84FrrbccovwzaOOHPK7E1eUeuf276q732zgKj546sk/CpNe9aq6NmoLjv32ceHU06pXw4orW8UVrpLjy/sfEC659NfJZfkzBr9if5o5YhhpyZI/h+sWLw7XXndd+OOfFoWHHnqo4aPbvuMd4aADZ1XuDRW4OvnEE4bdyvGo0jaJ80rbJaaPVgJX+31u37DHbrumH294Hlfy+vwXv1R374zTTi2vglZ3o6ZA4KoGxCUBAgQIEGingMBVOzW1RYAAAQIECBAgQIAAAQIECBAgMJyAwNVwOu4RIECAAIHiCAhctSdw9d3v/yD86KSTq744G2ywfnklp6rC1MXlV1wR9t3vC6mSZ09bCVztt+8+YY/dd6tro7bgIx/bKyxesqSqOL161fInnwzvLK0A9djjj1fVGSnwVFW5wcXvL788fPPoY8Ltd9xRdTeGon5z6cWV1bXi+9+y1TZ1K0gdevBB4W3bbFP1bPrio3t9Ilx33eJ0UXnFr2ZXuHrzm7YMXz/yiKrnG10ce9x3wqk/Pq3u1iUXXRBWLa00NtIhcDWSkPsECBAgQGAMAgJXY8DzKAECBAgQIECAAAECBAgQIECAQEsCAlctcalMgAABAgRyKyBw1Z7A1S9+eXaYc8ihVd+TFVZYIZw1b25Ya621qsrjxfLly8PHP/GpcP0NN9TdayVwNX78+PDd444NMTw11HH6GXPD0cccW3f7Ix/+UPj0Jz9RLm8U/nrJGmuEs3/xszBu3Li6Z5OCq6++Jpxx5pnhTVtuGd6zw7uT4qrPPy1aFD71mc9WlcWL2jDXDjvtUrcl4U47vicc8JUv1z0bC3518SVh/69+re5eKytcxbF97YD9ww7vrl9pLGk4jnHfz38hPPHEE0lR+XOTjTcOP/z+d6vKhroQuBpKRjkBAgQIEGiDgMBVGxA1QYAAAQIECBAgQIAAAQIECBAg0JSAwFVTTCoRIECAAIHcCwhctSdw9cc//il8+rP71H1f3vqWt4S4SlMMX6WP2QfNCeece166qHLeSuAqPjRhwmrhe8cdFzbccINKG8nJ/NJ2e0eWtt1rdCw4c25Yd911yreOOOrrYcFZP6mqtsvOO4Wv/Pf/qyqLF0+WVqO66Fe/CnPPnF8JjMXxffuYo8Nrp06tq3/33feEXaZNrys/6cTjw8CrX10p/+Sn9w6Lrr66ch1PYnjq+O9/L0yZMrmq/JZbbw0f/fhe4fHHq0NQyTPNrnCV1J89a2aI2xzWHnH1rM9+br/w2GOP1d4K+3/5v8POO+1YV96oQOCqkYoyAgQIECDQJgGBqzZBaoYAAQIECBAgQIAAAQIECBAgQGBEAYGrEYlUIECAAAEChRAQuGpP4Grp0gfDDjvtXLclXvwSxbBQDOasPvHF4Yqrrgq/+93CcNfddw/5/Wo1cBUbev6KK4apr50aNt9ss7DeOuuEuKrUFVdeFW686aaG79nsjW8Mxxz9ryDWe3beJdx33/1Vdb951JFhyy23qCqLFz/56c/CYUccWVceV9uKq1xt9sY3hE023iQ8+NCD4eabbw4/OvmU8Oc/X19VP64sFbfje+ELX1gpP/zIo8JZP/lp5To5iWOL2yZusskm4d577w2/W7gw/KEUcIurhDU6WlnhKnk+9mejjSaV/TZ+zWvKbldceWX43/+9thwwS+olnxMnTiyvXtbMdoLxGYGrRM4nAQIECBDogIDAVQdQNUmAAAECBAgQIECAAAECBAgQINBQQOCqIYtCAgQIECBQOAGBq/YEruIXZ9aBB4Vzzzt/zN+h0QSuWnnp+FVXDWecdmpYc801y4/FUNaeH/xwVRMrr7xy+NUF54XnP//5VeXx4umnnw4fKa0uVRuiqqs4TME73/H2MOfA2VU1/vKXO8OM3fdoGFqrqjjCxWgCVyM0WXf7G0cdUd5Gse7GEAUCV0PAKCZAgAABAu0QELhqh6I2CBAgQIAAAQIECBAgQIAAAQIEmhEQuGpGSR0CBAgQIJB/AYGr9gWu4qpV03fdPTz11FNNfXFesMoqYdNNNwmX/f7yqvrNBq5e8pKXhBVXXCHcc8+9Vc8Pd/G85z0vHDLnoLD1Vm+tVDvhxB+F7//w+Mp1PHnrW94cjjz8sKqy9EUMaX3kY3s1XP0pXa/R+WqrrRZOOuH48PKXr113+6CDDwm/PPucuvKhCt7y5jeFyy+/IiwvbXOYHK0ErqL/1Vdfkzza1GdcbWu/feu3jxzuYYGr4XTcI0CAAAECYxQQuBojoMcJECBAgAABAgQIECBAgAABAgSaFhC4appKRQIECBAgkGsBgav2Ba7iF2Xe/AXhm986ZsTQVdwm75vfOCrcdNPN4ehjjq36jjUbuNp0k43DzK99Nez1yU+HBx98sKqNRhcvetGLwmEHzwmvf/3rqm5/6KMfq1ut6qv7fyXs+J4dqurVXsTtAmcfdHC44cYba28Neb366quHY7/1zfDKV7yiYZ0Hli4NX/jifzXV5habbxaOOuLw8PZttw+PPfZYpb1WAlf/c8nFYb8vfDEsuvrqyvNDncR29/3s3uXtDYeqM1S5wNVQMsoJECBAgEAbBASu2oCoCQIECBAgQIAAAQIECBAgQIAAgaYEBK6aYlKJAAECBAjkXkDgqr2Bq/iFue66xeGAr80M9/71rw2/P2984xvC3p/+VNho0qRw+hlzxxS4+sH3vhseeOCB8J3vfT+cd/4FDbfji9sDvmu77cKHPrhnWGuttar6FANOO+y4c3jmmWcq5ePGjQvnnf2L8OIXv7hSNtRJXM3r5FNODaeednp49NFHh6oWXvjCF4b3vXeX8JEPfbB8PmTF0o24WtUxx367HF5rVC8Gxz70gT3DrjOml7c83Opt7xh14Oryhb8tr9J12ulnhFNO/XF47PHH614ZPbbYYvPw4Q9+IGyy8cZ195spELhqRkkdAgQIECAwSgGBq1HCeYwAAQIECBAgQIAAAQIECBAgQKBlAYGrlsk8QIAAAQIEcilQpMBVNydw+fLl4eZbbin/u+XmW0MpsxPWW2+98OqNNgobbTSpI1258847w+Ilfw633XZbePhvfwsve9nLwsvXXjtsVgp4xZBSp4/77rs/3HrbreGWW24Nfyn1JQa9Ymhr8sCrw9RNNw0rrLBCS12I7cVVtG4qOd57773l8axfMvzP0gpdq666akttNVN52cMPh2uuuabkd3u4p/S+NdZYI6xdCqjFlcTWWWedZpoYso7A1ZA0bhAgQIAAgbELCFyN3VALBAgQIECAAAECBAgQIECAAAECzQkIXDXnpBYBAgQIEMi7gMBV3mfY+LIgIHCVhVnQBwIECBDIrYDAVW6n1sAIECBAgAABAgQIECBAgAABApkTELjK3JToEAECBAgQ6ImAwFVP2L20YAICVwWbcMMlQIAAge4KCFx119vbCBAgQIAAAQIECBAgQIAAAQJFFhC4KvLsGzsBAgQIEPiXgMDVvyycEeiUgMBVp2S1S4AAAQIESgICV74GBAgQIECAAAECBAgQIECAAAEC3RIQuOqWtPcQIECAAIFsCwhcZXt+9C4fAgJX+ZhHoyBAgACBjAoIXGV0YnSLAAECBAgQIECAAAECBAgQIJBDAYGrHE6qIREgQIAAgVEICFyNAs0jBFoUELhqEUx1AgQIECDQioDAVSta6hIgQIAAAQIECBAgQIAAAQIECIxFQOBqLHqeJUCAAAEC+REQuMrPXBpJdgUErrI7N3pGgAABAjkQELjKwSQaAgECBAgQIECAAAECBAgQIECgTwSSH1cHBgbCzMFZfdJr3SRAgAABAgTaLZD8TTBlypQwZ86cdjevPQIESgICV74GBAgQIECggwICVx3E1TQBAgQIECBAgAABAgQIECBAgECVQPLjqsBVFYsLAgQIECBQOIHkbwKBq8JNvQF3UUDgqovYXkWAAAECxRMQuCrenBsxAQIECBAgQIAAAQIECBAgQKBXAsmPqwJXvZoB7yVAgAABAtkQSP4mELjKxnzoRT4FBK7yOa9GRYAAAQIZERC4yshE6AYBAgQIECBAgAABAgQIECBAoAACyY+rAlcFmGxDJECAAAECwwgkfxMIXA2D5BaBMQoIXI0R0OMECBAgQGA4AYGr4XTcI0CAAAECBAgQIECAAAECBAgQaKdA8uOqwFU7VbVFgAABAgT6TyD5m0Dgqv/mTo/7R0Dgqn/mSk8JECBAoA8FBK76cNJ0mQABAgQIECBAgAABAgQIECDQpwLJj6sCV306gbpNgAABAgTaJJD8TSBw1SZQzRBoICBw1QBFEQECBAgQaJeAwFW7JLVDgAABAgQIECBAgAABAgQIECAwkkDy46rA1UhS7hMgQIAAgXwLJH8TCFzle56NrrcCAle99fd2AgQIEMi5gMBVzifY8AgQIECAAAECBAgQIECAAAECGRJIflwVuMrQpOgKAQIECBDogUDyN4HAVQ/wvbIwAgJXhZlqAyVAgACBXggIXPVC3TsJECBAgAABAgQIECBAgAABAsUUSH5cFbgq5vwbNQECBAgQSASSvwkErhIRnwTaLyBw1X5TLRIgQIAAgYqAwFWFwgkBAgQIECBAgAABAgQIECBAgECHBZIfVwWuOgyteQIECBAgkHGB5G8CgauMT5Tu9bWAwFVfT5/OEyBAgEDWBQSusj5D+keAAAECBAgQIECAAAECBAgQyI9A8uOqwFV+5tRICBAgQIDAaASSvwkErkaj5xkCzQkIXDXnpBYBAgQIEBiVgMDVqNg8RIAAAQIECBAgQIAAAQIECBAgMAqB5MdVgatR4HmEAAECBAjkSCD5m0DgKkeTaiiZExC4ytyU6BABAgQI5ElA4CpPs2ksBAgQIECAAAECBAgQIECAAIFsCyQ/rgpcZXue9I4AAQIECHRaIPmbQOCq09LaL7KAwFWRZ9/YCRAgQKDjAgJXHSf2AgIECBAgQIAAAQIECBAgQIAAgecEkh9XBa58JQgQIECAQLEFkr8JBK6K/T0w+s4KCFx11lfrBAgQIFBwAYGrgn8BDJ8AAQIECBAgQIAAAQIECBAg0EWB5MdVgasuonsVAQIECBDIoEDyN4HAVQYnR5dyIyBwlZupNBACBAgQyKKAwFUWZ0WfCBAgQIAAAQIECBAgQIAAAQL5FEh+XBW4yuf8GhUBAgQIEGhWIPmbQOCqWTH1CLQuIHDVupknCBAgQIBA0wICV01TqUiAAAECBAgQIECAAAECBAgQIDBGgeTHVYGrMUJ6nAABAgQI9LlA8jeBwFWfT6TuZ1pA4CrT06NzBAgQINDvAgJX/T6D+k+AAAECBAgQIECAAAECBAgQ6B+B5MdVgav+mTM9JUCAAAECnRBI/iYQuOqErjYJPCsgcOWbQIAAAQIEOiggcNVBXE0TIECAAAECBAgQIECAAAECBAhUCSQ/rgpcVbG4IECAAAEChRNI/iYQuCrc1BtwFwUErrqI7VUECBAgUDwBgavizbkREyBAgAABAgQIECBAgAABAgR6JZD8uCpw1asZ8F4CBAgQIJANgeRvAoGrbMyHXuRTQOAqn/NqVAQIECCQEQGBq4xMhG4QIECAAAECBAgQIECAAAECBAogkPy4KnBVgMk2RAIECBAgMIxA8jeBwNUwSG4RGKOAwNUYAT1OgAABAgSGExC4Gk7HPQIECBAgQIAAAQIECBAgQIAAgXYKJD+uCly1U1VbBAgQIECg/wSSvwkErvpv7vS4fwQErvpnrvSUAAECBPpQQOCqDydNlwkQIECAAAECBAgQIECAAAECfSqQ/LgqcNWnE6jbBAgQIECgTQLJ3wQCV20C1QyBBgICVw1QFBEgQIAAgXYJCFy1S1I7BAgQIECAAAECBAgQIECAAAECIwkkP64KXI0k5T4BAgQIEMi3QPI3gcBVvufZ6HorIHDVW39vJ0CAAIGcCwhc5XyCDY8AAQIECBAgQIAAAQIECBAgkCGB5MdVgasMTYquECBAgACBHggkfxMIXPUA3ysLIyBwVZipNlACBAgQ6IWAwFUv1L2TAAECBAgQIECAAAECBAgQIJA/gSVLFpcHtWTxkpCc145yyZIllaIYuqo9BgYmV4oGJg+E9HXlhhMCBAgQIECg7wUErvp+Cg2gDwQErvpgknSRAAECBPpXQOCqf+dOzwkQIECAAAECBAgQIECAAAECvRBIwlQL5s8vvz4doupUf2I4KwlfCWJ1Slm7BAgQIECgewICV92z9qbiCghcFXfujZwAAQIEuiAgcNUFZK8gQIAAAQIECBAgQIAAAQIECPSxwGgDVpMnPN5w1JMnPFEuX/zwynX3Fz+8Sl3ZUAXTpk0v3xLAGkpIOQECBAgQyK6AwFV250bP8iMgcJWfuTQSAgQIEMiggMBVBidFlwgQIECAAAECBAgQIECAAAECPRZIh6yGW8EqCVXFENXkic8GrJJA1ViGkISxFi97NoAVr0cKY8UA1rTpz4awxvJuzxIgQIAAAQKdFxC46ryxNxAQuPIdIECAAAECHRQQuOogrqYJECBAgAABAgQIECBAgAABAn0mEINWcavAoUJWMWA1Y4Nl5VG1I1jVKk85ePVcCGve7RMbPp5sPyh81ZBHIQECBAgQyISAwFUmpkEnci4gcJXzCTY8AgQIEOitgMBVb/29nQABAgQIECBAgAABAgQIECDQa4HhQlbJClYxZNWLgFUzNvNum1ha/arxClhx1StbDjajqA4BAgQIEOiugMBVd729rZgCAlfFnHejJkCAAIEuCQhcdQnaawgQIECAAAECBAgQIECAAAECGRMYKmjVDyGroSiHCl/F4FU8rHo1lJxyAgQIECDQXQGBq+56e1sxBQSuijnvRk2AAAECXRIQuOoStNcQIECAAAECBAgQIECAAAECBDIiMFzQKssrWbXCF4NX8ajddtB2g60oqkuAAAECBDonIHDVOVstE0gEBK4SCZ8ECBAgQKADAgJXHUDVJAECBAgQIECAAAECBAgQIEAgowIL5s8PCxbMr+pdXNEqL0GrqoE9dxHDV7XBq7jildWuGmkpI0CAAAEC3REQuOqOs7cUW0Dgqtjzb/QECBAg0GEBgasOA2ueAAECBAgQIECAAAECBAgQIJABgSIGrWrZBa9qRVwTIECAAIHeCQhc9c7em4sjIHBVnLk2UgIECBDogYDAVQ/QvZIAAQIECBAgQIAAAQIECBAg0CWBRtsH5n1Fq5FoBa9GEnKfAAECBAh0XkDgqvPG3kBA4Mp3gAABAgQIdFBA4KqDuJomQIAAAQIECBAgQIAAAQIECPRQoHZVq6IHrdJTEUNX8UhvNWibwbSQcwIECBAg0FkBgavO+mqdQBQQuPI9IECAAAECHRQQuOogrqYJECBAgAABAgQIECBAgAABAj0QaLSq1Yz1l4UZGyzrQW+y/cra1a4GBgbCtOnTw8DA5Gx3XO8IECBAgECfCwhc9fkE6n5fCAhc9cU06SQBAgQI9KuAwFW/zpx+EyBAgAABAgQIECBAgAABAgTqBWLY6sDZsys3rGpVoRj2pDZ4ZbWrYbncJECAAAECYxYQuBozoQYIjCggcDUikQoECBAgQGD0AgJXo7fzJAECBAgQIECAAAECBAgQIEAgSwK1Wwha1aq12RG6as1LbQIECBAgMBYBgaux6HmWQHMCAlfNOalFgAABAgRGJSBwNSo2DxEgQIAAAQIECBAgQIAAAQIEMiVQG7aaPfWeMHnCE5nqYz90ZvHDK4fBRWtXuhq3GJw5OKty7YQAAQIECBBoj4DAVXsctUJgOAGBq+F03CNAgAABAmMUELgaI6DHCRAgQIAAAQIECBAgQIAAAQI9Fkh+sIzdsIVgeyajdrWrmYODYWBgcnsa1woBAgQIECBQ2gJ5VliyZEmYMmVKmDNnDhECBDogIHDVAVRNEiBAgACBREDgKpHwSYAAAQIECBAgQIAAAQIECBDoL4ElSxaHuLJV/LEyHjFsNXvqvf01iAz3tjZ0NW3a9DBt+vQM91jXCBAgQIBA/wgIXPXPXOlp/woIXPXv3Ok5AQIECPSBgMBVH0ySLhIgQIAAAQIECBAgQIAAAQIEagRi2OrA2bMrpTPWXxZmbLCscu2kPQJCV+1x1AoBAgQIEKgVELiqFXFNoP0CAlftN9UiAQIECBCoCAhcVSicECBAgAABAgQIECBAgAABAgT6QkDYqrvTJHTVXW9vI0CAAIFiCAhcFWOejbK3AgJXvfX3dgIECBDIuYDAVc4n2PAIECBAgAABAgQIECBAgACBXAkIW/VmOhc/vHIYXLR25eUzBwfDwMDkyrUTAgQIECBAoDUBgavWvNQmMBoBgavRqHmGAAECBAg0KSBw1SSUagQIECBAgAABAgQIECBAgACBHgsIW/V2AmpXuhK66u18eDsBAgQI9LeAwFV/z5/e94eAwFV/zJNeEiBAgECfCghc9enE6TYBAgQIECBAgAABAgQIECBQOIHddp1RGfOM9ZeFGRssq1w76Y6A0FV3nL2FAAECBPIvIHCV/zk2wt4LCFz1fg70gAABAgRyLCBwlePJNTQCBAgQIECAAAECBAgQIEAgNwLJj5JxQMJWvZ1Woave+ns7AQIECORDIPnbZsqUKWHOnDn5GJRREMiYgMBVxiZEdwgQIEAgXwICV/maT6MhQIAAAQIECBAgQIAAAQIE8iewYP78sGDB/PLAhK2yMb/p0NXAwECYOTgrGx3TCwIECBAg0CcCAld9MlG62dcCAld9PX06T4AAAQJZFxC4yvoM6R8BAgQIECBAgAABAgQIECBQZAFhq+zOfjp0NW3a9DBt+vTsdlbPCBAgQIBAxgQErjI2IbqTSwGBq1xOq0ERIECAQFYEBK6yMhP6QYAAAQIECBAgQIAAAQIECBCoFkiHreKdBVvfWl3BVc8FBhetFRY/vEq5H0JXPZ8OHSBAgACBPhIQuOqjydLVvhUQuOrbqdNxAgQIEOgHAYGrfpglfSRAgAABAgQIECBAgAABAgSKKLDbrjMqw5499Z4wecITlWsn2RGYdumGlc7MHBwMAwOTK9dOCBAgQIAAgcYCAleNXZQSaKeAwFU7NbVFgAABAgRqBASuakBcEiBAgAABAgQIECBAgAABAgQyIJBe3WrG+svCjA2WZaBXutBIYPHDK4fBRWuXbw0MDISZg7MaVVNGgAABAgQIpAQErlIYTgl0SEDgqkOwmiVAgAABAlFA4Mr3gAABAgQIECBAgAABAgQIECCQLQFhq2zNRzO9mXfbxDDv9onlqrYWbEZMHQIECBAouoDAVdG/AcbfDQGBq24oewcBAgQIFFZA4KqwU2/gBAgQIECAAAECBAgQIECAQEYF0lsJLtj61oz2UrdqBQYXrRUWP7xKudjWgrU6rgkQIECAQLWAwFW1hysCnRAQuOqEqjYJECBAgMBzAgJXvgoECBAgQIAAAQIECBAgQIAAgewIWN0qO3PRak9sLdiqmPoECBAgUGQBgasiz76xd0tA4Kpb0t5DgAABAoUUELgq5LQbNAECBAgQIECAAAECBAgQIJBBAWGrDE5Ki12ytWCLYKoTIECAQGEFBK4KO/UG3kUBgasuYnsVAQIECBRPQOCqeHNuxAQIECBAgAABAgQIECBAgEA2BWwlmM15abVXthZsVUx9AgQIECiigMBVEWfdmLstIHDVbXHvI0CAAIFCCQhcFWq6DZYAAQIECBAgQIAAAQIECBDIqEAvVreKW+DFY/GyVapUZmywrOq63y6GGtfkiY+HyROe6PhwbC3YcWIvIECAAIEcCAhc5WASDSHzAgJXmZ8iHSRAgACBfhYQuOrn2dN3AgQIECBAgAABAgQIECBAIC8C3VrdKoaB4rZ3ix+uDlnVOk6e8Gw4qZ/CV8+Oa+URxzZj/WcDZZ0cm1Wuar9RrgkQIECAQLWAwFW1hysCnRAQuOqEqjYJECBAgMBzAgJXvgoECBAgQIAAAQIECBAgQIAAgd4KdGN1q2aDVo0kYkCpk+GkRu9spSy9olQrz8VxdWrVq3SfBgYGwszBWa10TV0CBAgQIJB7AYGr3E+xAWZAQOAqA5OgCwQIECCQXwGBq/zOrZERIECAAAECBAgQIECAAAEC/SHQ6dWt4spP826fWIWxdOnS8OCDD4b4Gf/FY4011ij/i+eTJk2KH5Ujq6Gr2rElY7nhhhvKfU+uezE2q1xVvj5OCBAgQIBAnYDAVR2JAgJtFxC4ajupBgkQIECAwL8EBK7+ZeGMAAECBAgQIECAAAECBAgQINBtgU6vbtUokBTDSEkQabjxbrTRRlXBq7jN4Oyp9w73SFfvpQNN8cVxXNdff31TfagdWycCZVa5amoqVCJAgACBggoIXBV04g27qwICV13l9jICBAgQKJqAwFXRZtx4CRAgQIAAAQIECBAgQIAAgSwJJD82xj4t2PrWtnatNmzVSiAp6UhcGWrLLbdMLkMngkmVxls4SY8thseaDZGlX1Ebupo99Z4wecIT6SpjPk+HwuaeOW/M7WmAAAECBAjkRSD5G2jKlClhzpw5eRmWcRDIlIDAVaamQ2cIECBAIG8CAld5m1HjIUCAAAECBAgQIECAAAECBPpJINlOsBNBpmmXblihWLhwYVOrWlUeqDmJoasYvopHJ/pa87phL2vDVnFsoz1qA2XtDl2lV7maOTgYBgYmj7arniNAgAABArkSELjK1XQaTEYFBK4yOjG6RYAAAQL5EBC4ysc8GgUBAgQIECBAgAABAgQIECDQfwLp7QQ7ubrVaFa2qtWsDSa1u7+17xvuup1Bsvie9EpXndg2MenvwMBAmDk4a7ihuUeAAAECBAojIHBVmKk20B4KCFz1EN+rCRAgQCD/AgJX+Z9jIyRAgAABAgQIECBAgAABAgSyKZCsbtXukE87V4BKy6WDSb1a5So9tnYEyZLxpVfwavcqV+ltBa1ylYj7JECAAIGiCwhcFf0bYPzdEBC46oaydxAgQIBAYQUErgo79QZOgAABAgQIECBAgAABAgQI9FAgvbpVu8NL6VDSWLcSrCVKB5N6scpVslrU0qVLw1i2EqwdV3oFr3YH4GwrWKvtmgABAgQIhCBw5VtAoPMCAledN/YGAgQIECiwgMBVgSff0AkQIECAAAECBAgQIECAAIGeCaQDV+0OLnUqlBSx0sGkdq8ENdJkpINk7VzdKnlvEiZrd+Aqtp+scmVbwUTbJwECBAgUXUDgqujfAOPvhoDAVTeUvYNAEwLPPPNME7VUKYrAuHHjijLU3I9T4Cr3U2yABAgQIECAAAECBAgQIECAQAYFOrWdYHo1pU6EktKBq3avzDXSNKUDV+1euSu+OwlcxfN2h8mSwFVse+6Z8+KHgwABAgQIFFpA4KrQ02/wXRIQuOoStNcUU0CIqpjz3u1RC2d1W7y19wlctealNgECBAgQIECAAAECBAgQIECgHQJJ4KrdoaVOB67i2HfaaacyQSdWghrONh24+vnPfz5c1VHdS4fJ2h24Ss/LzMHBMDAweVR99BABAgQIEMiLgMBVXmbSOLIsIHCV5dnRt74UELLqy2nLXaeFsLIzpQJX2ZkLPSFAgAABAgQIECBAgAABAgSKIZDeTrDdwZ50KKkTq0DFGUpWgup24CpZJWrp0qUhjq3dRycDV7GvyVaPthVs98xpjwABAgT6UUDgqh9nTZ/7TUDgqt9mTH8zKSBklclp0amSgOBV778GAle9nwM9IECAAAECBAgQIECAAAECBIolkA5cLdj61rYOPh246sQqULGzeQ1cxbElq3e1e+Wx2HYSGBO4ihoOAgQIECi6gMBV0b8Bxt8NAYGrbih7R64EhKtyNZ2FG4wAVvenXOCq++beSIAAAQIECBAgQIAAAQIECBRboFPbCUbV9NZ1nVrhKgkl9WqFqzjOToTJOr3CVRK4iv2fe+a8+OEgQIAAAQKFFRC4KuzUG3gXBQSuuojtVf0rIGTVv3On50MLCF8NbdPOOwJX7dTUFgECBAgQIECAAAECBAgQIEBgZIFuBa5uuOGGcP3114/coRZrJIGrTqwCNVxX0oGlToTJNtpoozBp0qRyF9q91WNsNB2Gmzk4GAYGJg83XPcIECBAgECuBQSucj29BpcRAYGrjEyEbmRToNNBq063n01VvWpVoNPBqE633+p481Zf4CpvM2o8BAgQIECAAAECBAgQIECAQJYFlixZHA6cPbvcxU6EemLD0y7dsNz+0qVLQwwmtfNIh5K6HbhKB5Y6Hbhq91aPcQ7S/Re4aue3UlsECBAg0I8CAlf9OGv63G8CAlf9NmP62xWBdgah2tlWVwbvJX0l0M6wVDvb6ivEDndW4KrDwJonQIAAAQIECBAgQIAAAQIECKQEFsyfHxYsmF8u6VTgKlkJqtOBq06EklJUdafpwFInxpas3NXJrRKTMNy0adPDtOnT68aogAABAgQIFEVA4KooM22cvRQQuOqlvndnTqAd4ah2tJE5GB3qG4F2hKba0UbfgHWhowJXXUD2CgIECBAgQIAAAQIECBAgQIDAcwLpwFWnAkvpYFI7txVcY401wpZbblkeSSdDScN9WZIwWazTzlWuurVyV9L/gYGBMHNw1nBDdY8AAQIECORaQOAq19NrcBkRELjKyEToRu8FRhuUGu1znRxxFvvUyfFmpe0sBpVG26fRPpeVuchSPwSusjQb+kKAAAECBAgQIECAAAECBAjkXaAbgatomKykFM/bFUyKYasYuopHt7cTLL+09D/pMFm7VrlKB8niezoVhIttC1xFBQcBAgQIEAilLZZnhSVLloQpU6aEOXPmICFAoAMCAlcdQNVkfwmMJpw0mmcaqbSrnUZtK8uPQLvCT6NpZzTP5Ee+PSMRuGqPo1YIECBAgAABAgQIECBAgAABAs0IJD8udnqFqHYHk7q1AlQzhkloKdZtxwpe3QySzbttYph3+8TyMOeeOa+Z4apDgAABAgRyKZD8TSRwlcvpNaiMCAhcZWQidKM3Aq0GnlqtH0c1mmd6o+Gt/SgwmkBUt57pR89O9FngqhOq2iRAgAABAgQIECBAgAABAgQINBbYbdcZ5RvdWCGqXcGkdNgqdr6TK0A1VqsuTYfJ4p2xhK7SY+vGnAhcVc+lKwIECBAoroDAVXHn3si7JyBw1T1rb8qQQKshqE7XzxCNrvS5QKthqlbqt1K3zxnb2n2Bq7ZyaowAAQIECBAgQIAAAQIECBAgMKxANwNXsSPp0FW8bjWclF79KT7fjVBSfM9Ix1hDV3EbwUmTJlW2SOz0imPJeNL9njk4GAYGJie3fBIgQIAAgUIJCFwVaroNtkcCAlc9gvfa3gm0Ep5qtm6z9YYa9VifH6pd5f0vMJaQU7PPNlsvarZSt//12zMCgav2OGqFAAECBAgQIECAAAECBAgQINCMQLcDV7FPjUJXS5cuDfHfUEcMJMWwVfqYPfWeMHnCE+minp6nV4tKOjJSoCyOKwlbJc90K2wV3ydwlaj7JECAAIGiCwhcFf0bYPzdEBC46oayd2RGoNlgUzP1mqmTHnir9dPPOidQK9Bq8KmZ+s3USfrRSt3kmaJ+ClwVdeaNmwABAgQIECBAgAABAgQIEOiFQC8CV3GcjcJJsTyGrh588MF4GlZfffXyZwwkpY8YSJqxwbJMha2S/qUDTElZ8pmMbahxxXrdXrEr3V8rXCUz5ZMAAQIEiiggcFXEWTfmbgsIXHVb3Pt6JtBs4KmZeu2q0yxGM+9rti31siPQztBSs22NVG+k+2m9VuqmnyvaucBV0WbceAkQIECAAAECBAgQIECAAIFeCvQqcBXHHENXMfCz+OFVmibodiCp6Y7VVBwqUFZTrXLZqxCZwFVlCpwQIECAQMEFBK4K/gUw/K4ICFx1hdlLei3QTGBpqDo3/+XucPNf7ir9u3v4YTwz/G13CXRVYNzwbxsXxoVt3/SG8Mp1X96wYrNhqmbrNXxJQQoFrgoy0YZJgAABAgQIECBAgAABAgQI9FxgyZLF4cDZs8v96GWQqRy6WrZKw/BVDCLFI6srWpU7N8z/xOBVPBoFy7IwNoGrYSbPLQIECBAolIDAVaGm22B7JCBw1SN4r+2ewFBBqnQPhqtz0x13he+c8dN0decEciGw7ZZvCNu9+Y1DjqXZMFWz9YZ8Uc5vCFzlfIINjwABAgQIECBAgAABAgQIEMiMQDpwNXvqPZncoi8zWDntiMBVTifWsAgQIECgZQGBq5bJPECgZQGBq5bJPNBPAsMFqeI4mrl/we+uDBcsvLI87Feu++/9NHx9JdBQIK7YFo9XlFa32nv3XcJIgamR7se2mqkT6xXxELgq4qwbMwECBAgQIECAAAECBAgQINBtgYceeigc9+1jw+LFi8uvFrjq9gxk430CV9mYB70gQIAAgd4LCFz1fg70IP8CAlf5n+PCjrCZMNVQOOln04Grb+3/uaEeUU6gbwSO/fFZ5W0yk8BV0vHhQlPD3YvPj3Q/eUcRPwWuijjrxkyAAAECBAgQIECAAAECBAh0U+DSSy4Jp556Snjssccqr+3lloKVTjjpuoDAVdfJvZAAAQIEMiogcJXRidGtXAkIXOVqOg0mEUgHppKy9Odw92vvCVyl5ZznQWCowFUc23DBqeHujfRsHtxGOwaBq9HKeY4AAQIECBAgQIAAAQIECBAgMLzA0qVLy0GrKy6/vK6iwFUdSSEKBK4KMc0GSYAAAQJNCAhcNYGkCoExCghcjRHQ49kUqA1N1fZyqPuNys//3RXhwoVXlZuwwlWtpOt+FKgErtYpbSm4xy4NhzBUuGqo8qSRke4n9Yr0KXBVpNk2VgIECBAgQIAAAQIECBAgQKBbAhdf/Ktw6imnhCeeeKLyynXXWy/85Y47ytcCVxWWQpV24YIAAAWqSURBVJ3Mu21imHf7xPKYZw4OhoGByYUav8ESIECAAIFEQOAqkfBJoHMCAleds9VyjwQahabSXRnqfqPyWHbBwisFrtKAzvteIB24+szuOw+5qtVQ4amhyiPMcPf6Hm6UAxC4GiWcxwgQIECAAAECBAgQIECAAAECDQQeuP/+8qpWV155Zd3dHXfcKfziFz8vlwtc1fEUoiAduJp75rxCjNkgCRAgQIBAIwGBq0Yqygi0V0Dgqr2eWsuAQKPgVNKtoe41Kk/KBK4SPZ95EagNXCXjahSWarYsaSN+Nnomfb9o5wJXRZtx4yVAgAABAgQIECBAgAABAgQ6JXDRRReWV7Vavnx5w1d8beZgOOjA2eV7kyc8HmZPvbdhPYX5FRC4yu/cGhkBAgQItCYgcNWal9oERiMgcDUaNc9kViAJSTXq4FD3astrrwWuGmkq62eBoQJXcUyNwlLNlqVNGj2Tvl+kc4GrIs22sRIgQIAAAQIECBAgQIAAAQKdELjvvvvKQas//OGqIZtfddVVw/EnnBiSHxfzHLha/PDKZYfFy1YJyXksWPzwKnU+0SE5Jk94dvvFyRMfD8l5ci8vnwJXeZlJ4yBAgACBsQokfxNNmTIlzJkzZ6zNeZ4AgQYCAlcNUBT1p0BtUCo9iqHu1ZbXXsc2LvhdaUvBy579f8h/a//PpZt1TqAvBdKBq0/vtlNdyKpRWKrZsjRIo2fS94tyLnBVlJk2TgIECBAgQIAAAQIECBAgQKATAhdecEF5C8Enn3xy2Obf9KY3hc/us28lcBUrL9j61mGf6ZebSagqhoni0ShYNZqxxG0X45GnANbgorXKPgMDA2Hm4Kzy+PwPAQIECBAoooDAVRFn3Zi7LSBw1W1x7+uYQKOwVPKyRvdqy4a6vnDhVQJXCaTPXAjUBq7ioGrDUSNdN3qmFqe2jdr7RbkWuCrKTBsnAQIECBAgQIAAAQIECBAg0E6Be++9txy0+tMf/9hUs3t/dp/w5je/OSyYPz8sWDC//Ey/B65i0CqGrNoVsBoJMgawZmzwbAhrpLpZvT/t0g3LXZs2bXqYNn16VrupXwQIECBAoOMCAlcdJ/YCAkHgypcgFwK1Yan0oBrdG6ksfV/gKq3pPA8CjQJXcVy1AamRrhs9U+tT20bt/SJcC1wVYZaNkQABAgQIECBAgAABAgQIEGinwPnnnVcOWz311FNNN/vD408I48ePD0uWLC6tcjW7/Nzsqff05dZ53Q5a1SL3c/BK4Kp2Nl0TIECAQFEFBK6KOvPG3U0BgatuantXxwTSAan0S5otT9dLn8e2Llh4Zbjosj+Um7WlYFrXeb8KDBW4iuOpDUiNdN3ombRL7fPpe0U5F7gqykwbJwECBAgQIECAAAECBAgQIDBWgXvuvrsctFq0aFFLTb26tH3c4HPbx/Vz4CquZjXv9me3DWwJoEOV+y14FYNqg4vWLmtY4apDXwrNEiBAgEDfCAhc9c1U6WgfCwhc9fHk6fqzArUBqbRLo3u1Zenr9HlsJ15feNlVAldpVOd9L5AOXH1q1x2HDVnVBqZqryNGo7I00kj303XzeC5wlcdZNSYCBAgQIECAAAECBAgQIECg3QLnnntOOPWUU8r/f7Kttr37HnuEnXbaufLYbrvOKJ9PnvB4mD313kp5Vk96vaLVSC79ErxKB9ZmDg6GgYHJIw3NfQIECBAgkFsBgavcTq2BZUhA4CpDk6EroxOoDUklrTQqry0b7jq5J3CViPrMi0Bt4CqOqzYUlb5OnzeqO1RZLI9H7fPPlhbnfwWuijPXRkqAAAECBAgQIECAAAECBAi0LnDXXXeWg1bXXHNN6w8/98ThRxwR1ltv/crzyQ+M/RC4SoeEKgPI4Ek/hK7SlnPPnJdBRV0iQIAAAQLdE0j+HpoyZUqYM2dO917sTQQKJPD/AQAA//9paT+nAABAAElEQVTsvQl8VOW9///NLNnJRoCwLyJLWASs1lpebqj12utW1vZqKyrVW2urVQvYKqIgcPV1f+r/3uqtULfWC4raerWtolWrggsCggSUTSAkJCxJSMg2S/7P94RneHKYmcxy5syZM5/zavIs55znPM/7Gach8873m9EhDsIBAilMINRLOFi/2qfWeflqW62/9dFntGbdeo3QY/f8MoVJYeog0Eng//vjy7RzXyUNG9iP/n3mVQEsGRkZQevcqZ4L1g7VF66fz6XD8eWXX9Jvf/tbban3LVhA5eVj0mHZWCMIgAAIgAAIgAAIgAAIgAAIgAAIgAAIdEvg9ddfpz8+/1y314W7oE+fMnrs8ce7XLL6pZdo9eqXtL7VF+7ucs5KjQUb+9LW+hwrTSnsXMYUtdDCidVhr0nmyWnvDtMeP23adJo2fXoyp4JngwAIgAAIgEDSCTyw8H6qqKigsWPH0qJFi5I+H0wABOxIIEOIJRCu7LizabKmUC/fYP36PrUdrr5m7XoIV2nyekqXZarC1S0zruwiU6liVag6c1LPSW7B+iI5J6+xawnhyq47i3WBAAiAAAiAAAiAAAiAAAiAAAiAAAgYQeCVl1+mF19cFfNQ3/veZTT7hhu63F9RsZUeWLhQ61s4sYrGFLV2OZ/sxtb6bFqwsV+ypxHz863OFMJVzFuLG0EABEAABGxEAMKVjTYTS7EsAQhXlt0aTCwSAqoopV4frF/ti6YO4Uoli7odCOiFK16TKktFUtffI7mo98q+UNeq5+1ch3Bl593F2kAABEAABEAABEAABEAABEAABEAABIwg8MUXm2j5U0/RoUOHoh5u7tx5NHHSpFPumzVzhtZntahMqS5bSdBWk65e3FNML35TrE0PUeblLqEEARAAARBIZwIQrtJ597F2swhAuDKLNJ5jOAFVmtIPrj8Xrq2eC1Z/a+1n9Pa6z7VHIKWgnjTaqUggIFwN6Ee3zLxSW4IqSkVS55vU6ySHYH2RnJPX2LGEcGXHXcWaQAAEQAAEQAAEQAAEQAAEQAAEQAAEjCbQ1NREy5c/RR+vWxfx0G63m5597nlyOByn3CM/ZLSacCXT3p0y4Rg6eG0zhtZ1uVMfzYsFLz621nWmLpRSUpebYmxYSbpShauVq16McUW4DQRAAARAAATsQ0D+LISUgvbZU6zEegQgXFlvTzCjCAmocpR6S7B+tS/aOoQrlS7qdiCgClc3z7giIE6pslQkdWahXhesrfLSX6ues3MdwpWddxdrAwEQAAEQAAEQAAEQAAEQAAEQAAEQMJrA6//3f/THPz4f0bBnnXUW3XnX3UGvtWJawQUb+9LW+k7xKeiku+lUBSu9WNXNraecZkGJj3gFLKtIV1JkQzrBU7YaHSAAAiAAAmlKAMJVmm48lm0qAQhXpuLGw4wioEpT+jH158K15TlZ8lj6+pp16xHhSg8Z7ZQmEEq44kWpUlS0dQlFvU/26cdW++1eh3Bl9x3G+kAABEAABEAABEAABEAABEAABEAABIwm8OCDD9DWL7/sdtib5vyULr744qDXqcKVFaJcxSNbSdEqXskqGCgjxKtkS1dqdCsIV8F2GX0gAAIgAALpSADCVTruOtZsNgEIV2YTx/MMIaBKUfoB9efUdnf1YOfXrBXC1cdIKajnjHbqEtALV7wSVZKSdVmGOq/v5zYf6n2dPSe/hzt38ip71SBc2Ws/sRoQAAEQAAEQAAEQAAEQAAEQAAEQAIHEEnj9dRHh6vnOCFe9e/em2trakA/8r//+HZWWloY8Lz9o5AtWX7g75HWJPqEKQdE8K5GiVbB5xCqFJVtoU+eNdILBdhZ9IAACIAAC6UhA/hyElILpuPtYs1kEIFyZRRrPMZSAKkapA+v7w7XlOVnyOLIuS+6DcMUUcNiJQDDhitcnZShZdtcnmajXh+tTx5PXpUMJ4SoddhlrBAEQAAEQAAEQAAEQAAEQAAEQAAEQMILAtm3baOH9C7ShysrKaMnSZfTBP/9Jf/jDilOGH3baafTQQ0tO6Vc71ChXyYrCFKtsNWNIHc0YWqcux5R6qs2XoSCdoCkvDTwEBEAABEAgxQhAuEqxDcN0U5IAhKuU3DZMWhWiVBr6frXdXT3Yee57e93niHClQkY95QmowtVPp/9rTKKVKlmpdQknWB+fC9Uv77NjCeHKjruKNYEACIAACIAACIAACIAACIAACIAACBhNwOPx0Px5c6myslIb+re/vZfGjhun1Xft2kXLn/o97dmzJ/DYH0ydSjNmzAy0Q1VmzZyhnUpWFCYpA4WaX7D+ZMlhci5b67NpwcZ+shlxmYx5q4IYoltFvFW4EARAAARAIA0IQLhKg03GEpNOAMJV0rcAE4iWgCpGqfcG61f7uqvL87LksbkO4UqljLodCOiFK16TFKH0ZbBzah/X+ZD3dbZObcv+YNeq5+xYh3Blx13FmkAABEAABEAABEAABEAABEAABEAABIwm8MTvfkfvv/+eNuysH/6Irr766lMesXz5U/T2mjVa/wMPPkgjRow85Rp9h/ywkfvNFoJUGUg/r1Bts+cYah7cr6bqC3edPJcMqU0KbeXl5XTfgvvlVFCCAAiAAAiAQNoTkD8DIaVg2r8UACCBBCBcJRAuhk4MAVWIUp+g7w/XludkyePIur5cs249vfPxBu1Rj93zS/WRqMdBoEPc29LWQcdaOqixxU+NrR3U6umgNi+Rz98h9oPI6SByuzIoW3zlZWVQj5wMKshxUEFuBmXE8ex0vzUS4YoZSYlKlmpfuDqf40O9r7On83uofvUaO9UhXNlpN7EWEAABEAABEAABEAABEAABEAABEACBRBB466036Q8rOtMGnnX22XTnnXeFfMx7775Lq1atpCee/J+Q16gn1LSCZgpBqS5bSYbRSldmCmMq42nTptO06dPltFGCAAiAAAiAQNoTgHCV9i8BADCBAIQrEyDjEcYSkEKUflR9v9ruri7P60t+BoQrPenY2yxZdfiJPEKoavOQJlo1NJ8Urto04apDCFcswCnClTuDclm4ys6gwlyH+BISlujLFCKWQ5hXGbCvotoUVbiaM+37ATFKilChSn6IPBeuLiejXiv79Pep/XatQ7iy685iXSAAAiAAAiAAAiAAAiAAAiAAAiAAAkYQ4HSBv7lnvjZUSUkJLVm6jAoLC8MOvX//Pho4cFDYa9ST8gNH7jNLCFJlIHUuoepmzSvU88P1RyNdmSm1yehWPHekEwy3gzgHAiAAAiCQjgTkzz+IcJWOu481m0UAwpVZpPEcwwhIKUo/oL5fbQerh+uT57h8++PPEeFKDzvGtvCshGjVQSxZVdf7ROmnZhHlyuvjLxHZSpznayR/9qhY2uFIV/zlcmZQzgn5qqzISb0LHZp05XbGOKE0vS1a4YoxSXlKlmqfxKieC3Y+1HWy364lhCu77izWBQIgAAIgAAIgAAIgAAIgAAIgAAIgYASBe+bPo927d2tD/XruXJo06Uwjhu0yRjKiXKkyUJfJBGnMGFJHM4bWBTljja6t9dkivWC/iCdjhjymCm2IbhXx1uBCEAABEACBNCIA4SqNNhtLTRoBCFdJQ48Hx0pAyjjq/fq+cG15Tl/yePo+bkO4UknHVhcOFfmEUMUpA+uP++lwo58OCuHquJCtPMKy4mhWkR4sXWUKwaqPJlw5qUhEu+oh0gyykMXRrnB0T0AKV0MH9KWfTvvXgEzFd0ppKlQZ7Br5RHlPqHZ3/fK83UoIV3bbUawHBEAABEAABEAABEAABEAABEAABEDAKAIrli+nNWve0oabOnUaTZ8xw6ihTxlHfujIJxItBKky0CkT0XVYXbaS041mTWZEuVKFNkS3kruEEgRAAARAAAROEpA/+yDC1UkmqIGA0QQgXBlNFOMllIAUovQP0feHaqv9st5d+fY6EeHqkw3aIx+755f6R6MdAQGOWtUi5Ko6IVvtPeyjo01+LdKVV5yIRrbiR3H6QPaqskSkq7wsBw0qdRJHu+KUg4h0FcFmiEv0whXfJWWp7kr12nB1/Tluq4d8jtpn1zqEK7vuLNYFAiAAAiAAAiAAAiAAAiAAAiAAAiAQD4H33nuXnnziCW2IMyZMoPnz74lnuG7vNTPKVTRy0uoLO6N7dbsAC1wQaWrBRAtXKl9Et7LACwNTAAEQAAEQsCQBCFeW3BZMymYEbCNcsTTT1NREx44do8bGRu2rubmZWltaqd3TTl6vl/x+vyYVOBwOcrvdlJmZSTk5OZSbm0s9evTQ8sIXFxdr52y2z7ZZjpSj9AvS96vt7uryfKgSwpWednRtlq08XqLaYz6qafBTjYhs1dTq10SrKAJbnfJQjmbF0a76FDrFl0OkF3SKSFcZASHrlBvQESAQjXDFN0k5Sl+q5/T1YG3uk4ccS7btXEK4svPuYm0gAAIgAAIgAAIgAAIgAAIgAAIgAAKxENi/fz/NnzdX+719fn4+LVm6jHr16hXLUFHdIz945JsSGeVKjb4UboKpEt1KriGa1IKJ4qvOoby8nO5bcL+cHkoQAAEQAAEQAAGFgPy5BxGuFCiogoDBBFJOuGJpisUYlqt27Nih5Xbfu3cvVVZWav84i5cPy1hlZWU0fPhwGjVqFI0cOVITs1gOSCdBIF6OibpfSlH68fX9ajtYXfbJkseTdS7V+jsfb0CEKz3wKNpekUqwpb2Dvqr20oGjPi2ylY8tLAMOjnaV5XaItIIOGtHXRb0LHELC6oyCZcDwth1CFa7mTP1+4L1NvsdFWjIgea2+LuGp52VfqGvV83aqQ7iy025iLSAAAiAAAiAAAiAAAiAAAiAAAiAAAkYQuH/BfbR9+3ZtqNvvuIPOOec7Rgwb0RizZp5MW5iI6FJq9KXuJpQoKam758ZzPtkymcr3vgULqLx8TDzLwb0gAAIgAAIgYFsCEK5su7VYmIUIWF64YsGqtrZWE6oOHDigldXV1VRfXx+QYhLJ0+kU0XP69KH+/fvTwIEDacCAAVrJUbFwmE9AilD6J+v71XawuuyLpIRwpacdXbuhuUNLIbj3kJdqG/0i0lz0aQTDPdEpQl3liXSCQ3q5tEhXhUK+crvC3YFz8QpXTFCKVLKUVLtrh7pO9tuxhHBlx13FmkAABEAABEAABEAABEAABEAABEAABGIl8Pxzz9Ebb7yu3X7FFVfSv117baxDxXTf6pdeotWrX9LuTUSEKVUI6m6CiRC+untmvOeTmVZQZYtUgvHuJO4HARAAARCwOwEIV3bfYazPCgQsK1y1tbXRJ598Qh999BEdPnyYPB4P+XwiVE4SDxYJXC4XZWVlEYfeu/DCCzUBK4lTSrtHS0FKv3B9v9qWdVnyvbIeSQnhSk878jbHsTpwxEdVdT46dMxPx1r8kd8cxZVZrgzqJaJblRU5qX+Jk3KFgIUjNIGAcNW/L82Z9n3tQn5/k7JUdyXfIK8JV9ef47Y81Ptln11LCFd23VmsCwRAAARAAARAAARAAARAAARAAARAIFoCa8Xv+x9//DHttmSmg5MfQPJEjI4ypUpB4fgkQvYK9zyjzqkp/cKNOaaoRbCtDndJ1OfU6ForV70Y9f24AQRAAARAAATSiYD8eQcpBdNp17FWswlYSrg6cuQIcXrATZs20datW6m1tdVsHlE/jyNeTZo0iUaMGKHJV263O+oxcENkBKQcpb9a36+2u6vL88FK2ff2x5/TPz7ZqD32sXt+qX882mEIiOyMtOOgl7455KOmVr+WTjDM5TGfcjmI8rI7havhZS4qyIFwFQ5mKOGK75EiFJdqXY4Xrk+9X3+9bMtSjiPbdi4hXNl5d7E2EAABEAABEAABEAABEAABEAABEACBSAnU1NTQ/Hlzqbm5mTIzM2nJ0mVaZolI7zfyuoqKrfTAwoXakEaLQZFGgIJwFd2OqiIboltFxw5XgwAIgAAIpCcBCFfpue9YtbkELCFccQSrN998U8vZ3tDQkPRIVrFsQV5eHvXq1UuLesUClsMhDBAchhKQApR+UH2/2g5WD9Un+/UlhCs98cjaLFv5xNfW/R7aKaQrr0glKDKEJuQQbhBxasE+hQ4aO9BNRXkOEk0cIQhEKlzx7VKMClWq1+jrwdrcx4ccr7Nl7+8Qruy9v1gdCIAACIAACIAACIAACIAACIAACIBAZAQWL15EWzZv1i7+2a230nnnnR/ZjQm6Sk0taKR0FalwZXRkrQRhCjqsGmkq6AUnOo1KmQjZKhxlnAMBEAABEACB4AQgXAXngl4QMJJA0oSr9vZ2qqqq0lIGfvrpp+T1eo1cV1LH6tevH1166aV0+umnU1FRUVLnYqeHSxFKvyZ9v9oOVpd9kZYQrvTEI2u3C9uqzUO0/YCHdtd4SbhXIpVjZPfGchULVqUireCo/m7qme8gTjPIIhaOUwlAuDqVSSJ7IFwlki7GBgEQAAEQAAEQAAEQAAEQAAEQAAEQSAUCq1b+L7366qvaVL/3vcto9g03WGLa8oNInoxREacilZEgXEX2ElBlq2SmoYxstrgKBEAABEAABKxDQP6cg5SC1tkTzMR+BJIiXH3zzTf01ltv0c6dO+n48eP2oypWxNFb+vbtS2eeeSZdcMEFlJ2dbct1mrkoKUjpn6nvV9vB6rIv0hLClZ54ZO2W9g5qauugHdVe2nvIHKGyRIhWI/q5qLSHk/KyMhDlKsRWqcLVTVMvD0Sb4vctGXmqu5KH1l+j9slHy2tkW5ah+uV5O5UQruy0m1gLCIAACIAACIAACIAACIAACIAACIBAtATWf/YZPfLIw9ptw4cPp0WLH4p2iIReLz+M5IcYIUFFKlwZFf0poXBCDB5pFK9417i1PpsWbOwXmMXKVS8G6qiAAAiAAAiAAAiEJyB/xoFwFZ4TzoJAPARME678IpfYsWPH6O9//zt98MEH8cw55e7lVIPTpk2jESNGaLnpU24BFpmwFKT009H3q+1gddkXrlTPvfPJBvrHJxu1xz52zy/1j0c7BIGm1g5qbOmgXSK61f4j5ghXxSKV4GllLupV4KQe2RkizWCIyaV5dzjhitGwDCWFqFClvE4t9fVgbe7jQ47b2bL3dwhX9t5frA4EQAAEQAAEQAAEQAAEQAAEQAAEQCA0gbq6Opo/by7V19drvw96aMlSGjp0aOgbknCmomIrPbBwYeDJ8UpXkchIRqYwDEzcxEoka+TpxCtcqfLatGnTadr06SauEo8CARAAARAAgdQmAOEqtfcPs08NAqYIV83NzfT+++/T2rVr6ejRo6lBxuBZulwuGjVqFE2ZMkUTrwwePi2GkxKUfrH6frUt67Lke2U9WBmsD8KVnnhkbRauGpo7aHetlypNEq6KWLjq0ylcFeRAuAq1U0YLV/wcKVDJUj5b35b96j1qnx3rEK7suKtYEwiAAAiAAAiAAAiAAAiAAAiAAAiAQCQEHn74P+jz9eu1S2+aM4cuvviSSG4z/RojpSs1BV6ohRiVvjDU+Inu10eeCva8eNeoSl2QrYIRRh8IgAAIgAAIhCcA4So8H5wFASMIJFS4Ynnl0KFD9MQTT1Btba0R8035MRwOB11yySV02WWXIdpVlLspZSj9bfp+tS3rsuR7ZT1YGawPwpWeeGTt40K4OoYIV5HBMvmqSIQrnpKUpbiUdX2/nHqw88HOyT51HLXPjnUIV3bcVawJBEAABEAABEAABEAABEAABEAABECgOwKvvPIyvbhqlXbZhRdeRDffckt3tyT1/OqXXqLVq1/S5hBvBCpVFtIvKt6x9eMlqx1ujTyneKJbqWNDtkrWDuO5IAACIAACqU4AwlWq7yDmnwoEEiZctbS00Jo1a+jdd9+l9vb2VGBh6hyHDRumSVejR48mlrBwdE9AylD6K/X9alvW9SWPofapdf25dz4WKQU/RUpBPffu2i3tHcTS1dfVXtp72JyUgiX5DhrR102lPRyUJ1IKOjK6m2V6no9FuGJSUqrSl+o5fT1Ym/v4kON0tuz7HcKVffcWKwMBEAABEAABEAABEAABEAABEAABEAhOYPMXX9BDDy3WTg4aNIiWLF1GTqcz+MUW6jVSugoW6couspXcskSsEbKVpIsSBEAABEAABOIjAOEqPn64GwQiIZAQ4aqmpoYef/xxamhoCEgtkUwm3a5xu9106aWX0uWXX55uS49pvVKK0t+s75dtWfL1sh5LCeFKTzyydrtwrFo9HbSt0qOlFYzsrviuYtFqdP9O4SrLzVGZ4hvPrnfrhSteJ8tPUoCKtpT3S17y/lDt7vrlebuUEK7sspNYBwiAAAiAAAiAAAiAAAiAAAiAAAiAQCQEjh8/TvPnzQ1kvXjggQdpxMiRkdxqiWuMlK54QZx+Tx5jilpl1ValUWtUBS5EtrLVSwSLAQEQAAEQSAIBCFdJgI5Hph0BQ4Urr9dL60U+9tdee02TrdKOZowLHj9+PF1zzTXUu3fvGEdIj9ukLKVfrb5ftmXJ18t6NKW8FsKVnnhkbZ+fyOvroIoDXtpV4yWfqPs7Irs32qtYrHKJcFa9ChxUPsBNxXkO8RdzQiKKdqA0uT6UcMXLj0W8kvdJfBCuJInOEsJVVx5ogQAIgAAIgAAIgAAIgAAIgAAIgAAI2JvAY48+SuvWrdUW+eOf/ET8wfH3U27BRktXKQcgCRNWZavy8nK6b8H9SZgFHgkCIAACIAAC9iEA4co+e4mVWJeAYcKVz+ejF198kT7++GNi8SoZR15eHvXt25dKS0uppKSECgoKKDc3l7Kzs8nlcmmp+3iebW1t1NraSo2NjVRfX09HjhyhgwcP0qFDhwJijpnzZzmhsLCQrr32WuIUgziCE5AClP6s2t9dXZ4PV+rPQbjSE4+83SEEq20nhKs2Ee3KI6SrRByclTNbRLQqK3TSqP4uKsxFms5wnCFchaNj/DkIV8YzxYggAAIgAAIgAAIgAAIgAAIgAAIgAALWJPD66/9Hf3z+eW1y3508mW677RfWnGgEs1KlK7584cQqsmuEqghwJPQSNY0gZKuEosbgIAACIAACaUQAwlUabTaWmjQChghXdXV1tHLlSuIPlc04WKwqKirSpKqBAwfSkCFDaMCAAZpgpY+sEs18WMaqqqqi/fv30549ezQJi9Mi8pcZEhnLYTNnzqQzzjiDON0gjq4EpAjVtfdk9CruV68JVpd94Ur9OQhXeuKRt1mv2n/YRweO+uhIo58aW0XYqwQcma4M4nSCZUVOGtjTSblZiG0VDnN3whXfG02kK3m9fKb+fVjfDnWd7LdbCeHKbjuK9YAACIAACIAACIAACIAACIAACIAACAQjsG3bNlp4/wLtVFlZGS1ZuoxycnKCXZoyfXrpasaQOpoxtC5l5m/1iXIqQo5stbW+83UC2crqO4b5gQAIgAAIpBIBCFeptFuYa6oSiFu42rdvHz333HOanCRFlUTA4AhVI0aMoAkTJhBLVhwRqkePHlrUqkQ8j8fkSFgyAtb27dtpw4YNxHJZIg+nyIN28cUX05VXXpnIx6Tk2KFeX2p/d3V5PlypPwfhKr6Xy9EmvyZb7RPi1eFGn5DihBgX35Bd7uZ0gnlCsBpU6hLClUNLJ8gCFo7QBFTh6sYf/EtIuUqKUqFKfoL+nNonZyCvkW1ZhuqX5+1SQriyy05iHSAAAiAAAiAAAiAAAiAAAiAAAiAAAqEIeDwemj9vLlVWVmqX/Pa399LYceNCXZ5S/ZCuErNdLFst2NgvMPi0adNp2vTpgTYqIAACIAACIAAC8RGAcBUfP9wNApEQiFm4YinlwIED9OSTTyZMQsrMzNSiWE2cOJG++93vUnFxcSRrStg1fr+f+K903nvvPe0fjpySUMo5Rj+U1zt16lTKysoyeuiUHS8Ua7W/u7o8H6wM1sewIFzF95LhVIJNrR20u9ZL1fV+avP4yeuLb0x5N8tWnEqwKM9Bw3q7qFeBg7KEbMUpBnGEJgDhKjSbRJyBcJUIqhgTBEAABEAABEAABEAABEAABEAABEDASgSe+N3v6P3339OmNOuHP6Krr77aStOLey6QruJG2GUAjmr14jcnP++BbNUFDxogAAIgAAIgYAgBCFeGYMQgIBCWQMzC1RdffEEvvPACNTU1hX1ALCdZtGLJir+GDRtGnELQSgeLOdXV1fTVV1/RunXrNPHM6Pk5hDHyrW99S0sxmJ2dbfTwKTeelKH0E9f3q+1gddkXrAzWx8+DcKWnHn273dtBHOGKhau64z5qae+gDpFdMJ5IVw4hW7mcnbIVi1aDSp1UlAvTKpLdgXAVCSXjroFwZRxLjAQCIAACIAACIAACIAACIAACIAACIGA9Am+99Sb9YcUKbWJnnX023XnnXdabpAEzqqjYSg8sXBgYaUxRCy2cWB1ooxIZgQUb+wZSCPIdkK0i44arQAAEQAAEQCBaAhCuoiWG60EgegJRC1cspWzZsoWeeuop4ohPRh5ut5uGDx9Os2bNotLSUiOHTthYzOCTTz6hN954gxoaGgxnMn78eJo9ezaxhJbOh5Sh9Az0/Wo7WF32BStD9f3jk430j083ao9+7J5f6qeAdgQEfOKtgqNccXrB/Ue8VCfKViFhxfMWkiUiW+WKVIIDe4pUgoUOys/J0KJbRTCdtL8kUuGKQXHaP5n6T1/K82qprwdrcx8fcrzOln2/Q7iy795iZSAAAiAAAiAAAiAAAiAAAiAAAiCQ7gR27dpFv7lnvoahpKSElixdRoWFhbbFwtIVR7uqqKgIrHHGkDqaMbQu0EYlOAFOIciRrbbW5wQuuG/BAiovHxNoowICIAACIAACIGAcAQhXxrHESCAQikDUwhXLRatXr6bm5uZQY0bdz6LVhAkT6Nxzz6URI0ZEfb8VbmDZav369fTRRx9RTU2NYVNiIYHZTBe5y+38D9XugEkZSn+dvl9tB6vLvmBlqD4IV3rq0bc5khXLVcfbOqiqzkdHGn10rKVDi3TF0a+Exxnx4RYpAzOdRAUimhVHtOpb7KSe+Q5yiuBWnGIQR/cEIFx1z8jIKyBcGUkTY4EACIAACIAACIAACIAACIAACIAACFiJwD3z59Hu3bu1Kf167lyaNOlMK00vYXNBisHo0OpTCJaXl9N9C+6PbhBcDQIgAAIgAAIgEBUBCFdR4cLFIBATgaiEq88//5yeeeYZQ6M4FRUW0ewbZmupAzmNXqofHo+H/vznP4t89e8LiSQKi6SbhY8ePZp+/vOfd3OVfU+HYqnvV9vB6rJPLdU6E1TbXIdwZczriv9z4EhXbZ4OamjxU02DX4hXfjp63E8+n0Y+/IOETCViLQnRKoNK8hzUW0S1Ku3hpGw3kVukFoRsFR6fejZa4YrvZflTRqSSpexXS309WJv7+FDH6eyx53cIV/bcV6wKBEAABEAABEAABEAABEAABEAABNKdwIrly2nNmrc0DFOnTqPpM2akFRK9dMWLR7Srri+BYFGtkEKwKyO0QAAEQAAEQCBRBCBcJYosxgWBkwSiEq7WrFmjyUQnb4+91qNHDzrnnHPokksuoby8vNgHsuCdLOls27aN3nrrLdqxY4chM+zVqxctEOF100VQ0EOTElR3/ep1weqyTy3VOo+vtrkO4UpPPb42i1ct7R1U3+yng/V+qjzKEa/8VN/kI5cQp/iL5SkOVuXXJK0O8gohKy9biFZCsOpb5KD+JU4tulUPkUZQXhvfrNLr7mDCFRPg9xf5HhOs1PfJe9RSX+c2H/Lezlbn92B96nm71CFc2WUnsQ4QAAEQAAEQAAEQAAEQAAEQAAEQAAFJ4L333qUnn3hCa54hMjTMn3+PPJV2pV68YumKj3ROMxhMtOKoVtNEJg+kEEy7/0SwYBAAARAAgSQRgHCVJPB4bFoRiEq4am9vp5dEfvK1a9fGBam4uJjmzJlDgwYNCvohfFyDW+jmlpYW+uCDD+iNN94gr9cb88w48tcvfvELOv3002MeI9VvlBKUfh36frUdrC771FKt8/hqm+sQrvTU42+zdOUTX1VCttpR7aVtBzy0vdJLuVkZlJPJ0hULOqSJVm0eouY2P/UTktXYgZk0vK+ThvVxihSCGST+hyMGAvEKV/xIKUvpS/WcOjV5XXd96nm71CFc2WUnsQ4QAAEQAAEQAAEQAAEQAAEQAAEQAAEmsH//fpo/b672O+/8/HxasnQZ8R8Mp/Ohl66YRbpGu9KnD2QWiGrFFHCAAAiAAAiAgLkEIFyZyxtPS08CUQlXjKi1tZWeffZZ2rx5c9TEXC4XnX322TRz5kziejocLOxs2bKF/vKXv1BNTU1A5ol07fwP1tmzZ9OoUaMivcWW10kJSr84fb/aDlaXfWqp1nl8tc31VBCuOBIUR4E6JlL1HW3qoOOtfmoWUaR8wmric06RrTPT1SkzFeY6RKQoh0jFlyFS8emJmttm4errKi99trOdPt3Rrs3RLebJIhULV36RgtAr1tDu7RCilZvOHZVFI/u76LQypxYBy9zZnnwap0Zk3kdFVK66pk7WrSJVIqdGFLi1KF1ZItVhXpaDikT6w5J8Zp2h7cPJUZJXg3BlLnsIV+byxtNAAARAAARAAARAAARAAARAAARAAAQSS2DBgvvoq+3btYfcfscdIpPFdxL7wBQaPZ3Fq2CiFaJapdCLF1MFARAAARCwHQEIV7bbUizIggSiFq54DU1NTfSnP/0pKukqNzeXZs2aRePGjaPMzEwLokjslA4fPkyrVq2iioqKiB/Eka1+8pOf0JlnnhmIJhPxzTa7UEpQ+mXp+9W2rMuS75V1tVTr+mv4XCoIVyz/sPCz95CPvjrgpep6H9Uc81GbJl0J2UrIVZx+r1eBg4b0ctGoAW4qyesUsFhsStZRrQlXPvrn1jZ6Z3Nr2GmMH+KmSyfm0OgBLhqWZOGKo25xWsSvqz2aMFZ7jEU3P7WLPeDoXVkiSlexEK36FDqFHCZ4C0mMo3dlWcQzTbRwxRupj2ilbwe7JuwLIIVPQrhK4c3D1EEABEAABEAABEAABEAABEAABEAABLoQeP65Z7WMDtx5xRVX0r9de22X82h0EgglXvFZO6UaDJY6kNcI0Yop4AABEAABEACB5BKAcJVc/nh6ehCISbhiNHV1dfRf//VfdPDgwW5JlZaW0nXXXUfDhw/v9lqjL/CLEDk+EXZGSjX8oT9/OZ0iQo7JpgnPYfXq1VqaQZ5TuMPtdoswu9No8uTJ4S5Lm3Ny//QL1verbVmXJd8r62qp1uU1ap+VhSuO/FTf3EHVdT76ptZHB4VoVdvgFxGuOkSEK37tiyhRughXHHGJxauBpU4a3MtJPUW0K5aDknGkmnB1vK1Di2hVeUTwFnJbbYOPjjSKCFeiX4twJSJfcYgr8fZCWUJy4whXzLdXIYtuThokmHOEsfzsJFpuYoqJEK749aO+p6p1/Tlu86G/prPXft8hXNlvT7EiEAABEAABEAABEAABEAABEAABEEhHAms/+ogef/wxbeks1Ny34P50xBDxmlm64mP16s5S3sipBvlIZfEKopXcTZQgAAIgAAIgYF0CEK6suzeYmX0IxCxcMYJDhw7R8uXLqbKyMiSRoUOH0rXir1zKyspCXmPUiWPHjlF1dbU2ryNHjlBDQwO1t7eTx+MRqcnYhOj8gJ9lKxaa8vLyqKioiFgI69OnD/Xt21cTsYyaT7BxeD7r16+nl19+WUvPGOwalhCuuOIKmjJlStqkXgzGQe2TApTax3V9v9qWdVmq18s+LtW6vEbts6pwxS9pTiHI4s82EdVq0zftVC/SCWpRlnghIQ4ROE1Lbzesj5PGD86k4SJa1NDeLuJ+TuVn5pEqwhVHrWJxjQWrXQd9tHW/hzbv8xBHuuKUh+EOTpHIEcbGDnTRGYPdmuRWVuQMpE0Md2+izoUTrviZ/B4kZSi1lHV5jVqGq+vPcZsPdbzOHnt+h3Blz33FqkAABEAABEAABEAABEAABEAABEAgnQjU1NTQ/Hlzqbm5WctgsWTpMurfv386IYh5rRUVW6lia8Up4hUPmEryVSjJiteBiFZMAQcIgAAIgAAIWIsAhCtr7QdmY08CcQlXjIRlq//8z/+ktra2UwiNHj2afvaznwmRw/joOV6vV5OpamtracOGDbRlyxZNtJKizCmTiaCDRawRI0bQhAkTaMyYMcRpEDn9YSKkAJauOC0jC1jqwc9i0eqaa65Ru9O+Hmpf9f1qW9ZlyRBlXS3VurxG7bOicOU7IVvtF5GW1u/00B4R3epIo0ghKFLa8blwBwd2Y7GqIMdBJSL6EktAE4dmUpFIMchpB808UkW4Yq51xztoR7WXNuxup6qjnSkEfcLCYhEr3MGsneIbs+4tIl2deVomjRFpEZl1thCxknFAuDKXOoQrc3njaSAAAiAAAiAAAiAAAiAAAiAAAiAAAsYTWLx4EW3ZvFkb+Ge33krnnXe+8Q9JgxGDpRqUy5by1ZjiFhpT1Cq7k1ayYMXHi3uKaWt9TtB5QLQKigWdIAACIAACIGAJAhCuLLENmITNCcQtXDGfnTt30tNPP0319fUaLpaGzjrrLE0aKigoMBTh4cOHqaKignbs2EF79+4ljmSViINFK/4LHY7QNWrUKGJ5zGhxjD+Ef+GFF7RIXLwG5nbOOefQ1KlTKScn+D9gErHWVBhTClD6uer71basy5LvlXW1VOvyGrXPasIV+z2t7R2073BnZKuPv26nmnp/YG28hmiO8UK4YgmII171L+FUm+K1GM0AcVybCsIVR7fitI27DnrpSxHVav2udmoSKRtjOTja1TkjMoXg1hnpqiTfYRprdb4QrlQaia9DuEo8YzwBBEAABEAABEAABEAABEAABEAABEAgcQRWrvxf+vOrr2oP+N73LqPZN9yQuIelycih0g2qyzdbwFIFK54HJCt1N1AHARAAARAAgdQjAOEq9fYMM049AoYIV7zsTZs20VNPPaURmDRpEs2ePdtQQamuro7+8pe/0MaNG8nn88Usl8SyRSxCcbSradOm0dlnnx3LECHv2bNnDz366KPEEbvGjx9PP/3pTxMSUSvkBFLkhBSg9NPV96ttWZcl3yvrXKr1cOesJlxxBrv6435a+1U7bdnrpQNHfXS8tZuwVnpwSrtXgYP6Fjtp8qgsmjDETS6niIBlfFA65Yknq1YXrli28vpEJD8RSeyD7e20o8pDNQ1+avfGJlw5RKSr/iUOIbe56NxRIp2jKDkCFktuZh6xClc8R34/DFaqffp6sHaoPu632wHhym47ivWAAAiAAAiAAAiAAAiAAAiAAAiAQPoQWP/ZZ/TIIw9rCx4+fDgtWvxQ+izepJVGIl/JqYwp6hr9iqNh6Q81OpaUqPTXbK3LEUJVZwSrUGKVeg8iWak0UAcBEAABEACB1CAA4So19gmzTG0ChglXjOHDDz/Uok794Ac/MCRCE4tVHMnq008/pS+++IJaW5MfRresrIzOPfdcLe1gz549Ddn9bdu20QcffEAzZ86kwsJCQ8a02yBSjtKvS9+vtmVdlnyvrHOp1sOds5JwxQJQs4huVV3np7c3t4qIS14t2pW3u7x2enBKO1NEXcrNEqksx2XTt0/PFKkGibIzzTGArC5ceYXH1iiiW3Eqwbe3tNK+QyJto5CteB9iOZgqs+1T5KRLzsiisYPclCfaLLmZeUC4MpM2EYQrc3njaSAAAiAAAiAAAiAAAiAAAiAAAiAAAsYQ4D+Cnj9vrpbZgv8Ib8mSpTREZITAkTgCFRVbqWJrhfaA1atfStyDuhlZClZ8WXn5mG6uxmkQAAEQAAEQAAErEoBwZcVdwZzsRsBQ4crv9xN/uVyuuDk1NDTQH//4R9q1axe1tbXFPZ6RA3BqQRajLrvsMvrud78biPgS6zNY/OEIV263O9YhbH+flKP0C9X3q21ZlyXfK+tcqvVw56wkXIn/vKiqzke7a3z00fY22inS3LFrFasAxOvmgEVuZwZ9W6S6mzQ0kwaVOqhnD3NCXFlduGr1dNDeQ37aVumhT3Zw6kafxpu5xXpw9LCiXAdNLs8kTufYV8hXeUJ4M/OAcGUmbQhX5tLG00AABEAABEAABEAABEAABEAABEAABIwi8PB//Ad9/vl6bbib5syhiy++xKihMU4UBGQELL5FE7IqOoWsKIYIeilLVfJgqap8TGcbgpWkghIEQAAEQAAEUpsAhKvU3j/MPjUIGCpcGbHk48eP08cff0x/+9vfqKXl1HC4RjzDyDFGjRpF3//+92nw4MHkdCY+TM2mPR5t+hOGppecJeUo/d7p+9W2rMuS75V1LtV6uHNWEq58QrjiaEvbD3jo890e2n9Y5Lsz4HAKCWjsoEwaN8hF5QNd1E+kGDTjsLpwdby1g7ZWemnrfo9I3+iho02xp25UeeZnZ9CZpzFvNw3v69QELPV8ousQrhJNuOv4iHDVlQdaIAACIAACIAACIAACIAACIAACIAAC1ifwyisv04urVmkTvfDCi+jmW26x/qTTcIYsYKmHjI4l+6REJduyhFQlSaAEARAAARAAAfsSgHBl373FyqxDwFLC1d69e+mFF16g6upq4nSCqXLk5ubS5MmT6corr4w72lW4NbNsdfvy+i6XTBiWGWhPGOKmCcPElw1lLClHBRZ7oqLvV9uyLku+Rda5VOvhzllJuPKK/yy+2NuuyT/bDnhFxCVjBCCHCLB0WpmLRvZ307dOc9OQXhCu+DVxTKQTXL+7XUvduFOIbsdajOGdK9IIjh7gpjED3SLKlYtKC8yJKMZr4gPCVScHs75DuDKLNJ4DAiAAAiAAAiAAAiAAAiAAAiAAAiBgBIEvvviCljy0WBtq0KBBtGTpMlP+2NiIuWMMEAABEAABEAABEACBTgIQrvBKAIHEE7CEcOXxeGjjho304ksvpkRUq1DbMmLECPrhD39IvXv3DnVJXP3BhKtQA7KIdf1FubaRr6QcpV+vvl9ty7os+V5Z51KthztnNeFqgxCAvhDRljjSVW2DMQIQC1dDertEtCUXnXN6Jg3rA+GKXxMNQrj6WKQS5OhWew/5qNEg4SpHCFcjBOtyIVxNEpJkLwhXjNu2B4Qr224tFgYCIAACIAACIAACIAACIAACIAACtiPAGSjmz5tLtbW12toeeOBBGjFypO3WiQWBAAiAAAiAAAiAgN0JQLiy+w5jfVYgkHThqq2tjV5++WUtjWAqRbUKtXl9+vSha665hsaNGxfqkpj7oxGu1IfYQb6ScpS6Lq7r+9W2rMtSvZ77ZH+wUu2zmnC1flc7bfrGQ7sOeunQMeOEq4GlLi3K1bkjM2l4GYQrfr3UC+Fq7VedwhWnb2xqNYZ3thCuhouIYuUDXCKiWCb1LkSEK+Zt1yOdhSsZ1l6Gs5dtu+411gUCdiGgTy0xbfp0uywt5dbB75t4D025bcOEQYDU91G8hybvBYH30OSxx5NBwGoE8L4c3Y489uijtG7dWu2mH//kerr88sujGwBXgwAIgAAIgAAIgAAIWIIAhCtLbAMmYXMCSRWumpqaaMWKFfT111/bCnN2djbdeOONNGrUKHI4jBMpYhWuVLipKl9JAUpdC9f1/Wpb1mWpXs99sj9YqfZZTbjaLKItbdnnoW2VXjpYb0zqzc6Ugm6RUpAFIKQUlK8zLaXgrk7eLLgZllIwS6QUFKy1lIIiFWhpD+PeJ+Tcw5VIKRiOjvHn0k244g+2Vr/0kgayoqLCeKAYEQRAICkEysvLNYEA4kBi8Usxld9H8R6aWNYYHQTMJjBtWqe8ivfRxJKXP4viPTSxnDE6CNiBAL8vl4/p/BnXDusxag2vv/5/9Mfnn9eG++7kyXTbbb8wamiMAwIgAAIgAAIgAAIgYDIBCFcmA8fj0pJA0oQrDkn8pz/9iXbu3GlL8C6XS4t0dcEFFxi2PiOEKzmZ6y/Ko+un5Mqm5UspQOknqu9X27IuS75X1rlU6+HOWUm48okAS18d8ArZSqThFFGuKo8YJFwJ34flH05xN26QiwaUIMIVvyaaWjuE3OalL4XgVrHfQ3XHjYlwlZedQROFaDV2UKfkVpwH4Yp52/VIF+GK5YDVqztFq1B7ORIpCEKhQT8IWIrAV199FXY+kAbC4onpZCSCAN5DY0KLm0AgKQQieR+FeGXs1shfIocbFe+j4ejgHAjYmwDelyPb323bttHC+xdoF5eVldGSpcsoJycnsptxFQiAAAiAAAiAAAiAgOUIyH8rjx07lhYtWmS5+WFCIGAHAkkRrvx+Py1btowqKyvtwDDsGmbOnEnnnXde2GsiPWmkcMXPTKVoV1KO0rPS96ttWZcl3yvrXKr1cOesJFz5O4iq63y0u8ZH/6xoo50HfYF16NlE03YK3+c7I7PoTBHdalCpk3rmmyMAVR/10ddVYi1b2+idza1hpzxeCEqXTsyh0SIN3zCR8jAj7NXGnGz1dNDeQz4tmtg6kVqwpsEYwa1QCFbnl2fRGYPd1LfYQbki4pWZByJcmUmbyO7CVShJgD/QuvLKKzXY+HDL3NccngYCRhJ47bXXtOFkqY7N4hWEAZVI9PVw76E8Gr+P4j00eq64AwSsRIDfP/mD/mAf9uN9NP6dCiX9y59F8R4aP2OMAAJ2IiDfj2WpX1s6vy97PB6aP29u4Pf1v733Xho7dpweEdogAAIgAAIgAAIgAAIpRADCVQptFqaasgRMF67q6uq0NIJ79uwxBVpGRgZlZWWR2+0mrrPs1d7ern2ZMQGOdMXS1bnnnhv344wWruSEUkG8knKUnLMs9f1qW9ZlyffIOpdqPdw5KwlXYtpa1KUDQrp6+4s2qhBpBds8fuLIV7EebmcG5WRm0AVjM+nbp2cSR1sySwCyunDlEX5VvYhq9XW1l/6xpU2LKNbu5ddObLRZq8p0Z1DvQiddMj5LRBNzU35OBrnNCSgWmDSEqwAKUyp2Fa6CSQL4YMuUlxQeAgJJIyA/mNLLV+n8wVQ8mxFMEpCCFQSBeMjiXhCwLgF+/9S/h/Js71uwQEvbat2ZW29mwd5D8bOo9fYJMwIBqxMI9r7MP9umY6rBJ373O3r//fe0LfvhD39EV119tdW3D/MDARAAARAAARAAARDohgCEq24A4TQIGEDAdOGqqqqKHnrooYDsYsAaugzBgtPQoUNpxIgRWtm7d2/KzMzUZCsWrqRo09LSQgcOHKDdu3dTRUUFHTx4sMs4RjYuvfRSuuqqqwwZkqUrPjbtPlGKtHKd7XatjOebldMMSjlKvz59v9qWdVnyvbLOpVoPd85KwhXPkyWguiY/fbyjXUt1d0BEiTouUt/FepQWOESUJSd9d2QmnSGiSLldGeQyJ8AVWV244ohiHiFY7RepGz/c3k47hXhV0+DX+mLh7RDGVf+eThrWx0XnCLnttDIXuYRsxf1mHhCuzKRtzwhX+g+48OGWua8pPA0ErEBA/+FUeXm5EAbut8LUUmIO8pcdcrL8Pnr33XfLJkoQAAGbE9C/h/JyIa9Gvun4WTRyVrgSBEAgMgLp/r781ltv0h9WrNBgnXX22XTnnXdFBg5XgQAIgAAIgAAIgAAIWJqA/B0kUgpaepswuRQnYLpwxbw4H/yzzz5LjY2NceNjwap///40ZMgQGjNmDJ122mmUnZ0d9bhHjx7V5sVz279/Px05ciQg5EQ92IkbWPAaN24c/fjHPzYl3z3LWM/8o1l7+qbdsQlYHO3q0RsLY11ywu6TcpT+Afp+tS3rsuR7ZZ1LtR7unNWEK55rS3uHFnVpm4hwtWWfhw6xBOSLLvISpxF0iehWp/d1iTR9bioXqfqG9jY31JIUrj4Q6RH/saVVE4/4vxvxPy1lIEeS8mt7RTRWpN+7dEIOjTIxpSCz5uOIENwq9nu1iGLbKz3U2Oonb5TZBZl1tptovFjHmIEuwd0tIl2ZZLZ1LiPwHcJVAIUpFbtFuNJ/wMXRWGTqQFOA4iEgAAKWIqD/cApRWsJvD0cHfGDhwsBFEFYDKFABgbQkoH8PhXTV/ctA/rJYXsmyKr+X4gABEAABIwik4/vyrl276Df3zNfwlZSU0JKly6iw0Hq/GzZifzEGCIAACIAACIAACKQbAflvaAhX6bbzWK+ZBJIiXPEC169fT08//XRca500aRJNmTKFSktLKT8/P66x5M1er5caGhro66+/pjVr1lBNTY08FXXJb16zZ8+OSQCL+mG6G+KRr6woXUk5SrfMgDQl+9XrZF2WfI2sc6nWw52zonDFKQQbRVSrb2q99OG2dtotymPN0UlAnEYwLztDpBHMonNFdKvC3AzKF20zDylcfbStjd7f2iaia5FIr5dBTvHF0pVPSGRe8eXxEpUPdNNFZ2TTqP4uGlbm1IQss+ba5umghuYOTbj6cHsb1dT7tNSOLIRFehTkOKikh4Mmj8qkiUNFKkHBOkukF0zGAeHKXOp2Eq70shU+4DL3tYSngYBVCaTjB1Ox7IX+PRTCaiwUcQ8I2I8A3kMj31P5i2K+A8Jq5NxwJQiAQHQE0u19+Z7587QMEEzp13Pn0qRJZ0YHDFeDAAiAAAiAAAiAAAhYloD8dzSEK8tuESZmAwJJE66Y3Ycffkgvv/wytbdHHo2Jo1cNGzaM/vVf/5UGDx6c0C3weDzaHD/44IOoxat+/frRrbfeSkVFRQmdYySDS/kq2qhXj95URBOEGGKFQ8pR+rno+9W2rMuS75V1LtV6uHNWFK7Y8+F0d0ca/VpawT21PqoSqQWPikhMLF7xuWAyEEtM2ULy6SHkH46uxKkEx4iIUSwzcRpBh0kBlzg6VJtI1Vdd56M9NT7iqFHbxBdHgXKKIFtOkWNPE67EQnziWr6eU/GNHeSmoX2cNLiXS8hKRJki/aEZyhKz5DlUCsYcUWz/YZ82d5awmkS0q2Cs+TXFa8jLyhAym2Bd4qSB4msMr0FEEuM0gnw+GQeEK3Op20W4UkUBfMBl7msITwOBVCCg/2AKka667pr6HspnIFt15YMWCKQ7ga+++ooefvjhAAZEugqgCFTkL4m5g38WZfEfBwiAAAgkikC6vC+vWP6U9gfHzHHq1Gk0fcaMRCHFuCAAAiAAAiAAAiAAAkkgIP8tDeEqCfDxyLQhkFThyidMirfffpv4A5pIDpaYrrrqKu2Xa263eSJQXV0dffbZZ/Tmm29Sa2trt1MdMGAA3XzzzcRhmK10PPNOs0g5eDyqKVlFupJylH7y+n61Leuy5HtlnUu1Hu6cFYUryYEloFYRfYllq+1VXvpafO2o9oqIUB2adCWv45LdHk7XV1rgEBGiXFqkqNEiWlSPnAzKERKWWfIPy0nNIiUiy2EHRaSoA2Lux0W0Ll6HNs8TEhIXWo/4xiXLVblZJEQxJ/UTolhJvkMTmXjeJ27R7k/UN553u4i21SL80N01XiGIeWnXQS99c8invZb00hXPyyG+DRCi2Ih+Lhopvjh9Y3ameaJYKBYQrkKRSUy/HYQrVRTAB1yJeZ1gVBCwAwFIV8F3UZ9GENEBg3NCLwiAAGnSFX/Izwekq5OvCPVnUQirJ7mgBgIgkHgCLMPa9X35vXffpSeffEKDeMaECTR//j2JB4ongAAIgAAIgAAIgAAImEoAwpWpuPGwNCWQVOFKMv/zn/+siVdSgJH9smS56pxzzqGZM2dqwojsN7vcvXs3PfPMM3T06NGArKOfQ48ePei2226j/v37609Zpn37igaKJtrV9Rfl0fVTcpM6/1CvDX2/2pZ1WfICZJ1LtR7unJWFK23e4tsxEWmJ5aXqOj9ViahRTUJgamkT6fhEhCgWgZwicpWbhSWRRpBFJY5s1be4s+TIVomWrXgOnhOy0rEWPzW2dBCXHI2rQZScIlEvLPHa1KNTYBIRo0QqPk7Pd/KrMzWiloowE70/KwAAQABJREFUwRG6WPzieR4WkcWqj4ovwbym3i8kLCGMtZ9Yh7iGmXK6wNyszkhi/QTrMsG8t5DdEs1aZRaqDuEqFJnE9Ke6cKV+wMWEli9fnhhQGBUEQMAWBFTpqry8nO5bcL8t1hXPIuQvNngMiALxkMS9IJAeBNQP9xEtkEj9WRTvoenx3wBWCQJWIqCPdGWX9+X9+/fT/Hlzyev1Un5+Pi1Zuox69eplJfSYCwiAAAiAAAiAAAiAgAEE5O8lEeHKAJgYAgRCELCEcMVRozi14Nq1a0+ZJqfku+aaa2jixIki1ZjINZbko76+nl599VVav379KTNh2WrOnDl02mmnnXLOah3RphlMtnQl5Sg9R32/2pZ1WfK9ss6lWg93zurClTb3E988IuKVxytSDYroUSwFtYmoUV4hOrH8k5/toJ49HNRDCEuZIkCcyN5nmvzjF0JVo5DAOKrVPpGOr+64n9q9IvUhi1Y89ygOLZqV+KZFvMp00MBSJ5UVnViXkMrMOKR41S5Yc2SuI8f82pqYN5/jtI2cRpCjieUIyS3T1cnanNl1TwDCVfeMjLwilYUrRGUx8pWAsUAgfQhAujq51xAFTrJADQRAIDICdv1wP7LVd71KfQ9FlNWubNACARAwj4Ad35cXLLiPvtq+XYN4+x13iD90/o55QPEkEAABEAABEAABEAAB0whAuDINNR6UxgQsIVxJ/k899RRt2rRJNik7O5t+9atfWTJaFAti74rQy1Lacblc9POf/5xOP/30wPxToRJNmsFkSleSs56pvl9ty7os+V5Z51KthzuXCsKV5CICWmkSE0dc4rR9ImunSC3YISJcdUo/HHHJLeQfjnhllvzTLCJtccStQ0JKYhGsQUS14vnxvMT/Yj54DS5hjbHYVJznoF5CbirM7YwqxRGmzDg4Mhd/MetW5s0RxcSDXYJ3Z4SrDC3aFc/VSgeEK3N3I5WFK/VDLkQUMPd1g6eBQKoTQIQWRGVJ9dcw5g8CySSgfrifztECZ82cEdgGpGMNoEAFBEAgCQTs9AcFzz/3LL3xxhsaxSuuuJL+7dprk0AUjwQBEAABEAABEAABEDCDAIQrMyjjGelOwFLCFUeP4pR9O3bsoJ49e9JNN91EgwYNsuQetbe3a/84/cc//qGlObxW/OP07LPPtuRcu5tUKkhXUo7Sr0Xfr7ZlXZZ8r6xzqdbDnUsl4UrPJ5ltlqlYPjosRKtDjT46IFLwcYQryd2ouWWIHH0cUap/iUjbV+SkUhHFi9tWSN1n1BqNHgfCldFEw4+XqsIVZKvw+4qzIAAC3RPgn+X5SFdZQBUFkI61+9cLrgABEOhKQP1w3y4prLquMHwLP4uG54OzIAAC5hOwwx8UrP3oI3r88cc0eOn6M7r5r5zEPZGjkvNRsbXilIfIc3yivHzMKee1/jHlIc8FvQGdIAACIAACIAACKUcAwlXKbRkmnIIELCVcMT+ZXnDKlClUVlZmaaR+kQ/tr3/9q5bjnmUrFj9S9YhGunr0piKaMFTkpDPxCCXp6PvVtqzLkqcr61yq9XDnIFzFttGcXq+lnajyqI+q6nx0vNWvpd+LbbTwd3Gkq3yRKrFnvoMG9HRRkYh0lS1S+YlAUziCEIBwFQRKArtSVbiCKJDAFwWGBoE0IaBGaJk2bTpNmz49TVaO6FZps9FYKAgkmEA6i6vyZ1FEWU3wiwzDgwAIRExA/dk2FWWlmpoamj9vLjU3N1NmZiYtXbqM+vXvH/H6cWFyCEhxiqWqQL3iVMEq3tnxa5oPKWel07/d4mWH+0EABEAABEDAygQgXFl5dzA3uxCwnHBlF7CpuI5opKv3FvcydYlSjtI/VN+vtmVdlnyvrHOp1sOdg3Clp959myNbNRzv0CJa7Tvio4P1vgDv7u+O7QoWHgtyMmiQEK56FzqoRMhXLmdsY9n9LghX5u5wKgpXiChg7msETwMBOxNQIwGsXPWinZfaZW1SFOBORLfqggYNEACBKAika5Qr/CwaxYsEl4IACJhKIJXflxcvWkRbtmzWeP3s1lvpvPPON5UdHhYZASlV8f8X8lERo1w1dqw/8MAvv3QE6tFW+A9n+ChHNKxo0eF6EAABEAABELAEAQhXltgGTMLmBCBc2XyDo13epj0eun15fbe3TRiWSY/eWNjtdUZdIOUo/Xj6frUt67Lke2WdS7Ue7hyEKz318G1OJegXX5VCtNpd66XGFj8db2MFK/FHliuDeuQ4qF+xk4b2diLKVQjkEK5CgElQdyoKV/KHcEYCUSBBLwwMCwJpQkCNBJAuKbEgCqTJixvLBAGTCKRjlCsprSK6lUkvMjwGBEAgKgKp+L68cuX/0p9ffVVb5/e+dxnNvuGGqNaMixNLgCWrSAQrVaIaM8ZPsi3LaGYpJSxZbt3aKWXJdqixWMCCfBWKDvpBAARAAARAwHoE5Gc9Y8eOpUVCwMcBAiBgPAHbC1dtbW3EX16vlzgFoMPhILfbTTk5OeRyuYwnaoMRI5Wurr8oj66fkmvKiqUcpX+Yvl9ty7os+V5Z51KthzsH4UpPPXzb4+0QqQOJ9gjZasdBL/mEfeU7+UdV4W+O8yynEHSKb32FcHV6mUuLeMWpBXF0JQDhqiuPRLdSUbjCh1yJflVgfBBILwIyylUqpl6JZafkLzL4XkirsRDEPSAAAiqBVI6moq4j0roqrd599900cuTISG/FdSAAAiBgCgH5sy0/LBUiuK7/7DN65JGHNTbDhw+nRYsfMoUTHhKeQCSSFYtUUqyKRaoKP4PQZ1m6kuLVqlWhPz+BfBWaIc6AAAiAAAiAgFUIyN9TQriyyo5gHnYkYEvhqq6uTvyj4Ev6+uuvqaGhgVpbWzXhigUbKVzl5uZSv379RF7ychoxYgTkK92rO9L0gmZJV1KO0k0zIE3JfvU6WZclXyPrXKr1cOcgXEm6kZXHWjro8DEfVdX56MBRTiUouEd2qyFXicyCVJLnoP4lTpFa0EmlBQ6CctUVLYSrrjwS3Uo14QofciX6FYHxQSD9CKTah1Lx7hCk1XgJ4n4QAAGVQLpFClR/FoW0qr4SUAcBELAKgVR6X+bfkc+fN5fq6+spQ/zCbMmSpTRk6FCroEzLeUjRKliqQJaqZs70alzMFKwi2YiVKzvFq1ACFstX06Z3ph+MZDxcAwIgAAIgAAIgYA4BCFfmcMZT0puAbYQrjmBVXV1Na9asoY0bN2rRrCLd2uLiYrr00ktpwoQJ1KNHD+0foJHea+frIpWu3lvcK+EYpBylf5C+X23Luiz5XlnnUq2HOwfhSk89fPvQMT99c8hLRxr9VHfcpNBWuinlZWVQcb6TBvZ00uBSp/hvWndBmjchXJn7Akhl4Qofcpn7WsHTQMCuBFLpQ6l490AVBZAKK16auB8EQEASkOJqOkQKhLQqdx0lCICAlQmkSlrBh//jP+jzz9drKG+a81O6+OKLrYzV1nMLJVpZWbIKtSHh5CuIV6GooR8EQAAEQAAEkkMAwlVyuOOp6UXAFsJVZWUlvfnmm8R/GcLRrGI5+K98ysrKaPLkyXT++edDujoBMRLpyowoV1KO0u+tvl9ty7os+V5Z51KthzsH4UpPPXy78oiPth3wUmOrn9o8Zsa2OjkvtzODskQqwdN6u2h0fxeEq5NotBqEKx2QBDdTTbjCh1wJfkFgeBBIUwKp8qFUvNujCleQVuOliftBAAQkASlccTsV0lfJeUdb8ofRDyxcqN0GaTVaergeBEDATAKp8L78yisv04urVmlYLrzwIrr5llvMRIRnnSCg/vtAhSJFK6tFslLnGEmd5atgUa8gXkVCD9eAAAiAAAiAQOIJQLhKPGM8AQRSWrjy+/20bds2+sMf/hCzaBXsJTB69Gi64YYbiNMO4iC6fUUDbdrdHhZFoqUrKUfpJ6HvV9uyLku+V9a5VOvhzkG40lMP3vYLt8rr66B9LFxVeqmlvYN83JmEwyEiWjmFdDVMCFflQrhyuzLI6UjCRCz6SAhX5m5MKglX+JDL3NcGngYC6URAfihl9+gs8pcYI0eOpLvvvjudthhrBQEQSCCBdIkUqH4oDWk1gS8oDA0CIBA3Aau/L2/+4gt66KHF2joHDRpES5YuE78nc8a9bgwQOYFgEa3sIlkFowDxKhgV9IEACIAACIBA8gnI31WOHTuWFi1alPwJYQYgYEMCKStccQrBN954Q0shKMUZI/dn+PDh9IMf/IAGDx5s5LApO9YFvznU7dwfvamIJgx1d3tdLBeE2mN9v9qWdVnyc2WdS7Ue7hyEq8h2zCNkK5as9h320VdVXmr3Jke2Umc7pJeLRvVzU65IMZiVmJem+riUqUO4MnerUlW4YlGAhQEcIAACIGAEASlc8Vh2js6CKIFGvFowBgiAgJ6A1T/Y18831jaEq1jJ4T4QAAGzCVj5ffn48eM0f95cqq2t1bA88OCDNGIE/m1v1msk3UQrPddQ6QYR8UpPCm0QAAEQAAEQMIcAhCtzOOMp6U0gJYUrjmz19B+epo2bNgakmURsY0FBAd11113Us2fPRAyfUmNGklqQF/Te4l4JWZeUo/SD6/vVtqzLku+VdS7VerhzEK701IO324Rg1djSQZxScOdBL7GAlexjYE8nDS9zU4+cDMoT0hWOTgIQrsx9JaSScIUPucx9beBpIJBOBF577TXiLz4gXKXTzmOtIAACRhCw8gf7RqxPjoGfRSUJlCAAAlYnYOX35ccefZTWrVurIfzxT66nyy+/3Oo4bTM/+YGmuqCZM700a5ZX7UqLerCIV5Cu0mLrsUgQAAEQAAGLEZA/nyDClcU2BtOxFYGUFK44utWzzz5LGzZsSOhm5Ofn06233kocehkHUSTSVaJSC0o5Sr8P+n61Leuy5HtlnUu1Hu4chCs99eBtjm5V3+ynA0d99E2tzxLCVb9iJw0WaQVL8hxUIKQrHJ0EIFyZ+0qAcGUubzwNBEDAmgQgXFlzXzArEACB1CFw0003aZO184eV8hfBSMuaOq9LzBQE0pmAFd+XX3/9/+iPzz+vbct3J0+m2277RTpvkWlrDxbVKl1FKz10iFd6ImiDAAiAAAiAgLkE5L+zIVyZyx1PSy8CKSlc8Ra1tLTQihUraNu2bQnZMYfDQTfccANNmDCBMjIgakjIyUotKOUoOQ9Z6vvVtqzLku+RdS7VerhzEK4k7fBlsxCujjYK4arOJ9IKesnrC3+9GWfLipw0qNRJPfMdVCSkKxydBCBcmftKgHBlLm88DQRAwJoErBwFwChi/EHLAwsXasMhLatRVDEOCICAJGDFD/bl3Iwq5S+CIVwZRRTjgAAIJJKA1d6X+XfkC+9foC25rKyMlixdRjk5OYlEgLEFATU6IwMZO9ZPLFtxieMkAb14ZWeB/OSqUQMBEAABEACB5BOQ/86GcJX8vcAM7EsgZYUr3pIjR47Qk08+SVVVVYbukMvlounTp9Nk8ZdAOLoS2LTHQ7cvr+/aqWtNGJZJj95YqOuNrynlKP0o+n61Leuy5HtlnUu1Hu4chCs99eDt5rYOOiyEqyoR4Wq/+PJaIKVgn0IncVrB0h4OKhbSFY5OAhCuzH0lpJJwJX/4xodc5r5G8DQQSAcCEK7SYZexRhAAgUQSsNoH+4lY66yZM7Rh8bNoIuhiTBAAAaMJWOl92ePx0Px5c6myslJb5m/vvVcIP+OMXjLGUwggqpUCI8KqXrri2+5bsIDKy8dEOAIuAwEQAAEQAAEQiJaA/MwHwlW05HA9CEROIKWFK15mXV2dJl3Jf1BGvvTgV3I0q2uuuYbOP/98YvEKx6kEkpFaUMpR+tno+9W2rMuS75V1LtV6uHMQrvTUg7dZuDraZMEIVz1d1LNHBiJcKdsG4UqBYUIVwpUJkPEIEAAByxOAcGX5LcIEQQAELE7ASh/sJwoVhKtEkcW4IAACiSBgpfflJ3733/T+++9ry/zhD39EV119dSKWjDFPEFAj23IXolpF99LQi1eIdhUdP1wNAiAAAiAAAtEQgHAVDS1cCwKxEUh54YqXvX//fvrv//5vamxsjI3CibtYtpoyZYomXMU1UBrcfPuKBtq0uz3sSh+9qYgmDHWHvSbSk1KO0l+v71fbsi5LvlfWuVTr4c5BuNJTD95uESkF64530IGjXtp7yEcei0S4GlAiIlwVOKgEEa4CGwfhKoDClAqEK1Mw4yEgAAIWJwDhyuIbhOmBAAhYnoCVPthPFCwIV4kii3FBAAQSQcAq78tvvfUm/WHFCm2JZ519Nt15512JWC7GPEFAn0KQ0wfOmuUFnygJQLqKEhguBwEQAAEQAIEYCUC4ihEcbgOBKAjYQrji9e7Zs4eWL19O9fXh092FYsOyFacQvFr8BVB2dnaoy9B/gkAkqQWvvyiPrp+SawgzKUfpB9P3q21ZlyXfK+tcqvVw5yBc6akHb7d6OuhYSwdVHvHR7hqvJYSr4jwH9RZpBfsXO6lPEVIKyp2DcCVJmFNCuDKHM54CAiBgbQIQrqy9P5gdCICA9QlY5YP9RJKCcJVIuhgbBEDAaAJWeF/etWsX/eae+drSSkpKaMnSZVRYWGj0UjHeCQJ62erBB9u16FYAFBuBL7900L33ZgZuRqSrAApUQAAEQAAEQMAwAhCuDEOJgUAgJAHbCFe8woqKCvr9739PnLc+2mP8+PE0Z84ccjggZUTKzszUglKO0s9N36+2ZV2WfK+sc6nWw52DcKWn3rXt8xNxdKsjjSKdoJCt6o77qbHVT6KbMrpeampLbDH5fERe8TWo1Kl9lfRwUEFOBjmdGeRI5uRMJXHqwyBcncokkT0QrhJJF2ODAAikCgEIV6myU53z5D9i4X8XFRQUpNbEMVsQsDEBK3ywn2i8EK4STRjjgwAIGEnACu/L8+fPoz27d2vL+vXcuTRp0plGLhFjKQTkh5WyC7KVJBF/ydIVy1d8lJeX030L7tfq+AYCIAACIAACIBA/AfkzzNixY2nRokXxD4gRQAAETiFgK+GKV7d+/Xr605/+RO3t4dPdqSTGjBlD1157LT5QUKFEWL/gN4e6vfK9xb26vaa7C6Qcpb9O36+2ZV2WfK+sc6nWw52DcKWn3rXNslVNvZ++OuClz0WaSREsjvr2dJLbSUmTmli28ouv3dU+2rrPS0N7O2l4XydNGJZJp4syNytDzC99jSsIV11fw4luQbhKNGGMDwIgkAoEIFxZd5e2bdtGa9asoXfeeUeLGnz48OHAH7BkZmZSaWkp9e/fny688EK65JJL6IwzzhA/76Xvz1Gx7uSjjz5Kx44dC9w+e/ZsGjhwYKCNCgh0R8AKH+x3N8d4z0O4ipcg7gcBEDCTQLLfl1csf0r7GY7XPHXqNJo+Y4aZy0+rZ8kPKnnRY8f6idMIconDOAL6FIP3LVgg5Ksxxj0AI4EACIAACIBAmhKQP8dAuErTFwCWbQoB2wlXLNGsW7dOk64iIVhWVkZ33HEH5efnR3I5rtERMCvKlZSjdI8PSFOyX71O1mXJ18g6l2o93DkIV5LuybLd2yGiWhEdrPNR1VE/7TvkpW9qfbSn1ks9cjPotL4urczNTM6HcTy/420dtKPSR1u+8VBPEdmqt0gpeFqZi4b0dlH/Egf1EWkGS0V/tphjukW7gnB18rVsRg3ClRmU8QwQAAGrE4BwZb0d+uyzz+g3v/kNbdy4MarJsSS0cOFCuuKKK6K6z64Xt7a2alGWf/GLX4RdIotqVVVVgWv++te/0llnnRVoo5J8Arw/LB5ed911yZ9MkBkk+4P9IFMyvAvCleFIMSAIgEACCSTzffm9996lJ594QlvdGRMm0Pz59yRwpek9tPyQkimwZMWRrXAkhgCkq8RwxaggAAIgAALpTUD+LAPhKr1fB1h9YgnYTriSuN5880167bXXZDNoOWDAALrxxhupd+/eQc+jMzICZkhXUo7Sz0jfr7ZlXZZ8r6xzqdbDnYNwpadO1NjSQbUNfvrk63b6Yo+H9tR4qa7JTz7BtUyITKf1c2mCEwtNyTh4foeP+WnPQS/trPJqqQ0dwqrKdGVQzwIHTRjqpjOGuGncYLcmYzlFNK7kqGHJoEME4cpc7hCuzOWNp4EACFiTAIQr6+yLT+RcvvPOOyP+A5VQMz/vvPPoCfFBXzr/W+pvf/ubJq01NDTQrl27QqHS+iFchcWT1JMej4eefPJJeuSRR+j888+n5557LqnzCfXwZH6wH2pORvdDuDKaKMYDARBIJIFkvS/v37+f5s+bS16vV/sD4qVLl1Fpr/izCySSVaqOLT+g5PlzVKtZs7ypupSUmbcqXSG9YMpsGyYKAiAAAiBgYQLy5xkIVxbeJEwt5QnYVrjif3TyL8D//ve/B92krKws7YMGTo+BI34CiU4tKOUo/Uz1/Wpb1mXJ98o6l2o93DkIV53p+fwiUjanDqwWUa32HRJfh7n0ighXPqoXslWrR+TwE0dhXmc0qSF9nDS0j0tLMaidMOkbz6JWzHO3kK0OirkdEmKY2G7tYOkqR0S06lvsoP4i7eGgXi4aWOqkwb1EtCshYpXkO0yfr0lYujwGwlUXHAlvQLhKOGI8AARAIAUIQLiyzibdfvvtYWUrTiPIUYDdbjdVVlZSW1tbyMlzanb+I5eCgoKQ19jxRGNjI/30pz+lt99+W1serx/CVWruNKfU5A/Mv/76a20B//Iv/wLhKolbCeEqifDxaBAAgagJJEu4WrDgPvpq+3Ztvrff8Ss655xzop47buiegPxwkq+EbNU9LyOvgHRlJE2MBQIgAAIgkO4E5M80EK7S/ZWA9SeSgG2FK4bGQs0LL7xAa9eu7cKwsLBQ+6XqsGHDuvSjETuBREe5knKUfob6frUt67Lke2WdS7Ue7lw6C1csKvnFF6foaxMRs7fu99AmEdHqy30iclS1l3zipJ8vUA63iCLFafpGD3TROBFFyiGCXJmVso/n6xNf+0R6wy1invXHhQjW3nV+cqo8z/wcB40Q0bjOPj2TRvZ3iZSDTnI7RYpBMWeOeJVh07BXEK7kq8CcEsKVOZzxFBAAAWsTgHBljf3hKD733ntvl8lkiB94LrnkErrllluIBari4mLxM9DJH4IOHz5MvH8czYqjCOuPyZMn0yuvvNLlHv01dmvv2LGDzj333MCyIFwFUKRc5ZlnnqG77747MG8IVwEUSalAuEoKdjwUBEAgRgLJEK6ef+5ZeuONN7QZX3HFlfRv114b4+xxWzgC8oNJvgayVThSiTsH6SpxbDEyCIAACIBAehGQP9dAuEqvfcdqzSVga+GKUba0tGjS1YYNGzSy/OHBzTffTOPGjTOXdBo8LRLp6r3FsYW4lnKUHqO+X23Luiz5XlnnUq2HO5fOwhWn5qtp8NFeITDtOuijAyKqVdVRLx1p9FN9cydDlpzUg2UllzCsBosIV5xasFhEvCrIPfmhnXqt0XWWq+qaOmi/iMDFQlhzm0hz6Av+FJ4npxgsEvPrU+SkASLiFUe7YulqiIjMlZ/dGQ0r+N2p3Qvhytz9g3BlLm88DQRAwJoEIFwlf19aW1tp4sSJxAKVPEpKSmj16tUR/9to8+bNdN1111FVVZUcQis5BRuLKulyxCJccSQlTl0nj+HDh1Nubq5sokwSAQhXSQIf4rEQrkKAQTcIgIAlCZgtXK396CN6/PHHNBZItZa4l8Tql14SPx+/pD1g7Fg/Pfig+AtUHEkhAOkqKdjxUBAAARAAAZsRgHBlsw3FcixJwPbCFVNvbm6m3//+91pKjB/96Ec0adIkS26GHSbVXWrB6y/Ko+unRP/BgpSj9Iz0/Wpb1mXJ98o6l2o93LlUEK7YeWLxiYNNccQprmtfvLATBwcq4C+H+MYRp/iL2+rB9/hE6kCPCBPF8tJBkZpvR7WIaLXXq0W2OtYcOmKUOg7XSwsd1LdESEwiXV/fYmfCI13x3DmiFcthVUd8dLDOTx4RmSvSo1ikE+xd6KSJw9w0Yahbk7B69nBQlpuIo2Fp/E4MJtlymkW/aGhtZq48rJP1Cd4Wi5gF4UrZKBOqEK5MgIxHgAAIWJ4AhKvkb9Gzzz5Ld911V2AinDLw5Zdfpu985zuBvkgq69evp6uuuora209++HTmmWeGTOUeyZipdk0swlWqrTFd5gvhylo7DeHKWvuB2YAACIQnYKZwVVNTQ/PnzdV+x52ZmUVLly2lfv36h58gzkZNALJV1MgSfoMqXU2bNp2mTZ+e8GfiASAAAiAAAiBgJwIQruy0m1iLVQmkhXDF8Ovq6ujQoUPEf0ns4NA2OBJCIJIoV7FIV1KO0k9a36+2ZV2WfK+sc6nWw51LBeFKSlLNQpI63ibS/4k/nm/3iHR/wgBiIcgp7B+XszOiE6f7y8vKECJRhojw1Jk6T3LlcVhaqhay0rZKL+2p8dKBI1461OCno02dAhNfE8nBz8kRzxkuolwN7u2ivGyibPHMRBy8TparaoUgtnWfhw6L+bYIDtwf6cHRrnjOLFn1EuLVUBGha5iIdMUpB8uKHVo0LOeJtw6PiJrFUhqzbhZfzJr7OMUiHw5hs7kFb2acI8bMFxxY2uI+KxwQrszdBQhX5vLG00AABKxJAMJV8vfltttuo5UrVwYmcv7552vRrQIdUVQeeeQRWrZsWZc7PvvsMxoyZEiXPrs2IFzZZ2chXFlrLyFcWWs/MBsQAIHwBMwUrhYvWkRbtmzWJvSzW39O5513XvjJ4WzUBFTZim9+9dXWqMfADYkhcO+9mfTll52/lIV0lRjGGBUEQAAEQMC+BCBc2XdvsTLrEEgb4co6yO0/k+6iXE0YlkmP3lgYFQgpR+lv0verbVmXJd8r61yq9XDnrCpcsdrD8hPLPi38JWSrptYOahRfHJ2qTfTxeV6nUwhALFyxAJQr5J/8bIcmXeUKGYglIBaJ+N4GEcGq6qifdh/00mYR1WrfIa9I0RddpChmydGdWK7iZ5UPdtPpQlpi8Yjn0C5EJY4MZcTB+haLTFKEYtFq616PFt2qscVPbd5OESuaZ3HaUZczgwaJ9ILDRHrBsYPcmnxVIkQsXg+zbBNyF4tWzKzphHDF+yCWFuDNAldWJgnODuoh0hPyvcybJTe3GJ8ZJeuAcGUueQhX5vLG00AABKxJAMJV8vfl8ssvJ5ai5PHrX/+a7r77btmMquT0ePoP+p5//nm67LLLohonVS+GcJWqO3fqvCFcncokmT0QrpJJH88GARCIloBZwtXKlf9Lf371VW163/veZTT7hhuinSquj4CA/P8gvpTTCHI6QRzWIXDNNeIveU8c9y1YQOXlY2QTJQiAAAiAAAiAQBgCEK7CwMEpEDCIAIQrg0BimJMEEhHlSspRJ5/SWdP3q21ZlyXfIetcqvVw56woXLFsxdISR7SqPeYTUlSnLMWSlVdYPyz+SKlJLLUzHZ4QfDiNoJSvOMpVQY5D++L69gNe+uqAR5OtDoiUfI0tnRIXR46KJlIUC0ssEw3o2RklapIQ7EYOcGnRoJpaRaQsEUGLhTAjDg5WV5LnIE4H2FN8eYRgtf+wj7aL6FxbhHhVU++jeiGMRfs0XkOOkKVYTOOx+wv5arRYQx8R6Yr7WgVnFrpkVKvO1ILi9aW9kDp5M2ueH/NmwapHTgYVibn2LhBjipLPJ0u6gnBlxKsv8jEgXEXOCleCAAjYlwCEq+Tv7QUXXEBbt24NTOTWW2+l+++/P9COtjJmzBiqra3VbnM6nbRw4UK6+eabIx6mtbWVPvzwQ3rnnXdo7969WjRiv/ihqrS0lAYPHkwXXXQRcRSunJyciMd8//33A6kOCwsL6eyzzw7cy2nm33jjDfrnP/9JVVVVWgRkjn48fvx47euMM84gvifU0dTUROvWrdNO8/1qesbc3Fxavnx5l1uZN6dtlAc/t62tTTbprLPOoqKiokBbVvbv30/bt2+XTeJ0jSUlJYH2p59+KiIuvErffPMNHTx4UBujf//+WmrIqVOnUnb2yQ+jAjeJytGjR7UIZxs2bKDKykpyuVx0+umna19TpkyhkSNHqpdHVd+zZw+tWbNGE/o4qjQ/i1n26tVLm/+ll16qPSfSQfUMJk6cqL0u5P0s/L3yyivE4lt1dbW2ZrmPXI4YMUL8nC1+2A5xqOPz62/FihWBK7/1rW/Rr371q0A7Pz8/6rSbgZsNrpj1wb7B045qOPlhN78eYxVCo3ogLgYBEACBOAiY8b68XsjyjzzysDZL/rll0eKH4pgxbg1FQH4QyednzvTSrFniF4w4LEWAI1xxpCt5QLqSJFCCAAiAAAiAQHgC8uecsWPH0iIRNRUHCICA8QQgXBnPFCMKApFIV+8t7hUxKylH6W/Q96ttWZcl3yvrXKr1cOesJlyx/OQV6euOCeGH0//VHhOliEx1XERbYtkq0oMjUHH0JY68xPUNuzz05d52TVjiqFbRHiwVcUq+IiEolfZw0hCRku80ESFqeF+XJl+xwMVzPiLGZlmJpSUWpLxiQSyFRXq4hKkkPtPrjKAlnseRp1i2KhFf/LlKvZDPvqn1atLVvsNeqhTyGK+n/jhH/IouuhZLUZmCTVmxkyYMc1O/nhytSohdgn9zW+QiF3/gw5w50lUvIVwxHxawOOoXzzn0x0GRUonuOghX0fGK92oIV/ESxP0gAAJ2IADhKvm7eP3112vCkZzJuHHj6O2334453frOnTvFz2ROTQYqKCgIK7jIZ3LZ3t6uyUn/7//9P6qvr1dPnVJnaeeOO+6gOXPmUGbmyQ9YTrnwRMfo0aPp8OHDWmvChAmaBMQ/8z/55JO0dOlSYukq1JGVlUX//u//rj2PBSr9ESyql/4atc2veVWUYqGLRS15/PWvf9WkK9mW5dNPP00cfUweL730ErG8xfded911tHlzZzoheV4ti4uLxYeyj9CVV14Z6PZ6vfTQQw/R//zP/wRktMDJExUWw37+85/TnXfeScwh0oMFPhbt3n333W5vYXmOBT/+BV93h57BCy+8QJdccgmx2MWpMT/55JOwQzDrxYsX07e//e2g17EcN3/+/KDn9J2jRo2iDz74QN+dlLYZH+wnZWHKQyFcKTBQBQEQsDyBRL8v19XV0fx5c7Wfl/j3SkvEzzJDhgy1PJdUm6CaShCylbV3b+VKF61aJVIHiKO8vJzuW3C/Vsc3EAABEAABEACB0AQgXIVmgzMgYBQBCFdGkcQ4pxDoLrXg9Rfl0fVTTv0w45SBRIeUo/Tn9P1qW9ZlyffKOpdqPdw5qwlXLPscF2nsvhHp/mpFCj0Wf9pPRKGKRlximcghvnF0JvE5jBYRakeVh5qEGPX/s3ce8FIT6/t/OY3ee682QJSmIEpRsHOvKGAX1L8FFMV2AVFQmiKIYlfKFUUpFkS9FlSKV8WKoD/KFQGl995P4T/PnDM5k2x2N7ub7GbPvu9HzGQymUye7NnN7nzzvHDKijTKCresGsIBqmndTGrdOItqVkqTcBFgLqTXyxWDOy6OgzR8eyQolkv7D+en5gMI5STwAxMgJcBLcIqqImCrMgJiyk9XmI8toa/j2SRTLP4tNFq9JYeWrcuhFeuzZR20chrQqLg4XvUK6dSsfgZVq5AmQS/s73DIxqHSCtyukFawvHC4alAlQ+qDNIs4TjyDgat4qk3EwFV89eajsQKsgD8VYOAq8dfl2WeflRCKPhI4yOhwj77NizKgoWuuuYYAL0USgHRmzJhBNWrUCLmbFbiCo9UNN9zgCAhSHQPSmTVrFpUuXVpVyWUigSs4fsG9Cs5M4SJN3HROnDhR6rxv3z669tprTakkQ+1/7rnnSvesUG3UNkBsw4cPF98lnD+ogbEBhFIT1Kov69IOuEIb7BcKmtP7gYPXpEmT6PLLL9erZZmBqwBJfFPBwJVvLgUPhBVgBRwooD7PevbsRT179XKwR2RNxj31FP3yy89yp/932+3UtWvXyDrg1mEVYNgqrES+a6BDV1797fnupHlArAArwAqwAqxADAowcBWDeLwrK+BQAQauHArFzSJXwInLlVPoSsFR1lFY6/V1VVZL7KvKWOrlUNv8BFwB8tl9II92in+b9wjnJgEuwdUqUvhH1zFXzJHIVHwCTtoo0vFt35tH+4RjVjh4S7BPEqQC8AQgqbZIIVivKtIIZghnqwwqVwoOWmaSCOME8ARgDMeAK9eR40THRB1SF2IseeLAaIfjA0QCFCbMswhgEsAtwFX4V170j5SIqMM2u9gldEJawTVbc2ndtlxCqsStYh0aYgwnxIHEYWwD51dcwGLVBWRVs1I61RVpBcuKY2aIwWBbtAEnMIy/lnDNqlY+Xbhz5TuDRdtfNPsxcBWNatHvw8BV9NrxnqwAK1B0FGDgKvHXcteuXQTnHz2tHUZ10UUX0T333GNKv+fFaAELXXbZZTIFnN4/XLKQIgdPqcNd6ddff5Wp4qwgT506dWjevHkyTZ2+v162Alfo991339WbSNepxo0by2MEc9gaOHAgDR061LTfH3/8QVdccYWsy80V95MibZ4KPBSAVIh6fPvttwTHKRXROlyNGDGCXnjhBSN9I/qrWbMm1a1bV9YhHaP6XqOOBWew77//nvr372+CzQAh4dzRHudjF0ivpztk2bUBNAWAzxpw9MJ1hO5/iZSHuJaAvqwBbaFxsLACV3CjAkSmw1Y4l0aNGsnUiIDhrBqgb6Sj/P333wNSRb711lsGfIjUlgcOHDCGAic1PbUk0i7OnTvX2J7IgtcT+4k8N3VsBq6UErxkBViBZFDAy/fl999/j2YLABzRRaRZvuOOO5NBkqQbo/rcwcDnzDmadONP1QEjtSBSDCI4tWCqvgr4vFkBVoAVYAWcKsDAlVOluB0rEL0CDFxFrx3v6UCBcC5XZzbKomdvLR+2J7sf0LGTtV5fV2W11NujTtXbLfU6PwFXSCW4emsOIVUegKHjwokqGDAUVlStAeAmwE+7Req95X/n0CYBXgF8Qn2wSBffa8uVygeRzjopi5rWy6CG1dKl41Q6oCSxox2YhC7RL/7hfPDv4FGRZlAcHzAWHLwkRCbaZYpjAFDKFG7RSH+IlHwArLCOvoMdQ40Z4JaEvIRO6H/p2mxaui6blgu3KwBruQJWC3aOAL3KibR/cLaqK0AyuGplimO7EdCujEhNiPSCTQScVqFUfjpEN/p20gcDV05Ucq8NA1fuack9sQKsQPIqwMCVP67dU8IpYdy4cbaDadCgAV1yySXUpUsXmYrNLq2e7Y4OK/v06UNIo6cH3IcwpqpVzanGDx48KIEYOBHpAbem5557Tq8ylXXgCm5KCtoC1HXrrbfKdHTKJQvbAOMgZR/S7elRuXJlmbovWBrD1atX0znnnGPsgpSKa9asMdbtCtECV4C51HcTwHFwJUNfKpBuCMDcZ599pqrksnbt2rRp0yZZhr5jx46VcJ06py1btkgYDUCXHgC5fvrpJ5kuUq9XZWgGhw2lLeoBWiFlY48ePVQzucS4f/zxRzFJfIcxFmzAGL755htq2LChqb1asQJX+rUEADVq1CiCG5c6F8BvANyQ3hAAmh4jR46kO+8MPkn9+uuvS03VPvgbeOONN9Sqr5ZeTuz75UTVxPcpp5xiui5+GR+PgxVgBVgBXQGv3pd/W7ZMpAMeLQ9Vr149kUpwbNDPZX08XI5MAXa3ikwvP7UGbAXoCsGpBf10ZXgsrAArwAqwAn5UgIErP14VHlNRU4CBq6J2RX12Pm65XKlJBuvpWev1dVVWS+yryljq5VDb/AJcHTkuoCGR7m/d9hwJC2ULWAiOUG4FHKaQ7m/15hzasCM/3R+ALj2QFg8OTzWFo1Ut4foER6t6VTPkEk5QFUtHBiVJIEqcA0AruFzhfPJdrvKBLIBJcLnCEqBVcfFP1on1SALHgYvX5t35LlfrBVC2XpwjwDU4eu0tcApTfeKY5UTavyoCiDq5doZ0ucoQx0a9GwFYDCBZRXEMQGpVyuYDXTi3eAQDV/FQufAYDFwVasElVoAVSF0FGLjyx7XH/S/gk/fffz/kgDIzMyXU065dO8K/9u3bE6CiaGPRokXUs2dP0+4ApwBQhQrAUPfffz/BhQgB+AguV2eeeabtbjpwpRogNSBSCzZr1kxVBSyRVhGQjx4vvvgi9e7dW68yyvEErtRB4Vb12GOPSQ1UnVriul5wwQUSIFN1atmiRQsJlQGKsosxY8bQM888Y9q0ZMkS6aBlqixYAST3ww8/GJtwLWbOnEmA1IIF3NUwKQ3ISgXgsenTp6tV09IKXKmN1113HY0fP57w+rSLP//8U0KDunMZoC6MF68du2Dgyk6VxNUxcJU47fnIrAArELkCXgBXhw4doiGDBxnOliMEOHzyyadEPjjeI6QCK1YspxEC1EY0b55HI0cK+32OpFKAXa6S6nLxYFkBVoAVYAUSqAADVwkUnw+dMgowcJUylzpxJ+oEulo42vxUu3W0Co4KV6+3U2W1xL6qjKVeDrXNL8AV0uNt358n0uLl0A5R9iIAPgFEQmpBuFwdFIAXAvMT+IcUeyWz0qhlw0w6U/xrXj9Tuj8Jdkhu92JMbveZI6TDeQFc++GPbFoh3K7WbsumI8IxDGAWLMMAltUWKQTrCKAMqQQrCDDKi0DKxToCXKshADbAXYDK4hEMXMVD5cJjMHBVqAWXWAFWIHUVYODKP9c+JydHpoObMGECZWdnOxoYHKJatmxJ3bp1k2kB4T7jNHA8uGatWrXK2OWaa66h559/3lgPVUDqOqSwU9G2bVsJUNkBNHbA1bBhw6SzldrfbokUgXBtwme2iiuvvDLA+UptizdwBVhs/vz54h41+D0poKcBAwaoIcolNAJsBs2CBb4TwRlg586dRpP33nuPOnbsaKyrwpw5c+j2229XqzIF5IIFCwiuU+ECafvatGljSsU4e/Zs+dqw7msHXFWrVk2mSSxbtqy1uWn9448/pptvvtlUh/SKSKVoFwxc2amSuDoGrhKnPR+ZFWAFIlfAC+BqorjvWbz4OzmYPn360iWXXhr5wHiPsAro7laArQBdcSSfAj16lDAGPXPWbKPMBVaAFWAFWAFWgBUoVICBq0ItuMQKeKUAA1deKcv9mhQIl1qw7/mlqe8FpUz76CsKjtLrULbW6+uqrJZ6e9SperulXucX4AquTH+Lf3BjQgo+LwLuUntFWsEtu/No1YZs2g2wS0zUADiqXl64MVVPp0YiBR7crfCvctk0KiPS7kkgy4sBedBnvtvVCdovoKutewCwwekql9Zty6G1Il3jgSN5wg3rBJ1aL5MaIt2fcO0qmeUNCAXACtrWqphODQTc5dVxrDIycGVVxNt1Bq681Zd7ZwVYgeRQgIEr/12n5cuXS7egzz//3DF4pc6iVatW9Mgjj9B5552nqoIu33zzTelSpRpUqVJFOg45dcw6cuSIBHW2b9+uuqC33nqLLrzwQmNdFazAVaNGjaSrUjBHJLUflkhTN3HiRKMKKQPnzp1rrOuFeANXkyZNoiuuuEIfQkDZOiY0gBuV1bkrYEdRcfXVV0ugS20DjHfjjTeqVblECkFAd5s3bzbq8Rq49957jfVwhZdffpkAwKlAasQvv/xSrRpLO+AKgB5AvXBx+PBhCVcB9FPxwQcfUIcOHdSqacnAlUmOhK8wcJXwS8ADYAVYgQgUcBu4+vjjj2i6uG9CdBCpcwcMuCeC0XBTpwrosNXVV+eI+4vCewanfUTTDmnw9ChKkFeizk1PLdizZy/q2auXLjGXWQFWgBVgBVgBVkAowMAVvwxYAe8VYODKe435CEIBJy5XoaArBUBZxbTW6+uqrJbYV5Wx1MuhtvkBuBLDpf9tyaE/RLo/uFAhnaAXgeMg9d6OfXm0bO1x2n3whHS1qlM5nU6qmUFnCFerMxpmUEnhdJUl/hWF2H/4BG0TaQV//zubfv7zOG3di3SKedRMuHcBuMpIz09p6MW5wqQAaRIBXJ1WJ5PKCMcrwGteBwNXXits7p+BK7MevMYKsAKpqQADV/697jt27JAp57766ivpIHT8uPOUKoCAACpVr1496AnCdQnuSypCOUepNtbl3XffTbNmzTKqBw8eTA888ICxrgpW4Oqmm26ip59+Wm0OuZwxYwbdc0/hxCZS5X3xxRe2+1jhJsBja9assW2rKgEX6bDSJ598Yus8ZQcbAY6Dw1OoOHjwIDUU6fP0eFykykEqwnDRr18/evfdd41mSLH40EMPGesobNiwgQDa6RHKOUpvp8rr16+n1q1bq1XKysoi1MFBTQ87DXA/Fep1pu8PMGzjxo1G1dtvvy3d2YwKrcDAlSaGD4oMXPngIvAQWAFWwLECbgJXK1eupMcfGy6PXaNGDXriybFUsmRJx2Phhs4V0IGrOXPy01Y73zuylgCCZs3KEC6qZthK7wXQFyJe4Jd+7GjL6rywf7BzA1DWrFme5+elpxZkl6toryjvxwqwAqwAK1CUFWDgqihfXT43vyjAwJVfrkQKjCOcy9WZjbLo2VvL2yqh4CjrRmu9vq7Kaol9VRlLvRxqW6KBK7hO5QjA6n8CtgJ0lScsmmTqO6sYLq2Lh9cFcHSC1ohjZYhcgU0EdFRPuC/VFo5WSHtXsUwxSk/DP5cOmOBuAJgdOX6C9ghnL6RqXCucrjYIx6uypYpRKQFAiVP1DIICXIX+awrgqqkArsqVTKNM83yTJ+owcOWJrEE7ZeAqqDS8gRVgBVJIAQaukuNiw03qu+++o0WLFtG3334rU+zB2ShU1K9fn5DGDZODdnHZZZfRjz/+aGx64oknSE1QGpVhCq+99hoNHTrUaNW7d2968cUXjXVVsAJX2GfgwIFqc8ilNRUd0uxBB7uIJ3BVq1YtWrZsmd0wAurQVk8TCXAJLlfhAvDaG2+8YTR78MEHadCgQcY6CtCiZ8+eRl2lSpUIf9eRBtIP7t2719jt559/JryG9LACVwCzAFDZpZHU91NlpEPExLUKQFV4HdoFA1d2qiSujoGrxGnPR2YFWIHIFVD3M7E66+Cze8jgQQYs/Mijw0SKu+aRD4j3CKuADlt56W41c2aGBK3CDkhr4OV4tMPEVFSgVTDIKljnXp4bxgLoChHr32Kw8XM9K8AKsAKsACuQzAowcJXMV4/HniwKMHCVLFeqCIwzFpcrBUdZZbDW6+uqrJbYV5Wx1MuhtiUauMrOJToqgKA/tmSLf/GxuQaEBOiqUpk0aiGcnmpWBGiVJuEg6zUoSuvwDVsj0gqu3Z5Lh0TaxqPZqPE+qpeHw5VIX1gqLS5pBRm48v6a6kdg4EpXg8usACuQqgowcJWcV37//v0SvFq4cKF0e4LLkV2ccsop9J///IfKlw98eMIKQV133XXUoEEDu26C1sE9Sne4atOmDX366acB7a3HQgo7HRIK2EGrgMOXnrLu1FNPpf/+979ai8JiPIGrZs2aEfR3ErVr1ybdoWz+/Pl0+umnh90VjlaAnFTYAVdWCKpu3boBaQfV/qGWAJx0p6/Zs2dTly5dTLtYjwXnLh3aMzW2WbnoootoyZIlxpapU6dS9+7djXW9wMCVrkbiywxcJf4a8AhYAVbAuQJuAVcvv/SSAJsXygNfe+119M8waYSdj5BbWhXQgSsv3K2CAUk7d+6kXbt2EZYIpNhGVK5c2SjLCvE/L+EkdYxolnYQmTofdW6hzguOVzg3L1IpsstVNFeU92EFWAFWgBVIFQUYuEqVK83nmUgFGLhKpPopeGwn0NXC0VUDlFFwlHWDtV5fV2W1xL6qjKVeDrUt0cDVMZFC8OAR4TglnJfwLx5RXKQLrFwmnaqVT6MaFdKotHB6yhSp74QZU5GPHfvzaJtIqbh5Ty7tOiBotzhE1XLpdLJI2VixdBqVLem9ygxcxeGiaodg4EoTg4usACuQsgowcFU0Lj3cr5Ci7+uvvw44IaSgA7ijB4Ctxo0b61WulIO5K1mBq48++ojatWvn6Jh+Ba7at29PH374oaNzsAJXP/30kyO4zQlw9eijj9Irr7ziaByRNLJzPLMCV3Cseu+99xx3y8CVY6l815CBK99dEh4QK8AKhFDADeBq3rzPaeqUKfIobc86S6RMfjDEEXlTLArosJUXUJPutKTGCcdYBSWpOrslQH88wKDCi/GpvqNZWmErnBO+34U7N+t54dgjRx53HbrStR82fDg1bdosmtPkfVgBVoAVYAVYgSKpAANXRfKy8kn5TAEGrnx2QYr6cJauy6aBkwtTSNidb9/zS1PfC0qZNik4ylQpVqz1+roqqyX2VWUs9XKobYkGruCyBLeptdtzaJ34F48oWyKNGlbPoOoCuCovXJfikeYuHufl5Bj7BNyG9ILQeuveeAFXadSkeqZ0ESsvUhl6HQxcea2wuX8Grsx68BorwAqkpgIMXBWt6z5z5ky67777KCen8N60YsWKtHTpUipVqvA+Hqnwunbt6snJ//nnnwGOWlbg6osvvqAzzzzT0fH9ClxBvxkzZjg6By+Bq+uvv57mzZvnaByRNLr99ttp9OjRpl2swFW3bt3o7bffNrUJtcLAVSh1/L2NgSt/Xx8eHSvACpgViBW4goPn0IeHyE4Bkz/55FgqZ+MWaj4qr0WrgA5cue1upQM/GB9AJMBWkYQVTvILdGWFrfC9btWqVZGcGlnPzQvoSrlcIS34sOGPRTQ+bswKsAKsACvAChRlBRi4KspXl8/NLwowcOWXK5FC43DicmWFrhQcZZXJWq+vq7JaYl9VxlIvh9rmB+BqnwCuAADFC7gqU6IYNayWIdyt0lMPuBJa7z6UD1xtixNwVaVcIXBVgYEr65950q8zcJX0l5BPgBVgBVxQgIErF0R0oYvc3FzavXu3TF+SlpYWU48vvvgiPfbYY6Y+3nnnHercubNR98svv9DFF19srKNQrVo103q0K4sWLQpIwVIUgatIYKNatWpRdna2ISnS8CEdX7hw4nB19dVXE1IUqihdujThX6zRu3dvGi6cCPRg4EpXo7Ac68R+YU/+LTFw5d9rwyNjBViBQAVifV8eMmQwrVu7Vnb8r0GDqVWrVoEH4RrXFFCfMV6ATAr2wWCjAZLUSSIlX4cOHdSqJ25QRucOClbYyqljl13XOnSFtIKArtwMHXpjlys3leW+WAFWgBVgBZJdAQaukv0K8viTQQEGrpLhKhXBMXYeuiPsWempBRUcZd3JWq+vq7JaYl9VxlIvh9qWaOBKphQ8KlIKbo1fSsGyJdOoMRyuRDrBcsLtKiPdqnzRXT8gtN4jgKu1In3jFpFWMB5RDSkFa2VQBaQUFLCb18EOV14rbO6fgSuzHrzGCrACqakAA1eJu+779u2jyy+/nLZv30579+6lvLw8mRIQcFIscfz4cZn65ODBg0Y348aNo759+xrre/bsoZNPPtlYRwFPxFeuXNlU59YKA1feAVdDhgyhyZMnG5fqlltuobFjxxrrbhYYuLJXM9aJffte/VWrJsORVglpSjlYAVaAFfCzArG8L0+ZPIngxIm46qqe1EsAyBzeKaC7W7kNXOlQUiywlTp7HbryAkxSx3Gy7NGjhNHMjXMDTIbzQ7h9HdCnGi+7XEENDlaAFWAFWAFWIF8BBq74lcAKeK8AA1fea8xHsFEgUpcrBUdZu7LW6+uqrJbYV5Wx1MuhtiUauMoWzM8RkVbwj83ZtHpLYdoWqxZurpcTaQRPromUgulUungxSo/NAMHNoXne1+HjIoXjkTyhdw5t2h0f4Kq6cBI7rXYmwd2qZBYDV55f5DgfgIGrOAvOh2MFWAFfKsDAVWIvS6NGjejAgQPGICZMmEA33nijsR5t4cILL6Rff/3V2P3+++8ngDl6AJyAq5aK2bNnU5cuXdSqq0sGrrwDrrXr8WcAAEAASURBVABb6de2TZs29Omnn7p6/VRnDFwpJczLWCb2zT35d42BK/9eGx4ZK8AKBCoQ7fvywoUL6JWXX5YdIvXx4CEPB3bONa4qoCYa0anb6QQV5IO+586di0XM4TWY5GSAboNk6pj//Oc/VdF1By/daWzmrNnGcbjACrACrAArwAqksgLqPqh58+Y0atSoVJaCz50V8EwBBq48k5Y7DqdAJNCVgqOsfVrr9XVVVkvsq8pY6uVQ2xINXOWdIAJ09T8BXP0hgKtcUSGMCTwLZJepKJyWAABVK59GmenFKM17Bsiz84m04+zcEwToauXGHFq/M4egv3i5eBLFhK7pQtyaFdOpae0MgrNYZhzcxNjhypPLGbRTBq6CSsMbWAFWIIUUYOAqsRcbaf2Q3k9F165dacaMGWo16mXHjh1p5cqVxv5WhytsuPTSS+mnn34y2jzyyCN07733GutuFhi48g64WrBgASH9n4qSJUvSX3/9RbGmplT96UsGrnQ1CsvRTuwX9uD/EgNX/r9GPEJWgBUoVCCa9+UNGzbQkMGDKCcnh8qUKUNPCrfIKlWqFnbKJdcVWLFiOY14/HHZr9uuSl5BSRisApMS5XLlBUiG8/LSwYvTCkJhDlaAFWAFWAFWwKwAA1dmPXiNFfBCAQauvFCV+3SsgNPUggqOsnZsrdfXVVktsa8qY6mXQ21LNHClzvl/Arb6n3BdOi7crgAFeREAgDIEYFWlbBo1q5splynEWhmSHhf6/t+GHPpre47U2ivADc5hWZlpVBvAVZ0M6SZmDMLDAgNXHopr0zUDVzaicBUrwAqknAIMXCX2klvTwWE08+fPp9NPPz3qgcG1CvsjtaCK999/n8477zy1KpcDBgygmTNnGnWdO3emd955x1h3Upg+fbp0DKhbty7Vq1ePGjduTN27dw/YlYEr74ArTBC3atXKpPknn3xCbdu2NdWFWjl27Bhdd911VKFCBXkd69SpQ4D/6tevb9otnsDVtGnT6MEHHzSOf8kll9Abb7xhrPupEM3Evp/G72QsDFw5UYnbsAKsgF8UiOZ9efiwYfS//62SpzDwvvupXbt2fjmdIjsOPZ2gV+5WO3fupG+//dZVDU899VSZvhudjhx5nABexSt0cMmNVILWcSsHLy9gMgWKcVpBq+q8zgqwAqwAK5CqCjBwlapXns87ngowcBVPtflYAQo4dbnqc37JgH1RoaAptVFfV2W11NujTtXbLfU6vwBXG3bl0nrxb8/BPDogUt55EQCAyot0gtXKpVPD6ukixZ2oSMHIEY5iawVstVHove9wHh0VkJsXUTyzGFUsk0Y1RUrB+lXS45JOEOfBwJUXVzN4nwxcBdfGyy1InZWVlUXFixf38jDcNyvACjhUgIErh0J51GzdunXUvn17ys0VNzkFgVR/H374IVWqVElVRbQcJiYMXy5Ih4Mdy5UrR8uWLZNuDXpHgK0AXenx1ltvEdIROgkAXQB9tm3bZjS/5ppr6PnnnzfWVSFewNX69eupdevW6rBUunRp6fZkVNgUzjjjDNq8ebOxJRisFAtsVKuWd8BVnngC4ayzzqK///7bOIeWLVvSZ5995tjlypqWEO5YP/zwAzVo0MDoE4VYNMD+F110ES1ZsgRFGVOnTrUF9LARTm/33HNPfkPxf7wu8fr0Y0Qzse/H8wg1pmQGrvBetX//fipfvjxlZmaGOk3exgqwAkVEgUjfl998Yxr95z//kWffvfs/6PobbigiSvj7NAonGfMkuOTmaBXc4wWUpDtBxRu40p27AJIBKHMzFHCFPt0+N04r6OaV4r5YAVaAFWAFioIChfdCnFKwKFxPPgd/KsDAlT+vS0qNygl01adLKep7QakAXRQYpTbo66qslmijyljq5VDb/AJc7RKg1Y79ebRpd65Y5orxq7N2Zwknq8wMkd5OwD81KqSJf+lUqngq+luRSNtItG1fLm3dmyf+5dJ+Abi5rreQtkxx4W5VOZ2ql08XbmLFKEvoH49g4CoeKhceg4GrQi28Kq1Zs0a6p6xYsYLgwLFp0yY54VVM2PbVrFlTTuQ2bNiQ8ITjtddeS2XLlvVqKNxvEiqwatUqmj17tmnkSHeGCdNoY9GiRYR/1oDzD16Hr776qmkTYJLLL7/cVBfrCqAIOLfocYOY1GnUqJFeFbcyA1dxkzrogW677Tb64IMPTNubN28uoSk8QR9JWMEZ7Dtw4EAaOnRoQDcAdQDALF261NgGlypM3pQoUcKoC1Z4+umn6cknnzRt/uqrr6hFixamOqzEC7jasWOH/FtWAwA4hM+ejIwMVRWwTHbgCif0+eefE95H9LBLI6lvV2Vodv7559PWrVtVFSHV5Ztvvmmsq0I8gSv8TeBvQwXARICIfoxIJ/b9eA7hxpQswNXBgwdp4cKF9MUXX8iUqdu3b6d9+/YZpwcXN0yU4+++W7du8rVfsWJFYzsXWIF4KfDrr7/SRx99ZBwO94HW93FjIxciViCS9+XvxH3Pc89NlMdg552IpY5pB/XZ4nY6Qa9doHDSiUorqANXc+fOjUl/u529hMn06zJs+HBxz97MbghcxwqwAqwAK8AKpIwCDFylzKXmE02gAgxcJVB8PnShAuFTC56gZ26tQGc2ND8pqqAp1ZO+rspqiTaqjKVeDrXNL8AVXJYOHT1Ba7YJ5yUBXeWItHcAg9wKpBIEYNW4WgbVFCnuUM5Md6v35OoHcNWhYydoz6E8Wiv0BuiWk4fXjDvnIRgQoa1wtyqdRo2rZ1CVcmlUUrhdibm6uAQDV3GR2TgIA1eGFK4WkJYIP/whzdTixYsd940JsH79+snJVQavHMsWsiFgpbPPPjsgJVTInXy0EU+Z9+3b1zQiTE4h1VU0gclX9KenWUM/SFMFSAVABpxxNm7caHSPNGm//PILARJ0K0aOHCkmdZ4zugNAhvcjJ4CLsZOLBQauXBQzyq7gcgXwac+ePaYe0tPTqU+fPnTjjTeKVCXNTdv0FbymAU0BsAFooAdAgu+++04CBnq9KuP1jb8Bdf+NerglvfTSSyHfO95++20CAKkHUu/ok8f6tngBV4cOHZIwr35spKHDOQaLogBc4dzgLgbgTQWcJIeLySxMOgd7DwOIggnD5cuXq93k0i4FJTbEE7iaN28eXX/99ca4cG/w9ddfR/0ZYHTkQSGSiX0PDh+XLtWkOBz4HnroobgcM5KDHDlyhCZOnEgvvvgiHT161PGueJ/t1asXwRmwatWqjvfjhqxArArAsQ9AtIpo0vqqfXkZqIDT92W4dA4ZPIgOHz4sXJiL05Njn6RatWoHdsg1riugpxN0G7jSoSQvXKAghnKC8iL1XiixdZeoZAauevbsRT3F5y8HK8AKsAKsACuQygowcJXKV5/PPV4KMHAVL6X5OCEVCO9ydULAVlkCujK7TeiTNjiAvq7KaqlvR52qt1vqdX4BrgD75AjASqW6Q1pBQFixQkCY28U/uC1VEABQIwEAVRMAUFpBfcgLV4Q3Cr5KQlcAruB0dfBoHmULyM0NvTOEuOVKFqOqInVjg2rpMo0j9I5XMHAVL6Xzj8PAlft6r169mm655RaCM1G0AfDq8ccfp+uuuy7aLlJ+v5UrV9K//vUv+v7772nBggUhQQ0/i+UmcBUMtgJo8MorrxjuN0899ZSEVnRdAJAAJHEjkDbuzDPPNDnJ4G9m7NixbnQfVR8MXEUlm+s7/fTTT3TllVcGBQUAGsLhp3r16hIKgIsL3mtx/dauXUs5OTkBYwJwA3AGAFWoADgFgEqPMmXK0KBBg6hjx4500kknyTRc2dnZEg4E1IC0e+q+HPsh/SHchwBj2EW8gCscG65gu3btMoZRsmRJ6tKliwTIdu/eLSEkHawoKsAVwL1zzz03ACrFJH7//v3p9NNPN8A76PDee+9JQEVPCQnRbr75ZsJ7oV3EE7jCaxvnowde/506dSLcK8Ch7YknntA3J6zsdGI/YQN04cB+Bq7w/olroKcGjfSUAfSNGDGCHYYiFY7bR60AA1dRS+doR6fvy6NHjaLff/9N9tn/rrvlfY+jA3CjmBXQgas5c5yDsk4OrANXXkBJGEOigSukEgRM5kUo9y63QTiMVQFj7CbnxZXjPlkBVoAVYAWSTQEGrpLtivF4k1EBBq6S8aoV0TGHhq7yrYX6dCltSi2oT8BAFn1dldVS3446VW+31Ov8Alxh/ICA9h3OI6QX3LAzl3YdiN15KV24KsHdqk7lDKolnK0qlhZOV1nFJISFY6Zq4BWXLeYU9wqXq+3C4WrDrhwBXQlXMUBXMYgincSEvnWriNSNIpVg+VIilaBwt4ojb0UMXMVwAaPYlYGrKEQLscucOXPovvvuI7iL2EVmZibVrl2bkK4KTxBjYlifENf3gRMHHIDg1sERmQKjxKQB3B0UfMHAFcm0QnbOVldffbWEDeBuoQKpL9u0aSMn8lUdHIbGjx+vVmNa4nr07t3b1Mf8+fMlCGGqjOMKA1dxFDvMoeDoAzBGT38VZpegmwEZASa89NJLg7ZRGwDf4P0WDnJ2kZWVRdWqVaMtW7aI+63cgCalS5eWYBdScAaLeAJX+HsHsBks3nnnHQKEpKKoAFc4nxkzZtD9999vfAaoc1RLAEtw/wn2GuvRo4d83SAVo13EE7gCUIXXDV6fdoExrl+/ngAWJjqcTuwnepyxHN+vwBUg8+7duwd9TeOcAYQCssRrCRPU6jcFOz2ef/55vv+0E4brXFeAgSvXJTV16OR9eebMGfSB+A6LuEik0r355ltMffCKtwoUTjDm0ciRx109mJ66ziuHKwUlJcrhyivgSk8p6CVwhQs+c9ZsV687d8YKsAKsACvACiSbAoX3Q80Jv+lzsAKsgPsKMHDlvqbcYwwKBE8tWIi46NCV9UdMfV2V1RLDUmUs9XKobX4CrjDObDH/hHR3m3blylR3B4Tz0jHhdBVpyjvMb8BpCakDy5ZMo9qV0ql6+TTKyigm6nEkDgBux3NO0L5DJ2Qax90CvjoEpysBYkHvSAJgG9IIlhFaVyiVRnWE3pXKIm2jSCUYT9pKDJqBq0iuXOxtGbiKXUPVg50rELaVK1dOOl7dcMMNMgWQDrZg+4EDBwyHDT2NG7ahLdK8XX755VjlcKgAnhTdsWOH0TrVgSsALHBrsaYRvOmmm6STlR1UgNRCelo2pGTD+wWgk1jjjjvukGCK6gegx5dffqlWE7Jk4Cohsgc9KCZP4PI3a9Ys4544aOMgG7p16yZd05AS02kAwnnggQcIqUgjicaNG9OECRPonHPOCblbPIErvAdedtllEuy1G9To0aPp9ttvNzYVJeAKJ7Vo0SIJ7m3fvt04x3AFpFTFZ/WYMWOkm1mw9vEErjAGODXiPTlYiji8Vzdr1izYcONW72RiP26D8ehAfgSuAPnDgXLr1q2mswYgir9xTIbXrFnTBOUh9fWmTZvos88+oxdeeMF0z4RO8Lcwc+ZM6aRm6pRXWAGXFWDgymVBLd2Fe1/+WTjjjR8/Tu7VpEkTGjV6jKUHXvVaAfW54gXUE0/gyovxh9JeOUShjRfuXXCKVY61AOEAlLkZ+rUZJlJfN22a+Ps4N8+P+2IFWAFWgBVgBSJRgIGrSNTitqxAdAowcBWdbryXRwoEd7kywy3P3FpBpBjMDJggUhAVhqfKammtU/V2S73Ob8AVUtohteBhAV3tEQDQpj25tEc4Xh0+lke5EXw/LS5cleBkVaNCOtUUzlZlBHhVUvwD+4MUgxz5Cijo6pBwt9opHMU2C733HzlBR4Te5ldlaMUAtpUuIcA2oXV14WyF9ayMfK3jLTcDV6GvldtbGbhyR1FM7GIyVL0/o1c4VGHivl+/fhK6CnckwDBPP/20nLTX2wJw+eCDD6ht27Z6NZdDKMDAVaE4wWCr2267jQBc4HVqF3jNoY0eb7zxBl1yySV6VcTl/fv3SzBABwfGjRtHcONJZDBwlUj1gx972bJl9PHHH0uHtuXLlwdvWLAFDi4XC3cGOBR17tw5bPtgDZYuXSrfjwFrAkwIFpichJMS0iBaYVq7feIJXOH4cKu76667JLCjfz5hmzVlXlEDrnCOR44ckdAy3rv++usvVNkG3Cfh9geHSjhQhot4A1cYD16LQ4YMoTVr1gQMD2C2cpgI2BjHinAT+3EcimeHUhPjmIR96KGHPDtOJB2//PLLNGzYMNMu+PvGk8FOIGl8HiMt5UsvvWTqA+k34T7JwQp4qQAc1/QHTpDKt1GjRl4eMqX6DvW+vGfPHhoyeBDt3btXfh944smx1KBBg5TSJ9Enu2LFchohHjBAeAEs6VCPF05QXrtAhbo+erpEL9y7dODK7VSPOC/92jBwFepK8zZWgBVgBViBVFCAgatUuMp8jolWgIGrRF8BPn6AAvbQVSDasmBUVdPEOzrSJzpUWS317ahT9XZLvc5vwJU8D/E/ZFo5fPyETC+4T4BXB4Xz0tHsfEemHJH2DvAVYCEAWpjrhcOSclmCi1VpAfyUEQBQpTL4J1ythNMStnMEKgAdoekBAV0hjeMBAVxB72PC/QoOWNBa6Y294ViVr3c+VJUlna2Ek5jQu3LZNJFGME1uj7ezlTozBq6UEvFZMnAVu874sbpjx44mZwFALHA7gVNGpPHwww/TpEmTTLt16dIlYscVUwcptsLAVf4FDwZbDRgwIGBy1voSAQCIyVY9jRXSFU2dOtXaNKL16dOnS6hB7YSUbwBpypYtq6oSsmTgKiGyR3RQpPJD2iykYcXrEkukDYWLC1LE1alTh1q2bCmdWSLqOERjwFZwFwKsg2NiHY5Z9evXl5OStWrVIjuHuBBdJmTTtm3bZNo5TKpicgxaYRkMuEzIID0+6N9//00/CScPTDjitVOhQgXjOmKCGZP8yRBr166V9xuAyeBahDTF5cuX98XQQ03s+2KALgzCb8AVUk4CltTdrbp27Ur4rHUCgSpJ8PsCXC/heKUHXAbPP/98vYrLrAArkEQKhHpfHvfUU/TLLz/Ls7ntttvpAvHewRFfBd4V6Z3fffcdeVAvXJTQsXKC8gK40qEkr8Yf7IrowBK+x61atSpY06jqO3ToIO+VvUyV2KNHCTm2nj17UU/x8B4HK8AKsAKsACuQqgowcJWqV57PO54KMHAVT7X5WI4VGDhlHy1de1xrHwhcndkwiybcUk5rkzrAlTppgEDiN2ABWgEAOiGcl/Jo3+ETEsRCHSAhbE8XZA/clEoIR6syJYpReZHWrlxJpBIEZJWf0g5QFkdwBfAKPCG0hOZI6Qjoap/QG5ofFeDbMQG7qTSDgNeKF+hdTkBW5Uvl615aaJ8uhC4mwLZEys3AVfDr7MUWBq5iV/WWW26hjz76yNTRxIkT6brrrjPVOV3BpBf2taZYg4sWQCKO8AowcEUUDLaCI8e//vWv8CKKFkOHDqXXXnvNaFu8eHFasWKFI8c2YydLAekxf/jhB6P2mmuuoeeff95YT1TBL8DVr7/+KqEhL3TQn6LH60ClyfDiWNwnK8AKpJ4CoSb246XGvn37BAQpvkeWNX8Pd+v4fgOu4GLXqlUr0+n95z//obPOOstU52QFDpToCxqqgHur1flKbeMlK8AK+F+BYO/L77//Hs0WQCWii4Aq77jjTv+fTAJG+MUX8+jss86mch6BzTpw5YWLEiRTwBXKbjtB6e6aXo0f4w4W6tzchsl0kMwL5zF1Pmr8+O1i2PDHVDUvWQFWgBVgBViBlFOAgauUu+R8wglQgIGrBIjOh3SmQOehO7SGgcAVNvbpUor6nF/KaKecqVChympprVP1dku9zo8OV8YJFxRyhNsVnJYAWR0R8E+2WM8ugK3gcJVW4G6lQKCSWWlUPJPEv/wUgtb+4r2OdH0Axg6JNH0YP84H405PF5CYcOMqJUAlgGJw5PKDC1d2Dkl3K4wVmkNrjBkwFgLOVdA6E+MXGpcUoBtcxQC9+SEYuIrvVWDgKja9kdqnXbt2pk7OPfdcmjNnjqku0hWk0YJDgR5Id/TCCy/oVVwOokCqA1eff/45AQSES5UejzzyCN177716Vcgy4KpOnTqZ2sQCE65bty5gEhjp4s4++2zTMRKx4hfgaq5I5Thv3ufU/pxz6Jz251Cjxo1dk4OBK9ek5I5YAVbARoFgE/s2TT2rAix0x+23ic+as+X7aPv27V09lt+Aq4ULF8qU1uokkUIQn7VOUgmqffQlUhHic1kFpxVUSvCSFUhOBezel38T3zPHjBktTwhpdJFKMBJHvORUIrpRA7iaItL2tm3btuAz5RxX3UHjAVzpTlBQYe7cudGJYdkrXlCS5bCmVT2toJsuV/ECyRi4Ml1OXmEFWAFWgBVIYQUYuErhi8+nHjcFGLiKm9R8oEgVMKcWLCBZrJ0IKqfP+aUN6EqBUmimympprVP1dku9LhmAK6ssybIOqAqQ0vZ9ubRpd55c7tovHKME0ARnLkBhcIeqVj6dalZMp1oV0yhLAEwAmjiiV4CBq+i1i2ZPBq6iUa1wn6dEKoZx48YVVohStM4Cpk7EyhVXXCGfQkU9UlYhZQwmwayTaEiT9Mcffxi7t2nThipWrCg/Z9D+HZGqYOPGjTINFkAkpDlECqJQcfToUfrmm2/oq6++IvS/Y8cO8b6XJ231kUoL6WUA4iAdXCyhnkZFyq5NmzbJlF1wWEC/SJGENGFwaQDE1rBhw5CHwqRjdraw8xPRr18/k0PDE088IVNHqQ6gJdKQhQqcL9yGvvjiC5miYPv27XTo0CGpQY0aNeSYAMVVrVo1VDcxb8PrqW/fvqZ+MC6kBLOLYLDVqFGjxNPrd9jtErLuwgsvlDqoRrEAhWPHjqXx48errujkk082XuNGZYIKfgKuZsx421ChYaNGErwCgIUUcLEEA1exqMf7sgKsQDgF7Cb2w+3j9nYFXKl+S5cuTeeI90+8hzZt2kxVR730G3D16aefylSA6oSQJhRpJ6NNkfn6668THBBVVK5cWaZytUs/iuPgwQMVuLfEfabT0O/bsA8eYLBLL4x7r++++87oFveDTZo0MdaR8nXmzJmEz3Hc72ZmZsr73GbNmknH2GD3e4DSP/zwQ8I44BR2+PBhaiwg55NOOknCFUgXHi78PDa7sefm5hIe6oDLDb4DIi060uXi/hlpVnHvDfdL/M3gH5xNwwXu23XnUmjYSNy7IHC/OmXKFHlt8N0E2l511VUmVzbc/wPwV4HXnNW1DdfHzXRhAEWQHjhc+OW7QLhxhtpufV/Ga3bI4EGE7zWIESNHyfvxUH2k8rYvxffAyZMnGRLgO6p6KKK5SL0eaxROLuYRUvJ5FTqYpL5/x3IsHbZCP4lwt1LjV9AS1t1w8NLPzUt3K4xXH/vMWbNRxcEKsAKsACvACqSkAoX3RM0Jv19zsAKsgPsKMHDlvqbco4sKFEJXwYErHO6MRln0zC3lDcgKdTo0hXWEXqeXQ21j4EpK5+r/cgRMdUw4Q23bm0d/78ihrWIJ6OqwSNUH16jcgtR9GcKZC4BVKeEQValsGlUX4FW9KulUv2q6hLHgGsURuQIMXEWuWSx7MHAVi3okYSA4CajA5IRbT41icmnBggVyAgpP1ZYrZ58eB65Xjz/+uBqCPD4mre6++24JWxkbCgqYiBoyZAgNGDDAukk6Ik0WT/E+88wztHfv3oDtegUmZe677z667bbbAiAwvZ1d+bfffpPgzWeffWZ89tm10+sA/sCh6bTTTtOrjTLgHUwcOYmpU6dS9+7dgzb95JNPxI/eI+nPP/8M2gYb8DQ4ALZBgwZ5Bl5FAlzZwVaYJAUU2KdPn5DnEmzjtGnT6MEHHzQ2A/5bunQp1axZ06hzUsB9TevWreWkpmqP123//v3VakKXfgWudFHOPPPM/EmeczrICWV9m5MyA1dOVOI2rAArEK0C1on9aPuJZT8rcKX3VatWLWovnAMxWR4MWtbb25X9BlzZOVHic/vSSy+1G37YuoMHD8rPacAxlSpVCnl/Z33oAOmw4YLpNAD2APZRMX/+fIKjljXwUEGHDh2M6oEDB8qUxwCm7rnnHnr//feD3kviIQVA+EiRrENj7733Hj388MOm4xsHKCggBfKTTz4p4X/rNrXu57GpMWKJBykA0yGFswJt9O12ZTxQcNddd9Gdd94Z0v0IUBXu0VUgbTWgPbwOsVS/a6ntWF588cX05ptvyqq33nqLcE1VdO7cOeD7C6CtwYMHqyYxL0ePHk233357yH789F0g5EDDbLS+L0989llavDgfYOzTpy9dEuV7RZjDFpnNX34pgKtJhcCVfmJ4MAifJ/hcwQNJ0YT6TPEa7MHYdLgnFjcoPACivyfHY+yhtLU6eMUCXemwVfPm3kJwOCcdhGPgKtRV5m2sACvACrACRV0BBq6K+hXm8/ODAgxc+eEq8BhCKpAPXR20bwOLpIIAdDXh5sLJcvXDk1qimSpjqZdDbWPgqkBglxZwtAJYtetgHq3alEO/rj1O2/bl0R6xHipKi3SClUqnUfN6mdSqUQZVFgBWhVJp4oddkv9C7cvbzAowcGXWw+s1Bq6iV3jJkiV00UUXmTp49NFH5eSPqdLjFTvgCq5ModIPvvjii9S7d2/TyDZv3kzXXHONdDIwbQiz0rx5c5oxYwbB9clJPP3003ICy0lba5tSpUrJCaMuXbpYN8mns2MFrjB5B1At0pSQmJjExJEXqfGcAld2sBXgKEx+4rpGG3AuwDU+cuSI0cXw4cOlTkaFgwJ+/IZrmwpMggK8g5OBHyIZgCulE6BJ9XT9mS1bquqwSwauwkrEDVgBViAGBawT+zF0FfWuoYArvdNTTz2tAGA9x9ZVSW+rl9XkOGAh3QlKbxPPMkAauEodOHDAOCwgADiUYullJBK4uv/++yVIjgcTnIQOgwHcAcDjJACdLV68WMJndu3tgCu/jE2Nd/369dSzZ0+ZalLVRbKEmyvgKavDrurDDrgCIH799dcbv2mptmqpXw+/AVd+/C6gdItmqb8vlyhZgqYXgG4dhHvwgAH3RNNlSu0TCrjShYB7G8Ar3J/je6HTUJ8p8YCWrGBSNNCVDiThHOMxbida6uAS2kdzboDIdDdhOI4BuvIy9HEzcOWl0tw3K8AKsAKsgN8VYODK71eIx1cUFGDgqihcxRQ4h4GT99LSdTb2zxpwBRkERiWcrirQGQ0zjR+fFFgltxe0Z+AqMS8aOFcdFc5W63fm0pK12dLdaruAreBqBcerUJGRXky4WhWToFW18mnUSlxjwFclRF1WRqg9eZtVAQaurIp4u87AVfT6WkEn9ATHJrj4xDOs4xgxYoR0vELaELtAeh04IgBeUoFUHZdddhlt2bJFVcklHJyQtgWTeUgrgkmV1atXy/SCekM4RcybNy+syxOcAgBcWQPpYfAP0BbSdyCNIVLEWMeD/fAEMVKXYGx6wF1MuXLt2rXLNEZMmOntnxVPd+tP46OfnJwcmeZET1uj+sf5ITUNxrd8+XIJCmFCRg/og5Q2SLnnZjgBrvC6u/XWW6VDmTp2RkYGvfTSS9SjRw9VFfUSENqsWbOM/aEFUvBEEnBUgz4q4DAGpzG/RDIBV7pmlStXMdJlqRQ++na9zMCVrgaXWQFWwG0F9In9nr16ud29o/6cAld6Z4ClMUnerl17vdq2rCbH/QJcYZCAoPF5rwfue+BCeu2118o0cfo2t8qJAq5wv/P777/Tjz/+aJwK0uHhPvLYsWPSndR6j4aGcMLCPrgX1QOp5QB/4zsR7gWtccsttxBSItuFFbjy09gwXtxHd+vWjbZt22YaPtKiwU0M97V4rWA77r2RIhIaWgPOWMHgfStwBedSuP0iVWCwQCrH9u3z/968AK5wz4/7cvzDewK+W+iBY1q/B2C7X78L6GOPtKzelzsJ57BFBffuuO5PPjmWSojXAUdoBb788kvhcPVa6EaWra1atTaAXv37p6WZXFWfKfECl6zQFQbhBE4CiITPPR1IiteY7XSzq9PhJafnhXZWxy7UxQO2wnH06zFMfJa7kfoY/XKwAqwAK8AKsALJpgADV8l2xXi8yagAA1fJeNVScMwApO6bsi8QurIBriAPoKsWDfIpHAau/POCAWyFNIIrNubQf1ceE2kE8yg3F5ic80gXaQbT04rRuadl0dknZYk0g8LpSjhfcThXgIEr51q50ZKBq+hVRHq7V1991egAkxdr164lgC7xDCtwBaDq0KFDcgj16tWjf/7zn3IiCQAT4J0LLrhAphTRx4h0c0idoQdSqWAyDSlF9EC6GaTiQOpBPTCp99xzz+lVpvKqVasIzlT6ZBaegEdaOcBSdoHxjhkzhjChpQecCf7xj3/oVaYyALEdO3YYdXBAgEtTqHjttddkyhm9DfrBJFOLFi30asrOzpbpTpA2RZ+YApz29ddfR5XuzXQAbSUccGUHW8GFAOcDiM6NgLuDVW+cZ7D0jtZjHj58WEJ76nWJ7bNnz5avB2vbRK0nK3Cl6wXgSjpfiZSDds5hDFzpanGZFWAF3FZATez37NmLkgm4UjqUKVNGOpTgnuQ08flvF2py3E/AFWAZ3NtZoRqMH/cDHTt2lCncAIQ3btzY7rSiqksUcIXUgOo3FNyHIE02HGfh6onAfeaECRMk4K+DNkgpCQAJ+0IXpEsGIKVSduP+dqEAUu69916Cu6cKABNw6bTTzgpc+WlsGP8dd9whQTN1LtAIDl99+/a1deFBukE8GAHXVtzrqoCrDe779LSMapsVuNK/h+BvCu5agOG2bt0q9cW94M8//2z05QS4wjW1g+jUGNQS48O1VZCLtW+0Uykp1T760q/fBfQxRlpW78t4navX9SOPDgv7vSjS4xTV9l8J4GpShMCV0gLAn3K9OuOMM1S1aak+U+IJLwHymTUrQ8I++mB27txJeGgJgTJAJPV9QgetsD2e48XxnIYVusJ+1vNCnTo363nB0Qrn5rWzFcaAYOAqXwf+PyvACrACrAArwMAVvwZYAe8VYODKe435CC4ooH7wC4CuggBXOGSfzqXopvNLGT8Wok71g6VeDrWNUwpCHXcCaQR/+jObVmzIoXXbc+jg0TxxHSLrW6YQFLs0rJ5BTWpkUJsmmXIpMgtyOFSAgSuHQrnUjIGr6IXED9h4glsFJhP0p+1VvddLK3CljofJJzgIYeJBBSYr8KS3DlEtWrRIToaoNlgCnAJAFSreeecdQtoUpLNBYJIDLleAqOwCDgEfffSRsalBgwaENHh4sj5UYHIAk5/6RCLWde2t+0cKXO3evZvOOussqY3q64YbbpDAGdK3BQukxMOkFRzCVMBhrF+/fmo15mUo4MoOtsIBy5cvLye14MzlVsABBEChCkxIAjp0EoCr7rrrLqMp3CQw2aYmR40NCSwUBeBKlw+pBvF3co6ArxQEysCVrhCXWQFWwG0F1MR+sgJXuh6Ac/D+CYi1du3axiY1Oe4n4AqDw/0InCMBOIcKpBmEs1C7du3kZwRAGjuIJlQfaluigCt1fKR/AhhUtmxZVWVaWh+MUBsBAeEhg2DQOCB93APrTrHBHJ6swJU6hh/GtnLlSgnbqTFhOXLkSLrzzjv1Ktuy3XeLDz74gHBe1rACV2o77kGR4twKNGzcuJH0+1MrFNVZODHhO0asgeuI1IX6gx54CGXSpEm2r3k/fxeIRQv1vqz6uPba6+ifWopvVc9LewW++koAV+IhllijSpWq8j0Xnyn4zUCF+kxJBMBkByepcQVbxhtICjaOcPXRnFsirgEDV+GuJG9nBVgBVoAVSBUFGLhKlSvN55lIBRi4SqT6fGzHCig4CjuYoCsLrWPyShLbzmiURTd1LklnNMifzFX9MHDlWHpXGuIywd1q4+48WvD7MfrfphzafySPsoW7VbRRrlQaVSuXTl1Oz5LXt0RGMcpIj7a31NqPgav4Xm8GrqLXG+5BOmDVtm3bAJeo6Ht3vqfdpAggpm+++cYEVtn1iEkIuE7BfUoFUoZgYslJIDUf3K5UQAMAQtbJO0BZJ598Mh05ckQ1pX//+98EFy0nYX3iHMAY0iIGi0iBK7gcTJs2zegOY50/f75MRWJUBingNaA7SeEJcqQ8tE4wBdk9bHUw4Ap/u4DYdAcCvTNcC6RsUbCNvi2aMiA8TNSpwGTZkiVLAq612q4vr7rqKumMoOoGDRoknSXUuh+WOnDVslUrqijSEyUiVEoft44NYBDQAOCrzKxMGiEc5RAPPfSQTA3i1nG4H1aAFWAF1MQ+JpP1CeV4KgOwHPc/bgbAnPYF76P/79ZbZNd+A64wKNwXAW7G/YHTwP0UAJdLLrmEunbtSnBrdRqJBK5KlCgh77UAxgULwPBI861+Y1HtAGIBGg8V1ocE8IABnLSsYQdc+WVs48ePN6VCxOsYLlVOAt8P4Ayr3G6wz7hx4+RDBtb97YArfA949913A4Av675Y9wK4wt8C7s3hWqaiTZs2BGgMrkN24efvAnbjdVqn3pfRvor4e5eOv/JnLu3XSX3d+AmsYLuj9fxGBd2IIwXpG4MoaKR1G7y91qiw7/xOCtfzG3m1vnv3rpDpMTGaSCPfkbYDVa1SmfBdGpEI2EeNG3DS8uVpAY5Xartye4qn85M6dixLnBcCbl7BItEAmQ5c1RcPo5UuVSrYUH1ZrzuK+3KAPChWwCUFkO6zU+dOnPbTJT25G1bATgEGruxU4TpWwF0FGLhyV0/uzSMFrD/ivf7VYZq2QKRzCgNc5Q9HgFcNBXjVpRS1qJ//RZCBK48uVJBuc3KJtu/PozVbc2jRimP01/Zcys0TPxLl/3YTZK/Q1UgtWKZEGnVsVpxaNcykaiK1YOnixULvxFulAgxcxfeFwMBV9HpjEmf9+vVGB3CUmj59urEer4IdcIVJFqQJDBdwBsAkkgpAQoCFVHoVVR9sCYAKExhIQaICEycXXnihWpVLwCxIv4e0hkjnUrFiRfr9999N7lumHSwrVhcuuHZt2rTJ0qpwNRLgavXq1YQ0O3ramU8//VSeV2GPoUtww4Jbl4r+/fvLVIlqPZalHXD1xBNP0LBhw4LCVup4odKmqDZOl3AYg3uZ7hQAlzGAPKECTgatBMCk7pXgagVQS3cMCbV/vLbpwFW8jhnv48D5DA53CAau4q0+H48VKPoK6BP7Rf1s/QhcQXNA2HDwgcMpAN5IAk5RN954owSig7lG6f0lEriCuygAoHBhvR+sUaMG/fLLL2HvP62QOcDxV155JeBwdsCVX8aGByJwb4rXAZzPnLpbqZO0PlgSDFSzA6569OghU1urvkIt3Qau8D0D38mwVAFnVbjwBnsYwu/fBdR5RLNMpfflaPTxyz4jRx6PWxq7UOcMAEgPBVvpdcla9uO56cBVsurK42YFUkUBPKTw/Asvpsrp8nmyAnFXgIGruEvOB0xBBRi4SsGLnoynrCYR9bFL6Gp+4RN12KY956XBWIVUz41IMyjAKwaudCW9Lx/POUFrtuXSqo05IqXgcdq8RxBYLkSJzGIipWC+w1Xj6ulUqYz5xwsXDlEku2DgKr6XlYGr6PUGaKRPZgEywqRBvMMOuPryyy/pjDPOCDuUAQMG0MyZM412V155Jb366qvGupPC3XffLZ4cnWU0HTx4MD3wwAPGurVw7Ngx2rNnD2HSy2msWbNGpr9R7fHkvA55qXq1tE6wIa2IfKJbNdCWOH/ooALpdiJxh8B+M2bMoHvuuUd1Qeeddx69//77xnosBTvgCuev33vA0eyZZ56R54EUjCoANyGdX6dOnVRVTEtMxCKNoQqsT5gwQa3aLq0uaBdccIHpNWe7UwIqdeAKE81uOYNFeipwg9Od4CLd3649AEe4XNWoWYOmTJ4smzBwZacU17ECrEAsCqiJfTj8ROKUFMsxrfsCnlZgqXVbLOtwjUQqqOcmTpTd+BW4UueIewTA6kjljHsgPfWxahNsWa1aNUJ6ZEBGoSKRwBWcTwH1hAvccyDdogqn9+p4gOK+++5Tu8lUenBHsoYdcOWXseljhQsJ7m3wt+k07rjjDtO9bDB3Ujvg6nHhpomHD5yEm8AVHK0AiukuuHiIBPfSSJ8ZLPz+XSDYuJ3Uq/dltE1PTxevg3Jiqf0uJb5TmB4LjGhd7Jn/X8FQRF/mziJbj/HY1rFEum43dnye6GntnWgers1JwkkZD6xUqVyZnn76adncL8BVuLHzdncV0IGrZHO4wucKO1y5+3rg3vytQCnhQDf136/7e5A8OlYgiRVg4CqJLx4PPWkUYOAqaS5Vag9Un/TUlXj9q0M0bb5wuiqIcMCVslRqIRyRbuxUklqIVIOqb32pl+f/8CvN//FXeYSJD4e2xlfj4KVZgaPHT9Cyv7Pp9/XZErraeSDP3CDKtSyRRvC02hnUtG6GTCtYowLnFHQiJQNXTlRyrw0DV9Freemll9JPP/1kdAAAC85I8Q474Grt2rVyYiXcWKxPr8M5Sf9hPtz+2G5N99e7d2968UX3nvzavXs3wUkJDll64MdvAEV2EQlwNWbMGAkrqX6Q5hDpDiOJ5cuXU+fOnY1d4N60dOlSYz2Wgh1wpfdXWfxYD7gL5/zee+/RnXfeqW8mTJwuXLgwbHpJ005BVgBbAbJSUUGk3cO5w3EsWLRr144AzKmIJJWk2iceSx24GjZ8eMLs4ueKCd0ZM96O+ZQxqQY4ABM6rVq1lv2tWLGcUwrGrCx3wAqwAsEUUPcPPXv2op69egVr5mk9JsfvuP02V45x8smnyPdQvJfCIRBxzdW95dLvwJUcpPY/fA4DvEK6xcWLFxPurcJFMEcjtV8igSvcfzdo0EANJeiyl3gd4h5IBV6juNcNF7ivAnCkon379jJNs1pXSzvgyi9jU2OMdAloES60eIDi559/NnbHwxSos4YdcAUH3Ysvvtja1HbdLeAKDqzXXXedfJ2rAwGex0MR+j262qYv/f5dQB9rpGX1vqz2w9/NoMFDpNuwquNlcAXmf/WV+K4b2cNIdr3h+5hK8V2vfn2jifpMSWRKQWMwXIi7Ajpwlcjvn9Gc+LvvvCNSx74jd505a3Y0XfA+rEBSKHDLzX2lUygDV0lxuXiQSawAA1dJfPF46EmjAANXSXOpUnugCoCyqoD6afNFesEC6MopcCU9rwry2QG6urFzSTo9SLpBBq6sqke+fkQAVz+vyabfBHT1p0gruOegO8BVZnoxalIznU6tnUltGmdS7UoMXDm5OgxcOVHJvTYMXEWv5c0330wff/yx0UHDhg3pxx9/NNbjVbACVwBwVq1a5ejwp512Gu3cudNoi4kKJxNYxg6igEk83eEqWvAMEyVwDEN/f/75p/yHiR6ci93n7NatW+WT2vpYVDkS4AoTEQC6VCC9YMeOHdWqoyVS+OipbeBABTeJ4sWLO9o/VKNQwBV+vMekICZ+VVgdCVDfpUsXeY0wrlgC1wjOabq72Ouvvy7dBOz6xaQjwEQVsGFftmwZZWZmqirfLIsKcNWiRQsBWnWQkID19cfAlW9ebjwQVqBIKqAm9pMZuIL7poJV69atF3Cd1OR4sgFX+ongnmrlypX09ddf01cCJvj222+DpiiG+8pNN92k726UEwlc4T5RQXDGgGwKeAgAoJkKp2n1cF+oXs/YNxLgyi9jU+ccagnwDvfd6h9eF99//72tSxxSkA8ZMiSgOzvgCmCffm8asJNW4RZwhTTaVqdhpynW/f5dQJMr4qJ6HTdu0oTWiL8bBL6zArrCgxMcoRWYP18AVxG6P6secR+e/3nSgXB/bhfqM4WBKzt1in4dA1dF/xrzGSa/AgxcJf815DNIDgUYuEqO68SjTG4FGLhK7uuXMqO3mwjGyav6peuyadqCw7R07bFCTQqAKiQaNKKgTgeujO1iW4uGWQV9inK9DGreIIO+Xjqftq7fJLtghytDyYgKh4+doO9XZ9Oyv7Lprx05tO+QO8BVhuCrGlTLoFMEcNXupEyqW5mBKycXhoErJyq514aBq+i1xFPeU6ZMMTrAxA8mWeIdVuAKaW8++eSTsMNA6rnGjRuHbRdpA6S3A7wSLgBTzZs3T6Z6gSMXACUAPU7DLeDq/PPPl0/yOz2u03aYyATQFmsEA65q1qwpYasmYgJFD7h74En+jRs36tX06KOPmtIemjZGsILJyueee87YI5Qj2IMPPkjTpk0z2iJ147Bhw4x1PxWSGbhCCopz2p8jJ3UA4QULBq6CKcP1rAAr4IYCamI/2YArpD+EGyBg1WDph5U+anI8mYErdS5quXfvXpo6dSq98sorMuWzqscSaejg2Im0bNZIFHAFeBz3gMFcTvVxWoErpDm+/vrr9Sa25WiBKz+NzXpiBw4coDlz5kjXKtzzrFu3LuB6W/fR150CV7guuKcP5X6q9+sGcIW02nCp0gMpDZHa0En4/buAk3MI1kZ/Xz5wYD99/vnnsmnDRo2kY1n58gxdBdMO9fPnzxfA1SuhmgRsa9myFZ3TAS6zHYI+HKR2Up8pDFwpRVJrycBVal1vPtvkVICBq+S8bjzq5FOAgavku2Y84uRTgIGr5LtmKTliBVZZT95aL1MMLihIMRgBcJWbtVp0fYKyy+ZPoOfJdQBdRPWyG1G9nIby0AxcWa+As/VCh6sc4XCV7bLDVQadVieTWjfKYIcrZ5eDGLhyKJRLzRi4il7IiRMn0qhRo0wdYFIK6eTiGVbg6h//+IcJBAs2FjgNde3aNdjmmOqDPd2Pz0WkXUTKwUjcwDDRB0BMD7eAKzh6HTpUmP5XP0YsZYBGurtTtH3ZAVd16tSRk2YYu13AreLKK68kpIVRgbQqH374IQHIiyXggoA0gSowobZixYoAp4ljx45Rs2bNTC4JcE3wAvJTY4llmWzAVcWKFQ0XliZNTnJ06gxcOZKJG7ECrECUCugT+8mQUrBNm7YFoNU55NQBUk2OFyXgSl1uuB0BRtLTyGHb0KFDCe5B1kgUcAVAbv369dbh2K7HG7jy09iUIFu2bJHpv994442Ae2nVxrpEWuISJUqY7o+dAleROO3iuLECV++++y7169fPdAq4/0YKaydQHnb0+3cB08lFuGJ9X54qHhaaNy8fumrUqLFwuhoccA8f4SGKdPMFArh61QFw1UgAbACs4GiFvwGnoT5TGLhyqljRajdzZoZwoc6QJ5Vsafk4pWDRei3y2QRXgIGr4NrwFlbATQUYuHJTTe6LFbBXgIEre1241mcKWMEqNTxrPdaXBbhd2Ttc5Wb9Qdnl/kMKrtKNsCRpJQ7CwJVSOrblUZFS8Le/c+j/1ufQio3HaeeBwgnqWHrOyigmYatmdTOphUgJWaNCWizdpcy+DFzF91IzcBW93oCrunXrZuoAEBbS8rkVSN0BuAZp7k4//XTbiQMrcHXFFVfQpEmTwg7hl19+oYsvvtjULpQ7jqlhmJVFixZRlSpVAlpZHY8CGhRUIMUF0gKeffbZcoxwWID7hB7btm2z1QNtnKYUxOdyvXr16OjRo0bXAFncSHmHFINeAVd42hqvh1AxYsQIev75501N6tatK1PrOEnDY9rRsgKob/HixUYtnAVuuOEGYx0Fp+4Qpp0SuJIMwBUmDtVkTuvWrSNWi4GriCXjHVgBViACBawT+xHs6lpTuDzecfttQfs7+eST5YR4e+EKGE06LTU57jfgCuA4XEJj/XxHP2eddZYpdXAw59RYgStciz179hjXKti9zR9//EEdOnQw2pUqVUqmoDYqQhR69epFCxcuNFrY3a8YG7WC03sYP48Np4N7GziRwsUsVACwAjCCe8sLLrhAPpCB7yBIW63igQcekI5Ial0trSkFkT4aIL7TiAW4wgMGgOqOHz9uHA6pr/GAAV4nTiIZvgs4OY9gbezel6dMmUxfCJdhBB6EQHpBOxe7YH2mUn0o4KqSAKvw/RT35vj7iSbUZ0rz5nk0cmTh6ziavnif5FOAgavku2Y84tRTgIGr1LvmfMaJUYCBq8TozkdNLQUYuEqt6520Z2sFq9SJWOv19aXrjss0g8u0NIM6ZKW3lf1pXBYDV0phd5bHc07Q2m25tGpTDv3453HavDvXlY5LZBajs07KojMaZFGjamlUsQwDV06EZeDKiUrutWHgKnot8T7dokULmdZE9dKjRw/5FLlaj2W5evVqE2SESTQ8XY4UGXpEC1xhkguTXXqsWrUqoqdy9X3DlcePH09jx44NaFarVi0JVgGSgiMSllaXMEwYATrTww3gCv117NiRVq5caXSNNJEAivwSdg5XmNwCiBcqMPl00UUXEf7G9QiVAlBvF6o8a9Ysuvvuu40mmGzA5KQe1157LX355ZdGFVzNMCnm1/AzcHW6eJ9RKQPhOBFtMHAVrXK8HyvACjhRwG5i38l+braxA66qV69uOALWq1c/psOpyXE/AFeA/OGmuWPHDgmOu/H5DnGQdm/06NGGTsEAGitwdc011wSA3kYnNgXc6+mgDANXgfC6jWyOgfbNmzdL8H/Tpk2mbvBQQZs2bSRcpe67Tz31VOlopTe89dZbJbik6vwGXAF2w4MN+JtXge8USJlXo0YNVeVo6ffvAo5OIkijYO/LkydPoi+/+ELuhfTkgK7wgAuHWYEFC4TDlUi3qgJwIgArfPdp2aqVqo56WTi5yMBV1CIm8Y4MXCXxxeOhp4wCDFylzKXmE02wAoX3RM0DsokkeGh8eFagyCjAwFWRuZRF+0QC4KiC07XW6+uqDPBq4PSfCtys/jCEUtsLK4xSvrWVWGWHK02TGIo5wtBqx/48WrM1h75ecYzWbc+lnNwTimuLqueMdKIyJdKoU9Pi1KpRJlUpl0alixeLqq9U24mBq/hecQauYtPb6tgEdyTAMKVLl46tY7H3mDFjCE/i6/HOO+9Q586d9SqKFrhCJ5gwRAoZFbNnz6YuXbqoVdeWmES7+uqrTf3BAesV8QN2p06dTPV2K0g/eNlll5k2IUUK0uTZhVOHK+x7880308cff2x0c++999IjjzxirCe6EC1whXEDIoJTAdL76YFJUpx3tHHkyBEJxx04cEB2gVRMeN0rUG779u0SRszNzQeYAQvivSYWWCjasTrdz2/AVf369Q0XFsACbgQDV26oyH2wAqxAMAWCTewHa+9FvQKukNoNLlZI7xTOETKScfgJuMI91KOPPmoMH5DJ77//bqxHWwAsDWhaD7t7LitwddVVV8n7On2/YGXcl1jBcQau3AWu4GILN1s9AMUhHboTJ7SePXsSHGtV3HffffTwww+rVWOZCIcr3Gfi/DZs2GCMA9+9cM8MiCzS8Pt3gUjPR28f6n15snBE/vJLBV2dJNMLMnSlq0e0cMEC8b72svxe074AtCpevLi5UQxranIRXcyZU+i4HEOXvGsSKfDoo1niO3KafOBs2PDHkmjkRJxSMKkuFw82BgUYuIpBPN6VFYhAAXVP1Lw5A1cRyMZNWYGIFGDgKiK5uHGiFAiAowoGYq3X11Fesnk5Tfn5HbEscJ/QXKz0trI7bZsigRi4cueKQ8djwuVq0648WrD8GP1POF3tPZxH2aIu2qhQOo2qlU+nLs2Fw1X9TMoSblcZbHDlSE4GrhzJ5FojBq5ik/Krr74iTF7oEWxCQm8TrozPAKQM0ycSGjZsSD/88AMBbtEjFuAKT4b/9NNPRncAjQAcuR3XX389zStIXYG+K1WqJN2PkOLOSXzyySfUp08fU1M8uR8s9V8kwBVcHODmoALAGcAzv0QswBXOYfLkyYS0MHpgogDXAzpFG1bYcOTIkXTnnXfK7qzHhEvCk08+Ge2h4rKfn4CrpmKy8KSTTnL9vBm4cl1S7pAVYAU0BUJN7GvNPC0CuPq///tdwlZIw+py3CEuAABAAElEQVR2+Am4soPZ8dnesmXLmE4bELoOZQdzuMJDAXg4QEX37t1p6tSpajXkEve3rSzuMAxcuQdcLVu2TKYF1C8CnEmHDx+uV4Us435Yd0kdOHAgDR06NGCfeANXhw8fpn/+85+E1O4q4Dr05ptvBqR6V9vDLf3+XSDc+ENtD/e+PGnSa/RVgSMt7j3hdFWmTJlQXabUNgBX+L5UzaWHH6zi6dAKA1dWdYr+eo8e+c7FPXv2op4iBW4yhf7anTnLP7+dJJOGPNbkUICBq+S4TjzK5FeAgavkv4Z8Bv5XgIEr/18jHqFQIACOKlDFWq+v9587nJZsWW64VcldNL5Hb2vdxsBVgcAuL/YczKNf1mbT8g050u3qwNE8JbXjI4GDwL/G1TPopJoZ1LpxligLuysOxwowcOVYKlcaMnAVm4xw8Dn//PNpxYoVRkeAWRYvXkxOYSJjR61gncTCJkyS6GncVPNYgKsBAwbQzJkzVVfSPQsuWpHE9OnTZXoTnG+9evWocePGhEk3PZC6ECkMVUSaduaxxx4jpKTTY+PGjRTsCWM83Y6n31UsED+W4ykZu8D5QwcVFSpUkJM4kbiUwVECT/3j/KED/mEyyI0Ji1iBK9xPwF0MGuiBSRW4WJQqVUqvdly2TrCdffbZhlMYUjLib0BFKP1Vm0Qv/QJceakDA1deqst9swKsQLiJ/aKgkJ+AK6QSPOOMMyg7O9uQ9pJLLqE33njDWI+mgPvNl156ydi1Xbt29NFHHxnrqoD7MtyfqejQoQN98MEHajXk8r333jMgbdWQgSv3gCsr+A6NlyxZ4vi7CcBFpBnMyclRl4fuuecek6Oa2mC9HwwG6Kn21uVbb71FgLlUwMk32HcRfO/CAxhIG6gHwL/bbrtNr4qo7PfvAhGdjKWxk/fl1157leaLh4gQ+M4G6CqS70GWQ/JqBAro9+YjRx4X31eF/T5HyijAwFXKXGo+0SRWgIGrJL54PPSkUoCBq6S6XDzYJFWAgaskvXCpNuwAOKpAAGs91uFq1f9D7clC2Cup0It6PbZr2xQFhCb1shtRvZyGsoeJD7vvSqKGlgrLY9knZGrBFRvzUwtu25uXn1owgpNPFw9SZ6QXo/NOy6L2J2dR1XLpVK6U2Y0mgu5SsikDV/G97Axcxa43wBIAJnq0bduWMIGAFIORBiYRbrzxRhPMm5WVJdPEwBnKGrEAV9YJBvSNcV944YXWw9iuHz9+XDoUbNu2zdhuhakwEVirVi1jOwoTJkyQ52iqDLKCY7Rv357Wr19varFu3bqgQBPcHQBkqQBYhElJu0A/54iUQ/qkUv/+/enxxx+3a25bZ01FAvAKzmFuuGvEClxhwLg+HTt2NKWPRL31WqEukkCfK1eulLvgXPF+gnsdpG/Ky8ufMIDu0N/vwcCV368Qj48VYAX8roCTiX2/n0O48fkJuMJYreA86mJJG4z7hbPOOovgIqRi8ODB9MADD6hVY/n222+bXFFr1qxJcFayOrEaO2gFwDFWOIuBK/eAK6tjU6TpJuEWdf/992tXjKhfv340YsQIUx1W4glc4bU4ZcoU0xjwvvPEE0+Y6iJd8ft3gUjPR2/v9H35tVcFdDVfQVenyPSCDF3pSnpTZuDKG12ToVekEkRKQQQ7XCXDFeMxpqoCDFyl6pXn8463AgxcxVtxPl4qKsDAVSpe9SQ8ZytYpU7BWj/pp1k0+WeL1a4OVmlQlXVfHbg6s8Zp8hAHj+RR3ZxGdHz7MbmejMDVzp276ODBA0oy0xIT9IAM3A5MAquJe/0YmBtGasENu3JpiXC6+ntHDgG6OnzsBB0VMFawOJGXQ3mHt1KJrDSqU6ceVa+QRi0bZlKzuplUPJMoUwBYXgR+CJcOLsJSq0H9+l4cIiF9MnAVX9kZuHJH7zvuuIPef/99U2dNmjSR7lH1I/j7BKQDN6IDB8zvi8Emu3DAWIArvB9edNFFprQcgIW+/fZbKlEi3+LedFKWlaeffjogVRzSLLZo0cLU0gpA3XXXXSZXBFNjy4rVaUFtBiBjB6BhOwCq1atXq6Y0Z84cOvfcc411a2HYsGH08ssvG9UZGRli0mE+nXZa/uetscGm8N1339EVV1xhAuT09Ho2u0RU5QZwhQNaUwSpQeC8e/bsqVYjWr722mum9DLjx48nuA8MGjTI6GfcuHHUt29fY92vBQau/HplvBkX3h9mzJhh6hyuDtYUsaYGKbgybdo0+vvvv40zx3ud9f3d2JikBcACuntPo0aN6IYbbkjSs0nssJ1O7Cd2lLEd3W/AFT67zjvvPNM9CIAnpIgGIBMs9bKdCvheedVVV9GqVauMzQAu8Ddi9wDB999/H+BoCmciOBSFCtw3wQXUGgxcuQdc4fMNjlQqAMWvWbMm6IMKqh2WmzZtok6dOhFcrvQIlh46XsAVXNesKREvu+wymcbSjQcc/PxdQL8OkZYjeV9+9ZVXhCPufHmIU04BdDUkaifcSMeZyu3V58rVV+eI+9BCV7lU1iQVzn3mzAyaNStDnuow4SzZtGmzpDptTimYVJeLBxuDAgxcxSAe78oKRKAAA1cRiMVNWYEoFWDgKkrheLf4KhAARxUcXtX/svn/aMrP79Avm/4vcGAOgKuWNZtR31ZXUsuaTeWPqapfLOf/8CvN//FX2W8yAlejn3iS5n4YmKIAJ5SVmUlNmzWlNq1a0U033uBo8j9Q4MCaAwcP0gXdLpIb/j1lEjVr2tRohMtx5PgJ2nMoj/63OYd+FeDVlj25tPNAcGvvAxu+pw1fj6KKNU+j2//1ArVqlEkVSxcTzlZpMr2gN7gV0aKvv6aHBg2RKbX+uzD/hzHjRJK4wMBVfC8eA1fu6L1lyxY5wbR7925ThwCC8BQ/gJMqVaqYtukrcGMCpAMwSL3Hq+3hXIhiAa5wjF9++YWQgkY/LtwNMLERChazOhugr2BpZ2666Sb69NNP0UQGdEGaOavzldqOJWCwsWPHEtIr6mNTbTBuwGF20bVrV+myoLYBiBs1apRaDVgCcMPY9TSEGNvzzz8vnaECdiiowN8PJg33799vNMHk5G+//UblypUz6mIpuAVcYQz33nsv4brpgbSHgOQAGUQaeL3DzQouZIguXbrI1EbffPONXC9ZsiQtX76cypYtK9f9/D8Grvx8ddwf20MPPUSvv/66qWOk10R6ULf+dk2dJ+kK3t8AR6hACrHevXur1SKxjCSdVZE4YQ9PIpKJfQ+H4WnXamIcMALeR/wQSGk8adKkgKEA/H/wwQela2moz2GkJoTj6XPPPUd79+419QOXoyFDhpjq1Aru0+Doqh5kQj3uGwF4w1HJLnDvh/tiK8yDtgxcuQdc4f4U92R6BEsJqLcBbAfX1j///FOvlmU8EILvHNaIB3AFKBbAl/59APft7777btD04tZxhlv383eBcGMPtT3S9+VXXnmZFhakIUdaSUBXuJ/n8E6BwgnGPPF7QP53Ku+Oxj37RQEduJo5y/Jgtl8GGWIcDFyFEIc3FSkFGLgqUpeTT8bHChTeDzUP+fu9j0+Bh8YK+F4BBq58f4l4gFBA/+FHVwT1gK3uKkghqLNVRju9UjNRalmjKd3c5irRNxmglTqWOh6WRRm4MjQShRZiQveZp8e5MmkbCrjCMXMFW3VcOF1t3yecsHbm0lYBXG0TZThdHRFOV7m5J6ThWHpaMcoSDyT9Pm8CrVk2jy684ja6/vobqF7VdFFfTDhb6WfgfpmBKzJSVqjUFcGWUF9ts5bt1oPVob6oBQNX7l1ROKZgIlpPZad6h1sfoCY4JmEyqlq1arId9sG/hQsX0rFj+W6Fah8skbINk2ChHApiBa5wnGAgDpyKMIaTTjpJjgHpAfGamThxIn3yySemzz9AVB9++CFhEtIa1gltbIebDCCz888/39Qc8A7grFdFags4fgWLefPmEZyz7AKTMhiLHnC4atasmXBVPCgdHOAIocfs2bMJzlt64H0DfV155ZXiqdOmBJgKn70bNmyQk5sANo4ePWrsgifsXxFPh/fo0cOoi7XgJnB16NAhCQb+9ddfpmHBsQaaR+MqqacFwusUE7BwuUKEgwVNg0jwCgNXCb4AcTw8/mbxXqCDkurwSEukJidVXSovGbhK5asf+bmrv51kTI3j9Gz9CFzhc/eWW24h3C/YBT6bkZq5cePG8v4ToDVSqAGuwT/rwwKqj0svvZT+/e9/h0yPDDDdmmauevXq0l3pggsuEO7PdWjXrl0SvkaaOqTNxngRDRo0IP1+hIEr94Ar3K/jmusOhbinvfPOO2ngwIEBDrFr166l6dOn0+TJk+nIkSPqJWBa4mEGqzMkGngNXCF1dbdu3Uzfk+DC++ijj8r7VowXzt9Y6unBTYO3WcE9f9WqVU1b/PpdwDTICFeieV9+5eWX5HdTHOpU8d110KDBDF1FqHskzdUEI/aZM6fwe2UkfXDb5FOgR498N3H8xjBs+GNJdwIMXCXdJeMBR6kAA1dRCse7sQIRKqDuh5o3Z+AqQum4OSvgWAEGrhxLxQ0TqYACoKxj+HnT7wZshW06W2W01SsFXNWqlnCzan0VAbjK3yefwlLHwFIvFxXg6gIx4T5m1AhDFhTWrfuLFotUBa+8Nkn+wNaoUUN6fcrkmJ2uwgFX+iBweXYId6ute/Jo+/486XR1XEBXeXknhAOXcLEqWYyeebgnHdi/h9568y1q0qQBeeVopY8LZQauCiEqBVMFW0Ivtc1atlsPVof6ohYMXLl7ReF0hSfAMTkQa7Rp00bYzM8K67biBnCFyTbAMZg0sQuAOIDEcH4KptHbAURCSsVWwpEwWCBFEybarIF+AaHBVWbz5s3SKQFgkApATJgcwqQgIC8VocAIOD3A8SFYoL+hQ4eaNuOz9b777iPAYXaBccD1aufOnSbISm/71FNPSWcAvS7WspvAFcYCiK179+4B1/H222+n0aNHRzxcwIK9evWy3Q9jh1taMgQDV8lwldwZIxwxkGrLLgCMKoc2u+2pVsfAVapd8djON5qJ/diOGP+9/QhcQQVA+wDE7e6zolEJsD3uh8Kll8b9Iz7n7RyrQh0Xzq9wwXryySeNZgxcuQdcQVQ4wV5++eUBEBLu6QG7AYbD/TZSCFofFsGDFnBHg0OsCsBJcIFMTzc/VeY1cIUHKPDadjv++9//Ehyc9PDrdwF9jJGWo31fflk4HS9atFAeDg8Mwekq3PtBpGPj9vkKrFixnEY8/rhcgcNV8+b5UCrrU7QVUMBVskLqDFwV7dcnn12hAgxcFWrBJVbASwUYuPJSXe6bFchXgIErfiUkhQIKgNIHO+mnWTT551l6VUjgqlWt5nSzAK1aifSB6E/1abfU64oScPXE6JEmvdTKd4u/p/sffEg+DfuvBx+gnlddqTZFtYwEuMIBkGIQ7laHxfKo+AcHLIBY6WkCCluznB79V38xCV+b3n93NgnTq7gFA1eFEJWCqYItcVHUNmvZbj1YHeqLWjBw5f4VxaQTwBs8BY70FJEGJkEAC11xxRWm122wftwArtA3XF8eeOABwtPdkQQcEyZMmEDnnHNOyN0wKXfVVVdJh6yQDbWNALHQNyb+4IyA1DYqAKTpaQpVPZaAwuD2oANa+na4NkybNk2vMspIpThmzBjTk/TGxiCFChUqyJQ7OKbb4TZwhfEhVeP48eMDhgqNL7744oD6UBVwqsC1gOuXHnAw+/bbb/UqX5cZuPL15XF1cHCgU1BV+fLlpXvd4sWLjWNgchfOIBwkU6ZySkF+JThVINqJfaf9+6GdX4ErpQ3S+T3yyCMSolF1kSxxPzN8+HDh2ny9o3tQ9L1s2TL5sAGcrMIFvo/BGfNxARfALZWBq4WGZEihjYcTwsXcuXNNToz4vLK6uqo+pk6dKp2gVOpnVR9sCZgK1x4OtEizi7TRW7duNZrjO4I1VWFRAq7Uifrtu4AaVzTLWN6XX3rpRfp60SJ5WLjwALoqXrx4NMPgfcIooD5bAFtxWsEwYhWBzXo6wWHiM7dp02ZJd1YMXCXdJeMBR6kAA1dRCse7sQIRKsDAVYSCcXNWIAoFGLiKQjTeJf4KKABKHTkftsKEdb47laoHpBMQovLW1r3p1ja9TJCV6tNuqdelAnAFzW7v15+WLl1GXTp3orFPjAmQMZKKUMAV7PfxQyOcTJzECy+9TG+8OZ2uu+ZqGnjvPQG74Gljr36UCgVceXncgJN0ueL56e/Rn+s3UsPaNenWKy8xfuxXwBSWehmHD7eut7GW7daD1aG+qAUDV95dUTw1jomJKVOmEGCOUIHXMOAiuK7ceOONIVMIWvtxC7hS/S5dupSefvppWrBgQUjoqEmTJhKAQro969Puqi/rEnAOXLvgpLRt2zbrZmMdT1LjqXq4hWVkiNytIgD0WB204NbUQABqdoHJJQBaH3zwQcB5hIOB4OT17LPPyn2DpdrBMStWrCivGSYzypYtazeMmOu8AK6QcuWyyy6jJUuWmMaHtJC47nDyiiTGjRsnIUN9H0ym9u/fX6/ydZmBK19fHtcGh/RKbdu2Ne654fbWoUMHGjx4sHEMAFmvvfaasZ7KhVRwuMJ7vO7ugnRrjRo1SuXLHvW5xzKxH/VB47yjmhSHG95DDz0U56M7OxzSqwHK+eKLL4RDzSLb9Kl6T7gHbd26tXRDgttp5cqV9c2Oynv37pX3jkiDjbJdwAkLrkkK2MG9JgNXCw2pvACu0Pn69etl2ke8JoIF7mHxMAKcXvEghYoBAwbI1OZqHY6mgJH0KIrAFc7PT98FdL0jLcf6vvzSiy/Q119/LQ/bVKRjHizSC2YxdBXpZQjbvnCSkYGrsGIVgQY6cDVzVmQPu/nl9Bm48suV4HF4rQADV14rzP2zAvkKFN4LcUpBfk2wAl4pwMCVV8pyv64qoAAodPrL5v+j/nOHF/RvJqzsgKsXug+XrlbYQfWDpV4OtS1VgKtnJz5Hb8+cRaeLPL5TJr1K27dvp+dfyP+x718PPRB0ovutt2eINFT/o7PPPosuv+xSeV104Gra1ClUv0F94XTyBi1YuIjWiwl9/OhcV1jsd+9+OV11ZQ8qVbKk3M/uf72vvZ7++usveln8ENW6VUvZZNlvv9Gb09+iP/5YTVsFUFBWTNwACLi6dy/q1vUCAw5C4x9++JE+/s8n4oftSrbAljrmqNFPSGDgmmt6UzPxdCFCAVclS5agRfO/kj8KIv3iEpESbNu27dLy/WSRDqBXr6vowq5dTcdV/fpxycBVfK8KA1fx0RtP/SNlB1LmYYkf8TGpC2gJExsNGzb0XZoGgJvfi7SueI/DhDTW69atK8EwvKcBynEKp1pVBgy1du1a2bfqH7AP0su0a9dOLq37RLsOlzEcC6kAkfqwdu3aVLNmTQPkCtUvADGkT1m+fDnhGu7fv1+ODecPQK5evXqE1Cwcya8AA1fJfw2dnAEm9zHJrwJg5fkirfUZZ5xh3HvjbxqOLVWqVFHNUnaZCsBVyl5cD0481ol9D4bkepfJAFzpJw3AGiA9XIpwL4d/e/bskWmcq1evLtNFw8UI90VuBB5egmPg6tWr5fd1uCThfhFAF4OMbigcfR/4/WTdunXyH+DjzMxM+TkHeBDXRz3gEP0Riuaeyf5dwI335RdfeIH++9986KpZs+bC6UpAV/z9x9UXvJpkRKecVtBVaX3ZWbKnE4SoDFz58qXFg/JAAQauPBCVu2QFbBRQ90LNxdzvqFGjbFpwFSvACsSqAANXsSrI+8dFAQVHmWErHDo4cIXUgS/+4zFjcke2LiCyGLgKvGx39LuLfhU/Fnfu1On/s3cecFLUXhx/V+lFmvRiQw+sFFHsSlEEBanSEVDEgiIoIk1FmgoqCgrYG4oIFrD+bdhAxEaRLr1ztDu4xj8vR4bs3O7e1tmZ2V8+n70kb5JM8s2wzO789j2aOP4J8fB8A3Xumu9y/9OP51NFHw/FBg8ZSj8s+pE6tL+FhgzOD0WlC66mPjuFWMzEwihv6YILzqepz0zx+oXS5i1b6JYOnaTY64uFn0oPLy+/8iq9NHOWDH/I47EYgb+kU6nJxRfTlKefNEQK78/5gCY99bQUMXzw3ruqWYH8qmubEf9aedzYx+naa66Wx5Xgin+FPGrEcHpg6EPkK1xAt6630j13DSwwrh0NEFxZuysQXFnLG2cDARCwJwEIruy5L5GcFd+PsZc8Fr2q9JcQybPQgD2+LV68WJlp+PDhNGjQIKMerwUIruJ150NbdyQe7Id2Zut6OU1wZR0ZnAkEQMCOBCL1vjx16nO06Icf5BL5QRiHF2TRHlJkCKxYsZweFd6BOUFwFRmmdh1F927Vvn0Hai88BzoxQXDlxF3DnEMhAMFVKNTQBwSCJwDBVfDM0AMEgiUAwVWwxNA+JgRYIFVQbMVT8S646ttQhBBskP+hSom1ZGsIrrzu36+Ll9Cg+wdTbm4uDX1gMLW/pV3EBFcs1NotvJ40b3YdtWjenC4UAqt1whPKjz/+RG8K71j8q+Arr7hchjE0e3Jhj1vseatli+b06OhR0qtUm7btpIiuY4f2dH3LFlT3rLNEGKwt9NkXX9Arr74m13fXnQOoR/d8sVgkBFf8y2FxUsoSvyjuemsXuqzppXS6CIWyTIRg/Fyc94svv5Lnvffuu+Rxr5BtZITgytrNgODKWt44GwiAgD0JQHBlz32J5Kw4XGbHjh2NITl0qQqT8+qrr3qEB2Nvfr/99pshkDc6xVkBgqs42/AwlxupB/thTiOq3SG4iipeDA4CIBBhApF8X5763LO0aNEiOcP6wjPeQ0J0Bc9okduwkw8aYxNWkH8nqv1WNHILs+lICQkkfjRr/eR0wZVTwwkyNQiurL92cMbYEIDgKjbccdb4I3DyPggeruJv97FiqwhAcGUVaZwnLAIsmuIwgiy68kwFBVcstuKXElqpnPupMud62d8xN4cU3CzC+/308y/0wvTplJl5VITcqk2vzppFHEJv/frIeLhithw28MEhD3DRI33+5Zc0ctQYuRe9e/WkAbf39zg+YOBdtPT3ZfTE44/RdddeI0MesgCLxU7vvPWGR1uucHjDBOHx6uab2hhhCiMhuOKxWQw2ZvRIatGsGVc90viJk2juh/NkSMFnJz8twyt6NLBZBYIrazcEgitreeNsIAAC9iQAwZU99yWSs+KHjvPnzzeGHDhwII0ePVrW09PTqV69eh6eQt955x26ToRkDiZxCFIO3aUShytMOvE0hz1sscBr4cKFtGnTJiHU3ym9a5133nnELw5ryKG3Qk0cNvXHH3+UoWDZixeHDuMwqMVEaOwyZcoQhw9r3LgxXXbZZTKMbSDnKUxwxevgfzsqcVjuYJlx3y3Ca+zKlSvVMDLM1YUX5ofqNoxagcVwv/76qxGWlkNllS1bljg0LYfHulJ4xG3YsGFA3jeY1YoVK4zR2XMse0IrLPEPQb4UnxW4rwrTdeTIEREmvLycB4/B8zj77LMdE9a7sDUXdjySD/YLO1esjkNwFSvyOC8IgEAoBCL9vvzcs8+Ke4180dW5554nwwtCdBXKzhTsE2svV7oQqODs3Gdp2jSXHngg2/KFuSGcIEOD4MrySwcnjBEBCK5iBB6njTsCEFzF3ZZjwTEgAMFVDKDjlMETeGnxuzTzt/e8dPQUXN3WIF9sxQ3NgiqzzXxcr+tltwiueP3qgZRioYfiq18vjSY/9aR8YMTH10dIcHWe+GXeS9Nf8OnB4Lmpz9Mbb71Np4iHOAs++ciY46FDh6hZyxtkvy8/W0AlSpSg+R99TGPHjZcPd+bMfkc+ROO5+kuRElz17NGdBg64w+upmGPf/rfTP8tXCG9dV9CkCeO8trOLEYIra3cCgitreeNsIAAC9iQAwZU99yVSs2Lx0bnink8PvfzRRx/RJZdcYpyiV69e9Omnnxr15sLz6VtvvWXUAymYRV3//fcfsSdS9q51//33S2GRv3HatGkjRWDsYSvQxGERn3zySfrss8+MzxeF9eW1PfLII8RevvylwgRXLDBjUZmeFixYQI0aNdJNhZbvuOMO+uCDD4x2I0aMoHvuuceoqwKv8VnxsHfJkiXK5DNngRmHhezRo4fX0OCqI++xHj7yqquuovfff18dLpDzNfT666/TCy+8ILzYbi5w3Gzg627YsGHUzMuPIsxtnV6P9IN9O/KA4MqOu4I5gQAI+CIQjfflZ599hn4SAm9OLBjn8IL6d3m+5gK7fwK64Kp+feu9XEFw5X9/InFUZ+zkcILMAoKrSFwRGMMJBCC4csIuYY5uIADBlRt2EWuwOwEIruy+Q5gfzVgyW768ozgpuOrboBPd1vBkbHZdNKX66ja9zMf1ul52k+BKcVA5/1Lu7LPrUkPxC/He4iEYe7ZSaX2EBFe39+tLt/XprYYtkC9d+jsNuOtuaX9l1gyql5Ymy599/gWNHD2GLm7ciJ57Zoq07d69m9re0kGG9qtSpYoMfdhChCqsVKlSgXGVIVKCq5emvUAXiHCIvtJLM2bSzJdfoSJFitAP3/7PVzNb2CG4snYbILiyljfOBgIgYE8CEFzZc18iNauZM2dK0Ysar2bNmjJkIHtkUok9T7E4RyX2Hrp06VKqXr26MhWaexNcTZ06VQqi1P17YYPwfSN74jrjjDMKa0pPPfUUjR8/vtB23hqwEIxDKV599dXeDktbYYIrbsQCpeXLlxtj9O7dmyZOnGjUCyscPnyY0sT9dWZmpmzKD23/+OMPqly5skdX8x56HPRTadq0qRTO8Y8jvKVgBFfyRwzCU9rHH3/sbSi/Nt2jmt+GDj4YjQf7dsMBwZXddgTzAQEQ8EcgWu/Lz4jvwH7+6Sd56vOEh04OL8j3TUjhEVAPG3mUDz88Gt5gQfbWxUBBdnVk81h4uFLerRiYk8MJ8vwhuGIKSPFAAIKreNhlrNEOBNQ9UP36CCloh/3AHNxJAIIrd+6ra1bFIQQHzB8p1FC+lpR/gMVWfRudDCPIrdVDF5WbbcruLddtbhFc1axZg8Y9/rgBkp9/VatW3UNkZRwUhfUREly98NyzIuRIA31ojzKHMrz6umbED1jGPzGWrrn6Knn84UdG0Fdf/48eGHw/dWx/i9FnmXhA9MCQB+mQeHikEnspuPyypnTD9S3prDPPVGaZR0JwlZKSQt989YXfX+//IsKu3DPofnlO9sjFoWXsmiC4snZnILiyljfOBgIgYE8CEFzZc18iNSuzKGjIkCE0dOhQj+Gzs7OlF6y9e/ca9vvuu48efvhho15YwSy4elzc27InKT1xiD8Oe8fn8eUhiT1Esacof4mFViy4Mqc6derIkIEsWOL7V/aytXHjRtq+fbu5KdWqVUuG5vPlmSIQwdVLL71Ew4cPN8bmsH58b8H3p4EkDt2oe7Nij1mzZ8/26Mqex/r06SPXow7w+MyJ18lh/Jgnr3PZsmXG5yzV9sYbb6RXXnlFVT3yYARX7Hlrugh1rqdTTjlF/OjhAhmysWTJkpLzqlWraN26dXozWR43bhyph98FDrrAoNbmdK8N/rYCgit/dHAMBEDAbgSi+b78zJTJ9PPPP8sln3/+BTK8IERX4V0BuperTp1yqHPnnPAGDKI3BFdBwAqhqc7XDfdJEFyFcBGgiyMJQHDlyG3DpB1IAIIrB24apuw4AhBcOW7L4mvCLLZi0ZU/wZUSWzEZJZTSy75syu4t121uEVxdKx6ujBv7WMAX0PoICa7eev01OvNM3x4E+EHVNc1aUEZGBo0ZNZKub9mC+IFc8+tb0ZEjR2j+hx9QFdMv8Dnc4MefLqDPPv+cVq9e4/Fw6KY2rWmIEGmlpqbKtUZCcFWqVCn66vOFpHtpMINcsXIl9erTV5o/mf+hX69b5r5W1yG4spY4BFfW8sbZQAAE7EkAgit77kskZsUh96699lpjKL5f+u2334i9XJmTWVDD3qbY21Kg4iGz4IofPPK9JCcWEo0aNUqG8VP3bCwQ+vrrr6X92LFjHtNhO4fq8ZZY0MOeqXJyTj6IY9HPmDFj6NJLL/XWRYZLfOKJJ8S96WqP47NmzSIOZegtBSK44nCN/CtEvj9W6Y033qCWLVuqqt+8bdu2tGjRIqPNjBkz6OabbzbqvMazzjqL+P5apVtvvVUK4ThsoDmtWLGCWFC3ePFij0McjrBBg4I/sghUcMVCLg7FqBLv7WOPPUbdu3cXPxAppsxG/rn4HMDz0IVuLAxjr2m+vG0ZnR1aiOaDfbsggeDKLjuBeYAACARCINrvy1MmT6ZffskXXfF9CIcXVPc4gcwPbQoSUA8c+YiVXq50QVDBWbnPYrWHKzd5t+KrAYIr9/2bwIq8E4DgyjsXWEEg0gTU/Q88XEWaLMYDgZMEILg6yQIlmxGQoQR/O/Hrax8eri6qWo9eaDPGmLkSSrFBlVVutim7t1y3QXBF9PG8D8Wvyr2H7evRuw+tWvUvdRBeqFjoxIm9T10rRFScHhzyAN3Srq0se/uzVvw6/dZu+eFlpk19TjyouYh+XbyE7r53kBRqsWDLX+KHQz8s+pE+/uQTWvr7MtmUv6i/f9C9sqwEV9WqVaUP57zvdSgOtXJdi+vlA7txYx+na6/JD/3y3fff05AHh8k+773zFtWuXdtrfza+9/4cevLpydLN+4/ff0u+PBn4HMDCAxBcWQhbnAqCK2t542wgAAL2JADBlT33JRKzeuihh4hFRSqxIIlD9nlLLNa58sorPQ5xKDsWHgWSzIIr1WfKlCnUtWtXVS2Qz5s3j/r37298PuAGLCp65plnCrRlA3t70sPa8T0gC3zYu5S/dPDgQSnI2rlzp9HMH49ABFc8EIcR/ETc66rEAi6dubKb8y1bttBFInS4+mzDHlg5PCGHwFbp999/pxYt8u/b2cb78/777/t9oMvr5D5r165Vw0jPUuxhypwCFVw999xz9Oijjxrd+boaPHiwUfdWYLFfq1at6OjRk2GB2NMWe9xyY4r2g307MIPgyg67gDmAAAgESsCK9+XJk5+mX3/5RU7pwgsvlKKrQOeHdgUJxMrLFQRXBfciUhadrRu8WzEXCK4idXVgHLsTgODK7juE+bmFAARXbtlJrMPOBCC4svPuxPHcPMRWzMGH4OqXO+Z4UFIPE2SX4/mdfNmU3Vuu2+JVcMW/pm/ZqrXkO2vGi3Su+GW9OWVkZtI11zWXQiVfgquWLZrTo6NHmbsa9bkfzqPxEydRcnIyLfj4Iypbtgw9+dTT9N6cD+i2Pr3p9n75XqNUB/51P7f19qu+SaIfC6yKFSsqQgB+KcVPX371NQ0fMZJSRViUH777xmu/n8WXV/fel/9Ax5fgaviwh4i9Z/lKfA4+F4c0fPP1V301s4UdgitrtwGCK2t542wgAAL2JADBlT33JdxZsdco/oVcenq6MRQLZzp37mzUzQX2hsVCGZWaNm1KLIgKJHkTXPF47777bqHdhw0bRizuUom9N/H/0ebE4h32+JQp7nNVCkbEYw4BWLFiRWKhmbcUqODqiy++8BCUsWCKxyxdurS3YQ0bC9HGjh1r1Hv27ElPPvmkUefC1KlTpecuZeT2LE4rLL3++usegqhzzjmHvhc/VjCnQAVXXbp0oa+++sro/u2331K9evWMuq/C7bffTnPnzjUO8zXiTfhlNHBwwYoH+7HGA8FVrHcA5wcBEAiGgFXvy5OffkqGKOa5XXjhRTK8YDDzRFtPAuqhI1ut8nKli4I8Z+POmpUertzm3YqvCAiu3PnvAqsqSACCq4JMYAGBaBBQ9z7wcBUNuhgTBPIJQHCFK8GWBIxQgmp2XgRXfRt2JH7pSQml2KbKKjfblN1brtviVXDFvK665jpiUdXdA++k7t0Keg2YMXMWzZj1Mjf16eGKxVFPTpxAl17SRLbT/3AIkL7976Dde/bQFZdfJtvx8TY3t6MdwjPAay/PEmFhzja6vPLa69KT1M03tSkgxOJGS0T4moF33yvFVV9/+bn89f7KlauoZ5/b5BivzJpB9dLSjPG4kJubS/cMul/25bovwRWHvJnx4rQC4Q25z48//kQPPPiQHGvQvffQrZ07sdm2CYIra7cGgitreeNsIAAC9iQAwZU99yXcWbHQhQUvKhUvXlwKgfyFdGPRE4uf9PTTTz8Jz6Zn6iavZbPgiu8zf/zxRzrttNO8tteN33zzDXXsePKzA4es27ZtWwGvpHytDh06lDZu3CjD1Z1yyin0999/G+Gq9TG9lb/77jtq3769cYjDXG/dutWo64VABVcc9u/888+nXbt2Gd3ZOxd76fKXLrnkEg8vVAsXLqSGDRt6dDEL0W655RaaPn26RxtvFf5xxnvvvSc9wNaqVUvm3kL/BSq4uuKKK2ilCNGt0tNPPy3DCaq6r5xDUnI4wtrCC1mdOnWoevXq8scZvto72W7Vg/1YMoLgKpb0cW4QAIFgCVj5vvz0U0+JcL6/yimy98qh4jsopNAIxMLLFQRXoe1VYb10rm7xbsVrhuCqsJ3HcbcQgODKLTuJddidAARXdt8hzM8NBCC4csMuumwNBbxb8fpMgisWWvVr1MkQVSkESiglu8DDlcQydtx4mv/RxyJM3jVCTPSYQhVQPkR8gfPd9z8QPzR7dsrThpcr9jL18aef0vgJk4xxfHm44gZFixalxx8dQ00vvcR4qMWhBB986GHaLEKdVKxQgWYKL1pVKlemNWvWUtcePaXtk4/meXik4vPNFR4Q2LsVe5y64fqWxgMVfhA15rGx9LnwAHCZ8JTw9JMT5dyysrLopnbtae/evVRXeCt4atIEYvEUJw59+Myzz9FHH58M0eJLcMXta9SoQRPGjaUzTj+dq9Kz1yIhtmLvVuzhoWHDBvSM+OVhivCmZecEwZW1uwPBlbW8cTYQAAF7EoDgyp77Eu6sWFjEAiOVOnXqJD0mqbq3nIU65557LvE9mkrsUUn3xKTs5twsuDpd3JP9ciLMjrmtub5p0yYRurqBh/m///4jFon5Snx/t3//fqos7lEDTevEPW6TJid/aMD3rbpQSh8nUMEV9xkzZowH28suu0x4ZfhQH86jbA4VeMYZZ9DPP//s0YYrLK4aMWKEYWeB2MiRI6WXK28eZY2GARYCFVx1796dPvvsM2NU9kDG3reuuuoqwxbvBSsf7MeKNQRXsSKP84IACIRCwOr35aeeepKWLF4sp8r3NEOGPhjKtNFHEFAPHhnGY49lCY+teVHloguDonoimwxulYcrN3q34i2E4MomFzKmEXUCEFxFHTFOAAKSgLrvgYcrXBAgED0CEFxFjy1GDpFA42ntCvbUBFcNqtanF24aI9voAis26HVVVrl+nG3K7i3XbfHs4Yp/2d+5a3cpLOIHLvXSzqEyZcuKX5H/QRkZGcKbQB068/Qz6PMvv/Tp4Yo9PrGoiZmWKlmSzjvvXFq3br30YMV7wrYXp79giJhmvfwKvThjJrW9+SYa9uBQbmIkFkjd1rc/bRQPxzjxA7JGQuSUmXmU/v7nb5mzuGvS+HF08cWNjX4fzptP4ybkC7A43GD9evWlN6o/RTgb9nDFwq3ly1fQf+JBnDfBFQutLm7ciOZ8kB+upLJ4AHT66aeJcDh/S9EWn4jFXNNfmCrFacaJbVqA4MrajYHgylreOBsIgIA9CUBwZc99CWdWmzdvlgImdd/MY7EAiIVAhaXevXvTJ5+cFLyXKVNGhvfj+zh/ySy4YjHO+++/76+LcYyFXnXr1jXqXFizZo0IZ13WwxZOhc8xf/586SFLH2en8NzKHrXMKRjBFf8b0tnyeOzZqWrVquZhZf2hhx6iWbNmGceGDx9OgwYNMuqqwEK0Ro0ayft9ZeOcPWrdfPPNxCEbOVxgqClQwdXs2bPprrvu8jgNf/5o3rw53XDDDXSN+PFIMMI3j4FcUrH6wX4ssEFwFQvqOCcIgECoBGLxvvzUk5NoyZIlcsoNhNfKIUM8vzcLdS3x1k/3csViKxZdRTNBcBV5ujpTN3m3YlIQXEX+esGI9iQAwZU99wWzch8BCK7ct6dYkf0IQHBlvz2J6xl59W7FRDTBFYutWHQlzSe8WMkKN9PqqqxyvT3blN1brtviWXDFzJaJMB2jxzxG23fs4KqRWjRrRvfeczfNeuUV+mDuhz4FV59+PJ9Wr15DT095hvjBnJ7OEqFjHhz6gOE5i49x+D8OAzhZfInUtOmlenNZZk8Dz059Xp4zL8/zF2j8y/3Hx4yWQjBzx7kfzqMpQvh19OhR41CRIkWoW9dbqU+vnsKrVi8ZOsaX4Or9d9+md2e/RywIY+GXSvwg6DIxT/a4Va5cOWW2dQ7BlbXbA8GVtbxxNhAAAXsSgODKnvsSzqwmTZpEEyfmC9p5HBaoL1261MM7qa/xvxAeSbt29QxXHUiIPLPgqlu3bjR58mRfp/Gw8z0gz1FPfF2Gcv/GnlXZOxZ7s1q7dq18/SZCW69atcr4jKGfZ4e4j05KStJNshyM4Io7tGjRgthzlUrsieruu+9WVSNnb7T8y0UWgHEqTJzFQiwWRvlKVapUkYKn6667jq688koqVaqUr6YF7IEKrtjj2dVXXy0+N6wuMIYysPCLBWA8j8aNG9veq6yad6TyWDzYj9TcAx0HgqtASaEdCICAHQjE6n35SXEP9ttv+aKrhg0b0QNDhtgBh+PmoItaOnXKoc6dc6K2Bv760vQVZtTOZYeBxVel4t43ejNxs9iKqenXJn8XjQQCbiUAwZVbdxbrshsBCK7stiOYjxsJQHDlxl116Jp8iq14PScEVyqUoFqiEkZ5q6tjKpfDnBBksU3ZveW6zemCK8UmnDxLPLRZv349bdiwUXiVKkZnCW9OHP4vmMSepNaJMdi7FT9wqlWrpvQKFcwYelsWXq1fv4HWb9hA5cuXIxZvFfbAjIVSfH4WfnFYwXPOrkulS5fWhy20nJGZKb0hsDesMqXLiPOeQfwQykkJgitrdwuCK2t542wgAAL2JADBlT33JdRZsei9ofCqoIvp+T6MQ/wFkvi+UBcOcZ8LL7yQWIjlL5kFV+zFafDgwf66GMfCEVyxmIrn9pfwjsr3xLxuFl0FmiIluHr11VeFJ4uTD1XT0tI8Qjqq+SxYsIB69uypqlLI9N57vh/WsNiJw0EuWrTI6OOrkJycLMVOLBZjD1iF3X8HKrji87G3rRtvvJG2b9/u6/SGvaTwksseztq1aye9YPEPKdyeYvVg30quEFxZSRvnAgEQCJdALN+XJ02aSEvF/Qkn9lQ5+IGT9wfhriue+qsHkLxmK0ILxhPbaK7VraEEFTMIrhQJ5G4nAMGV23cY67MLAXW/g5CCdtkRzMONBCC4cuOuOnRNXkMJqrWcEFz9OuADZZG5EkYpo15XZZVzG1XmXC/7OwbBlaKL3C0EILiydichuLKWN84GAiBgTwIQXNlzX0Kd1XfffUft27cPtbvPfl999ZUMZeergVlwNXr0aBo4cKCv5h72YAVX/Flh4cKF9Pzzz9PixYs9xvJXYTH/wYMHPZpESnB14MABqlevHvEPD1TivWDhlZ7MIRtffPFFKUzS25jLLKJ76aWXaMqUKbR3717zYa91Fl/xnowaNYq47C0FI7ji/szuscceo3feecdjnd7GVjZmPm7cOOrYsaMyuTKP5YN9q4BCcGUVaZwHBEAgEgRi/b48aeIE6V2U19JIeH4cPPiBSCwrrsawOrRgXMGN0mLd7t2KsUFwFaWLB8PajgAEV7bbEkzIpQQguHLpxmJZtiIAwZWttiN+J+PXuxVjEYIrs3craRYPQvSkRFT6MV82ZfeW6zYIrnTCKLuBAARX1u4iBFfW8sbZQAAE7EkAgit77kuos7r99ttp7ty5oXb32Y/DDLLgx1eyUnD1wAMP0GuvveZrKoa9bNmyUvB08cUXU8uWLWW4vUsv9QyLvXPnThnWz+h0ohBsSEHuZmZ/1113ScGTGnv//v0ynCB7reLEYqTly5dT0aJFVRO/OYcjZOHbnDlzpPcsFnkVlpo2bUqzZs0SXmfLF2garOBKDcDnnTdvHs2fP59+/fVXUutRx73l/fv3pzFjxvgUf3nr4yRbrB/sW8EKgisrKOMcIAACkSJgh/fliRMmCK+hS+WSGje+mO4P0PNnpBi4YRz1EJLXEu3Qgm7gFcs1xIPYivlCcBXLqwzntpIABFdW0sa54pmAuteBh6t4vgqw9mgTgOAq2oQxfqEElm77hwbMH+m3Xd8GHalfo04F2ihhlDqg11VZ5dxGlTnXy/6OQXCl6CJ3CwEIrqzdSQiurOWNs4EACNiTAARX9tyXUGaVnp4uBT26l6WqVatSiRIlgh6OhUi6N6hixYoR/7/pK+SzVYKrJ598kiaIB5jmxOtkYRV7lGJPU5xXq1bNoxlf65dddpmHLZKCq2+//ZY6dOhgjM9zWrZsmSHoeuWVV2jo0KHG8e7du9PTTz9t1IMpqNCP//vf/4hff/zxB7EnLG+pRo0a9OOPPxLvoZ5CFVzpY2RkZMhwh9988w19/fXXIsz5Bv2wR5lFbDNnzvSwuaVihwf70WYJwVW0CWN8EACBSBKwy/vyhAnjadnvv8ul8X3KffcHFm45kiycPpZ6EMnrQGhBe+7mP/8k0ogRqcbk3p3tO1y20cihBQiuHLpxmHbQBCC4ChoZOoBASATUfQ4EVyHhQycQCIgABFcBYUKjaBIo1LuVOPmvd3iGElTzUaIpb3V1TOXcRpU518v+jkFwpegidwsBCK6s3UkIrqzljbOBAAjYkwAEV/bcl1BmxZ6MHnroIaNrUlIS/fnnn3TqqacatkALb7zxBt1///0ezTk0nHqA6XFAVKwQXLGwqFMnzx96VKhQgaZPn05XXnmleUoF6hx+sFWrVh727du3e/W6FIqHKxY8XXTRRbR161bjHAsWLKBGjRrJert27eiHH37weswwhljYt28fseiJvV+x8El9nlLDjR8/nm677TZVlXkkBFceA4rKxo0bpReuN998U3rvMh///vvv6ZxzzjGbHV9X/y7at+9A7TXRneMXpi0AgisNBoogAAK2J2Cn9+UJ48dJATZDu7hJE7rvPs/7K9vDjPEE9dCCPBWIrmK8IV5O37btSW+tbr4X4qVDcOXlAoDJlQQguHLltmJRNiQAwZUNNwVTch0BCK5ct6XOWlAgYqt+DTvJcILeVmb+kl+vq7LKub8qc66X/R2D4MobedicTACCK2t3D4Ira3njbCAAAvYkAMGVPfcllFlde+219Ndffxldr776anrvvdB+YX748GHpJSozM9MYr27dutKbkWHQClYIrjis4RdffGGctVy5clLcwx6cAkksfurZs6dH023btlFKSoqHjSuhCK64H4vSdK9Vd999N40cOZI4DN/ZZ59NOTk53IxOO+00GY5PViL85+eff5bCNH3vOKQii+j0FA3BlT7+Cy+84BFSkY899thjdMcdd+jNXFG204P9aAGF4CpaZDEuCIBANAjY6X2Zv+Nk0RV7o+TUpMklNOi++6KxbNeOqYuu6tfPk6Ir1y7WYQuLl1CCalsguFIkkLudAARXbt9hrM8uBCC4sstOYB5uJgDBlZt31wFrazytnd9ZstiKQwkqcZS5sdmu11VZ5dxXlTnXy/6OQXBlpo660wlAcGXtDkJwZS1vnA0EQMCeBCC4sue+BDsr/j+NBVZ6YsGLHuJOPxZImcVC7777rkfT+fPn06WXXuph44oVgquzzjqL9u/fb5y7c+fO9Nxzzxn1wgqjR4+m559/3qPZli1bqEiRIh42roQquGIPT8qjFY9z+umn0y+//EIffPCBh9Do4YcfFh4ufD9sZY9VLC5bvXo1rVmzRoZ3ZPaBpkceeYRefPFFozmHWPzuu++MOhcCEVytXbtW9uOc58HeqVg0FWjia5KvTZX69etHTzzxhKq6JrfTg/1oQYXgKlpkMS4IgEA0CNjtfZm9YE4Q3ib//DNfdHXJJZfQvYN83wdEg4nTx9SFLhBd2WM3401sxdT169DNoRPtcYVhFrEkAMFVLOnj3PFEAIKreNptrDVWBCC4ihV5nJcC8W417aZHqUHV+oY4yoxNiaaUXa+rssq5jSpzrpf9HYPgStFF7hYCEFxZu5MQXFnLG2cDARCwJwEIruy5L8HOigU8M2bMMLoVL16cVq5cSZyHmlgo1Lp1a4/uN998s8d51MFoC66ys7OpatWq6nQyZ09S3bt397D5qmRlZRE/3Ny0aZNHkw0bNlDJkiU9bFwJVXDFfdu0aUPsZUqlRYsW0ZNPPknz5s2TpsTERPr999+pWrVqqkmBnAVX9erVMzxicQMWTLFwKpDE1wJfEyqxCIw9fOkpEMHVZ5995sGYvYrxdcVrCCT16tWLPv30U6PpoEGDaPjw4UbdLQW7PdiPBlcIrqJBFWOCAAhEi4Ad35dzc3NpwoTx9JcI98zpEiFgv/feQdFC4Mpx1UNJXlynTjnUuXO+51BXLtbmi9LFVnx/OnLUaJvPODLTg+AqMhwxSuwJLP3tN3rzTU8PyPqsduzYIZ/RJSQkUOXKlfVDHuWePXvRBRde6GFDBQRAIHAC6t6mfv369PjjjwfeES1BAAQCJgDBVcCo0DDSBAL1bsXnVeIo8xzMdr2uyirXx2GbsnvLdRsEV2bqqDudAARX1u4gBFfW8sbZQAAE7EkAgit77ksws2IxEX85o3t/Ys9W7OEq3HTxxRfT+vXrjWFSU1OFd4Y/qUKFCoaNC9EWXPE5LhRf5LJHKpUGDhxI7LUqkDRq1CivPPj6ZxGROYUjuGKvYOwdTKWhQ4fStGnT6NChQ9J05ZVX0pw5c9Rhn3nHjh3pm2++MY5zSMUpU6YYdX+F2267jT766COjSZcuXejZZ5816lwIRHDF1xaHQlRz536Bek7jB8vnn38+7dy5k7vJNHXqVBnuUNXdktvxwX6k2UJwFWmiGA8EQCCaBOz6vixFVyK8oAoBfemlTemee++NJgrXja0eTPLCILqKzfbGq9iKaUNwFZtrDmeNDoE7bu9P6enpIQ9evnx5ev6FaSH3R0cQAAEidV8DwRWuBhCIHgEIrqLHFiP7IRCId6vFA+YaIygBlGE4UTDb9boqq5y7qDLnetnfMQiuTsBG5hoCEFxZu5UQXFnLG2cDARCwJwEIruy5L8HMikPNqQeLqt97771XIMSgOhZM/swzzxT4lR17KGJPRXqyQnDVo0cPWrhwoXFaFkqxIMns+cpoIAoyhM+ECTR58mTjM4Z+fOnSpVSzZk3dJMvhCK6OHDkivVNxzqlEiRKkylxn8VX79u256DeZBVHc+KmnniLm4C998skn1KdPH4/1zp07ly6//HKPbubxr7rqKnr//fc92nDlzjvv9LAXK1ZMeq0699xzC7TVDWava+xJ7I8//qAyZcrozVxRVv/+2rfvQO2F2NGNCYIrN+4q1gQC7iVg5/flnJwcGV7w77//khvQtGlTuvseiK6CuRrVw0nuA9FVMOTCb6uLrXi0eAurB8FV+NcQRrAPgddfe014QT7pjTjYmbVu3Ya6dusWbDe0BwEQ0AioexoIrjQoKIJAhAlAcBVhoBiucAJLt/1DA+aP9NuwX8NO1K9RJ6ONEkcZhhMFs12vq7LKuYsqc66X/R2D4OoEbGSuIQDBlbVbCcGVtbxxNhAAAXsSgODKnvsSzKzMnpBOPfVU6YUqKSkpmGG8tmXvROyliD0yqFSjRg36TYQg0MPKWSG4MguEeD5nnXUWPfbYY3TNNdeo6cmcPTOxOOvFF1+kJUuWeBzTK1988YX0nKXbuByO4Ir733PPPfTOO+9w0SOVKlWKli9fTixaKixlZGRQy5YtZQg/ve11111HdjFzawAAQABJREFU999/v/Rqpsbhh7fr1q2TYq7Zs2d7hCJs3LixR1g/NZaZpy/B1apVq+j666+nw4cPq67Ens44nCPve61atSglJUUeO3r0KLGIjcM9fv/990Z7Ltx3330eYQ49Djq8YucH+5FCC8FVpEhiHBAAASsI2P19mUMlc3jBf/7+W+JoetllwjvmPVagcc051ANKXhBEV9Zsq1lsNVJ4kE1Lq2fNyW1yFgiubLIRmEZECKxZs4ZGPBJ6uPcnxo2n0047LSJzwSAgEK8E1P0MBFfxegVg3VYQgODKCso4hweBYL1bcWcljvIYyItdb6fKKtfHYZuye8t1GwRXZuqoO50ABFfW7iAEV9byxtlAAATsSQCCK3vuS6Cz4hB7DRo0kJ6cVB/2SDRmzBhVDTvnUHYsTNLT22+/Tc2aNTNMVgiu+GTdxC9oP//8c+O8qlCpUiUp/CldujRt27aNNm3a5OFVisVh7JWLxUMLFixQ3WjcuHEFvIPxwXAFV7/88gu1bt3aOI8qBBMWkPts3rxZct67d68awiOvWLGiFDvt2LHD4xpQjVgcx+utXLmyMhl5oIIr7sD7zwIr9hhmTizsq1KlCmVmZpKvebJI7I033qDk5GRzd1fU7f5gPxKQIbiKBEWMAQIgYBUBJ7wvS9GVCC/I30twukx4orzrrpMhia1i5eTzqIeUvAaIrqK7kxBb5fOF4Cq61xlGt57Ag0OH0H///Rf0iVloxYIrJBAAgfAIqHsZCK7C44jeIOCPAARX/ujgWMQJBCK2Mnu34kkoAZR5Qma7XldllevjsE3ZveW6DYIrM3XUnU4AgitrdxCCK2t542wgAAL2JADBlT33JdBZcYi58eM9v+jkMHv8ZU2kEgt2evbs6TFc8+bNiQU7KlkluNq3bx/dcsstxsNJdX5/OXtgYo9LV1xxhRT9sHcolRo2bOgRplDZwxVc8TjsWWrDhg1qSJlzuL+LL77Yw1ZYhT103X777VJ8VVhb/Tif//nnn6fatWvrZqMcjOCKO70mQk6MHj3aw9OVMZiPQkJCgngA2okmTpwYkFcvH8PY3uyEB/vhQoTgKlyC6A8CIGAlAae8L7NHzgniPm758nzR1eWXX0ED77rLSlSOP5d6UMkLqV8/T3g+zXL8muy2gBEjUsW9d6IxrXj0bKUWD8GVIoHcLQTmzfuQ3vXimbmw9XEoQQ4piAQCIBAeAXUfA8FVeBzRGwT8EYDgyh8dHIs4gcIEV97EVjwJJYAyT8hs1+uqrHJ9HLYpu7dct0FwZaaOutMJQHBl7Q5CcGUtb5wNBEDAngQguLLnvgQyK74vZsEQe3NS6ZxzzikQyk0dCzVnDwznnXce7dmzxxiCPUZxWEH2oMTJKsEVn4u9LHHYvLFjxxKHPPSVmAULlVjwozwrsceoiy66yKMLC5rMoqRICK4mT55MTzzxhHGuOnXq0OLFi416MAUOGThnzhyaMmWKDB3oqy/vC3s8Y4Ech5pkwZOvFKzgisdJT0+nGTNmyNf+/ft9DU1Fixalq666SoZWbNSokc92bjnglAf74fCG4CoceugLAiBgNQEnvS8fO3ZMhhdcIUIOc2KB+J0DIboK5prRRTDcj0VXLL5CCp+ALrZKS0ujkaNGhz+og0fQr7V3Z7/n4JVg6iCQT2DXrp10z93Be1d8/oVpVL58eWAEARAIkwAEV2ECRHcQCIAABFcBQEKTyBAoTGzFZ4HgKjKsMQoI+CMAwZU/OpE/BsFV5JliRBAAAecRgODKeXuGGecTYK8Q69evp40bN8oXe78qV66cDJ/XpEkTr2H0rGQ3ffp0GjFihHHKYcOGke5dyzgQRIHFZlu3biUOJckvLh89epQ4tOCpp55KvO4KFSoEMWJoTZk9h55Qc+B5pKSkyHlUr15dzqNYsWKhDe7AXk56sB8qXgiuQiWHfiAAArEg4LT3ZSm6EuEFV6xYIXFdceWVdOedA2OBzrHn1IUwvAiEGAxvK9mjFYutVILYKp+Efp1BcKWuDuROJzD28cfp77//CngZF1x4IT300LCA26MhCICAbwIQXPlmgyMgECkCEFxFiiTGKZRAYYIrX2IrHlh5nDKfxGzX66qscn0ctim7t1y3wcOVmTrqTicAwZW1OwjBlbW8cTYQAAF7EoDgyp77glk5n8DVV19thD5MSkqiZcuWUZUqVZy/MKygAAGnPdgvsIAADBBcBQAJTUAABGxDwInvyyygniBEVytXrpQcr7zyKhpw5522YeqEiehiGJ4vRFeh7dq77yYLb7LJRuf27TtQ+w4djHo8F/RrDIKreL4S3LX2b/73P3rxxekBL+rOgQOFN8YrA26PhiAAAr4JQHDlmw2OgECkCEBwFSmSGMcvgcLEVtx58YC5PsdQAihzA7Ndr6uyyrmvKnOul/0dg+DKTB11pxOA4MraHYTgylreOBsIgIA9CUBwZc99waycTWC5CAvEIfVUatasGb399tuqitxlBJz4YD/YLYDgKlhiaA8CIBBLAk59X87MzJThBVedEF3xvcQdAyC6CuZa0gUx3I9DC7LwCiEGA6OohxDkHhBbeXLTry8IrjzZoOZcAiz47dO7F7E35cISezV++ZVXpXfjwtriOAiAQOEEILgqnBFagEC4BCC4Cpcg+gdEoPG0dn7b+fNuxR2VOMo8iNmu11VZ5fo4bFN2b7lug+DKTB11pxOA4MraHYTgylreOBsIgIA9CUBwZc99waycTeC+++6jN99801jEa6+9RjfccINRR8FdBJz6YD+YXYDgKhhaaAsCIBBrAk5+X87MzBCersbTqlWrJMarhMfMO+4YEGukjjr/ihXLiYUxKkQjTx7ervxvobcQguzVKi2tnv+OcXYUgqs42/A4Wu7U556lRYsWFbpieF8sFBEagEBQBCC4CgoXGoNASAQguAoJGzoFQyBc71Z8LiWAMp/XbNfrqqxyfRy2Kbu3XLdBcGWmjrrTCUBwZe0OOlFwxYRmzpxpLSicDQRAwNUEILhy9fZicTEg8Mknn1D//v0pOztbnr127dr0yy+/EIcVRHInASc/2A90R9QXwXXr1qUhQ4YE2g3tQAAEQCAmBJz+vpyRwaKrccT36Zyuvvoauv2OO2LC0skn1cUxvA6IrrzvpjmEYFpaGo0cNdp74zi36tcUPFzF+cXgsuUvXbqUJk2cUOiqHhr2MF1wwQWFtkMDEACBwAioz9n169enxx9/PLBOaAUCIBAUAQiugsKFxqEQCNe7FZ9TCaDM5zfb9boqq1wfh23K7i3XbRBcmamj7nQCEFxZu4NOElzpX+pAcGXtdYKzgYDbCXz00UfEL05u/dKYf+X+6Jgxco0sFGDBABIIRILAvHnzqGjRolS9enVau3Yt/frrr/TKK69Qbm6uMfykSZOoV69eRh0F9xFw+oP9QHZEfREMwVUgtNAGBEAg1gTc8L585MgR6elq9ep80dU111xD/W+H6CrYa0v/LkX1hfAqn4RZaMVWhBBUV4n3XL+e3PrZ2fvKYY0HAgPuuJ3279/vc6kVKlSgqc+/4PM4DoAACARPQH3OhuAqeHboAQKBEoDgKlBSaBcSgUh4t+ITKwGUeRJmu15XZZXr47BN2b3lug2CKzN11J1OAIIra3cQgitreeNsIAAC9iQQD4IrJq/CYbVp04b4hQQCkSDAQqpPP/3U51Dnnnsuff7555SSkuKzDQ44n4AbHuwXtgvqi2AIrgojheMgAAKxJqB7b3W6eOTw4cM0ccJ4Wr16tcR6zbXXCi+at8casSPPrwtl1ALiVXjF4QNnz04mzlVir1YIIaho+M716wiCK9+ccMSZBF5//TVa4OezLX+PcmvXbs5cHGYNAjYloD5nQ3Bl0w3CtFxBwLaCKw6NsGvXLqpcuTLCIjj4UouEdytevhJAmVGY7XpdlVWuj8M2ZfeW6zYIrszUUXc6AQiurN1BJwmu4J3F2msDZwOBeCIAwVU87TbWGmkCw4cPp5deesnrsMWLF6eFCxcSP8BCci8B/cH+yFGjxH7Xc+Vi9QeM8Lbqyi3GokDANQTc9r7MoisOL7hmzRq5R9deex31E6GLkYInwP+XcZozJz/ncv36eVSvXh517pzDVVcnCK3C3179fgiCq/B5YgR7EVi7dg09Ij7f+krjxk+gOnXq+DoMOwiAQAgEILgKARq6gECQBGwjuGKB1bZt22jx4sW0cuVK4g96Tkk1a9akiy66SHxwqkdlypRxyrSjPs9IebfiiSoBlHnSZrteV2WV6+OwTdm95boNgiszddSdTgCCK2t3EIIra3njbCAAAvYkwOHO+MEUi0JGjhptz0lGYFbqSwx4Z4kATAxhEJg2bRqNHDnSqKtCuXLl6O2336YGDRooE3KXEnDbg31f26Q/YITgyhcl2EEABOxAQP8xgVuEsIcOHZLhBflhOKfrrruO+vaD6CrU603/P00fw60er7wJrXjdTvcAp++dVWX92oHgyirqOI+VBB58cCj9t3FjgVOefvrpNPaJcQXsMIAACIRHQH1XCQ9X4XFEbxDwR8AWgqvMzEz68ssvpdgqKyvL33xtfax69ep0xRVX0AUXXGDreVo1uUh5t+L5KgGUee5mu15XZZXr47BN2b3lug2CKzN11J1OAIIra3fQqYIrhMOy9jrB2UDA7QTiIRQW76H6EgOCK7df0daub8OGDfTOO++IkCz/0NatW6latWrUtGlT6tGjB5UqVcrayeBsMSGgP9h384M3eFuNyeWFk4IACIRAwK3vywcPHqQJIrzgurVrJZXrmjWjvn37hUAIXRQBXTyjbJy7RXjlS2jl9h/a6HsZ6bJ+zbj5vi/S3DCecwjMnzdPfL59u8CEu3XvTjfe2LqAHQYQAIHwCKjvKiG4Co8jeoOAPwK2EFy99dZb9NdffxkCGH8Ttvux5ORk8YGpE51//vl2n2pU5xdJ71Y8USWAMk/abNfrqqxyfRy2Kbu3XLdBcGWmjrrTCUBwZe0OOklwxWTUDTjEAtZeJzgbCLiZgO6Zxe2/cIZYwM1XMtYGArEjEC9eAplw504dJWjci8buesOZQQAECiegfkzgRlGJFF2J8ILr1q2TIJo1a0639e1bOBS08EtAF9HoDZ0YbtCXyIrX5fbPe/reRausXysQXEWLMsaNJYFdu3bRPXffVWAKz78wjcqXL1/ADgMIgEB4BNTzHgiuwuOI3iDgj0BMBVd5eXnSs9XXX3/tb46OO8ahHfqLOPecx2uKpHcrZqgEUGaeZrteV2WV6+OwTdm95boNgiszddSdTgCCK2t30KmCK6Y0ZMgQ4oddSCAAAiAQDgG3egDwxgSCK29UYAMBEAiXgHqwHw8PMdWXwRBchXvVoD8IgEC0CMTDjwkOHDggwguOo/Xr10uMzZu3oD633RYtpHE1LotpOM2Zk5/ri7er+IoFVpxmz04WHlfzy/q8WXjYvkMHET6+nm5GOUQCEFyFCA7dHEXgibFjhROOP405X3jhRfTgQw8ZdRRAAAQiR0B9xobgKnJMMRIImAnEVHC1e/dumj59OnGMeLcl/qDRq1cvty0roPVE2rsVn1QJoMwTMNv1uiqrXB+HbcruLddtEFyZqaPudAIQXFm7g04TXEEsYO31gbOBQDwQiCfPLLyf6osMiAXi4erGGkEg+gR00erIUaNc/zAT96LRv6ZwBhAAgfAI6O/LbvY+k56eLsMLblCiqxZCdNUHoqvwrp6Tvfn/uxXLV3gVXqlWHHaQEwux+GVVUqIqFlhxUnXz+SG0MhOJTB2Cq8hwxCj2JvDNN9/Qi9OnGZMcOPAuuvyKK4w6CiAAApEjoL6nhOAqckwxEgiYCcRUcMX/qS5cuNA8J9fU7733XqpWrZpr1hPoQiLt3YrPqwRQ5jmY7XpdlVWuj8M2ZfeW6zYIrszUUXc6AQiurN1BpwmumA5CuVh7jeBsIOBmAvHgAcC8f+qLDLbDU6CZDuogAALBEoiXB/uKiy64atOmDfELCQRAAATsREB5HeQ5uVlwxetLT98vPF2Npw0bNnCVWrRsSb1795Fl/IkcgUDEV3w2JbqqVy9ffKXqKg9mRrqIisvLl+d7rtLt3saDyMoblcjaILiKLE+MZk8Cx44do149e8hndAkJCfTGm29RcnK+yNOeM8asQMC5BNT3lBBcOXcPMXP7E4ip4Oq5556jzZs3259SiDNs3bo1XX755SH2dma3aHi3YhJKAGWmYrbrdVVWuT4O25TdW67bILgyU0fd6QQguLJ2B50ouNK/3IFYwNrrBWcDAbcRiDehAO+fLhaAlyu3XdFYDwhYS0B/D42HcIKKrvpCmOu4F1VUkIMACNiBQDy+L+/fz6KrcbRx40a5BS1bXk+9eve2w3a4cg5KfMWL8xZ2MNBFm0VYhQmpfI3LAisOFZhWLz/31Q72yBHQv5Nzu6gzctQwkhMJ3N6/H3EI27Jly9L0F19y4hIwZxBwBAH1+RqCq9hv13Gegvxzci6mav4Br8aTfWQpwVQ/UfUwi4pH3XsXWCNAIKaCq2HDhlFubm4ElmHPIS655BJq27atPScXpVlFw7sVT1UJoMzTNtv1uiqrXB+HbcruLddtEFyZqaPudAIQXFm7g04UXDEheLmy9jrB2UDAjQTi8YGU2kf1ZQbXIRZQVJCDAAgESyCevKjobCBc1WmgDAIgYCcC8fq+vG/fPhle8D8lurpeiK569bbT1rh6LizA4cT/P+bnK2QeyT8srFKpfYcOsshCKyTrCUBwZT1znDE2BIYMeYA2b9pENWvWoomTJsVmEjgrCMQBAfUdJQRXsdnsPOGYNEtIYTKOHae9h/Nov3gdyuTXcTp8VL3yZDkr5zhli4jS2bnHKSfvuNDQEOWIl5A0SJ1WUhJRsnBKmpyUIPIEShWOAVNTiIqnJlCJIolUomgilSyaQKWK5b/Klkig8qWSqLSoF0kWfUR/pOgQiKngaujQodFZlU1Gbdy4MbVv394ms4n+NKLl3YpnrgRQ5lWY7XpdlVWuj8M2ZfeW6zYIrszUUXc6AQiurN1Bpwqu9C94IBaw9prB2UDALQR0wVW8/TIXYgG3XMVYBwjEjoD+HhpP3q0UcfWlMNdxL6qoIAcBEIglgXh/X963d2++6Oq//+Q2XH/DDdSzZ69Ybkncn1v3hmWGocRZym4WT7HHKpXMx5QdeWwI6N/HxdvnaG/E9Wt5xfJ8saFu89YHNmcQYM+JGRkZVLx4capdu7YzJo1Z+iWg/3+i/p/RbX4742DUCKjP1hBcRQ2xHFjooyhXiKuyhWjqmHhlCeFUjhBOcX40+7gQXBEdyMgXWmUcy5MCrExhzxRCrMys/Fe2FFgJoZUYLEeMxWItFl2xwyt+CZ0VsegqSYiukkQ4VhZQpQjRVZGUBCqWkkjFhPCqqHgVL0JCgJVAJYXQqkyxfCFWUSHM4napso/IT5RZvMXjIYVHAIKr8Pj57R1vgqtoebdiyEoAZQZutut1VVa5Pg7blN1brtsguDJTR93pBCC4snYHnSq4YkrqZpzLeNDFFJBAAAQCJRDvD6SYk/5FeZs2bYhfSCAAAiAQCIF///2XJmm/8o7Hh238IO3RMWMkLoRnDeSqQRsQAIFoEsD7cj7dvSy6EuEFNwmPJJxuuKEV9ejZM/8g/oIACESEgP45Ml7vARkkc1ixIvLe3CKySRgEBEAgKAIqPK3yoBhUZzQOm4B6xgPBVdgo/Q7Awir2YsWiqj2H8mif8GR14EgeHTzhyYpFVcqDleG9ir1YCVEVe7FikRV7smLhVn4u6uKMXFZJ6K1I6KzyXyJYYKIQSnFdCrCExyv2fiXL0gOWEGOdEFcVFeIqKcASHrDY61XZEolUoXQilSvJYiwh0BIiLaTwCEBwFR4/v73jSXBVmHerfg07Ub9Gnfzy8ndQCaDMbcx2va7KKue+qsy5XvZ3DIIrM3XUnU4Agitrd9DJgis86LL2WsHZQMAtBHSxFa8pHr8kVnupwrNyHcJVRQU5CIBAYQRYbMUP9znFo3crxUd/4AjhqqKCHARAIBYE9FCC8fy+zOz37NkjRFfjafPmfNFVq1atqHuPnrHYFpwTBFxJQL//iafP0vwdJERWrryksSgQKEAg3u+lCgCJsgGCq8gClvon8Ye9UbGAil/s0SpDhAdkcdXew8dp14Fc2itEVxw+8KAQYB0WQqxj2Sc1CpGdUeGjsRcrFlVxqMFThNiqXKkEqlQmSYquyhRPlCEHU0XIQQ5RKL1gibISdxU+OlowAQiuongdxIvgqjCxFSOG4CqKFxqGBoEgCUBwFSSwMJs7WXDFS9e/6IF3gTAvBnQHgTggYP71/8hRoyie3YfrwlXefoiu4uAfAZYIAmES0EWr+CLa0+MqRFdhXlzoDgIgEBIBiGALYtuze7cML7h582Z5sNWNN1L37j0KNoQFBEAgaAL693DxILjyJbTi7yA5KU/Rqh40UHQAARCICQH+flD9iIg/43pL+LzrjUrkbRBcRZYpe6DiV7rwXrX7YP5rz8Fc2n9ECK6EuOqIEF5xqMBjwqMVC7GyhecrFmexB6tYJeH8SoQfTJAer1hYJUMQCgFWMRF2sJTwcMWiK/Z4VVG8KpXhPOmE16xYzdh554XgKop7Fi+CqwHzR9LSbf/4JBmu2IoHVt6ozCcx2/W6KqtcH4dtyu4t123wcGWmjrrTCUBwZe0OOl1wxbTUTTmX8aCLKSCBAAj4IoBf/xcko39hDuFqQT6wgAAInCQAsdVJFnoJ96I6DZRBAASsJID3Zd+0d7PoSoQX3LJli2x0442tqVv37r474AgIgEBABPTPj24WXHkTWvHnZQisArpM0AgEHEdACbDM4isWXaXVS4vrH2tGezPV52mEFAyNtJRJiT/ZuezNiuioEFNlCm9Vu4UXq2378mjb/lzakS4EV8Kb1WEhtsoWIiunJA5JWDw1kUoK0VWVUxKpStkkqlY+iaqWS6JiLMgSL/Z6xSItFm1x+EIk7wQguPLOJSLWeBBcWeHdijdDCaDMG2O263VVVrk+DtuU3Vuu2yC4MlNH3ekEILiydgfdILhiYurGnMsQXTEFJBAAAZ0Af3HCX5qoX6/hV2o6HU9vgXxkyJAhhF/oejJCDQTinYD+UD8tLY1Gjhod70iM9Zu9BeJe1ECDAgiAQBQJ6J6t8L7sHfSuXbtkeMGtW/NFV61bt6au3SC68k4LVhAIjEA8CK70NSoquL9TJJCDgPsJ6J991WrxPaIiEflcPdeB4Co0tnl57J2KRKjAPBkqcOeBPCGwEuEChYerg+LFIquMrDwZMpBFWdzeKYkFVMlCSZUiRFUccrCEEF6VKpYoPV6xp6tTRdjBU8tynijaJAgvWUJ05ZTFWTxPCK6iCDweBFeNp7XzSzAS3q34BEoAZT6Z2a7XVVnl+jhsU3ZvuW6D4MpMHXWnE4DgytoddIvgyvygC15arL2OcDYQsDMBFlnxAymV8CWJIuGZm79UxhfKnnxQA4F4JoCH+oXvvvleFO+hhTNDCxAAgdAImH9IALGVf447d+6kiRPG09atW2XD1m3aUNeu3fx3wlEQAAGfBPTPjW70cKWvjyHgns7npYADIOB6AmbhFb5PjM6WQ3AVGtccEQYwR4QBzDxGlCnCA27dl0ub9+bQpj2c59LhzHxvVjGMFBjawvz0YjFVkgg9mCqEVezpqlr5ZKpVIYlqVkySoQdZkJVyIjQhvF15goTgypNHRGtuF1wF4t1q8YC5EWGqBFDmwcx2va7KKue+qsy5XvZ3DIIrM3XUnU4Agitrd9AtgitFTd2gqzq8tCgSyEEgPgngy5Hg9h1fLgfHC61BwO0EzIJVPNT3v+Nm0RV+AOCfF46CAAgETwDvy8Ez4x47d+6Qnq62bdsmB2jT5ia6tWvX0AZDLxCIcwL6Z0a3Ca7wnWKcX9xYPgh4IWC+94LoygukME3qvRceroIDyR6sOEzgVhE2cIsQWO09lF8/mJlHh06EDmSxlZAbuCpx6EAONViyaH6owbIlEqls8QQpvqomQg1WKp1IFcSLvV0hnSQAwdVJFhEvuV1wZZV3K94YJY4yb5LZrtdVWeX6OGxTdm+5boPgykwddacTgODK2h10m+CK6elf/nAdv0ZjCkggEF8EzL/859WPHDWK0tLqxReIEFZrfg/lIfA+GgJIdAEBBxPw9h6KL5YD31D1hbHqgfdQRQI5CIBAqATwvhwquZP9duxg0dU42r59uzTedNPN1OXWW082QAkEQCAgAvrnRTcJrvR1sWie7984RwIBEAABJgCvz9G7DtTnZwiu/DNm4RRrp7Jy2HMV0Tbh0Yq9Wq3ZkUNrtuXQ4WPHRdhA1hb4H8dtR1lYxV6t6pyaTGdUFh6vhLermsLrVfEiCVREhBlkcRaLtOI92VpwddFFF9l6f/hXO/xh0ldys+DKSu9WzFcJoMyszXa9rsoq18dhm7J7y3UbBFdm6qg7nQAEV9buoBsFV0xQ/6JEEcXDLkUCOQi4l4C3h1HskaV9hw4QWwWx7WYvLdyV30P1XFbwBwRAwHUEzJ4BeYEQWwW/zbgXDZ4ZeoAACBQk4O3ellvhfbkgq0AsLLaaIMIL7lCiq5uF6KoLRFeBsEMbEFAE9Hsctwiu9DXBQ6naaeQgAAJmAvpnZdyLmemEXofgqnB2rKHiEIIsttq0O1eGDdyZnks7D+R7tWJvV9m5xylXtIkzvZUUU7Goqkxx4elKeLuqKLxbVSqTSDVEyMGaFZKpVLEEKpqSL7wqnLR7W9hacDVx4kRbk//888/p66+/9jlHtwquAhFb9WvYifo16uSTTbAHlADK3M9s1+uqrHLuq8qc62V/xyC4MlNH3ekEILiydgfdKrhSFPUvTZSNvzzRX8qOHARAwJkE1IMonj2X9YQvQHQawZe9vYfyKBBfBc8SPUDArgTU+yZ/eazKaq4QrCoSoeX8Hsppzpz8XI2C91BFAjkIgIA3Auq9GO/L3uiEb9u+fZsML6h+pHzzzW2pc5cu4Q+MEUAgTgjonxHdILjS1wOxVZxcxFgmCIRBQPd0he8cwwCpdYXgSoPhpchhAfPySHqwOpR5nP7amE1/ite+wyKMoAgpGG8erbwg8jCVLJpApYsl0tnVkum8Wql0atlEIcZKoFQhuhKOsCghTr1dQXDlcZkEV4lXwdWA+SNp6bZ//MJaPGCu3+PBHlTiKHM/s12vq7LKua8qc66X/R2D4MpMHXWnE4DgytoddLvgimn6etilSCvxlaojBwEQsDcB9RCKZ6mX9VnjSw+dRvhl/Uto82j8HspJ5ebjqIMACNiPgP7eqZfVTCG0UiQik+M9NDIcMQoIuJmA/l6sl9Wa8b6sSEQm56gQHF5w586dcsC2bdtRp86dIzM4RgEBlxPQ72ucLrjS18LbNnPmTJfvHpYHAiAQCQJ9+/Y1hhk5ahQ86hs0QitAcOWbG4utMrOO05Gjx2n9zhxavytXhhLkcIJsPype8ebRyjet/COpyUSpIpRghdJJdCp7uhLhBWtVTJaer8qVFMIrcSweEwRXYex6PAquYuHdirdIiaPM22W263VVVrk+DtuU3Vuu2yC4MlNH3ekEILiydgfjQXCliHKIrBXLVxTwMqCOIwcBEHA2AfUgileRllbP2Yux6ez5C2n5XrpihU1niGmBAAiESoDfQzkhBGuoBAvvx++hnMwerwrviRYgAALxSADvy9Hd9a1bt8rwgruU6KqdEF11gugqutQxuhsI6CIlpwuuOnfqaGzJkCFD8CMigwYKIAAC/giwOJ49XXHi+7WRo0bLMv6ERgCCK+/cWGyVI8IE7jt8nHYfzKPf12XRsg3ZdDRbCK3EC8k/gUShq0oSf2pXSqK0Gil0mshrVkwSIQYT49LTFQRX/q8Xv0fjUXDVeFo7v0wiHUpQnUwJoFRd5Wa7XldllXMfVeZcL/s7BsGVoo3cLQQguLJ2J+NJcGUmq4QDbF8B8YAZD+ogYGsC6gEUTxLigNhslRIOsACLE95HY7MPOCsIhELA/B7KY0CoGgrJ0PuoHwLgPTR0hugJAm4igPfl2Ozm1i1b8kVXu3bJCbS75Rbq2LFTbCaDs4KAQwi4RXClr4PDPauQzw7ZBkwTBEAgxgQ49DO/OMHLfnibAcFVQX65IoTggYw8IbbKow3CqxW/tguvVtv350oRVo44juSfAIcO5FfZ4onC21UiVS8vBFfC21W1cklU5ZQkKiFCD8aTrysIrvxfL36PxpvgKhDvVpEOJag2QImjVF3lZrteV2WVcx9V5lwv+ztmR8EVzz0hXgOhqs1HHjIBCK5CRhdSx3gWXIUEDJ1AAARAAARAIA4IHD58mPre1keutEfPnnTDDa3iYNVYIgiAAAhYT+Crr76kmTNmyBO/MG06lStXzvpJ4IwgEMcEtrDoSoQX3L17t6Rwyy3tqUPHk15v4hgNlg4CXgnoQiUne7hS3q3q1q1L7N0KCQRAAASCJcBerlQoaCe/Hwa77ki3h+DKk6h4vE7Hco7Tlr25tGl3Lv35Xzat2JJDucLlVZ7FQispSDqhSvIQJ2kVrSgXYvjeMgpC+6AtkdcXi8Siq5oVkqlutWQ6v1aKDDHIXrDiRcoAwVUYV108Ca4CEVtFy7sVb5ESR5m3y2zX66qscn0ctim7t1y32UFwlZ2dTZs2baIlS5bQqlWr6MiRI1SxYkVq2LAhXXDBBXTKKadAgGW+OFD3SQCCK59oonIAgquoYMWgIAACIAACIOBoAhBcOXr7MHkQAAEHEYDgykGbham6lsDmzZuF6Go87dlzQnTVXoiuOkB05doNx8LCIuAGwZW+Bni3CutyQGcQiGsCemhBeLkK/VKA4OokuywhtOLwgTvT2bNVDm3ak0s7RHn3wVyhGWAtwsm20SyxCInD8RVLTaCiKQlUJIUoNTlBvIhSkhLEsfzjiSJn0RK7iuKMwyCyKExEQpR5tihkizXxurJyiI5mHadM8WI7t7VqPcyqeJEEKiO8XbGHq9Mrs7erZFkuXVx4ujoxf27n1gTBVRg7G0+Cq8JCCTLGaHm34rGVAIrLejLb9boqq5z7qTLnetnfsVgKrniO27Zto6lTp9KCBQvkr8EOHTpEubm5lJqaKn+ZWadOHerduze1bduWSpQooeNBGQS8EoDgyiuWqBkhuIoaWgwMAiAAAiAAAo4lAMGVY7cOEwcBEHAYAQiuHLZhmK5rCWwWPySdMIFFV3vkGjlsOT88RQIBEPAkoIuVnOrRRXm3gtjKc29RAwEQCJ4AvFwFz8zcA4Krk0QOHz1Oq7bm0KptObRGvP7bLVRKFicWTiUKFVWKEFedUiJRhuQrVSyBSopXCSFa4leKFF8lUHISUbJoK71EiY5CGiDFVDki5xeLq44czaPDx8RLrO3AkeO0X4RJzBD2nBOiK4uXJ0VXp5ZNpLpVk6nhGalUVQiwhIYsXzhm9WQsPB8EV2HAjhfBVay9W/EWKXGUebvMdr2uyirXx2GbsnvLdVssBVf//fcf3XTTTbRy5UrKysoyL1/WE4XEtWTJknTXXXfR4MGD4R7fKyUYdQIQXOk0ol+G4Cr6jHEGEAABEAABEHAaAQiunLZjmC8IgIBTCUBw5dSdw7zdSIC993N4wb1798rlsZerW4S3KyQQAIGTBJwuuNLnD8HVyX1FCQRAIDQCuperkaNGUVpavdAGiuNeEFzlC5PYkxWHEdywK1cKrfYeyqP0I9GLIcgiKfZOxZ6rpJCqqHiWX1SUxYvzkkUSpVco9gyle7ji9uzhikVZUqgkyiqxx6pcMWXp6UqUT3q3OunhKuOYEGGJFwuw8vM8IcrKr7MHrBzRmceIVioiPHbx+qqeIjxdVUmm2hWTqUb5RDqlZKL00iXFY9E6eQzHheAqDPjxILiyg9iKt0gJoMzbZbbrdVVWuT4O25TdW67bYiW4Ys9WTZo0IXa7HUhKSkqiMWPG0H333UfFixcPpAvaxCkBCK6s3XgIrqzljbOBAAiAAAiAgBMIQHDlhF3CHEEABNxAAIIrN+wi1uAmAvzjUhZd7du3Ty6rQ0churoFois37THWEh4BXbDkRA9X+vxnzpwZHgz0BgEQAAFBoG/fvpJDWloajRw1GkyCJADBFdE+4fVp2YYcWr45m7buy7UkhCCLrZKFYorFRxVLJxF7faosXpxzvZzwbsXH2YuVChnIWyu65SejoAxaLsRWnE5k+WVRYW0Di7FYbLVHCMp2HRCv9FzaKfKdIt8nPGCx6IpDDkYryWmLPyWFwKy8EFmdKTxdXXxmqhBeJQkRmXs9XUFwFcYVFQ+Cq1iHElTbowRQqq5ys12vq7LKuY8qc66X/R2LheAqIyODevbsSXPmzFFLFS4DE6SQqnLlylS0aFH5xcSuXbtkeEHVqFSpUjRv3jy65pprlAk5CBQgAMFVASRRNUBwFVW8GBwEQAAEQAAEHEkAgitHbhsmDQIg4EACEFw5cNMwZdcT2LhxI00U4QWV6Kpjp07Urt0trl83FggCgRDQBUtOFFwhnGAgu4w2IAACwRBAWMFgaBVsG8+Cq6PZx4XY6rj0bPXPpmxauyOHDmTke3wqSCo8C3ulUgKrckJoxOECSxVPoNLFRC5EV6VEXlrUZfhAIUYqnpoQFQESe8E6lsOhBvO9Wh3KzKNDmcfpoHhxmXNmwCK0dCHAOiYYZYn2kU7sqauYWCOHFDynejKdfmqyKIsQioKLPy1ZpOdh1XgQXIVB2u2CqwHzR9LSbf/4JdSvYSfq16iT3zaROKjEUeaxzHa9rsoq576qzLle9ncsFoKrzz77jDp37kwHDhyQS2ax1RlnnCHDC954441UpUoV+u2336S46pNPPqHMzEzZjv+w2Orrr7826iiAgJkABFdmItGtQ3AVXb4YHQRAAARAAAScSACCKyfuGuYMAiDgRAIQXDlx1zDneCCwceMG4elqPO3fv18ut1OnztS2Xbt4WDrWCAJ+CThZcKXPHeEE/W4zDoIACARBAGEFg4DlpWk8C644ZODq7bnilUOrt+XQ1r050gMUi5IinZTAqIoIpXdWlSSqVSmZKp+SSGWKJ8rQgCzI4mf97PmKw+pxHq3E6+MXe7tiLUR+LkIrCnEVh1Lctl9wETw27hICNMHokBBnRTrx8nidLDRjDmdWTqaLTs/3dMX2KC4/0ksJaDwIrgLC5L2RmwVXdgklqMgrcZSqq9xs1+uqrHLuo8qc62V/x2IhuOrXrx+99tprlJ2dLZfKXq2GDRtGffr0oZIlS0ob/9myZQs9+OCD9Pbbbxs2Die4bt064j5IIOCNAARX3qhEzwbBVfTYYmQQAAEQAAEQcCoBCK6cunOYNwiAgNMIQHDltB3DfOOJwIYNLLoaR+np6XLZnTp3obZt28YTAqwVBAoQ0EVLTvNwpc8d4QQLbC0MIAACYRBQYQXbt+9A7Tt0CGOk+Osaj4KrbOGx6fCx47R9Xx79LTxbrRGerXaLsHrs2SlSiQVTHCKPQ+ex1yb2alWhlAgXWCY/dCCXWWxVrEi+tMgOAqOcXA43KDxbZRyXIQZ3pufJ0IN7DuXSfuEJjMVX2aJNJEMOFhGerkoWS6BqwtNVvZop0tNVxdLMxg5EInU1CAGZEJ1EXrYW4PyGDh3qt+XEiRP9Ho/1QbcKrgIRWzH7xQPmWrYFvi5Ts12vq7LKebKqzLle9nfMasHVkSNHqHnz5vTTTz9Jvqx4bdmyJb366qtUqVIlaVN/8vLyiL1h8c3G9u3blZnee+896oCbDoMHCp4EILjy5BHtGgRX0SaM8UEABEAABEDAeQQguHLenmHGIAACziQAwZUz9w2zjh8C69evl+EFleiqc5cudPPNEF3FzxWAlZoJ6KIlCK7MdFAHARCIVwIQXIW+8/EouDosPDbt2J9La3fm0LL12bRhVy7lCDdP4pF6xFJyElGKEBOxR6vThDer2pWEV6sKSVJ8lZIswgsKRRYLsqLpySrYxbAgiBmwx6uc3OOUmSVEaYLTVma1LZfWC04Zx/LEK3LSIeXNi8VntSom0xlVkuncmslUU7ByU4LgKozddKPgikMIcijBwpJVoQTVPJQ4StVVbrbrdVVWOfdRZc71sr9jVguuduzYQa1ataLff/9dLjMlJUV6tpo+fbqsm//8+eefNGDAAPr555+NQ48++iiNGDHCqKMAAjoBCK50GtEvQ3AVfcY4AwiAAAiAAAg4jQAEV07bMcwXBEDAqQQguHLqzmHe8URg/fp1MrzggQMH5LK7dLmVbrr55nhCgLWCgEHAyYIr9VC/bt26NGTIEGNNKIAACIBAuAQmTZpEHFowLS2NRo4aHe5wcdVfvTfXr1+fHn/8cVevPVeIiY4J71a7hDerf7fm0BoRSnDj7hzaczBySqviqQlUQnhsKs8erUon0anCoxWLriqJvLzwasXHOV6eE/w3scerdOH1a//hPCG8yqMd6bm0W7DafTCX0o8Ij1fiWKTcNhVNSaBTBLMaQmh1fu0UGWKwlOBYjHm5IEFwFcYmuk1wZVexFW+REkeZt8ts1+uqrHJ9HLYpu7dct1ktuNq7d6/0aPXbb7/J5SYKCeytt95Kr7/+uozvamawZMkSuu222+jvv/82Dk2ePJkGDRpk1FEAAZ0ABFc6jeiXIbiKPmOcAQRAAARAAAScRgCCK6ftGOYLAiDgVAIQXDl15zDveCOwbh2LrsbRwYMH5dJvvbUrtbnppnjDgPWCAEFwhYsABEAABAoSgOCqIJNALfEkuDqafZwOZh6nDTtz6dc1WbRWhBJkL04cYjBSiUMGcni8009NprpVU0QowQQqKkRDqezVKkl4tXKQfojFVCxSY+9fWTkkWW3clSP5rRPewdYJjvl6ivDpMZck8aeCCCd4bq0UOrtaCtUoL0IwirobEgRXYeyimwRXgYqtGlStT9NuejQMaqF1VQIoc2+zXa+rssq5rypzrpf9HbNacMVhAlu0aEFff/21MccLL7yQpk6dSpdccomH6CorK4tefvlluvPOO422HIKQwxE2adLEjAt1EJAEILiy9kKA4Mpa3jgbCIAACIAACDiBAARXTtglzBEEQMANBCC4csMuYg3xQmDt2rU0YcJ4OqREV12F6KoNRFfxsv9YZz4BJwuuOnfqKBfRpk0b8W+3DbYUBEAABCJGAIKr0FHGg+CKQ+RxqLy9wlPTBiEYWrs9l1ZuzZZhBflYuF6aOCxeqWKJVFaExateXoQOFOEDa4i8uhBelSiSHzqQ2zg5sSQtV3i82nkgl3am58nwghuE6GrPoTzaeyjXCEUY7hpLFE0Q4QST6TQhWKtXPZnqnJpEKUKoxiEanZwguApj99wiuApUbMWoWGzFoiurkxJHmc9rtut1VVY591VlzvWyv2NWC654Luyh6pFHHqGMjAyuUnJyMnXs2FG64j3zzDOpaNGilJ6eTt9++y0NGzaM1qxZI9vxn+rVq9PmzZuNurdCrnjX3L17N23dupWOHj0qRVwlS5akWrVqUZkyZbx1gc1FBCC4snYzIbiyljfOBgIgAAIgAAJOIADBlRN2CXMEARBwAwEIrtywi1hDPBFYK77jlKKrQ4fksrt27UatIdyIp0sg7tcKwVXcXwIAAAIg4IXARx99RPzi9O7s97y0gMkXgXgQXOUIsVWW8G61YVcuLRaerdYIz1bpR/Io85jQAvgCE6CdhVTs6KRWxSQ6o3K+UKiOEFyVFuHwUkWYPKEVEscDHMzmzViYlpUrvF1lE20X4QU5zOA/m7Jp+eZs4QHrOHEIwnBTsnBoVSw1kSqLMIxNzkqlc2umUMmiRMWFcM3JCYKrMHbPDYKrYMRW/Rp2on6NOoVBLPSuShxlHsFs1+uqrHLuq8qc62V/x2IhuGI32q1bt6aVK1caS+bQghdddBHdeOONVLFiRVq1ahW9+uqrdOjEFxDcMCkpiYYPH05jxowx+pkL/Gux//3vf9IL1jfffEN79uyR/Vioxeds0KABXXXVVVSpUiVzV9RdQgCCK2s3EoIra3njbCAAAiAAAiDgBAIQXDlhlzBHEAABNxCA4MoNu4g1xBsB/mEphxfk+yVO3bp1pxvFd5ZIIBAPBCC4ioddxhpBAASCJQDBVbDETraPB8HVgYzjtEMIhNZsz6E/NmTT1n25UiDE4fJCTRwCL0GIg04pkSjCBiYRi6xYcFVFCIUqleEQgs4WCPnjwsKrAyI048GMPPp3Ww6tFq9dwvPV7oN5IjyjCEEoRFmhCtmYGgvUygqu9YXY6pzqKVLMVqVsoqOFaxBc+buiCjnmdMGVU8RWvA1KHGXeErNdr6uyyvVx2Kbs3nLdFgvBFc/1tddeo/79+xOHDdQTK2k5qTnqx6699lopwmLxlDlxqMIPP/xQHv/ss88oJ0e8K3pJZcuWpc6dO1OfPn2oUaNGXlrA5HQCEFxZu4MQXFnLG2cDARAAARAAAScQgODKCbuEOYIACLiBAARXbthFrCEeCaxevVqIrsbTkSMnRFfdhejqRoiu4vFaiLc1Q3AVbzuO9YIACARCAIKrQCh5bxMPgqvNe3JpmRBaseCKxVbpQijEoiF+hZo4xB2HuqtbNZnSaqRQNRE+sMopidITE4utWJDl5sTerFhYlR9SME94ucqhv4W3KxZhZWYdJw7VGGpidEWEd7CKpROpRsVkanJmihRfsd2p3sIguAr1ahD9nCy4cpLYirfIm7jIm11vp8oq19uzTdm95botVoIrDvU3bdo0GTLw2LFjPH2/qWnTpjR27FjinEMQ6onXM3v2bOn56t9//zXWrrfRyxyykMVb7CmLPV4huYsABFfW7icEV9byxtlAAARAAARAwAkEILhywi5hjiAAAm4gAMGVG3YRa4hXAqvFd5gcXvDIkSMSQfcePahVqxvjFQfWHScEILiKk43GMkEABIIiAMFVULg8GrtZcJUtwtxlilCCa7fn0i+rs2RIwYOZeXRM2EJN0rOVUP5UKsOerBLprCrJVLdaCpUvmUCliydSkvB6FU+JxVVHhV8YFrOxt6ste3Np2/4cacsRqqtQRW3MsWgqi66S6NK6qXRBbQ4tmODY0IIQXIXxr8KpgqsZS2bTjN9mB7TyWIYR1CeoBFC6jctmu15XZZXr7dmm7N5y3RYrwRXPNyMjgxYuXEiPPPKIDCHINnNKTU2ltm3b0sMPP0znnHMOpaSkmJvQ8uXLacCAAfTjjz8Se7oKJBUpUoQGDhwoQxSWK1cukC5o4xACEFxZu1EQXFnLG2cDARAAARAAAScQgODKCbuEOYIACLiBAARXbthFrCGeCfz77yrp6Yq/I+XUo0dPuqFVq3hGgrW7nAAEVy7fYCwPBEAgJAIQXIWETXZys+CKvS1xmLtVW3Pot3X5oQRZBBTgY3CvUJVnq/OFAOiCOql0qhBdsSemIsLPSbLweOVUD0xeFxuAkcMyMs8DgjWHbmQvV79vyKL9h/Mo49hxCjVsI3NkcRuL2M6vlSK8iCVTjfLsRUy4FnNgguAqjE1zmuCKvVrNXPIecR5IalC1Pk276dFAmka9jRJAmU9ktut1VVY591VlzvWyv2OxFFzxvFgglZ6eThwGcNKkSfTXX39JW2JiInXt2pUGDRpEaWlpxAIpFW6Q+6mUnZ1NQ4cOpalTp3qEESxTpgx169aNunTpQvzA54knnqAffvjB4ML9K1SoQG+++Sa1aNFCDYfcBQQguLJ2EyG4spY3zgYCIAACIAACTiAAwZUTdglzBAEQcAMBCK7csItYQ7wTWLWKRVfjKDMzU6Lo0VOIrm6A6Crerwu3rh+CK7fuLNYFAiAQDgEIrkKn50bBFXtVYh9W2/fn0mrhdYlf7H1pnxABhZNYBFS+VKJ8XSjEVhfVyfe6VEx4Yoo3oZWZIwurOMzgyq3Z9Od/2cRhHLeJ8I1HhTex4+JYqD7FiorQgrUqJtFppybLsIIcwlHIHxwXshGCK/MVE0TdSYKrYLxaMQI7ia14PkocxWU9me16XZVVzv1UmXO97O9YrAVX+npHjBhBTz/9tPR8xQKrbdu2UWHep9auXUv33nsvLViwwBiqWrVqNGvWLGrWrJl448r3f5iTkyNFWUOGDPEQZo0fP57uvvtuKl68uNEfBWcTgODK2v2D4Mpa3jgbCIAACIAACDiBAARXTtglzBEEQMANBCC4csMuYg0gQLRq5UoZXlCJrnr26kXXX38D0ICA6whAcOW6LcWCQAAEIkAAgqvQIbpRcMUel1gAtFqEufuZQwnuzKG9Qmx1VIS/CzVJj0vC5VJ94WmpXo0UKQKqWSGJUpMT4i6MoDeGUuQm8O45lEc7D+TSP5typKerA0fyKFsIsfh4KIlDC5YqlijDNzatW4QanJ4imacKj2JOShBchbFbThBcBevVinHYJYygvjVKHKXbuGy263VVVrnenm3K7i3XbU4XXH3yySd0zz330IYNGwx8gwcPlt6yzB6xWHR13XXX0XfffWe0veyyy6SXq1q1ahk2fwUeIytLBHTVkl3FWuz9a/fu3bRv3z46evQoVa5cmapXry5nzsf4pVJSUpL0IqbqTs4huLJ29yC4spY3zgYCIAACIAACTiAAwZUTdglzBAEQcAMBCK7csItYAwjkE1i5coUML8jf4XHq1as3tbz++vyD+AsCLiEAwZVLNhLLAAEQiCgBCK5Cx+lGwVWmEFYdOXqc/tmcQ4tWHaOte3MpKye88HYliiZK4U+D01KkZ6tTSiRQ2RKJce/ZynzlHRMerY6KR+f/bM6mpeuyaNv+PNonRFjMP5QknIpJj1YcWvDSuqnU8PRUOkVwLyv4OylBcBXGbtlVcBWKyEphsKPYiuemBFBqnio32/W6KqtcH4dtyu4t121OF1y98sorUnDFD3VU+vPPP+m8885TVY/87bfflqEKlbFmzZr0xRdfUN26dZXJb87ikrlz5xptkpOT6eGHHzbqdijk5uYSuyP/5ptvaMmSJbRS/Epu//791L9/f2IPXyy0+uWXX+RxNd/69etTu3btVNXROQRX1m4fBFfW8sbZQAAEQAAEQMAJBCC4csIuYY4gAAJuIADBlRt2EWsAgZMEVqxg0dU4OnbsmDT26t2HWrZsebIBSiDgcAIQXDl8AzF9EACBqBCA4Cp0rG4UXO05mEdbRDi7lVty6M+NwrHGwVzKE3qfcLws1amUTLVFWLuzRUi7s6slCy9LRCnCu5WzZD+hXyeB9mTPYnkC9LZ9ebRxdy79K0IMLhfCt4OZeSHzZ+9iHLbxnOopkv0ZlZOpTqWkQKdki3YQXIWxDXYSXLHI6vety+n3bcuJy6Eku4qteC1KAGVel9mu11VZ5fo4bFN2b7lus4vg6siRIzR8+HB68cUXpTem1NRUWrZsGZ111lnEoiZf6eWXX5YhBXXB1fLlyyktLc1rl3fffZe6dOliHKtRo4YUXJ199tmGzV/hnXfeoW7duhlNOPRhRkaGUbdD4Y8//qChQ4fSokWLSLki53mxMGzs2LHErKdMmUIjR440pstM3nzzTaPu5AIEV9buHgRX1vLG2UAABEAABEDACQQguHLCLmGOIAACbiAAwZUbdhFrAAFPAvy95sQJ4w3RVe8+fahFC4iuPCmh5lQCEFw5decwbxAAgWgSgOAqdLpuElxJH0riDwt9lgsPS2tESMH1IpzgocwQvSsJoU8RIaoqXiSBzq2VIsIJplDVcklUrZyIc4fkl8Ah4WEsXYQTXCHEVkvXZ9PO9Fw6cuw45eSGthcpSQlURXCvUT6JGpyWSufXTpHexUSUR0ckCK7C2KZYCK50YRVPPVRxlb7sBlXrU99GHYlzuyYlgDLPz2zX66qscu6rypzrZX/HYim44jlu2rSJ3njjDSyIBWsAAEAASURBVFq6dKkUWG3evJnyRIDaxEThXu/SS+nMM8+k5s2b0/XChXaZMmXMiGj+/PnSwxWPo9KIESPo0UcfVVUj53FbtWpFn332mWG7+OKLiUVYtWvXNmz+CmYPWSy4Uq6+/fWz8hjz+uqrr4xrQJ1bF1xNnjyZmJNKt956K7311luq6ugcgitrtw+CK2t542wgAAIgAAIg4AQCEFw5YZcwRxAAATcQgODKDbuINYBAQQLLhYf9CUJ0lZWVJQ/26XMbNW/RomBDWEDAYQQguHLYhmG6IAAClhCA4Cp0zG4SXLEXK36xR6Uf/z1Gm3bl0n4h+uEwd6GkZKGrqlgmiSqXTaTzWHBVM4VKCPEVC7CQ/BPIFsKqrByi/4T4bdXWfOHbxl05dFgIsUJJQvJAJYskUrmSiXRZWipdclaq8DKWQCkOcXQFwVUou36ij9WCKxZXDZh/0uNOGFM3urLIatpNBYU3RgObFJQ4yjwds12vq7LKua8qc66X/R2LleCKxU/Tp0+n559/Xoqu2EsU2/SUIPzssXersmXLUpMmTaSI6vzzzxeqz/z/DLg934gMHjyY1q9fb3StU6cOzZgxg6699lrDxjxeeOEFuv/++40vK/jgqFGj6IEHHqCSJUsabf0V7C642rNnD1WtWlWGDVTraN++PXXv3l2GWWRhGXu4guAq/xpS11JhObNUbcxlb3VfNra7LUFw5bYdxXpAAARAAARAIHwCEFyFzxAjgAAIgEAgBCC4CoQS2oCAMwnw9y0cXjA7O1suoM9tfeWPUp25GswaBPIJQHCFKwEEQAAEChKA4Kogk0AtbhJcHRMCn6NCXLVsQzZ9u/wYbd+fS9k5x8Wz80BpnGzHj9GLpCTQWVWS6UzxqivCCHIZKTgCew7lyfCCK7Zk0x9iX7ieK1RxQnIQdGJvVv9n7zzgoyjaP/4kuVQg9N47hN5VxEYTFAUB4bWhvmJDQXytiIKoKKiIYkOKihUVyx8LSBdQmkiR3mvogQDpCf/5TZhlbnN3ub3cXe4uz3y8TNmZ2dnfHPGy993fU0xAV50SoumKRlEUHxsmX5YnKoQBDFwVQPRgBq6CwdVK3xoFR+ltKJvb9boqq1zvjzbV7ijX2woDuMKNggcffJA+/vhjY53ma3dUr1SpEn3//fd0+eWXU1JSEo0fP54++OADOnPmjF13gDEVK1ake+65h/r27SsBo0mTJsmxOtQFxyy4OsH1yt0U6MAVrkcPeQhADRpBM5UYuBJxiS9Ce+7m0E71NZcd1Z21oT3UEgNXobajfD2sACvACrACrEDBFWDgquAa8gysACvACrijAANX7qjEfViB4FVg48aNMryggq7+e9991LVrt3wvCA+mxsXFEe6lcmIFAkkBBq4CaTd4LawAKxAoCjBw5flOhBJwdercBTp6Joc27s+kNTsz6KSAe3Lk9/3W9MHXnzbhnFQ8Jpwuqx8lQ9iVKREm3ZWszcS9AcCdF65Wmw9m0YodGXTopAgtKOpwwLKaJAQnXK2ai3CCzWtGUY1y4eIVHBZXDFxZ3W2tf7ACV4PbDqDB7QZoVxL4RQVAmVdqbtfrqqxyjFVl5HrZ1TF/A1fJycn09NNP0/Tp0+2cpszX7qxerlw5euWVV2jChAm0bds2oxtgGHXNRqMoIDShroc6FhUVRfeJmxRwuKpQoYJqzje3AlzhZoi+JqwxIiJCrsl8IliEnz17VoYnVFAYwhWWKFGCYmNjZffs7GzCSyXMFxkZKatZWVnSIWzs2LH04osvqi50++23S70qV64sz43zWwGuzOfExNAUzmPmhGuFUxnmx/Wgjr5YP67DvFY1HmvCy1uJQwp6S0n35mHgyj2duBcrwAqwAqwAK1CUFGDgqijtNl8rK8AKFKYCDFwVpvp8blbAPwps3LBBhhfEvT+k++4bTF26dnV68l27dklIq2PHK+muQYOc9uMDrEBhKMDAVWGozudkBViBQFeAgSvPdyiUgKv9J7Jpe2I2bTuUKcPYnU31wNpKSBkhwtcViwmjcvERdLVwU7pcQFf4ChbtnDxTYHtiFq0UwNXuo9l09HQ2paRbB65wZltEGNUqH0G1KthkmMfmNfN+1+7ZCn07ioGrAugbbMBVsLla6VujQzmu2vV+qqxyjFNl5HrZ1TF/AleAcD766CMZGvD48eP6pUoYB65UeAHMAbRz8OBB6WRl11FUdLgKQA9gosaNG9P27dtleEJzf3MdINPVV18t19GhQwfzYZd1d4AraL9p0yZauXKlYfuNNSPUIZymAB+pBE02b95Ma9asoYULF0qIDNASUs2aNenaa6+lLl26UKNGjWj9+vWyr7rBgqfUevbsSYDHFi1aRDt27KCffvqJfv31VzW9PF/v3r0pPj6eWrduTe3bt3cbuEpNTaXFixdLTdX7CfAU5klISLCDpE6dOkXr1q2j5cuXy9ehQ4cI64yJiaGmTZtSjx495FoQ7nD+/Pl04MABuUaAVri2Tp06GWsuaIGBq4IqaG08A1fW9OLerAArwAqwAqxAUVCAgauisMt8jawAKxAICjBwFQi7wGtgBXyvwIYN60V4wdeMBzHvG3y/vF9oPvOunTtp3PhxlCyiAeCe22RxH7Z48Uv3Ic39uc4K+FsBBq78rTifjxVgBYJBAQauPN+lUAKu4KK0ZlcG7TmWLcMJpmV4BvUglGANAfXUBtRTI5IaV7MRwtmJr6k5eajAkdM5tPtYlgThNgkHMriPeZIE0kCli0UIGC6crmwcRVc2jBIbI//zZDq/jWHgqgBSBwNwBciqdZUmQedoZd4WBbPk1673U2WVY6wqI9fLro75E7gCkPPoo49KIEetD2uDa1W3bt0kPNS8eXPpiHTixAn6448/CB80Vq1aZYBL6K8SIKKOHTtS//79qXv37hJYAtC1YsUKh/0xDmEEe/XqRQ899JAEgABCWUnuAFcbxJNnTz31FC1YsEBCR5gf8NTo0aNp4MCBEkJCG2CradOm0SeffCJhqvT0dDTbJThJQZNHHnlE6oaQgWlpabIPrv23336TANcg8cQajukOWHYTicqIESOk25U7DldYC+Ct5557jvBkHPYLYFfnzp3ldbRt29Zw6gJcBsey7777TkJyyqFLPz/26vrrr6fBgwfTa+IGEbRBAvz23//+l9577z29e4HKDFwVSD7Lgxm4siwZD2AFWAFWgBVgBUJeAQauQn6L+QJZAVYgQBRg4CpANoKXwQr4QQE8iDnutVelwz1ON/j++8V9ui7GmXcK2Gr8uNcI0QVU6n/rrdS3bz9V5ZwVKHQFGLgq9C3gBbACrEAAKsDAleebEgrAlfj6VYQOJPp7dwYt2ZROh07l0Lm0HMq6FPDIbYEAVhUToQRbiLB1TQVsVaNsBFUpw9ZWbgvopON54WiVdD6HNh3Ioj+3Yo9ERCrBXGHvrCQgCZEirGBcdBh1bR5NXZpFi4ckwsgW4FvEwJWVXTb1DVTgCpAV0n3tbiVVNi096Ko6fKQv3tyu11VZ5Rinysj1sqtj/gKuEF5v0qRJMtyd/od/qVKlaMiQIfTAAw9Q1apVDYgHawaQBOcnhBCcM2eOcUMBx+Li4ujuu++WIFK9evWkKxZgI8BOgJAAXcE5KikpSc4JF6yWLVtKJyXAXTVq1JBOWZjLSsoPuEKYQ4T0+/HHHwkOUUjly5enl19+me644w65brTB/QkA1NSpUx26eKGPnuCOhWuGixV0QfIVcIW1/f777zRy5EgJggGgApiG840fP57gCgZnMaR//vlHhoiEExb22FWCcxmcrPbs2SNf6MvAlSvFguMYA1fBsU+8SlaAFWAFWAFWwJ8KMHDlT7X5XKwAK1CUFWDgqijvPl97UVRgvXiYdZyAqtTDjvff/wBdJx6OxP1CwFZnz561kwUPnn44+SOP7oHaTcQVVsBLCjBw5SUheRpWgBUIKQUYuPJ8O0MBuErPvEBp4utVuFstFsDVsTM5lJl1QUJYVpQBbBUlYJ4yxcPpcuGc1LpOFMXHhVEJEV6QU8EUyBTwW5rYp+2Hs2jF9gzaK9yuklMvEPbOakJoR7iQXdk4mjqJV3xsmNynQN4lBq6s7rLWPxCAKwVUwcWqddUmcnWqTVtq0BcVHGW+EHO7XldllWOsKiPXy66O+Qu4Onz4MD3zzDP02WefGZcJaAduU1OmTJFQknFAKwD+Qag9OEbhSS6VihcvLtuef/551WTkAJ2OHDlCcMmCmxMstBHGr0KFCvIF1yhPkyvgCtf46quvSscqfMmEBFeoiRMn0l133UXFihUzTot1v/HGG4ZblToAAAlwGHKE6cM1qL0EsAQ9VF0Hrt5++21asmSJDGWI0IoqVa9eXYb/Q2i/fv36SejLlcMV5kYoxOHDh9Pq1asNxyyEbPxEOHHpzlZHjx6V0Bt+V6g1qfPC0Qp6IyUmJsp9QBl6AIxTTlwMXEGV4E4MXAX3/vHqWQFWgBVgBVgBXyjAwJUvVOU5WQFWgBXIqwADV3k14RZWINQVWLfuHxleUN2L6927D82fP5/OnbOHrZQOd99zr3SdV/VAzjdv3iSXt3nTZvEgbW4ZDXio1t2UkJAguyYk5H6XgEpCkwRxf/RS3d25uJ/3FWDgyvua8oysACsQ/AowcOX5HoYCcAVw5/T5CxK4Wro5XTopeaKILYIEvBNOlUtH0FUJ0dSubqQMI2gx0JMnp3Y6Bg5QGQIeA7DkaQKgBFcoYQQlr8fTebwxbu/xbOlEtiMxiw6dzKazYu88SdirtnWjqG29aKpSCnsWXujX5uo6GLhypU4+x/wNXGE5fx/+N2Rcq/KR1+6w+gPZrlFUzO16XZVVjrGqjFwvuzrmL+BKhRNctmwZliMTIKBvv/2WbrzxRtXkMMcXNkOHDpXAj7ouwEeAmOAQ5Sqhv9Wwga7mcwZcwbULYNW4ceMoJSVFTgGgDG0Io4drVQlPnXXt2pX27dunmqRDF+Czhx9+mBo0aECAwgBGLV26VDqDwUnKnHTgChrB+QrAF0AulRDC8IUXXqCKFSvKNcAlyxVwtXv3bkJ4wr/++suAoqpVqyYdxgBdKWcrzI95J0yYYMBUaINLGUIE9unTR0J00B4g2jfffCPDJwIi0xMDV7oawVlm4Co4941XzQqwAqwAK8AK+FIBBq58qS7PzQqwAqzAJQUYuLqkBZdYgaKkAO4TIrygO6lKlSo04a2J7nT1ax8AVQqssgJUFXSRALIUfNWvf/+CTsfjLSrAwJVFwbg7K8AKFAkFGLjyfJtDAbhKTMqh/QLe+XdfJq3bmykgHhGrzoMUFxVGtSvaqG4lmwgnaKP6IidASh7M5Y0hQJHgAAU4ac9Rz4mrCiXD5XWVFG5dcPCCk1dhpZNnc2jfiWzadiiL1ou9Ono6mzxBrgCRNawSSQ2rRlKDKhGibCvU68pPTwau8lPIxfHCAK5cLCekDymIyHyR5na9rsoqx1hVRq6XXR3zF3D1xx9/0P33308IuacSwgnCiQrQTX7pzTfflGH5Tp8+LbsC/OnduzfNmjUrv6FePe4IuMIXSjNmzJBglbL0hqvWmDFjZNhA8wJuueUW+umnnwz7b8BjCKkIWArOXeYEoOXxxx+XT6upfUUfHbhSYwBBvfTSS6oq4anXXnuNKlWqZLQ5A64++OADuvXWWwn/9lUCqIV6ixYtVJPMDx48SP3FTQmEblQJYQ8BnOH6cP16QrjBr7/+WrqcAcBSiYErpUTw5gxcBe/e8cpZAVaAFWAFWAFfKcDAla+U5XlZAVaAFbBXgIErez24xgoUJQV+nv1/9Pnnn7t1yY888ihd2amTW3191QmAFWAbJHcBq9Qml76cS2uS5dbSYjblRjaI3WR/b9LV4H79csErBrBcqeSdYwxceUdHnoUVYAVCSwEGrjzfz1AArnYeyaZNB7IEmJRJu45mUWq6JwgPUam4cGojXK2a14ykKmUiqHy8oHoKMcHd6mzaBVr0bzr9uTXD45U0rGqTIRKrlY2gWBGKD+5QhZUAkJ1LE59lD2TK8I9wvMqRTIa1FQEawx5VKWMTTleR1LZOpDA8sTaHP3szcFUAtRm4KoB4FofqEI0+1Nyu11VZ5Rinysj1sqtj/gKuFixYQPfddx/t3bsXy5EJEBDgG3ccqCZPniwdlY4dOybHAri6+eab6fvvv784m38yR8DVzJkzZbg+hPtDAjT10EMP0ahRo+zCCOIYwh327NmTFi9ejKq89s6dOxPmLV++vGxz9APX+cQTT9CePXuMw94ErqAlQh5+9dVX8r2DPalZsyYBwoIblxmgmjZtGj333HOEsIJIGPvYY4/R6NGjpTuXsUitgGsHDIbwh8oFjIErTaAgLTJwFaQbx8tmBVgBVoAVYAV8qAADVz4Ul6dmBVgBVkBTgIErTQwusgJFSIFt27bKsILq/lp+l15fuOm/9NLL+XXz+nEFWbkCrBRUBaAqtWkuYJWmgVYFXVTMRfAq9t/cb+cAZLmCsRi+KqjirsczcOVaHz7KCrACRVMBBq483/dQAK427s+k1Tszad/xLDpyOke6QllRBCEDAfCUE4AVQgm2qRNFJWLDqFh0IVpBiQsAcIVwiXPXpdGSTelWLsmub5PqkXRVk2iqUS6C4OJVmMBVlviompl9gbYfzqY/tqTTriNZlCIAOYRNtJKwZwDkShUXe9Y4Su4bXK8CNTFwVYCdYeCqAOJZHKrgKPMwc7teV2WVY6wqI9fLro75C7j6888/6cEHH6SNGzcal1miRAnaunUrwdraVYJrFNyiECoP7kxIAID69esnXZNcjfX2MTNwBfAL4QLVDQ6AR3feeaeErXRXKbUOOEIBPNu0aZNsApz11FNP0fPPP6+6OMwBcw0YMIB+/PFHwxnLm8BVmTJlKCkpyXjf1KpVSzqK9e3b1y4colocYDLsh7puuFvhd0b9+vVVF4c5QLMhQ4YYT7IxcOVQpqBqZOAqqLaLF8sKsAKsACvACvhFAQau/CIzn4QVYAVYAeGEPY+mTpkilXj/gw8Jf9tzYgVYgdBWAPdSx497zbgn5+7VPv3Ms9SqVSt3u3vcLz/ICoCVgqu8CVZZXTBALEBYrgAshq+sqpp/fwau8teIe7ACrEDRU4CBK8/3PBSAq1U7M2jJ5nRCaMHklBwC1GMlAbaKFKH2KpUKp+6tYqhD/SgJYBUubpULXJ0RwNXPa1Jp/gbPgSs4dnVuHk21ytsoTkBkkYXocKX2ZbcIkbh8W4YAr7Lo5NlsCV2pY+7mCI8YHUnUvWUMXS9ehQmS5bdmBq7yU8jFcQauXIjj5UMKjjJPa27X66qscoxVZeR62dUxfwFXeJIJDkjz5s0zLjMqKkpCO48++qjR5qiQmJhI6KOHDwSoA3Dp3XffdTREtuk6AIzyRjIDV+Y5EYIPrlB9+vQxH5J1OEONHDlShlJEA8CmhQsXEoCl/NLgwYPp008/JYTnQ/ImcGU+N8I1Tpgwwem6hg8fLrVXrl7t2rWj5cuXE8IjukqnTp2SYQvheIbEwJUrtYLjGANXwbFPvEpWgBVgBVgBVsCfCjBw5U+1+VysACtQlBVg4Koo7z5fe1FUYMuWLRK2gou81QTYCtCVr5Ir0AqQVdKAdAFaWfwG0VeLdTBv6ZlRsrX0N9F5jjJ4lUcSjxsYuPJYOh7ICrACIawAA1eeb24wA1c5whQJcNVf2zNowcZ0Op6cTekZF0SIOmt6ANyBW1I14QDVSbgltayd+z0tA1fWdLTS+9CpbFq/N5O2CeBq/3HxOfd8jpXhsi8crWwRYdSlWbQAymIETCa+MxchEwt73xxdCANXjlRxs42BKzeF8kI3BUeZpzK363VVVjnGqjJyvezqmL+AK4A2AI0+/PBDY20IWwdQZ4p4GrN58+bmy5f1tLQ0+vzzz2WoukOHDhl94uPj6cUXX5QQl9F4sXD8+HFatWoVAQQ5efKkiHsaTlWrVqU2bdrIJ7liY2PNQ9yum4Erm80mXZ1wwwMJ9dtuu41effVVh85dCKcHty7ogQRHKLh+ATzKLz3yyCM0depUSk/PJYG9CVw1ENbi+/btM+aGXi+//DINHDjQocPV0KFD6f3336fs7GwZFvG6664TT9bOz+8S5PyA0X777TfZl4GrfCUL+A4MXAX8FvECWQFWgBVgBVgBvyvAwJXfJecTsgKsQBFVgIGrIrrxfNlFUoEt4mHWccLZCvdKPU0IK4jwgt5MzkCrYICsnOmQH3zVr39/Z0O5PR8FGLjKRyA+zAqwAkVSAQauPN/2YAauAFulZxEtFaHp5q5Lp9MC2lHf7VtRJFaE2ataxkZ1KkaIcIKR1LCqzcpwn/VFSMFQdbg6kZxDO49m0fZDWbTpYCYdFaEgPUkCk6BrRLjEa5rEUKliYTIUJBzLAi0xcFWAHWHgqgDiWRzq7BeouV2vq7LKcUpVRq6XXR3zF3CFsIAzZsygZ5991nB3wroA3Fx//fUSxsKTVggVqBLCB8LVCuDPzp07jWvCcTgpAThCOL5rr71WQkFwfvruu++kw9SBAwdkiDzchADYhVB/5cqVoxYtWtCwYcPo8ssvl+3qXO7mZuAK68e/lR49epB6ugyhEseNG0f33ntvHpAK47FmBY9Vr16d8GGqZcuW+S7hrrvuoq+++oqUq5Q3gSuEDgRkNWnSJEPnhIQEgiNXhw4d8mj1xBNP0DvvvGO4bQGYW7lypUM4S7+wPXv2yJCLcMNCYuBKVyc4ywxcBee+8apZAVaAFWAFWAFfKsDAlS/V5blZAVaAFbikAANXl7TgEisQygoAahr32mvGg5KeXuuVV3aiR/KJNODu3KEIWjm6dsBXzlyvGLxypJjrNgauXOvDR1kBVqBoKsDAlef7HszA1fn0CyIc3QVafTGkIMIJepLiY8OpQRWbBK0airyGcLoKhBTKwFWyCJV49HS2dLhatSOD9p/wzMUVwFV7EQKyQ/1oGRKyYslwgvNVoCUGrgqwIwxcFUA8i0MVHGUeZm7X66qscoxVZeR62dUxfwFXWMPBgwfpoYceol9++cVYH9oBT1WrVo1uuukm6tq1K5UpU4Z2794t4aKlS5fS2bNn7fpjDBLgrLJly1J/8VTRww8/TB999BF9/PHHTvtjDByo6tatKwEvOFFZDTXoCLjC+nBuOFCpVKpUKfrjjz+oWbNmqknmcMK64447aO3atbKOsIoAs+AWBTDMWcKXVgMGDJDOUGpvvQlcQYuJEyfSjTfeKN3BsA5o0717d/rss8+kzvraAMEBKsO6kACO/fjjj9S6dWu9W57yzJkzJXC2f/9+eYyBqzwSBV0DA1dBt2W8YFaAFWAFWAFWwOcKMHDlc4n5BKwAK8AKSAUYuOI3AitQNBTIyMigOXPm0Nw5v0k3/4Jc9YS3Jjp05Xd3TmegVdKt6SJsYIa70wRdP0fgFR5WTUhoQgxeub+dDFy5rxX3ZAVYgaKjAANXnu91MANXp85doH0C1Nm4L5NW7Uinc2kWYwlelK1M8XBqUzeSmlSPFE5XEVQ+PjCInVAGrtKzLsj92iYcrhb+m047E4VVmQcJVECzmpHiFUW1K0RQLfGyBcb22V0NA1d2clirMHBlTa+C9FYAjXkOc7teV2WVY6wqI9fLro75E7jCOmbPni0dpuB05EkClARIBzca4JqlUlxcnLTU1tvUMUd5nTp16M0336Sbb77ZJehkHusIuIKL1uHDhyX0hQ9GSvvy5cvLMH16CEMc69atm134Pbhtwf0LIJgj6ArX9MYbb9Arr7xCycnJxpK8DVwhdCNAMLh1ISyjSqNHj6bnn3/eDk4DOPXkk08SnMSQAI7dfvvt9O677xL2wlHCnHDGwnnUPjFw5Uip4Gpj4Cq49otXywqwAqwAK8AK+EMBBq78oTKfgxVgBVgBEvcW5tHUKVOkFO9/8KF8gI11YQVYgdBWYP68eRK+Ongw956c1avtLiIN3HPPvVaHyf7qS019cKiDVvq1ouwIvOrXrz9DV2ahnNQZuHIiDDezAqxAkVaAgSvPt199NmnatKmMluT5TP4feUSEodt+OIu2Cmhn3d4MShGOV1YTfDwqlIygTo2jqGWtSBGWLpyKxzg397A6f0H6hzJwlSXwhAwBXQG0AnAF8CpT1NFuJWGnGlSJpAYiDCTcyRqJly0wDMrsLoOBKzs5rFUYuLKmV0F6K0DHPIe5Xa+rssoxVpWR62VXx/wNXAG0efvtt2n8+PF2oQXN1+6oDiesNm3a0GvCQhtz/Pnnn3TixAnKzs5r1Ye+cL+CWxZC8AH2SUpKspsWsBWgK4BO7iZnwBX0hqPV/fffT9u3bzemu+WWWyRMhZCGKg0fPpw+/PBDCYihDZAV3L3GjBlD9erVk2H50AatAFjNnz+fXnrpJdq4caOaQubeBq6++OILQhhHuHWNGjVKOoWp9cG96oYbbjBCPp48eZL+85//0IIFCwx4qkKFChKmGzRoEAE2wx4gIdRjYmIiTRE3gRGyUIfGGLiSEgX1Dwaugnr7ePGsACvACrACrIBPFGDgyiey8qSsACvACuRRgIGrPJJwAytQZBRYvmwZzZk7h3Zo9yHduXjcc/xw8kdUsmRJd7rLPnC1GvPii3b9ixpoZXfxosLglVkR9+oMXLmnE/diBViBoqUAA1ee73cwA1cHTmbThr2ZtOtoFm05mEWpGdaAK8BW4eJHFeFq1a1FNLWtE0WRNhHpKUCAnVAGrvCOxfXtOZZFi/7NoM0HM6XjVXqmxT0U89SsYKOa5SOouXC6wisyQPZP/1fJwJWuhsUyA1cWBStAdwVHmacwt+t1VVY5xqoycr3s6pi/gSusBQAU4JsPPviAEGIPdVcJNwIQou+6666ThHKjRo0kGAQI6NNPP5XgFUAhlWJiYqhLly4ydF/79u0l2PT777/TtGnT7KAlODFNnTqVbr31VgMkUnM4y50BV+gPsAghDUeOHGk4RAE6AtSFUIoIZ4h07NgxuT4doMI1dujQQTpuAQADiAQw6e+//6bvvvtOhmOUg7UfvgCuMD3cx5577jmaNWuWdBJDW9WqVen777+ndu3aGS5cALOeffZZOnXqFLrIhJs1cMi65pprCAAWEty/Fi5cSL/++qsBmckD4gcDV0qJ4M0ZuAreveOVswKsACvACrACvlKAgStfKcvzsgKsACtgrwADV/Z6cI0VKIoKrF37twg1OIfWr1/v9uX37duP+ov7oe4kHZBB/9Qm2SJ0YDqliZxTXvCK3a5cvyv099PXM79x3TnAjg4ckPtvBg9O48WJFWAFWAFvKcDAledKBjNwtedYNq3ckUEHRFjBXUeyKM0irGOLCKPYqDCqXi6CrmsaLR2uwkU4uvDAMLiSQNKZ1Av085pUmr8h3eNNBoTUuXk01Spvo7josIACkgDNrdyeIYG5Y8nZdFZcr9UEYK5K6QhqXSeS2orQkFG2ANlA7UIYuNLEsFpk4MqqYp73V3CUeQZzu15XZZVjrCoj18uujhUGcKXWs3jxYvrhhx8IMNS+ffsoPT3dWDf6hIv/MwDgadGihXRXGjBgAFWrVs0AfuAA9csvv9Djjz9OO3fuxBCZEKJv+vTpBDBLT4B++vfvbwcIPfDAAzJUH9yw3EmugCuMP3LkCI0YMYLQD9eDVLlyZRlqr0+fPsbacRznxpdR5gQ3LIBIgMjUHOiD0ISoq3B8vgKucK7ly5dLXVevXi33BHvRs2dPeuedd6h27droIh2wECIQIJvZZQxwmXL1Onv2rLFmtAFMQ0hIJAaupAxB/YOBq6DePl48K8AKsAKsACvgEwUYuPKJrDwpK8AKsAJ5FGDgKo8k3MAKFFkF8FDr3Dm/0YoVK/LVoHjx4jT5oykuH0CFqxXgmM2bNxvzHR6TwqCVocalArtdXdIivxIDV/kpxMdZAVagKCrAwJXnux7MwNV2EY7uj80ZdCQpm/YL6Aoh6qyk6MgwKiNCCNYSDklXipCCCdVtFEioTqg7XGGvjoqwkBv2ZdBWERpyrwDoTp61GFNQzFEuPoLKlQinyxpE0RUNGbjK82/gqaeeytOmNyCkWiAnBq78tzsKjjKf0dyu11VZ5Rirysj1sqtjhQVcqWs9ffo0/fPPP7Rr1y4JKS1dulQ6XgHYgXtSs2bNqGHDhtSgQQMZak+NU/nMmTNp6NCh0jVKtQG2uueee1TVyKEJwvkhHKFKTZo0IThlIZSfOyk/4Apz4IkynH/dunUGrHTZZZdJWAkhEVX67LPP6Mknn6SjR4+qJoc53K8wLioqSjpeKQjLl8AVgCi4dY0ePdoI/QhYCpAYwg3Gx8fLtQKUGzt2rHQaU+tyeBGiEc5jffv2lfoA0kFi4ErKENQ/GLgK6u3jxbMCrAArwAqwAj5RgIErn8jKk7ICrAArkEcBBq7ySMINrECRV2Dv3r3S8WrRooUutbhr0CDxcOUNDvvoUAw6wNUqUcBWnFwrYAav2O0qr176e4sdrvLqwy2sACtQNBVg4MrzfQ9m4GqzCCP4+/p0OnE2m44m5VBmtjXgCm5PlUpFUJ2KNmpfL5IaVMmNsuS5mt4dWRSAqxMCsNomYKtth7Jou8iPnLbuABsfF07xseHUSUBz1zSJIoB0gZbY4aoAO8LAVQHEszhUwVHmYeZ2va7KKsdYVUaul10dK2zgSl0zwgoiFB8clFJTUyWIAwirSpUqhiuU6qvncFcaNmyYdINS7YCAatSooap2+U8//US9e/c22qpXry4dtsxuWEYHUwGAlw5zARhKSkqy6wXt582bR3fffTcBKENCv8GDB8tQfXDtQoIrFACzCRMm0IIFCyglJe+NC4xDeMSnn35ahvibMWOGEZYPTl6AxUqUKCHnw48xY8bQa6+9ZtRvv/12eumll6hSpUpGG5yzJk2aJPuqRoRV/OSTT1RV5mfOnKH//e9/9M033xhhHzHPCy+8QHfeeafxFBxCJCJc4Lhx46TTmDlEJICx8uXLyzH33nsvAUaF2xgSIDLoNHHiRFn3xo9Jn8+infsPUu2qlem/t/Qw3j9YBxJyvazaXOX6MXPZUd1ZG9pDLTFwFWo7ytfDCrACrAArwAoUXAEGrgquIc/ACrACrIA7CjBw5Y5K3IcVKJoK4AHPuXPnSPjK7EwPRXCPb+Lb7+QRR31xqQ4k3ZouQgjmOtWrNs6dK2CGrhISEuiFUaOdDyhiRxi4KmIbzpfLCrACbinAwJVbMjnspD63NG3alF5++WWHfQK1cf2+TJq9Jo3OpFygpHPZlGWR1SkRI8IJijB79SvZqEUtmwSvAulaiwJwlXQ+hxAaErDVxv2ZdEiEGLSaYkRYSISGRFjIri2iKYaBK3sJ2eHKXg+uOVdAwVHmHuZ2va7KKsdYVUaul10dCxTgCmt8/vnnJXwE8Aig0eHDh6lMmTI45DQBEgJwlZycbPQBxHTllVcadb0AQOu+++4zmmrWrCluPsyVLlpGo4sCYKUTJ04YPQDuOIK7cCMjMTHRLtQeHJ5Kly4tISM1AfYJcwISmzVrlgzld+rUKXn9zZs3l5ASPijAXQoAFMLzqb2FRhUqVJChF9V8ALzQTyWMwzkjIiJUkxwPvRQMhgPoV65cOaOPKuB8AMrUOXG9ALxKlSplQEvoC8gK/f7++28ZJnLTpk0SnEOoRgBj/fr1k6EVsWboB6gOCfPBxjy/fZad3fzBwJWbQnmpGwNXXhKSp2EFWAFWgBVgBUJIAQauQmgz+VJYAVYgoBVg4Cqgt4cXxwoEhAK4tzdHhBqcO2cO4TOanh5+eAhddfXVRpP60hINcLVKGpDOIQQNdawVzODVCyJiQEJCE2uThGBvBq5CcFP5klgBVqDACjBw5bmE6rNLMAJXa/dk0g8r0+hcWg4lp+RQtsVodCWFM1LdShHS2SqhWiTVKHfpe2DPFfXeyKIAXCWnXqBDp3KBq7W7M2jfcevAVWREGNlsYdSteTT1aCWAKwFfBVpih6sC7Ag7XBVAPItDFcxiHmZu1+uqrHKMVWXketnVsWAHruAMBeAKgI9Kffr0oa+//toObMIx3GBo3LgxHTp0SHWlG264gSZPnkxVq1Y12rgQ3AowcOXf/WPgyr9689lYAVaAFWAFWIFgUICBq2DYJV4jK8AKhIICDFyFwi7yNbAC/lEAD0sCugJ8dfz4cXnSunXr0itjX6XNmzcRQJjNmzfLdg4h6J09MUNXHGKQ5Pvsu+++lQJzSEHvvM9CYRZ8b/PWW2/ZXUr37t2pQ4cOdm3uVHbu3Elffvllnq7//e9/PfoOCJE91qxZY8yHB9GHDh1q1AtSQPSSDRs2GFNcd911To0EjE5cCEkFGLjyfFuDGbhasyuTvv0zlVIyhDFHeg7lWASuShcPp8ZVbdRIvOoJl6sqZRi48vyd5NnI8+kX6NiZHNqZmEV/bc+g3UezLE8ULoxJwgVjdb2ArW5oEyPdrixP4uMBDFwVQGAGrgognsWhCo4yDzO363VVVjnGqjJyvezqWCAAV3CDgivTKPGkz9SpU2XIPISaW7lypXSegjOUCgFn1gguWI899hh99913xjUjZB/csvr27Stdm3LE/6X2799P7733Hn344YfGFDabTbYhpB3Oxyk0FGDgyr/7yMCVf/Xms7ECrAArwAqwAsGgAANXwbBLvEZWgBUIBQUYuAqFXeRrYAX8r8BC8QDrHBFucL9w3B84cKB8cFWtgkMIKiW8kzN0Za8jO1zZ68G1Swo0a9aMjhw5YjTcdttt9Pbbbxt1dwvjxo2jN954I0/3sWPH0uDBg/O059fQs2dPWr16tdGtd+/eNGXKFKNekMLDDz9M3wrYVaXnnntOftel6s5yfCcGI4I777zTWRduDzIFGLjyfMOCGbhatTODvl6WSumZFyhVQFc5F6zpULZEODWtHkmNqtmodoUIqlTKc+AK4Qwzsy+Il8izBGOQz1IiwomihStTpI0oQtBCAIbMqSg4XGHfTp29QDuPZNHSLekyN+uQX11KJ6AruFsBuIoTDleiGlCJgasCbAcDVwUQz+JQBUeZh5nb9boqqxxjVRm5XnZ1rLCBq5MnT9LixYtpyZIlNHv2bBlaD2sPDw+njh07Uq9evSTV36pVKwJ4ZU54Ouvdd9+VsXkxl0pxcXHUqVMnuuKKKyTAhacFtm7dauiCfi1btpTuVu3bt1fDOA8BBRi48u8mMnDlX735bKwAK8AKsAKsQDAowMBVMOwSr5EVYAVCQQEGrkJhF/kaWIHCU+CtCRPEA68rjAUwbGVI4dUCQ1eX5GTg6pIWXLJXYMiQIfTNN98YjXDgW7Hi0u8n40A+BThjrV27Nk8vuEfNnDkzT7urhrS0NKpTpw5lZmYa3QCBAQbzRrIKXGEdMBQAUHa1CAc7Y8YMbyyD5wgABRi48nwTghm4Wrkjg75YmkIZwhQJ0BUAJSupnACuWtSKpMYXwwlWKCkoKA8TnJrOpeW+zosQh/mFN0TYu1LFwqlETBhFRwrwygHrVRSAqzSxb2fO5wJXSzan0/bD1h2u1JZd3zImF7iKDhMQm2oNjJyBqwLsAwNXBRDP4lAFR5mHmdv1uiqrHGNVGblednWsMIErhPbDkwXff/+93dMLug5woQIYhQ/cCBUI9ypzAtH/5JNPSperjIwM82GH9XLlytGIESPo/vvvp2LFijnsw43BqQADV/7dNwau/Ks3n40VYAVYAVaAFQgGBRi4CoZd4jWyAqxAKCjAwFUo7CJfAytQOAro4AtWwLCVb/chZlMEVXkhzjhJUQ0vqL/vOKSg8XbgglAAMNQjjzxipwUeoC9btqxdm6vKqVOnqHHjxiIsV964XHiYH+EGo6OjXU1hd2zp0qV0yy232LWtX7+eqlSpYtfmacUKcLVlyxa67777aPv27fJ0PXr0YODKU+EDcBwDV55vSjADVwhB99mSFFLuUlaBq/Lx4dSqdhQlVLdRtbIRBADL03Qm5QIlnc+h0xdfcLpylYoJ0KpiyQgqUzyM4gQgBLcrcyoKwFW6cAM7l0oypOCiTWm09ZDnwFW3FjHUo3UMFRfa2gTAlldRs8L+qzNwVQCtGbgqgHgWhyo4yjzM3K7XVVnlGKvKyPWyq2OFBVwlJSXRoEGDCDGwEVIwv1SjRg0ZJvCOO+5w6HS1ceNGeuqppwjvW3XtzuYsXbq0/HA6fPhwqly5srNu3B6kCjBw5d+NY+DKv3rz2VgBVoAVYAVYgWBQICUlhV4a86Jc6o039qKOV14ZDMvmNbICrAArEHQKMHAVdFvGC2YFAkKBzZs30ZgXcz+rYUEMW/lnWxi6ImLgyj/vtWA8S2JiIjVv3txu6XBwAljkbpo1axY9+OCDRvf4+HhKTk426oC64HTlbho/fjy9/vrrRvcGDRrQ8uXLjXpBC1aAq08++USaDqhzMnCllAiNnIErz/cxmIGrP7dl0KeLUqSbVLaIJ2jR4IrgaNWmDoCrSKpSJpzKFvccuDolQKsTIjTeieRskefIsIKudiU+NlxCXoC+AAjFCJcrcyoKwBXcyeAOtkuEFFywMY22HPQcuOrSPJq6C5craBslQjUGUlhBBq7M724LdQauLIhVwK7OACFzu15XZZVjCaqMXC+7OlYYwBXWNmzYMHrvvffyPG0QJn6D4KVfg5IX0NXvv/9ODRs2VE1Gjv779++nL774gsaMGUPp6enGMVXAvFWrViXE8e7WrRvB5YpT6CnAwJV/95SBK//qzWdjBVgBVoAVYAVYAVaAFWAFWAFWQCnAwJVSgnNWgBVwVwGGrdxVyjf9ijp0xcCVb95XoTJrx44dDQcnXBOinowePdrtyzOHJURUFB2YQrSTV155xe35evfubQdYPfDAA/Tyyy+7PT6/jgcOHCAYE6hUqVIlqlChgqra5Qxc2ckRchUGrjzfUl8BV1lZWRQeHi5fnq/O9cjlWzPoYwFcAUzK8RC4als3ipoI4Kpy6XDhNuU5cHXyXA4dS86h42fES0BXGcK5yVUqGRdONcrZhMtVblhBhBg0J1zXmdQL9POaVJq/Ie939ub+zurNa0ZSZwEj1Spvk25ajsIXOhvr6/ZModN5EXgLwBWuccvBSyForZ67c7No6iaBq1yAjYGriwrCbcdVAh0dyImBK//tjoKjzGc0t+t1VVY5xqoycr3s6lhhAFerV6+mnj170okTJ4xLRujAXr160dNPP021atWiP/74Q36gPn78uNEHhZtuuol++uknuza9gjjWmBfxvj/++GPCkxH4n2KjRo3omWeeIXxoj42NpYgIBwFl9Ym4HLQKMHDl361j4Mq/evPZWAFWgBVgBVgBVoAVYAVYAVaAFVAKMHCllOCcFWAF3FGAYSt3VPJ9n6IMXTFw5fv3VzCf4dlnn6WpU6cal9CuXTsZIcVocFHA92FNmjQh9X1SnTp15HdEbdu2NUbVrVuXVqxYYdRdFTIyMghz6A/2f/3119S5c2dXw3x2jIErn0kbEBMzcOX5NvgKuMLvlIEDB9Lll18uXx06dPB8kU5GSuBqYQoJ1kq8XANOjqaAw1W7ernAVaVSDFw50sjXbXC4ShEOVzu94HB13UXgqmQsA1d2+8bAlZ0cXHGhgIKjzF3M7XpdlVWOsaqs53pZ9dHbCgO4wpMFkyZNMj6sxsXFSTtUfKDWY2jv27ePEEJw2bJlhjTFixenkydPUlRUlNHGBVZAV8BT4AoOaCqpssrR7qxsPmaeQ9VDNWfgKlR3lq+LFWAFWAFWgBVgBVgBVoAVYAUCXQEGrgJ9h3h9rEBgKTBwwK3GgjiMoCFFoRTM0NULo0ZRQkKTQlmLP0/KwJU/1Q6+c/3222901113GQvHd0C7d++2+87IOGgqbNiwwQ6GuvPOO2nChAkEaGvv3r1G77///psQSSW/tHLlSrrxxhuNbljLzp075cP8RqMfCwxc+VHsQjgVA1eei+4r4Aor6tu3L2VnZ8vFFStWzICv2rRp4/mCtZEArj4RDlcSuPLQ4arNRYerKuxwpSnrv2K6cLg6K1y8AFwt/jedth7yPKRg5+YxIqRgtAwpGM0hBS9tIgNXl7TgkmsFFABl7mVu1+uqrHKMVWU918uqj97mb+AKTwbgKQAFUQFiueqqq2jKlClUv359OwnwP7Iff/yRBg0aROfPnzeO/fLLL9Ihy2jgAiugKcDAlSaGH4oMXPlBZD4FK8AKsAKsACvACrACrAArwAqwAg4UYODKgSjcxAqwAg4VUF9I4iDDVg4l8nujDl0lJCTQC6NG+30N/j4hA1f+Vjy4zpecnEwNGjQwAAesfvbs2XTZZZfleyETJ060Cxf40UcfUZ8+fWRElenTpxvjEXnonnvuMerOCub5OnXqRN9//72z7j5vZ+DK5xIX6gkYuPJcfvX5pmnTpl4N+YkVDRgwwDAO0VdYsmRJ+XsJ7lctW7bUD1kq/7Utg2YsSaGsHKKsbESusjScyseHU6s6UZRQzUbVykZQuRIcUtCaggXvnZpxgU6fz6Edidm0dEu6yD0Hrrq2iKEerWOoREwY2USQrksWIQVfZ0FnCBNgicW3Z0FPeWk8A1eXtOCSawWcvU3N7XpdlVWOM6iynutl1Udv8zdwBUvXHj16EJ4kQEIoQTxtoH/olQcu/li7di0htrbqj+Zx48ZRfv++9Dm4XLQUYODKv/vNwJV/9eazsQKsACvACrACrAArwAqwAqwAK6AUYOBKKcE5K8AKuFJAh1wYtnKllP+PlZ4ZRaW/iZYnLgrQlf5e/HrmN/4XvABnVA5xN910E+HFyTcK4LujNWvWGJOPHDmShg0bZtSdFXr16mWECwwPD6ctW7ZQmTJlaM6cOfL7JzWue/fu9Pnnn6uq0/zWW2+lRYsWGceff/55Gjp0qFFHARFatm/fbrQhfGHp0qXl93Q///wzffvtt3Tw4EGqXr26cLBLkNFcqlatavTHffXExESjXq9ePapdu7ZRP3DgAG3dulXWFyxYQNOmTTOO4VyPP/64UUdkGMAf+aVTp07R/PnzaenSpXTkyBE6ceKEdBArX748NWrUiLp27Upw74mIEN/2c/KbAgxceS61L4Gr2267jVJSUlwuDr9nVNhBQF9W0oodGfTFHymUKRgdOCVZJVrKCeCqRa1c4KpGuQgJYFk5v9735LkcOpacQ8fPiFdyNmWI9bhKJePCqUY5G1UUYQ0BCMVE5cWDcD1nhPvTz2tSaf6GdFfTuTzWvGYkdW4eTbXK2yguOowiA+jXE8IJHhe6AbQCQLfrqOfAVfeWMXRDmxgqJq5R/G+MgSv1rsgPCAFJHchp7ty5hP+JO0vt27enfv36OTvM7RYUUACUeYi5Xa+rssoxVpX1XC+rPnpbIABXCBuID4v4IGxO+HA9ePBgWrdunXHo9ddfpyeeeMKoc4EV0BVwBVypsICOctWGuVRZ5Xqbueyo7qwN7aGWGLgKtR3l62EFWAFWgBVgBVgBVoAVYAVYgWBRgIGrYNkpXicrUHgK6IBLapNsShzj+ovDwltp0T2zDl3169ef+vXvH7Ji6O9HBq5CdpsLdGGvvvqqDAWoJgEA9OWXX6qqwxzOWA0bNqSsrNwvulu1akW///677Hvu3DnpmpWZmSnrcXFxtGPHDkKIQGcJ8wB+0qOuAL4ywxTvvvsuvfjii8Y0P/30k3S9eeSRRyRsZRy4WIiMjKRnn32WHn30Udny8MMP2/V77rnn6LHHHjOGTZ06VfY3GlwUAEsBonKWAFbhO+nPPvvM0MlZ37p16xIAsxtuuMFZF273sgIMXHkuqC+BK4Q4xe8Xd1OFChUkfAVXvsaNG+c7bJUArr5aliphqzThlITQglZSWeFo1axmFDWuaqNaFSIk/GRlvN43lIArd8A1EXzLK+lc2gVKPJ0tgavVYj/3HMsNQWl1cnwXfX2raLpRAFdxAl7z1vqsrsNZ/4B2uML/PAM5LVmyxAj75midDFw5UsWzNgVAmUeb2/W6KqscY1VZz/WyuQ+O+Ru4QkhBPEWA95da2xVXXEGTJ0/O84EVH2zxFAAcrvDBWKWFCxfStddeq6qcswJ2CjBwZSeHzysMXPlcYj4BK8AKsAKsACvACrACrAArwAqwAg4VYODKoSzcyAqwAhcV2Lx5E43RYIDds86yNgGqQOUX4ih2U65lQyhDVwxcBegbMICW9eeff9LNN99srAiOUdu2bTMekDYOaAW4SelhAuH8BLBJJTiS/fXXX6pKs2bNoquuusqomwuIuoLvsFQqV64cbd68Oc8aHAFX8+bNI7Q7S++99x7BPQvJX8DV8uXL6e6776bTp087W5bD9v4C/kRoRVdwmsOB3GhZAQauLEtmDPAlcHXvvfcSXOE8SVWqVDHCDtavX9/hFGt2ZdI3f6YSwtKdT8+hHBFa0EoqUzycEqpHUsMqNqpXyUaVS+c1NXF3vmAGrsCpAbLKFqwTnMIy83HngnVURLgI2yfkgltWpM1z+io5NYcOnsym7SKk4NrdGbT/uHXgCnBVuAZcxTpwC3N3H33VL6CBK5COgZxAb+sEt3mtDFyZFfG8rsAj8wzmdr3uqKza9FwvY369jrK/gSusAbDhm2++acS+xVMFcLFCO6xLVcKHWFg2rl+/XjVRqVKl6PDhwxQbG2u0cYEV0BVg4EpXw/dlBq58rzGfgRVgBVgBVoAVYAVYAVaAFWAFWAFHChw/doz27tsrD7Vo0ZK/EHMkErexAkVYAfUlJCQ4LJyt0oTDFafAVCBGwFZVBHSlUrC5P6l155czcJWfQnwcD+zDXSo1NdUQY9myZdLBymgwFYYPH24XJhAAVocOHYxegIZeeeUVoz5kyBAaPXq0UTcXAEXpxxHp54MPPjB3k2CV7nA1ZswY6XiVjW/9HaRixYpJcAvfhyHlB1x98cUXxrrT0tLo7NlL0CwgqJIlSxpnAdABhy1zAgAG2Aq66gnfr8EVC6EOz5w5Q//88w8dOnRI7yLLMD6YOXNmHtgsT0duKJACDFx5Lp/6rAMHupdfftnziRyMxPfWx48fd3DEWhPCiiLsIJyv6tSpYwxeuzuTZq1MpfPCJemsAHeyLQJXpURYv3qVbRK4alTNRtXL5oLbxgksFIIZuIIzGMIynk3LoSPCber0eddCAq8qFhMuwxOWFdAawDVPHaVwrr0CskJIwfV7MyV8ZUF22dUmts0WEUbdWkRTz1YxDsMzWp3T2/0DGrjy9sX6ez4GrrynuIKgzDOa2/W6o7Jq03O9jPn1OsqFAVwB0Lj66qvtyGB8QITV64MPPigtXn/88Uf65JNPZAxptWas//bbb7f78Iw2TqyAroAvgCs9tCDOlV/dUR99jaFUZuAqlHaTr4UVYAVYAVaAFWAFWAFWgBVgBVgBVoAVYAVCQQEdbEm6NZ2SBth/2R4K1xhq16BDV4AgXhg1OtQukfT3ZbBBZQMH5LoSwS0JL06+U2DAgAGEKCcq4eF9hPZyllq2bGnAQvHx8dIRy2azGd3xQH+XLl2Men7h9/AdlApJiEFwrMKazMnscAWgSplY1KhRQzp1lS1blvbu3Uu//PILde7cmSZNmmRMkx9wZXQUBXxX9uSTTxpNPXr0oBkzZhh1R4WUlBQJeCQmJhqHEdZw6NChBBcws3PVgQMHCOEQ4TKmJ2fXr/fhcsEUYODKM/1yhCXUS2NepC1btsgQfk899ZQMmQnoERGU8CpIefbs2XbRlzxbpf2o2rVrG85XSTmV6f/WpFFySg4lCXAnyzGraT+BVisRG0a1ygt3KwFdNa9po9oVLv3e07q5VQxm4AqgWlrmBTp1Lod2HsmS0JWriwZcBVitVLFwqlomQr6E4ZVHCefcdSSbth3Oos0HMulwksVNFGeNjgwTL6IuzWKoW8toihH1QEsMXPlwRxgZ0XKOAABAAElEQVS48p64OlCkz2pu1+uOyqpNz/Uy5tbrKBcGcIXzjhw5UsaMxv/w3E21atWiBQsW2BHA7o7lfkVHgYICVzpMpcoqVyrmV0c/cx81NtRyBq5CbUf5elgBVoAVYAVYAVaAFWAFWAFWgBVgBVgBViCYFdBDCaYKV6tE4W7FKTgU0EMLvjBqlHCfaRIcC3dzlQxcuSlUEe/2/vvv0yjx/lcJsJOzMH1bt26lTp06qa5044030scff2zUUcD3UY0bN6aTJ08a7YCwEPLLnABwNGjQQLo+qWO4/12xYkVVNXIzcKUOIBzh9OnT7YAmOEzBSUqP8OJr4Grs2LH01ltvqWURYDQYHTRr1sxoMxfwfR3cvSZPnmwcwrWvXLmSAJRx8o0ChQVc4d9GLpQEOCmbsgEpXYSVFKh0Kb/YJ9u+b+4YATaJ8VnimPM8d251DpnjnGo+l/klcOrSerJFCD7XTka+2S3vzHrFFVdQnaadaH9Oczp5NpuOnc6hzGxh1WQhFYsOo8oCGKpb0UZt60ZSfQFeeZoYuPJMuRNnc2jroSzaLoArQFdHhcOW1QRwrkRsOF2VEEXXNomWAJbVOXzdn4ErHyrMwJX3xMX/1Bwlc7ted1RWbXqul3EOvY5yYQBXWAdI/zvuuIPmzJlDsEN1lQCu4ImAcePGUZ8+few+qLoax8eKpgKOgCsFP7nKzcegnqM2vd1cRl0lNVbVQzVn4CpUd5avixVgBVgBVoAVYAVYAVaAFWAFWAFWgBVgBYJRARVeB2vnUILBt4N1+pYwFh1sLlDGwp0UGLhyIgw32ymA+80IZacSHsRfvXq1qtrlZjjLmRsWIqvMmjXLGAsQCd9PmZP53HCbW7JkibmbrDsCrsqUKUMIgaiDVQ4Hi0ZfAlf79+8nAB3p6enG6eGuNXDgQKPuqtC/f39avHix0WXYsGHSRMFo4IJXFdCBqzuFm5tb0JIDWCkXeHIANgmIyg50ulgPZmDJqxvgp8lq1qxJ11xzjYwAhd8VAHSWbMqQjkwHT2ZTRpZjVsDZ8uCEVLZEONUWwNUVDaMoQYQV9DQxcOWZckfP5NDGfYCtMmnPsSw6kWwdAsQelhGvyxtEUUexj3C8CrTEwJUPd4SBK++JqyAo84zmdr3uqKzaHOXO2goLuMK1njhxQsa0/vXXX2nPnj0GDKbrgFjSCDWID58333wzFS9eXD/MZVYgjwJWgSsFRplzTOyqTZ1Y9VF1lTtrV8dDJWfgKlR2kq+DFWAFWAFWgBVgBVgBVoAVYAVYAVaAFWAFgl0BHWjhUILBuZulZ0ZR6W+i5eL79etP/QT4ECpJf38GG0zGIQX99y7Ed1lmR6pNmzZRhQoV8iyiX79+dkDU2rVrqXr16nn6ff311/Too48a7Y6csHBwypQpNGLECKPfkCFDpOOT0aAVHAFXb7zxBg0aNEjr5bzoS+DKDJgBYPvmm2+cL8Z0ZMOGDTIEompG+MGNGzcSIBFO3ldAB668P3vhzhgREUEI8RkRYRN5BEWIsk20uc5Ff9HXJsa4zMVcfwgg8tixY1S5cmUZ7lWdL/ecuef2tPzMM8/Qzp07CyQgzq0gK7O73O6j2bRiewYdELDV7qNZlC7C4llJtogwgstVjfIRdF3TaGpRK1J8pym+17QyycW+DFx5IJoYAlBu1c5M2nowk44I+ArhIa2myqUjqJJ4ta0TSe3qRVKUzZMdtHpWa/0ZuLKml6XeDFxZkstlZwVDmTuZ2/W6o7Jqc5Q7aytM4ArXe/r0aVq0aBH98ccftG7dOtq9e7d0vCpdurS0Nm3durX8YIc43OaY0ma9uM4KQAEGrvz7PmDgyr9689lYAVaAFWAFWAFWgBVgBVgBVoAVYAVYAVaAFXCmgIJCcHz3rLPOunF7gCsQqqEFGbgK8DdeAC1v8ODBMvydWhLCBAKS0lNKSgrVr1+fEK4PqV69evTXX3/pXYzy0aNHqWnTpka9RIkStGPHDgGBRBhtKNxzzz30888/G23ffvuthCWMBq3gCLiaP38+tWjRQuvlvOhL4Arfqx04cMA4uTPnL6ODgwKMEA4ePGgcmT17Nl122WVGnQveU8AVcIX3qIKIPIWWJOikgCeVS/ApH6BJAk8uACkAVGo+mQOsUoBVbu49lRzPpFw98e/75ZdfdtzJw1YAVwhb6kkCNHr11VfL3x8xMTEOp9h/IpvW7c2UsNU2EZYuNcMacAW4yhYeRlXLRlD3FtHUpm4URYSTfDk8oYtGBq5ciOPkEIKX7T2eRX9szqAtArg6k3LB8h5i6prlbRKaa14zklrWtFEkA1f2ij/11FP2DSFWY+DKexuqYCjzjOZ2ve6orNoc5Y7acL4FK9bSwlX/yFO/PWKYeQl+qcO2EvGrExMTpesVPiDDyQpEMp5awP+MiopbkF8ED/GTuAtcqfeUsxwymY/pbUpG1UfVVe6sXR0PlZyBq1DZSb4OVoAVYAVYAVaAFWAFWAFWgBVgBVgBVoAVCGYFdJjFH+5WMZtyQYXYfyMoZlNuGJu0JlmGhEkDciEIoyGICrg2Z9eV2jSb0ppk+/RqcP4qL8TJcyCk2QujRvv0fP6aXH+PssOVv1QPzvN8/vnnNHz4cGPxDzzwQB6YYt68eXTbbbcZfe6//3565ZVXjLq5APhh8+bNRvPcuXMJYJKeGjVqRCdPnpRN+F4K7jbR0bmOc3o/lB0BVzAUAMzlTvIVcIXv1+DypYeLg+EBABArCU5diE6j0ttvv22nt2rnvOAK6MDVtOkf20FLReV7Jk9V9CVw9dxzzxHc9dxN8fHxhptV3bp18x2WeDpbOCMhHF0WbdiXSSnpFoErcQZAV5VKRdBVTaKppXC4io8NozjhemU1MXBlTbFsYWSVJV47E7No4b/ptPVQpnAoE23Z1vewXmUb1RevxlVt1KhaJEXac8DWFuaj3uxw5SNhMS0DV94VVwFR5ln19vzK6rij3FEbzhUIwJX5mrnOChREAQauCqKe9bEMXFnXjEewAqwAK8AKsALOFNi3bx9t376djh8/QWfPJlOpUqWobNly8kndWrVqOhvG7awAK8AKsAKsACvACrACrAD5y90KIe8AWMVeBK5cSe8P8MvV+a0cA+RUema0W9eFeX19baHocsXAlZV3ZNHuu3//fmrTpo0hAtyWfv/9d6OOAtxnpk2bZrQhbGDnzp2Nurnw4osvSkhKtT///PM0dOhQVaVt27bRlVdeadTzC8NnBq7Kli1ryQ3HV8AV7il07NjRuA4AO08//TSFhwvrGwtp4cKFtGLFCmPEsGHDaOTIkUadC95TQAeugg1G9Z4Kns3kS+Bq1KhRtH79+nwXht9VADqvuuqqfPvqHU6ezZEOSRv3ZdHqXRl0Ps0arKPmKlsinNrXj6LmNSKpYqlwKlPc2r91zMPAlVLTvTxTcPcpwpEMzmQLNqTTdgFeKQ7DvRlyewGNa1I9UrxsVLeSjepUhKOclRn805eBKx/qzMCVd8V19g9Rb8+vrI67ys3HGLjy7j7ybIWvAANX/t0DBq78qzefjRVgBVgBViD0FEhLS6MpU6bKsAV79uxxeoF16tShnj17EJ7adWZH7nQwH2AFWAFWgBVgBVgBVoAVCGkFdJDFVyCQVSBJF9xXa9LPUZAyILLS3zh2sTlx4gSVK1fO6fS+urZQdLnS36fBBhUooPGmm24ivDj5XoF27drR3r175YlsIvwZ3KNiY2ONE+vHo6KipBuVftzoeLEAl6e+ffsazWag6pNPPqEnn3zSOA5AC1CUs2QGrrAe3RHK2TjV7ivgas6cOXTnnXeq03gtR0hHhHbk5H0FGLjyXFNfAlcvvfQS/f333w4XV6lSJcPNCtGaPElnBWB1IjmH1gjY6o/N6XQ21TPgqmRcuAR2GlWNFNBOBFUtY53YYeDK2g6ew94JYA7A1V/bM2jvsUsOr1ZmgkNZWxEKsm3dSLlvlUuLMJ3WeTkrp/SoLwNXHsnm3iAGrtzTyd1eCoQy99fb8yur465y8zEGrsyKcz3YFdCBq/v69pSXo2xX9Vwvo5Ozun7MXHZURxuSmi+3Fro/GbgK3b3lK2MFWAFWgBXwvQKrVq2ST4ju3bvP7ZMBvHrjjTeoadMmbo/hjqwAK8AKsAKsACvACrACoa2AgkF8Bf+YgSRASAi7hRxJ5SgDTlKAUsOGDdEkU6oIw5c4JkVVAybXnaSwKFwLnG5UWRYu/kC4MST9ulD3le762l4QThsJCcH9NwADV3i3cHJXgSeeeII+/fRTo/sPP/xgOFDhYSV8R6kS3GW+++47VXWYI9RevXr1KDU1VR6Pi4uTkFZkZKSs4+EmnEOl/MLwmYErgHi645aax1nuK+Dq/fffJzjzeDs1adKEFi9e7O1peT6hAANXnr8NfAlcjR07lnDfTk+dOnWSoJXuwKcft1LOEIxOqnBJWr41g35fn0ZnzueQJ8hVMRFCsGb5CKonHJKai7CCyK2mU+dy6FjyBfHKpuNnsikjy/VKAHnVKGejiiXDqURMGMVEwavJPl0QU5wRENnPa1JpvnCB8jQ1rxlJnZtHU63yNhku0RxyD+H90jIvEK5h55EsOiJCNbpKgJxKFQun0uIaAKfhhTYrCefaeyzbCAeZmOT6nM7mxnk7NY6mTgnRVLZ4mFxXuMW1OJvbm+2FClwhtmdmpgjYGKLpiiuuoN69e4fo1fn/shQIZT6z3p5fWR13lZuPMXBlVpzrwa5AfsCVAqHczaGH6msuO6qjDUkfk9sSmj8ZuArNfeWrYgVYAVaAFfCtAufOnaPx41+nmTNnemQ5jad7hw0bSoMHD7YcFsC3V8azswKsACvACrACrAArwAr4WwEdYvEF+GOGrQAjbd261a3LBKCkw0mBBl3pQBMuaPny5XbwmKuLNF+bL7QPNZcr/b3KDleu3l18DArMnj2b7r33XkOMESNG0PDhw2V96tSp9OyzzxrH8nOjUh0HDhxICxYsUFXpSAVnKqRmzZrRkSNHZBnuNRs3bpRlZz/MwBW+L50yZYqz7nnafQVcvfPOOwRnHpXg/lWqVClV9TivVasW/fLLLx6P54HOFWDgyrk2+R3xJXA1fvx4+vPPPwkPPl5zzTXyFR8fn9+S3D4OUAih6f7alk4LNqZLtyuATjmuWac880dHhlE5EVawRrkIuqJRFDUToQWtpmQBRp0WwFeigJUOn8qWAJOrOYIduKoQH0Hl48OpggDGULYKXAGw2nQwi7YLh6tdR7MI4SGtJoBVEeIHYLLOzaKpmADXYsVeWl2L1fN60r9QgasPPviAXIWE8OSCAmkMPjwAuuLkHQUUCGWezdyu6ipHf1V2Jzf3YeDKrDjXg10BBq78u4MMXPlXbz4bK8AKsAKsQPArkJ2dTQMGDKQNGzY4vRg8aVutWjU6fvw4JSUlOewXERFBU6dOoY4dOzo8zo2sACvACrACrAArwAqwAkVDAeVuhavdPeusVy/aDFtZAZL0hehwUqBAVzpsBVcrXJvVpF8XxvoCutLXGWyQkllPBq7MinDdlQKnTp2ixo0bU05O7hfZ119/PX322WdyyB133EFz5841hufnRqU6fvTRRwSzDJVQfuyxx2jfvn3Utm1b1UwAsyZNmmTUHRUCFbj6+eef6Z577jGWDOh12bJlRp0LgacAA1ee74kvgStwHgCt8HvIl2nljgxa+G86HT2dI8IK5hBALCsJIehiBKhTRTg1dW0RTe3rRVkZLvvCaeu8CJN3SMBWe09kS+ctxRM4miyYgSuATlVE6D68ShcXTlfC7coq5LTveDat2pkhHa6OCkjNk3CQtogwgltXj1YxdH3rGFkWSwvIVKjAFf4HBgLb1RsyIFVzY1Hh4eGSJK9YsaIbvbmLOwo4e5+Y21Vd5ZhblT3JGbhyZ3e4TzApwMCVf3eLgSv/6s1nYwVYAVaAFQh+Bd599z3CE6fmVKNGDRo6dCg1b96catasYbhlnj9/nubMmSNCKcwwnAQAW73++ni68cYbzdNwnRVgBVgBVoAVYAVYAVagCCmwefMmGvPii/KKvQ37mGGrn376qUDK4kEBFWrQ22u1ujD92qw4djk6D65JfwjisAibmCbCJ3or6S5XwR5WkIErb70ris48nTt3Nh5WwveRuBcNAKtBgwZ05swZKUTlypWNPvkps2PHDjsjiV69etH06dPp66+/pkcffdQY/uGHH1Lfvn2NuqNCoAJXW7ZsoauuuspYMu4fwBwkNjbWaONCYCnAwJXn++FL4MrzVVkbuX5vJgG62i9Ap2NncvIN52eeHbBQpIB34Nh0dZNoals3UobeA4TlboLTVqZw1zoizn/wZDYlp+RQSoZz+CsYgSvoFGUT4Q/Fq1rZCPmKFaEQ8XI3AYbLEq+diVm0bEs67RAhDAFbpYuQhlYS1hIfGy5eYXSN2DPsm/hVTe6vxMrZCt63UIEr0NeTJ092+kRywS+v8GZAbOR+/foV3gJC8MwKljJfmrldr6tyQXIGrsyKcz3YFXAGXKkQf1Zz6KHGmMuO6mhD0sfktoTmTwauQnNf+apYAVaAFWAFfKPAv/9uov79+xNcrlTCzc9BgwaJp2qHUUxMjGrOk+Mz/4QJE2jatOkyv/767nn6cAMrwAqwAqwAK8AKsAKsQNFSQAdYfOluVVAoSe3KzTffrIrkbTDJmNiNQp2+JYxeBQXJMJHudOULBy+13oSEBHph1Ghj7cFW0N+vwebWpZzkbrrpJsKLk38UGDNmjJ3TFJyiT548Sddee62xgP/85z8OH2oyOpgKrVq1ooMHD8pWhApbuXIlDRs2jL788kvZhvv6gJbKli1rGmlfDVTgKj09nfBAl3IGw6p/++03Owcv+yvhWmErwMCV5zsQCsDV9sNZtHF/Ju0U8M7eY7nuUp4oAremyxtEUas6UVS+RBiVEs5NVtMJERrviHDaOpGcTSfPOYe/ghG4grNV8ZhwKiFAp5oi/CJCMAJ8spIApqUJN7DNIpzgQhEGcrcIJ5gtYkBaDQOJtVQsFUGVSoVT+/pR0pUMTmWBmgoVuMINcdhYhlpM2woVKtADDzxAJUpc+qMkUN8AwbQuBU2Z12xu1+uqbM4xB9rM7Y7qC1aupYUr/5GnfXvEMPPpuc4KBJ0CZuBKgU+e5hBAjTWXHdXRhqSPyW0JzZ8MXIXmvvJVsQKsACvACnhfAdz07N27D+3atctu8uefH0l33nmnXZuryuHDh6lKlSquujg8tnbtP9IhCzeW8XdBlSqVqXbt2nT55ZeLp6jEY1ROUmZmJqWkpBpHbbYIKlasmKynpqbS4sVLaPfu3eLp4tNUuXIVceP7GqpVq5bRXxXwRO3SpUvp8OFEio6OpoYNG1DTpk3ljWDVx5yfPXtW3CS+9JRY8eLFjLVu3ryZ1q1bT3v37qHixYuLJ5wbUpcunclms9lNk5EhrNkXLpK6Izwj3MPq129ArVq1lOuw6+ygAq3++WedeCJ4N+3ff4DS0tKoevVq4lWd6tatK0M/OhjGTawAK8AKsAKsACvACvhFAQWBeNsxypsOULoQuhuUL8Ak/VzOyr66Nt3By9swmR5WMJhdrhi4cvau5HZnCixevFg+tKSOf/rpp3To0CEaMWKEaqKpU6eSDnMaB5wUhg8fTp9//rk8img++Fu1S5cuBPcrJLhOL1iwQJZd/fAncIXrfuKJJ4zl9OjRg2bMmGHUzYXWrVvTgQMHjGYVOtFocKPw1FNPib/fD8u/ffH3b7t27eTLjaHcxaICDFxZFEzrHgrAlQzjJ0CrTQcyaeO+TDorQvt5korFhFGDKjZqWCVSvGxUu4Lze33O5j+ffkE6Nh09k02HRXhB1LPEfTlxa8wuBRtwJX7VSxewiiUjCK8yxcPEyzrhdPp8jgi7mENbBSS3dlcGJSZlS21M8thp5agCuKpeJZt8Na5uo4RqkQQIK1BToQJXShQ8IQFCOisrSzUFbY4b2QMGDBA3xxsG7TUE6sIVDGVen7ldr6uyyjFWlR3ljtoYuDIrzvVgV8CXwJUZojLXde1cHdP7BXuZgatg30FePyvACrACrIC/FPjhhx/o6aefsTvdZZd1EKECP/UpqD1nzlz64IMP5BO6die/WAG8dffddwuXrbscruP//m+23Y3dtm3biid/v6Bvv/2W3nprIp04ccJuWnwGuuOOOwggGRIApSeeeJJ+//13u36oAPQaMmQIPfTQgwZIpXfq2rUb7du3z2iaMeNTAUzVpJdeeonmz897AxwhJqZNmybgq/pyDG7OP/PMswT3aXNCn7feekvAV7l9zcfhQoa/5adMmZoHklN9ca0I6zh8+GMMXilROGcFgkABALAAKY8cOSJ+hx2Xv3/Kli1H+B3SokVzl26DQXB5vERWgBUoQgro8IovgStvOEDp26KDSd525dLP46ys3KK85dqlzuNLmEwPK9ivX3/qJ1xzgzHp71l2uArGHfT/mvH3JB50wYM0SI899hjt3LmTfv75Z1kHMLV161YqXbq0rLvzY/bs2XTvvfcaXWfOnEkDBw40vluD29XIkbl/zxqdHBT8CVx99dVXNHToUGMV3bp1oy+++MKomwsIj4gwiSrhu92//vqLKlWqpJpc5ps2bZIuYuo7RXSeNGmS1MnlQD7okQIMXHkkmxwUCsAVIB64Sa3dnUnLt2YQ6p6kSBEqr6yAiBAur2Mj4XRVO9LyNHjmES+ARHBvwlrSRahBhNLTUzABV+LWHUUImilGhA6sU8FGtcrbRGhBAFj6FblXRrjFjfuzaPvhTNojIDlP98omgKuWYn9a1Y6i6sJpq7rYM6wzUFNAAFe4kYP/kS1cuFDebA5UsfJbV61atQj/E69Xr15+Xfm4BwroH1z04eZ2va7KKsc4VXaUO2pj4EpXm8uhoIAV4EpBUeYcOrhqUzqpPqqucmft6ngo5QxchdJu8rWwAqwAK8AK+FKBIUMeoXnz5hmngBMT6lWrWnerMiZxUQAwNG7cOPrkk09d9Lp0qHv37jR+/DiKjY291ChKjoCrrl270quvvmrXz1x5/PHhdNddd0l35JUrV5kP29XxUM9LL42xa0PFDFy99dYEevvtt4Wr1b48fVUDALJvv/1GuF+tE2EhHnP54BNCOP788+w8Llu4qf/II48Ix+qlalqXeWRkJH344YfUqdOVLvvxQVaAFShcBf755x+aPPkjeY8ODn2OEn4Hdux4hfwSDoApJ1aAFWAFAlkBHV7xNrikoCTA9cuXL/eqDDqY5G0nqPwW6it3K3VeX8JkyuUqmMMK6u9ZBq7Uu4bz/BS45ZZbpFsy+l1zzTWE+9HqwZ82bdrQnDlz8pvC7viZM2ekqQT+ZkbCQzQK4EIdD0tdeWX+f9v5E7j68ccfafDgwVieTHCqBqTjLOHBgssuu4zOnz9vdOndu7d4oGiKUXdWgC4I07ho0SKjC8IrIpxjVFSU0cYF7ynAwJXnWoYCcIUQdSnitWZXJi3ZlE7Hk3MoK9t6mDq4JsUKqKhcfARd0TAXuCoWHUZx4mUlwa0pSQBgR88gtCBgsGxKy7R3uQom4AqwFdZbWoRYrFw6giqLMH5wvLLiKJUlgLNMAZ7tPJJNK3dk0B4Bo50SGqWKfbOacN4oAccBiuvYKFqsS4R/FKCctV2yetaC9Q8I4AqXANAF4RAQYhA3fpOTkwt2ZX4cjSeI8T/vZs2aEW4kc/KNAgqGMs9ubtfrjsqqzd18/oq/OaSgWXSuB7UC7gBXCohylkMA8zG9TQmk+qi6yp21q+OhlDNwFUq7ydfCCrACrAAr4EsFWrZsJULzpRinaNSokbhB+pNR93Zh9OjRwonqK0vT4kb2a6/Zg1Rm4AquVDk5OcaDHs5OgM9DeGgH4RnyS5jzl19+pjp16th1NQNXgNTccY7G37AInahuoNtNaqr06tWL3nzzDaMVf0fdffc9EsgwGi8W8PQ0jqu/tfTjFSpUkNdQsmRJvZnLrAArEAAKAK56880JMnQMfn+5k/DvffDg+wS4OSxPqFJ3xnMfVoAVYAX8oYCvwgnqbkredoGCLjpw5W1nrvx014ErgGQK2shvnLvH8RlfRQfxNkymrz3YYCWlHwNXSgnOrSgwceJEeuWVV+QQfEeJsPcqIcze008/rapu5z179qTVq1fL/vrfmXFxcTK0oDtgkT+BKzhG33777cb1lShRQn7fXK1aNaPNXDCvD8fvvPNO6RhdrFgxc3dZx9+6cNLS3bFw4H//+59wj7Z37HY4ATd6pAADVx7JJgeFAnAl/tlJV6k1IkTdwo25YerOp+fkcZVyRyXAPCViw6ldvUjpoITweeXjBV1kMQGwShHGgkeE09Xe41kyzKC8H3ZxnmABrnBvEk5W1crYJGxVMi6M4sXLKtwEPc6nkQz7uFhAcQeE01W2B1AczhsRIdy2BG7TrUUMdW0RTTZR98Rty+KWFqh7wABX5qvATWJvf5g3n6OgdTzVxzeLC6qi++Md3bTHaHO7XndUVm3u5gxcub9H3DM4FNCBq8H9bpCLVgCU1RyD1Rhz2VEdbUj6mNyW0P3JwFXo7i1fGSvACrACrID3FDhzJpnatWtnN2GfPn2EA9Vrdm165fjx43Tu3Dm9yWW5Ro0aRlg+hNt7+OGH7frjix88pXr11VcTwgls2vSvcIt6R4TV+seu35QpH8k+qtEMXKl25ACkMF9SUpJ8qhjOUM5S9erV6ZprrqHU1BQBJv0qcnt3Gdw8HjXqBbvhZuBKHYQz1VVXXSXD+K1atVI84bxJHcqT40Y5nuxt0KABrVq1Sj6Vq3cCVLF8+TLCE7sq4SGpRx8dSkePHpVN/fr1pR49elKrVi3lE73bt28X4RTetXviFx1x81qFUlRzcc4KsAKFq8CyZcvEv8sX6NChQx4tpEmTJvTGG6/LUDYeTcCDWAFWgBXwkQKbN2+iMS++KGf3NrTka+AKi7755pvl2lObZFPimEsPJchGH/5QLlE4hbdDJWJOHSbzNnCl78sLo0ZRQkITnDKoEgNXQbVdAbNY/M2K6DuO0q+//prnb21H/cxtr7/+unB4Hm9upi5duhDC97mTzECTuw5Sam78zf7tt9+qKj333HMyZKLRoBUAv5pdtxAOG3+PlypVSj4UZXahBpiGv5sRglFPeDBqxIgR1Lp1a+n2jO8y8HDYkiVLCLps3LhR7y7+Dm4lXb+cQVp2nbnikQIMXHkkmxwUCsCVuvpNB7Kke9LeY1l0TLhLAfLxJMVEhlG9yjaqL16Nq+XmVgEjhBDMFEDRyXMXZHhBOF6dTcuhDOHyBEAs0IEr8WtNhhEsFiMAKwGgVboIngF0ihb6WE0nz+ZIyGrLwSz6Z0+mcCHLljpYnQdAXLxw2yojHK2uSoiiTo2jxffJ1ty2rJ7TG/0DFrjyxsXxHKGlgAKkzFdlbtfrjsqqzd2cgSuz4lwPdgUYuPLvDjJw5V+9+WysACvACrACwanArl27JLCjrx43OO++e5DeZFd+7LHhhJvH7qYff/xBfOmSIB/Y6NatO+3bt88YWrt2bZo582t5I9ZoFAU4vTzyyKM0f/58oxlh8aZNm2bUnQFXuBk8aNBdRj/cAO7V6yajrhfw1PH99w82mhz1BRQ1Y8anRh8UHAFXALe+//578XBQvOyLa3jssccE8DXXbiwqCC/4/fezqEyZMvIY/kYaOnQYzZ1r3/fLL78gc+iwkydPiielnxFfBt4krqtXnrkx13333SdCWywzjjVv3py+++7SDXPjABdYAVagUBRYsGAhPfTQQ07PjS+X4AoA8PLw4cN2bgn6oPbt29Nnn80oUg/W6NfPZVaAFQhMBXRwxdtgj+6k5AsXKCiqQu8VFnDli1CJuC5fAleYX4V6DNawgvr7NthcupSj3E033UR4cfKfAnAuxgNECAWoJ4BGW7duNR480o/lV16zZo34G71Hnm5jx461C92Xp4PW4E/gCn/3Nm7cmE6dOqWt4FIRn2f3799P0dHRlxpFaf369TRw4ECnBiDx8fEE8w31sJHdYFHBg0uzZ882/qY2H+e6dxRg4MpzHUMJuNp7LJu2HMqk7YezaGeicJVK8wy4sglDq1IifF6lUhF0ZeMoal8/SkI9gutxO+HMAKtS0y/QmdQLdPxMNh0SbldnRRn3wwAN1Shno4olw6mEgJpiRChDc8J4jP15TSrN35BuPux2vXnNSOrcPJpqlbfJ8IhmNyjAYYDTEOJv55EsOnI6W4QLDJNgFUIIYo2lxHpLxIbJMIIAnKymPQKCWydAK8y//0SuDlbnQH+bcNuqWiaCqgvtWteOpFZ1Ii27bXly3oKOYeCqoAryeL8poAAp8wnN7XrdUdlVG46p4ypn4MqsONeDXQEGrvy7gwxc+VdvPhsrwAqwAqxAcCqAJ0p79sx13lRXMHLkc3TXXZeAJdWucqvA1UcfTaZrhIPUihUr7eZFuL758+dR1apV1dR2OcCijh2vlPAVDqD/n38up9KlS8t+joCra6+9liZP/tBuHlQQktDsNoU1oa/ZAfTmm3vTli1bjDmqVq2SxzHKDFzhJjLAsRYtWhjjUFi0aBE98MCDdm24jq+++pJatmxp1+7I/QthFLF2qwlhHQCsqYQnfteu/TvPtarjnLMCrID/FMCXUTfccCPhd5ye8HsEbn833NCTEPYJjn9I+BJrx44dIuzgF9LxRDn24Yn+adOmGv30ubjMCrACrEBhKqCDK7tnnfXqUhi4Kpicyr3L285jWJVy6GLgqmB75MloBq48Uc17Y+6++27hlPyL3YR4MGb69Ol2be5WAHHhs+Dp06fthvz5559Uv359uzZnFX8CV1jDihUrqH///qQ+p5rXtXjxYoI7qzkdOHCA7r33XoKTs5UEdyxcY+XKla0M474eKMDAlQeiXRwSSsDVieQcOpyUI8PW/S3CCwIgArQE+MlKgosSXJwAGV3WIIra1I2UgFS8AI6spsxsonQBMyWdz6Gjp3NkjnCHMZHhVL1sBFUIJODq/AXafTST4EZVPCZcOFuFiXCKEVS2RDjFCj2ihbuV1QRHrzQRWnHr4UxavSNDwlanUy5ITazOhf7RtjBqUiOSmlS3UZ2KNqpdQRBYQZAYuAqCTeIlXlJAQVCXWvKGFMQxvZ8qq1w/rtpc5QtWrKUFK9fKU749YpjM+QcrEMwKOAOu1Jd8znJcs/mY3mYuO6qjDUnNk1sL7Z8MXIX2/vLVsQKsACvACnhHATyJ265de7vJ+vbtS6++OtauTa9YBa6mT58mQwy89tq4PDed+/fvp0+dp/zrr7/R+fPnjfYvvvjcCMvgCLh6660JAlawB8gw+P3336eJE9825kFh4sSJAjbL++TwO++8I27evmf0Baz0zz+5f5eoRjNwBaeuuXPnqMNGnp6eTu3bd7ALU1i3bl367be8DmEIZYi+GKPSs88+Q/fcc4+qGjmADcBc+LwDkAxPDGOdCN2AL7mqVasq9tA+LCRcIMqXL2fMwQVWgBUoHAWGDHmE5s2bZ3dyhEEdO/YVGT7F7oCpAvDqwQcfkv/WEWaVw6eYBOIqK8AKBIQCCv7wBdSjh67zlcOVgpIKy+EKm+iLkIIAOODEg+Rt5zHMqcNwwRhWUAcF2eEKO8rJXQUAVj399NN23d966y2644477NqsVAAhwb1JJTykZAVK8jdwhXXi79Nnn32W4KJtTlOnTjXCtZqP4TtCXCv+ZkeIRjxs4CwhTOGTTz5JHTp0cNaF272sAANXngsaSsBVWiZRinCUQsi6xf+mCaemHMrKyQ3hZ1UhQFexwnWqmYR7IqmmAHtqlLMO9wD4Ekug1IwL0t3qhICZ4CAFW6aqwj2qfHzgOFwlCeBq7/EsSk7JoSrCRaqCWBvAq7joXGcraGI1JQu4CgDXvwcyZbjHo+La4agFTawmnD42KpyuFKEEr2gYJYG4UsU8WJTVE3uhPwNXXhCRp/CfAgqMMp/R3K7XVVnlGKvK7uQ6cFWvRjXzqbnOCgSdAjv3H5Rrrl2tMg3ue4MBPykIKr8cg819lAiq3Vk9v3Z1PJRyBq5CaTf5WlgBVoAVYAV8qUCLFi3tgCAAOwgD6CzhZumaNX87PHzu3Ln/Z+884KQozz/+3N5eA+7g6B3pcoAFFYgttoCJig3sNWKLJUajidGAKCYmaqJJ7NgrxRI1llj/CREbKiKHFOkc5YCjXd+9+z/PLO8yOzvbZ2bL/N6Pw7zztnnf77u33s3+9vfQ559/HlL3+uv/1L6he80114aFzAtpGMfFX/5yL5144olaSzPB1ZtvvqGFFjAO9fLLL/PD39+FFIvoScRPxjRz5kz6/e+nBIvjEVyNGzeORVp/D/bRZ4466mgtJJgqE5GXiL3Mkjh6VVdXB6vMBFcSekHcqyKFVgh2NmQ+/vgjLZShoRiXIAACDhJ47bXX6KabQj+Qkw/Q5L0rXvGUuB1ISBYJsZJIam5uprlz59KqVav5PWk9368duwJ0pxEjRpg6DujHFkFoUxM/Zd+TiouLgmFhNmzYoLkarFixQvuArF+/fjR+/HgOr9peNQ+e5f8PCxZ8q71/denShd+vB2tuf8q5MNhwT0aeHe3cGeqOo8K2Sp04PUgo2KqqDZqgdNSoUUFRrn6smpoaLUTtunXrNVHroEGD+P9LQ7W169tFysv65f97Im5VIXH69OlNEkpWBAwSNggJBEBgLwGnBFfy8y8hu6xM+rB7dgjGos1VOURJGzvEZHYLrvRiOAiuou209XXqZw4hBa1nixETJyC/E27cuFF7xiAOVPK7rtnvhWYjyxeL5s2bR/L7pbjBFhSwGIN/t5Rjn3320UKjmvVDmX0EILhKnm0uCa5EyCPHd2v4b8rvm2jtFh/tYMGPuCwlkyT0Xq9OXs2JShyVhvUu0FyeitntKdHk2+N0JSECRXQlmk0RC5WVeKgtRzIVRy1jErGWUyEFG31EtQ0ttJldwsSRS4RgHdt5qJAdpYwhCI3zNLuWfRCx27qtLfQDhxFcxmEel3CYRxFzJZNE7NWmiEMbcqjHIzjMoziPFbPjlhm3ZMa3uw8EV3YTxviWElACKeOgxnL9tVlelcVz1guujPfFNQhkMwGrBVdGsZWwMSuLVp7NPCPNHYKrSGRQDgIgAAIgAAKhBK688kr64IMPg4XyUPPDDz/QHFSChXFmnn/+BZo2bVqwtfxO8tlnn2ofRp999jk0f765UCvYIUbmzjvv5FAFAVcsM8HVggXfmIoQzARXixdXamEKjbdMRnA1efJkFlHcaBxKuzYKroT3r351nWnbWIKr1177J91yyy0kwolEEwRXiRJDexCwnsDRRx9D69evDw4s75FPP/00jR1r37f0JSzNjBmP0zPPPBMi6AxOgjP77bcfXXvtNSThWczSLbfcSrNnzw5WXXXVL0jey+699y/0/PPPsxiLYxnokojB7rjjDpow4SStVMLFXHnlL2jp0qW6VoGsfAAnrorHHXdcWJ0IS484InRO33+/WHM/mDbtdlOhxUEHHcTrfSwoYHvggQfpoYceCpuj3EyEYdOnT+cPAcvC7i0FO3bspGeffVY7RLRllmSt4kQ4efIlCO9oBghlriNQWbmIbt/zu6AdLkoCdMDppRrXLVu2aMIkKyHrRUlOC670giU7BFeHHXZYUKxgdahH2QP9/CG4svJVGXssCK5iM0ILEACB5AhAcJUcN+mVS4IrkVWJSGnFJj99u7qZlrPQZ021n3azkCiZxH8GUzG7XJWyy5OEFTxoQCGLkPI0IVKi48m8xNVJC7HHj8pEkCQiogIv34PFVmaiJqcEVzKvZhZcNfkD4jS5r7h7ybxkjsIh0SSirTr+83vRHmerqm1+Flu1UjPfI5lUkJ+nhV/s2dFLhwwqoFH9C8jjCTBMZjyn+0Bw5TRx3C8lAkogZRzEWK6/NstHK1N1+vNjL3Ps6wjvEWxWGJhOhHo112A7KYjRVvXBGQRCCBj+p5cnnpSR0p6qaG2OHTuKBrDLlRJF6c8qL8OrvPGsrzPmza6lTCU1lrrO5TMEV7m8u1gbCIAACICAlQTmzHmZfve7UPenI444nB5//PGEbzNx4iT69ttvg/1GjBhOr7zyinYtrkz//ve/g3WSueaaq0OuY10cf/zxNHjwYK2ZmeBq0aLvtG/BGscxE1wtXbrE2Ey7fumll2jKlKnBungcri6//HK64Ybrg330GaPgStZ8zTXX6JsE8/JBWHX1luC13uFKPuyXscRpRSURCVxwwYU0bNi+mtBMvkksvwO9/fY7JN8O1icIrvQ0kAcB5wmIE8tJJ00IufEZZ5zBgp87QsqsvBA3LHm/+eyzUOdBs3vI34rXX389XX75ZWHVRsHVFVdcQeJeYHxP13cU8a78f6Rbt67a+1QsVz69g6Eax0xw9cILz9Mll0wOeS9U7dVZhGOPPPIw3XPPvTH/XyZuV2+88XqYAFdEYhdeeBGtWxdwqlZjRzpLyNZXX32VunbtGqkJykHAFQT0YdnsEPUIROUEZbfgyq75R3sh2CkmU6ES7RSSqfnL3wQTJ02KttSMq9O/dhFSMOO2BxMCARBIEwEIrpIHn0uCK0Vh844WWr3FT4vX+Wjh6iYtpJ2IiJL56F3ERiL2GdrTS0N7eWlANznyyctKJG/iEQY10ZWIrcThqoUnpcbPZ/GQMTkluJL7aHMSRnzk5bVSPq/PbE7GORqvpb8c4pS1nkVWleua6dtVzbS9VsI7BuqMfWJdy8fZInwb0qOAhsg+8DGoRxLwY93IxnoIrmyEi6GtJ6BEUMaRjeWRrvXlKm/VWeZkHEtfFi0vdSqpMdR1tHMibaONgzp7CSQiLorUVl+uz8vM1bU6m5WpumTPZmPqy4x5s2spU0nNQ13n8hmCq1zeXawNBEAABEDASgIi4Dn55FM4zNSqkGHvuON2OvPMM0PKol2Ii8j9998f0uTaa6/l8HdXaWV/+MMf6Kmnng7Wy4fSc+f+N3idaMZtgqtZs2bRrbf+PgTT3//+N82hJaSQL95//336xS8C3FUdBFeKBM4gkB4CIj7605/+HHLzBx98wNTZKaRRkhfiOjVp0hm0ePHihEb405/uolNPPTWkj1Fw5fV6yefjr+rGSGVlZVRYWEgiioiVJNzMv//9boho1kxwFe+9BwwYoInCYt1X6kX0JuI3lWS+8v9FfYhXVZefn0/iGmaWjjzyCHbXmmFWhTIQcA0BvWjFLsGS3knJyrCC6XS3Ui8QJSaTaytdrpxam5q/hCifMvU2taysOOtfuxBcZcWWYZIgAAIOEIDgKnnIuSi4qmvkkO8cuk9CC/53cROJu5KfbZzEySmZJC5P7dsEwtmNZnelQwYXUhsWAJUURbO0iHwnTZTE1Zq4ic8iupLDmKTeiZCCcl81JzUHbU7qIoFzIJQgUSU7W83/oZnWbvXT5h1+amDHK7lHMknmIi5jEkZQXMY6l+VRp1IThVoygzvUB4Irh0DjNtYRMBMZGcsiXevLVT7SWWYcqS5Wub5vtLyxTq71Sd1HX4Z87hGIJj4y1umvY+VVfaSzkIxUF6ncrI/aEdUn0nWsclWfa2cIrnJtR7EeEAABEAABOwksWLCAzjrr7JAPkeVD7csuu4yFO1dqH5hHun8Lf33sxRdfpNtvvyP4e7y0lQ/P//WvN6lNmzZa148//pjHuzxkmPvuu49+9rOfhpSpC/kw/89//rMWTkoct4zJbYKre+65hx599LEghvLycvrkk/+FubJIA6O4TcoguBIKSCCQPgLTp9+phfXTz+Cjjz7U3iv1ZSovoUPjdVeSPqWlpcFQUXJt9j5wzDHHcJi/CSRuenV1tfTFF1+yC9Q9JO54Ksk477zzNnXp0kUVcSjT0JCCqsLDsQZGjRrFx4EcLnAZv898rKrCzvJ364gRI2jMmDEs8F2phbI1Pnv529/+RscfPz7Y10xwpSo7deqkidW8/PVnCYurX4Nqo84StvDoo4+isrL23PZ9DutYpaq0szgnyv+v9ElCKErYQhGuFRcXs/PX5cztUG0NDQ0NzO4LzUFr2bJl+m70j3/8ncaNGxdShgsQcBMB9eFi/XA/bbi9zralK2GP3MAqYZJygJIx7XSBkvEjJb2YzCoHL73YSu5rlxBOxlb7AsGV0HAuIaSgc6xxJxBwGwEIrpLfcfU7kfwNJGHMcyGJk5KPQ9d9v95H85Y20arNfs1hScLcJZvE7anAm0cH7FOgHT075lOP8nzNBUoEWXYkJwVXVs2/praVtu1q0cRu81cE3MVEbCVCrGSScG9b5GGRVT4dOjQQ1rGkMOB4lcx46eoDwVW6yOO+SRMwPghTAxnL9ddm+Whlqi7WWe4dqY2+zpg3u45UJuXGpO5pLMd1ZhMwCpIizdasnbFMf22Wj1am6mKdZX7GNpHK9OWSl6T6Bq72/hupfG+L3MpBcJVb+4nVgAAIgAAI2E9A3KnEpcqYxCXkV7+6jvbbbz/q0aNHsHrHjp20YME39Je//JUqKyuD5ZKR3zueeeYZ/mB9dLBcHEGOPvqYkA/FRdQlDi9HHXVUsJ1kpO0vf3ldMFyV3Puqq37B/Y8OtnOb4MroICZCBxEOiLBNn2QvzjvvfNq9e7e+GIKrEBq4AAHnCVx33XX01ltvB28s7k9ffvlF8NqYkZB9xx9vLkg1tpXrY489hh566CGtqqqqio455lgOp7D36auIW//617+G/b24Y8cOOuWUU1mEtD447A033BASWtBMcCUhA1966UUaOXJksN+zzz5Ld9wR/mGCvNc/++wzdNBBBwXbyv8jRISmT7/85S+193pVFklwNX78eHZUvI/kfVCSvN9JuEb9GtQY8v8N+f+MOFNJqq2tpRNPPDFEdCVr+fbbBcE2qu/ChQvprrv+xIKz35GIB4xJQrdOmHAybd68OVh16aWT6cYbbwxeIwMCbiOgPly0W3BltTBJL0pKl9hKvVaUaEmurXDwclJIVj6zkMpnFWlLyTaXKDhcqVcgziAAAiCwlwAEV3tZJJpTvxPlkuBKhEriZiUh7Raz6GpplY9+2OjTRFeJ8lHtxWVJhFXdO+RTNz4O7F/ARyEVFxD/fcbPF1VDC8/ZJriS+S7d4GN3Kx+t2OSjVdV+qme3MT9XSF0yqaggj3qxuK1fF68mdBvW26uJ3ESIlU0Jgqts2i3MVSMQSWxkLNdfx8qreuNZbmgsi3Rt1lZfJnlJqn/gKvxalZu11dchnzsEogmQjHXRrlWdOgshlTeeo9XF01bf35iXa0lqnMDV3n8jle9tkVs5CK5yaz+xGhAAARAAAfsJiMhp0qRJ9N13iyLeTFxC+vbtSxs2bIgaIur888+n3//+1rBxXn31VfrNb34bUi4fgssDqLFjx7A7Sxf6/PPP+fiMRNClT8cddyx/aL5XEOY2wZU4uFx55ZV6JJpQ7bLLLtUcZrZu3cqOV/NYHHAXSd6Y4HBlJIJrEHCWgIiJ3n77neBN27cv01ySggWGTKKCK3kffeWVl7VRxCnqH/94IDjioEGD6LXXXo3oVvjuu+/SNddcG2wvwoPXX/9n8NpMcPWb39xEl1xySbCNZOR9e+zYsSFuiVJ+00030uTJkyUbTNu3b6cf/ejQkLYSylBCGqpkJrjq3r07vfnmG+xWVaaaaWcRRj3xxBMhZdL2jTdeJ/l/lz6ZuX9FcxvT9zXmRWAmQjOVROD1yCMPq0ucQcB1BJTTjhOiJb24JxVhkl5sZbdQLN4XhBWiq86dO9PQoUOD7odO7wkEV/Hudurt1M+duFjKgQQCIAACVhGA4Cp5krkouFI0trPb0obtfs3p6uuVTbSxpkVzvko2tKCMWyKhBPk4gMVWIrrqzKHtytvlUUF+Hn/RxlrhlYiUJDTiW1810IcLG0k0S6IhkHLtUAuNcd6vXwEdu18R7cOipTYcBrEg8B2fGL3iq1bz2MXz3FXfoomtFqxu1sII1tQK7/jGMbYSAZvwbMehBIf3LaBhvQqof9d86t0py5RWexYGwZVxh3Gd8QSMgiU1YWN5tGtVZzzLWMYy43W0Nvq6aHljnVxLUvcKXEX+N952kUdATToIxCs0MmtnLIt0rS9XeXWWNat8pHO0Nvq6aHmpk6TuEbja+2+k8r0tcisHwVVu7SdWAwIgAAIg4AwB+bBcBDsvvxz40D6Zu4ow6t5776WSkhLT7iK4EuFVIqlnz570z3++FvKhudsEVyKI+8lPxpmGGJOwjXV1dVGRQnAVFQ8qQcB2AkZhjtww2s9looKrAw88kGbOfElbx2mnnRYmnp00aWLENdbV1XNIvX8F60UIu2jRd0EHKTPBlYTw6tKlc7CPylx44UU0b948damd5837hCQEoDGdc8657PL1ZbBYQh4+/HDApUsKzQRXZ511FoewnRbsozLz58+ns88+R11q53POOZtuu+22kDK5+PTTT+mCCy4MKX/11Vdo+PDhIWVyIfswd+5c5vmdxlRcrcrKSqlPnz6a8+PWrdvY6SvAXdr379+f3n13r7BOypBAwE0ElPDDCXGPcNULk+Q6EeGVUZAk/as4DGIDh0NMd9I7eMlcJLygrE3O8SS9iEzaO7Uf+nlPmTqV3QHD31fjmX862sDhKh3UcU8QAIFMJwDBVfI7lMuCqyYfUX1Tq+Zw9cmSRi20YC07LjX5RLqUXBJnpXy2upKQgj3LPTSUhUAV7LrUrjiP3a7y+HPP5MY16yVipl0NrfTvbxrpP5WNmlOUhObTDv41kKVXmvDKrK++zE7BlZrPcnYQW8ZOYquqfbSana0UZ1lDMkncxAqZpwjaDh1axAK3AiorydM4JzNeuvtAcJXuHcD9EyYQSWxkVq4vM8tHK0u2ThZk1tdYrhaub6vK1DlanWqDc/YTiCZAMqvTl8XKm9VHK1N16ix0VV6d9WWKvr7OrF61i1Wnb5creQiucmUnsQ4QAAEQAIF0EPjkk0/YoWoKrV27Nu7biyjq97//vRbWKlonn8/HDiZ/oqeffiZas2CdfAAu4aD04Qyl0m2CK1nzmjVr6KKLLjYVXUm9pMLCQhKHsccffzxQsOffaMKOkIa4AAEQsIXAjBkz6M9/vjtkbAkBKKEAzdLGjRtZLBQuLFJtFy78lqqr937wftxxx2nvlVI/evQYEgepVNL//jeXBVVdtCGMgitxjPrii89Nh7/ppt+wm9ZrwToRWongyixdf/0N7Fb1ZrAqHsGVuCfKe5wxrVu3TgujqC8XsZWIroxp1apVNG7c+JBiM8HV7NlzaNq0adTU1BTSNtpF79696cMPP4jWBHUgkNMEnBZcCUy905VcizBJ0vfff6+djf+I0EqJrVSdOFvVnNmYEWIrNSe9eEmVKdGVmfBK1iRJ72ol106JreRe+jlDcCVEnEnq5w4OV87wxl1AwE0EILhKfrdzWXAlVETwIwKg+SuaaBmHu9vALlfixKScmZIlJ05RcgzrXUAj+ni1MIOdWBxU5CXN7coK4ZVolepYIPbl8mb6hl2j/GzNpR0sthKhk6xB3LqkXeCf4ClkWYN75NMhgwpZIJZPxezO5bXAJEruL/fewS5W2+taafG6Zqpc56MtO/20bXeLNr+QSSR4UejNI+HZp1M+jR1aSCPZ5UrEbiLEysYEwVU27hrmHCJo0uMwCpT017HyZvWqTJ3lXiqvzrHK1Pz07fV9VH2kMn29Wd44rlkblKWPgFGIFM9MzPrEKtPXq7w6yz3N8qpMnfXtVJk66+uM+XiupY0k/XiBktz/F4Kr3N9jrBAEQAAEQMBeAvX19fTQQw9zOKY3aP369RFvJh+mn3baqXT11VdHdLUy6/zRRx/RjBmPRwyp1bVrVxInk0su+bnpuG4UXAnH6upqZn0NffPNN8G/R6QFCwAAQABJREFUkRRfcTO49dZbqE2btrwnp6li7QzBVQgOXICA4wQqKyvplFNODbnvueeeQ1PZASTRJI53Rx55ZIjg6oILLtB+/qVu2LCKRIcMa//+++9pIWSlwii4GjlyJDshzgnrIwVGwdWoUaPYAepF07bJCK4ee+xR+vGPfxw2npng6qmnnqRDDz00rG08gisRa73wgvm8wwbUFUBwpYOBrCsJKOGHkyIfAW0UXenh68VJSpSkr8+UMIL6Oenz0dYmAiz5XdxsXTKG0/sAwZV+55zLq587CK6cY447gYBbCEBwlfxO57rgSshsZ1HQ+m0tmtPVN6uaqIrzIlwSwVCyycth+bys/unIoqDOZR4a3MNL+/byUqd2HnZi8pDUW5EkJN+WXS1UwyKmgLhqb0hBuW7ZI3zS6rT1hC+qQ1sPC8I81LbIY4loSe7QyM5hDc38xYH1zVrIxo3bW2gjh29s4PJGdhBL1tlKMROGw/pIKEEvDegmoQTzSbRWVgjZ1D2cPENw5SRt3MsyApFERsbyaNeqTp1lciqvzmZlZnX6dvHmpZ0k/XiBktB/Y9WHtsZVthGIJUAyqzeW6a9VXp2Fh1k+WplZXaRxjOVyLUk/RqAk8G+kcn2bXMtDcJVrO4r1gAAIgAAIpJPAsmXLNJcA+cBKwg5KOKWOHTvyh/rDaMiQIRF/B4lnzvLBd2XlYqqqWk+NjU3sZNWdevXqTQcdNIq8Xv4KG5IpgZ07d7LoaoEW9kv2YtCgQSTCBjf+3mcKCIUgkIEEjjrqaH6vqwrOzOPxsKjnee1nN1gYR+Y///kPTZ58aUjLRx55mI4++mit7JBDRvN79Y5gffv2ZRxC74LgdTyZyZMnB8WuRsGVPnyhcSyj4Gr06NH03HPPGptp17/61fUhoQzjcbh64onH6fDDDw8bz0xw9eyzz9KYMaPD2q5cuZLGjz8+pFzvcGXGtz+HCjz//PM0EVoLP32vqtrAoQnn0Ucffcz/72oMjgXBVRAFMi4loIQfTgt9FO5o4iTVRp0z0dVKzc14TmRd0jdda9MLri648ELaZ599jEvJ2GsJKSjiaEkvzZyVsfM0m5j6uYPgyowOykAABFIhAMFV8vTcILiSEIJ1/KfQys0++nx5E63Y6Ked7HIl4QZTTSKsKmA3JhFcVbDblQiDeoi4iUMMikuTnY5MmuCKlyCiK78shQ8JM2hM4gxVkB8Id5iKQZSMLAKwZua5lQVgW1kItpCdt75b46PdDS1UxzxTFVop1672LLiSMIIj2Nmqb5cAU1FcpTJ/IxcnryG4cpI27mUZgWgiJGOd/jpW3qw+3jJZnFlbY7mCoG+ryiK11dcnmo90n0THQXtzAlZ/kBRpPLNyfVmyedVPnWWVKq/O+jJjPp5raaOSfkxVlutnCK5yfYexPhAAARAAARAAARAAARDILgKz+cNcES/pU79+/ThE6j+D4iZ9nVleXO5OO+102rRpU7C6pKREE//IWdLJJ59CixcvDtafdNJJdO+99wSvE824TXB1ww2/1twdFScR//7f//0fh1gMhOtS5XK+8847Q0LkQnClp4O82whUVi6i2zkMp6Sq2+vSGp5PBEqSiheFi/cldKCkBg4jmG1JBE0l33HYmj3rKuFrEVeplO6wiHrBlZpTNp4huMrGXcOcQQAE7CAAwVXyVN0guBJBko+Pag53J2KrZRt9mtvVJnZkCpcnJcZSBFX8/SRq38ZDHdndqn9XLw1i8VWPcg91ZeerogL7JEIyd03gxBktL1M3WZC4Qsk8U3WHEoHXDg4fuJ3FVks5POOSqkAIweqdLdTMv+b5WfVlcvuEgEqowqbmVk0g1rdzPg3s7qWDORxiBYdtREjBhFCiMQikTiCaiMhYF+1a1amzzMwsH2+Zsb/xWj+O1KkUqVzVq3O87VR7nDOTQLyio0jtjOX660TyZm3NyoRiPOWKtr6tKlPnaHWqTa6dIbjKtR3FekAABEAABEAABEAABEAg+wlcccWV9OGHH4YsZOjQofTHP/6BRowYEVJuvBAXp1/+8jpauHBhSNWUKb+n8847L1j2pz/9mR5//PHgtThczZo1i/qzS5NZEvHWQw89pI0hbnnG5DbB1aRJZ9CCBQuCGI477lh68MEHg9f6jFHcBsGVng7ybiOQSYIrt7HPlPXmguCqoqKCpky9LVOQxjUPOFzFhQmNQAAEkiAAwVUS0PZ0cYPgStGpa2ylbbtbWXTloy9XNNPKTT4Oi9fKYqFUZUIBMZN8vikiIXG72oddmfpwvqxNHhWx05U4TIlgKFXRk1qLU2flntXInCRc4KYdLbSJwwcuWtusHT5WYYnrlRVJBF0+ds+q532S+5axiK1neT4dXlFIowYWcqjGPGrHzmHZ6HQFhysrXiEYIy0EIomPjOXRrvV1ZnmzMlmsKldnBUB/HSlv1laVqbO+ryrDOfcJRBMjmdUZy/TXsfKp1qvd0I8jZcZr1S5Wnb5druUhuMq1HcV6QAAEQAAEQAAEQAAEQCD7CUho1hNPPIm2bdsWspj8/Hwt7N+JJ55AgwcPpuLiYq3e7/fT2rXr6OWX59CTTz5FTU1NIf3Gjh3LDktPhfxNKKFax40bH9KuW7du9OKLL5AIgvSpunoLnXvuuSR9JMTh+PHj6KqrrtLCxap2bhNcXXDBhewY9qlaPokL2bvvvqPxCRZyZvbsOexYdou+SOP74YcfhJThAgTcQgCCK7fsdOR16gVX2RZSUK2qomK4ymbNGYKrrNkqTBQEso4ABFfJb5mbBFciDBKB1UZ2tqpc66PlLLxat9XPIixW96SYWAakCYFKiz3Uvm0edS7Lp67tA4KhXp0C7lftigJhBlO8lWPdRQAlIqtaFkBVbfNTVY2f2Yngyq+FExRuKgRgqpPSXMhY+CbCLhFc+XxEhewO1p4Fa0N6FtC+vb1aeMEhvbLT6QqCq1RfIeifNgLRREnGumjX+jqzvFmZLFqVq7MCkeh1pH6qPNbZeL9Y7VHvLIFoAqRoM4nUz6xcX6bPy/j6a5VX50j1yZRLH0n6sQMle/+NVre3Ve7lILjKvT3FikAABEAABEAABEAABEAgFwi89957LGq6OuJSRHy1zz77aAIfEUI1Nzebtm3bti29+eab1KtXz7D63/72ZnrllVdCyiXk4MEHH0Rjx/6Ix2yiefPm0ddffxMm4rr55pvp4osvCvZ1m+Dqj3/8oyZuCwLgzCWXXEITJ55OAwcOpB9++IHef/8Duu+++zi8Q+jXjuFwpaeGvBsJKOFHzRmNVHNmqEDUjTzctma94GrK1KmUjeKlbNwz9XM3YcIEkgMJBEAABKwiAMFV8iTdJLhSlHbWtdDqar8muFrEwqs1W/zkZ3WRhLOzKrVlcVVZW4/mciWOV+LU1LnUQyK6KuBI0vkc48+KMH9WzVeNIwIqMfxqZqcpET/t5BCCW1lY9QOL01Zs9lE1O1xJCEErk3ZP5i/irkY+GvhXc7m/pAJ2B+vMoRl7d/LSEex0NXpIoeZyJXyzyekKgisrXzEYy3ECkcRGxvJo1/o6q/ICQj+W2XWkMinXJ+M4+jrks59APCKkSG2M5fpru/NCXn8PtRNmZfHUqTa5eIbgKhd3FWsCARAAARAAARAAARAAgdwg8NFHH9FUDlm0cePGpBZUXl5Of/jDH+jYY48x7V9fX0+nnz6Rli9fblofqfCYY47h8IIPhvzd6TbB1cqVK+lnPzshTEwlzETkVltbGwkfHK4ikkGFWwgo4QcEV27Z8dB1QnAVysOpK/VzB8GVU8RxHxBwDwEIrpLfazcKrppYzCNCovXs2vTt6mZattFPNbv9tKs+IPJJnubenhJCsLCAqLTEQx3E8YrFVl3Z9ap7Bw/1YPFVeTsPFbODkzd/b59058TRqpmdpcTRSlzAxMlKQghu5mN7bQvtYKGahGWsZ1GUVSkgtiKSPalvCAiuJEyhOF5JYnNrKikUpysPu1wFnK4q+hTQ0CxzuoLgKrCf+DdLCUQSI5mVG8v013bkFVL92FJmvFbtYtXp2yGf2wQSFS0Z2+uvrcoL8Uhjqd3Q16sydY5Wp9rk6hmCq1zdWawLBEAABEAABEAABEAABHKDgAh37rnnHnrhhRejPrPQr1b+xjv99NPopptuog4dOuirwvISvvD666/n8HifhdWZFZxyysl0xx13UFFRUUi12wRXsviPP/6Yrr32l9TQ0BDCQn/Ro0cPGjNmDL322mvBYjhcBVEg41ICSvgBwZU7XwDlMwupfFbg/yFwuHLuNaB+7iC4co457gQCbiEAwVXyO+1GwZWiJSHxvlvjo++rfLSa3ZtEZCSiIxEBWZ1KS/KoCzs19e3spSHseCWiKykTMZGIivL58PDf0JKX8ISctTXJGmWt2sH/iLtXM5si17GYqmaPo9XKzX5NlCaOVqJdsIOLdl8WW0moxzoWvDXx2SyJMK0LC9Z6dsqnI4cX0aH7FlIbdrmSw2ZUZtNJuAyCq4SRoUMmEUhEvGRsG+1aX5doXvjo+yheZmWR2qo+ZudI45i1RVnmEUhUeBSpvVm5sUx/nWheyEXqY6xTlPXtVZk6R6tTbXL1DMFVru4s1gUCIAACIAACIAACIAACuUVg/vz59MADD9Jnn30WMXyghBncf//96de/voHDAh4cNwAJeffcc8/T888/R6tWrTbtN2zYMC2E4CmnnGJa70bBlYCQfbnhhl9TVVVVGBdxFrv11t/Tu+++Q3fd9adgPQRXQRTIuJSAEn5AcOXOF4BecPXSzFnuhJCGVaufOwiu0gAftwSBHCcAwVXyG+xmwZU4NYmYSEIKLmSnqx/Y6aq2scVSBye1M4UcGk/EQWUssurYLp9drzzs2iTOTVzG7k1lkmc3rLbFeVTEzlcFNjpficjJx4esX8IrirPXzvoWLb+9tpXdrFppW61fO+9uCLhaWS22UoIvCR0ooQTlEHctv8QzNEkSfrFE+DGrYex0Jce+vb3aERCrmXTKoCIIrjJoMzCVxAlEEx+Z1RnL9Nf6vMxEfx1P3tjH7FrKJOnHC5SE/hurPrQ1rnKFQCxRUqR6Y3m0a32dPi8M9deR8sZ2ir2+vSpT52h1qk0unyG4yuXdxdpAAARAAARAAARAAARAIPcI1NXV0ZdffkkbNmyg6uot2t+KHTuWU/fu3TWRVWlpadKLlucdCxYsYNHVKk1AVFBQSD179qD+/ftTRUVF0uO6oaPsx9dff00rV66iXr160YgRw2nQoEFuWDrWCAIJE1AfLtYP99OG2+sS7m91Bwlx18BzQXKGAARXznA23gWCKyMRXIMACFhFAIKr5Emq34lGjBhB06dPT36gLOwp0h5xeNrCoqsvlzfTd2ubqVoLnxdwupI6q5PmxsT2VSUsqmrPoqtOpXnUrT2HG+SjW4d8vmbRFQuLxPlKXK5EaCR9tM9RVV4mJYV7TvppKmFU8ExcG/hPc6iS8ib+lVMcpXZwWMXNO/zamjfv5DNz2MqHhFts4YZ2rD8waxZWseBLjkYWWtXuCSUoE1XzVu2MZxFXdWnPoRnZIeyoEUWa25WwKubwjXa7ghnnksg1BFeJ0ELbjCQQTZxkVmcs01/r87JY/XWkvLGd2XWkMimXpB87UGLtv3aPb+1ss2c0u4VE0cY3qzOWRbtOtk52x9g3UpnaSbP2qs4NZwiu3LDLWCMIgAAIgAAIgAAIgAAIgAAIgAAIgECmEFAfLqZbcKUX/ig2uea6JWKynlPaqOVp53SvUc8dDlchW2PrBQRXtuLF4CDgagIQXCW//ep3IjcKroSaCHwkjF7VNj+t3dJCyzf62PHKRzvY5WkXuzvZkUQr5c0POFmJWKhtMWkiqzZFHmrHDleBIyC8Uo5X4npV6CU+8sirhSDk8IM8kIQglMTL0NyhJCygj12ixMFKO/O1lMkhLlK1vKbaxlbazSInOeo4L65ecpZDXK8aWYwlXGRMq5OMq4UR5Dk2NrH4i+/VxC5X4mwVS2wlcxFRlTiFlbIbWEWfAhret4CG9PTS0F5eZhoQqFk9ZyvGg+DKCooYI60EoomJzOqMZYlcJ9JWoBjbK1CRylW9/pxIW30/5LODQCJipEhtjeWJXCfSVoga20cqU/TN2qs6t5whuHLLTmOdIAACIAACIAACIAACIAACIAACIAACmUBAfbgoc1nx8q60TEkv+jFOIN1CMON8kr3O1DX2YAFYCQvBxDlxytTbkl0e+iVIAIKrBIGhOQiAQNwEILiKG1VYQ/U7kVsFVwJEhEUtLFASZ6cFHFqwcp2P1m310abtAaereIRAYWCTLBAxVbviQIjBzmXigCUirMAhzldyFOiFV3tCD8ocG5vlCAimGiVUn5YPuFk1sLhJwgdu3e2nmtoWzd1K2jutMdgrtmKBFwu+mngOsgOJMhaxmThdiSvY0SOLtEPEayJKy0SnKwiukvyBQLfMIRDrzcKs3lhm97XQMt7DSDBWvbE9rnObQCyhkll9PGX6Nvq80Ez02qyPcVeMYxrr3XANwZUbdhlrBAEQAAEQAAEQAAEQAAEQAAEQAAEQyBQCc2bPpjlzZmvTqeKQgk6H84smRFKM0u0CpeaR7DmT1zjg9EDo24kTJ9HESZOSXSL6JUgAgqsEgaE5CIBA3AQguIobVVhDCK4CSETwIwKlTRxScEONn1Zt9tOaal8gzN6ugNOVCLPsTiIkEhcrcbQSJyclIpKyAj4KWWCVz43E2SrE4YonJ2ImcbXSzhwP0C9uV1pZoFzEWOJgJetsYKGTtHUqyZz8PCfN0YrvLa5WzT4RuiUutpI5i6hKE6CxGG1Ev4DT1eAeXhrMblciuhI2mZQguMqk3cBckiYQS6xkVm8ss/paFmMcU7/AaHX6dvp8Mn30/ZHPHAKJCpGitTerM5ZZfa1IGsdV5XKOVqdvl+t5CK5yfYexPhAAARAAARAAARAAARAAARAAARAAgUwiUFm5iG6fNk2bUjoEV8phKRqTbHe5ikdwJetPh8MYBFfRXnn21UFwZR9bjAwCbicAwVXyrwAIrsLZiSBp+QYfLePj+/V+LcygfP7O2iCkJAk0s8BKc93isIUNfPhYDGZVEperrnwcNaKIjtmvKOACxsK0THK6guDKqt3GOGklEEuIZFYfT1kybcz6KDjR6hJpo9rinHsE4hEpRWtjrDNeCzFjWaxrsz6RyqRcJeO4qtxtZwiu3LbjWC8IgAAIgAAIgAAIgAAIgAAIgAAIgEA6CaRbcKUEP7EYpEOMFGtO8dbHIyqTsZxeYzGHEuzJIQUlTZk6lcMKDtfy+Md+AhBc2c8YdwABtxKA4Cr5nYfgKpydiIG27m6hLTvF6aqFVlf7qYpdrzbW+DTRFYRX4cwilQhLEVtpgisWsomrlThrSQhHq1K7Yg61yE5Xw/sW0Eh2uxrETlcDu+dTMYcYzBSnKwiurNptjJN2ArHETGb1yZYl288IyWwcY5tUru0eP5W55VJfu4VFscY3q0+2LNl+xv00G8fYxi3XEFy5ZaexThAAARAAARAAARAAARAAARAAARAAgUwhoMQf6XCSildwlQ73Lav2J941Oi240jtvvTRzllXLxThxEFA/cxMmTCA5kEAABEDAKgIQXCVPEoIrc3YSYrCZhUGbdrRS1TY/fbWiiRasatbEQhKmDykeAnnUyCEM6xpbtLOIriS0oNVJix7I/3Rtn0/dy/Pp8Ap2uhpZRGVt8sjLoRczwekKgiurdx3jpZVALIGRWb0TZQqK2b1Undk50fZmY6AsOwgkIlCK1jZSnbHceC2UUinTUzYbR1/vtjwEV27bcawXBEAABEAABEAABEAABEAABEAABEAg3QTUB4zpEFzF6/4EwZX1rxIIrqxnGu+IEFzFSwrtQAAEEiUAwVWixPa2V78PjRgxgqZPn763AjnNzWpXfSvtrGuhlZv9fPhoPYuvRIDVxE5NIiBClMHQF4oInLz5edSx1KMJoLazU9j6LX7atquFdta3aMxCe1h3JU5X7Uo8VNGngPbbJ+B01b9rPpUU5WmiK02YZd3tEhoJgquEcKFxNhCIJlKKVJdIuVlbszJhFalccYxVr9rh7G4C8QiYIrUxK0+lTHbCrL/aoWh1qo3bzhBcuW3HsV4QAAEQAAEQAAEQAAEQAAEQAAEQAIF0E5gzezbNmTNbm4bTwqZcF1zpw/bF2menHa6U89bEiZNo4qRJsaaHegsJQHBlIUwMBQIgEEIAgqsQHAldQHAVHZc4XYkr0w4WXW2vbaX5K5rp65VNtIvFQ7WNLLiC4ioEYFFBHpXwMbinVwvvt3FbCzuDNdFaDsu4eYefGtjxyq4kTlZydGOnqx4d82ns0CI6akQhlbfzpN3pCoIru3Yd46aNQCwRU6T6RMoTaatAROqj6iOdk+0XaTyUZxaBZAVK0fpFqjMrNysTQomWK6qR+ql6N54huHLjrmPNIAACIAACIAACIAACIAACIAACIAAC6SRQWbmIbp82TZtCpgquas5opJozm9KJKal7612kog3gtLuYXggGwVW0nbGnDoIre7hiVBAAASIIrpJ/FUBwFZudSIQam1upnn8lW1PtozXs2LR5R4smIKphB6caFmL5W1o1R6zYo+VWC801iv8pY2epsjYe6lImzlYe6sWCpz6d8mk7C9VEbLW0ykeL1zZr3OpYqObjcI12pbbsdFW6x+lq//4FNLC7l/qx01XbNDpdQXBl125j3LQSiCVSilYfqc6s3KxMLTxaXSJtVFuc3UsgHhFTtDaR6qwqVzsTaTxV79YzBFdu3XmsGwRAAARAAARAAARAAARAAARAAARAIJ0ElADEaeFPpgqSrNqLeNfntKBMP6+XZs6yarkYJ04C6udtwoQJJAcSCIAACFhFAIKr5ElCcBUfO3GykqOJhULNHE5Qwguu3OSnHzb56IeNPmpoJmph0ZV9MqL45ul0K3GU8vA/Iq7q18VLg3rk0+AeXpLwfuJ2JcKqRmbz7apm+r9FjbSMhVfV7HRVb6PTlYfnJPPq2iFfE34dMriQjhhepInBJOSh1DmdILhymjju5xiBWIKnaPWR6iKVy6Ki1cVTHw1MrLGj9UVd5hJIVaAUq3+k+kTLhWCkPopurHrVzo1nCK7cuOtYMwiAAAiAAAiAAAiAAAiAAAiAAAiAQLoJqA8ZnRZc6Z2WYjFw2n0r1nziqVdh+2K1dVpwpQ/lCMFVrN2xvh6CK+uZYkQQAIEAAQiukn8lqN+FRowYQdOnT09+IJf01IRXvNYtO1uoeqefqmpaqGqbn7ay09V2PnbVt9LuBnZwEvFVjqqvRNDUpsjDRx6Vt/VQBz56lPPBAqdufO7O50KvCLFYhMYMWjgk41p2BVu8jl2u1jUHna7q2emq2WanK3He2rd3AR04gJ2uWAjWp3O+JgYT0ZWTuisIrlzyBuHGZcYjUorWxo46/T5EG1/fDnkQEALxCpqitbOjTu1OtLFVG7eeIbhy685j3SAAAiAAAiAAAiAAAiAAAiAAAiAAAukkkK6wgokIrpwWg6W6H3oXqVhjOS24UkIwhBOMtTP21ENwZQ9XjAoCIICQgqm8BiC4So6ez0+aqErEVXJIqMGVm/20biuLsLa1UD2HIGzNQccrESl5WEnVrT2Lqzp4aEA3Lw3slq+Jrtq3yaMCFlp58/M0FyklaBLdWSM7WtU2Ei3isIL/Yaerpet9LFrzk4QXtCt52M0qn5VVXTjEYe/OXhrFoqvDK4q0eTvtdAXBlV27jHEzgkC8oqZo7ZKtUwCi9VdtIp1T6RtpTJRnHoFUxUqx+kerj1YnpGLVx9sm86g7NyMIrpxjjTuBAAiAAAiAAAiAAAiAAAiAAAiAAAiAgJ6AEoE4LWzSuy3p52OWX/HyLrPijCzL1HXphWBTpk6liorhGckvlyelftYQUjCXdxlrA4H0EIDDVfLcIbhKnp30FIemJg4xWM2OV5u2+2nzDnG+Yrer2hbaWSdirBYWGkmbVs3tKdtcr8SlKp//ETcrOcraeKh9SR51LsvnQ9ysAo5WJYVExRxCMFK4PuV0JYK0RWsCTldL1jfTRnYIa2Axlu1OVzzvob0K6KCB7HTV3Uu9OQRiO16HU05XEFyl9nOG3llAIF7RUqx2qdYrVLHGUe1wBgEzAvEIoKRfrHap1qu5xRpHtXPzGYIrN+8+1g4CIAACIAACIAACIAACIAACIAACIJBOAuqDRpmDk8KmXHS50ouaYu1putytZF4IJxhrd+yph+DKHq4YFQRAAA5XqbwG1O9BCCmYHEURUImYyMfCq2Z2vRLHpt18iJBoTbWf1m/z0QYWYkmoQREVSXi9bEkiRhKxVRG7VnXjMIHdOVxgP3aK6tsln0qL86iUw/VJ6MACdrQSNykRZ0VLwqqBnb+ExffscDW3spG+5zCDW3f5qZZdwuxKMjcvT06crvrw/PfvX0CH7ltIPTvm8/piz9uKeUFwZQVFjJHxBBIROcVqG6teYMTTJhq0VPtHGxt1mUsgVfFSPP2taiMU4xkrc2k7NzMIrpxjjTuBAAiAAAiAAAiAAAiAAAiAAAiAAAiAgJ5AusIKyhxUiDv9fCLlq26vo4bh/EleBqdMXY9e3IZwgul7AakP9YcOHUo33nhj+iaCO4MACOQcgbvvvpuWLFnC7oUVNGXqbTm3PjsXpN6bIbiyhrKEGmxiYdW2Xa173K78tGW3X3O7ElGRuF3Vy8GuTo3setWcAc5XopMKiqsKAk5VJYV5VMKOVu3kYGFVx3Ye6lQacLTqyuEEi7ldETtaJZqU09X6bX5auLqZFq/10dIqZ5yuxKGrQ1sPDdnjdDW4h5d6sOiq1AGnKwiuEn2loH3WEkhExBRP23jaKFiJtFV9cAaBWAQSETzF0zaeNmpOibRVfdx6huDKrTuPdYMACIAACIAACIAACIAACIAACIAACGQCAeW8k8lhBZ2eW6L7kkgoQRnbSTcxvfMW3K0S3Vnr2qsP9SG4so4pRgIBEAgQgOAq+VeCem+G4Cp5hvqeescrCTUo4is576xrYSenVg436OfQg4GwgzW1fg45GBBd+dPofCViK3GAEhFVBxFW8dGV3aBEWNWFwwaK0KrQm0eFIrLicwEf4mYVy9FKz0WfF0b17HQlIReXVfnof4ubaPG6ZtrKbISHXUncrLzsxiWhEPt28dLIfgU0dmghu17Z73QFwZVdu4pxM5ZAIuKneNrG08YMRrL9zMZCWe4TSFbgFG8/q9vl/o7Ev0IIruJnhZYgAAIgAAIgAAIgAAIgAAIgAAIgAAIgYDUB9WGjjOukk5TeeSmeNWWq6CpRsVW6wgnC3SqeV5l9bdTPGQRX9jHGyCDgVgJKcIX3+cRfAeq9GYKrxNnF00PkQ5rAiB2tdrDAqGa3CK9aqKa2hXbWt2gCowauk0MTaLHjlY/FVxKiUNyy/GwJJWcpa+GBWvkcryRJhFQiipKwgN78vWfvHuFRAZcVskuViKiK+SwOUGXs9tS+bZ7maCWuVuXsCFXWxqM5YCUrsDLjJExEZLaBQy8uWNVElex0JeKrDex81chiLAm/aFfSO12NGVJIQ3p6OWSih9pxmES7EgRXdpHFuBlNIBGxUyJtZdGJtk8ElJ1jJzIPtE2NQLzipmTukujYibRPpG0yc8/VPhBc5erOYl0gAAIgAAIgAAIgAAIgAAIgAAIgAALZQEAfVtBpUVOiYiWn5xdr/zJ9/np3K3wQH2s37a2fM3s2zZkzW7vJjBkz7L0ZRgcBEHAVgcmTJ2vrxft84tsOwVXizBLtIdKhFhYXiYiomcVTEkZQziKqamBx0e56dsBi8dV2TYQlYQdbqE4LPchOUCzEquNDztJPxFeiBYglR5LPazVHJxZXFUl4QD7ayMGiKnWWUHrtWUwlgioRWkmdiLAK2AWqwKvOLNbiMUS8ZXVSQrTtta30w0YfffI9C6/WsNMVi9J2Mw+7knK66l6eTxJWcAQ7XR08qIB6d+LF25QguLIJLIbNDgKJCJgSaatWn0wf1RdnEIiXQDJCqET6JNI23jm7qR0EV27abawVBEAABEAABEAABEAABEAABEAABEAgEwmoDxxlbpnsciXzE9FVzZmN1MDndCVx5yqfWUQlfE4kwd0qEVq51VYvbLzxxhtJnK6QQAAEQMAKAhBcJU9R/f4Dh6vkGabSUwRUIq6SUHoiutrF53oRW+0RWYnzlYTfE2GWj0MTikhLpEgiVpKDDbCCSURRoosSJyrJi7NVPv+aVsTiqeI9oqsSdrIS8ZUcbdnRqayEnZ1YbCX5Yna6skNYFZygSUbWIE5XEmbx65VN9N0aH/2woZmq2OmqqTkgUjPpZklRKa+9C4dN3L9/AY0/sIiGstOVXeuH4MqSLcMg2UwgGVFUMn30jFLtrx8LefcQSFX4lEz/ZPq4Z0fiWykEV/FxQisQAAEQAAEQAAEQAAEQAAEQAAEQAAEQsIuAXgzitIuU3oUpkfU5LV5Sc0s0FKLq5/R89VzheqJ2IX1n/c8YBFfp2wfcGQRyjcDrr79OckiaMnUqVVQMz7Ul2roeCK5sxRtzcBEcSbhA/x73Kx8rqESAFDhYXBXMB5ytRGClHVyu9eMLGUOSElixQZXmbqUJsDjv4Yy4OgWOPXlps8fNSoUblPp0JJm/CMy27W6llZt8NG8JC69WN2vhF3fZ6HRVIKEUCz20/z4FdPqhxTSSna7EzcuOBMGVHVQxZtYRSFYAlWy/RAE5dZ9E54X2qRFwSsyU7H2S7ZcaldzrDcFV7u0pVgQCIAACIAACIAACIAACIAACIAACIJB9BNSHjjJzJ12u5H6JhuaTPpLSKWIKzCD+f1e8vCv+xha0HHB6qTYKxFYWwLRoCPUzJu5WIrpCAgEQAIFUCdx99920ZMkSbZiXZs5KdTjX9Vfvy3C4yp6tDxNcydRZtKQXXHkkDGD2LEkTjYnIrHpnC83/oYkWrmqmFSy+Wr/Vz6EU7XG6KmS3r7ZFHtqPHa5OG1tMw/tCcJVFLxlMNZsJpCpsSrV/NrPD3DOLQCpiqVT6ZhaFzJgNBFeZsQ+YBQiAAAiAAAiAAAiAAAiAAAiAAAiAgLsJ6B14nHa5StY1Su2YncIrmVvJdxxCcFaRul3CZzvnZzYZuFuZUUl/mfpgX2YyY8aM9E8IMwABEMh6AggnmNoWqvdlCK5S4+hkbzG0Elco7ZAbSwEn5WglQiu7QuNpN7LpH+V0tWVXC63e7KfPljbRtyy82r67RQu3aPVtO7T1UK9OXi2k4NEjC2lQd4QUtJoxxgOBqASsFE5ZOVbUSaPStQSsEkhZNY5rNyLCwiG4igAGxSAAAiAAAiAAAiAAAiAAAiAAAiAAAiDgMIE5s2fTnDmztbs6LRJKVXQlk5Y5a+czm7RzKv/IfMpnFlEJn1NJTnOE2CqV3bK3r17UiLCC9rLG6CDgBgL6cIJwM0xuxyG4So4betlDQERX4nS1lUVXXy5vpgWrmjjMoJ+drnzUxE5XPg69mGrycszFogLSxFYVfbyas5WEE+zewaZ4gjxhhBRMddfQP6cJOCGWcuIeOb1JOb44J0RQTtwjx7cp6vIguIqKB5UgAAIgAAIgAAIgAAIgAAIgAAIgAAIg4CgB9eGj3NTp0IJWiK4ULHHpahjuo/oRfq2oga8jJbmvJHGyKl7kTVlkpe7jtNhK7qtCCUoe4aWEQmYl9fOFsIKZtS+YDQhkIwHlbiVzx/t9cjuo3pPhcJUcP/SynoCETGxsbqXNO1poTbXO6aqWna7qWI2VYipr46Gu7T20b+8CGj2kkAZ2z6eO7TxUUmhfEEYIrlLcNHR3B4F0i6LSfX937LLzq8wEoVMmzMF58s7eEYIrZ3njbiAAAiAAAiAAAiAAAiAAAiAAAiAAAiAQjYDehcfp0IIyLytFV5HWKetK1bkq0tiqPB1iK7hbKfqZe9b/fE2YMIHkQAIBEACBRAnA3SpRYubtIbgy54LS9BJQTlfbWWT12bJm+uqHJg4z6GOnKz81s8uVL/J3CCJOXJytCr1EPTvm06AeBTSyn5cOHFhIPdjZyu4QjBBcRdwWVICAOQGIn8y5oDR7CEBk5exeQXDlLG/cDQRAAARAAARAAARAAARAAARAAARAAARiEVAfQEo7p12u5J5OiK7kPnaldIit9MwqKipoytTb7Foexk2RgP7na8aMGSmOhu4gAAJuJAB3K2t2Xb0fw+HKGp4YxToCyulq4/YWWstOV18sb6IFK5tJRFjJOF21Z2erzu3zaVhvLx08qJD6d8unLmUealNkn7OVogHBlSKBMwgkQQDiqySgoYvjBCCwchx5yA0huArBgQsQAAEQAAEQAAEQAAEQAAEQAAEQAAEQyAgCZ515RnAe6RJdlc8sst2JKrhIizLpYCVT7zGlTZDVlKlTqaJiuEUrwjBWE4DLldVEMR4IuIsA3K2s228IrqxjiZGsJ8DRBcnPblY761vp0yVNmuhqTbWP1m3xk58VWfE4XXk5anYBu1v17BRwttqPna1EcCVhBe12tlJEILhSJHAGAQsJQIhlIUwMFTcBCKviRuVoQwiuHMWNm4EACIAACIAACIAACIAACIAACIAACIBAXAT0opB0hBZUk9SHyVNlmXjOFEYTJ06iiZMmZSIizElHQH3IL0UILagDgywIgEBUAhBbRcWTcKV6L4bDVcLo0MEhAuJ01eRrpapt7HTFQqv57HT1NTtd7WSnqx11LTFn0b4tO1uVemhYnwI6SJytuuZrYisnnK3U5CC4UiRwBgEbCUCAZSNcFw8NgVV2bD4EV9mxT5glCIAACIAACIAACIAACIAACIAACICA+wjMmT2b5syZrS08HWHyFPFMF11lChuIrdQrJjvOehe5G2+8kYYOHZodE8csQQAE0kJAL7aSCbw0c1Za5pFLN4XgKpd2M3fXopyuahtb6ZPvm+gzdrtau8WnCbDE6cpvorvK9wScrXp1zqeB3QvogP4FdMjgAk185ZSzldoRCK4UCZxBIA0EIMRKA/QsvCWEVVm4abopQ3Clg4EsCIAACIAACIAACIAACIAACIAACIAACGQYAfVhpEwrncIiuX+mCa/E1armzEZq4HM6kpEHPnxPxy4kf0+9i5yMAtFV8izREwRyncCSJUvo7rvvDi4ToWODKFLKqN9x4HCVEkZ0doCAOF0186+b67f6aU21n75e0URf/cBOV+xyJYeIsvSpAztbdWJnq4q+BTRqQAHt09VL3cs9VFyYR3n6hg7kIbhyADJuAQIgAAIg4F4CEFy5d++xchAAARAAARAAARAAARAAARAAARAAgewgoD6QlNlW3V6XNoGRomUUGqlyp87pFlrJOosX5VPPKW2CS8aH70EUWZXRu8jJxCG6yqrtw2RBwBECRrEV3Aytw65+v4HgyjqmGMleAj52s2poaqX/LW6iuXys3+qjKhZhSbm4XeV78vgg6s3OVgO6eVlsVUhjhhZQOQuwnHa2UiQguFIkcAYBEAABEAABGwhAcGUDVAwJAiAAAiAAAiAAAiAAAiAAAiAAAiAAAhYSMDrxZILoSpbntPAqE4RWalsHnF6qsoQP34MosjJjFF1NmDCB5EACARAAAWMYQbzfW/uagODKWp4YzX4C4nTlZ6erNVsCTlffrWmmhaubqWZ3C9XUtpA4W3UWZ6s+XhrZr4D6sbNVD3a2KmFnq3QlCK7SRR73BQEQAAEQcAUBCK5csc1YJAiAAAiAAAiAAAiAAAiAAAiAAAiAQJYTMIpCMkV0JVhFeCWpeJGXStj5yaokAitJ6QwbaLaWHuxspdaJD9/NCGVfmfHnC6Kr7NtDzBgErCYAsZXVRMPHg+AqnAlKsoOAOFr5+NfUT5c20Wd8SKjBDdtaNHFVv675dCCHETx4UCGVFnMIwfRprTSYEFxlx2sKswQBEAABEMhSAhBcZenGYdogAAIgAAIgAAIgAAIgAAIgAAIgAAKuI2AUhWSS6Eq/GXoBlpQrcZK+jTGvxFVSLgIrSQ17BFfaRYb8A7FVhmyEDdMw/nwNHTqU5IDblQ2wMSQIZDABCSEoYis5q4SwsYqEtWcIrqzlidGcIyBOV3JIOMH12/y0s76VdtW1UmlJHrtc5VH3DvnUvaOHCr0suHJuWqZ3guDKFAsKQQAEQAAEQMAaAhBcWcMRo4AACIAACIAACIAACIAACIAACIAACICAEwSMopBMFV1FYlGsc8DKREFVpHlLOcRW0ejkRp2E75SfscrKyuCCRHClxFfBQmRAAARyjoCZ0KqiooImTppEFRXDc269mbAgCK4yYRcwBysIaKEG2fXKw+qqfI8VI1o3BgRX1rHESCAAAiAAAiAQRgCCqzAkKAABEAABEAABEAABEAABEAABEAABEACBjCaQ7aKrjIYbYXIQW0UAk6PFxp8xWaYSXalzji4dywIBVxFQLlZGRyuBgJCx9r8UILiynzHu4AyBVna74v80N6t0hxA0rhiCKyMRXIMACIAACICAhQQguLIQJoYCARAAARAAARAAARAAARAAARAAARAAAYcIGAUhNWc0cii+Jofu7p7biCNXzyltggvGB/BBFK7IGH/O9IvWhxoUERYSCIBA5hNQAivjWT9zvM/radibh+DKXr4YHQSEAARXeB2AAAiAAAiAgI0EILiyES6GBgEQAAEQAAEQAAEQAAEQAAEQAAEQAAEbCRjFIBBdWQsbYitreWbzaMaftWxeC+YOAiAQTkBCB0pC+MBwNnaWQHBlJ12MDQIBAhBc4ZUAAiAAAiAAAjYSgODKRrgYGgRAAARAAARAAARAAARAAARAAARAAARsJmAUgkB0ZQ3w8pmFVD6rKDgYHE+CKFyfkZ+5yspFfFS6ngUAgEA2ExCRlQisJFVUDM/mpWTt3CG4ytqtw8SziAAEV1m0WZgqCIAACIBA9hGA4Cr79gwzBgEQAAEQAAEQAAEQAAEQAAEQAAEQAAE9ARF/3D5tWrAIoqsgiqQyEFslhc3VneRnEAkEQCDzCUBYlVl7BMFVZu0HZpObBCC4ys19xapAAARAAAQyhAAEVxmyEZgGCIAACIAACIAACIAACIAACIAACIAACKRIQH1wqYaB8EqRiO8sIQTLZxZRCZ9VmjJ1KpxPFAycQQAEQAAEQMBCAur3lhEjRtD06dMtHBlDgQAIKAIQXCkSOIMACIAACICADQQguLIBKoYEARAAARAAARAAARAAARAAARAAARAAgTQRQIjB5MAbXa1UqCm4oSTHE71AAARAAARAIBYBCK5iEUI9CKROAIKr1BliBBAAARAAARCISACCq4hoUAECIAACIAACIAACIAACIAACIAACIAACWUnAKLqqH+6nDbfXZeVanJh0jyltQlytJk6cRBMnTXLi1rgHCIAACIAACLiWAARXrt16LNxBAhBcOQgbtwIBEAABEHAfAQiu3LfnWDEIgAAIgAAIgAAIgAAIgAAIgAAIgEDuE6isXEQivKqsrAwuFiEGgyi0DFytQnngCgRAAARAAAScJADBlZO0cS+3EoDgyq07j3WDAAiAAAg4QgCCK0cw4yYgAAIgAAIgAAIgAAIgAAIgAAIgAAIgkBYCRrcrmYTbhVfFi/KpfGZRiKuVhBCcMvW2tOwRbgoCIAACIAACbiQAwZUbdx1rdpoABFdOE8f9QAAEQAAEXEUAgitXbTcWCwIgAAIgAAIgAAIgAAIgAAIgAAIg4EICAaerRWFuV4Ki5swm1xCJJLSS8IEVFcNdwwELBQEQAAEQAIFMIADBVSbsAuaQ6wQguMr1Hcb6QAAEQAAE0koAgqu04sfNQQAEQAAEQAAEQAAEQAAEQAAEQAAEQMAxApHcrmQCuSy8MhNayZonTpxEIrZCAgEQAAEQAAEQcJ4ABFfOM8cd3UcAgiv37TlWDAIgAAIg4CABCK4chI1bgQAIgAAIgAAIgAAIgAAIgAAIgAAIgEAGEDATXsm0ci3UYCShFcIHZsCLEFMAARAAARBwPQEIrlz/EgAABwhAcOUAZNwCBEAABEDAvQQguHLv3mPlIAACIAACIAACIAACIAACIAACIAAC7iYQTXhVP8JPDcP9WQdIRFaSymcWUcmevFqECK0QPlDRwBkEQAAEQAAE0ksAgqv08sfd3UEAgit37DNWCQIgAAIgkCYCEFylCTxuCwIgAAIgAAIgAAIgAAIgAAIgAAIgAAIZQiCS8Eqmlw2uV9FEVrIGCK2EAhIIgAAIgAAIZBYBCK4yaz8wm9wkAMFVbu4rVgUCIAACIJAhBCC4ypCNwDRAAARAAARAAARAAARAAARAAARAAARAIM0ERHhVWbmIj8qwmdSz21XDcJ9WXnNmU1i90wUisir5Lp+KF3nDnKxkLhBZOb0juB8IgAAIgAAIJEYAgqvEeKE1CCRDAIKrZKihDwiAAAiAAAjESQCCqzhBoRkIgAAIgAAIgAAIgAAIgIBtBLQP9xeFf7hv2w0xcMYQkNBeSCAAAplJIJr4Ss1Y3K8k2R1+UO9gJfczhgqUMkkispKEsIEaBvwDAiAAAiAAAhlNAIKrjN4eTC5HCEBwlSMbiWWAAAiAAAhkJgEIrjJzXzArEAABEAABEAABEAABEHATgbPOPMNNy8VadQREHDFl6m26EmRBAAQykUA84is1b3HCkqTcsILlIwLl6tp4FrcqlcS1SlIkYZVqJ2eIrPQ0kAcBEAABEACB7CEAwVX27BVmmr0EILjK3r3DzEEABEAABLKAAARXWbBJmCIIgAAIgAAIgAAIgAAI5DABcbe6fdq0HF4hlhaNAARX0eigDgQyk4ByJYwUetDuWcv7RkXFcKoYHjjbfT+MDwIgAAIgAAIgYA8BCK7s4YpRQUBPAIIrPQ3kQQAEQAAEQMBiAhBcWQwUw4EACIAACIAACIAACIAACCREQC+4+u1vb6bBgwcl1B+Ns5PAPffcQ4sXL9acaeBwlZ17iFmDgCIg7+OSKveEhg1eVyYfKla5Vsm4Slyl8nJGAgEQAAEQAAEQyH4CEFxl/x5iBZlPAIKrzN8jzBAEQAAEQCCLCUBwlcWbh6mDAAiAAAjYTkBCp8yZM9v2++AGmUdg4sRJNHHSpMybGGYEAjlIwCi4GjRoYA6uEksyErj33nshuDJCwTUI5DABJcKKtkQRViGBAAiAAAiAAAi4hwAEV+7Za6w0fQQguEofe9wZBEAABEDABQQguHLBJmOJIAACIAACSRNQD36SHgAds5rAlKlTNUeFrF4EJg8CWUAAgqss2CQbpgjBlQ1QMSQIgAAIgAAIgAAIgAAIZBEB9dxtxIgRNH369CyaOaYKAtlDAIKr7NkrzBQEQAAEQCALCUBwlYWbhimDAAiAAAg4RkA9+JEbnnrqqY7dFzdKL4FXX31VmwAEV+ndB9zdPQQguHLPXutXCsGVngbyIAACIAACIAACIAACIOA+Auq5GwRX7tt7rNg5AhBcOccadwIBEAABEHAhAQiuXLjpWDIIgAAIgEDcBNSDn2HDhtENN9wQdz80zF4Cy5cvp7vuuktbAARX2buPmHl2EYDgKrv2y6rZQnBlFUmMAwIgAAIgAAIgAAIgAALZSUA9d4PgKjv3D7PODgIQXGXHPmGWIAACIAACWUoAgqss3ThMGwRAAARAwBEC6sEPBFeO4M6Im0BwlRHbgEm4jAAEVy7b8D3LheDKnfuOVYMACIAACIAACIAACICAIqCeu0FwpYjgDALWE4DgynqmGBEEQAAEQAAEggQguAqiQAYEQAAEQAAEwgioBz8QXIWhydkCCK5ydmuxsAwmAMFVBm+OjVOD4MpGuBgaBEAABEAABEAABEAABLKAgHruBsFVFmwWppi1BCC4ytqtw8RBAARAAASygQAEV9mwS5gjCIAACIBAugioBz8QXKVrB5y/LwRXzjPHHUEAgit3vgYguHLnvmPVIAACIAACIAACIAACIKAIqOduEFwpIjiDgPUEILiynilGBAEQAAEQAIEgAQiugiiQAQEQAAEQAIEwAurBDwRXYWhytgCCq5zdWiwsgwlAcJXBm2Pj1CC4shEuhgYBEAABEAABEAABEACBLCCgnrtBcJUFm4UpZi0BCK6yduswcRAAARAAgWwgAMFVNuwS5ggCIAACIJAuAurBDwRX6doB5+8LwZXzzHFHEIDgyp2vAQiu3LnvWDUIgAAIgAAIgAAIgAAIKALquRsEV4oIziBgPQEIrqxnihFBAARAAARAIEgAgqsgCmRAAARAAARAIIyAevADwVUYmpwtgOAqZ7cWC8tgAhBcZfDm2Dg1CK5shIuhQQAEQAAEQAAEQAAEQCALCKjnbhBcZcFmYYpZSwCCq6zdOkwcBEAABEAgGwhAcJUNu4Q5ggAIgAAIpIuAevADwVW6dsD5+0Jw5Txz3BEEILhy52sAgit37jtWDQIgAAIgAAIgAAIgAAKKgHru5mbBVU1NDX3zzTe0e/du6tGjBx144IGUn5+vEOEMAikTgOAqZYQYAARAAARAAAQiE4DgKjIb1IAACIAACICAevADwZV7XgsQXLlnr7HSzCEAwVXm7IWTM4HgyknauBcIgAAIgAAIgAAIgAAIZB4B9dwtEwVXb731Fvn9/pjQioqKqKysjNq3b6+dO3bsSCUlJTH7SYPVq1fTo48+Ss3NzcH2ffr0oV/84hcZIbpqaWmhlStX0qJFi2j9+vW0a9cuqquro+LiYiotLaXu3btTRUUFDR48mLxeb3ANyGQWAQiuMms/MBsQAAEQAIEcIwDBVY5tKJYDAiAAAiBgKQH14AeCK0uxZvRgEFxl9PZgcjlKAIKrHN3YGMuC4CoGIFSDAAiAAAiAAAiAAAiAQI4TUM/dMlFwdeutt1JTU1PCOyDuVCNHjqTDDjuM+vXrF7X/448/TkuWLAlrc84559ABBxwQVu5kgbhuiehs+/btMW/bpk0bGjduHI0dO5Y8Hk/M9mjgLAEIrpzljbuBAAiAAAi4jAAEVy7bcCwXBEAABEAgIQLqwQ8EVwlhy+rGEFxl9fZh8llKwErBVWtrK8m3cOUsR15ennZkYkgG/Vxl69RcM/EBtWIqZ/08JZ9sguAqWXLoBwIgAAIgAAIgAAIgAAK5QUA9d8slwZV+Z/r27UsXXXQRtWvXTl8czN9///2ac1SwYE9mwoQJdPjhhxuLHblubGykZ599lpYuXZrw/Xr16kUXX3yx5vSVcGd0sI0ABFe2ocXAIAACIAACIEAEwRVeBSAAAiAAAiAQmYB68APBVWRGuVYDwVWu7SjWkw0ErBBcSaiHZcuW0epVK6mmeh15Wmqp1bebfK1FVFDSnko79tG+XTxkyJC0hzqQbwjLw+uqdauobudmym/ZrW1Ti6cN5ReVU/de/bS5xvo2tN17K0wlxMWqlSuoesNqIv9uamnmgwqokJkWtO1K/fsP4KN/Ug/UIbiyewcxPgiAAAiAAAiAAAiAAAhkNgH13C1XBVdCX0IEXnHFFVRQUBC2Ge+88w59+OGHIeXyZaHrrruOunXrFlLuxIWEC5wxYwatW7cu6duVl5fTZZddRp06dUp6DHS0lgAEV9byxGggAAIgAAIgEEIAgqsQHLgAARAAARAAgRAC6sEPBFchWHL6AoKrnN5eLC5DCaQquJIQDLNeep66FG6iccN2U5dSP3nYeEkiGbAhE7W0Em3akUdfrCmlH3b1pElnnE1Dhw5NC41Zs2bRd19/Qof230EH9W2gkoJWyt8TccHP85T5LlhbSJ+sakd99z2CTj75ZJLwDE6nLVu2kIS38NSvpvHDdlHfTs1UwPMUrjxFauV/qnfl0Tfr2tLibd3px8f+VAuZkcg8IbhKhBbaggAIgAAIgAAIgAAIgEDuEVDP3XJZcCW7JiEGzz///EcsSzYAAEAASURBVLAN9Pl89MYbb9CXX35Jzc3N1LFjRzrxxBNJeKQjzZkzhz7//POItxbRmAjB5O/FhoaGiO0GDhxIl19+ecR6VDhLAIIrZ3njbiAAAiAAAi4jAMGVyzYcywUBEAABEEiIgHrwA8FVQtiyujEEV1m9fZh8lhJIVnBVU1NDH3/4Lm1e+j6dsF8t9e4gUqDoafUWD71b2Ya67TuejjzqJyTfvrU7SRi+xYsX04dvvUSDS1fSMcOaqdAb/a71TURfr/HS/I296PBxZ/ED95Gm34iOPkritbt372am79Oqb/9FJ4zcTf07x2ZaU+uhdxYVk6fLYXTksSdQjx494roxBFdxYUIjEAABEAABEAABEAABEMhZAuq5W7YIroqLi+nWW28N2Q9xhdq6dStt2rSJ3nvvPaqtrQ2pVxe33XZbxC/TSLh5EVwVFhaq5o6fq6qqSEIcylyMqaKigo477jjq2bMnf7HJo7WR9f73v/+lL774wthcu77gggvSJhwznZCLCyG4cvHmY+kgAAIgAAL2E4Dgyn7GuAMIgAAIgED2ElAPfqwQXK1du5aqq6u1b4EJkS5dumhhmDp06JBRgOTDdpmrfFtNHhK1bdtW+/aahLYqKirKqLnKwyB5wCNzFUFD586dqXfv3tS1a1fKy2MbliQSBFdJQEMXEEiRQDKCqx07dtDzT/6DDu3+PQ3t3qw5RcU7jSYf0VerC+mzzfvSxLMvol69esXbNal2n346j77/5HkaP3QrdWvforlExTvQtto8+tfCUmo34Gd00kkn2RoOsbGxkZ58/CEaUvwNje3fQG0Kwx+0R5q3j3VZyzfm0yuV/ejsCy7T/v8Wqa0qh+BKkcAZBEAABEAABEAABEAABNxJQD13yxbBVUlJCU2bNi3iZsnfqQ8//LAmwDI2uuSSS9LmtGyci9n1I488Qj/88ENY1SGHHEKnn366JrQKq+SCf//73/T++++HVUlIwV//+tckIRKR0ksAgqv08sfdQQAEQAAEcpwABFc5vsFYHgiAAAiAQEoE1IOfZARXYgu+bt06WlK5gJYunEv5TZupc2krdWrXQs3+PNqyy8NHHrXrPJD23f8wGjZ8f5KHEelIO3fupFUrV9CiBfOoasXX1LGkkefaQu2KWkmcS3bU51FNQ1sq7zGURh96DA0eMjQt37oTUdWGDRtoyfeLaPl3n1Dj9lXUtayFSkskJFcrbdmZr821pbgnDWOmQ4btR3369EkIKQRXCeFCYxCwhECigisRhv7tvrvpZwOX0wF9WT2VZFpZ7aEPNoyhCydfZYt7lHwz+Ntvv6W3XrqPrjmuntoVxy9gMi5p1vwy2mfMJTR6zFhjlSXXTU1N9NSTT9AAz1w6rqI56TE378yjl5dU0NkXXUuxBMUQXCWNGR1BAARAAARAAARAAARAICcIqOduuSK4kk1566236OOPPw7bn3HjxmkuUfqK7du3k4i0jEmcpCR8X6y0bds2Wr16Na1fv57at29P8mVJ+UKREjmJ85b8/axP8neatNWnRYsW0dNPP60v0vLypcZrr702rNxY8Mwzz5B81mhMJ5xwAv34xz8OKd68eTPV19eHlIlrVrTnd7JGY5IviMoXL+NJ4o4tX9qUQ5zExJVZGMuXYeXe0ZLMVeZsTH379tW+7CnPKuX579KlSzXWEk5xwIABwS/c6vu1adNGu6e+TJ+XvZI9M6Z27dql9MwYgisjUVyDAAiAAAiAgIUEILiyECaGAgEQAAEQyDkC6sFPooIreWDyr3/9i/yb59KBPXdQv04tVMaiIGPy+Ym213loeXUBzd/Ul/pXHEo/+clP4nqoYhwrmWsRAyxcuJA+fu9NGt7+exrYpZl6ckgus1BXdU15tL7GQ0uqy+iHhn3pZz87gQYPHpzMbZPqI25bb7/9NtVXzaXhnas1ph3atlC+wcjKzy4rwnT9dg7FtaEbte97KP30pz8l+RZiPAmCq3gooQ0IWEsgEcGVvG+99+47VLLhBTpiCMfdSyHxc1H6eGkR7ep0Gp18yikpjGTeVR6UznnuH3TKkO+pG4tDU0nb6/Lopa970E/PvE57iJ7KWGZ9586dSzXfPkE/HVlL3ujPm826B8vk/3RfruT3352jafJlV0V15ILgKogNGRAAARAAARAAARAAARBwJQH13C2XBFeRxEuHH344TZgwIWSfI7lDiTOUuLdHSgsWLKB//vOfYWIqae/1emns2LGaQ7KIqCorK0OGkeeOcujTAw88oAm39GWSnzRpEonDVawkz9IeffTRsGYiipoyZUqIC/0TTzxB33//fUhbCdV4++23h5TpL2666Sb9pZY/4IAD6JxzzgkrVwUihBLnrU8++YQk7KNZElZjxowhEYZJ3iwJv6eeeiqsSpzOdu3apa1bvsiq0pAhQzT28veuMckXbX/zm98Yi4PXr7zyCn366afBa5WR1428fpJNEFwlSw79QAAEQAAEQCAOAhBcxQEJTUAABEAABFxLQD34iVdwJUIACW/35CN/ocN6r6XDBifmEvLCZyXk73wknXHmubaH7xMHrndZtLD8y9fogkN3U3mbcEFYpI3fujuPHv1fZzr+1J/T/vsfEPPbYJHGibdcvrH3/NOP0dA2C2nciMSYvrmggFY1jqRLrrg2LtEVBFfx7gragYB1BBIRXImg9bkHfktXHFmTkjBIP/vfzm5Ll1x1s+Ui0jdff4U617xMYwcl78Kln+fqrR56sXJ/uuGGGywV5sqD6Ocfnkan7buE2rKzoRXpgQ9LaNRxl9OPfvSjiMNBcBURDSpAAARAAARAAARAAARAwBUE1HO3XBJc/e9//9PEUMYNPO+882i//fYLKU5UcCXPHaXPBx98EDKO2cWoUaM0R6eNGzeGVBsFV+J2LKIo+btQn+SLi7feemvcf3vec889pk5Q8vdrt27dgkM7IbgSR6sXXnjBVEQWnIguI45XIt7Sz1NVRxJcSYjI2bNnk15sJX3kGfLFF19Mf/3rXzWXfjWOOkcT0915551hjmfiwHXLLbdQaWmpGiLhMwRXCSNDBxAAARAAARCInwAEV/GzQksQAAEQAAH3EVAPfuIVXMk3tN595WGaULGJ+ncJfVARD71mdrz6fIWXvtpxMJ1/4cUxwzHFM6ZZG3mY8s7b/6Lmla/S8fs1JfUB+7baPHptQRkNPfRC7QP1WBbcZvOIp0weDM1+7iH6ce/lNKK3jzwGR6tYY7SwdmDBGi/9Z8NwOvv8yVG/IShjQXAViyjqQcB6AokIrma99BwNbnmbDuxnjYhJVrNxh4f+u+N4Ou2M84KhF1JdZUNDAz3219/SxWM2auFZUx1P9X/hi/Y04pirwx7Uq/pkzu+99x75VzxN41nQmuBbbMTbbd7poTnLR9FFl1xFEjbBLEFwZUYFZSAAAiAAAiAAAiAAAiDgHgLquVsuCa7MBEWyoyJqktBw+pSo4CqSA5J+zFh5o+Bq2bJl9Nhjj4V123fffennP/95WHmkAnHcErGZMZ166qkhX8Qx42Olw9XatWu19cjf5IkkCeEo4RONoqtIgqu8vDwSAZwxDR8+nC688EItrKSElzQmszCL0kbCQt5///3G5toXwy699NKw8kQKILhKhBbaggAIgAAIgECCBCC4ShAYmoMACIAACLiKgHrwE4/gShyjXnjiPprQ/0vqkIBblBGo/K3+6YoC2lh6Op3CIa7kD3irk4QR/Pb9f9DZh2xPWMCkn4uERHz4k5504lm/pH322UdfZVl+xiMP0GGd/0fDuvPNkkzCdPmmfPp4y+F0wUWXRHUPg+AqScjoBgIpEIhXcLV79266c+qN9Luf1VBpcfiDzRSmQDMXDaHjz/w1tW/fPpVhgn0lDMDGLx+gkw9sskzEJIOvr8mnb/POpp+ecFLwXqlmbr3ld3T1YSupe/vEhcLR7v3C/C50+Ck3Ud++fU2bQXBligWFIAACIAACIAACIAACIOAaAuq5Wy4Irpqbm2nOnDn09ddfh+2f/E109dVXh5UnIrjavHkzyd9QZiIfGbiwsFBzdt+xY0fYffQFRsGVhNx77bXX9E20/OjRo2nixIlh5ZEKPv74YzITGB1xxBFaiD3Vz07BlbD5+9//TuvWrVO3CznLl4HkC6PybMEsKXcqfV0kwZW+jT4/cuRIOv/880lctu66666w/RowYABdccUV+i5aXsIfyuvBmM444ww6+OCDjcUJXUNwlRAuNAYBEAABEACBxAhAcJUYL7QGARAAARBwFwH14CcewdUHH7xHvmVPayHvUtVI+fgz75e/6UQ/Ovk3/EF1P0uhy4OX52fcQ6fsu5x6dkj9w/VVHOLq/zaNpnMvvoa8Xq+lc/3mm29oxX/vp5MPqKN8T+pDv/p1CfUafQWNHj024mAQXEVEgwoQsI1AvIIreU/46t2/0s8PT+ybqvFM/J/flNCBJ02z7D337rvvpvOHf2W5iGlnQx69uWJ/Onvyby0R5K5YsYKeffgOmnpyXTyYEmozf00htQ66mg4+ZIxpPwiuTLGgEARAAARAAARAAARAAARcQ0A9d8sWwVUyGyOh+cQ5qVOnTmHdExFcvfjii6Zirg4dOtCZZ56pfREyPz9fExOJ01SksINGwVUkoc+xxx5L48ePD5tzpIKvvvqKXnrppbDqgw46SJufqrBTcBVpDuJaddZZZ1HPnj21aWzatIneeecdEjGVMV1++eU0cODAYHEswZV8UVa+uCX76/f7tS8cnXjiiVr/Bx98kFatWhUcSzIi+BK3M6MTtAjFxJ1Ln+Q5q7QVB7BUEgRXqdBDXxAAARAAARCIQQCCqxiAUA0CIAACIOBqAurBTyzB1erVq+mxv0+nW07YRSWF1riurNrioVmLK+im3/7e0j2Qb9t12fkaHTGkOSV3KzUpCdk3+8s21POgn5N8a82qtG3bNvr7X+6ka3+8nspTcAzTz2cDhw176ovedMPNd2rf/NPXqTwEV4oEziDgHIF4BVfyQLRg3XN07LBmyyf3GYdz9Qy9mg4Z/aOUx5awrVOnTqXpJ6ygfItNChub82j2wp507tX3WiK4mj9/Pi1nYeuZhzSmvG7jAMvYWfCHwtPo+BPNvxUNwZWRGK5BAARAAARAAARAAARAwF0E1HO3XBVc9enTh84991zq2LGj6cbGK7iqrq6me+65J8wtqW3btnTNNdeYji/P/z7//POw+xoFV6+//jrNnTs3rJ0xFGBYA0OBPE979NFHDaVERtcouwRX4jD25z//mYwOXyKGEsFbaWlpyNyk/UMPPRTmhmV0oIomuDrggANIxFVlZWUhY6uLefPm0auvvqoug+ezzz6bDjzwwOD1rl27aPr06WH7q8ITBhsmmYHgKklw6AYCIAACIAAC8RCA4CoeSmgDAiAAAiDgVgLqwU80wZXYVb/xxhvUa9dMOmgfn6WoHvqohMafM4Xkj30rklhm/+2vd9H1Ry6jYgvNqDbt9NCDnw5mgcFt2je1rJirfBtvzeeP0tmjrXWyeW5eMfUdfQkdeeSRptOE4MoUCwpBwFYC8Qqu5IHxAZ5XaFC35EOMRlrIwnVeqiqdRON/dkqkJnGXb926lZ588E769THr4+4Tb0M/GxPO/KoDnX3tw5YIruRbz8Vrn6TDBlsvYlvNDohfNYyjU8+42HR5EFyZYkEhCIAACIAACIAACIAACLiGgHrulouCq6KiIjrttNNIQsxFcoSPV3Alwin5e9iYorlQrV+/nu6//35jFzIKrl5++WX67LPPwtqJI9SoUaPCyiMVSBi/v/3tb2HVgwYNossuuyxYbpfgKpLgy7je4EQ4YyaIktCMIn5SKZrgSgRe0VJtbS3dcccd1NISGmFAhFrnnHNOsGuk/T3vvPNov/32C7ZLNgPBVbLk0A8EQAAEQAAE4iAAwVUckNAEBEAABEDAtQTUg59ogqu6ujp67skH6MLhX1JRgbWovljppTUlp9Hpp59uycCLFy+mVXPvo59WbLdkPP0gf/mgE026+GaSb++lmnw+H8166Tn6add3qbxt6EOJVMdeW+Ohf637sfawR2y8jQmCKyMRXIOA/QTiFVw999xzNLrkLRrS3XrB1bcsuFrL77cnTEj9/XbLli30xAN/oJuOs15w5eOlPz2vLf38psctEVxJ+IgOG5+kg/tbKxiWV83abR76dOdRNOncy01fRBBcmWJBIQiAAAiAAAiAAAiAAAi4hoB67paLgiu1iT169KALL7zQ1IUqXsHV22+/TR999JEaMni++eabqby8PHitz0jYPPmby5iMAiRxYBLhkTGdccYZdPDBBxuLI16vWbOG/vGPf4TVDx48mC699NJguV2CKzPxlNz0xhtvpC5dugTvr89ICD8J5WdMt956a9C1KpLgSlz+TzrpJGPXsGuz9UqYSXHFVs8ln376aVq0aFFIXxHsSTjBgoLUHzZDcBWCFhcgAAIgAAIgYC0BCK6s5YnRQAAEQAAEcouAevATTXAlH6x/+PLddMaIVbYs/uEFh9Ell15pyR/Y4hrVceMDNMwGscKH37ehtiOuoDFjxqTMob6+nl574T6aNOQb8oZrolIe/4FPh9L5l/46+PBEPyAEV3oa/8/efcBHcZ1rA38kIZCERJNoEsUgWsA00zsIMNV0ML0ZG+PELbbv/ZKbXNtxnDhOc4w7tjFgem8Gg+mmF1Fs6qV3JECAkBCS0Lfv5I7uarWrbbO7M7PP/H75pJ2dOXPmf7jjT2effQ9/p4B/BFwNXEk1wQpp89CqpvbhIAm4Ztd4Dm3bdfL6puUZ9t67b+HtXue8bsu2gYeWW19wsCJGvfQvTQJXsnRE6sHPMaDpQ9tLef36pGVJwbMRUjVsoN22GLiyy8KdFKAABShAAQpQgAIUCBoBdd7NSIErCcvYbhKqCgsLU5ank78HbbcyZcrg17/+NSIiIgq85Wrgavbs2Th06FCBc6WtP/zhDwX2Wb9wNXC1du1abNy40fpU5fdevXqhS5cuhfY72iGfNc6cObPQ240bN1aWVVTfsBdAcnYv//Ef/6Genv/TtkrU8uXLIfOetptU6QoJCbHdrbzOysqC9Nt2mzJlCmrUqKHsdhS4cjWQduDAAcybN8/2Epg8eTISExMhXzp966238PBhwb/Jpd9SZUyLjYErLRTZBgUoQAEKUMCBAANXDmC4mwIUoAAFKGARUCd+igpcXbt2DbuXv4v+j9/widn0Q40xdOxLiI6O9rr9NatX4PFHc1G1nLZVo6RjR6+G41q5iUhKSvK6n7L04Xez38Xg+mcQ5oPA1fT9j6HP8NdQoUKFQn1l4KoQCXdQwOcCrgauduzYgUt7PsOwFlma92nj0XBUavcb1K/fwOu2c3NzlWUDXm53HGWj8rxuz7qB9KwQLDtRF6Oef8vhpLH18c5+l78HNy/9G36VVPhDAWfnOnv/0MViSE94Fu06dLZ7KANXdlm4kwIUoAAFKEABClCAAkEjoM67GSVwJWGrt99+2+H4SGjmiy++gFR7st06duyIvn37FtjtauBKluqTJfust4oVK+K1116z3lXgd1cDV1u3bsWqVasKnCsv2rVrh/79+xfa72iH/L2+bNmyQm+3bt1aWVpRfcNXgatp06bh1KlT6mW8+mkdpnIUuJIlASX05WyTUJcE47Kzswscqv57OHHiBL766qsC78mLiRMnol69eoX2e7KDgStP1HgOBShAAQpQwEUBBq5chOJhFKAABSgQlALqxE9RgSspP/3z2j+gZ/27PjH69khd9B3+a8i34bzdViyZjRYRq1C5tPaBqzOpoThWbDT69OnjbTdx9+5dbJr333iq/lWE2v8SmlfXWPxTAto89Tri4+MLtcPAVSES7qCAzwVcDVylpaXhg3dfxX/1y0C4xmHMpUcqov3g3ztcasBdhAULFiA+cwXa1yo4qepuO7bHp6aHYuPN7hg2cqLtWx69loDrm7/7D7w36CbCwzxqwuFJq34qg190+w/lW7v2DmLgyp4K91GAAhSgAAUoQAEKUCB4BNR5N7MErmTkpMLVH//4x0IBG5mDeuWVVwoMrquBqz/96U+Qv4ett8ceewwvvPCC9a4Cv7sauEpOTsbcuXMLnCsvGjZsiDFjxhTa72iHo0pZtksY+ipw9de//hUpKSmOuufW/gEDBqBt27bKOd4GrqQRexXK4uLiIJW77C3pWLJkSfz+97/PX3LQrc7bOZiBKzso3EUBClCAAhTQSoCBK60k2Q4FKEABCphRQJ34KSpwdeXKFexZ/kcMaHTTJwRfJjfC0+NeQkxMjNftr16xFI1CFvikwtWRS+G4WnYsnnzySa/7ee/ePUuFq3cwpMF5n1S4+mpvdUuFq1+jUqVKhfrKwFUhEu6ggM8FXA1cSUc+mfp3dI3fj7oaLo36IDsE80+3x/DRz6JEiRKa3O/ly5ex9Kv/xpQu6Zo+xzYcj0T041M0Wb5VvdEZM75Bg2Lr0fwx7ZZqzMkFph14HKMmvmx3+Va5NgNX6gjwJwUoQAEKUIACFKAABYJTQJ13M1PgSkbyo48+KlTlKjQ0VKl0VLx48fzBdjVw9fHHH+P8+fP558kvUrX99ddfL7DP+oWrgatbt27hvffesz5V+d1ZBS3bE2Q5Qfm80XZ75plnULdu3fzdvgpcTZ06FfKlWC02rQNXP//8M2bMmFGoa2+88QakMpdtmK5NmzYYOHBgoeM93cHAladyPI8CFKAABSjgggADVy4g8RAKUIACFAhaAXXip6jAVWpqKjYv/guGPF64XLgWcJ8eaIkJz/4SERERXje3ccMGVLnzFepUtHwSrvG29WQkwutNhkwKeLtlZGRgycy/YniDoyimcRUb6dvHu+pizLOv2w0BMHDl7ejxfAq4L+BO4Oqnn47gf7Z8gP6N0y1L6rl/LXtnzN0dgaotn0X79u3tve3xvq8++wc6VtiH2ho9czMsywlO3V4dz0x5A/JtWK02mdxdOf03ePqJWyheTJtWNx4rhpulB2Dw4MEOv5XLwJU21myFAhSgAAUoQAEKUIACRhVQ593MFriaP38+9u/fX2hYpkyZgho1auTvdzVwNW/ePBw4cCD/PPlFviz0zjvvFNhn/cLVwJWc8+677+LOnTvWpyu//+pXv0K1atUK7bfdIZWTpY3c3ILzjSGWP9plCUbrOU1fBa4WL16M3bt323ZNWXYxPDy80P6idkRFReX3WYsKVzk5OcpYSfUz6+2JJ54oNK7yvlQukwpmWm0MXGklyXYoQAEKUIACdgQYuLKDwl0UoAAFKECB/xVQJ36KClxJNaa50/+FSU8c0Xz5uyu3Q/F9ypOYMGGCJmMiZcJvHZiKrvXua9KedSOfbCmNLgNfhVh5uz18+BALvv0MQxO3o4RGH/6rfbqdGYLph1vi5ZdfRlhY4fWzGLhSpfiTAv4TcCdwJROV82dPR7eKG1AxJs/rTp64FobZB2viD+/80eu2bBuQbyBvXvI3jG1xw+tw2CPLrX65JRINk7QJtlr3NS8vD4sWzEM9rELDKt5XuTqXGoaZB6rh1df/q8jqjAxcWY8Cf6cABShAAQpQgAIUoEDwCajzbmYLXH344Ye4dOlSoQGVJQVlaUF1czVwtX79esj/bLfXXnsNUonK3uZO4GrlypXYtm1boWaaNWuGp59+utB+2x0bLF/w/P777213K5WtpMKV9SaVnqTik/UmVb9kGUZHmyy9Z7s1adIEI0eOzN+9Y8cOLFu2LP+1+ou34SUtAlfSl0WLFmHPnj1qtxz+LFOmDH7zm99Y5hA0+oaZ5UoMXDnk5hsUoAAFKEAB7wUYuPLekC1QgAIUoIB5BdSJn6ICV8oH1QsXWD6oXo6GCQW/yeWtzMztJfB4t1cg33jSYrt27Rrmff13vNJJ22pcafdD8MG2GvjP3/43IiMjve6qmMpEUqnrM9G6ZrbX7Vk3sOxACZRpOBGdO3e23p3/OwNX+RT8hQJ+E3AncCWdOn36NFbM+QDDn0hF5TKPPO5nWkYIlv5cFX1Gvm5ZjsH+JLXHjVtOlG/3ysR1xLVF6Fo/G+GFM54uN7/zTHGczeuIkWOfdfkcdw6UyfhvvvgnRjY579Wys7mW4ZibXBHNuj9vCeDWL7ILDFwVycM3KUABClCAAhSgAAUoYHoBdd7NTIErWaLv/fffx6NHhf9WfeuttyDVk9TN1cCVfIFy7ty56mn5P6VKc79+/fJfW/9y7tw5fPLJJ9a7lN+7d+8O+Z/1JpWX/vKXv0AqzltvxYoVw3PPPVdktaWbN29CljyUKlfWmyyhKAGzSpUqWe/GihUr8OOPPxbYJy9sbdQDpG9vvvmm+jL/p23g6uzZs/j000/z31d/SUxMxOTJk9WXdn/KFz+l8rMs02i7aRW4kvnGL774wrb5Qq87deqEPn36FNrvzQ4GrrzR47kUoAAFKEABJwIMXDkB4tsUoAAFKBDUAurET1GBKwE6fPgw1i/6J17rUXBiwhu82xmh+MemeLzzp79700yhc+Vbdn2r7kTNCoUnfgod7OKOH46G4065gcrSUS6e4vSwGzdu4NN//A5vPlW4pLnTkx0ckPkQ+OPKknjjv/8B+caYvY2BK3sq3EcB3wq4G7iSUKZ8W3jxN+9hQpubKB3pfqWru5Zqd59uK4Pu/Z+1hFqb+ewGpSLXvLmzUCFjA7o/ng1PvqO673wxbL/eFM88+zyio6N91tcrV67g2y/+jPGtUlGhlPv/jciyZI4/3RiFeq2GokfPXk6/kcvAlc+Gkg1TgAIUoAAFKEABClDAEALqvJtZAlfyRcevvvrK7vJ8EuZ5/fXXC4yLq4ErCR39+c9/xoMHDwqcL0vlSQWpmjVrFtgvYa8//elPuHv3boH98sJe4Er2O6oQJdcYNWoU6tcv/IUa+Rvyyy+/LBS2kvZat26NQYMGya8FNkfXkeXoW7VqVeBYeTFz5kzI55i2m23gSv72/uc//4mUlBTbQ9GjRw8kJSXZ/RtVAmNSdUtWMJBqWOXLly9wvlaBK5nHkGUX7Y2J9QVtq6BZv+fp7wxceSrH8yhAAQpQgAIuCDBw5QISD6EABShAgaAVUCd+nAWu5I/6WTOno0nkZjSpluPRB+rWyLJ01KrDUajcYgpatGhp/ZbXv8s3vlbM/gfGt07xKKRg24FblupWi4/Wwshnf4OSJUvavu3V63nz5iHu7iokWSrDhHqSUrC5+uYTEUiPG4C+/QbavPN/Lxm4+j8L/kYBfwm4G7hS+yVh190b56NJ3DnLUni5iAh3HrzKseSIfrpcDLuvVEOzjoPRvHlztTmf/czKysLypQuRe20rutS5h4qWMJMrqwNIBa5tp0vi8qPGGDl6PEqVKuWzPqoNnzp1CtvWzUOdqONoWj0XJUs4N5WqVievF8O2c5VQp/lTkG9ZyzehnW0MXDkT4vsUoAAFKEABClCAAhQwt4A672aUwJWMhgSJbDcJ0Vy+fNlu0Eo9duDAgWjTpo36UvnpauBKDna0rKAsx9etWzfUqlULEo6SeS353E8qQ9vbHAWuJKQlgSWpfmy7SbWqxx57DNWrV0dCQoISajp//rxyjezswpXpIyIi8J//+Z925wlPnjyphLRsryHn9OrVS1mGUP6Glvs4ceIE5Hh7m23gSo45c+YMPv/8c0i4yXarU6eOEuiS/oeFheHixYu4cOECdu/eDQm0ySZfzvzlL3+J0qVL55+uVeBKGnS0dKN6MXuhPPU9b34ycOWNHs+lAAUoQAEKOBFg4MoJEN+mAAUoQIGgFlAnfpwFrgRJSmcv+PIty3JMl1z60L8o2Mu3QrEptT2Gj5ns0ofWRbVl770d27ch99Q0dKidZe9tt/bN31sSNds+jxYttQ2GSSekpPf0z97H4Do/Iy7G/Wor1jeSYZn/mXGwIUZOeLHAxIn1MfI7A1e2InxNAd8LeBq4kglhWa5hy8a1uHZyCzrWykDtSjmWZ3DhPst8a/KFYvjxZDiqN+yB1u2TULGi9ssIFr7yv/fIN5FlSYdt6xejVM5xtE7MQfVY+8+1W5ag1Y8nwvHT9Rg82W8cHm/YGDL57I9NJqbv3LmDnw4nY8/GeWhTMx2Nq+Ygqrj9qx++GIYfT4UjPK4xevQZgmrVqtk/0M5eBq7soHAXBShAAQpQgAIUoAAFgkhAnXczUuDKk+GJj49XgjwSiLLe3AlcOapyZd2eK787ClzJuTInJhWr7C2H6Erb6jEDBgxA27Zt1ZcFfkqYSpZclIpS3mz2AlfS3qJFi7Bnzx6Pm65Xrx4mTpyYf76WgSsJeU2dOjW/bdtfnnzySSU8Z7vf29cMXHkryPMpQAEKUIACRQgwcFUEDt+iAAUoQIGgF1AnflwJXMmH1FIW+/iOWRjSNM2j6lESCLhgCVt9e6A6xkx8wa0Prt0ZLCUcNmcGmpTcgcaWilxhoe6c/e9js3JC8MOxEkiLeRLDhg1TvkXnfivOz5AKNnvXTcOgRikoW7LwN9SctSBn3EwPxdRNZS1hq5eVb8oVdQ4DV0Xp8D0K+EbA08CVdW/k28QbN27ElfPHEBd1H7GRDxEZmmn5v/9HyHgUhct3y6J6Yl1l0le+2RqoTSaX5W+wLZs3Ijz7OuJKZiE6LN3yDdxHSM+Nxu3MEriTG4eWlhBru3btEBkZGaiuQpZW2Lp1K06fOITokBTEFEtH2ag8ZIdE4U5WcVy9G4XS5R9Dhw4d0KBBA7f7ycCV22Q8gQIUoAAFKEABClCAAqYSUOfdzBy4KleunBK2iomJKTR27gSu5OQDBw5gwYIFXgWiigpcyTWOHz+OWbNmwV7lKnnf2davXz+l6nFRx8l9SFV7V7YQS3loexWrHAWuJJgmyxA6qvBV1DWlwtXzzz8PGTN10zJwJW3+5S9/Uf7WVtu3/ilVwWJjY613afI7A1eaMLIRClCAAhSggH0BBq7su3AvBShAAQpQQATUiR9XAldyfG5uLpKTk7F3/Zfo+/gdVC1nv3qJHGtvO2SpvrLrei30HjwJVatWtXeIZvvS0tKwaP4sxD/agycff4hiboSuHuQAy5IjEV2zH7r36IMSJUpo1i/bhuRbdUeOHMGWlZ+jb4PbqFnBPdPjV8Ow6VwVtOk2EjIZ42xj4MqZEN+ngPYCWgSupFcyCSvfkr19+zakopT8rSMT2FKWXwJMSUlJdpc00P6OnLco/71ITU2FPIuXLFmiTLj27t0b8m1aqbwlSxzoZZMlFr7//ntl2QhZglGW0JDnviyzULZsWY+7ycCVx3Q8kQIUoAAFKEABClCAAqYQUOfdzBi4kr/p5MspXbt2dThv5m7gSgZdgkQSiMrIyHD4b0CWGezRo4eyhJ3tQc4CV3L8lStXsGzZMqVKs+35zl6PHTsWMp5FbfK3+7fffqvM9xV1nPzdOWLECGzevLlQXxwFrqQ9aX/Xrl1YvXq1Uj2/qGuo78kXswYNGlQgbCXvaR24cjTmMg/84osvqt3R9CcDV5pysjEKUIACFKBAQQEGrgp68BUFKEABClDAWkCd+HE1cKWeK3+Mr1wyF20SzqNZdUuVlfA8hDoINGXnAveyQrD+aDQu59bHuHHjfPJtJrVv1j9zcnKwcOFC3Lu4Db1+cRvlY/IcLoco1bceZIfg0u1QzNkXh9ad+ihlrv0VCpDJnrmzZ6Je9FF0qmMxLZ7nsDJXjsU0/aFlSa5TETh0q7ZiWqVKFetbd/g7A1cOafgGBXwmoFXgyl4HZTkBmWyVZfKkCpaEhSR4Jc+uUEcPZnsN+WCf9Gv+/PkoX768Un1v8eLFmDRpUsBDYdIv+TbzoUOHMGfOHGWyXCptSZBN+qdVyJaBKx/8o2KTFKAABShAAQpQgAIUMJCAOu9m9MCVLBVYsmRJ5X/yBRoJ78j/oqOjixwNR+Gb119/XfnikKOT5cs7EoiSOSzr5f8kaFW9enXI0nRSGeqjjz4q1ESvXr3QpUuXQvvt7ZDPD/fv349Tp065HFySdqT9nj17Kn2w1666T75guWLFCuXvdXWf/JS/P2Ue76mnnkKlSpUwffp0HDt2zPoQ5UuVI0eOLLDP9oX8Dbt+/XplLuDGjRvKF2Wtj5Fxq1atGiSEVrNmTeu38n/XOnAl/fjb3/6W3776i9yrBPR8sTFw5QtVtkkBClCAAhT4XwEGrvhPgQIUoAAFKOBYQJ34cTdwJS3eunXLUup7H1LP7cfDW8dRo3wuSkc9QqkIQAJBdx6E4JZlqbsLt0ugZKXGqP14GzRu3ATFihVz3CEfvCMTM2fPnkXyvh1Iv5qMiiVSEBvzSFkSMcISFEu3hMHuZITi6p1Q3EU8yiY0Qau2nVC5cmUf9KboJh8+fIjt23/EjfOHkHv7KKqUyVT6WSoiD48sgbA7mZa+Wv537mYJhJerj4TEpmjTpi0iIizoLm4MXLkIxcMooKGALwNXUulKJnBl+YYWLVpAlimV4JVMrNauXVuZBLe3tIOGt2e3KXn2btq0CVevXlWWZZWJcVkSUforla4CsUkIVyaTT548qVTekglu+dawPBdlgn348OGaLnHIwFUgRpnXpAAFKEABClCAAhSggH4E1Hk3PQau9KPkuCfp6enK35TyUyo7y1yd+sWiHTt2KKEs27OlYlTTpk1tdxf5Wv5WvH79ulJR+v79+8rfhfI37Zo1a5RKyPZOlsCZBKKioqLsvV1gn3xBKiUlBdJ2fHw84uLinIa1CjTgwgupMi3XkPuQPskXn6RqswTT/LnJfPF7771X6JK/+93vUKpUqUL7tdjBwJUWimyDAhSgAAUo4ECAgSsHMNxNAQpQgAIUsAioEz+eBK5UQPmDXv6YP3bsKK5fu4rTlg+ur1g+YI+1TB506ZKE9u07KB/8qxMi6nn+/ikVTaSvJ0+ewCnLh+3Hjx/DhQvn0aDB45blDauhcZOmyqSHVIXx92SErYX0UyaTJHy1ZfMm3L17BzVr1ER8QoLlm2nV0ax5C6UCiyfVtxi4stXmawr4XsCXgSvpvYQ1ZbkCmXyWb9lK9aZr164pFZwOHjwIWSavY8eOfq0sdfHiRaxdu1apwCdhK9mysrLw+eefK/1p27atss8f/488/7ds2aL8T5YxkG/3ygS3hFWluuDSpUsxZswYZTJay/4wcKWlJtuiAAUoQAEKUIACFKCA8QTUeTcGrlwfO/mSjsx3yZeIitrmzp2L5OTkQodMmTIFNWrUKLTfkx0SxJJKW3v27LF7erly5TB+/HilSpXdA4Jwp8wDyJetrLfExERMnjzZepemvzNwpSknG6MABShAAQoUFGDgqqAHX1GAAhSgAAWsBdSJH28CV9btye/ybS1ZQur48eNKuWopXV2rVi3ld39Xt7Ltm/r65s2b+OCDD5Slt+TDdvn2m7My6Oq5vv4pwQmpviIVak6cOGEJhV1QghJSEt3ZZJMrfWPgyhUlHkMBbQV8HbiS3sq3b2fOnKksjycBK3VLS0vD7t27cebMGWXpBfkWrnyj1pVv4aptuPtTvr0rk989evQoNNEt/42YNm0aBg4cqPTH3bZdPT4jIwOylIEsyyDP07JlyyrLxMo3fGWTEJZUP5SgmiwjKCZabwxcaS3K9ihAAQpQgAIUoAAFKGAsAXXejYEr18ftm2++UZZ7Hzt2LGJjY+2eKMvDy9+c8new9SZ/50olJa3nH3fu3KlUlpYvSFpvEuwaN26cT/++tr6e3n/PzMxUqlvJT+tt8ODBaNWqlfUuTX9n4EpTTjZGAQpQgAIUKCjAwFVBD76iAAUoQAEKWAuoEz9aBq6kffmge968eahYsaKyXJNUFpEqWI0aNUKHDh2gVjux7ou/fpc/+qVvLVu2RL169fDdd98pIbGhQ4cq36DzVz9sryPf4NuwYYOyHJgsAybVX9avX2+pvlUVnTp10qxvDFzZyvM1BXwv4I/AldyFhEkldCXPWevQlbwnz2VZ3k+WHDx//jzq16+vTHiqASQ5RotNQqPTp09XlnCQPtirbijhL/nvwoQJE7S4ZIE2pIqWTIbLBHylSpUg/3177LHHCi1dICGs1atX46mnnvLZErIMXBUYGr6gAAUoQAEKUIACFKBA0Amo824MXLk29Fu3bsWqVauUgyMjI5X5MPl7TubG5G9L+dKkfElR5stsw1ZyUpcuXSBfWPTFJl/YmTVrllKRXtqX5emHDRumebjLF333dZtSCUzmG6QamFS7tt7ky6MSgpPx9NXGwJWvZNkuBShAAQpQwCLAwBX/GVCAAhSgAAUcC6gTP1oHruSKEiCaPXu2MgHRtGlT3Lp1C+vWrVOCVxIoatasmfJNNX+Hr2TyRpbr6927dz6MBLDq1q2rBATyd/rhFwl/SRDtwIEDSjUrCYDJxJB8E09KcEuVGAmCaWnEwJUfBpaXoICNgL8CV3JZefZ++umn6N+/v1JZ0KYrykt5Bq5YsUKp8CTPZ/mffHNYi2fNtm3blMpS8g1WR5tMjH///feQ6lsyQe3J8qjWbcvkrjwvJUwmQS55niclJSmhX+vj1N9lkv7rr7/G6NGjlTCWul/rnwxcaS3K9ihAAQpQgAIUoAAFKGAsAXXejYEr5+MmFd7lb1nbKlJypoStQkJC7L6ntlymTBm89NJLPq1gL3/DypecZF6zZ8+eSp/U6wfjT/nS14cffqjMQ0gVaXtb69atMWjQIHtvabaPgSvNKNkQBShAAQpQoLAAA1eFTbiHAhSgAAUooAqoEz++CFzJNSRkJRMR8se1lI6WP77lQ3GpbiL/k7CR7G/cuLEmH/Sr9+Xo55EjR5RvwU2ePBklS5bMPywlJQULFiyABASkIoqvNwk77Nu3TwkHVK9eXQlF1KxZU+mTGG3fvl0JQsiSW1ovdcjAla9Hl+1ToLCAPwNXcnW1elPHjh2VSn6Fe/TvPXfv3oU8E06fPg2pTCXfGpbwVUxMjKNTitwvS8lKdSkJijp7dsn15NvBssRh+/btPZqozs7OVgKr8t8TCZpJ1a7ExETExcU57KcsM7h06VJ0797dYSDN4cluvsHAlZtgPJwCFKAABShAAQpQgAImE1Dn3Ri4cj6wMjcnX4yREI+7W0REBF544QW/zOnJF4jsVXJ2t89mOD41NRXvv/++w1uRudfXXnvN6fyAwwZcfIOBKxeheBgFKEABClDAEwEGrjxR4zkUoAAFKBAsAurEj68CV6rjBx98oHwAn5CQoO5Sfkoga/PmzcoH5lLZST4sL1WqlNfVTgpc5H9fXLt2Dd9++y0mTZoE+dab7Xbs2DGl9PWvfvUrj8MGtm1av5ZggITNJEwlgYR27dqhT58+BSZpJGx16tQp/PDDD0o/tag2Y90H+Z2BK1sRvqaA7wX8HbiSO5Jv3r733nt48cUXYfvstXfH8vyRJfbkGSUhKKk8ZR1MtXeO9b779+8rywT813/9V5GBJ+tzpDLVtGnTMGrUqEJL/lkfZ+93qZi4cuVKyAcXTz/9NGRpRPnGs7Nt6tSp6NGjh3KPzo719n0GrrwV5PkUoAAFKEABClCAAhQwtoA678bAlWvjmJGRoczdydyVq5t8gVG+QFm+fHlXT+FxGgkUFbiSStZjx46FzDn7emPgytfCbJ8CFKAABYJagIGroB5+3jwFKEABCjgRUCd+fB24On/+vBJmGjBgAKSik+0mf6AfPXpUqbJSokQJ5YPwBg0aIDIy0vZQj19L2KpJkybKh/OOGpGlsG7fvo2nnnrKpQ/uHbVjvV9CDxLmkiCVfAOuVq1a+cEy6+PkG3J79uxBcnIyhgwZ4rOJIgaurNX5OwX8IxCIwJXcmSzJIEv3SSBJwqyubPLMkqX55Fkhy5vKBwNSNUoCTY42CZQuXLhQWcpPlot1Zztx4oSyhOr48eOLvIZUBjx79qyy/KqEVyVE1qZNmyLPse6HnCOVDGW5Qamo5Y9vJDNwZT0C/J0CFKAABShAAQpQgALBJ6DOuzFw5d7Yyzzi7t27lb9NpTqy7SZfUKxXr55SMV9sXfnyjW0bfO29gKPAlaweMGLECFSuXNn7i7jQAgNXLiDxEApQgAIUoICnAgxceSrH8yhAAQpQIBgE1IkfXweuxPLq1auYMWOG8u2m+Ph4u7wyiZKVlaVULZHl/zp37oykpCSEh4fbPd7VnVIFRdqWJfqK+pBdQgPz58+HBAa8/QaWXE+qxezYsQNdunRRKlpJtRgJMNjbJNwgla0mTpzo0+UVGbiyp899FPCtQKACV1K1Sp6lS5YswauvvupyOEk05FksISUJgUoYVAKrskShveCVVJvKzMxEv3793J7olrCpPCdPnjyJMWPGFHrey1KBW7ZsUSohynO5ZcuWKFeuHGTJCFc3cZD//kiQV57vRf13wNU2XTmOgStXlHgMBShAAQpQgAIUoAAFzCugzrsxcOXZGMvfi/KlIPlypFS/knm1mJgYlC1b1uH8mmdX4lmeCEjVavkM9saNG8jNzUXFihUhc74VKlTw29/d0m8GrjwZPZ5DAQpQgAIUcFGAgSsXoXgYBShAAQoEpYA68eOPwJUAnzlzBuvXr1dKfcfFxRVpLtVMtm7dClkKUD5cl6okUiZcKmC5usmH7Pv371c+qJcP8l2pmHX9+nV89tlnSjCsRo0arl5KOe7evXvKPUrFlpSUFOXbdlJJxVmfL126pAS9Ro8erUxOuHVRNw9m4MpNMB5OAQ0EAhW4Urv+448/KhUEBw0a5NGSqfI8PnDggPI/qXYlz+Jq1aohOjpaqYQllaNeeeUVt5YgVPum/pQ2pN3mzZsr4S2ZsD148KBSHVC+udy2bVvlvwXq8a7+lEn5uXPnKtUVJcDrr7CV9I+BK1dHicdRgAIUoAAFKEABClDAnALqvBsDV+YcX96VPgQYuNLHOLAXFKAABShgUgEGrkw6sLwtClCAAhTQRECd+PFX4EoCUBJGWrp0KSZMmAApMe1su3//vlIdSz7slwooUp1EKka5Ut1EqrPMmTMHI0eOtFuVxdG1JSy1ePFiSABKAgXONunj2rVrlf41bNhQqaIiZbNd6ePly5eVZa4GDx6sBBicXcvb9xm48laQ51PAfYFAB66kx7t27VKqXcnzUL4V7MkmwSt5ZskyqfJTgrO3bt2CBLnkm6zebPKt5VmzZinLF0rb8q1l+VCiSpUqKFOmjEdNS6VBeZaXL19eqZjoqMKgR427cBIDVy4g8RAKUIACFKAABShAAQqYWECdd2PgysSDzFsLuAADVwEfAnaAAhSgAAXMLMDAlZlHl/dGAQpQgALeCqgTP/4KXKn9PXr0qFIpRUJGrlSdUs+TEuJSqeXnn3+GBJsaN26sfJBur4KULG81e/ZspVqKLIXl7iZLWEmQQJbICgsLK3C6BMekakpqaqqy1JaEmFq0aOH2B/oSCJMlDKXqSq1atQpcw1cvGLjylSzbpYBjAT0ErqR3smzpuXPnlDCpK4FQx3cEyNIO77zzjvJTql717NlTCbbaex4X1Y4sQSDP0507dypLC8oSgr/85S+9DqBK/6TN8+fPK6Hbovrgq/cYuPKVLNulAAUoQAEKUIACFKCAMQTUeTcGrowxXuylMQUYuDLmuLHXFKAABShgEAEGrgwyUOwmBShAAQoERECd+PF34Eo+CN+8ebPywb9UW3H3g/+7d+8qy2NJcEBCUbLcYKNGjVCqVCnFMTc3V6miVbp0aaUalidVTeRDf1mG6rHHHlPaUAdIljg8fPgwLly4oFTokvdluSt3ryEVXCRs1b17d6Uilr+WuWLgSh1J/qSA/wT0EriSO16zZg2ys7OVMKmnAnK+hLekul/v3r0hIVoJNt28eVMJj0oYNjY2tsjm5TkulQvlWRoSEoI6deooz1J5Rp09exZDhgwp8vyi3pQQ17Zt25Q+SfUt9b8NRZ3ji/cYuPKFKtukAAUoQAEKUIACFKCAcQTUeTcGrowzZuyp8QQYuDLemLHHFKAABShgIAEGrgw0WOwqBShAAQr4XUCd+PF34Eq90e+++w4SYJKl+4oXL67udutnVlaWsmSUVL6S8JL87+LFi9i+fTsmTZrkVlv2Dv7Xv/6lBBPkA3sJGOzevRtPPvkkOnXq5FZ1Luu2pfrWV199BanwJUsP+nNj4Mqf2rwWBf4toKfAlYSlFixYoCyX2r9/f4+GSEKnO3bswLPPPptfAVCCtNK2BJ1kiVX574qEncqWLVvgGhJmXb58OQ4ePKhUBpRndlRUVH47EpZauHChEtiSZ60nm1S2On78OMaOHZvfrifteHsOA1feCvJ8ClCAAhSgAAUoQAEKGFtAnXdj4MrY48je61uAgSt9jw97RwEKUIACBhdg4MrgA8juU4ACFKCATwXUiZ9ABa7k5tatW6dUSXnqqafcrhJljSOVVvbt26f8T5bqa9u2rVehKGk7LS1N6d+hQ4dQpUoVJRwgk2TuVuSy7qcsnTVv3jzUrl0b7du3Vyq7WL/v698ZuPK1MNunQGEBPQWupHcS+pQKe9WqVVOWQnWnwl56ejrmzJmjhKni4uIK36xljzyPT5w4oYSqKlSooCz9WrJkSSUEJZWtpDJg69atlaCVvQbkGf7ZZ59BAmFSwVAqYLmySehLglz79+/H8OHDERMT48ppPjuGgSuf0bJhClCAAhSgAAUoQAEKGEJAnXdj4MoQw8VOGlSAgSuDDhy7TQEKUIACxhBg4MoY48ReUoACFKBAYATUiZ9ABq6kmsmyZcsgVU+k0pU3m1S7ev/99zFixAhliau9e/fiiSeeQN++fREeHu5y0xKKkgorV65cQdOmTSFhhGPHjuGFF15wqx3bC0r1FwlbSUhBKreEhYXZHuLz1wxc+ZyYF6BAIQG9Ba6kg/LMlWdvfHw8OnbsWKjP9nbIs/GTTz5RQlrNmjVzGoS6dOmSElo9ffo08vLylKpVUtlPAlfONgldybKu8t+F6OhoZ4cr78vfflKJcOLEiQFbRtC6owxcWWvwdwpQgAIUoAAFKEABCgSfgDrvxsBV8I0979h/Agxc+c+aV6IABShAgSAUYOAqCAedt0wBClCAAi4LqBM/gQxcSWcfPnyI2bNno06dOmjXrp3L/bc9cMWKFZAqKl27dlXeknCALDUolVYk5NS4cWNUrVq1UNUTCQLcunULV69eRXJyMm7evKkEtdq0aZMfsPr222+VcyWY4Gq1Fev+Sdhq/fr1yM3NhVTzCtTGwFWg5HndYBbQY+BKxuPevXtKJSl5rrVo0UIJlzoaJ3l2rVmzRjmmd+/ejg5TglwStJKQqjx75bnbvHlzZWnBzZs3K8uySsirVatWypKqRVWh2rNnj1IVS0K0zkKz169fx6JFi5TAbbly5Rz2z59vMHDlT21eiwIUoAAFKEABClCAAvoTUOfdGLjS39iwR+YRYODKPGPJO6EABShAAR0KMHClw0FhlyhAAQpQQDcC6sRPoANXAiLLVElVqXr16kGCTu5uO3fuVD7clw/mS5QoUeB0Wd7q8uXLkCorEgBo2LChEgAoXbq0UsVKKmGdP39eCQZI5RVZPtC2+pT0T5bgkrCULJHlziZBBanUIteTyla2/XOnLW+PZeDKW0GeTwH3BfQauJI7keejVN6TJU5l+T5H288//4zDhw9j6NChdpd/lYpUEnC9du2a8qyTDxTkWWpbnUoCtmfOnMHZs2eV52/58uWVJWDtLU8ox0oVrqioKEjIy9HSh9LWkiVLMGDAACQmJjq6Bb/vZ+DK7+S8IAUoQAEKUIACFKAABXQloM67MXClq2FhZ0wmwMCVyQaUt0MBClCAAvoSYOBKX+PB3lCAAhSggL4E1IkfPQSuREY++P/www+VClVSbcXVSlKy9N/HH3+M3/72t0qFq6KUZdnBdevWYeXKlZD7vnDhAgYOHIjWrVvI5doHAABAAElEQVSjePHiRZ2Kc+fO4csvv8Qbb7yhBAqKPNjqTVniSqpnjRkzxmpvYH5l4Cow7rxqcAvoOXAlI5OSkoJPP/0Uzz//vN1AaWpqKj7//HM888wzqFSpUv5gypKwEkZdvXq1Uh2wQ4cO6NWrl9NnqdqAVBeUKlYSqqpRowb69eunLDtYrFixAs9/6Vv37t1Rq1Yt9dT8nzdu3MCsWbMwfvx45dz8N3TwCwNXOhgEdoECFKAABShAAQpQgAIBFFDn3Ri4CuAg8NKmF2DgyvRDzBukAAUoQIFACjBwFUh9XpsCFKAABfQuoE786CVwJV7y4f3ixYuVJf2kEpWz7cGDB0plrLZt2xZZ2eTRo0eQD+YPHDig/IyMjET16tWVqley3F9CQgJatmzpNEi1e/duSGhp0KBBkDacbTt27FCuIVVhIiIinB3u8/cZuPI5MS9AgUICeg9cSYelStTatWuVKnzWVaJk2UEJNCUlJSkVCOVYWYJVlgyUqoGy1J9UBpSlA6USlSebLP8qVa+OHj2Ku3fvKu01aNBAWXJQ2pMKhbJk7KhRo1CqVKn8S0g1LQlrSd9kSVq9bQxc6W1E2B8KUIACFKAABShAAQr4V0Cdd2Pgyr/uvFpwCTBwFVzjzbulAAUoQAE/CzBw5WdwXo4CFKAABQwloE786ClwJYCyNNVf/vIXTJkyBVWrVi3SVJbCKlu2rBIScFQRSz7Ml8DA9evX0alTJ2X5KgkJqJtUvZKggYSjJOQlH+o7akvOkQ/4pcqLVMUqajt06BBmz56Nd955J6DLCFr3kYEraw3+TgH/CBghcCUSEmCS5U+l0pUaKJVqgPKMlSUHJbS6detWyPKCXbt2RZMmTZTjbJdg9VRVll+V57Us/SoBK6l61bNnT+V5K2HXbdu24ZVXXlFCXnLcP//5TyX8KkvRFvXM9rQ/3p7HwJW3gjyfAhSgAAUoQAEKUIACxhZQ590YuDL2OLL3+hZg4Erf48PeUYACFKCAwQUYuDL4ALL7FKAABSjgUwF14kdvgSu5aVm+T5b+6927N6pUqVLIQSpW7dy5U6mIMm7cuAJLWMkyVRKukg/tpR0JA8gH8o0aNSpwnG2jUulKKmBJ1RZpo3bt2krVrNjY2AKHSiBs5syZSsirbt26Bd5TXxw5cgT79u2DVLaKjo5Wdwf8JwNXAR8CdiAIBYwSuJKhkTDV8uXLMWHCBMhyrdu3b0ezZs2UZ6lUu2ratCmk+pQ/nmsSWpXrlyxZUnmGS1UtCbt27twZq1atUipgtWvXTrf/ohi40u3QsGMUoAAFKEABClCAAhTwi4A678bAlV+4eZEgFWDgKkgHnrdNAQpQgAL+EWDgyj/OvAoFKEABChhTQJ340WPgSkQl+CTVVeSD/9KlSxdAliWmvvvuO4wePTq/EosccOnSJaxZswY3b95UqlnJMlNlypRRQlcFGijihVRZkUovEtiSqlcS+BowYIDSjnpaSkoKpk+fDgl7VaxYUd2t/JSwlYTFxo8fD9uwVoEDA/CCgasAoPOSQS9gpMCVDJYETzds2KAs5SfVrVq0aKEEVuV5pla+8teg5uTkKM/jkydPQgJYEqKVZ3KtWrXQp0+fIkO0/uqjo+swcOVIhvspQAEKUIACFKAABSgQHALqvBsDV8Ex3rzLwAgwcBUYd16VAhSgAAWCRICBqyAZaN4mBShAAQp4JKBO/Og1cCU3Jf8tX79+PUaMGKFUNpF99+/fV5bq69atm7Lk1O3bt5VgwMGDB5GWlqYsfSXVrLRY5urBgwfYu3evEryS5a1kCa34+Hil4ooEq6TiypAhQ/KvdfbsWaU6zJgxY3QXthI7Bq5EgRsF/Cug98BVeno6UlNTsXXbVly6clFZ1q9U2VIIKxaGO7fuoER4CdSskYhmTzyBhIQqyvPPH4Ky3Ks83w8fOYwjPx3GzVs3ERUdidJlyyDt5m1kPchGjeqP4fHHG6JO7TpKMDc0NNQfXXPpGgxcucTEgyhAAQpQgAIUoAAFKGBaAXXejYEr0w4xb0wHAgxc6WAQ2AUKUIACFDCvAANX5h1b3hkFKEABCngvoE786DlwJXd5/PhxbNu2DRJikg/T58yZo4SeqlevjsOHD+Pu3buoVq2asuSUhKG0CFrZ6krw6urVq5AqK+fPn1eqWskyW7JsYHh4uFJpRZbb+vbbb9GjRw/UrFnTtgldvGbgShfDwE4EmYBeA1dSzS85ORl79+9BXJVyiK8aj/hqlRBVMip/hGR51Xt37uH6lRu4ePYyUi/fQqcOnSwhp8eV53H+gRr/Is/b79d9j7zwXFSvVQ1VayRYwlYlUcwSAlO3zIwHSL2RikvnruDq2WuoWT0RSUlJyjNZPSaQPxm4CqQ+r00BClCAAhSgAAUoQIHAC6jzbgxcBX4s2APzCjBwZd6x5Z1RgAIUoIAOBBi40sEgsAsUoAAFKKBbAXXiR++BKwHcsmULtm/fjnbt2mH37t0oX768UpGlTZs2aNWqlV8/YJeqK1J1S5bdkqWt9uzZo4TBNm7ciN69e0OCWHrdGLjS68iwX2YW0GPgKjs7WwmIpqbdwPDnhrq8NJ8EsGZ/Ng9NGjyhhJtCQkI0HbpHjx5h69atWLxkEZ55ZRwqV63sUvvSr01rtuDqmRuY8vwUvy99aK+TDFzZU+E+ClCAAhSgAAUoQAEKBI+AOu/GwFXwjDnv1P8CDFz535xXpAAFKECBIBJg4CqIBpu3SgEKUIACbguoEz96DVzJB+9STercuXNISUnBnr17lCWu7t65i8SaiUrQSvoeFxfn9r17e4KyzJWlupYEAy5fvoRIS0WY2HKxqFe3nhIGkwpXlSpV8msQzJV7YuDKFSUeQwFtBfQWuLpz5w4WLFyACo/FomWH5m5XqrqfnoFdm3cjLLs4nur7FKKi/q8ilrdy33//PS6nXkL3AUmWEFi4W83l5j7CkX1HcPbYRQzqPwgVKlRw63ytD2bgSmtRtkcBClCAAhSgAAUoQAFjCajzbgxcGWvc2FtjCTBwZazxYm8pQAEKUMBgAgxcGWzA2F0KUIACFPCrgDrxo8fA1enTpzF//jxk5T5E9/5JqFi5PCIiIwCppmKpZHIr9TaSdx7E8SOn0OAXDTBy5EifLCVoOyAZGRnKkoanTp9EC0tQoX7TeoiJic7vlyxxdfzICSTvOoyIYhF45plnAhIIs+23+pqBK1WCPyngPwE9Ba7UylaVEuPQpFVjt8NWqpoEYrdv2IVKpSqjffsO6m6vfu7cuRNbd23FqOeGIdzNsJX1hU/+dApH957E5OcmW+/2++8MXPmdnBekAAUoQAEKUIACFKCArgTUeTcGrnQ1LOyMyQQYuDLZgPJ2KEABClBAXwIMXOlrPNgbClCAAhTQl4A68aOnwJVUjtqy1bIsVOplNG3bBNVrVi0yEJCTk4P9O5Jx/sRFtGvdHo0aNUKxYsU0h5ZlBPdaKmwdOLwftRvWRsMn6hcZCJAwwukTZ/Hz/qOIj6uCrl27aloFxtMbZODKUzmeRwHPBfQUuDp67CgO/LRPqSDl+R39+0ypdLV89ioMG/g0EhISvGouLS0NcxbMRo+h3RAdXdKrtuT5+/2yH1A2Ihb9+vWz5HS1XfbQ1c4xcOWqFI+jAAUoQAEKUIACFKCAOQXUeTcGrsw5vrwrfQgwcKWPcWAvKEABClDApAIMXJl0YHlbFKAABSigiYA68aOXwNWFCxcwY+Y3aN21Jeo3qedWcOpu2j1s+/5HlC9TSVniShOg/23k4cOH+PLLL1EuoQxadmyOktGuL58lgbA9W/fhnGWJq2cnPWsJEliqYQVwY+AqgPi8dNAK6Clw9a+pH6Bjn3aIr1pZk/E4YakmtW9TMl566SW3ntm2F1+zdg2KlQ5Bo+aP277l0esHmQ8w9/NFGDd6HCpX1uZe3e0IA1fuivF4ClCAAhSgAAUoQAEKmEtAnXdj4Mpc48q70ZcAA1f6Gg/2hgIUoAAFTCbAwJXJBpS3QwEKUIACmgqoEz96CFzdunULs+d+i6T+HRFXMc6j+3z4MBurFnyHmvG10L179yIrY7l6AVl+a968eYgsVxwdnmzn6mmFjju09wj+5+AZjB8/IaCVrhi4KjQ03EEBnwvoJXB17NgxbNz+A4aMG6jpPa+YsxrtW3aE/LfEk00qUn3w0QfoN7I3YkprF0o9ffIs7l99gO7dunvSLa/PYeDKa0I2QAEKUIACFKAABShAAUMLqPNuDFwZehjZeZ0LMHCl8wFi9yhAAQpQwNgCDFwZe/zYewpQgAIU8K2AOvGjh8DV0qVLEF0xEo1aNPTqpqWqyfdLfkDH1p1Rr149r9qSk48fP4b9P+9Dt35dvG5r+4ZdiEI0evbs6XVbnjbAwJWncjyPAp4L6CFwJaGmTz/9FC26NkX1xGqe34ydM8+cOoc7F+5Znm297LzrfNfevXvx8+kj6DX4SecHu3HEnbS72LshGaOGj3LjLO0OZeBKO0u2RAEKUIACFKAABShAASMKqPNuDFwZcfTYZ6MIMHBllJFiPylAAQpQwJACDFwZctjYaQpQgAIU8JOAOvET6MDV/v37sfHHDRj53NOWJanCvL77G9dSsOeHA5g4fiJCQkI8bi83Nxd/+su7GDyuP2IrxHrcjnpiZkYm5n+5GC8890vExMSou/36k4Erv3LzYhRQBPQQuLp9+zY+/vQjPPvGBM1HJf3efexYsxsjLcGmsDD3n+ESBHuicyM8Vqu6pn17mPUQS2Yux8svvKppu642xsCVq1I8jgIUoAAFKEABClCAAuYUUOfdGLgy5/jyrvQhwMCVPsaBvaAABShAAZMKMHBl0oHlbVGAAhSggCYC6sRPIANXWVlZ+OTTT9B1UGdUrFxek/vKyc7B90t/QOumbVG/fn2P21yxYgXu591DUu9OHrdhe+JPycdw5tA5PPPMM5oseWjbvrPXDFw5E+L7FNBeQA+Bq6tXr2LRsoUYNmmQ5jeYmfEAW1b9iNHDx3gUuHr33XfRf3xflNcg2Gp9c3l5eVg+dxWGDxiJ0qVLW7/ll98ZuPILMy9CAQpQgAIUoAAFKEAB3Qqo824MXOl2iNgxEwgwcGWCQeQtUIACFKCAfgUYuNLv2LBnFKAABSgQeAF14ieQgaszZ85gy85N6D2sh6YgVy5exarZa/H73//eo3YfPHiAv/7tr5j42hhL1a1iHrVh7yQJAMyYOgcTxkxAhQoV7B3i030MXPmUl41TwK6AHgJX8n/72/f9iJ6Du9ntozc7syyVpH5YshFjR453+3mZk5OD3/72t3j1D79CaFioN92we+73S9ejdZN2qFOnjt33fbmTgStf6rJtClCAAhSgAAUoQAEK6F9AnXdj4Er/Y8UeGleAgSvjjh17TgEKUIACBhBg4MoAg8QuUoACFKBAwATUiZ9ABq42b96EsLIhqFO/lqYO8iH+p3+ehldeehVxcXFut33s2DHsTt6JXkOfdPtcZydsWfsjHqtYEy1btnR2qObvM3ClOSkbpIBTAb0ErjZv34inRvZ22l93D8h6kIUflm3G2BHjPApc/e53v8Pzv52EyMgIdy/t9Pi1S9ahQ4vOqFmzptNjtT6AgSutRdkeBShAAQpQgAIUoAAFjCWgzrsxcGWscWNvjSXAwJWxxou9pQAFKEABgwkwcGWwAWN3KUABClDArwLqxE8gA1crV61AtccTUCmhoub3vnTWCrRr0QENGzZ0u+39+/fjxv1reKJNY7fPdXbChbOXcOPUTfTp08fZoZq/z8CV5qRskAJOBfQQuLp27RrmLJiNUVOedtpfdw/IuJ+Jbat3YNTw0R4tKfjee++h59PdULlKJXcvXeTxjx49wpJZyzF5/BS3g2BFNuzimwxcuQjFwyhAAQpQgAIUoAAFKGBSAXXejYErkw4wb0sXAgxc6WIY2AkKUIACFDCrAANXZh1Z3hcFKEABCmghoE78BDJwNX/RfDRs+wvEli+nxS0VaGPrOkslqQqJaNGiRYH9rrzYvHkzwmNDUKteoiuHu3XMndt38fP24xgwYCBCQkLcOtfbgxm48laQ51PAfQE9BK7S09Pxt7//FZP/8xnNl+67m3YXyVt/wpCBQxAa6v6ygDNmzECNhlVQt2Fd93GLOEMqb61d/AOeGz+5iKN89xYDV76zZcsUoAAFKEABClCAAhQwgoA678bAlRFGi300qgADV0YdOfabAhSgAAUMIcDAlSGGiZ2kAAUoQIEACagTP4EMXM2cPQMtuzZD2dgymivs3rwHZSPKo0OHDm63vX79epSrUQpVqie4fa6zE+6m3cP+jYcwbOgwj8IJztov6n0GrorS4XsU8I2AHgJXcmcSbKr6i3g0aPoLTW/08L6fgPQwdO/e3aN2T506hXVb1mLw2AGahlBvpd7Gvg3JGDN6rEf98vYkBq68FeT5FKAABShAAQpQgAIUMLaAOu/GwJWxx5G917cAA1f6Hh/2jgIUoAAFDC7AwJXBB5DdpwAFKEABnwqoEz+BDFwtXLwA9VvXRVyFWM3vdd2yDWhUpwkaN3Z/WcDt27fjUXQ26tSvpXm/0m7dwYndp9GvXz/N23bWIANXzoT4PgW0F9BL4EqCTSvWLsPo50doepOzPpmHYQOHoVq1ah61m5eXh6mffIgeQ7qiTDntwrcH9xxBTEhptG3b1qN+eXsSA1feCvJ8ClCAAhSgAAUoQAEKGFtAnXdj4MrY48je61uAgSt9jw97RwEKUIACBhdg4MrgA8juU4ACFKCATwXUiZ9ABq5WrlqJhF9UREK1eM3vdc7nC9CvV38kJrq/LGBycjLOp5xBmy6tNO/XyaP/g4epuejatavmbTtrkIErZ0J8nwLaC+glcCV3NuvbmajTLBGP1aru9Y1KUGrPj/uRcvYmxo8fj7CwMI/blJDrzcwbaNXJ/SVg7V007VYaFn65FFOefwGxsdoHeu1d03YfA1e2InxNAQpQgAIUoAAFKECB4BJQ590YuAqucefd+leAgSv/evNqFKAABSgQZAIMXAXZgPN2KUABClDALQF14ieQgas9e/YgJeMamrd7wq2+Ozv4/v0MTH37U/zhD39ATEyMs8MLvX/+/HksXbUUI54bUug9b3csm70SLRu3RtOmTb1tyu3zGbhym4wnUMBrAT0FruTZtnrdKgwe39/r+5JqfVtXb8eIYSMRHR3tVXv37t3D19O/Qud+HVApoaJXbWVnZ2PJzGXo0aU36tat61Vb3pzMwJU3ejyXAhSgAAUoQAEKUIACxhdQ590YuDL+WPIO9CvAwJV+x4Y9owAFKEABEwgwcGWCQeQtUIACFKCAzwTUiZ9ABq6uXr2KZZZg06Dx2i6v91PyUVz4+TImTJjgkV9ubi7ef/999B/fB7Fx5Txqw95JWQ+yMP2fs/DqK79GqVKl7B3i030MXPmUl41TwK6AngJXjx49wvLly3En6zaS+nRGiYjidvvsbGdmRibWLduIFo1aahYePXr0KFZ+twJPjeyN2PKePXfl/nZv3YfM1CyMGKHt0onOTGzfZ+DKVoSvKUABClCAAhSgAAUoEFwC6rwbA1fBNe68W/8KMHDlX29ejQIUoAAFgkyAgasgG3DeLgUoQAEKuCWgTvwEMnCVk5ODr7/+Go+3rYfEujXd6r+jgx9kPsCKud9haP9hqFSpkqPDnO7fu3cvDh7fj77DeiM0NNTp8a4csOm7LSieG4lBgwa5crjmxzBwpTkpG6SAUwE9Ba6ks1IBauPGjbh6+zI69+6IyMgIp/dgfcBty3J9axevR+NfNEGHDh1QrFgx67e9+l1CV1t3bkHnPu1RNrasW209fJiNbeu3Izc9DwMHDPS66pZbF7dzMANXdlC4iwIUoAAFKEABClCAAkEkoM67MXAVRIPOW/W7AANXfifnBSlAAQpQIJgEGLgKptHmvVKAAhSggLsC6sRPIANX0ueUlBR8/uWnGPn80ygZXdLd2yh0/ImfTyHl9C0MHjy40Hvu7vj4k4/QIukJVE+s5u6phY5PuZ6KdYs34Ncvv1boPX/tYODKX9K8DgX+T0BvgSvpmVSC2rFjB1avWYU+w3pZAq81EBpWdLBUKv9dPHsJ65dtwoCnBkA+NPDFdubMGXz11Vd4on1jtOzQHOHFwxESEuLwUnIvabfvYKUlaFujSk0MHTrU4bH+fIOBK39q81oUoAAFKEABClCAAhTQn4A678bAlf7Ghj0yjwADV+YZS94JBShAAQroUICBKx0OCrtEAQpQgAK6EVAnfgIduBKQzZs342raZcsSV5288rl25Qa2rv4RI4aNRFxcnFdtyclnz57F2g1r0HNod0RFRXrcXnZ2Dr5bsBbtW3ZA/foNPG7H2xMZuPJWkOdTwH0BPQau1Ls4f/48du3eifSse6hUtSIqVC6PmDIxiCoZhby8R3iQ8QB30+7h3P+cR1rKXUSXiEHXrl1RoUIFtQmf/Lx//z62bt2KlLQbCI8qZgm9VkXpMqUQFR1lCWAVt/QrExnpGbh47jJuXElBXlYI2rVthzp16hQZzvJJZx00ysCVAxjupgAFKEABClCAAhSgQJAIqPNuDFwFyYDzNgMiwMBVQNh5UQpQgAIUCBYBBq6CZaR5nxSgAAUo4ImAOvGjh8CVLHE1d+5cFC8dhi69PQtdpd64iRkfzcbLv3oZCQkJnpAUOicvLw+7du3CmvXfYeLLYxHpQehKlk2cM20hEqskYuDAgQENAzBwVWiIuYMCPhfQc+BKbl6ev5mZmTh+/Dj27tuLtDtpSEtLQ5hlKdVSpUsjvnI8mjRugsTERJQsWVKzJVadwUvlKunX5cuXsWPnDqTeTEVqagpk6cAyZUqjYoWKSKxZC02bNkVMTAyKW4JYetoYuNLTaLAvFKAABShAAQpQgAIU8L+AOu/GwJX/7XnF4BFg4Cp4xpp3SgEKUIACARBg4CoA6LwkBShAAQoYRkCd+NFD4ErQHjx4gEWLFiE0CqjftB4qV6nkkuXDhw9x5sQ5nDh4Ch3adITcj9bb7t27cfTUT6jTqBaq16qG8PBwp5eQsMCVC1dxeO/PqFK+Kjp37uzSeU4b9uIABq68wOOpFPBQQO+BK3u3lZqaim+++UYJiUrQSg/bnTt3lP9GDBgwALGxsXroUpF9YOCqSB6+SQEKUIACFKAABShAAdMLqPNuDFyZfqh5gwEUYOAqgPi8NAUoQAEKmF+AgSvzjzHvkAIUoAAFPBdQJ370EriSO5FqUMeOHcPK1StRt3EtPNGmibK0laO7PHPyHHZt2o0qlaqhW9dulqonZXxSQUoqXV25cgVrv1+DLDxA+25tUbGy4yW17qTdxeY1W3Hv5n0MGTQEVatW9VtVGEdWsp+Bq6J0+B4FfCNgxMCVSFy8eBHLly/HqFGjULZsWd/guNiqVOGaNm0aWrRogebNm/vkOe9iV1w+jIErl6l4IAUoQAEKUIACFKAABUwpoM67MXBlyuHlTelEgIErnQwEu0EBClCAAuYUYODKnOPKu6IABShAAW0E1IkfPQWu1Du7e/cudu7ciWPHjyGqVAQqxJdH8cjiluWtYnA/PQN3Uu/g0rkrKFEsAklJSahXr57fPoDfu3cvduzYgWKRxVAxIQ7FI0qgdLlSeJT7CLdTbuP65RTcT8tAq1at0KFDB/WWdPGTgStdDAM7EWQCRg1cyTBJdb/r16+jT58+CAsLC8jI5ebm4ocfflD6MXbs2ID0wZOLMnDliRrPoQAFKEABClCAAhSggHkE1Hk3Bq7MM6a8E/0JMHClvzFhjyhAAQpQwEQCDFyZaDB5KxSgAAUooLmAOvGjx8CVerOyzOC1a9eQkZGhLDm4efNmJVxVp04dpeJKoJaVkgBASkoKZImrffv24fbt22jXrh0iIiJQqlQpVKpUKWDhBNXO3k8GruypcB8FfCtg5MBVVlYWFi5ciISEBHTp0sW3UA5aP3HiBA4cOKAsbyjPWKNsDFwZZaTYTwpQgAIUoAAFKEABCvhGQJ13Y+DKN75slQIiwMAV/x1QgAIUoAAFfCjAwJUPcdk0BShAAQoYXkCd+NFz4MoWWQJDssTViy++iOLFi9u+7ffXqamp+Otf/4o33ngDcXFxfr++uxdk4MpdMR5PAe8FjBy4Uu/+3XffxdChQyFhV39umZmZkODSxIkTER8f789Le30tBq68JmQDFKAABShAAQpQgAIUMLSAOu/GwJWhh5Gd17kAA1c6HyB2jwIUoAAFjC3AwJWxx4+9pwAFKEAB3wqoEz9GClw9evQI27Ztw7lz5zB8+HCUKFHCt0hFtH7//n3MnTsXHTt29HsIoYhuFfkWA1dF8vBNCvhEwAyBq8uXL2PdunXKczcyMtInTraN3rt3D3PmzEHTpk3RsmVL27d1/5qBK90PETtIAQpQgAIUoAAFKEABnwqo824MXPmUmY0HuQADV0H+D4C3TwEKUIACvhVg4Mq3vmydAhSgAAWMLaBO/BgpcCXieXl5SpWrihUrok2bNgEbhCVLlijLB3br1i1gfXD3wgxcuSvG4yngvYAZAlfy3JWwa3JyMl5++WXvUVxoQQJeskSrhK1CQ0NdOENfhzBwpa/xYG8oQAEKUIACFKAABSjgbwF13o2BK3/L83rBJMDAVTCNNu+VAhSgAAX8LsDAld/JeUEKUIACFDCQgDrxY7TAlRCnpaVh/vz56N27N6pWrepXdQkeSOhg7969GDduHCIiIvx6fW8uxsCVN3o8lwKeCZghcKXe+cyZM5Wl/ZKSknwagjp9+jQ2btyIUaNGISoqSr28oX4ycGWo4WJnKUABClCAAhSgAAUooLmAOu/GwJXmtGyQAvkCDFzlU/AXClCAAhSggPYCDFxpb8oWKUABClDAPALqxI8RA1cyCufPn4d8+P/CCy8gNjbWbwNz8eJFLFu2DGPHjkXp0qX9dl0tLsTAlRaKbIMC7gmYKXCVlZWFefPmoUuXLqhWrZp7EC4eLUvGzpgxA5MmTUJCQoKLZ+nvMAau9Dcm7BEFKEABClCAAhSgAAX8KaDOuzFw5U91XivYBBi4CrYR5/1SgAIUoIBfBRi48is3L0YBClCAAgYTUCd+jBq4Eu4jR47g6NGjGDp0qE+rrahDm5OTowQBOnTogDp16qi7DfOTgSvDDBU7aiIBMwWuZFgk7Dp37lxMmTJF89CpBLpmz56Nfv36IS4uztD/Chi4MvTwsfMUoAAFKEABClCAAhTwWkCdd2PgymtKNkABhwIMXDmk4RsUoAAFKEAB7wUYuPLekC1QgAIUoIB5BdSJHyMHriQAtWLFClSoUAHt27f36WDJtRYuXIgyZcqgR48efgl4aX1DDFxpLcr2KOBcwGyBK7njnTt34sSJExgyZAiio6OdI7h4xMqVK1G8eHHlGeviKbo9jIEr3Q4NO0YBClCAAhSgAAUoQAG/CKjzbgxc+YWbFwlSAQaugnTgedsUoAAFKOAfAQau/OPMq1CAAhSggDEF1IkfIweuRP7+/ft4++23MWHCBMi9+GrbvXs35P9v8cwzz/jqEj5vl4ErnxPzAhQoJGDGwFVeXh7WrFmjBFDbtm1b6J492SEhri1btuDXv/61ErrypA09ncPAlZ5Gg32hAAUoQAEKUIACFKCA/wXUeTcGrvxvzysGjwADV8Ez1rxTClCAAhQIgAADVwFA5yUpQAEKUMAwAurEj9EDVwKekpKC5cuXY+DAgYiNjdV8DG7evImlS5cq1VykwpVRNwaujDpy7LeRBcwYuJLxuHfvHmbNmoWkpCTUq1fPqyG6fPmy8gwfN24cSpYs6VVbejmZgSu9jAT7QQEKUIACFKAABShAgcAIqPNuDFwFxp9XDQ4BBq6CY5x5lxSgAAUoECABBq4CBM/LUoACFKCAIQTUiR8zBK4EfP/+/Th+/DhGjRqlqX96ejo+/PBDJczlywpamnbaQWMMXDmA4W4K+FDArIErIbt+/Tq+/PJLjB8/HgkJCR4rTps2DR07dkTdunU9bkNvJzJwpbcRYX8oQAEKUIACFKAABSjgXwF13o2BK/+682rBJcDAVXCNN++WAhSgAAX8LMDAlZ/BeTkKUIACFDCUgDrxY5bAVXZ2NhYuXIiYmBj06dMHoaGhXo/Hw4cPsXjxYlSoUAFdu3b1ur1AN8DAVaBHgNcPRgEzB65kPE+dOoW9e/di2LBhKFasmFtDnJOTgyVLliA8PBz9+/fX5LntVgd8eDADVz7EZdMUoAAFKEABClCAAhQwgIA678bAlQEGi100rAADV4YdOnacAhSgAAWMIMDAlRFGiX2kAAUoQIFACagTP2YJXIljZmYm5s+fj+bNm0MmtLzdkpOTcfHiRSXAFRYW5m1zAT+fgauADwE7EIQCZg9c5ebmYvXq1ShRogR69Ojh8gjn5eVhz549OHHiBEaMGKGErlw+2QAHMnBlgEFiFylAAQpQgAIUoAAFKOBDAXXejYErHyKz6aAXYOAq6P8JEIACFKAABXwpwMCVL3XZNgUoQAEKGF1AnfgxU+BKxkSWuJKqVOPGjUPJkiU9HqYLFy5g0aJFGDt2LOLi4jxuR08nMnClp9FgX4JFwOyBKxlHqQb45z//Gf369UPTpk1dGtqMjAx89dVXeOaZZxAVFeXSOUY6iIErI40W+0oBClCAAhSgAAUoQAHtBdR5NwautLdlixRQBRi4UiX4kwIUoAAFKOADgYKBq7dQv359H1yFTVKAAhSgAAWMKaBO/JgtcCVVUw4cOIDt27fj2WefRWRkpNsDdPPmTUyfPh2DBg1CzZo13T5frycwcKXXkWG/zCwQDIErGb87d+5gwYIFGDBgAMqXL1/kkKalpWHWrFno0qWLJtUIi7xYgN5k4CpA8LwsBShAAQpQgAIUoAAFdCKgzrsxcKWTAWE3TCnAwJUph5U3RQEKUIACehFg4EovI8F+UIACFKCAHgXUiR+zBa5U65UrVyq/9u3bFyEhIepul34uXLhQCWo3aNDApeONchADV0YZKfbTTALBEriSMTty5Ah27NiByZMnOxzCnJwcpXpghQoVlMCVu89nhw3r7A0GrnQ2IOwOBShAAQpQgAIUoAAF/CygzrsxcOVneF4uqAQYuAqq4ebNUoACFKCAvwUYuPK3OK9HAQpQgAJGElAnfswauMrOzsa8efPQsmVL1K1b1+Wh2bRpE1JSUpQqLcWLF3f5PCMcyMCVEUaJfTSbQDAFrnJzc7F06VKEh4ejd+/eyk/b8Tx48CBOnTqFoUOH2r5lqtcMXJlqOHkzFKAABShAAQpQgAIUcFtAnXdj4MptOp5AAZcFGLhymYoHUoACFKAABdwXYODKfTOeQQEKUIACwSOgTvyYNXAlI3nt2jV89tlneOmll1CuXDmng3vy5El8/fXXePPNNz1aitDpBQJ8AANXAR4AXj4oBYIpcCUD/PDhQ8yePRvNmzdHw4YNC4y5PINWr16N0aNHIzY2tsB7ZnvBwJXZRpT3QwEKUIACFKAABShAAfcE1Hk3Bq7cc+PRFHBHgIErd7R4LAUoQAEKUMBNAQau3ATj4RSgAAUoEFQC6sSPmQNXMqDJyck4cOAARo4cWWSI6s6dO8oyV4MGDULZsmVN+W+BgStTDitvSucCwRa4kuG4desW5syZg7Fjx6JUqVLKCMkzdtasWejVqxcSExN1Pmred4+BK+8N2QIFKEABClCAAhSgAAWMLKDOuzFwZeRRZN/1LsDAld5HiP2jAAUoQAFDCzBwZejhY+cpQAEKUMDHAurEj9kDV8K4fv165OXl4cknn3So+s0330AsWrVq5fAYo7/BwJXRR5D9N6JAMAau5Hl75MgR/PDDD3j22WcRExODJUuWID4+Hq1btzbiMLrdZwau3CbjCRSgAAUoQAEKUIACFDCVgDrvxsCVqYaVN6MzAQaudDYg7A4FKEABCphLgIErc40n74YCFKAABbQVUCd+giFwlZmZic8//xzdunVDgwYNEBISko/56NEjbNmyBfL/b3jxxRfz95vxFwauzDiqvCe9CwRj4Eodk40bN0IqW8XFxUGeP7KUYHh4uPq2qX8ycGXq4eXNUYACFKAABShAAQpQwKmAOu/GwJVTKh5AAY8FGLjymI4nUoACFKAABZwLMHDl3IhHUIACFKBA8AqoEz/BELiSUb5y5QqWLVuGfv36oUqVKvkDf/LkSezYsQPDhw9HRERE/n4z/sLAlRlHlfekd4FgDFylp6fjwoULSElJwebNm3Ht2jX07dsXCQkJyvO3XLlyCA0N1fvQedU/Bq684uPJFKAABShAAQpQgAIUMLyAOu/GwJXhh5I3oGMBBq50PDjsGgUoQAEKGF+AgSvjjyHvgAIUoAAFfCegTvwES+BKJCVwJBVXnnvuuXzY9957D2PHjlWWusrfadJfGLgy6cDytnQtEEyBKwlYfffddzh37pxSTbBixYqoVKmSsqRrWloarl+/jsOHD6N06dIYOXIkJHhl1o2BK7OOLO+LAhSgAAUoQAEKUIACrgmo824MXLnmxaMo4IkAA1eeqPEcClCAAhSggIsCDFy5CMXDKEABClAgKAXUiZ9gClzJ8oFr1qyB/OzYsSMWL16MxMREdOrUKSj+DTBwFRTDzJvUmUAwBK5k2cDk5GQl1Cr/TWnRogWKFy9udyRyc3MhlQVXr16tPH+TkpKUAJbdgw28k4ErAw8eu04BClCAAhSgAAUoQAENBNR5NwauNMBkExRwIMDAlQMY7qYABShAAQpoIcDAlRaKbIMCFKAABcwqoE78BFPgSsYyKysLM2fOxO3bt9GkSRN07doVYWFhZh3mAvfFwFUBDr6ggF8EzB64kmfp/Pnz0bBhQzRq1AgxMTEuucqzeP/+/dizZw969OgB+W+RmTYGrsw0mrwXClCAAhSgAAUoQAEKuC+gzrsxcOW+Hc+ggKsCDFy5KsXjKEABClCAAh4IMHDlARpPoQAFKECBoBFQJ36CIXB169YtpKamYu/evbh37x7u37+PzMxMpQJL2bJlIcte1a5dG1WqVHE5LGDEfygMXBlx1NhnowuYOXAlz1EJFvXv318JXHkyVrLU4DfffIOhQ4ciISHBkyZ0eQ4DV7ocFnaKAhSgAAUoQAEKUIACfhNQ590YuPIbOS8UhAIMXAXhoPOWKUABClDAfwIMXPnPmleiAAUoQAHjCagTP2YOXGVkZGDLli24fPkyatSooQSqYmNjERkZqYStJCyQnZ2NGzdu4MKFC7h48aJyXLt27RAREWG8QXXSYwaunADxbQr4QMCsgSt5vsqyrPJsbd++vVdyx48fx7Zt2zBq1ChERUV51ZZeTmbgSi8jwX5QgAIUoAAFKEABClAgMALqvBsDV4Hx51WDQ4CBq+AYZ94lBShAAQoESICBqwDB87IUoAAFKGAIAXXix4yBq7y8PKWS1RdffIH69eujd+/eLo2JVL5asmQJcnNzMWTIEERHR7t0nlEOYuDKKCPFfppJwKyBq8OHD0OCUsOGDdNkuFavXq1UH9SqPU065UUjDFx5gcdTKUABClCAAhSgAAUoYAIBdd6NgSsTDCZvQbcCDFzpdmjYMQpQgAIUMIMAA1dmGEXeAwUoQAEK+EpAnfgxY+DqyJEjSmWrpKQkyP2FhIS4xbhz504kJycry2SZaYkrBq7c+mfAgymgiYBZA1fTpk1TnpEVKlTQxEmqDX788cdKgCs+Pl6TNgPZCANXgdTntSlAAQpQgAIUoAAFKBB4AXXejYGrwI8Fe2BeAQauzDu2vDMKUIACFNCBAANXOhgEdoECFKAABXQroE78mC1wdevWLcyYMQNTpkxBiRIl3A5byYBJhayzZ88q1a4mT56MmJgY3Y6jOx1j4ModLR5LAW0EzBi4OnXqFDZt2oRJkyYhNDRUGyhLKwcPHsTNmzfRtWtXzdoMVEMMXAVKntelAAUoQAEKUIACFKCAPgTUeTcGrvQxHuyFOQUYuDLnuPKuKEABClBAJwIMXOlkINgNClCAAhTQpYA68WOmwJUsBThnzhzIZFbTpk29dt+8eTPS09PRt29fr9vSQwMMXOlhFNiHYBMwY+Bq+vTpaN68ORo2bKjpcEpgVp67gwYN0rTdQDTGwFUg1HlNClCAAhSgAAUoQAEK6EdAnXdj4Eo/Y8KemE+AgSvzjSnviAIUoAAFdCTAwJWOBoNdoQAFKEAB3QmoEz9mClzt2LED586dw9ChQxEeHu61+YMHDzBz5kw8/fTTKF26tNftBboBBq4CPQK8fjAKmC1wJcHWN998E7///e+VKoJajmlmZiaWLl2KESNGeFSdUMu+eNsWA1feCvJ8ClCAAhSgAAUoQAEKGFtAnXdj4MrY48je61uAgSt9jw97RwEKUIACBhdg4MrgA8juU4ACFKCATwXUiR+zBK5ycnIwdepUjBkzBnFxcZrZHTp0CBs2bMCrr75q+AAAA1ea/bNgQxRwWcBsgau7d+9CwkRvv/22ywauHijP8blz52L06NGGf94ycOXqqPM4ClCAAhSgAAUoQAEKmFNAnXdj4Mqc48u70ocAA1f6GAf2ggIUoAAFTCrAwJVJB5a3RQEKUIACmgioEz9mCVydP38emzZtwvjx4zXxURt5+PAh/vznP+OVV14xfJUrBq7UUeVPCvhPwGyBq+vXr2P+/Pl46aWXNEd89OgRvv32WyU4GxISonn7/myQgSt/avNaFKAABShAAQpQgAIU0J+AOu/GwJX+xoY9Mo8AA1fmGUveCQUoQAEK6FCAgSsdDgq7RAEKUIACuhFQJ37MErjatWsXQkND0bJlS82NlyxZgtq1a6Nhw4aat+3PBhm48qc2r0WBfwuYLXAlFa4++ugj/Pa3v9V8iCXgOnv2bCU4y8CV5rxskAIUoAAFKEABClCAAhTwo4A678bAlR/ReamgE2DgKuiGnDdMAQpQgAL+FGDgyp/avBYFKEABChhNQJ34MUvgat26dahbty6qV6+u+VAcOHAA9+7dQ6dOnTRv258NMnDlT21eiwL/FjBb4EpCUW+++aaypGDx4sU1HebMzEwsXrwYo0aN4pKCmsqyMQpQgAIUoAAFKEABClDA3wLqvBsDV/6W5/WCSYCBq2Aabd4rBShAAQr4XYCBK7+T84IUoAAFKGAgAXXixyyBq7Vr16Jx48aoXLmy5qNw5swZnDx5Ej179tS8bX82yMCVP7V5LQr8W8BsgSu5q6lTp6Jz586aV/1LS0vDmjVrMHz4cAau+H9AFKAABShAAQpQgAIUoIChBdR5NwauDD2M7LzOBRi40vkAsXsUoAAFKGBsAQaujD1+7D0FKEABCvhWQJ34MUvgatmyZcpygvHx8ZrDnTt3DkePHkXv3r01b9ufDTJw5U9tXosC/xYwY+Bq//79OHbsmOaVqA4dOoQrV66gV69ehv/n8/e//10xql+/Pv77zbcMfz+8AQpQgAIUoAAFKEABClDAPQF13o2BK/fceDQF3BFg4ModLR5LAQpQgAIUcFOAgSs3wXg4BShAAQoElYA68WOWwJVUuJJ78cWSgrt27UJ2djY6dOhg6H8jDFwZevjYeYMKmDFwlZOTg08++QRjxoxB2bJlNRsZaVMqCdasWVOzNgPVEANXgZLndSlAAQpQgAIUoAAFKKAPAXXejYErfYwHe2FOAQauzDmuvCsKUIACFNCJAANXOhkIdoMCFKAABXQpoE78mCVwtXPnThQrVgwtWrTQ3HvJkiVKkKtZs2aat+3PBhm48qc2r0WBfwuYMXAld7Z7927cu3cP3bp102Sot27dCqmc9eqrr2rSXqAbYeAq0CPA61OAAhSgAAUoQAEKUCCwAuq8GwNXgR0HXt3cAgxcmXt8eXcUoAAFKBBgAQauAjwAvDwFKEABCuhaQJ34MUvg6vTp09iyZQsmTpyoqfuDBw/w7rvv4pVXXkFsbKymbfu7MQau/C3O61EAluVIf8Yf3n5bofh//+83qFUr0RQs6enpmDdvHpKSkryuSHX58mVIsHXSpEmIjIw0hQ8DV6YYRt4EBShAAQpQgAIUoAAFPBZQ590YuPKYkCdSwKkAA1dOiXgABShAAQpQwHMBBq48t+OZFKAABShgfgF14scsgStZ8k8+4JZgVEREhGYDeOzYMWzbtg3PPfecZm0GqiEGrgIlz+sGs4BZA1cypjdu3MBnn32G0aNHexy6yszMxDfffIOuXbuiTp06pvmnwsCVaYaSN0IBClCAAhSgAAUoQAGPBNR5NwauPOLjSRRwSYCBK5eYeBAFKEABClDAMwEGrjxz41kUoAAFKBAcAurEj1kCVzJqmzdvxrVr1zB48GCEh4d7PZBS3UqCAE8++aTHYQKvO6FhAwxcaYjJpijgooCZA1dCcPXqVaxatQotW7ZEgwYNlKVdXaHJy/v/7d07i2VlFsfhraCiCAaKijcQbJHWSCNRMTLRwEQdMyNjP4H2BIZ+AWPDnsRAvEAHfgNRZgo1EBFEEPESeKN1Zk4xu6iymKZ1n7Xes8//EYaq6q5z1n6ftRlmtj+q/j19+umn0zvvvDNtfl3rww8/PF155ZWX89JVfI/gahVrcpEECBAgQIAAAQIEygTm526CqzJib0xgEly5CQgQIECAQKGA4KoQ11sTIECAwOoF5gc/+xRc/fbbb9Mbb7wxPfLII9OZM2cW7+iDDz6YNr/q6qmnnlr8XrvwBoKrXdiCa0gT2PfgahNOff3119Pbb789bf47+Mknn5xuvfXWS675m2++md59991pE7U+8cQT05133nnJ71/jXwqu1rg110yAAAECBAgQIEBgewLzczfB1fZMvROBPwoIrv4o4msCBAgQILBFAcHVFjG9FQECBAjsncD84GefgqvNkja/4mrz01aefvrp6cYbb/xLe9sEBF988cV0/vz56YUXXphuuummv/Q+u/YiwdWubcT1JAjse3B1fIeb///15ptvTjfffPPhT7u6++67D3/i1e+//z5tfu3rJrT68MMPp88++2x69NFHp8cee2wrP43w+DXsyueCq13ZhOsgQIAAAQIECBAgMEZgfu4muBrjb2qGgOAqY89OSYAAAQKDBARXg+CNJUCAAIFVCMwPfvYtuNr8i/2PP/54euutt6bnn39+uuOOO/70Pg4ODg5/+som2toEA/vyj+BqXzbpHGsSSAquNnu5ePHi4U8G/PLLL6cffvjh8Otffvlluvbaaw//c9ddd0233377dM0116xpjX/6WgVXf5rMCwgQIECAAAECBAjslcD83E1wtVdrdZgdExBc7dhCXA4BAgQI7JeA4Gq/9uk0BAgQILBdgfnBz74FV7PS5l/2v/7669Nzzz13+OsFr7rqqumKK66Y//rUx81PtdpEAe+99970ySefTC+++OJ0ww03nPq+Nf+B4GrN23PtaxVIC67WuqdtX7fgatui3o8AAQIECBAgQIDAugTm526Cq3XtzdWuS0Bwta59uVoCBAgQWJmA4GplC3O5BAgQINAqMD/42dfgaoO5+fVV77///vTjjz9Ot9xyy3Tfffcdfrz66quPrH/66adpE2d99NFHh99/2223Hf6aq+uvv/7oe/blE8HVvmzSOdYkILha07a2d62Cq+1ZeicCBAgQIECAAAECaxSYn7sJrta4Pde8FgHB1Vo25ToJECBAYJUCgqtVrs1FEyBAgECTwPzgZ5+Dqw3l5idXbaKqzf8uuHDhwvTVV18d/uSqza+3+vbbb6eff/75MMR6/PHHp3vuuWfaxFiX+klYTespGSO4KmH1pgQuKSC4uiTP3v6l4GpvV+tgBAgQIECAAAECBC5LYH7uJri6LC7fROAvCQiu/hKbFxEgQIAAgcsTOBlcnZvOnr3/8l7ouwgQIECAQIDA/OBn34OrP67y4sWL03fffXf46wM3P8Xquuuumza/bjDhH8FVwpadcdcEBFe7tpGe6xFc9TibQoAAAQIECBAgQGBXBebnboKrXd2Q69oHAcHVPmzRGQgQIEBgZwUEVzu7GhdGgAABAjsgMD/4SQuudoB+2CUIrobRGxwsILjKXL7gKnPvTk2AAAECBAgQIEBgFpifuwmuZhEfCWxfQHC1fVPvSIAAAQIEjgQEV0cUPiFAgAABAqcE5gc/gqtTNHv7B4KrvV2tg+2wgOBqh5dTeGmCq0Jcb02AAAECBAgQIEBgBQLzczfB1QqW5RJXKyC4Wu3qXDgBAgQIrEFAcLWGLblGAgQIEBglMD/4EVyN2kD/XMFVv7mJBARXmfeA4Cpz705NgAABAgQIECBAYBaYn7sJrmYRHwlsX0BwtX1T70iAAAECBI4EBFdHFD4hQIAAAQKnBOYHP4KrUzR7+weCq71drYPtsIDgaoeXU3hpgqtCXG9NgAABAgQIECBAYAUC83M3wdUKluUSVysguFrt6lw4AQIECKxBQHC1hi25RgIECBAYJTA/+BFcjdpA/1zBVb+5iQQEV5n3gOAqc+9OTYAAAQIECBAgQGAWmJ+7Ca5mER8JbF9AcLV9U+9IgAABAgSOBARXRxQ+IUCAAAECpwTmBz+Cq1M0e/sHgqu9Xa2D7bCA4GqHl1N4aYKrQlxvTYAAAQIECBAgQGAFAvNzN8HVCpblElcrILha7epcOAECBAisQUBwtYYtuUYCBAgQGCUwP/gRXI3aQP9cwVW/uYkEBFeZ94DgKnPvTk2AAAECBAgQIEBgFpifuwmuZhEfCWxfQHC1fVPvSIAAAQIEjgQEV0cUPiFAgAABAqcE5gc/gqtTNHv7B4KrvV2tg+2wgOBqh5dTeGmCq0Jcb02AAAECBAgQIEBgBQLzczfB1QqW5RJXKyC4Wu3qXDgBAgQIrEFAcLWGLblGAgQIEBglMD/4EVyN2kD/XMFVv7mJBARXmfeA4Cpz705NgAABAgQIECBAYBaYn7sJrmYRHwlsX0BwtX1T70iAAAECBI4EBFdHFD4hQIAAAQKnBOYHP4KrUzR7+weCq71drYPtsIDgaoeXU3hpgqtCXG9NgAABAgQIECBAYAUC83M3wdUKluUSVysguFrt6lw4AQIECKxBQHC1hi25RgIECBAYJTA/+BFcjdpA/1zBVb+5iQT+GFydOXMPlACB1157bTo4OJjOnj07vXLu7wEndkQCBAgQIECAAAECBI4LzM/dBFfHVXxOYLsCgqvteno3AgQIECBwQkBwdYLDFwQIECBA4ITA/OBHcHWCZa+/EFzt9XodbkcFjgdXO3qJLqtQQHBViOutCRAgQIAAAQIECOywwPzcTXC1w0tyaasXEFytfoUOQIAAAQK7LCC42uXtuDYCBAgQGC0wP/gRXI3eRN98wVWftUkEjgs8/7fnjn/p8yCBZ555dnrm2WeDTuyoBAgQIECAAAECBAhsBObnboIr9wOBOgHBVZ2tdyZAgAABApPgyk1AgAABAgT+v8D84Edw9f+N9u1vBFf7tlHnWZPAP86fX9PlutYtCJy9/+x/f6Xg/Vt4J29BgAABAgQIECBAgMDaBObnboKrtW3O9a5JQHC1pm25VgIECBBYnYDganUrc8EECBAg0CgwP/jZjNxEV/7JEDg4ODg86CvnzgkBMlbulAQIECBAgAABAgQIECBAgECzwPzcTXDVDG9clIDgKmrdDkuAAAEC3QKCq25x8wgQIEBgTQLzg581XbNr3Z6A4Gp7lt6JAAECBAgQIECAAAECBAgQIHBcYH7uJrg6ruJzAtsVEFxt19O7ESBAgACBEwKCqxMcviBAgAABAicE/vWvf05+xdUJkqgvXjn396jzOiwBAgQIECBAgAABAgQIECBAoEtAcNUlbU6ygOAqefvOToAAAQLlAoKrcmIDCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEjgkIJWut9gAAH+xJREFUro5h+JRAkYDgqgjW2xIgQIAAgY2A4Mp9QIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAg0CkguOrUNitVQHCVunnnJkCAAIEWAcFVC7MhBAgQIECAAAECBAgQIECAAAECBAgQIECAAAEC/xMQXLkVCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsIDgKnj5jk6AAAECBAgQIECAAAECBAgQIECAAAECBAgQGCAguBqAbmScgOAqbuUOTIAAAQKdAoKrTm2zCBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEBFfuAQL1AoKremMTCBAgQCBYQHAVvHxHJ0CAAAECBAgQIECAAAECBAgQIECAAAECBAgMEBBcDUA3Mk5AcBW3cgcmQIAAgU4BwVWntlkECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQKCK/cAgXoBwVW9sQkECBAgECwguApevqMTIECAAAECBAgQIECAAAECBAgQIECAAAECBAYICK4GoBsZJyC4ilu5AxMgQIBAp4DgqlPbLAIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAcGVe4BAvYDgqt7YBAIECBAIFhBcBS/f0QkQIECAAAECBAgQIECAAAECBAgQIECAAAECAwQEVwPQjYwTEFzFrdyBCRAgQKBTQHDVqW0WAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQICA4Mo9QKBeQHBVb2wCAQIECAQLCK6Cl+/oBAgQIECAAAECBAgQIECAAAECBAgQIECAAIEBAoKrAehGxgkIruJW7sAECBAg0CkguOrUNosAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQEBw5R4gUC8guKo3NoEAAQIEggUEV8HLd3QCBAgQIECAAAECBAgQIECAAAECBAgQIECAwAABwdUAdCPjBARXcSt3YAIECBDoFBBcdWqbRYAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgILhyDxCoFxBc1RubQIAAAQLBAoKr4OU7OgECBAgQIECAAAECBAgQIECAAAECBAgQIEBggIDgagC6kXECgqu4lTswAQIECHQKCK46tc0iQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAQXLkHCNQLCK7qjU0gQIAAgWABwVXw8h2dAAECBAgQIECAAAECBAgQIECAAAECBAgQIDBAQHA1AN3IOAHBVdzKHZgAAQIEOgUEV53aZhEgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECAiu3AME6gUEV/XGJhAgQIBAsMDx4CqYwdEJECBAgAABAgQIECBAgAABAgQIECBAgAABAgSaBR544IHp1VdfbZ5qHIEMAcFVxp6dkgABAgQGCXz++efTSy+9NGi6sQQIECBAgAABAgQIECBAgAABAgQIECBAgAABAqkCDz300PTyyy+nHt+5CZQKCK5Keb05AQIECKQL/Prrr9OFCxem77//Pp3C+QkQIECAAAECBAgQIECAAAECBAgQIECAAAECBBoF7r333unBBx9snGgUgRwBwVXOrp2UAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAIGFAoKrhYBeToAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAjoDgKmfXTkqAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAwEIBwdVCQC8nQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQCBHQHCVs2snJUCAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIEBgoYDgaiGglxMgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgkCMguMrZtZMSIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQILBQQHC1ENDLCRAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBDIERBc5ezaSQkQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQWCgguFoI6OUECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECOQICK5ydu2kBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgsFBBcLQT0cgIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEcgQEVzm7dlICBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBBYKCK4WAno5AQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQI5AoKrnF07KQECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECCwUEVwsBvZwAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAgRwBwVXOrp2UAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAIGFAoKrhYBeToAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAjoDgKmfXTkqAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAwEIBwdVCQC8nQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQCBHQHCVs2snJUCAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIEBgoYDgaiGglxMgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgkCMguMrZtZMSIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQILBQQHC1ENDLCRAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBDIERBc5ezaSQkQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQWCgguFoI6OUECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECOQICK5ydu2kBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgsFBBcLQT0cgIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIEcgQEVzm7dlICBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBBYKCK4WAno5AQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQI5AoKrnF07KQECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECCwUEVwsBvZwAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAgRwBwVXOrp2UAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAIGFAoKrhYBeToAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIBAjoDgKmfXTkqAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAwEIBwdVCQC8nQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQCBHQHCVs2snJUCAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIEBgoYDgaiGglxMgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgkCMguMrZtZMSIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQILBQQHC1ENDLCRAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBDIERBc5ezaSQkQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQIECAAAECBAgQWCgguFoI6OUECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECBAgQIAAAQIECOQI/Ae3+lpSUnb2ggAAAABJRU5ErkJggg==)" + ] + }, + { + "cell_type": "markdown", + "id": "mXtb21lMj_rU", + "metadata": { + "id": "mXtb21lMj_rU" + }, + "source": [ + "At a high level, the pipeline has the following steps:\n", + "\n", + "1. Read the review data from Pub/Sub, the event messaging source. This data contains the chat ID and the chat history as a JSON payload. This payload is processed in the pipeline.\n", + "1. Pass the text from the messages to Gemma with a prompt.\n", + "1. The pipeline requests that the model complete the following two tasks:\n", + " * Attach a sentiment score to the message, by using one of the following three values: `1` for a positive chat, `0` for a neutral chat, and `-1` for a negative chat.\n", + " * Provide a one-sentence summary of the chat.\n", + "1. The pipeline branches, depending on the sentiment score:\n", + " * If the score is `1` or `0`, the chat and its summarization are sent to a data analytics system for storage and future analysis.\n", + " * If the score is `-1`, the Gemma model drafts a response. This response and the chat information are sent to an event messaging system that connects the pipeline and other applications. This step allows a person to review the content of the response. " + ] + }, + { + "cell_type": "markdown", + "id": "nETbaxwZk7us", + "metadata": { + "id": "nETbaxwZk7us" + }, + "source": [ + "## Build the pipeline\n", + "\n", + "This section provides the code needed to run the pipeline.\n", + "\n", + "### Before you begin\n", + "\n", + "Although you can use CPUs for testing and development, for a production Dataflow ML system we recommend that you use GPUs. When you use GPUs with Dataflow, we recommend that you use a custom container. For more information about configuring GPUs and custom containers with Dataflow, see [Best practices for working with Dataflow GPUs](https://cloud.google.com/dataflow/docs/gpu/develop-with-gpus). To faciliate rapid testing of the pipeline, follow the guide [Run a pipeline with GPUs](https://cloud.google.com/dataflow/docs/gpu/use-gpus). The guide includes the steps needed to build the container image.\n", + "\n", + "After you configure your environment, download the model [gemma2_instruct_2b_en](https://www.kaggle.com/models/google/gemma-2/keras) into a folder. In this example, the folder is named `gemma2`." + ] + }, + { + "cell_type": "markdown", + "id": "jMrjYGW9spFG", + "metadata": { + "id": "jMrjYGW9spFG" + }, + "source": [ + "### Build the base image\n", + "\n", + "Add the following Dockerfile to your folder, and then build the base image. Use the Dockerfile to build the image as you create the `pipeline.py` file. The images are broken into two groups to facilitate testing and development." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "29dOdG_kxzTv", + "metadata": { + "id": "29dOdG_kxzTv" + }, + "outputs": [], + "source": [ + "ARG SERVING_BUILD_IMAGE=tensorflow/tensorflow:2.16.1-gpu\n", + "\n", + "FROM ${SERVING_BUILD_IMAGE}\n", + "WORKDIR /workspace\n", + "\n", + "COPY gemma2 gemma2\n", + "RUN apt-get update -y && apt-get install -y cmake && apt-get install -y vim" + ] + }, + { + "cell_type": "markdown", + "id": "3eWt8AatpEuG", + "metadata": { + "id": "3eWt8AatpEuG" + }, + "source": [ + "When testing the pipeline code and when launchig the job on Dataflow, test and launch from inside the container. This step prevents dependency mismatches when running the pipeline on Dataflow." + ] + }, + { + "cell_type": "markdown", + "id": "lyS0uYpsoeOW", + "metadata": { + "id": "lyS0uYpsoeOW" + }, + "source": [ + "The `requirements.txt` file contains the following dependencies: " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "K4gJQ0e9pCR4", + "metadata": { + "id": "K4gJQ0e9pCR4" + }, + "outputs": [], + "source": [ + "apache_beam[gcp]==2.54.0\n", + "keras_nlp==0.14.3\n", + "keras==3.4.1\n", + "jax[cuda12]" + ] + }, + { + "cell_type": "markdown", + "id": "k9gGtkqvn6Ar", + "metadata": { + "id": "k9gGtkqvn6Ar" + }, + "source": [ + "The next step includes the files needed to construct the pipeine. The content of the `pipeline.py` file are contained in a later section of this notebook." + ] + }, + { + "cell_type": "markdown", + "id": "aqPS_p3Pp37b", + "metadata": { + "id": "aqPS_p3Pp37b" + }, + "source": [ + "Replace DOCKERFILE_IMAGE with the image that you built using the first Dockerfile." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "EdUxmUjqx58r", + "metadata": { + "id": "EdUxmUjqx58r" + }, + "outputs": [], + "source": [ + "FROM \n", + "\n", + "WORKDIR /workspace\n", + "\n", + "# Copy files from the official SDK image, including the script and dependencies.\n", + "COPY --from=apache/beam_python3.11_sdk:2.54.0 /opt/apache/beam /opt/apache/beam\n", + "\n", + "\n", + "COPY requirements.txt requirements.txt\n", + "RUN pip install --upgrade --no-cache-dir pip \\\n", + " && pip install --no-cache-dir -r requirements.txt\n", + "\n", + "# Copy the model directory downloaded from Kaggle and the pipeline code.\n", + "COPY pipeline.py pipeline.py\n", + "\n", + "# The colab was tested and run with a JAX backend to let Dataflow workers\n", + "# pick up the environment needed to include in the Env of the image.\n", + "ENV KERAS_BACKEND=\"jax\"\n", + "ENV XLA_PYTHON_CLIENT_MEM_FRACTION=\"0.9\"\n", + "\n", + "\n", + "# Set the entrypoint to the Apache Beam SDK launcher.\n", + "ENTRYPOINT [\"/opt/apache/beam/boot\"]" + ] + }, + { + "cell_type": "markdown", + "id": "i63FUxXwsSsO", + "metadata": { + "id": "i63FUxXwsSsO" + }, + "source": [ + "### Run the pipeline\n", + "\n", + "The following code creates and runs the pipeline.\n", + "\n", + "- The `pip install` steps are needed to run the code in the notebook, but aren't needed when running the code in your container.\n", + "\n", + "- Without a GPU, the inference takes a long time to complete." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "ebb44382-ee7b-4cec-af67-1fe220cfb40d", + "metadata": { + "id": "ebb44382-ee7b-4cec-af67-1fe220cfb40d", + "tags": [] + }, + "outputs": [], + "source": [ + "pip install apache_beam[gcp]==\"2.54.0\" keras_nlp==\"0.14.3\" keras>=\"3\" jax[cuda12]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "oPgRBScKThZg", + "metadata": { + "id": "oPgRBScKThZg" + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "os.environ[\"KERAS_BACKEND\"] = \"jax\" # Or \"tensorflow\" or \"torch\".\n", + "os.environ[\"XLA_PYTHON_CLIENT_MEM_FRACTION\"] = \"0.9\"\n", + "\n", + "import keras\n", + "import keras_nlp\n", + "import numpy as np\n", + "import json\n", + "import ast\n", + "import re\n", + "import logging\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference import utils\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.options import pipeline_options\n", + "from apache_beam.options.pipeline_options import GoogleCloudOptions\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.options.pipeline_options import SetupOptions\n", + "from apache_beam.options.pipeline_options import StandardOptions\n", + "from apache_beam.options.pipeline_options import WorkerOptions\n", + "from apache_beam.ml.inference import utils\n", + "from apache_beam.ml.inference.base import ModelHandler\n", + "from apache_beam.ml.inference.base import PredictionResult\n", + "from apache_beam.ml.inference.base import KeyedModelHandler\n", + "from keras_nlp.models import GemmaCausalLM\n", + "from typing import Any, Dict, Iterable, Optional, Sequence" + ] + }, + { + "cell_type": "markdown", + "id": "0gicDesYWdbu", + "metadata": { + "id": "0gicDesYWdbu" + }, + "source": [ + "Set pipeline options and provide the input Pub/Sub topic. The options that are commented out enable running the pipeline on Google Cloud Dataflow." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "wpG-ltLiTxWM", + "metadata": { + "id": "wpG-ltLiTxWM" + }, + "outputs": [], + "source": [ + "options = PipelineOptions()\n", + "options.view_as(StandardOptions).streaming = True\n", + "options.view_as(SetupOptions).save_main_session = True\n", + "\n", + "# options.view_as(StandardOptions).runner = \"dataflowrunner\"\n", + "# options.view_as(GoogleCloudOptions).project = \n", + "# options.view_as(GoogleCloudOptions).temp_location= \n", + "# options.view_as(GoogleCloudOptions).region= \"us-west1\"\n", + "# options.view_as(WorkerOptions).machine_type= \"g2-standard-4\"\n", + "# options.view_as(WorkerOptions).worker_harness_container_image = \n", + "# options.view_as(WorkerOptions).disk_size_gb=200\n", + "# options.view_as(GoogleCloudOptions).dataflow_service_options=[\"worker_accelerator=type:nvidia-l4;count:1;install-nvidia-driver\"]\n", + "\n", + "topic_reviews=\"\"" + ] + }, + { + "cell_type": "markdown", + "id": "g8sWSMRmW-Ab", + "metadata": { + "id": "g8sWSMRmW-Ab" + }, + "source": [ + "Define a custom model handler that will load the Gemma model and handle inference calls." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "vRVCN3qBUAt9", + "metadata": { + "id": "vRVCN3qBUAt9" + }, + "outputs": [], + "source": [ + "class GemmaModelHandler(ModelHandler[str,\n", + " PredictionResult,GemmaCausalLM\n", + " ]):\n", + " def __init__(\n", + " self,\n", + " model_name: str = \"\",\n", + " ):\n", + " \"\"\" Implementation of the ModelHandler interface for Gemma using text as input.\n", + "\n", + " Example Usage::\n", + "\n", + " pcoll | RunInference(GemmaModelHandler())\n", + "\n", + " Args:\n", + " model_name: The Gemma model uri.\n", + " \"\"\"\n", + " self._model_name = model_name\n", + " self._env_vars = {}\n", + " def share_model_across_processes(self) -> bool:\n", + " \"\"\"Returns whether to share a single model in memory across processes.\n", + "\n", + " This is useful when the loaded model is large, preventing potential\n", + " out-of-memory issues when running the pipeline.\n", + "\n", + " Returns:\n", + " bool\n", + " \"\"\"\n", + " return True\n", + "\n", + " def load_model(self) -> GemmaCausalLM:\n", + " \"\"\"Loads and initializes a model for processing.\"\"\"\n", + " return keras_nlp.models.GemmaCausalLM.from_preset(self._model_name)\n", + "\n", + " def run_inference(\n", + " self,\n", + " batch: Sequence[str],\n", + " model: GemmaCausalLM,\n", + " inference_args: Optional[Dict[str, Any]] = None\n", + " ) -> Iterable[PredictionResult]:\n", + " \"\"\"Runs inferences on a batch of text strings.\n", + "\n", + " Args:\n", + " batch: A sequence of examples as text strings.\n", + " model:\n", + " inference_args: Any additional arguments for an inference.\n", + "\n", + " Returns:\n", + " An Iterable of type PredictionResult.\n", + " \"\"\"\n", + " # Loop each text string, and use a tuple to store the inference results.\n", + " predictions = []\n", + " for one_text in batch:\n", + " result = model.generate(one_text, max_length=1024)\n", + " predictions.append(result)\n", + " return utils._convert_to_result(batch, predictions, self._model_name)" + ] + }, + { + "cell_type": "markdown", + "id": "cSbAFPXmXPMc", + "metadata": { + "id": "cSbAFPXmXPMc" + }, + "source": [ + "We definte a prompt template to format a given input as well as instruct the model on the task being asked of it. This block also has an example input to the model." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "hqh-Ro5-UNqy", + "metadata": { + "id": "hqh-Ro5-UNqy" + }, + "outputs": [], + "source": [ + "prompt_template = \"\"\"\n", + "\n", + "Provide the results of doing these two tasks on the chat history provided below for the user {}\n", + "task 1 : assess if the tone is happy = 1 , neutral = 0 or unhappy = -1\n", + "task 2 : summarize the text with a maximum of 512 characters\n", + "Return the answer as a JSON string with fields [sentiment, summary] do NOT explain your answer\n", + "\n", + "@@@{}@@@\n", + "\n", + "\"\"\"\n", + "chat_text = \"\"\"\n", + "id 221: Hay I am really annoyed that your menu includes a pizza with pineapple on it!\n", + "id 331: Sorry to hear that , but pineapple is nice on pizza\n", + "id 221: What a terriable thing to say! Its never ok, so unhappy right now!\n", + "\"\"\"\n", + "\n", + "# Example input\n", + "chat =json.dumps({\"id\" : 42, \"user_id\" : 221 , \"chat_message\" : chat_text})\n", + "print(chat)" + ] + }, + { + "cell_type": "markdown", + "id": "8UFfKvSeYn0b", + "metadata": { + "id": "8UFfKvSeYn0b" + }, + "source": [ + "Define pre and post-processing functions. `CreatePrompt` creates a key-value pair of the chat ID and the formatted prompt. `extract_model_reply` parses the response, extracting the JSON string we requested from the model; however, the LLM is not *guaranteed* to return a JSON-formatted object, so we also reaise an exception if the reply is malformed. This helper is then used in the `SentimentAnalysis` `DoFn` to split out the sentiment score as well as the summary of the text. The sentiment score is then used to tag the output, allowing the pipeline to handle different sentiments in different ways." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "dDIys7XaUPvl", + "metadata": { + "id": "dDIys7XaUPvl" + }, + "outputs": [], + "source": [ + "keyed_model_handler = KeyedModelHandler(GemmaModelHandler('gemma_2'))\n", + "\n", + "# Create the prompt by using the information from the chat.\n", + "class CreatePrompt(beam.DoFn):\n", + " def process(self, element, *args, **kwargs):\n", + " user_chat = json.loads(element)\n", + " chat_id = user_chat['id']\n", + " user_id = user_chat['user_id']\n", + " messages = user_chat['chat_message']\n", + " yield (chat_id, prompt_template.format(user_id, messages))\n", + "\n", + "def extract_model_reply(model_inference):\n", + " print(model_inference)\n", + " match = re.search(r\"(\\{[\\s\\S]*?\\})\", model_inference)\n", + " json_str = match.group(1)\n", + " print(json_str)\n", + " result = json.loads(json_str)\n", + " if all(key in result for key in ['sentiment', 'summary']):\n", + " return result\n", + " raise Exception('Malformed model reply')\n", + "\n", + " # @title\n", + "class SentimentAnalysis(beam.DoFn):\n", + " def process(self, element):\n", + " key = element[0]\n", + " match = re.search(r\"@@@([\\s\\S]*?)@@@\", element[1].example)\n", + " chats = match.group(1)\n", + "\n", + " try:\n", + " # The result contains the prompt. Replace the prompt with \"\".\n", + " result = extract_model_reply(element[1].inference.replace(element[1].example, \"\"))\n", + " processed_result = (key, chats, result['sentiment'], result['summary'])\n", + "\n", + " if (result['sentiment'] ==-1):\n", + " output = beam.TaggedOutput('negative', processed_result)\n", + " else:\n", + " output = beam.TaggedOutput('main', processed_result)\n", + "\n", + " except Exception as err:\n", + " print(\"ERROR!\" + str(err))\n", + " output = beam.TaggedOutput('error', element)\n", + "\n", + " yield output\n", + "\n", + "gemma_inference = RunInference(keyed_model_handler)" + ] + }, + { + "cell_type": "markdown", + "id": "Yj9aQ0q8YLOn", + "metadata": { + "id": "Yj9aQ0q8YLOn" + }, + "source": [ + "Finally, execute the pipeline using the code below. To use the example chat input created earlier instead of a custom Pub/Sub source, use `chats = p | beam.Create([chat])` instead of the Pub/Sub read." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "1fb47a17-9563-46f6-9768-73f4802694e8", + "metadata": { + "id": "1fb47a17-9563-46f6-9768-73f4802694e8", + "tags": [] + }, + "outputs": [], + "source": [ + "\n", + "with beam.Pipeline(options=options) as p:\n", + " chats = (p | \"Read Topic\" >>\n", + " beam.io.ReadFromPubSub(subscription=topic_reviews)\n", + " | \"Parse\" >> beam.Map(lambda x: x.decode(\"utf-8\")))\n", + " prompts = chats | \"Create Prompt\" >> beam.ParDo(CreatePrompt())\n", + " results = prompts | \"RunInference-Gemma\" >> gemma_inference\n", + " filtered_results = results | \"Process Results\" >> beam.ParDo(SentimentAnalysis()).with_outputs('main','negative','error')\n", + " generated_responses = (\n", + " filtered_results.negative\n", + " | \"Generate Response\" >> beam.Map(lambda x: ((x[0], x[3]), \"Generate an appology reponse for the user in this chat text: \" + x[1] + \"\"))\n", + " | \"Gemma-Response\" >> gemma_inference\n", + " )\n", + "\n", + " generated_responses | \"Print Reponse\" >> beam.Map(lambda x: logging.info(x))\n", + " filtered_results.main | \"Print Main\" >> beam.Map(lambda x: logging.info(x))\n", + " filtered_results.error | \"Print Errors\" >> beam.Map(lambda x: logging.info(x))" + ] + } + ], + "metadata": { + "colab": { + "provenance": [] + }, + "environment": { + "kernel": "apache-beam-2.57.0", + "name": ".m121", + "type": "gcloud", + "uri": "us-docker.pkg.dev/deeplearning-platform-release/gcr.io/:m121" + }, + "kernelspec": { + "display_name": "Apache Beam 2.57.0 (Local)", + "language": "python", + "name": "apache-beam-2.57.0" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.14" + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} From eaa3b56dbb1990526c03de9fe61ff2b8af1e226c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 14 Aug 2024 11:52:21 -0400 Subject: [PATCH 78/78] Bump github.com/nats-io/nats.go from 1.36.0 to 1.37.0 in /sdks (#32174) Bumps [github.com/nats-io/nats.go](https://github.com/nats-io/nats.go) from 1.36.0 to 1.37.0. - [Release notes](https://github.com/nats-io/nats.go/releases) - [Commits](https://github.com/nats-io/nats.go/compare/v1.36.0...v1.37.0) --- updated-dependencies: - dependency-name: github.com/nats-io/nats.go dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 53aa6f23bfaa2..3a5851cad12f2 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,7 +45,7 @@ require ( github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 github.com/nats-io/nats-server/v2 v2.10.18 - github.com/nats-io/nats.go v1.36.0 + github.com/nats-io/nats.go v1.37.0 github.com/proullon/ramsql v0.1.4 github.com/spf13/cobra v1.8.1 github.com/testcontainers/testcontainers-go v0.32.0 diff --git a/sdks/go.sum b/sdks/go.sum index f49007e6b7d9a..284a50039541a 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1077,8 +1077,8 @@ github.com/nats-io/jwt/v2 v2.5.8 h1:uvdSzwWiEGWGXf+0Q+70qv6AQdvcvxrv9hPM0RiPamE= github.com/nats-io/jwt/v2 v2.5.8/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= github.com/nats-io/nats-server/v2 v2.10.18 h1:tRdZmBuWKVAFYtayqlBB2BuCHNGAQPvoQIXOKwU3WSM= github.com/nats-io/nats-server/v2 v2.10.18/go.mod h1:97Qyg7YydD8blKlR8yBsUlPlWyZKjA7Bp5cl3MUE9K8= -github.com/nats-io/nats.go v1.36.0 h1:suEUPuWzTSse/XhESwqLxXGuj8vGRuPRoG7MoRN/qyU= -github.com/nats-io/nats.go v1.36.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= +github.com/nats-io/nats.go v1.37.0 h1:07rauXbVnnJvv1gfIyghFEo6lUcYRY0WXc3x7x0vUxE= +github.com/nats-io/nats.go v1.37.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= github.com/nats-io/nkeys v0.4.7 h1:RwNJbbIdYCoClSDNY7QVKZlyb/wfT6ugvFCiKy6vDvI= github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw=