From a2681f6a85aaad21179f91e03a91b4a05158841e Mon Sep 17 00:00:00 2001 From: Rui Fan <1996fanrui@gmail.com> Date: Thu, 19 Oct 2023 20:38:15 +0800 Subject: [PATCH 001/104] [FLINK-33316][runtime] Avoid unnecessary heavy getStreamOperatorFactory This closes #23550. --- .../flink/streaming/api/graph/StreamConfig.java | 16 +++++++++++++--- .../streaming/runtime/tasks/OperatorChain.java | 6 ++---- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 267289c181f30..2fb5b81d4a506 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -78,7 +78,12 @@ public class StreamConfig implements Serializable { // Config Keys // ------------------------------------------------------------------------ - @VisibleForTesting public static final String SERIALIZEDUDF = "serializedUDF"; + public static final String SERIALIZED_UDF = "serializedUDF"; + /** + * Introduce serializedUdfClassName to avoid unnecessarily heavy {@link + * #getStreamOperatorFactory}. + */ + public static final String SERIALIZED_UDF_CLASS_NAME = "serializedUdfClassName"; private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs"; private static final String NUMBER_OF_NETWORK_INPUTS = "numberOfNetworkInputs"; @@ -368,7 +373,8 @@ public void setStreamOperator(StreamOperator operator) { public void setStreamOperatorFactory(StreamOperatorFactory factory) { if (factory != null) { - toBeSerializedConfigObjects.put(SERIALIZEDUDF, factory); + toBeSerializedConfigObjects.put(SERIALIZED_UDF, factory); + config.setString(SERIALIZED_UDF_CLASS_NAME, factory.getClass().getName()); } } @@ -380,7 +386,7 @@ public > T getStreamOperator(ClassLoader cl) { public > T getStreamOperatorFactory(ClassLoader cl) { try { - return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl); + return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZED_UDF, cl); } catch (ClassNotFoundException e) { String classLoaderInfo = ClassLoaderUtil.getUserCodeClassLoaderInfo(cl); boolean loadableDoubleCheck = ClassLoaderUtil.validateClassLoadable(e, cl); @@ -400,6 +406,10 @@ public > T getStreamOperatorFactory(ClassLoad } } + public String getStreamOperatorFactoryClassName() { + return config.getString(SERIALIZED_UDF_CLASS_NAME, null); + } + public void setIterationId(String iterationId) { config.setString(ITERATION_ID, iterationId); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index b581990e8c84d..1afee0f75b1f3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -641,9 +641,7 @@ private Map createChainedSources( @Nullable private Counter getOperatorRecordsOutCounter( StreamTask containingTask, StreamConfig operatorConfig) { - ClassLoader userCodeClassloader = containingTask.getUserCodeClassLoader(); - StreamOperatorFactory operatorFactory = - operatorConfig.getStreamOperatorFactory(userCodeClassloader); + String streamOperatorFactoryClassName = operatorConfig.getStreamOperatorFactoryClassName(); // Do not use the numRecordsOut counter on output if this operator is SinkWriterOperator. // // Metric "numRecordsOut" is defined as the total number of records written to the @@ -651,7 +649,7 @@ private Counter getOperatorRecordsOutCounter( // number of records sent to downstream operators, which is number of Committable batches // sent to SinkCommitter. So we skip registering this metric on output and leave this metric // to sink writer implementations to report. - if (operatorFactory instanceof SinkWriterOperatorFactory) { + if (SinkWriterOperatorFactory.class.getName().equals(streamOperatorFactoryClassName)) { return null; } From d00f031785867d465d4a32fd937fe2cbd6722ca3 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Fri, 20 Oct 2023 09:32:55 +0200 Subject: [PATCH 002/104] [FLINK-33321] VertexFlameGraphFactoryTest#verifyRecursively fails on Java 21 --- .../webmonitor/threadinfo/VertexFlameGraphFactory.java | 8 +++++++- .../threadinfo/VertexFlameGraphFactoryTest.java | 10 +++++++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactory.java index 266f379c6995a..87d725c209f37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactory.java @@ -105,6 +105,9 @@ private static VertexFlameGraph createFlameGraphFromSample( private static final Pattern LAMBDA_CLASS_NAME = Pattern.compile("(\\$Lambda\\$)\\d+/(0x)?\\p{XDigit}+$"); + private static final Pattern JDK21_LAMBDA_CLASS_NAME = + Pattern.compile("(\\$\\$Lambda)/(0x)?\\p{XDigit}+$"); + // Drops stack trace elements with class names matching the above regular expression. // These elements are useless, because they don't provide any additional information // except the fact that a lambda is used (they don't have source information, for example), @@ -115,9 +118,12 @@ private static VertexFlameGraph createFlameGraphFromSample( // lambdas, so we have to clean them up explicitly. private static StackTraceElement[] cleanLambdaNames(StackTraceElement[] stackTrace) { StackTraceElement[] result = new StackTraceElement[stackTrace.length]; + final String javaVersion = System.getProperty("java.version"); + final Pattern lambdaClassName = + javaVersion.compareTo("21") >= 0 ? JDK21_LAMBDA_CLASS_NAME : LAMBDA_CLASS_NAME; for (int i = 0; i < stackTrace.length; i++) { StackTraceElement element = stackTrace[i]; - Matcher matcher = LAMBDA_CLASS_NAME.matcher(element.getClassName()); + Matcher matcher = lambdaClassName.matcher(element.getClassName()); if (matcher.find()) { // org.apache.flink.streaming.runtime.io.RecordProcessorUtils$$Lambda$773/0x00000001007f84a0 // --> diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java index c0a3816052324..070daa901cf0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java @@ -58,7 +58,9 @@ void testLambdaClassNamesCleanUp() { private int verifyRecursively(VertexFlameGraph.Node node) { String location = node.getStackTraceLocation(); int lambdas = 0; - if (location.contains("$Lambda$")) { + final String javaVersion = System.getProperty("java.version"); + if (javaVersion.compareTo("21") < 0 && location.contains("$Lambda$") + || javaVersion.compareTo("21") >= 0 && location.contains("$$Lambda")) { lambdas++; // com.example.ClassName.method:123 // -> com.example.ClassName.method @@ -72,10 +74,12 @@ private int verifyRecursively(VertexFlameGraph.Node node) { new Condition() { @Override public boolean matches(String value) { - String javaVersion = System.getProperty("java.version"); + return javaVersion.startsWith("1.8") && value.endsWith("$Lambda$0/0") - || value.endsWith("$Lambda$0/0x0"); + || javaVersion.compareTo("21") < 0 + && value.endsWith("$Lambda$0/0x0") + || value.endsWith("$$Lambda0/0"); } }); } From 4d37b8c34ff062b7505ab8c0ca8f2181768aab60 Mon Sep 17 00:00:00 2001 From: zoudan Date: Thu, 19 Oct 2023 17:48:05 +0800 Subject: [PATCH 003/104] [FLINK-33313][table] Fix RexNodeExtractor#extractConjunctiveConditions throws an Exception when handle binary literal Close apache/flink#23551 --- .../planner/plan/utils/RexNodeExtractor.scala | 4 ++++ .../plan/utils/NestedProjectionUtilTest.scala | 10 +++++---- .../plan/utils/RexNodeExtractorTest.scala | 21 +++++++++++++++++++ .../plan/utils/RexNodeRewriterTest.scala | 7 ++++--- .../planner/plan/utils/RexNodeTestBase.scala | 6 ++++-- 5 files changed, 39 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index 482ce56dc6389..481cbda8b8265 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -37,6 +37,7 @@ import org.apache.flink.table.types.logical.YearMonthIntervalType import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.util.Preconditions +import org.apache.calcite.avatica.util.ByteString import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex._ @@ -502,6 +503,9 @@ class RexNodeToExpressionConverter( // convert to BigDecimal literal.getValueAs(classOf[java.math.BigDecimal]) + case BINARY | VARBINARY => + literal.getValueAs(classOf[Array[Byte]]) + case _ => literal.getValue } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/NestedProjectionUtilTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/NestedProjectionUtilTest.scala index ec8214f5b91bd..9cd44c9fea71b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/NestedProjectionUtilTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/NestedProjectionUtilTest.scala @@ -87,12 +87,13 @@ class NestedProjectionUtilTest extends RexNodeTestBase { "$2", "$3", "$4", + "$5", "*($t2, $t3)", "100", - "<($t5, $t6)", + "<($t6, $t7)", "6", - ">($t1, $t8)", - "AND($t7, $t9)"))) + ">($t1, $t9)", + "AND($t8, $t10)"))) val nestedField = NestedProjectionUtil.build(exprs, rexProgram.getInputRowType) val paths = NestedProjectionUtil.convertToIndexArray(nestedField) @@ -101,7 +102,8 @@ class NestedProjectionUtilTest extends RexNodeTestBase { Array(1), Array(2), Array(3), - Array(4) + Array(4), + Array(5) ) assertArray(paths, orderedPaths) val builder = new FlinkRexBuilder(typeFactory) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala index 2eb87e35cc89d..bd5f15d3bed06 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala @@ -33,6 +33,7 @@ import org.apache.flink.table.planner.utils.{DateTimeTestUtil, IntSumAggFunction import org.apache.flink.table.resource.ResourceManager import org.apache.flink.table.utils.CatalogManagerMocks +import org.apache.calcite.avatica.util.ByteString import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex.{RexBuilder, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName @@ -145,6 +146,26 @@ class RexNodeExtractorTest extends RexNodeTestBase { assertEquals(0, unconvertedRexNodes.length) } + @Test + def testExtractConditionWithBinaryLiteral(): Unit = { + // blob + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(5), 5) + + // X'616263' + val t1 = rexBuilder.makeBinaryLiteral(ByteString.of("616263", 16)) + + // blob = X'616263' + val a = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1) + + val relBuilder: RexBuilder = new FlinkRexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + extractConjunctiveConditions(a, -1, allFieldNames, relBuilder, functionCatalog) + + val expected: Array[Expression] = Array($"blob" === Array[Byte](97, 98, 99)) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d) @Test def testExtractCnfCondition(): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriterTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriterTest.scala index 0cea5c8be699c..57a0edda39ebf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriterTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriterTest.scala @@ -39,12 +39,13 @@ class RexNodeRewriterTest extends RexNodeTestBase { "$2", "$3", "$4", + "$5", "*($t2, $t3)", "100", - "<($t5, $t6)", + "<($t6, $t7)", "6", - ">($t1, $t8)", - "AND($t7, $t9)"))) + ">($t1, $t9)", + "AND($t8, $t10)"))) // use amount, id, price fields to create a new RexProgram val usedFields = Array(2, 3, 1) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeTestBase.scala index ec326c2a5403d..1ba968b325390 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeTestBase.scala @@ -39,14 +39,16 @@ abstract class RexNodeTestBase { val typeFactory: FlinkTypeFactory = new FlinkTypeFactory( Thread.currentThread().getContextClassLoader) - val allFieldNames: java.util.List[String] = List("name", "id", "amount", "price", "flag").asJava + val allFieldNames: java.util.List[String] = + List("name", "id", "amount", "price", "flag", "blob").asJava val allFieldTypes: java.util.List[RelDataType] = List( DataTypes.VARCHAR(100), DataTypes.BIGINT(), DataTypes.INT(), DataTypes.DOUBLE(), - DataTypes.BOOLEAN()) + DataTypes.BOOLEAN(), + DataTypes.BYTES()) .map(LogicalTypeDataTypeConverter.fromDataTypeToLogicalType) .map(typeFactory.createFieldTypeFromLogicalType) .asJava From 36caa02bf73bb4e1644451f71ca07860360ec0d8 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 17 Oct 2023 15:23:15 +0200 Subject: [PATCH 004/104] [FLINK-33291][build] Sets the enforced range for Maven and JDK within the release profile --- pom.xml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 1812449bdf15f..01536764538fa 100644 --- a/pom.xml +++ b/pom.xml @@ -1365,11 +1365,13 @@ under the License. + + - 3.8.6 + [3.8.6] - 1.8.0 + [1.8.0,1.8.1) From c1c780225cd5c622aa6f2f052d5b926fc4c74dbb Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Thu, 12 Oct 2023 14:58:44 +0800 Subject: [PATCH 005/104] [FLINK-33116][tests] Fix CliClientTest.testCancelExecutionInteractiveMode fails with NPE --- .../flink/table/client/cli/CliClientTest.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java index 5cf6b527a6cce..f2920c21d180f 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ResultKind; @@ -294,14 +295,13 @@ void testCancelExecutionInteractiveMode() throws Exception { try (Terminal terminal = TerminalUtils.createDumbTerminal(inputStream, outputStream); CliClient client = new CliClient(() -> terminal, mockExecutor, historyFilePath, null)) { - Thread thread = - new Thread( - () -> { - try { - client.executeInInteractiveMode(); - } catch (Exception ignore) { - } - }); + CheckedThread thread = + new CheckedThread() { + @Override + public void go() { + client.executeInInteractiveMode(); + } + }; thread.start(); while (!mockExecutor.isAwait) { @@ -311,6 +311,8 @@ void testCancelExecutionInteractiveMode() throws Exception { terminal.raise(Terminal.Signal.INT); CommonTestUtils.waitUntilCondition( () -> outputStream.toString().contains(CliStrings.MESSAGE_HELP)); + // Prevent NPE when closing the terminal. See FLINK-33116 for more information. + thread.sync(); } } From a58a35e2b1abda28ae7d884a433ca26624c7a4c4 Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Fri, 20 Oct 2023 12:56:21 +0200 Subject: [PATCH 006/104] [FLINK-33238][Formats/Avro] Upgrade AVRO dependency to 1.11.3. This closes #23508 Upgrade AVRO dependency to 1.11.3 to mitigate scanners flagging Flink as vulnerable for CVE-2023-39410 This also solves the problem where Union types are not inheriting type conversions Co-authored-by: AndreiLeib --- .../formats/avro/RegistryAvroDeserializationSchemaTest.java | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java index c11c4bfb6b6f4..424e44817ee67 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java @@ -85,7 +85,7 @@ void testSpecificRecordReadMoreFieldsThanWereWritten() throws IOException { + " \"fields\": [\n" + " {\"name\": \"name\", \"type\": \"string\"}" + " ]\n" - + "}]"); + + "}"); RegistryAvroDeserializationSchema deserializer = new RegistryAvroDeserializationSchema<>( SimpleRecord.class, diff --git a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index dd601b9119f9e..f4fd1a6308df5 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -13,7 +13,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.confluent:common-config:7.2.2 - io.confluent:common-utils:7.2.2 - io.confluent:kafka-schema-registry-client:7.2.2 -- org.apache.avro:avro:1.11.1 +- org.apache.avro:avro:1.11.3 - org.apache.commons:commons-compress:1.21 - org.apache.kafka:kafka-clients:7.2.2-ccs - org.glassfish.jersey.core:jersey-common:2.30 diff --git a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE index 21f85619d82fa..4cf05a46b4a55 100644 --- a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.avro:avro:1.11.1 +- org.apache.avro:avro:1.11.3 - com.fasterxml.jackson.core:jackson-core:2.14.3 - com.fasterxml.jackson.core:jackson-databind:2.14.3 - com.fasterxml.jackson.core:jackson-annotations:2.14.3 diff --git a/pom.xml b/pom.xml index 01536764538fa..8a9c3f0e3a567 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ under the License. 5.4.0 - 1.11.1 + 1.11.3 2.14.3 1.2.0 From 9471f81a2fb53f5c8274bbbe99161d8f1b27bafc Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Tue, 17 Oct 2023 23:18:42 +0200 Subject: [PATCH 007/104] [FLINK-33308] Upgrade lombok to 1.8.30 --- flink-core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 598f0ed4995cf..ce90b2a1e6d16 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -147,7 +147,7 @@ under the License. org.projectlombok lombok - 1.18.22 + 1.18.30 test From 72e302310ba55bb5f35966ed448243aae36e193e Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Wed, 18 Oct 2023 22:28:43 +0200 Subject: [PATCH 008/104] [FLINK-33307] Disable spotless on Java 21 --- pom.xml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/pom.xml b/pom.xml index 8a9c3f0e3a567..5a806eb4dc696 100644 --- a/pom.xml +++ b/pom.xml @@ -1124,6 +1124,30 @@ under the License. + + java21 + + [21,) + + + + + + + com.diffplug.spotless + spotless-maven-plugin + + + true + + + + + + + fast From c7beda0da81ffc4bbb01befafd2eed08b7b35854 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 20 Oct 2023 15:09:21 +0200 Subject: [PATCH 009/104] [FLINK-33327] Window TVF column expansion does not work with an INSERT INTO --- .../calcite/FlinkCalciteSqlValidator.java | 8 +++++- .../plan/stream/sql/ColumnExpansionTest.java | 25 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidator.java index 0b0075a4f6476..a8cd9265e693a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidator.java @@ -54,6 +54,7 @@ import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlSnapshot; +import org.apache.calcite.sql.SqlTableFunction; import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.SqlWindowTableFunction; import org.apache.calcite.sql.parser.SqlParserPos; @@ -441,7 +442,7 @@ private static List getExplicitTableOperands(SqlNode node) { } final SqlFunction function = (SqlFunction) call.getOperator(); - if (function.getFunctionType() != SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION) { + if (!isTableFunction(function)) { return null; } @@ -459,4 +460,9 @@ private static List getExplicitTableOperands(SqlNode node) { }) .collect(Collectors.toList()); } + + private static boolean isTableFunction(SqlFunction function) { + return function instanceof SqlTableFunction + || function.getFunctionType() == SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION; + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/ColumnExpansionTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/ColumnExpansionTest.java index 290b981583e92..331cc13f36c03 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/ColumnExpansionTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/ColumnExpansionTest.java @@ -260,6 +260,31 @@ public void testExplicitTableWithinTableFunction() { "agg"); } + @Test + public void testExplicitTableWithinTableFunctionWithInsertIntoNamedColumns() { + tableEnv.getConfig() + .set( + TABLE_COLUMN_EXPANSION_STRATEGY, + Collections.singletonList(EXCLUDE_DEFAULT_VIRTUAL_METADATA_COLUMNS)); + + tableEnv.executeSql( + "CREATE TABLE sink (\n" + + " a STRING,\n" + + " c BIGINT\n" + + ") WITH (\n" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + // Test case for FLINK-33327, we can not assert column names of an INSERT INTO query. Make + // sure the query can be planned. + tableEnv.explainSql( + "INSERT INTO sink(a, c) " + + "SELECT t3_s, COUNT(t3_i) FROM " + + " TABLE(TUMBLE(TABLE t3, DESCRIPTOR(t3_m_virtual), INTERVAL '1' MINUTE)) " + + "GROUP BY t3_s;"); + } + private void assertColumnNames(String sql, String... columnNames) { assertThat(tableEnv.sqlQuery(sql).getResolvedSchema().getColumnNames()) .containsExactly(columnNames); From 76dda5d3cf7c4ecf255370679b27e11dd974a293 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Sun, 22 Oct 2023 18:33:35 +0200 Subject: [PATCH 010/104] [FLINK-33318] Expose aggregated collector metrics and measure timeMsPerSecond --- docs/content/docs/ops/metrics.md | 51 ++++++----- .../org/apache/flink/metrics/MeterView.java | 39 +++++++++ .../metrics/job-manager-metrics.component.ts | 6 +- .../runtime/metrics/util/MetricUtils.java | 32 +++++-- .../runtime/metrics/util/MetricUtilsTest.java | 86 ++++++++++++++++++- 5 files changed, 182 insertions(+), 32 deletions(-) diff --git a/docs/content/docs/ops/metrics.md b/docs/content/docs/ops/metrics.md index 2bdb44cac9853..ac106f6e6c927 100644 --- a/docs/content/docs/ops/metrics.md +++ b/docs/content/docs/ops/metrics.md @@ -315,7 +315,7 @@ public class MyMapper extends RichMapFunction { .getMetricGroup() .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram)); } - + @Override public Long map(Long value) throws Exception { this.histogram.update(value); @@ -333,12 +333,12 @@ class MyMapper extends RichMapFunction[Long, Long] { override def open(config: Configuration): Unit = { val dropwizardHistogram = new com.codahale.metrics.Histogram(new SlidingWindowReservoir(500)) - + histogram = getRuntimeContext() .getMetricGroup() .histogram("myHistogram", new DropwizardHistogramWrapper(dropwizardHistogram)) } - + override def map(value: Long): Long = { histogram.update(value) value @@ -464,7 +464,7 @@ class MyMapper extends RichMapFunction[Long,Long] { override def open(config: Configuration): Unit = { val dropwizardMeter: com.codahale.metrics.Meter = new com.codahale.metrics.Meter() - + meter = getRuntimeContext() .getMetricGroup() .meter("myMeter", new DropwizardMeterWrapper(dropwizardMeter)) @@ -687,7 +687,7 @@ Thus, in order to infer the metric identifier: ### Memory -The memory-related metrics require Oracle's memory management (also included in OpenJDK's Hotspot implementation) to be in place. +The memory-related metrics require Oracle's memory management (also included in OpenJDK's Hotspot implementation) to be in place. Some metrics might not be exposed when using other JVM implementations (e.g. IBM's J9). @@ -715,8 +715,8 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM @@ -829,15 +829,20 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM - - - - + + + + + + + + + - - + + @@ -1904,7 +1909,7 @@ Please refer to [Kafka monitoring]({{< ref "docs/connectors/datastream/kafka" >} - @@ -2147,7 +2152,7 @@ Metrics below can be used to measure the effectiveness of speculative execution. - @@ -2162,7 +2167,7 @@ To enable the latency tracking you must set the `latencyTrackingInterval` to a p At the `latencyTrackingInterval`, the sources will periodically emit a special record, called a `LatencyMarker`. The marker contains a timestamp from the time when the record has been emitted at the sources. -Latency markers can not overtake regular user records, thus if records are queuing up in front of an operator, +Latency markers can not overtake regular user records, thus if records are queuing up in front of an operator, it will add to the latency tracked by the marker. Note that the latency markers are not accounting for the time user records spend in operators as they are @@ -2170,17 +2175,17 @@ bypassing them. In particular the markers are not accounting for the time record Only if operators are not able to accept new records, thus they are queuing up, the latency measured using the markers will reflect that. -The `LatencyMarker`s are used to derive a distribution of the latency between the sources of the topology and each -downstream operator. These distributions are reported as histogram metrics. The granularity of these distributions can -be controlled in the [Flink configuration]({{< ref "docs/deployment/config" >}}#metrics-latency-interval). For the highest -granularity `subtask` Flink will derive the latency distribution between every source subtask and every downstream -subtask, which results in quadratic (in the terms of the parallelism) number of histograms. +The `LatencyMarker`s are used to derive a distribution of the latency between the sources of the topology and each +downstream operator. These distributions are reported as histogram metrics. The granularity of these distributions can +be controlled in the [Flink configuration]({{< ref "docs/deployment/config" >}}#metrics-latency-interval). For the highest +granularity `subtask` Flink will derive the latency distribution between every source subtask and every downstream +subtask, which results in quadratic (in the terms of the parallelism) number of histograms. Currently, Flink assumes that the clocks of all machines in the cluster are in sync. We recommend setting up an automated clock synchronisation service (like NTP) to avoid false latency results. Warning Enabling latency metrics can significantly impact the performance -of the cluster (in particular for `subtask` granularity). It is highly recommended to only use them for debugging +of the cluster (in particular for `subtask` granularity). It is highly recommended to only use them for debugging purposes. ## State access latency tracking @@ -2194,7 +2199,7 @@ This configuration has a default value of 100. A smaller value will get more acc As the type of this latency metrics is histogram, `state.backend.latency-track.history-size` will control the maximum number of recorded values in history, which has the default value of 128. A larger value of this configuration will require more memory, but will provide a more accurate result. -Warning Enabling state-access-latency metrics may impact the performance. +Warning Enabling state-access-latency metrics may impact the performance. It is recommended to only use them for debugging purposes. ## REST API integration diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java index ddb7e3c4fd30f..61a087cfde907 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java @@ -71,6 +71,10 @@ public MeterView(Counter counter, int timeSpanInSeconds) { this.values = new long[this.timeSpanInSeconds / UPDATE_INTERVAL_SECONDS + 1]; } + public MeterView(Gauge numberGauge) { + this(new GaugeWrapper(numberGauge)); + } + @Override public void markEvent() { this.counter.inc(); @@ -98,4 +102,39 @@ public void update() { currentRate = ((double) (values[time] - values[(time + 1) % values.length]) / timeSpanInSeconds); } + + /** Simple wrapper to expose number gauges as timers. */ + static class GaugeWrapper implements Counter { + + final Gauge numberGauge; + + GaugeWrapper(Gauge numberGauge) { + this.numberGauge = numberGauge; + } + + @Override + public void inc() { + throw new UnsupportedOperationException(); + } + + @Override + public void inc(long n) { + throw new UnsupportedOperationException(); + } + + @Override + public void dec() { + throw new UnsupportedOperationException(); + } + + @Override + public void dec(long n) { + throw new UnsupportedOperationException(); + } + + @Override + public long getCount() { + return numberGauge.getValue().longValue(); + } + } } diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts index 8888fa314fe29..bb759e2694041 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts @@ -112,7 +112,11 @@ export class JobManagerMetricsComponent implements OnInit, OnDestroy { this.listOfGCMetric = Array.from( new Set( this.listOfGCName.map(item => - item.replace('Status.JVM.GarbageCollector.', '').replace('.Count', '').replace('.Time', '') + item + .replace('Status.JVM.GarbageCollector.', '') + .replace('.Count', '') + .replace('.TimeMsPerSecond', '') + .replace('.Time', '') ) ) ).map(name => { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index 520a54ec6035b..0074dfbe07e58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -129,7 +130,8 @@ public static void instantiateStatusMetrics(MetricGroup metricGroup) { MetricGroup jvm = metricGroup.addGroup("JVM"); instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader")); - instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector")); + instantiateGarbageCollectorMetrics( + jvm.addGroup("GarbageCollector"), ManagementFactory.getGarbageCollectorMXBeans()); instantiateMemoryMetrics(jvm.addGroup(METRIC_GROUP_MEMORY)); instantiateThreadMetrics(jvm.addGroup("Threads")); instantiateCPUMetrics(jvm.addGroup("CPU")); @@ -222,16 +224,32 @@ private static void instantiateClassLoaderMetrics(MetricGroup metrics) { metrics.>gauge("ClassesUnloaded", mxBean::getUnloadedClassCount); } - private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) { - List garbageCollectors = - ManagementFactory.getGarbageCollectorMXBeans(); - + @VisibleForTesting + static void instantiateGarbageCollectorMetrics( + MetricGroup metrics, List garbageCollectors) { for (final GarbageCollectorMXBean garbageCollector : garbageCollectors) { MetricGroup gcGroup = metrics.addGroup(garbageCollector.getName()); - gcGroup.>gauge("Count", garbageCollector::getCollectionCount); - gcGroup.>gauge("Time", garbageCollector::getCollectionTime); + gcGroup.gauge("Count", garbageCollector::getCollectionCount); + Gauge timeGauge = gcGroup.gauge("Time", garbageCollector::getCollectionTime); + gcGroup.meter("TimeMsPerSecond", new MeterView(timeGauge)); } + Gauge totalGcTime = + () -> + garbageCollectors.stream() + .mapToLong(GarbageCollectorMXBean::getCollectionTime) + .sum(); + + Gauge totalGcCount = + () -> + garbageCollectors.stream() + .mapToLong(GarbageCollectorMXBean::getCollectionCount) + .sum(); + + MetricGroup allGroup = metrics.addGroup("All"); + allGroup.gauge("Count", totalGcCount); + Gauge totalTime = allGroup.gauge("Time", totalGcTime); + allGroup.meter("TimeMsPerSecond", new MeterView(totalTime)); } private static void instantiateMemoryMetrics(MetricGroup metrics) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java index e0c70638ed9e9..c98637fc37cde 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.memory.MemoryAllocationException; @@ -42,10 +43,15 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import javax.management.ObjectName; + +import java.lang.management.GarbageCollectorMXBean; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.metrics.util.MetricUtils.METRIC_GROUP_FLINK; @@ -110,7 +116,6 @@ void testMetaspaceCompleteness() { assertThat(hasMetaspaceMemoryPool()) .withFailMessage("Requires JVM with Metaspace memory pool") .isTrue(); - final InterceptingOperatorMetricGroup metaspaceMetrics = new InterceptingOperatorMetricGroup() { @Override @@ -126,6 +131,42 @@ public MetricGroup addGroup(String name) { assertThat(metaspaceMetrics.get(MetricNames.MEMORY_MAX)).isNotNull(); } + @Test + public void testGcMetricCompleteness() { + Map addedGroups = new HashMap<>(); + InterceptingOperatorMetricGroup gcGroup = + new InterceptingOperatorMetricGroup() { + @Override + public MetricGroup addGroup(String name) { + return addedGroups.computeIfAbsent( + name, k -> new InterceptingOperatorMetricGroup()); + } + }; + + List garbageCollectors = new ArrayList<>(); + garbageCollectors.add(new TestGcBean("gc1", 100, 500)); + garbageCollectors.add(new TestGcBean("gc2", 50, 250)); + + MetricUtils.instantiateGarbageCollectorMetrics(gcGroup, garbageCollectors); + assertThat(addedGroups).containsOnlyKeys("gc1", "gc2", "All"); + + // Make sure individual collector metrics are correct + validateCollectorMetric(addedGroups.get("gc1"), 100, 500L); + validateCollectorMetric(addedGroups.get("gc2"), 50L, 250L); + + // Make sure all/total collector metrics are correct + validateCollectorMetric(addedGroups.get("All"), 150L, 750L); + } + + private static void validateCollectorMetric( + InterceptingOperatorMetricGroup group, long count, long time) { + assertThat(((Gauge) group.get("Count")).getValue()).isEqualTo(count); + assertThat(((Gauge) group.get("Time")).getValue()).isEqualTo(time); + MeterView perSecond = ((MeterView) group.get("TimeMsPerSecond")); + perSecond.update(); + assertThat(perSecond.getRate()).isGreaterThan(0.); + } + @Test void testHeapMetricsCompleteness() { final InterceptingOperatorMetricGroup heapMetrics = new InterceptingOperatorMetricGroup(); @@ -296,4 +337,47 @@ private void runUntilMetricChanged( "%s usage metric never changed its value after %d runs.", name, maxRuns); fail(msg); } + + static class TestGcBean implements GarbageCollectorMXBean { + + final String name; + final long collectionCount; + final long collectionTime; + + public TestGcBean(String name, long collectionCount, long collectionTime) { + this.name = name; + this.collectionCount = collectionCount; + this.collectionTime = collectionTime; + } + + @Override + public long getCollectionCount() { + return collectionCount; + } + + @Override + public long getCollectionTime() { + return collectionTime; + } + + @Override + public String getName() { + return name; + } + + @Override + public boolean isValid() { + throw new UnsupportedOperationException(); + } + + @Override + public String[] getMemoryPoolNames() { + throw new UnsupportedOperationException(); + } + + @Override + public ObjectName getObjectName() { + throw new UnsupportedOperationException(); + } + } } From 6cdfca2a6aba1a387b866d9358df9c1ee7f1c138 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Sun, 22 Oct 2023 21:32:01 +0200 Subject: [PATCH 011/104] [FLINK-33310] Scala before 2.12.18 doesn't compile on Java 21 --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index 5a806eb4dc696..a8c4dfcf1c6b6 100644 --- a/pom.xml +++ b/pom.xml @@ -1130,6 +1130,11 @@ under the License. [21,) + + + 2.12.18 + + From 3596a94fe415fdc834cec346cc3910dd775f5239 Mon Sep 17 00:00:00 2001 From: lincoln lee Date: Mon, 23 Oct 2023 16:56:55 +0800 Subject: [PATCH 012/104] [FLINK-33314][table] Fix the named parameter example in window tvf This closes #23549 --- .../docs/dev/table/sql/queries/window-tvf.md | 4 +- .../docs/dev/table/sql/queries/window-tvf.md | 4 +- .../stream/sql/WindowTableFunctionTest.xml | 62 +++++++++++++++++++ .../stream/sql/WindowTableFunctionTest.scala | 29 +++++++++ 4 files changed, 95 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/queries/window-tvf.md b/docs/content.zh/docs/dev/table/sql/queries/window-tvf.md index 8af3015fd2220..986aab222c200 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/window-tvf.md +++ b/docs/content.zh/docs/dev/table/sql/queries/window-tvf.md @@ -312,13 +312,13 @@ CUMULATE(TABLE data, DESCRIPTOR(timecol), step, size) Flink SQL> SELECT * FROM TABLE( TUMBLE(TABLE Bid, DESCRIPTOR(bidtime), INTERVAL '10' MINUTES, INTERVAL '1' MINUTES)); -- or with the named params --- note: the DATA param must be the first +-- note: the DATA param must be the first and `OFFSET` should be wrapped with double quotes Flink SQL> SELECT * FROM TABLE( TUMBLE( DATA => TABLE Bid, TIMECOL => DESCRIPTOR(bidtime), SIZE => INTERVAL '10' MINUTES, - OFFSET => INTERVAL '1' MINUTES)); + `OFFSET` => INTERVAL '1' MINUTES)); +------------------+-------+------+------------------+------------------+-------------------------+ | bidtime | price | item | window_start | window_end | window_time | +------------------+-------+------+------------------+------------------+-------------------------+ diff --git a/docs/content/docs/dev/table/sql/queries/window-tvf.md b/docs/content/docs/dev/table/sql/queries/window-tvf.md index 95cf450844164..712c27d22278b 100644 --- a/docs/content/docs/dev/table/sql/queries/window-tvf.md +++ b/docs/content/docs/dev/table/sql/queries/window-tvf.md @@ -312,13 +312,13 @@ We show an example to describe how to use offset in Tumble window in the followi Flink SQL> SELECT * FROM TABLE( TUMBLE(TABLE Bid, DESCRIPTOR(bidtime), INTERVAL '10' MINUTES, INTERVAL '1' MINUTES)); -- or with the named params --- note: the DATA param must be the first +-- note: the DATA param must be the first and `OFFSET` should be wrapped with double quotes Flink SQL> SELECT * FROM TABLE( TUMBLE( DATA => TABLE Bid, TIMECOL => DESCRIPTOR(bidtime), SIZE => INTERVAL '10' MINUTES, - OFFSET => INTERVAL '1' MINUTES)); + `OFFSET` => INTERVAL '1' MINUTES)); +------------------+-------+------+------------------+------------------+-------------------------+ | bidtime | price | item | window_start | window_end | window_time | +------------------+-------+------+------------------+------------------+-------------------------+ diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml index 8994ab6b40947..fabcb77e269e9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml @@ -191,6 +191,37 @@ Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, wi +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime]) +]]> + + + + + TABLE MyTable, + TIMECOL => DESCRIPTOR(rowtime), + SIZE => INTERVAL '15' MINUTE, + `OFFSET` => INTERVAL '5' MINUTE)) +]]> + + + + + + @@ -335,6 +366,37 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], wind + + + + + TABLE MyTable, + TIMECOL => DESCRIPTOR(rowtime), + SIZE => INTERVAL '15' MINUTE, + `OFFSET` => INTERVAL '5' MINUTE)) +]]> + + + + + + TABLE MyTable, + | TIMECOL => DESCRIPTOR(rowtime), + | SIZE => INTERVAL '15' MINUTE, + | `OFFSET` => INTERVAL '5' MINUTE)) + |""".stripMargin + util.verifyRelPlan(sql) + } + @Test def testHopTVFWithOffset(): Unit = { val sql = @@ -199,6 +213,20 @@ class WindowTableFunctionTest extends TableTestBase { util.verifyRelPlan(sql) } + @Test + def testHopTVFWithNamedParams(): Unit = { + val sql = + """ + |SELECT * + |FROM TABLE(TUMBLE( + | DATA => TABLE MyTable, + | TIMECOL => DESCRIPTOR(rowtime), + | SIZE => INTERVAL '15' MINUTE, + | `OFFSET` => INTERVAL '5' MINUTE)) + |""".stripMargin + util.verifyRelPlan(sql) + } + @Test def testCumulateTVFWithOffset(): Unit = { val sql = @@ -230,4 +258,5 @@ class WindowTableFunctionTest extends TableTestBase { |""".stripMargin util.verifyRelPlan(sql) } + } From d59196cc23e58ca7d86d7ce2924defb8ba289c3e Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Mon, 23 Oct 2023 11:05:45 +0200 Subject: [PATCH 013/104] [FLINK-33329] Upgrade commons-compress to 1.24.0 --- flink-dist/src/main/resources/META-INF/NOTICE | 2 +- .../flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE | 2 +- .../flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../flink-s3-fs-presto/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE | 2 +- pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE index 8eb3dbc5dc798..a5358c47c3048 100644 --- a/flink-dist/src/main/resources/META-INF/NOTICE +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -12,7 +12,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-cli:commons-cli:1.5.0 - commons-collections:commons-collections:3.2.2 - commons-io:commons-io:2.11.0 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 - org.apache.commons:commons-lang3:3.12.0 - org.apache.commons:commons-math3:3.6.1 - org.apache.commons:commons-text:1.10.0 diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE index 0236725e0a47f..c5afbb58e3690 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE @@ -18,7 +18,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-collections:commons-collections:3.2.2 - commons-io:commons-io:2.11.0 - commons-logging:commons-logging:1.1.3 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.12.0 - org.apache.commons:commons-text:1.10.0 diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE index c16ab1adc986d..69137435b28a7 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -24,7 +24,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-io:commons-io:2.11.0 - commons-logging:commons-logging:1.1.3 - joda-time:joda-time:2.5 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.12.0 - org.apache.commons:commons-text:1.10.0 diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE index 3356afa220538..fc834f1e4d467 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -36,7 +36,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.airlift:units:1.3 - joda-time:joda-time:2.5 - org.alluxio:alluxio-shaded-client:2.7.3 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.12.0 - org.apache.commons:commons-text:1.10.0 diff --git a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index f4fd1a6308df5..2231d190de11d 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -14,7 +14,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.confluent:common-utils:7.2.2 - io.confluent:kafka-schema-registry-client:7.2.2 - org.apache.avro:avro:1.11.3 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 - org.apache.kafka:kafka-clients:7.2.2-ccs - org.glassfish.jersey.core:jersey-common:2.30 - org.xerial.snappy:snappy-java:1.1.10.4 diff --git a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE index 4cf05a46b4a55..c0ca160784f8d 100644 --- a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE @@ -10,4 +10,4 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.14.3 - com.fasterxml.jackson.core:jackson-databind:2.14.3 - com.fasterxml.jackson.core:jackson-annotations:2.14.3 -- org.apache.commons:commons-compress:1.21 +- org.apache.commons:commons-compress:1.24.0 diff --git a/pom.xml b/pom.xml index a8c4dfcf1c6b6..cab5a38103f56 100644 --- a/pom.xml +++ b/pom.xml @@ -720,7 +720,7 @@ under the License. org.apache.commons commons-compress - 1.21 + 1.24.0 From 8af765b4c9cd3519193b89dae40a8f8c2439c661 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Mon, 23 Oct 2023 12:22:12 +0200 Subject: [PATCH 014/104] [FLINK-32671] Document Externalized Declararative Resource Management + minor Elastic Scaling page restructuring. --- .../docs/deployment/elastic_scaling.md | 113 ++++++++++++----- .../docs/deployment/elastic_scaling.md | 118 ++++++++++++------ docs/static/fig/adaptive_scheduler.png | Bin 0 -> 538791 bytes .../static/fig/adaptive_scheduler_rescale.png | Bin 0 -> 859268 bytes 4 files changed, 161 insertions(+), 70 deletions(-) create mode 100644 docs/static/fig/adaptive_scheduler.png create mode 100644 docs/static/fig/adaptive_scheduler_rescale.png diff --git a/docs/content.zh/docs/deployment/elastic_scaling.md b/docs/content.zh/docs/deployment/elastic_scaling.md index 2b7475a1af001..d9b09ba7e8ca2 100644 --- a/docs/content.zh/docs/deployment/elastic_scaling.md +++ b/docs/content.zh/docs/deployment/elastic_scaling.md @@ -25,17 +25,91 @@ under the License. # 弹性扩缩容 -在 Apache Flink 中,可以通过手动停止 Job,然后从停止时创建的 Savepoint 恢复,最后重新指定并行度的方式来重新扩缩容 Job。 +Historically, the parallelism of a job has been static throughout its lifecycle and defined once during its submission. Batch jobs couldn't be rescaled at all, while Streaming jobs could have been stopped with a savepoint and restarted with a different parallelism. -这个文档描述了那些可以使 Flink 自动调整并行度的选项。 +This page describes a new class of schedulers that allow Flink to adjust job's parallelism at runtime, which pushes Flink one step closer to a truly cloud-native stream processor. The new schedulers are [Adaptive Scheduler](#adaptive-scheduler) (streaming) and [Adaptive Batch Scheduler](#adaptive-batch-scheduler) (batch). -## Reactive 模式 +## Adaptive 调度器 + +The Adaptive Scheduler can adjust the parallelism of a job based on available slots. It will automatically reduce the parallelism if not enough slots are available to run the job with the originally configured parallelism; be it due to not enough resources being available at the time of submission, or TaskManager outages during the job execution. If new slots become available the job will be scaled up again, up to the configured parallelism. + +In Reactive Mode (see below) the configured parallelism is ignored and treated as if it was set to infinity, letting the job always use as many resources as possible. + +One benefit of the Adaptive Scheduler over the default scheduler is that it can handle TaskManager losses gracefully, since it would just scale down in these cases. + +{{< img src="/fig/adaptive_scheduler.png" >}} + +Adaptive Scheduler builds on top of a feature called [Declarative Resource Management](https://cwiki.apache.org/confluence/display/FLINK/FLIP-138%3A+Declarative+Resource+management). As you can see, instead of asking for the exact number of slots, JobMaster declares its desired resources (for reactive mode the maximum is set to infinity) to the ResourceManager, which then tries to fulfill those resources. + +{{< img src="/fig/adaptive_scheduler_rescale.png" >}} + +When JobMaster gets more resources during the runtime, it will automatically rescale the job using the latest available savepoint, eliminating the need for an external orchestration. + +Starting from **Flink 1.18.x**, you can re-declare the resource requirements of a running job using [Externalized Declarative Resource Management](#externalized-declarative-resource-management), otherwise the Adaptive Scheduler won't be able to handle cases where the job needs to be rescaled due to a change in the input rate, or a change in the performance of the workload. + +### Externalized Declarative Resource Management + +{{< hint warning >}} +Externalized Declarative Resource Management is an MVP ("minimum viable product") feature. The Flink community is actively looking for feedback by users through our mailing lists. Please check the limitations listed on this page. +{{< /hint >}} + +{{< hint info >}} +You can use Externalized Declarative Resource Management with the [Apache Flink Kubernetes operator](https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.6/docs/custom-resource/autoscaler/#flink-118-and-in-place-scaling-support) for a fully-fledged auto-scaling experience. +{{< /hint >}} + +Externalized Declarative Resource Management aims to address two deployment scenarios: +1. Adaptive Scheduler on Session Cluster, where multiple jobs can compete for resources, and you need a finer-grained control over the distribution of resources between jobs. +2. Adaptive Scheduler on Application Cluster in combination with Active Resource Manager (e.g. [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes" >}})), where you rely on Flink to "greedily" spawn new TaskManagers, but you still want to leverage rescaling capabilities as with [Reactive Mode](#reactive-mode). + +by introducing a new [REST API endpoint]({{< ref "docs/ops/rest_api" >}}#jobs-jobid-resource-requirements-1), that allows you to re-declare resource requirements of a running job, by setting per-vertex parallelism boundaries. + +``` +PUT /jobs//resource-requirements + +REQUEST BODY: +{ + "": { + "parallelism": { + "lowerBound": 3, + "upperBound": 5 + } + }, + "": { + "parallelism": { + "lowerBound": 2, + "upperBound": 3 + } + } +} +``` + +To a certain extent, the above endpoint could be thought about as a "re-scaling endpoint" and it introduces an important building block for building an auto-scaling experience for Flink. + +You can manually try this feature out, by navigating the Job overview in the Flink UI and using up-scale/down-scale buttons in the task list. + +### Usage {{< hint info >}} -Reactive 模式是一个 MVP (minimum viable product,最小可行产品)特性。目前 Flink 社区正在积极地从邮件列表中获取用户的使用反馈。请注意文中列举的一些局限性。 +If you are using Adaptive Scheduler on a [session cluster]({{< ref "docs/deployment/overview" >}}/#session-mode), there are no guarantees regarding the distribution of slots between multiple running jobs in the same session, in case the cluster doesn't have enough resources. The [External Declarative Resource Management](#externalized-declarative-resource-management) can partially mitigate this issue, but it is still recommended to use Adaptive Scheduler on a [application cluster]({{< ref "docs/deployment/overview" >}}/#application-mode). {{< /hint >}} -在 Reactive 模式下,Job 会使用集群中所有的资源。当增加 TaskManager 时,Job 会自动扩容。当删除时,就会自动缩容。Flink 会管理 Job 的并行度,始终会尽可能地使用最大值。 +The `jobmanager.scheduler` needs to be set to on the cluster level for the adaptive scheduler to be used instead of default scheduler. + +```yaml +jobmanager.scheduler: adaptive +``` + +The behavior of Adaptive Scheduler is configured by [all configuration options prefixed with `jobmanager.adaptive-scheduler`]({{< ref "docs/deployment/config">}}#advanced-scheduling-options) in their name. + +### Limitations + +- **Streaming jobs only**: The Adaptive Scheduler runs with streaming jobs only. When submitting a batch job, Flink will use the default scheduler of batch jobs, i.e. [Adaptive Batch Scheduler](#adaptive-batch-scheduler) +- **No support for partial failover**: Partial failover means that the scheduler is able to restart parts ("regions" in Flink's internals) of a failed job, instead of the entire job. This limitation impacts only recovery time of embarrassingly parallel jobs: Flink's default scheduler can restart failed parts, while Adaptive Scheduler will restart the entire job. +- Scaling events trigger job and task restarts, which will increase the number of Task attempts. +- +## Reactive 模式 + +Reactive Mode is a special mode for Adaptive Scheduler, that assumes a single job per-cluster (enforced by the [Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). Reactive Mode configures a job so that it always uses all resources available in the cluster. Adding a TaskManager will scale up your job, removing resources will scale it down. Flink will manage the parallelism of the job, always setting it to the highest possible values. 当发生扩缩容时,Job 会被重启,并且会从最新的 Checkpoint 中恢复。这就意味着不需要花费额外的开销去创建 Savepoint。当然,所需要重新处理的数据量取决于 Checkpoint 的间隔时长,而恢复的时间取决于状态的大小。 @@ -116,35 +190,6 @@ cp ./examples/streaming/TopSpeedWindowing.jar lib/ [Adaptive 调度器的局限性](#limitations-1) 同样也适用于 Reactive 模式. -## Adaptive 调度器 - -{{< hint warning >}} -只推荐高级用户直接使用 Adaptive 调度器(而不是通过 Reactive 模式使用),因为在一个 Session 集群中对于多个 Job 的 Slot 的分配行为是不确定的。 -{{< /hint >}} - -Adaptive 调度器可以基于现有的 Slot 调整 Job 的并行度。它会在 Slot 数目不足时,自动减少并行度。这种情况包括在提交时资源不够,或者在 Job 运行时 TaskManager 不可用。当有新的 Slot 加入时,Job 将会自动扩容至配置的并行度。 -在 Reactive 模式下(详见上文),并行度配置会被忽略,即无限大,使得 Job 尽可能地使用资源。 -你也可以不使用 Reactive 模式而仅使用 Adaptive 调度器,但这种情况会有如下的局限性: -- 如果你在 Session 集群上使用 Adaptive 调度器,在这个集群中运行的多个 Job,他们间 Slot 的分布是无法保证的。 - -相比默认的调度器,Adaptive 调度器其中一个优势在于,它能够优雅地处理 TaskManager 丢失所造成的问题,因为对它来说就仅仅是缩容。 - -### 用法 - -需要设置如下的配置参数: - -- `jobmanager.scheduler: adaptive`:将默认的调度器换成 Adaptive。 - -Adaptive 调度器可以通过[所有在名字包含 `adaptive-scheduler` 的配置]({{< ref "docs/deployment/config">}}#advanced-scheduling-options)修改其行为。 - - - -### 局限性 - -- **只支持流式 Job**:Adaptive 调度器仅支持流式 Job。当提交的是一个批处理 Job 时,Flink 会自动使用批处理 Job 的默认调度器,即 Adaptive Batch Scheduler。 -- **不支持部分故障恢复**: 部分故障恢复意味着调度器可以只重启失败 Job 其中某一部分(在 Flink 的内部结构中被称之为 Region)而不是重启整个 Job。这个限制只会影响那些独立并行(Embarrassingly Parallel)Job的恢复时长,默认的调度器可以重启失败的部分,然而 Adaptive 将需要重启整个 Job。 -- 扩缩容事件会触发 Job 和 Task 重启,Task 重试的次数也会增加。 - ## Adaptive Batch Scheduler Adaptive Batch Scheduler 是一种可以自动调整执行计划的批作业调度器。它目前支持自动推导算子并行度,如果算子未设置并行度,调度器将根据其消费的数据量的大小来推导其并行度。这可以带来诸多好处: diff --git a/docs/content/docs/deployment/elastic_scaling.md b/docs/content/docs/deployment/elastic_scaling.md index 5259c1066fa08..bb38d7549bbde 100644 --- a/docs/content/docs/deployment/elastic_scaling.md +++ b/docs/content/docs/deployment/elastic_scaling.md @@ -25,22 +25,96 @@ under the License. # Elastic Scaling -Apache Flink allows you to rescale your jobs. You can do this manually by stopping the job and restarting from the savepoint created during shutdown with a different parallelism. +Historically, the parallelism of a job has been static throughout its lifecycle and defined once during its submission. Batch jobs couldn't be rescaled at all, while Streaming jobs could have been stopped with a savepoint and restarted with a different parallelism. -This page describes options where Flink automatically adjusts the parallelism instead. +This page describes a new class of schedulers that allow Flink to adjust job's parallelism at runtime, which pushes Flink one step closer to a truly cloud-native stream processor. The new schedulers are [Adaptive Scheduler](#adaptive-scheduler) (streaming) and [Adaptive Batch Scheduler](#adaptive-batch-scheduler) (batch). -## Reactive Mode +## Adaptive Scheduler + +The Adaptive Scheduler can adjust the parallelism of a job based on available slots. It will automatically reduce the parallelism if not enough slots are available to run the job with the originally configured parallelism; be it due to not enough resources being available at the time of submission, or TaskManager outages during the job execution. If new slots become available the job will be scaled up again, up to the configured parallelism. + +In Reactive Mode (see below) the configured parallelism is ignored and treated as if it was set to infinity, letting the job always use as many resources as possible. + +One benefit of the Adaptive Scheduler over the default scheduler is that it can handle TaskManager losses gracefully, since it would just scale down in these cases. + +{{< img src="/fig/adaptive_scheduler.png" >}} + +Adaptive Scheduler builds on top of a feature called [Declarative Resource Management](https://cwiki.apache.org/confluence/display/FLINK/FLIP-138%3A+Declarative+Resource+management). As you can see, instead of asking for the exact number of slots, JobMaster declares its desired resources (for reactive mode the maximum is set to infinity) to the ResourceManager, which then tries to fulfill those resources. + +{{< img src="/fig/adaptive_scheduler_rescale.png" >}} + +When JobMaster gets more resources during the runtime, it will automatically rescale the job using the latest available savepoint, eliminating the need for an external orchestration. + +Starting from **Flink 1.18.x**, you can re-declare the resource requirements of a running job using [Externalized Declarative Resource Management](#externalized-declarative-resource-management), otherwise the Adaptive Scheduler won't be able to handle cases where the job needs to be rescaled due to a change in the input rate, or a change in the performance of the workload. + +### Externalized Declarative Resource Management + +{{< hint warning >}} +Externalized Declarative Resource Management is an MVP ("minimum viable product") feature. The Flink community is actively looking for feedback by users through our mailing lists. Please check the limitations listed on this page. +{{< /hint >}} {{< hint info >}} -Reactive mode is an MVP ("minimum viable product") feature. The Flink community is actively looking for feedback by users through our mailing lists. Please check the limitations listed on this page. +You can use Externalized Declarative Resource Management with the [Apache Flink Kubernetes operator](https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.6/docs/custom-resource/autoscaler/#flink-118-and-in-place-scaling-support) for a fully-fledged auto-scaling experience. {{< /hint >}} -Reactive Mode configures a job so that it always uses all resources available in the cluster. Adding a TaskManager will scale up your job, removing resources will scale it down. Flink will manage the parallelism of the job, always setting it to the highest possible values. +Externalized Declarative Resource Management aims to address two deployment scenarios: +1. Adaptive Scheduler on Session Cluster, where multiple jobs can compete for resources, and you need a finer-grained control over the distribution of resources between jobs. +2. Adaptive Scheduler on Application Cluster in combination with Active Resource Manager (e.g. [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes" >}})), where you rely on Flink to "greedily" spawn new TaskManagers, but you still want to leverage rescaling capabilities as with [Reactive Mode](#reactive-mode). -Reactive Mode restarts a job on a rescaling event, restoring it from the latest completed checkpoint. This means that there is no overhead of creating a savepoint (which is needed for manually rescaling a job). Also, the amount of data that is reprocessed after rescaling depends on the checkpointing interval, and the restore time depends on the state size. +by introducing a new [REST API endpoint]({{< ref "docs/ops/rest_api" >}}#jobs-jobid-resource-requirements-1), that allows you to re-declare resource requirements of a running job, by setting per-vertex parallelism boundaries. -The Reactive Mode allows Flink users to implement a powerful autoscaling mechanism, by having an external service monitor certain metrics, such as consumer lag, aggregate CPU utilization, throughput or latency. As soon as these metrics are above or below a certain threshold, additional TaskManagers can be added or removed from the Flink cluster. This could be implemented through changing the [replica factor](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/#replicas) of a Kubernetes deployment, or an [autoscaling group](https://docs.aws.amazon.com/autoscaling/ec2/userguide/AutoScalingGroup.html) on AWS. This external service only needs to handle the resource allocation and deallocation. Flink will take care of keeping the job running with the resources available. +``` +PUT /jobs//resource-requirements +REQUEST BODY: +{ + "": { + "parallelism": { + "lowerBound": 3, + "upperBound": 5 + } + }, + "": { + "parallelism": { + "lowerBound": 2, + "upperBound": 3 + } + } +} +``` + +To a certain extent, the above endpoint could be thought about as a "re-scaling endpoint" and it introduces an important building block for building an auto-scaling experience for Flink. + +You can manually try this feature out, by navigating the Job overview in the Flink UI and using up-scale/down-scale buttons in the task list. + +### Usage + +{{< hint info >}} +If you are using Adaptive Scheduler on a [session cluster]({{< ref "docs/deployment/overview" >}}/#session-mode), there are no guarantees regarding the distribution of slots between multiple running jobs in the same session, in case the cluster doesn't have enough resources. The [External Declarative Resource Management](#externalized-declarative-resource-management) can partially mitigate this issue, but it is still recommended to use Adaptive Scheduler on a [application cluster]({{< ref "docs/deployment/overview" >}}/#application-mode). +{{< /hint >}} + +The `jobmanager.scheduler` needs to be set to on the cluster level for the adaptive scheduler to be used instead of default scheduler. + +```yaml +jobmanager.scheduler: adaptive +``` + +The behavior of Adaptive Scheduler is configured by [all configuration options prefixed with `jobmanager.adaptive-scheduler`]({{< ref "docs/deployment/config">}}#advanced-scheduling-options) in their name. + +### Limitations + +- **Streaming jobs only**: The Adaptive Scheduler runs with streaming jobs only. When submitting a batch job, Flink will use the default scheduler of batch jobs, i.e. [Adaptive Batch Scheduler](#adaptive-batch-scheduler) +- **No support for partial failover**: Partial failover means that the scheduler is able to restart parts ("regions" in Flink's internals) of a failed job, instead of the entire job. This limitation impacts only recovery time of embarrassingly parallel jobs: Flink's default scheduler can restart failed parts, while Adaptive Scheduler will restart the entire job. +- Scaling events trigger job and task restarts, which will increase the number of Task attempts. + +## Reactive Mode + +Reactive Mode is a special mode for Adaptive Scheduler, that assumes a single job per-cluster (enforced by the [Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). Reactive Mode configures a job so that it always uses all resources available in the cluster. Adding a TaskManager will scale up your job, removing resources will scale it down. Flink will manage the parallelism of the job, always setting it to the highest possible values. + +Reactive Mode restarts a job on a rescaling event, restoring it from the latest completed checkpoint. This means that there is no overhead of creating a savepoint (which is needed for manually rescaling a job). Also, the amount of data that is reprocessed after rescaling depends on the checkpointing interval, and the restore time depends on the state size. + +The Reactive Mode allows Flink users to implement a powerful autoscaling mechanism, by having an external service monitor certain metrics, such as consumer lag, aggregate CPU utilization, throughput or latency. As soon as these metrics are above or below a certain threshold, additional TaskManagers can be added or removed from the Flink cluster. This could be implemented through changing the [replica factor](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/#replicas) of a Kubernetes deployment, or an [autoscaling group](https://docs.aws.amazon.com/autoscaling/ec2/userguide/AutoScalingGroup.html) on AWS. This external service only needs to handle the resource allocation and deallocation. Flink will take care of keeping the job running with the resources available. + ### Getting started If you just want to try out Reactive Mode, follow these instructions. They assume that you are deploying Flink on a single machine. @@ -113,40 +187,12 @@ Additionally, one can configure [`jobmanager.adaptive-scheduler.min-parallelism- Since Reactive Mode is a new, experimental feature, not all features supported by the default scheduler are also available with Reactive Mode (and its adaptive scheduler). The Flink community is working on addressing these limitations. -- **Deployment is only supported as a standalone application deployment**. Active resource providers (such as native Kubernetes, YARN) are explicitly not supported. Standalone session clusters are not supported either. The application deployment is limited to single job applications. +- **Deployment is only supported as a standalone application deployment**. Active resource providers (such as native Kubernetes, YARN) are explicitly not supported. Standalone session clusters are not supported either. The application deployment is limited to single job applications. The only supported deployment options are [Standalone in Application Mode]({{< ref "docs/deployment/resource-providers/standalone/overview" >}}#application-mode) ([described](#getting-started) on this page), [Docker in Application Mode]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#application-mode-on-docker) and [Standalone Kubernetes Application Cluster]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}}#deploy-application-cluster). The [limitations of Adaptive Scheduler](#limitations-1) also apply to Reactive Mode. - -## Adaptive Scheduler - -{{< hint warning >}} -Using Adaptive Scheduler directly (not through Reactive Mode) is only advised for advanced users because slot allocation on a session cluster with multiple jobs is not defined. -{{< /hint >}} - -The Adaptive Scheduler can adjust the parallelism of a job based on available slots. It will automatically reduce the parallelism if not enough slots are available to run the job with the originally configured parallelism; be it due to not enough resources being available at the time of submission, or TaskManager outages during the job execution. If new slots become available the job will be scaled up again, up to the configured parallelism. -In Reactive Mode (see above) the configured parallelism is ignored and treated as if it was set to infinity, letting the job always use as many resources as possible. -You can also use Adaptive Scheduler without Reactive Mode, but there are some practical limitations: -- If you are using Adaptive Scheduler on a session cluster, there are no guarantees regarding the distribution of slots between multiple running jobs in the same session. - -One benefit of the Adaptive Scheduler over the default scheduler is that it can handle TaskManager losses gracefully, since it would just scale down in these cases. - -### Usage - -The following configuration parameter need to be set: - -- `jobmanager.scheduler: adaptive`: Change from the default scheduler to adaptive scheduler - -The behavior of Adaptive Scheduler is configured by [all configuration options containing `adaptive-scheduler`]({{< ref "docs/deployment/config">}}#advanced-scheduling-options) in their name. - -### Limitations - -- **Streaming jobs only**: The Adaptive Scheduler runs with streaming jobs only. When submitting a batch job, Flink will use the default scheduler of batch jobs, i.e. Adaptive Batch Scheduler. -- **No support for partial failover**: Partial failover means that the scheduler is able to restart parts ("regions" in Flink's internals) of a failed job, instead of the entire job. This limitation impacts only recovery time of embarrassingly parallel jobs: Flink's default scheduler can restart failed parts, while Adaptive Scheduler will restart the entire job. -- Scaling events trigger job and task restarts, which will increase the number of Task attempts. - ## Adaptive Batch Scheduler The Adaptive Batch Scheduler is a batch job scheduler that can automatically adjust the execution plan. It currently supports automatically deciding parallelisms of operators for batch jobs. If an operator is not set with a parallelism, the scheduler will decide parallelism for it according to the size of its consumed datasets. This can bring many benefits: diff --git a/docs/static/fig/adaptive_scheduler.png b/docs/static/fig/adaptive_scheduler.png new file mode 100644 index 0000000000000000000000000000000000000000..4962392ae58751964dcc1e8d267ac54c614f3ceb GIT binary patch literal 538791 zcmdqJc{G-9+ctb65h+R%nNmun_+==Wk`PHKM22WGH<-#?goIQYC`u#=k)h19W)U)U z87ot!GDY!j2fFY3dB5*j-(TN)pY>YnUiYu*y3X@B_G90-ecQJE2;8?BscXX$@_42P5X!r;LF@;S2N7t8(`%WS8k#S{{0s*+r4N0a+QamkPux*dGLk6zy(v2Ez`}_sTP)& zZQs9N-hE9(NKvui?m6{;eX+}i*{2wL9vi#*^2%*XCdUTUl$5BckrXdl^^mQt?Y)$g z(3daO-$u#vT4zi=40ZbPW%m=494?KxA2%lc^*Jh8p~@$lci>@S+!_blN>2SAKP5NO z^`JiOC@Uw6pRL#4y^A)PWcv;eJ94jBao?dOzpLs2)9}cMvD~++2WlMb>{Og?pi^(% z;kIHwyMBYCN7t-expJSL9s?^Y>-eu^wwDDRwzRwu zzQMpcXT#X_n3z@9Z{94S(f(wvEVzH4?%cU^Szqh_J|G|C%-Y7d8LF!@Kh%y6*?U)3 zNI`-B;lqb%e_GssvfbEt@chKYxdvwj28MZzjEB3c;|9mZ97gNk+z=~?kM?A7adBC_ zX3Y)*gDfqN4OslUcTJv6+S=I-jdz50bjL@lMk{zUI)0A4ub+7T!5N!t-?@2tAK*ET z9z7~9Az>9CMKj&CYuCPYoa;nIcX)a#XlQEcn3%8$&2w~g6t}Kfw{`2*&vgpFv0>$* zicqFJeM+TK3NR4mx=YeI63(R1+l(<{rbwCI}5`#*3_Pz`qeq~^`@D$mGS=l z^YCW-_wR2VYw@c1-tf$rwmHKn*roX_(?G$rLe$xBD;BQY%Kq)!x99G^{&DVW^s}r8 z+3gg+_wJ=6a<{k7Fn`L6C=H;Rv$&=e!LWx@mPsvtoigwc*6$PO`A6L)t8Kh zMMehI*Bh3Wm$(1^^>eHHxOjDSb#haARTamI6)QgRIe8}D*kp1kC4Yh?EG&#$(bGML zKPNq%Q(9V@UD>=b-Kze@nYC1F&Om<2M{PwmYOCv@VEN@0ACo)R%q>B7%SnEjnF!p! zd-nq8fo8TdXU=Rie0r}z)~4=B(T}fhBwPp0t(wY$mJbbo)b1z^TuOatm#oUn%rwk( zl~5ROWpMxTYONhh@XsGV2B#(m7$|pl_rcF+DtAxb*lZSp6XHHxr8eH?FF@A!{!ah) zmtn$BGcxQ(wi($BH0M1^PA_cWic;atakeJ>7bFzmoJ-JS~6_kzMY$gM?>#cZ&#O6!v5yE z5A|3i>dLKK_5JsU8lD}$|7(%0p&|E=pFi7LTLmX|i}5J`PB=7bE)Z()96DxZp*jy# zi(bBb)F7MZHa4FkTYXJiVF%8^`@7rM_6;;4uj`o&e5>e#L8KG7Y&_3V$;Bx7`R ze0-e0ZQHh@#zvzEu3tt`QBjDkrBUa4MCZ+$N3u+$v|Vv;NYRiZ?^XK2H}essJbL$|JGLXBS()4iHh!KS-VqNIo6rQk6xMidU!Z3 zGpt%HAt9mY>ERNbdSSa>JeKNZZo0*b7nhcn3a(x~@7h{zRYOC=YZ=K+#s?0B5AX8z z_NKbsBQ~%3^5skV zK}n7E2wQJ|eJaBSgXHUWy`NG9Mh9BFXjOS3AtA1VMNC6DX014hC!9_^*|!e+a^dRL zCHVfdiCnnO4{0sot&J+rKRmp-@#v<$Hep?CWhYA)3d9k zz$>z(GT9`@qOUQ-FvmG5Jqdx0HD9|*SeS#4PmD1FG5GS`JHL>SMW*gUyVixj7O9Ao zv|1vv_fCKh15+`w9~T#wSk(6Q>sjKpqc28A@|FklGV${AHcX|bOFl8mVCj5+cc+O7 zU$)CY2qP<(`RnVVGP#?Cg@qlvs~NGdw`3gnZ-^Ps*Vfjyn!d=!%f4jEdBjIcX5lz; z>iEw}#Zz4$gzjuRqskNfr{Q>XTiJ56>$P^d$(bUpfBLdo1bqv!81?M|W4+tlMxT6l zI(brDHDqO==FLsC(%N(_d;3jD*!E|WzDGK3S2Ui>yEQ}NGo>+G&8oOrFMjX+{QRkG zy@hUWZki7)q~1p>hUR-rT8#A6-ct1P46=*%oUjs=^c=_=Golq_XUkIFFJ5fJQ(wGv ziDgf;0^f}rH%J2Au;JRo7U%DVazEd0NgFlu7+I$O#L%a@TGuMFpg>VsS=s#9vAQqr zjEsyQK7PEAkul;lxeTw1lcV{EfkHvqx_$ct?P!G$A9E&zf9zZ=@B8S{x;+t+3oOeo z7gfh;>KrG7dv45!w}@0{-mu`uh4>qc(L(`OOl^%lP?2?3yy~KYGNB z-yR$pDbw7XR}*XM;UQlhDiFHUf6=Rnk1x&~H#~T-sHrKZp(Fm`LyMo^-i`O?3|zT; zmyP=H=~G2wO8zA}hWY8)(S^0G+i>6tYZX~hT?j2ipcJ{lv7fYkX=JITh_dJ4aXxD!H_~EO3pPed2EIJrYzrDSM1bF-H z<53(J=Cduic<~~=HzoM({oM`gH*AQsGdOrq)X~X_kd>Nq+~-x&yF$XkmlBM5@F2x_vy{{U z$NtX;DRw-X1AhV$@4DC<8UV8F3gUXz%*)5eN2j#WICGDrq~yyUq*P7)v697# zoVRb^e#bwFjKM|uhKGj_pJf!3lanjWv}|Yy|Jw12&{I%|E)uGty>grZ)-khRkr>UV)dOX@g#BblTWACg} zlSL%SdMAcD96l^W1+ei-2VA^3pR9qX`k7OwI8L29^)8cj!2%A7Y`1{Na9FxWb?j~| z!)U{C0P9D!)~T*T?ILt{0|Nv1CG59o%CuXhpkT@@L|(e9;aNPpazMeXsVSg2xv=HafYofLZtUR|q2P0LF%n(^K7CEnv*4?;`7zPVY3Fi^o4n~XI43&bt{ z3*zwSelK3UpaeyyOBq+ZeY=NZ$Kg#CmOMLNc|$dyhnqV9!5&C}Y06qbK_!YE zmDV=(J@WIBHnZueKV`$+ALDH5=6zygW7FOGXu))2)8hk)^QfU8Z#J7{HREluJ;#oT zY%-U}Czi;f*TgOSlvK{|`-ussL;bN_pir23-+S=P!1#>pPI1~RUF>xuDAto(N z*Y(^v7OMS|nO9z-!rw~fIDL4g?Q;xm%J<0OT%v^)-H&GA7x(VnI0@X_oo%mqh>D6< zcfIFXKdIT96pPde2?n6;=y*nXL1(jEQOo7KoA3)?YaFABi@XGTB+d9O^Ksy`}+B{q4@zg6U+bgjkmO- z!m{krvg%V4!`;FrEw95jq}nw}=T1%Zl9JuxHO)r3BHd6Fz#`?1G{!*b>gnki8?&03 znf*rV5D*kpVi_twAFY&Dq;yeBi$Gaf**jD14!2*E0|gHN|8RCsoH%hMq`;|HA33ir ztHrZqZM6G&6u3KEPtki!jxtg;H8tuL=ohU<iFYu zECrN|8b_yF`s&)+T^1HG^#xu9fhn+HdJqf)(VR&;Q12)lh-w5O-1v_91` zE+OjElY_!Ny}i5k@4pUsY*F{*V0%r1UiHv*jkt8rjANHhQx<5@V=^=K7YZmGa*n!k z$H9XK(>e-%eT$^`9`8v^v7b<|Rxl6?2Z2L#`hVh&P+MB|0i)g#^oLwP1jX;) z_h;X~{t$Wg(w#dik?b^YYz)hcjEJ}dyqfXOuBYHBW(kdP<_UZ&Wqs;ZI-6723R*n`uUVyvB{AN0x8O)KJj1CA}I1|{jfGst-9 z51-w?e_vfgV+o*B^>Mc6*{T`Q(b3!r?$6WbXQGepqUB_91DHNaNx>rP7#XdP@|@_7 zToiom>Q&$1V6AIbm66v_@Bo+nlL&o@K?NgE>e+CPo6k$bu?P zjXxO2e?+WQ+^|8@(AYQ#=cfngklo)jmrXxrs$qbMt{AntX!`A+2lVqXq_*6 z`#Iq-LQQrxpKdE=Bs~E_vw}&$-5cozX`Ksi|J-$G6$>k?&i?(r?d=v|w!pLTEfv{Q zh~{Y?x=>09bb`Y4?=zt44rAe~hu_;B0C{I=*}+?9bK-<|R1`mD4r;G@xCFRqICr&Qh|1JotFPX>g?IG)lN0FwTJ&qj2J#U z#s*Y(v=2B^;^=Em+t1HaUpZAq$}Bd{wDpOLTXXDPgdfli|DVaRo1<%2uQu=ca3B35 z;X=_FdGEL|e@DXpb^sN;u#%Ax55Ur?FE2Sr9Y73PeWlq(qMav4&h<-}+6Pcvymcv$ z9w~va%La7JTmDJbV?uHzA79(IZ}X{u;9yjV4NpOdi8uibT?T6Fc(pd#*C^@6!#8yS zmoA+*O1fB3;MKG_na%WE_ZoM=Mc1zT{!dX&D1-RAz1rIT2??U#zkiRgGcYo`1cD4J zXyRftO1Q`j+FOs@x&*y=aU4F!-a58duU=98$(39KD!Q0x`$F_EDJXh0~ zZ{LdR>N0mTKgOR0c>U=$!~5Ae7hJf&fQllTw9G>h9L26fhgM<>fV1gP0>BME)v_^f z*#zPRFn7b9J9ose&H}c@%-Mg|%#p%n) zz4z17mH_Pgq@--7XJ9a1$@f?{!Q9q%Av$(WF0SS_{^^~}J|;QNZGbrHSUHdb=Z3p{ zBrMB6OC4TM%WHNUkl$#QzY>hVfooO8zinriiKD8oaF)dHw*u8zEm;7Z2K1w}Zy!6F zuk_DbV(LW2#QYGWZ{NP9es5u$K(!zpiP%`k93YXU_hh?E{=5I6-orgRP#;Ljp$tBq zneQ#M^WuqT#wtl%e0*%jj~{n+bye?OZua~n-Te9U0T*iRJLf*+QOSpo&P+4Eu-D(S zXCc*!W!VKp4D3Kh(Q;fqnZ^e+3WAmftO4s=_DJW!V7+CikhC-xKR>@%a-`N@UZr`f zywpqxJOgVLx;Dy{9!05IT3%lM$gyL^RaH7c^1{}9=ugq+kqjZsGyTc-)zvj5S*O=; z`P<2>|LbGQL->Nm$6a%O|9JDEww95~9d8d>c)2?YZGboE4SxSce99@Nc@zac*7a++ z0aoqbA5uLdk@T~YOmX@v%dz)&79ef)^|NQ(Mgj*~kg%#=jq`pYXZzn5@R@B*=RO1f z*Hbw6EOVR2!SO#;dzstKfPnuQQqcczkHd#FrmCz=kM3HAKvL@7{`H`47E#CoWls)1 z8w5dOlX`P3v&l&jM>MXp#dA^&dmk7R6JxB<-zV$rM)OB;9`31SMLEo>i5_*%`t!mn zrILPbkDU(9s$%_?$mjL>9^3gn#}5&+WhU`ao)c{5d`t0`-`{I`|MZ^$FVf7vjgVp`>1_S_^&b-S)^0S-`xFQ!meDQG&MD? zrt$QlS7mQ@e?8rX3&6IMcZok4t%L#1*4K z2nE#6eN|KQ?uCg@sz%&ZSG?@M}1Dj+0eK1KAq-A%wnfO;HM zYg-%kbL|Hh{HUn_6jzMav1tG6y`Cqxcp zkSo`(&wKYp16`^vpPw?F&^$<~5x&m~?|MaR-<98GuRT@LGs8Aa04I zac$J=_YhN!wUl?FaLC{FY&>r>;|`!5|4gGDO@SZ_q02{c`nR5@JUz}xTql(77v0&s zGWMnDf6$x&B)rig1Bi6HYR@m8q$zl=tV)hf2%JiW(YLAj;Nl+N5e~%1`(Y zs*v=weFS%-i&M!ILw5zFyVbFaeLfqHFUZ@Bgw22P*I!|-61ikX#-z~3k)@_Sq?i>* z^Qk52TLLd4W86*GQd46>f3*W14w7O-^ae#lL{=f`pdYEu1XUzD_>ZKdZ*;U-Wbcje z@V2C^W_oHNuk=Dv!_j$xA3|Tgh_G&ek>?>^($mvliHcgZbLY-Ix5Vj|Z#Gqd(8eWZ zL|aSKq%X(tDv5|k1K&A{N3$k;tGCx_AxAK1~NYbh6md;#*fN#6lwwEaP-LNTl) zk-lX-5g!bW&4Ig5=AmC?s_b7KY#u-evGR=5K!QY>OQGAZ!RZFE%>r~&!2M( zc>R`GxNzaqlWod`)p1@49-f-s5u+(@GP~;?-6Y0C5`;!2{jD zjTk@NMmJ*j8=dL?wU!G=15P_4yGTk9Jy|KRPh1}o>@sQ| zRRut%gL>Q7l;@U(KSoOm`uEA^iHQl_9B^rw3CZlrPYhF)DR#gThtx`hZdhY?PtT>P zKa)WqNLf*0pf!DO(0iP?e|;ImaNoql#G3M^Cgb0uO%BR&y01b24IC}!#1AeOKYSK& zmj##&z`wlRG#pt|iXBQkmooFv>kTQ)RB=PYK~7Fi$%N^G=|9>OJ9PiFJ~}$Ov`W1* z{Wr#5fPBLrzkGdljV4~9z{@N3cqRYAKWS;N^;|NW3#mj~13lH6$$XWwtk7D$XAxO$Z=fyE8%y#NU9rrI7p3u~*U@wI6a@J6`e$Waup zwC(eRGjprRF>A}qt|at2C?CojTpb(eC!3!Vk{{^>l2a8$BsReb+3J(_193VFkrN8me*tz^aqsHHxvQlUl@)(#F zwWe0^Upw#F+A}$Zeea07X?m6R>|0N!3bPMnb8}E7-&&!pcpFfTq$+3L7 ze+9qWLR4{Bby$K40&hHA;aI{a#W~t3X z1Eyq8q5HE!MfFH?(!ev3g)XN~SvgO`TtiVXfE7dMZ0yVZdyaihINjMo;x{9QMD>i6 zj6EKG7(#NArrxmhCKS$9Tej$d%nL+nVxg3wt^z%)!YiXW0x%+kvkr~W#%6=o=3Mh4 zdO9V!vCn6cl5?E1gb#@wBH&y`qtPPtPi(n zNIOdJ4Id5{O0(T#H+iJaWmGAW=b4hJ41HbQt8j%ZrJ(ru&C;(!rAxKd$2w3+vh8U% zz%*%8psLzuDTMNE*9U7600g8A!datfGn7kUV@R6N3|NGGi47pOGU%Z;j4AZqPqFY| z0+-<}l|ha5^kQS1zT81O1APEee;H{1IICpvgd1u8IlnwSfiUJ#sTO4{R>phxhH%Td zKAqu(<&!ZwTLJ-LPF!AC z%Ta_BtmNn@dC0_sus28>f`Wn+Ae3Rg`!=wc^jLLD2~`oU3<)?7S0Y~^#O6`hiurr@ z?qyd_&$dV3e{Q6Qk$Sgx_d=??yd07#FwE5NUm$oio{dz0{CMiukE8Vs4Ml~8R8dh8 zo?iUi==VUFIW6DblBi~`0)t5e(uDqzJ$JTp03gFRqtx5%t4|=y&YcW^#7Aqbr6@r~ zMYG2bZnwXP%NBb9k0DhEw;R;bzPe6g(nQO$7ZN4xz=36i6AN0txwyQ1K3NM;ZG>JI>BO6E#MfR|3+C|yz03_b zVTj@gNtQsa#)SG-|Z656zBk&ccIMWM3QpUr>J#}`o)zc*&EYQ!x; z|Hb{!5%p^mAQr&B`T6+lNbHS>h-inouH1LQ(pae*DdBh^SQuiSJXr|zW%$&b4xnxx z7(Qey;Z>{Xt*opN^QulxQtq1;9fnBrBUun>b`(;6Kv) z$?{!740obH6HEaW0o@C)8To zHl#7a_*0D6s)AMOP`2X&tVm;jqusUt7v4G$)oX9#4Y*IAKi^6y?&4y4sJ7f&PYOCZ zM(ln4@kaJoHgxs*guVJ!8Qt%lhwC=DQV@xm{`I*Dj$z5`Yxygt~x12F~Ie{JlLrg1+AD^i*s` zc_ptR&x#dA_4V91W7TJnod|7;-V})wQTY7l^1pdXg_^Sl)3R{TuHYP?e}rRJP)24s z(oNdF@kZSP2N==)K{_mHY~;aNg2L;AR&)4-wKd#$Y~VboYRit!U;!IzYrvi$WUlTP zx(O;fcF<{PXzYVHIGsLyJ}D`Qvs`+Sztk>cVb25WmyxeC zH4P~)Rt6QT;d2^BZ|IV7)FHjsZ{8414#XlXek+hg!uV-4S{oc7OP4N9I<$E1h*ahI z&dyX+ne>2(fPeG!HS?`vVr*!927mm3t)Gw3K+CjFHoC61hoP2yqrUz{bX2@IfNQLH zdSFbgfy1RzO87GGa(p;`LGNvw10*Ubmhte`ty@TQx;#5RyQ&OjGy!&}xOtJ44-a2M z3oQ=?=Rq`*jbdU)#~uRCcfsf^Cof+Gb9U1d|6k}zW1ZVID;-tU`H<^41Oztwl}1#< z2MB?w5QYSn3sPm^0ASF5`TqTl@Z4NEK$jR~UHD$tZQQ5=?>V@>!?XI?!PNA0+q8Gc zeL#rfXx0IvVKBw@dXp03QMU^VueH*UlL|1Dd#teV;K zodWR^_tY%FBmh$V^<`M-t=qSkz#j@5gjjP6i2j53?nFg}pr{)Tm+brYk=V&90FV3u4dC#gBxB{p<`2M-?C3?(6(Y(1Bg$ z*V57=AAt`1K3OmFp>lHQF{*6WJrM=QI|)>(3-htLgTrDtiwPz{&;gP)=Kqe=J$Nt> zjvo^6n>TCg!!v*C>+5B3w%)sUPbuPIHkwM^N1vZdW=*9+szON#yMEpG`gP7j(C;AL z165xL4_^i-w8rmcgqpFjvDjzyt1;WPnSG>e>da}`PW#q}V*@URgzSwBkvRT-rLVug zC8TQN53<+OCZyjZ`8l!6S`AbOX=YD#SFcF`;3vyxYHHf`F@7;wD9q>pEefOMK$NI! zY3*`$mWI{}lN$Z;Lbxq>Hv4gWXF8RbogEOeld{*;a!Sq%v{ zw>+1{z7hVxTa68$H+|fad?ObZI0}@)%iO&Rv^}cXh&cRjcsjgZ7;)B*EKVK z(T;JCa_~lsSWLwA3zLEqwI4osw-ha zFx7$)bQ2~S7TdY_BOG>`&rXmzGJqu6>e9awyp2DY&O6&(7U8s!hn*Ph3xYMC==O+0 zC?F*mr8uIeWfNs%iaKCDd9{m&=qtl*!J6o3Htre4DyS~Eft6NNNx^GM4!TPa%*2JS zUM&E&zy2DQ7ch{?Wr?d-t+KvLm@{yd(0Dh4vC?hGN4X)=FH`^@WI!BI7&R{v-4{=Y zSsXdSA5FmT2M2))^eAGXeH*PfePeRDfIz5#=Qthg@(kc9V*8*jskpo64mYAUq34)2 zU%V2F9y|8=y#go@L6BJl8Nq{w)cPS#b2<6qThew$IZ?fPjbZRG_4?zKJ5L+FGdWyTBR>`CfT=@BQ1izFAo^ASVbr z(tQ?3A>Cs|&0a_%$f2P)2BPXAfq^^F|6tcKokA>o-7moY)kp*E!zvV}f#!#hYw(ei zUV!CV#)zgZ4n2km&&-Fj{d{8L1{jzRpE%)KB8}b}6HYJRyeWbC)>n=HtV-rI2+(4L z_F_1-NFHJFYkxT5423Coq|O-3`PIyAva-Y`?}IFSZmd6;iQiSJ@i=-l^i1H%tn8$3 zj(#sehZ2B(l5}+_#&AS$g-0!-=2?2Wij`Hj*j>%0X0K@v@|mvRZXbUNWB9RNc+eMO zmZbv1UxfQNl6(4Q3yF^ZvgOOy`jtfJlp=#;Bt?;UmHkTKUQOF)xOB-F=&+`nN=C3E zTtFz2nHy6debVtt>1oV(st`%Hdxp z9xVuiQdY=X4EP_jkhoo=}LJ)mn5fP0c2B$mo5KQABlnEH{^71~u1?iM6ZG47X+v0Jyb}4|SxW3Y{l;2w9U< z*~m!aYgTb6@b5UHYWfj|M<0Y)T3CR-+{hdJ zI7oD(o|=TEkn2vi7Sf^3k}+N@EX+XMI-uVlCniRI7XU}GH%d;li8S|s6Gc!I0G86$ z@Y%0MEkKH)C?siU>Y2L<5l^sFoNhzxpV-~k#80)WkU>Zk6X5F;BQK1v-nem*1U$$W zn67q?_0wji3X7Noa_u}^yW?&S{_2eGAW|K|vHoEu@)WGf4om)#0e$6<91g(DKp_Vb zq8|fJFn-KW_s?jPVct0@X@};^L}NslKy)g`N}c-lb_IbzAY{BT zXh){kpr%0V-4UmYVJlu`X2OKQ-;~y&x@*^Aj4}~R3dzp^67Jx(QONyl?T#RY%qY~; zixtKxXeQD%)*0!2Fd3J_zr8BYG}Cqk&c+aU_6Im_X=grR4i+Nw*RNlRruZSx&1Fxd zG#ivT0Ak(r8XYY*(XD|-OGxI>^>gKBE_+qv8X zOt7&j3tYOE82!sI4k`-V5;<|ab_m3E93=?O?o5f77b)#@ zN<^-esm(%}k$|)NFt8#8X7}AYEwE++@9G5pPHY*=J%p(s`Jf&~q(PLza98!=!-v>+ z{~u#o4tfGcy`Zo-De^`*t?VRyd2ZljAs7<=-+9z2ST%%7YojG3KS`ebapb;kf-k%B z9l5i*3s?N3Vm;8-CN%Z?r*AJwAx5HB~Gog8nOp7IAfj(OY# zaKHMn(-Q&^&D94Wr`*v-8-i*;i(i6J3ODK;Z`Ms()eCWR20Ju>t%}s6rS{H z3S(H~CDCgs;yc3JkI@sEV{f?^@<=ix$9$-&DuQfa|NN~#)I>u0PCuOZ*3of+j35AF zlFuRK9Nh#Sy#t5TLARx*hLb|G-`p+{?b^QJiE-8I*cu%)|)#}I@TG7j_z&M9c~ zNW>@683GE>KcCoA4wasc!a|Q@+$tDa@KR!1hSdUnx{Eu2X+UVGKXM)nM$jL8pwcme zEM#7=AWSu+qP(2`x$HlaFbh!>nbOCKs6snORbSNFbG?>g2ND}8cwv<&^`L@C3Qf(8 zlcTU4SVFm=1hsy7pnwqv8lWW_$qUN&3(?TTCDc^j*~S3mK|Eh5TOf+d(1DRbUT9Y{ zM`y3ZUev*YkAGdJJR5M=0F(esKb`{oBnSii5U6HmGHgn6HBvd^xW?Ha(0)5251C>a zSnu%6GK}e?NN+vWJ`ZN{4d|E6O9ME#rK~ws!JUJj7)kY4?6-(C^IxBX&so*M&2 zTGWL10Cj^9i@;#3%j%=grOc6;(mHP6x@C?aOtX{e*=dY|LPsG8s^YZ;axun$*3;nH zIglZsV-jtf$RnV!CIcuC&aP%|KJ=X3%zcOv!Z$LIH|91NDNIB!H*?4PFA@YI>a2`r zqA(Uujj!Drje%N}pM`vP6D_ZiI)vc>EBWuBAt-~EXQ)JPCLFQFGoDIu&)=>M`E|`Q z5FSXy#hzntspL&23>3NW0&|8lq=K(6@HiL3p#(}MU@sdKCfmmJ1xwfNRY7b)68!)= z7&la8oMmys%1_d%&6teB^i6VIIi_)~#8YeoIgX>FqvQ}_UJ*l^7KiYsaM&Yqi?$Es zURbbHL|pvgqIVcLuaqBKN%}nUHw=;w6~Wp=s5f#Jh#vIMQ5Lf9g@sBhWgUqVgb*HZ zO8Ke@cyPk<N8vjg?DSr9+3ZyGWs*vR6Hmk(ZKbm58_Ukgl=3X8l zPuy5S7#WC9Fz|uVQB_fy2d-i0oWSZkKd4+xDGVjPw#aZFO6@Xdo~ z;g;OlfAmvKgTZu?$ue@nfhq389LF2@y{G5#SDflqcK=|c44W^G0z{$%3RDFN9+_rW z2I9hyf477=X+gd6&YeT(cu47}@P8LKYC}3132n%Me!&KsFJw#JT@-AQ}&7!%Ub- zQPR>1c6@U6>Q!FP+e4}zT`(0U_U}A+a0Qqnyhtc{O0X;>?d#}ppkE;LF5q}qcQ6B)YD|&)ci?I%#-mZH#C~)-o}3f^cx^X$&Dp5qg|nLKQQr=m-93=l>^3bi|tgG&bTlm(eMaA%5I069oU=iw4WH zZf-$H*04>j#T3H^iWzKS>alk`GbF1inWCHYdZ3VAhY&Gx5}=qt*^_<F5zs+6zC9U=o#JI$o8CMs+Sl*$gt}XCz4t< z2E01{fjTV#<0U)`@W{=~UK1Ap#U3f+z~%L4r zU-vse2f8I?!v)_yMIk`D&_}}(^Z}J>IdHx#0N(`>JB%#1Vv<^`Qx0$W8*WU}2kQx7 zS1v<3Wh0y(*j;oY_Z~bTw>FV(86tc+45<<6+a0_1T}yc8NbP>>KX-7g3K zQ#3v4HN6<+o74&z7RZ!Wg9O}eCo!+wkV&SOWo3;bmQ3PXN%JEX!{o-M3AvqicF({pEUl{j2cSYJV)?FLKFwA$t^NB5HjOi^<5&#?2-sG)`;oasjkIZSM zqxUKWWfI|gC94T}{^`*-TyQaxxm0ppLZo|ODh#o|e*NmIj$<7jjxc*SfQJJN^@TH( zP~o7=i1)Ow&cE)_lP9<$fe+@d4_OXoRSgciIg_4stQfoaSs%+4s|R*1nA zuRr9;4v-M~0Ak-Qk@ zFDyi4lHsY$S4P#qM>2Or4l&bZ7}47Q*-ETWSA=$wB#$E0;usB$r~l}}8N*)JMf4Fz zgodyIgr9*d%DH^GFzh!Va>+PX>({Swef>+S#IjGeo#zy^QT=(MPL3b8l)ZPh`T(%1 zsH&2SBUrAi-E)hAl>#kWK!Cakv|1!-qr0LzjXiL|2CfAtf|g;lte#bfP;|t%iwT-{ zb+@;j5mA`@{*3EWTAlA8X2!B=Tqwd$ojv=@PTP|Fmc~y zUp6})7Xg$M&R7ac7_+JKP%cTEd>JMvmQaW%_sDbG-GIdcNzfd0J2{U&%n4k^K2Pc#OH* znF7!r`QwrV6*aXER4V``@-%RxKShNjjWHMJQk@Fto*Ge6rRD-J#fyvh*8ll~S&I52 zoIeOt2XcA4!WvODZ#rji%i8_;cqUMj@TdxT;_kYI{BnkG^5B@MUUYHh0zSSEeyq!5 zuRyEjfp!gUI+R;_nnz#?AVci!vO37{lb(dl{QOyGZ;t2jC z!0G(?^Z4>3kVS2BdI8jG=ys{8S>hNWQx=klfa?)4DGZqbx|Px0a=f*AeL0{@6?ZU@Vh?lRVz<%nKJ3a9IdR$9 zJa}JnlNex29Q6W3Vq?EFe#?oST+#&=l9f7n`gAcm^ot;CIT0EdPp&>a0AMh8>0y;e z7mgu@F3Q2Ik#mDOL^w&a0?+y!m(ApHh$4kscc650!=#p(vqxnt0r}7Z>f6j;0Z)a0 zD~!K_)@6sQ>sAyi$jY%4w&Yb~|37#hS_P!bD38&DB>kTFoj61&3wV*LEh@>Q1kFf1 zRUQNirG=Fusgn(fjus%-9Y7<9L5*Yy#Jx&H>9uo-43Oo-gO*k<^xDm16+u;dV;^Qneips>ABZb z!)s>R(=K7Ypa>TNG|^DRtJlv)+?aPc-6^!?;M`)vO2L*@r>B70NXrNezI-{I=CmqC zKL{$3st{q9CMGw+iGFXSNB%AkbtmuCWqb4Y4Ln*#c-q zbahBj&^5nOGFa+778uaD3oA72LL;$#^r#;kttF^u*b|KThSPsRQQrGPdxm9Fah$Us zSYdwCcFB^(I5RAPZV{ozRk6SXVx(A9ehql{mF+cpTcv^0_ z894>V`kD-yXmc0zWyjAF)7d2;AIWWF4UQiPgRe}=6Y9@CmOxxgbi`)q_HEm&CYK-@ z9^(oH9-my70aJ39Ai@d<6z}Z~9$1nE1$?Nne-oY`KUF5P;$RiuN%V2#XyAU2642SP zCtYEKMiA|UD9O2G0K19j)>W4~@j;Lz*REt|q^N@da)56bl;?-lv={>Y(T|3(6k6xx z%&v!u3*YS4WfcMRJq2$^$}+y<0;E82Z)?NRKtSs-X4y(y6<4<6SpobT^XGvUyNYS> zB)>a%?*?PZA3#1Q%cJ1F_1N26K0xvqr8oKa8f}+<)M>UESO)0q_aN z!zCe)(GJe8n$_H$k&%vl*FOfUEwOocJp_Swz8!MJJY0f_NqqGAmu|RetJDl(?h8tk z2HojT$tG1O#s@5`iR5fWw()LFzPU5m6QW#9HJRNwCi3dk7N-CHsi?+{7jy{p@FRNy>ap z7ZF^9xjx_qN)X0k9<7zQUk!xo?j5QA{{A;o&Ibe-2}9k03y!P^grJAGxDt-o!ioyD zh~jKW$c_CXvx`w>H+z~Mg(16}L(2Pf_cYIQo=qptw`0|%O%x{R7sNs!`b5fXB@znx$rhN|$(Th+QhIxBuq_5Gm;BhPCrXhsKD93cW8R0t zC}PZX7K#i`AQ7gz$(T+(S5%3Z2#^;V~$(0=2z%o;IVa;L#w z4(wJ}SHkU<@M)g>UM~SLm{cHL@6T{P*|`9XlAlMrX5d!WjX$U&hOjjgJ`4enTX4_BG4Li2=Of8Dja~n4x;p<74&SX^Z>RXAR78_IY_}H zy-Uotf0d+&H)qu{h(`tlz)NsNAZM5h*Kv&zGG3-j0{%R2s z4Z0toL|QvKgc)DLBhc8-Kf8cVg0tIo5wzDJFsM1)!(>*GTmu3cj>!WB0S`_~uII%n4| zh|GzT=_MPg0AdsHlWkT3BZb(^Kyx5V3SOu~TVC4k3z;$KL=UE-QH*a4V{inLFIu_P8`qxz zrYf-nqStA($RPxpT|55vaPKE=mP;P)?)C|!(vLkRh(YQ7>(|OS59D2u<7Vz@`1{SF z7R0JlQ&d}1(@I9|U>bpW;|OX{hYu5fA-Yz~lGJ?bxU1;JkM{BUc?%qD^y=z|Q!!#5 zWH*hwCrQDA)dQg)uG|jWU1Sd{^4lMB`yrBY8G7+RTo5($XjdjjdgJ^0XQQS{e#WY) z!!$~SbZ~(cz^A04fNH>i8C^P|D)FYpz^CA2H~N)g@CyB8uKgrIh7STU zJRX-3i1FZse#)m3Z{qsv8X+>LLyQUtB#e13#!@?<VZzP?_a?lioAnK^TJg1gYoWCXr|V~@Ll0^qzsM_-Jt1RUhWSzi_LMF9+8 z^hOAi&RD<_L>@+Z7eg3_%l+HMPEpuTj~t2RJxwLD^Icz#Nl5WVDF5f+LP6ct?nhL(&T1HGei z+>4L*2S-IZB;qVUY`TG(Z9hElk3fUt%MAVGJ6S9)VW7OO^XcBK&B zOe{0p*g_%FH0$W=lWX5FC`SlsAUAU14=xC}FxNKY3Nl5im2f2FW-AVL5FxtiD_5*o zfKMhPvWQ(c?sBC!6SH)qQj)%A+bLZCJ?rHq!_5{DUUA>xM&M2+0RaI!m+*kDi(r6t zV2Z#=hqSf3lGh;-V<$(C#tlCb1=%gWWy=K!W$+R}N@l~o2_nuD3?upAI!)3htEglZfn^|k0V+MY=ZaGrJ2nn%T!9E@imK&pzMYXlKMwtI89NkOkWjvF96}6a{7Lf!|C zFrrBkW)6mAI4Ij-se`P6ilFYN_ZV53Hc5CGw=sM4I=nb#aFP>K2;t2yb4$>pOcT&V z^MgQyyXHx1hNih|^cKFBoCjQK0ZYnBTxy243HRBrme0z{DoFaHmsnd}Q-wxYHvz%u z;gKgP=G0N`%41EXbtk6WeQyL`pY+v(+S)0!lr}6&^>h_1@o}BFCJ9-Ug511WeiXor zRykAEm}bM1T-bUFs4RgHYpRbrZC=M0gVUvmTCX`WjI6r(7JG$q-NRMw+lfW z)@M6y046J9$BuU?%yChKS4E=R2^%zVIpN}f<$%!ycF=?nIpEFgq&f2;-3BmcAhmgOERa%x;~;TF5{+l#pJ| zdGvV-&4b_bh|OAFZf?oj{KiYK$Qamvo!$$6dB!swCBH%b+t;sB3$9KMcb|DPz%a8v zGx*0z90#rpB4P+aluQAGjtRRI8d|9(V8AS-^}m>U4{)si?|)onla;-9$j&BmXB8@x z>`nF_*|JAg%E%@nTagixtW*fePGm5J-H@`IwXvW8Bf<5`#=!Zbhq`(nx1#%8@>kTu zc>MTwS-&Z$0dabs2unk;N6n6sUh1(63dVoA|M$!X00N*nK^T*q{IUg;86aFf1MMbP zSJ#l5noXxmFpZABr`J#_gTUr0A6TrYX=o@x0ECBr3eFjPT~~G{VXuMaA}m`_hUf&d zj&Ri3{Bxk&3MV@K$&>RI?<^ z4MYGARPP)gZ@P_MuM_2@!iPHA+D}wGXjfdRiZ&^!Ss4}*7N2{{)jOd<3bczu8b&#NR4hISxVIt|rPg{1Qdu(cXz9s7eH2kf){DoK#HnP7;AV@v<{Ft$N;o~K2BzaTCa)$ zzqzroy{l`1#OTtgJE{p_$M+2ooD}>%WH%0miXcZ&BTU9SOFz6Kg!4~tn8@6TdKX~H z2U>f5!la|aLt*#%9$EP*Nq9eXEdQP|Ja6>KOW+msr6#2~cmDi&6i@$!Qb%WKmHEZ% z*GBcu2-QPWLkln1u;IfZvK|?;6+vCMG}*Yt)MVcT+I{_6vORc95HPQRxtW!@yTjWf z015ih0nj}I4JqP!fi6NVC#D7j{_TbS-*4m;1Y6+ZflPcLcGeBBoa0N^xq>QKoiYak zV-Tln88qwZ00#}9i4+V6&Rz2Ra~XaSINy)}PKMkxm0wmQ%LM};&{TMc(+iwPEt&7H zZ|#y<60c$O3w30mm_+@ff8~`sKRayAm)J1SA@WF?nf`g5S@tiJ$h%qtw$=;o;a>P=L?~8YM9g z4Gm?Ujr51hjE5H`8}N8o3p!RB)SS%?4Q4QyjAmV-c%1m2#wdo2@nKp5Y9=%l4d}JB zB_^gfyO9hY^y&XB1JTJZnF9N&zSG~~H>1c6s19Nd-SL$6fR;) z$~iQ-0gZ69;s#{2vtn(YcTl?q?%w~A_9z&i{su>4ruqm_PZ5?hR7c266P11-1c3v& zNMZ!UtXB8`vwUcY8$gRVK@`Z!QF{}BA4t^@!UWD48foe;xUdIC|GSmwntWfz>%2)YzD6@s8%3^p#lxcY8B980zw4B9-x@8sA$ub zJ1{#s?DjuW5?ugx`v~BJcqtRW`Vy=o;Awp2o~?!%WLo`pvxm*#Fo4R5%cP) z-;fCg%I*l+asGx}pI=xwWHABp4tOpLATXf}3WJA*$VVi8VUa&1F@ZY;#vA{C1oqMc z&JtOCg8I%{M*|EpBM2c0$Lh4`Ttj zB>r%Ol7%IgTj3(i5ngkm$9p%Lz`PD>olzaGrjI~X!NA|YF=?1@Ld`Rz|3=M{!NHnN zZxAE^mkIL&;IN0rv_O#sr)h@i^=UX6{%6q1_)O1+nu4gX0`6Vk&e_mm#i&`+)RYAP zl6Kznswz_O=pEZ~Fooh$J--IvIW+aRwleL`VDUYEe$@`Dsmp;3gLwYnc zG>BlM{sldSqFxdDs!QbrQ82Cr@qXIM&n|(P`*S4#A~YNheFnH$*wDO;1qmI}T>=T# z=sW@K#H(zzP&JChZ^OCWxVeGasm|!AETzJNt3t@K=-aV`=MOey2K*3)wV9we)fa)y zGqn}s2bmMjph<(f02GpXzN6|>-Lg#V-gtHO3B&QxCr7E53pV5-qJ{e#S49Ai@KM5i)duK6v%3SIGMnMraLh zL$rDs$go6ht8&?)c;|~;+fGg00sW1QK&R5Gs0u+m=+J%?^H^kh@sp99WFetxS0m}iXK|8zP3x`Y&HDp>i zS}Ox?v8dLACh>(Qx%c6ZF9W4LSs)3244SQM6#6DUtqYUVFbVwMR4!e=@ z2cN{evN9C2mhYxJ1Q!od!XKSxVkItsK{bl3Aj$yQEh`Lm?o5Km22|Iz$sI=i=}X5* z4}*9NSc(KkK|6$zlG1a^8)Pbo+yMav;;7J-M|2`26L4S;it>ZX=;^p8^@h=UVp?`jw*U%1uj; z38==4l3Sq;uPx0p-gV5}Ov&kM#>Vre}+(VD!adM=IVF zRQO|lHlv174cGNa120W+ABg>}bkpKng$NDzF}3)ib7jf!X-r`DQT+Iq5nhll4p~ff zIYF(4W%PFvY(zllFnZRnYKl7?T%~jVD1b?GCb()5&CRkPVMv1d6huTYB%B9P0IChdRG{J) zvMlm31Ah*m`F?2_LA(uh9P}9gHn9U}L*vONL~Ld-?G9AK7|l1}4&V)6IC_xhYoe_k z3hiSsvib%UE3n<6m6d{ESqB2ghcMgvpRaLgdS+&yi9;+@T8`9bA-5+uf2R1@{~De| z?U;^O6^y(iW(Q6Ze2z@NVYxeHBOqaS5jnhFdh7GzX)JIk&4WoLXXojsFv-$uk_qGu z5H-3`S7jQ1hEl{>7YglgI29a1LZncb!E&PRkW**CN*@a;v_R?-0s%60rgZfH3`~(v zTSH@GJ`mZi9v&%FLA{A3a{w$<;d%pehXf(Kmvi71y~4f$*R(d-cnQFogia< z$%xU(dg8?v;p}&wlN&nDiXvHSPEUi-&~eFAVkiKs#UztMy8-sk&(pjee+!@S>}R1L zzX@Z_tnx@Ns}1qp(Y<{J>(93534zArZO{&dW|Y?OGYasQg*29mgp++Xc=LEfJcO)v z=bH5;`_&aQ+dLtn>ldCsKiN3E!%Imx{Dl{WI8d_X7SnugcKPIKsqu-+Raysczl)}4 z(0B9_voLE5f*6Bm6w{8S4CKG!6<-O+P)OQse*tWQA@wkX@)9jp7Di#9&aYA-i5AtV664sIK7Xp^YO)YeP^Af%E_L z?=(Nu{UehG^*mH!Js&-V}@B0ue0(-L(n6GW(la zw*`m}s#|H?^aFi3CouTMOe{R1g&b!KR^X9G3j!9!;_)e99mk49ue_P=s-F)FrWOmny zphHE`QPGVT>&ng+G7WQ`o+h3Lx(3rEz+OR8gI$p&VSyd>z(Vc`iO_0*I`F~ zLQnrqgI4af_k18PvYvJJ(iDI{KRH?(8`FDRt<_+vXN1edgyUF#Oi;f1Xq$(RHzlwp zzo_5D#GK#(@wU0$asOi5%hwcjUf0F~Tj{0J#2WjtTtP934my;yATa6%RX4IW26P6a zz95BWf#S3q#^{l=9+LP%7um@jy8ZUqz!(6f%>m62=yAbTC^jf3L!rL2M9Pz$ok*A< zbHq)SErkvr91Y;wRDpN~CyF8<$R@3F0qju<8$#903)lu(uVjpk2u&KxVz$QzpF@bN zKx8-2)sMGZ8tcp$E(!~WfGS6a0TL~*tFzp;k4NH1r)G(P7K!1JM=;25C+0LK{7SsD zrw1RZuvfR}6ZzU`p7}Um1t#cvu6#V)XOLjzgJcXwLDJ;|Wnfr55?BU%q#LNvHZ~R@ zN`e8xA}rX@*4*h0?*QQn)TRR|2^wie7Ea(5X0UA?fZ+Hkfa3-Jlc1VGDoikiq=HX` z_6FfC4U}Po!k4$QEph zg(Z&`UkpuN8o~Zlnc!V(ijkywD@n6ne^0X!=j5+-RB&bK)AG|^@BQ=-YGe7}{q@NE z+p{IXZ++%a$@?mgmb*)(TKM3+QdM~+!l9pXFQ>yTLE{++X{i4P#UDVUH{dsf)+vJZtLKy! zG@X4ip+BO2sutiqI+dVCwg5Ivpq?B9z7cwSP*%KylPB!4pn=;1Amaww#q62L2ap*? zfnlH^I1j`kJs;k`jf0Ix$ocE}&_Le9_;)y!xEYbS`?cg5T-H=2l@%!%1?bVi!Q-Kn zsjd-C!^z)ME*cmedK}d(UdK~b68bnI!|{=ejq|BBr*f-&OI>F+$c%I!%LoF_{@I>2 zbKn|ru$y<%HDRL+p#!tAZ~h1#9vn`6TN5LLgmIZ7epY_L zip!eD+5GqwCC^oqt-VsnWY4$Z7jt*j4hX{|wrBhHnFrs=*}&@Kwyf+|%&S2mz6)gz zH%+ug=8ZBMaWq4R7cjf0%^e_mrKi4 z72Whx2H!z$L3MuM_2mc?)r0AC7cWj@>COtFJn%xHKNGA zM^=H9YWT~Oq6-EGO%+aJ95D=d?Vf)T^?W)Fw)uQVGqE6gjRS8QC!hG}5=}%npqGc1 zKOWYHEa`JlndFd{+8I@4!r1B0vyh=w1$WQ$yUV9plpfI^dKEYku0 zIH~+5oH&lkT-GHOI&UGzNkq89*?^wB6$hXCd%l~ubuL5%d0pcojO#PYt0FyfCQ!e@ zKwJ0zekAVLpP}vTvpqt5uxX@{U*iLOv5grq{AwQ|vo}llE&rftUD&!sq&|?R z+jn|(T!t%Fz0cg*95+QJ-_B!5l!iJD2DhGNXw5vX^{TjSFYdk-XH^(e=1>_~J(}S9 z=>ktAkk$O}T4n5;Zh7afFx=`9;*X5T;Qhc5=nWq}ue{2ILeP{!@640rU&X@Wdy6tv zLHcjhytD5pG;_JyQNF!%>vrMV$y>i{hfv>(I_Ji6qe8!H^W-HNrxdrWinj-No3QZ< zRN{Vt!g&|G55P^qG|%6M6uG9Nu~hlJ`EzK!j2?{J!w?7wJT^g~ZDHC*1zg*a9t`G3 z&7jc$R<)nJf_>GM#&w_{g%oNKCSWRE0I86{#Kv@D3X=E>kgJ)4S_6&zL)#5S7*s-Y zs64Q(i3d0guL(jOL82sN5{a1Qul|txGCLYUi@VMSwiTVxDavJnS;BdjdzeFJS zd#O-ZY(O=;mKp9ZboxRQ5W-FQ96UrM+Zyz(Q<)Tf`AkZ0){iTX`!9PCrcY@ec8 z@!x-MDmzGdOm(UwHl13(gO3)!l1G>`<}@KuD8b_gN{Z?0X?(4OLbn^*6r4u_tMu~*X33*pIu6^CJT!#2y0$H6o5swR4~XB+p6wmC2Olr zh;6=qZ`_3;BUZTH>HNzLSE~+Tfm`Q|Mk7bU8cT%W7v(kg#1mp&8|aXwv#annpODby zpAD$E6U)|qFP|&lfG-@|GUU^;qj7|c~o9fTSC@pQVmdU(UX8LgdT#K$?B=%ft32XK>(I}5PgHHd1V4Ed_pliZeu?UP8S zS5wp*L4TWTs;fgh`Y)vsbuoVGONWUku%kT(PSG1R;7L5o!Lo;M0-NsUq7)8i-AUs+ zW?fAvo7ZZIKFJ?AqC?&QBDQ1=?#=t80a%8PH8}Fen~8HU*}pd|GH6Q)rbC>9jq?;s zO$mSDHFwAQgCajlOb~;u5Dc09bh$#=Y2Cm;5(O`NEZ1QBVb)cuimY|FQ|No(hChq< zwC`PHvnvJv+nh1oqrdh;)Mk~+^DI&AbIX)Q8A?skm_i}}a%}thv z5ev-@eVuaU$7au?(z@t{ytiV5ZvDCEulnUOzG5IdreQ3ZJWfTA)0>s{Oge=H%{ayT zA`FkV!H6b4**9Ni_YAQ}4u*pAP%NFAcab9v;=HqRA)=ALJ4Yg zgRwdaJjn3#kJ`zuji$SwWw0SV3TilGFzER}MdJAn3pG#2m)4Y9`kd$Hb{8Y+!!6ht zDDH>-j4-0!Bw*A3NzWP=KABL?MnTYgzd16Z-7K2w;;gjY-giGQ%p^0hs&K`xtUvo_ z^o|sHj_LB8=Svrd3o6|td@k{o6D6xH$1lup`Nm8r{8|1*3)Rkjvak*v6&kH&yPw3Jfx+}!9#=4*S56z7t^#@C0d zDP0c4FzPnps?^7i#rTVUVAd#0bmV4=^++2Q#L{T2_j(aUb@4VM?2NbvlW^qmDm5ct z4ClP6{F#FG`b@vaUZ{Ly9>o;dZiWR9bI*VFd0FLKhUs6pMxKMpdayskXtdL|BlC^GSZ(M%(^#4v;4h!>*k8sk2JmmU2(7RG8JcYrRm~gy^-Yso{V~;&eh9_ap zV!PX=54Y59)%kvpiFrH{e94K0eSzB|IV^qo7W;jSJkizV(2zWBzPyp-E2r&Pe+jM+ zI=$q=4-K(k>kfa_sWtk>lk?A4bK(ypWzj0#t7*29u6*;_;uZ45Wq6}o)xAbcVYB$P z&(E;xbmM7_d2AC-T=x$jSuR;i#aARtPm-riV0DnGU8Y=}WFl5g8eYG3%Pm+qynH$T zVA%5<$Alsuo%dKh7uDuH;>QGo)q|-=CG(p>o~AgVWb11`xAAV$#C*9n`o>I{Sm1OX zQ%_4N$%^w zFBSh3#&)K4CO)g*8dJHaVu+E+#`qR6WT+VXTGS7-LtM|hO0#|HEb*mC>#pwOK)qfrYB#*;MI|%Y2m3Ldn_-bEP;o6Qv^KFjh zL@H>z%+wq-t6!~BG~d9J2`)ooVyU?Y7k|MFD>KY8ipuW~CVmGmziu!mYn;3?=PJbz zy%De|D+NE85&gULMj+v~-%$kVJOsNmk+y@W4Oj5RnS&WF6;6tFa#hCrFlxytyH?$? z#cVm@U2?`3W(2Ltzl_*#`zFGxF_PT1e3RY9ND{Rrx(J)iB3SWlUrpRQn3ze36Kulg zn7IGclMVkgV>b@l8;!bfliVhXGa{9P$!z#}pYE6n!_{Hzw%+^*=|hZOu|*qG#&M&Q z@0Eo^(;;4ZpEaVc;t|y<1!CztTy;bfZqKmiqBjn+Cv}En-;&zWWWP|TDwCB#A(F&?GAt30g zpSwKx;ATL*vjHu!Dj7DLSdjA2f9X@Z_uV72a__U;hjnK5M`leGip9OJrDbxZ#lS{njt4E#J%wWoFh`x;;kAbJLb@du7C0l|C&=&H9(D&G;>q=2Q7<^@Ad6E;zJ6Z+j{OUci)=3) z^g^&qm5Ky4m4{~i1XU8kAZaVGOC<(KgaOn@JL^;P{&|9uT!N7ezs<4@Y5iP*0&T=( z$HULcqRD0l-e-w)gF_w%y%J=*T*$$ea*h3QbQHGeRCR!xSMq`OBau5*7)gFBZC+dB zGo@CiH}#132u6i3M@X*JlMuZOAHZJ=sdPRa8saH>|CXqtzo&9js3h4@NpyL>5g)6_ z;05BFs*VyVDqnnhclNI=MdI{tRlC1EJIO+9p02B=MvUEm)r}%=bIg8}nBWr@)$13_ zlpgcApCg!CB;^Tce;ZwWOFqldfuX6K!lx!A*)A*{Gbg=Qs zzPhp|&2N3q@1EvYT!h1&|6l@pJ&ichTTaDB@LLR)Y;caA3QbCB=5Y9hugIHg-9NLo zJt$s25Y%+lxI0HxSaNf{N$^OHl4fcMqprk8vtF&+j2o`D>-#m~%+;UYhjI_GuS@@+ zWb_)=l7jr>I;c!D8*+x^jZn}g5yX&B7qU@9Tk2cLnP0P{!L*U)-|8ovJFbvk3Wv?s zUX4*Zk{w)WMxic2TbZuLK4V?KmI@OKM(hP9wxxwk6J%C+4u7Y$y#tMT4IDhnG-L1f zljDsO!N1igsfGQ6@qMPpqG$to!0}dYH~LAy_U14s@TPGwCr$w%yH2+TVNX-#9Gv)0 zm>i1rYe^_fkRiNd%nhu(mMREz z{M5>#A_-FK^wf-_%5}HQE>$P1g-c&5HCr>6s;#lFj|3b%W={u@<%#oKx5=bJ9Z7%T z3KO+FqED;j6f2h5E-}$6>>4*;_yyWVM)(!=gumWPjSi5AAM7EB$$VKgYE-*bZz7=h z==P&H99x;6bw9UpKXA!N>vs6e;9Btgq^VE7T9kfyih+1=M3Y|qXQFttGRb~USmg{x zkRkR3Zb=edoQlX?vr!m{6&XcS{ZA=db^8(_y}^=IzaML!3R=RForii-lvHOcS8En& z9+ai`M;1KkuK6_UY(3#7%s7_6Xtj+y@aLD&D=w;KD|SwjRU6WlW7c^JiE?Ht3kR-u zmLjyeEX5;NI4Pixv|>MYwbE&SnRHe_V!-oUi!y>h53EE<6v;4~Q$$V#tZy7E9;6Fd z86KT_St&ANUIC%}l*f3YiiA>;>`2;L6SG0V11t9X21WQ2ZUf1~rImND{=|fYs41Dn ziG=#RG!cM>*OL5yaIaupmd{l2{%*2p$SzuHrB-fgtFFdH;h5Dg*`KqRKbQ=6SGfIX zDEC+_$Zx`_VeaIZ6ix~qDh5#seLa^BamK=T!+$0YE!DZ3?M0m+y+9lA?F`EsyJy+; zlO2AFjNSKnVsBF7M4IQ#Y+ND3CXTvd^+MmV?P#ZGaZ4zmAYQ4_D6-hDVW}YQ*Yw|3 zpF-0Uuu$v-E;vz-KN6LqQnTUbFf$w8&G*2ZBoPjo&UI-VC>uLfSUB$P)i^Pr=!{k) z<{qux{R$3-zaCcwST{y>BDhWm4kZ^?FHd_pe_ zM}iafjnPk8?r7r3zg6nHtR_IyRWJpypfo+IwWm~?er$=yU`Zx~B__KcBk9jGFA9g@ zvG%Fh;v0<`s`WzJ3m!c-r~htX8FygB=&h4GJrDWtyF&kB)I(XZ9&6v1E=Nyw`Ig!{ zY}@i$NpeNGTBhh)39kUxgbe!(f(K8W1?euGeIOfkKH|#MOEbM$!O4i!=sy$~PH1kI zicvRpPbYqp9!rR;e`+@37~kkp`t2I#(~Oz<8|^rpYsVM=l_XJC?BubOub60>it`DS+*F?p1XR zCwutx*0&t)T(_zl3O;c(4)~EZ5WZGeQ1qb)zOykRpG<4#!-sve%4c%H&{y$;uq?OA#gX1_+GgiQOWM zB>6mq$HONwRWK)|u-AK|dPLXI=^eQOQ0`}I`R7_T0)+&dmI~s7j$DEdCZxb2@6^D0 z7py3xnQ(k@_TczV+y3ptwv)dnSXkSBO#O0O#I<8<+RwfpGPw>Y;+e+@)W3n49f2s0 znZpIiG=DXO2p)RCWZWDoiNWb39wxT333@H7kafnvofRh_r9EDw(r+TlJg*vam?n7b(NqCQqYH>kP@awtH&Qk&0m)L04 zOYU6LdHZtJmyF7Z?_m>1StD#?Aqo?~4Z+frMC zxO7?(D=yOcCD*&Tnwy+(Rx#|%xfBnet{L?j+h$7hz9uA}lXFF>G5v06&8*L~AQ!FA zq1w-@B5qQQijw~MM|R;JVw~3{WJK9RLlEorullJ@itbLj$~ES3I;8$-o(ZoAAU}WF zvFmEW^-Mn3(n;BCE(U4Tk1$-8^zX60NINFsPkO2UilKIr1~(=Y%44Nb{BBQ(&0`w$ zO?DC3bM;Tzv#+r;$BDFm82#+)txsnVfYU}N5Q-eU*^rrj3)B6b#R3ops<)Fl4H1t7|S>6frqEkB*4emv4~{U|BPpCh;(S%Ctv-}~&>r5JKoE1O9m z+tW_}ij*`93SvI_bK~UY-noPh(JX5<*VnA=4OJ=>oYV)uBUF@ z&X%!x@lm297;xveS8YcEXc_9Lmm3U^S-hI^?nrApvzzo*=(LUB7sw{_scR_r&mhDfh?BB+1u(7q4ZOotK11V*Uj{QhGPAa3$$%-IqUnU_0T|AR(a9FbPboO1n^?1fJ~&1Feq_ZZ>dWhH~h9o}W zfd(I4?7WGsdNjUy6wl>OrJc|Z9+#%OOt`)M`_y}oB>^)Ly1U%D@eY$IsF%K02Wag> z49-7SMF5_%0gDA2<}WmLzYkm??F7)tfQCv@KyqJzOx~@C#}rN^^qOqhlOC8nM(37o z-PfD(E|W`^Zj5-~`X=5A+{<@(p|C~Y;GLQu?sT0Nu{;x)l|Bp}cYSV67@E#2# zlsiHrzf37z0#^;)z}UAyw@i_;6GATx(9!`JAaoEYpBu#!tH!+y2xBHV_;CKbg87UB zVugT5_AX3J+4`M7voV(!M)KCV`DY~1L=^D5V3{f~)p2kXCHM*lqdVa}ZI4KSQP@Aw z5Y3UWn-atLFZcsyFajg1ds~?^4^RMx+3y}J(SCnoxU#NN>g~#wh=9y+g?ZOGIK6_< za+u!A`TGsBjKa^sc;)2iZu6aewwh80Jy%AW5z%;OZ)4J4E5-<;FB>*E@Y9UlMaEqo z=6_ed>-_-ayHrpz&&ucbd4$e$p{qda%R$os1$kU&x?bZz>P=qiJX>S#hT7Gaobw(T zH}6)P@0jf!RiGVJ9D1IXfW!p>%h}zEt=}mU9eY!65!QNxTknt2)_{b|S2!cKqkgd2%A3%uL=F z$%!6IXbl$o#cav9-MX3SO3?pG>BE=zOFafl^5YLVk%lf3(JJ5ISTHQdi@ zQc?5Opwu5IEV;$1O70A-_2kb%2*f_|#la5sfIi6IbFU=}lf%aRG6?33JKnmAg*wUR zZ%K?(u;P~oKQXQn$d}pFNHvhdy2&=~W@I(=a#SXI7x&EqM|3WCxW*p?oz0-yw67~r zV)^Sy>vV>;MYE9+T4{E>N)dL}J2VlLjdo9JPz}cT1h;Zo{3z(Z(vkQ+Cm@%&EUMC6 zGp6rk`^QOF#j8VFg~n~&EZ>C;WLv!nkBsHt8U8`jwu>`;yz^=fykS-HZ@vM~<3#ZF zq=GVkruM-iOpJEIvtH+Fqlr~al zIM2b-4s}=aU{{Z?Z^5bziOdq~8hBHn<*-^jl)t7nf_(l5dHX`G^)=+cjc>vYUIW>O zq5={CZ~qq9exH?`e6yc8v-ioz`CYOH6nl%0{|_>p0%|K4?#pJSPYcnQBlQL}i3itu zqPs1lyDgTs@z`Gdu(j$~+13)E8DJq6VH^v=;-kG_4DimA?`qrKD4e;3a`v>3(3(mu z!HO$*C|et;A%jyVORZLM3G0FQ`#d5oFruJQ6G)mcB)i2=rw zCBaPZ(ua`?FXu*SSkJWk)gT)3UJgxcE(HCjDFja+5^xjqJxNev|AajazbsqhvSjzA zhWjZ#HD+>*V*|wJ_F{YuY-7^PZWpt+SU-+UXx}2Rh+iG~{K#z0mzlC64#GB~7wN_{ zVhPIaN8i|5StkM@RsGA#&?efx2Y5)5wRDTI`?9}nQp*7(Fg23pG1$<5DWBfVdp4{>*zCPMY?$gfd z_EGfD3_f}%I{2>yUF~njtmKcf4D7Bd$8p&_p9L!C)@3=0CqpK|mk-)jXZ>Bg zt+GIjaMOwo3UvYbgR%baKp=f{%RyvgADopqpa&?y(NZ22ONiMz~MJMg*OnxV}jb+COgpnrf;wTvJ?FmM1Vahh9tIa`ae?k1wRy!uNNl6^=Vl(Zh)PDs9^~hnrSi1gT&-$ z<<8v9@z0rGai)hB{{fIH&W0hVW?fBiv;0;{2fUbOz#n}Re&F;4-4=T2N+w5Rjcz4H zCl*Fu!VK7|e~3`aZX_?ys=(Y5Q{o_}bk~2$MHNcsZD!v@(Cu}lps;+oI)KK5)wb05 zOy#1vFnN`1tyAuWT8YrXoY5d8*ho8x;BH7P+Eq?bA|`EgYDc51TGU7tZte#D2$?6A?Mc~5U@wk zsCxS6L_zh%mtp2MdH+kEak;Nl-|FXOzy%X#EWmT@%LRA85fuibwy}@ofKa=`> z(do~vQucb6^jc0C^=Gcg3es%dY*y)69%mswg7J`db#_Mg8dmjCCwrR=cxI#i4sH)G zVpR#wqPW{0;$q_z&wreL7=zthuiEoas}I_oosa(Rp8=l5AfdBh&7dRsXZLL)o`n~V zt?)OXlf7+3sTsBEtC8grOK3EjJ)St-Oo0C@VtSJiXNz1af3Al1b1A3cyUWcT-OdNS zJT6z4*_jh-`ptg{a(@@u$-&;r>9^K96wkguSA>!HbDjSOCku4C#F-vj$iFPf<0y+( zVlR0d*YNxOg-vv38n*8G%ziO+?t<;2@u-kTo$a)L&1$zdr|`b%%*r{f{hzs^cOF?-f*TykGS5+Wj;B<7dimH z5IF!Cg3X}f%fkZ{mp|NQ6kZmSAm*4?3;NZL=Qi!TIO{)i0DTmBY{fU1{!WG=WIV}g z*$ODy?JdE14Upw!cOD+f9dELnkRylTCI4qx?u4y}KZIP8R@-Rl<8e{Ti;X&N7EF#w z1lIi;e;2YG;iFjo>%+x1^~|QTa5mYjG!BD9`OBK zu362j5KWHX3h!ENpI`&DVShyk1)Os6pO`?6QGtIH;G%Uqa*PCX8%B`fN#nC2kCy4} zN+0t+fmZ?3mdM@|!5~Ka> z4Lqb&VmHStaW>Eqyxp;8E>^3R>xvCs>W<|2cGq{8V7{W6xtG z!luolJyEo`hT?J?W5J8{l5_m*#44%CFC5FM>TkZ&kP{jNhbv=j#ol9EPCY%?*`I^P zNl&Et4q1#!a=(q4K_D51>Ep%Yqv%-NZZ{Fb3sKmrA%ru@YF7NT2NtDIJ+(4jGj{gR z(CP{?9!w=`^;oA;K44&?BMTp3OmC6Ly^=}L1>M>vJGL~Q z7xoXF7q#QYZO*~dAG*>X#5-DFuw1?`-# zO2_@`FIu~NTnc<8Egt!D%0_s>GVK_N&GR`QHrJBHi_4{zE~8A{$t-aGdu(1vZM@!c z>tTxuAq!=2ZaDQ1Ho@*%M!%)}-Nch!QbyM|r=&n;yxDsr$bAgQG9)&&4YV903JQrQ z`&B1?ChfFX9&cec`Qp@K|17E*P?H@}L&d*-3cT{SrzzLpB8G)VZs)qS=hHgfW4@5bA!o&g4fbIGuyYT?f88gav9#1GR5+z$pkW#EoY2xOBed8H;Fb!Kl}hgp2U>vGQ48-DDr;=VS?*Ba#geKm62V@O}h_`gLzHX_K#3r zM5MYK{KRZ=rt@3W7ew&@HU!C_rXUcC&MWDl%M#$FJ@Y493hYa3`UIPiPw8IV`fQN} z;(MTcX3FyXe06u-r!B53;KzlAJ4~tg>jS%W>!)u3%t+5KcAE92E4?kShqg@3LNl=wX%}65@t-k zOEV6uOZ<*b>Dyk_IiLX1vQ5;78y{4wH{b<~MmfUCzWXxk&#aLRtt6+GkWE#qz2Hc( zQRQyBx}mDw{B$r?tx6%isSBe{j4ED=veENSSJ{j^EGD8jGCxQvTTEQHO|YtKp8u+> z&#{`BJy4(Ql(HA&R6m!rXA>3u{@!uuYI$yhDh8jNDxT-b^`=N=fPymUmqVw%%|)G0 z>(GbevSfMbUq$Vu*Xg>sU+X)%xXY#n0_&6_;7bvuO~7=&QsVm4iVG#p-IQ_{?`ZQK zHIMI?#Qg(5+p6dLa)&1k^2pt&9p@r)iqUIXUVH`L?3QM-RDwFaQ_PZ$i~aPX;rHQ1 z^L&Ab=?b%HFjf;L4kj@k-Kx4MUp3c+9)y6%VdFWf_#XRv>*7;c`vKAk>R-Uc624t% zWcDY_HO>FNP{nQNR7V9n^qWp`c`xclu>X z+p83|ea1{>)56Kzp^ABNWUbHr{P{zmOGk7Tn_$VA>BH3{;=>ale6-sZ(T zkig5iUZe~uUV(AIad%3W{mD?O~n1OPr`;A9DxFED7O7B{J+>Sq*7Qa)Ce23a)|zx>OSUlXggw))!s5 zy1O9R7Gv`whV*a?Po$Q~>ND>nB56-6f%ydD$sA~gw#fe!sZ5O92DdynJc(2S@=87M)71Y?Cj5Ht!>pDU zB;#A%1{IsK9JMm=nJuE;kMvsPAH?h~ga8p$^azg)`(c53N+Z2TOl8*D<~w0Wr-YwX ztoq#)V+zH36z8kNKJ3)g%4?zALa6yF2~wY4ASZo+kwV7*Ze=ZmU5!P*)e{8n%;3IP+$ieKtT2G}8mo73Wk)07+ z0d_b9YldK5)nb)9S~W@YnVCV$%~u|U0)^Uq)q>V_4p^*gqu!uXa^A*lTp1Of^<(xp z*2cTX6gft;jRnfc{;iwu7crIA`XjSa>q>`P4GQ|Y%{n4u7e_>!iaU*3I3^;CGtoO} zb-=Ri<703b&D&gQ^d!vi+}#s+$q4}cQLMDuKZRD4dBj4?SpLz+@pp^&cL))KqSWQ9D0B<)_Mo|qX*Yo+sGIeWcW zeA^eA8;pKE?a;{D-AoLA0F8fY2&gA`S05trN#92(*om$_{MFbW_^&sH&xV9Y5axr) zQv@ez;_RMCE#-x|!j`NTPN=hF;{P)Op-*rl`?mrSh=gwt&W(RF0&iA&OR8}mb*~U} z`g+xP15+n+v{AG;_x?gvxZRVrzZn|1x%#go>3!knI)+YQ$p847Fu^qGyIY2_Tn3;EYh^INUh zOim2IMociY@}56OA7K;H)`4%LKPcdDBpyb<5N?bdNt(hQ6qv9P+` zT<78?Yv%+(`#I1?3Ur$Ul4v1%T6#+0_m`i{mQ6U zz?vmX$JO)M|3^6sKsMw-5vSyNERtlLY$=?CfeBc^(iN7s5D8l5Mau3}b&&?|UV%0< zf*ZR#ui8NC92$JQ7~BQ=!5YP`ZMcwhaC4ae9mq{Wz`Hk;IUNqnjSs{8Bw&^fUjjXo zk5AwlAGbDwtfB{iEzE)u`z|!%SQrFi@Y#q2Vu)2OyYH8;u)SJ@{MclkTxtFgQW2)4 zOk`KaOp0D$`M2@f_}O}`XVvfP#brg8>A9D8oxi*hx8GTk3OmB<7#!Y%8ZN^nCjWpn zTo}*wo6T9qwm7DIP}39%E)v{bg`CIpgxTFq@MVery;k0b`3lS(^v(KSDNPA!?q}j^ zRj9eU#wn-@mbq*8!+2HKAv-liO1C;2J{i9&hc{egpWZUC(|LVruef1;EAP|HW=#so zSm)MA>aBw1IX61pM1Fqy(Whbe+}l(P*&#nxsxwvPENV)JTi2f&k0k4N%0Y9hY(S}WmP^dq5q?h5Qw@c-hn*~bOf(YnjgS!8+QA}jGro4KWG@;NiS}HC6Ec+&5yPni(h>tA9(1o1xbz)vR*7d zy)%bby-tU`n{yMTk2sm0Nb=ZoIoWY>Y#V*OAilP$#I87fF#R`@@fDn)Fw+ybnw&98 z7AqO>7fy$)!CQ80yAixUG!J%pR^98+nPczXZH^(~6+kM-7##UtfRN3uL=Kx~F0=Be$wmF*0kl4Tp7 zm9!)`xnCC42Hzhhw{#WsOtYSrCL;;k!YRmi{_NHmB(}kM-%N`49tDP@j!N_H_pAj~ z%YI2^QF_hp-*!z`WooG@_|CAoxtBX#31ePtVl!Hk{z~Gf$X&udQWU1->{po=b))j5)6~&y}mH!&y)sQP^=ld~Z+pUh8lr zd2A^&ac*!;$ZEW00`cBwEan1Z(jSv@< z3X2jACuUi3Gj16CbG2}?0SGGTXHpDsMW{I5NVxqwSmAdt%e(7%r+-@vHUi{S)$Sm| zJ-jO^{jBBKlhNmw9JLE+LHBuroUm4&V*ggINQN#4yz1?!8QuCQMss?rBUv(+2f9-pN1 zH(EfZe?B}Z>09G2-e(4PLD>GY#Dz2AP}P!3hUtM7S$*|*>)P}<@ETbvRW5X}vXBMQa+J@2z55NAX{^GEbg3$Su&#y%5H zuL7qV1B*66+>wSFnDY?hxq7W0qM_6OuvaG{?6M9g+Kw19va`8deEST0B-{2r(v&yv zesl@Cf~0r9pWkVTMNK14)4lh1)Lx7omUX*BGYQ-h>D-}G3x7@pYjg3`- zB0=W&S84hOms|~nsMql6{x0FJ+v^YMSFJq_QNw3D+r}WB$5E;t`}@?Erd!OknShv3 z`&K%oaahjs)$Uhw50ocUOR8~j!^xWNu?{z`3hu2fX0p>=m%10KjMeTtdc%|L24Q~F z)q_MDS*tJ~k6H%5eC6iIiMXz>$+uA@1>KYgY;}T#@PWmF)TQmyK7X|PZ*1n?sZP)5 z;#&*EA2|H${QHtlbJwGimY}y(0(|D}@?`nvbFX%6DCT&c5n3-bvClcb5x99bnT)SM zsBXYaM)?gl5JJcEfYdo^`BRkG+q1OLw_DQ)dlY*?Orwg=HV<%tD~i#*M>KJ%>Mr7l zLnq?)wzjvcf-iYg1L2jeGjjs9vlQ3qR(NiQDT^kaO#uo=6Os=Lp_ud9BACF{WLr#P4wEa;n>{2a0hOtR%md>p8GF5#sTNZ@>Rn z5s)J3F9pB}K0npeUWK|9YX|6T6o=28^*SOAT)L9)>nm>#KK~<*ZIBFd$S^c!{M(Fa z``bW033;|sUC^!GI0E^c-tQ+>l)$f{90Q|voTmGq(|Q09{U^nUUHJc)dhd9u|M!3V z*qeil?7cEe+2Pn3AxTErl##u%_sD7($sQ5eWMpL)8A%A)nWNy#*ADd)?#tTks2TrJA}CRc)_# z{|gdEbeEGb?Dwr?Jka?Lk5$`|?AiTV`9lYx2ygS0H*E+oRr_6DSur%94fsZkviGze z_;PRc>zP`6Q-{p&k)4B0bZq3!n#*a1tdWzQAJSbaW-uo2V*PwF3Ouk-pzz%_dgP?} z^6{`om}ir5%T*W$RO`c}+VWxgWoOCHY^h&7n(4eOSj#=2wZDzs?tep>gN5W^?fI#c z-79W!X)4fBq~9mX%VP({!xM7vLthq9ZU7nL7(WNu>TawE&8Es!+#!s76ulc z`E6{GmHpI>RnnZ3KCaJsU!Lurn|x>TNTiaj~*i*(nn_H@$yfk-111oUURvWcVPs!@RHl z4Hq7&Xyt3|F?cyhqd4KMf|17h693mX+uk@CEm!@**l`ujchc{TcvmG2aXiyPp_>1x zb79YHjeN6h+ca5ch*re~5d9$!^KI|}@U2vrO{Jql>kBN$Of%!f*1Yh88)LlL*A#^5 z+_!27hrfRZdSXNQjjM@1COd4-nE7^h^S3&Dq&XFMM*Iz6p9L6FQD+4nIF2iWl8dCk z%OLH;(f&*MmT9HqQVpIbTbntWhY@SP{vbpk*6`^(glAvaKYqWU&r=a<+sX)-@!vlDLe?9gX zzvDgQi5RvfcV?lt;dt`Ca#k&04*`E0K*;<9;5h^n#zi68Xvb_swXwBHCYSo!@WTEd zcs2rZ%a=2~zQzQcz;`+gZ_2|Tv9&4jcAiYrOTEJ6jqv`jBa?4GX+e-eUB>GR)AX5s z;O$u0A=vg5{jp*87Rd7dseu4*|1qEy zMhFIO>h_*{Sey(DK%&BLe_&vN3BrywY}Zev$gUt#S`|7oTK8@v@rS|wzr*b zwqZuKc{rrY&z z(1f_ZlutI{@&T^b`qLt-hqpCNi86MCzt{ahhh+}ngNzHKtvM3F1dyu1nW}9C`p7RH z!Kl)eo<;W4-E@P;Et+{{>@11wEODTL)Dn~!U@1FAH99f?rP?8=iKd+Yo=LRG5(Kj`kO%&#&qmlfYa zC-!1?HkOXe#%+e=)1Ig+6AC>t0s{H<%?a#a^NhK+)DVRkFw*4dby@21XyF*B9|Gm- zps{W#$pG$!xwi+`h^htWk&@?O)(bt0_FtHdJMrjD5l`D>8kmh{I}!u;Ij4OV77F~SEyjis_4@4CSp*EJy?4Y^5d2ut7!(GV6|`ok_88u*=!$@C2ga@A?0j z`@WSmUiHkN4gdplg6vOqRD18$LosB4f#t(6(KLY2m`#J~&ab9%M2oIaV(|R=6fij7 z;t_Mj1gMhl{_uS}J!z9__G1Sm?GScy)Teg!U3dU|9`k<>5Nl||-=Fj(;nX^kSlLHiyjQxmVK60M6|s@p z`UrQz-mgI*?)V`@9*vv#N+6y!YmNNQ>{inI_EokXFTlX(`)o67Zj98&ad;Eu?v8y- zV2Zs)Zkf(2V-`ketulI>)YC*?$#oKj9#^I)53XkilmxZPHqy_>S$<&?%+%E{R&I)P zV>F3X2nd*~s$WGlWrZ0nLK90l_4t#`?oh`;q^1Wq;O(_^6ZI!PcG4-`BVZci-J`DR zZn`%@xDunjq7mvm^S*B-*(r)6P5S~9P0k*pIk3mP=fIv-eSFO*+2* z+{P{-^|!Zm4$uTM?SR2`YhE9mv9}h1Uf9(8qJUIG511<06~h~qIWu_c9i-#0rjEJz zOR?{#e@*D@X*BS6|M8uG@)MxiDp>@$J7yK~<|Erm|FU>ituEg2b1is??zk#X*;c;J zb5HYwfx*6hcJuEd^{;uDrXCNiU)KLTDg$;+FFe0GmdZ5HZ_RsXGJ9<3w!XaC_gcOs z6R~#2y?hEI>K%R4aqbM~CjLgCI1s3r1VOLgGB;NMi4*vccKZ*49S0q{K4EU@woG;e z-62<|PLySMqWr0k<y<XhnPIR8{-ebw)!4| zxHZHXxKZjC)47l)b5d&aKzdg;(L zuPq!3+4lMe#f@7Bez{hG`}wRwQVDF2{b7us%^AOS^OEC(Fr-&0Ef4RW*ondm+1 zFwf>M5IiSA0sr^jip6IR%dO|UU6N4!L@08Omq5zE@f#Y}BRjL}DK^#c7(E6UE$W5K zJEAVv7j##n;U82|gh)0ELBCQ$n!h)ZAhL>YfQ{uF2*4m=q*w9I)wEU_FJoYg^2icI zG!g#zdDCD3Q{!M!s)622Fg)K$=Hj&oCUhSYBzdnOn$_zk7HHCc~#BJmgHG# zIi4SNJ=_)4zMvCTlKTsiukp+^2!ou*@@iDB(?-{E($c*vvB> zuHh5zDV1&;+wG`7+52V1T5WcQv71c z(&q^t~JrYW1C_BTo0;&TLQ&Z&CmbE>P%yG%ZB*%#x+b{&V6lT3!BS zr5zs^iU`>NFYw7C3Je*2djL*499uAEotJJlWJN(0r;V%rZGTk0#dfDjjB=etxe^Ea z)iMW1psXi+E{i_7#cQAvapl;Mmb}H2sm#S-k>IH^N=do)Q2d5aRlry|))cwSch8z$ zjt`vFc}I7JASDY|FjB zt3ju{?sl!8*r`KIuge{L&JLKqZI{{I`WpobXQ`88P&bul0DeW2rz1TdG)*KaPq ze7z{?1AS%`T;!={{~!V<%4w=$>9OXIrtuc?GkhBqrn#f*?c&-MBy6)*(Jc!g93w1c{ejOH2`n`8WxNB4};Ai*%ur1*?2UZz%#Y=pnfBsUw8_`mr z+F`UCte=4g`e<1s=(im+MfmJpRM4e_X&iD%6k;|-OyqNjd(=+;&u#U`wbPx5e7a2v zA&QZj)((VJ{7~Rw4YF;#67(i#y3}7Vo(|7OVZl?+d?w>Uwrixe?sFsPIl@dEm3dZ- zKm4`{af;LM+oM(U;SBpsO?^A${dHzt(IhzYnA)9hfSw`=JcHne#f*3?Ro`pKic2Vq z%=4kqR+FVws*o_ZT*jDSu0A1GL58Z|S`3ph&T}Awc$M)(Hj${UYX=ig_%2W97E#UE zMT5Ho;QA;#4JL985nYiK#_y!0HpF6>pJF9fVs1wb+mB{VK*MVw08YLTEoqhR?#6Xt zb8U$d1)g9Pet7vyb;RN`UBvQVAx%L~U}dfep|32f`1If7f}jb;$&U$-lw~79w(*=I zn79i3^dt7TppAv(hA~I7giXvggI3ysY-mDWZc;x-y83YA7K3_WPu18HFe=0OjaS7a zcY=a%Odo4R9T#R(eevL6vtBQz;^$!2zyFLQEQKN;i94!HvuHw}v*Wj*uaI&ZAqic zx4^WUYj{5KnJ`~*4!3|KCET8z3^C!~#j^XWhjd}g)fZr6hz~y6(SVy3Y`!4pvTR$W z^!nHF=P9^T`vEaxs}K0;{^_zHxPd!d=&9b2wWUgB_lxd+-Jmc(IPSZ?&y5o(c%_|I z2NmojGvW1z*=1)ZFwRFd0Wd={*mNxq{fnK|@V3|O)Llr~2rx>Cpa9=izg!tvPl5Tb zRplE5h;sztYSMfNhp$eWc}?O0)W-=k&gq5yokGN}0tMA%%`cCyI)LX-G`Ge(yQ_QW zkt&;(nW+MTBhVC7!t)rN*LNrbOtYSFY8AX(6SDcrlF77KPBLM4h52$A-p7${dKC&lM_NxL+#W9;ivE0O3D> zW!B#vc1Kx;UF}DV-iWls6?iQ0QJ#t;6cZ9cA-OkjX&0F3`0ebEwpCNk%~&X8%(es^ z2@BOw9tXJl65hY89S`ZFv5LS5Bz6MOEyAhRh9|$D4Yd93Z<|A?q#po85`*RKPf^ng z#B)<)F^PE2GI#t5Kr9Aoi%z89hs%k{d)l>OK9Vs-))rAh!PY0(PvZ&=0y5Y`&sI`zsDw;Pc4ubesz0e}LqsNNTp987S?}PXK-P0}!|Q5CT?=Y`WrW z{cyh0-0z>tEjl*HlC*m6iQE@UgiBqJmaLIVraBg=VyX*aI@XFgI`0+yK@_XbwE#L^ z$A79Os*Pz@X63aP;6iRuyz3;8PI%CQz4&bc?2c~o!nNF=Kj8HVoMesdSk)WmhTds; zWaKgZhZxmziZ%;t$`0Ejm>GD2X)5rzO}FK*seF+|?X|wOTt9fhFH%M=HWc4{Hxm>q z(nO|ug*r8CZ}3f*?#?dw+8_p^FfD4+&pzhj!mdvryR^n|iL8_acqvS!yK#TKziJmO zT}AhmBQIIKn6b&hh(19ztt5s00|QlzBy}`M&6Br9=c~q96~;d55>z|MwA#xw9t9JR zc8(B2;RQ8QQ>@vB`CwG*Qg(Xu7W zwQ+?OQ{XClF`C%8l~Nn(0*~M0X57$Q=b%iic3ubfoaq(t0p;J}uaYvJel1vvIPA-I z`WPO^G|Yx*SJ)W`0D1qVj=+if*LsZChKE z8|nmO8D4*NB_J8CIXDj`nHFG6c^4E*Ksg`z2fpZYh1Z%Zs~LO;0PJq)_{qzlV_yGF zlUYpLY!n-zdZE7NcW>Zx&&>q&h!w@r@CpCabXGYQaDAojU4>Oc+ zEsg=lLdS9Le~!*MB+h7$g;Rq{xb3gonRNpu&?*pkAOf)au2~lJ6G8t@ko#aB z`Tg1wGv|&UrX$auh|qrq(D+A-lImPkXIycA#asL_H%fp*N(~?3mD~wJ)j8y(4RvhX zka$^a??S6W-+w=q_(L3SJs51DmKnHhJ%WIj0(}kikkS}k6Qnw63E*{ZjPlL-kP68k zOFz+ps^%g`v5a96(jzBI4iU}Nc&RI#5JXQaDbySpqtD)}{88b>oh=qmCORh#JcWth zI3bm#8O6UBSZQuJLDr0TpfS1 zO^GZ1cfyIM>1#-;u*$LODF$)zVy;2~!WQ#dO<<)G7mK$;GQ| zQ7RwoY-A*ML*03Ja>K zWz}vdR_6v-{fsV~SS1gjIvj=m{#8~{lBEqBV??2BUvYbAlZ>c`w><1 z)SR%+AsD6Ag&*pDN6~trrTWan3VykkHn1vG7hR9pMj8n>ERgAd=8x?vuyzY-<9Fg_ zyr;-`Gd-w~`M=N>NaS}&XGV&TD-59+g#Y-;;-a<$B`)CXQ8>Ae>3(qAE$Zkk(jp!&f7-mpl{xLaKY#Ciq) zll1O91_{U`qCQYdI(#YXfM0lOEz}7k?L!{6m!tQ1*a+@1f3&4WG{nf`C7z*e3MMGV zO)uoF?yv|mfL&xzbp@`DF8D*8x0pBJumpx}F0Cg`)i1|2gQ74dxB8V*G@%P~YU6D8Bw;^L9c#JhCtzThG+wtNpH8sQJYmwc3@AnANh+CK;UNpo-vN~CV z7X-WhyYXHW-qvIZv)R*YeSCCo;J*YntI|}Ko7G_yXYq4#Oz@=Nc_+ON$Je1of*ZXz zM;%ZGW#rA7-AR>i(&`J|xiST9=^dIvV=GfMl1xd7yHs<2g$K@9!DEdt7s!SsT{X5X zp%rcEK$`_j|C!#T$nl6DzJ%brV(ixVi1A3&zhDv4v7WU#4VxTGNgHbsn9~UK_c(H>p-}Be;$=v?lf*9phmt!4O zqt%-89%SzCe6_nzzjozemr5v%iz+RW;)1eP?8XH^R6$0~E?}mdpPa-fou-6^;k$Cx zh)~S;o4I1QWmRU6@5?l!e;{jOQjTtCiPz;%bpbpf7`bDwIk%m7y#Thi0rp{@c{R`S zQNX9a%tE{R0A)6w+`o*WIUlt70*?5=$3RflRXQ(gAfj;~_*gpt0FfQLCHvmt?jK{M z%RHI@kndJ+LfdqDMn)&hLziTsUu=MAL|5PggnlKRoxm(`OfhbLdEE)#Zt$eO#X?Ro zuynAbAnR8e$Q#0l`jC`4*^o;7po@_uqWP=?*~Q zJQ74^^2?VoLH543dJ<_@zHuWVe9jOSCphaVyDt7&o8%;LVx3Tzy;6TbX-u2ef+@N) zY(CnF#I!la=TS^Q;te&R0s{iqCw#Gy5GC3i%jaxtTrT|2$+6AFtCRbSax;)rFwxgn z-x7K6E$ph|hIBK6Ep5t`GErr|LHPc%B_C!|G{A1dG``W4GZY24?qbso=Ye)I=6FPl zp0mz#pGE-n$_ytgdjH4Egz4IVa-)53;zJ5a`6fA-~X` zO7$abwe`JefWSp*BW2wSbV=6CpE&>rpmxI&7)&9S;u%PQ(1C$$z-e$m)fe{K2{f<6WvFr#GmYJ3RcDqKV2JgrwnW#yHsF=Kq+9r7UUAk7L_GHCn ztXBQe51kK=kpja-WnuZt+T+MYGPGbL*A+kX4NBBP)+e$AfS7&)!CEeKc;g(MrRi~VzpMtyy@=lKGeK}+=PGE)a2hGXm>;e9^x`OtgGloA`ubA z8FVG6?)1Z@s=-x%fb@H=wPTeYtv@%yNBAUDJL33DJ^;6|H z$iD!L#hwaUelXs%g)3|_q-8{Jf~wf4Rq~1kz;6aWnqzozFOPeUu#!$*fycc2BnHk6;RZGW!O z5nr`p2-iwu8)z%G42^tF2lbba5vyVc?BdQ{=K){=e)zN*P068qIqc(`vJDW4LX?KR z261x`@>9{Bo-Ik{Ow&q*X=>tBijelGP=vcUZ;v}unKWbrINs8wdmw`?p4qJfR8Y$y zSf8B$3@B&$2Jqa%vt&8Kck=d#a_7k&%?qVbZ8UoSSjnJY1?Gf9-@bw;TB2U2ba`LN z;@z#Mbenmos^|mgKJlafou~bITQY%>3f|g=Y1TP^aI52*Z=9IfYxHT$-U#j~yDnXZ z1vT%jMd|AwYdfqRphA?+JUbwTD@YR$p$}0A^8@MF`)(AVTvXYEgeV{aXFNvb!FtdR zAr%oYM=*nsjuJ6ko4S9zEC~%X0KU9`&A|2N@>x#}x%(7=7y{s#9HFTG6XeWmA#*$8 z$qO#JccxK=Svt>1Z~?JkK(F4kB&Npfc`op%ll7grHZofu0kRy8Sn+O|dD5`>E>R*5 zq)I^t&RHUab_t+-IBy1rJF14g@m~mm`I@xXNpGgo6rHg+$AqU@!3`oVYS*?mwS9T_ z)asQo%i=ivK@i}`gEy8air*8;z|P`sK(2O0yc49&39Ym=WbHM&g9Ocymj@i#Y$$znq^|udYq4pIy2M@&@qUzbeG$n$<87qEiz4*x_%eiD%zLgyLJOfn!BK>19ujqvu*=9Lc6T zUZzI4@r;Ut5-W}hy_h@&1sE1~>Ei_O{VAm~Lsxx8U$ao>dUl4cQhxs6+xI-d zUL3nro=}c+{DAHED$_5#QmuASU=C1DjcEhWo9;#gdeoVqaZyFt&{KO!8i|=SbVFF$ zN;;9Ew``=a?JFCPb7X*F{Wj(CaGE+V;!J4z72-LpW%z#Q`2~r-hz)vBYr{hh*ERyg zjqriX5FQ!mw*}IJSAF?bdM&**4KQ&MkQ=79b)GP!jfxk7tJ!TsJWYnuKtG*nqzL&k>DeRph?9 zzm{r251J4JKtfVq3iSE+{{(B5OX9)*Yr7KisxiaDk>+|A$aw9{@jFHa#Oig6(3L&n@_j< z7#PQ0td%4LpAvu87Ai3V>#qLQUTH_V(d3)T;p%2F~XhSD1u|Rp=Z8e`)dF8at-o0f}ZHOv*aNj zO79CgPG8Ie21X8LUEvjU_Sz;{Pv!mk9*)tl6JC|6dhtLz!z1Q?Fwy}YJgG&TpG>B5 z34`5I;E^sVORvv+Y1^-8Wc+^W>GgFNYmw(SD97~KKVN_Wpg1K~x7nb9e5KWpziQ}w zb2$>n9HmC+);c!}wszI^=EkE-gSL&`4|3Rn`h_(uk6?O*IIU1l1ODDZ&J7IFaimfgi$$J=^7 za4+P6H_>bqn|)Ddob7qzNKyHHMKlD`0)z*Anfu40$3z2fx~kIBv^c0JG36$x0sdEB zWL!^(&MefEB~l&%lyVswoEbVJX_lBIop{a|I%;Ji0uGBGe=n||gOp{s!Rs8b*>ZIj z%%`kz#b{R6C*WyHn{6i59mz=A^+bI5!{C;Jw7pFV+D3bb`h%DO&da8o`-O!%xv?F24kH*d}N zaVDA~r>cSw(O(?afwcsQ1_TI^0ujs;vZI1w=<;uj{1v;5qzOT98Y>d8uIbas_^q9` z4pB4ME&hPmNn*rsc!n^760I9CMc!)JxQVFYVV|?^Ys1NkYlRmNN==L63WGpmG7o2( zy}b4KK9ctWX&9`BGv0TtFP=aAW!-41`r6P6m^lO}QDp1Lev7E5eBa*_XP5wKnJW@3 z0Mg9b#dGZ+CK=pIzb23b=#e@>Cu`JKb!r++90R$M${ghA&}%V=d-$t@bbDHB#`YQj zG9*0}{?N^2Sw#0@MBto|?i zxIW@nvPw7>)QF(Vh9;q=j#xG^XC&9X?S4H(YAX4&pcba{kwS|v!*awrJIP=*IdRw>S#kzL&>>YN{%BbQ00F4vp089f57D%Xn)jalql zq&6D^7w-eiKsHlqnHOaYa*IN1`jsQV8~&>P9ZmA@mw{YbIiDkJr!rb~zdWJ7xN;MJ zXKRUxZtphyW)w|UnEH1GQ-CQUAdx;L0@yoi4lHPoS>i3>rdUk-`xjax4=dT;z0vbi8R&CMt_ zl+sj3tP07crpTDa^!t*pnuItruZc$a{QqE_8w+x zb;c1kw|zc3+zCTcNsyCTiSVa)4gnpzgoK`1fZ~0Y4i_s3`p^cKzjT#Ab}oRDTs9B_ zfAeRY7vVf_xKYF)72Mfp=;608;J-lU6mlYPj#$KhE_?2Hv!WMCiUZM~c@(qPS*{C& zRc%jz!dYUt7@>FeUe2BAbtj{1jfIx>eBXjn3-~*LHz9NhGhS&lRC4b?FFl%#mZr_b z2P69ZAWF3C>d#4W5RBTM=u9M(DWVY=i3b_D;MCTZhnQZ8@DPacoh{)HCia7}cxjA^ z8VAQT)$5?gvMR~^?J$)d|EJcuXEitPhL?i!qE96DGr9c{<@BTn!>%OF>%B=*RZd2> z&sfs*3Fc|FVYk*91g|y<@xv34B+d*Ofe?n;=)C@d6i!L9Ru`qFDwq$ih~#}zk$z^4 zS;54q%8y^*hK85ENF`!sl@iHc!w_!1!?XV^#*SN4_ey@~bD_))GH)t1NVfQSn}>e% z)b?wMqYSzDjCH8xc%>-)q~P&aS2Okd5?sMer}Y}CB?ZGrRx@jgNQ7Ut7;gClTlsBZ zQm!>7%}s3ev4xwfWPZ~e4rqbnKeL5Iq#rLl)HFci%sZ3GI-#+ZkiMuNgA0+t>Y`p| zsRw_q-G?Y0*?mXsQ1We{qtyrv2Mu!V@htVGN!2)Ez_K3Y2RRCag&uGa3?87S zTys2o-QNyB7B$HqT&KBoc>{Nx6rD6I8&@3(@IBR|sXWjalf8Br9&}Jt03cC=8)`&S zf5L2lbd<%d7I3A>QgHSX$+ewDY8N#%!h8XtnW4LV_3Wk6pu|}FJ(&>J_^ed4?Nkb> z;F=n93(W|`i+B@|T?pClK@yOFc*Otx6!{m5Xm@VR&Vsd~!Tc|s8!~S}ws;TF`H#T0 z{UQ!Y4zHp74NPcT#}LFgv{EDK%;!=e7 zSqBlBH|e+ljyIg2g$z(00Uh_r0uUACox3V>l|GZ@7FY_!Cdeqffq$)=iF9^Z^Vekv zA3tfC%xSx2gE1AqmY)Qx3^wf!2y+uO+ETqB&$R6|8zFm$NOZEHC&MOV^yBBwY?O&L z(aXcW(ib5>4Q^fcOTAWNU5jFnaj-E)wdrIgWfBH@(;bS+cXZ^C-$7cM=*CW*%>|=;+=^9Hi@YHC7%>N27Qkjc$(-Sa%g-}DJdp${50D5mDt{%3Juq1L) z#plS==NSs;@e2ftCX5y(9PgfpWG%9_nmlbTKIZUIGsH?t8ZPr7nXC$zEh#e=$jyD|+>TA)6EHVzmRc$?@~ zO$o~^f{;>y=hDe<2%djI)-CjAIuyC8kon11{BqPD$0SHW5WEx#5LA+uE58Nu@S_j6 zTC4qW9!9uB@ZJK@37v4W5h$V&QUaN{kOPPXG@m4xkbVGDhJ!F?<&7&0j$r#f>XB#- z(c%dM9UGeNuK%tZ#S`QzU70%zn08cdnh{1EXP{l;Dc7^846iT1pnDc<9w?}EY5a0v zQysg&T}-`j4}862tQa(|*PTKK;HCgMG#9Og5UCPspHC6fDWbPtzT0uEBfT4iJ4ceM z!1t<&TYzt=TAabRJ`&-opF#@}y4}Do;ifS66n0!d_XS!l$6gf(iR2Jd$ZUej`T4iQ zfu1m}`9TyUV1NO_l!vU)A$%bW2WI<5zX%ml0(bRQo8YFMX|jd#TZ3OW#tXZb^%i(i0(Ah$H)V*H+^)f&b zz;jRwX5>ABXdfP19pX;l5vz=zHu;`NL0bCHNm8v+2^zX@5Gb`$0{|P$RGd@kRDa3h zyol&Xr23u@cHQsXs%rd1MpC{BWHvRtZ^6ibv8@6&+17vE?o16&wPF88q?fgXbnW%G zhWr;ipXs^TXpUvtNXYa7XIA#4z`vPF~RN-F|y~vjTWwH4eo;Z;{!ea1QZhA|ZRG zo?|y97os^+fMFiI@E8%2AcCf|2_F0gJqV`@b=RUX>LAtzYOS)svv){r1(=%ot!9`&i;n@42Yo z!aVXyo1{QnujfO$z$OKL@eREZ4-i{_nRQ5$gXJn80ttSAuHN%kkCm3keaMj1#H+QT zSyoxFM&FXCOpt5Pvt_59*}I$qct*YymA7-*bjPU*)%2~6PVJeVBT1=6DSgtC5U>rcI z=A~ZjTpQMngKg}@&-IAzr-6|(j4OZXZ3;CP^F2q2yPxA26ln8MgB(OxT<&u!ljwOh zmVIG@$$hEh-q)t-__t4qnS4pj|Qz7EHk*hjfqy)<^Qxgba=4XANGqR zC&9ALp*s5KTqD=QuCAk8rWFm*-7frKL{1c8Ni-7UfE3oqys7Q8!gfxj!#2YSS^wkM z7}bS@O+y4yJVZ&lG)C=8bRxpwX$Xp$fd5=U`7jUH+8qSnhvUf&Q4|(1VD!t8B6``3 z@c)D~oS!xiDq%4LO8nJCr5|9$X|JD&F~z5m)N%{l2;DtaC2+4JW?)W_u|W-Q^nw0B zOSe#GhQ;>24G%pew*jB$+}A#K-&k(xVu>!n>tc_2%6HNxA+ES?Ly3=4#)LsXT)l44 z7cZ?b_mlbS6~jTpdv9L1iO}p5L$+=i0j$9I?d*WHte++I6iYxj#UIP%v1*ibrVO(F zX{C%`z|k~Bfx%-?xRwiajK+ms(y??(R%N{-y^xA5 zHgqdeib?8{l)%mr&$~7pugXy-@@6DuVgXBFfsQhU5N$)2Za@X{H|E<)!cQ>L>B097 z#26;&Nh7uXQs_406Jo`-UdTj;HX2V@Y-+uwfTcRqK%aTm0Yz#X{LJlB{ z$kheMI(?q+op00<1=;;)x4{h<;YOx+nZiM7=mS`m*%J`!r2=*;25qI>gXf zOuYvYWzSjV%q%$E*dh-Ilt?2v;o~zB|6rMll~W24{B+$+G^%h)s-146mft&=_rpc# z!XMfTO`lIlF}i!;HjTbe8j_=i_c@0^Dw^?*l!;1>Q0~*ATzMv}!0YZ_?>tF#T`yg(RHD0qU^pG>t)fqPhxPNmMgBlO3MF5Non-JW4v~bGW1#rdINps zVZaKhvj#Cx*xftBS=PDy;DBQW;1lm(K>p5YLxt2DCwWu)&CJ8=1e7zN1WDHlSN^1t zTDMHHHs%)M{=zq5cGiMN*jLw}VcJJ|t~19bo) z!UOfX!UU1-P@Oh(F_m2bugDR?E*)}A2dW4i^uG98xWDrh)QfT(me-Vh=Vytv0ZU$) z&%lJ}Ms}>(Txt5qg;;QZD57=yMx4I_^l*N*Tl$8OSjdCNMSPrKQcTsjVOJr7|GecY z)3{PeQkb(oR-OKb8y!ZFPI959YO^T9QJQFIFRg(N*x}~y_De??>5~huxdpWL$(b6# zkOi&jWbWc~k2K*T&Jt9+vR!a&GZ2BxmRT`B5@cvkRO`v0TlBkA%sRPoqoc);-%_$# ziE>Iv4~yNrO#)#SpU5b{4wWd*vNV@A{5{wx1Ua3<)3_L06aD0&!d$Ndev4{uf%Yy; z7@LkvcE7@i2d#HmhFraF@Qhj`A<)7qN*pX1ZbJ!<1VB4H+hIvi%g-fu@}dwTx)2~H z+2DnqWPUp<-uf2%{atHOhRGkY?_TW27I6X;oYnB}J^AKH^wlTeBCD0YXgKf%fJC4Q zBROTb-i`Wfx>j65`=@L;T7<8T(C^Oe)mgqXV}5Z9Qo<9KAOCCB62=A<3g^`BY{d`v zn4=wjxjXoM9<%kEO-Q_<#q?!C7=Rs*UulB|_KYpYJIn34Qw}mx>|GzG&UtjTV5jJ# zWfrRcI|_TT;r|=i&c~~+c}ORX3I9CLz+ZNr2X@<7YrhQL4o{oBEQm57?O*L#Fo8L_ zdOIoST-UF9?u~QW+|1|n| zuQQl4;c`x_V9!m0C5LdkrPC9l*IMnrLO|-VYa6JfVKoBg3XaMpNYH01HG}weFbH27 zb4hfa^B=S=?r+fla4A;|Er&Dm@S>&h)61aDTEUnR*J);!eZHr;^P001g*)*1LnZTu z_yM7X@Qp;s@`?9SxZV{x1q_+@2|mVwZSKRX!5+%QZc%V+OfY~O0lGr=5s5TS5IjCp z4_E#N>x4kEou`$3d%UtJvkYdJERQSHiNi-WH3U;~2u z=7v(>(KegbDAcxUXgw6J4>F1ddh=%fMFl(nx8EP||H@w6R>rsA^1GU0+nZsSeHcD8 z27bU@xEZvVkxIZA3MzoN$72;ehy*~t1myeh)fZ@_OFdf#{qbWitW^00mE^zD@!}TO z{}h9L_TrfQ5oK6h0|w0T%B23`hQ;?2mgZZ}anZ45WG~7~S2i4Hk!&V|U}xj#-r^g; zfavLcqB8*A(~=!D`r3_j_fmgM91!cy3#5Jw_Q6f}pnBzOx$O8%O1?%@(i0@U{=zr_ zI=1Lr&Qwo&#F~w}n(+lrKVYN9HrS6-$sg7xaSwa|5fzTsGLVc5VD~v~HTG)MBo##0j=QegO@8+BPn|Pv}^(lZj+>@vnp4K z;$3t`8{Fl4Oh&Z{>D5#Sb|=EoX)Rcw67v|vaAB;<$x5S6UBv6sKt3<1^Wre@)=09! zQuX;W_yck1c)uD+f#nUD>}I@wEGoCH23I3!X%eR%$6~I=S%*qLeuz+BsPauHu#he~ zvikJeClwU+?igoA(;;$3wlD8cL>kSDNUrq>$R?WC8)v!gLhjb%T}k3rqghtck;_5@ z2G`IiE-IpFP>-HviiW~UKuCL{`1gaNU+x^7h;$Bgyq%q0OQrKYE%?z(a4}ul|y1pbl;<<$OI4 z&kIY{xW%LT3X&^+kYaB-kPn+m4F(kTn?N$D1Up83HDx?$v|r{`?6{11P^04KoNEhp z1@}HQN`EfL!or5P4JWBHS=%h9jyR4wD;zCQ{QCLpTz-8;nFe`=6WLaY4N-hs0dCl} z1lw!qCvVJDZZ#e`8>0qVzQUy$rBwE!aKRrC&;@T@f|IvU*fCsGv1Al059+V5Xf`6t zGz-bUoub0gJO=g3cw;{&8m}Rvq!LH%*URX5LXew;Ck;1q2pDM9I}s#of|E8^{{trr zW8zDp61H&I-%#bVpR);!I?wRKz&8)ryzsv-&oPCN_*bO?ZwHzR(T4$tv+2f0pKtyp zw1bQm4V+acw%~qPQpJ$~;#!qaF)%TmE~no=-*OtY29r7jLTRk7+e z)uKc;*l2OXk2}G})@@1~Ic68fycYUMwkh&_?B$mWO@KXq7eT2j?PfojWUO434=iE; zU3wog>{m2TlQXym`VMTaI+EVqwY?0VY*dpy3PrsMGINznx&t=JKg@cef@CwfQj}!GB{9fInv(Jch6~{L4giQtR;EVoR&m&xN6C==;5Hl$~ z(ixTZYu)p;C3;!@U5)V^U(P1E;ppiw0=-Fs_nh;g{zNZ4CgzB}h1g43)&P{T5xBL4j@AVve* zS2f)82q3aWPk0H|AK-_vKg>o^F}%rJcVUvK)z)uP=ca~Zi87<76zl`hXdLYWUCM@} zp?24M7=>XhxoYOazr5KcY=pUZjOG8BK!WtC1Q(vBaaSNfUrOhMNgmIoHnqQLj&i0$X?XjCKCSzK>1o@w7T`*>k!~?u_UQpNCkq}{ix2Q4DY{zq`Jbx5rm)> zc&{vb?g@)zrQIu=D@Cdq(sQ>yeleFJx2}`n{tNfT(^(uA<;>ni*Du`D``ust3d4)# zooBXkB?uF5AW|;KKxCLS<5x5gFYtf)Z=svB?~pM!o|h+q?N6|Ir8x^%IsP z-;otlA|~JjW<)k4r=_RQ?|%M&ljvQ-|LIi0t$eMJYK4vg{kM2ggR)#m&o1BMKnY_9 zM6v->^Yya<4|Q-Y*>e!z(umKtNr)ZBht=0R19+&L67x;Mp7zJ92I4p)>gS4rw!tX? zof|mQoHWv@aIn5AmDx+fxW=Ci3uA1AzqI3-46LT*q(15m@bDm?VUMI$L#SZN1; zT+N2U3heueZuHse!=(0J8M|B};R!je>bSd7p4m8TE9u505C^nWU1%PmVn=BmIm8eH z*C0S`*5qOG)U4OFnA)?**tTjK3p$RXjN%j$U2z;xgy34gqMoWfhOn3c_U7ToMy&3d z2FM+()dgfdAKjjMmleRpo;;uik9TBnVu=BrAiLarG&wDaL2Rw&q-Trn_$39aHkyy1 zJ%5w{#)P=sv?2Z6*`PM3DGTNg9ck8eB-rK=tm(`RGrVF0EbhN`RC5V-1meg=mL_Dr zv67aU_~l)YM$g9mW}?mrTib^pP8fY3Efq($&JAFfW{B;?7_Q!B(E3Ed710bQ2uO-MNl0h$@ z>pD_*jS?1bzSg%){uI~U4qHXo(zPl<<6-DiU7AAdC;s&fH$NZEPk^oA6oBl#ixY&F z@I)H$Do>c}Ks;$)AjM8WlJExx_v-{BLoDpzCWqcFS~(w}esvxyqu2!2MY9ifw}TwP z3z;09ztn7rM+mA@t-p20`>jm9WFIfnl$9urSuuBbl!f@*=BGi&ed@c9QRiDR(Qp1C zOC@cmnU^p(*M@~_pq^cNi|qpgEMknzr`&k9`y0E;Fq&{{5*j$jst3O*WTjrDR1?l9 zGV4x1cM%pcy;XxW({u5yoY-!o5zR88L?%HE^%eXqzd(X6}7 zWSKZ@(1vrq3G2q!>~58i`RvOw-T98HeA(Pp@nW(<2VL23ifdAUmKIGqS}uQhCi@(` z`)|u8f#OM4=Q?VO_{V+$dr9Fg|5FC>yx+@kaXa?pBgC8NlTz zI$MySmWbwi9%3mR7cLUWU`x^UZ>L~by4neLD?$Q{X-UZ3mM?C%KW+7^eAArr6;+cg zDjXcJ!4glB%Faf>BPePj)ahmV$0>pJRL3NFo&k|(!2_sSy4(dpT`)LlflLG`Gdtvk9?Kp zOyFHW)<08O^a9NC5p2QTPbF0FVCjzcU;=ZK`1g$1O=0Th-rY&?4u9bgJC_rO2eJ;h zzKsOyttlRe4o1+%rvoUrCllJsJP2DlXby-0M4NI&flxlmEv6}JTtYgR`~fQ0^=p0S zbcayel+ma*kfdA-^Tn(61>wi}a*44E;EgF0X$Gok>p*N%L^kc*CKwwH1Y*kXiFrQ+ zjY_#;$gA_YcCQ#|O~-90XR-3~sv10|=pGVPTKoIeC~+2t4c*HZ!os~lA{qK^^?oC2 zs(xOQoR%TJWDlW+?<(bww1e%|2MEDV-`lW&YibVsbt`9t-PxGBNB8cm-I$D?fSBIN z+w^{^PK=Y}+q_;aNnyHhZHKG#VK=UOlM+}u!!u%{U+geJrw;24oeu!b&@B*WH*le4 z6IFAVQ-^m4{0n#*eoAc&I%;k_wq#?DdgLay`fE7F=&h?Dt&QFX7&I4L=iHn$h=Do= z?CcRse8(N87SiBujv<44WxGoOAOlQ#_&oM8wyi@gg#k3&zIXF#qd`&G9IVS_KsMw$mwxK%_06_>ao)AB(s5v2|YERZk9rp16rPN~p zk?23l!rU9mNqP^4SLFGYVZ=y-5o6w^57|c!%cT*>0<_`hGIi`4qu!l$mk5^0@qbt* zd>Bw6Mx5VF)+dxNpesN195roqkS7Eyww`XQLr%xJr{%BavbFe}Iti^F4v0I^s2#W*A|;Z>EVZ82K`6IWM$AF7A$2LxCAkY7 z2$pi2{FNafAseO_A;v#W_zBQnj9seu`KjSzaBg{Nf9U%>boYMHH2NUsu+iYBM;;a3+!fut^&Pm>+v5kOkL`#! zm%R^{tin>*NPrImDy*AVe^wt^776y6Cw(6$FCWsp>Douu9l@%|xAfcIz0-n==Nrr~ zL&0Ka+w!#UfsgO-bX>b4T^z_pBaQ^)2PJW2P@A6ReBYJ8%ax3eSmx|^1)<x*b^x+%e?Mc8%PVbpVJTF?&zCZr0&HJSVI z*5_+7+BgTcyQ#c1PkPS5kR=+JZG!mUK0{O;0`S3+&I_=b>^$MW){s;bUv!i-%tmMm zoaGG9fxjLzL7bQH`7eiD=W!+>56l_C2VmzjsYGR4nf})rAE4Vo zQ#|%4>bXhABCT8aIBI>kzqkG7M}Qt8B^vezM6~SCkV5+>_wkp z=R?Ysm^HqeXi^M*ppw*P$O`YCYrN7LQ49~O(=40ZiCt;4^qxlWsYtbM`@pc(ZCKCS zA2&k9OO*_4vB|5-VEvsC|0t|1q~q`vXCblIinreL1y)W}Ip znfy6ymY6ye5BB!HFVvYJ7^YCB*T3$#dd`)psD{ZCC8mzIT7;<{9%w-`leCrHCs!}> zqHGQ0rj?J(m7@_cHgBZd_TDFg9_qxPA#ramoecrMA%x}1d51=wkQ|G!J~}$OTg*>* zuZW1U>gz9IduuIrzilZ2MoN%_^z zTF>meTJy{&RV;1Q`kHB1Oy4Hd&(C|5<>;6A-KszMG|#u_UeGaH`99`OFAfQm>1#9Z z^c;BC9@bZnh-%fVxlb?5JZ|msHZQi_H0f0U6X;2;vLLG#sPKcW_w$zWN?4TN*?TRpp6h9t^nCgW(i4y=kTStBEMW`qMG6{33*W zxTSyyb&HCo5Ye)X?h}LB*0zOJgLT=JH@U75`10#vg%@-A99e$Dmc6?UlSbKcz4gjxRrV@U*8< zu#eskHbNuCYWDH@*#{A_O*~kh!X|-@U^NTWRYQ8$TwnE^S+|8S4_qx+N;D9Nm%*gp z?#!-K?nY@T^=?_#N@;ioSjr1Wjh0=Yqj>3gKdyg_*U8Rn8pELd_PRV*VomeEE-v#HpF#A=`T%`v7p)*wbLv4Sm z<9rUUbB!!<7)$q*D0{(6wZd)Q3N`A4ZAmP~b;cmsCZ#cgMBRq9o^4LQhx$0Ak^!%M7S2Z8>FJB}~Adtl-Ce&SCC zORRAFln9YyGps?U+TZYnd0n3;ta#SETBpOQ^m(?f-h&S>Rl>EUNTzFoJ|PO1bFF_C@zU?>|A$ zN#IW3-OJ0be=3J2bDBE>haWbl4Y@j7*q^>glFxQ_^u&>lAO;GdWj`~{rO|)*l4D zJ1%s9Bs+B86SoFmGFz*x&6vt{mqR8NTk`nrp?HF+_fQ3Ixhiki%nnJDBcP#?0^J($>^yBQ_1MwMUw7Z8EbVW@oifLCt+^^! z$TodXWlboA?z%51cr9)OBWShTC`zZ2>n_aWOU9bKjP%4l@~oP-!R&qMzQA(#wzWs= zu9iTw^qe>tQG}zhMeJ%yriaCZDFIT;Gj=ZH!rI$?M}ea+3r(nu zsQxT3FI}ZbKFz+Gd;f%^(SN1!2TignovbHKo|I=j>MHSETh7YsW0Pk}Tl<#NWGx7> z3wYJ%sC&xDveb+%s^Cmg>+@0!ffCR93n&bul}V=t-}S7%feJ5H>lRVPvi^9!>XbeH zW?`Jbj3y(mz9fBRoaRY8VG(<$;b_vO(wMYRe75k*9Ef^5VYgU23!fE@vL@c^mSB=r z?w2ANL!h2xGbglO1XvY4waG>R#mp-^BIkFI!nRI!0Hh2YSFp%6jUL8rohN?f7pSo` z+Y~t=kx5RtxVUlNUm3!{ReS&=?-QX;9j*pW!q1*PBO)d?vakrT=E3lr*v=~}dmi1- zDOE`F#-@`=9^||3`c_|K{(OH;f!;Y6%rb|<=jo%AbMGLK{=EMiB5F`Zn=lzIf?h?U z31bwb2v{3b^8TgMVM%16srRvQVL)n$X>DusytJJv)$G35p9i4=;PO*0!UHP!B9#IT z?;B^4lbGl&ssYbXAVc>9i|@g}<{TR<7@l3Uy}xhj?EC}VGL1U<5M~ZGws)<1<`;Xj za0i`$_muvfSr2ZH1QZh+H_q&fmbJ=dkmH-TK`gz(qN)=Krm^@mVR-1s=z)mw9|Ps+ zaxo|a-+wi#`XHj8rE2-me6ILYnGz43QYQ8$yWXhN(yFNvxLSwhY*B?Vgn{mFem}pp zRhO{#S4DGEQxcmYA}*br0Z3`i&COkdwb?v!fcv~3Bo5yFR}y}ME&Kss(Jaxbr{PP^ zq~^Le*9SM$Y;ahcY5BtG)1LYD>yrgNLGG$+r`gTZmFIlNNP7{7d?EIHa?;W$pawa+ zx>n2T1W0dFTBl1imZE2gho{nczu&Gh%9y$jCh(sb-GCRqBVKCwjI<1sAarXS$^}} zy^m;XYpYxm+t*NB`oFWYjnli=3I_l6BH%oCZ6=e9d6O6xwEm46ZH}7KI&v%8Qhq=E z`IQh}m}!&X9C7&+I!|ivYf`M<$(i*GEaPa)a-|-#n(g-%JQV`8gK9Ar5Pa10DwtuBk1*b%!TkO78 zoy%d~tTxI)-SS{esiQ#9LTpS@F16Qx)Gk8VRuX@JjDZ3412m0q4y3xpYNcn)cte6x z*!GQZ+V zl21jjWo;1fUVs!6(c!CF6<^uZXnn>gDxdCzygw9BM|Oe6h~TLR`T-A+FvQ$}2$73y z@B#eA(?!hwxL<*YH{2M{T7n>^n(D}#8Bq-4xsmQ0BZZ6nrJ;qOW_k$Z;>pHKA=M#k zCY(pDpXoIf#s^u5l$WGi*1mDH8edMfCcoIqEQjvqaem=`Xk>5Jww_gY(u$^=0JUj@oYfgy_+3jGfaU? zpss24Uc%Q`6s~WCoBCNzm#+gCfc%1j;EW7%xN9-En0HSmL z@bxL8k716;g$_&qLAFLEg91lU8jIYs`2}ai5XT`(dU%SW!(U+D!$kMR#+|eG6AHNf zIkfAnf#)}FbU**~zqo6l0q@{#JARHOxL4_Rn^5g@?*ZK4?LL*M!!vKfp_c1l$-8sg z`S0T52#B^R5{v}*3kYOz78p+XDf7(r=WH(vSd>kc3(9`r+Ez{3fofLIG6R>v;O?)` z)sB4!Al^7jYO|dE_}ZGcM_fIOBP?s#qJ<2=rFa#Pq$iyl^J5xEo&qun$_vecsnJ^0}7K%avZg964omp<#pd`cU$V&Kue~)jr8ppAM^6(*3M`1i=l}^RYsvNw~jte=WA(h+sq=jRd$?q zi!Htbjrie`@J(m0DGSKDH0SE8yZEj>vi{j&^wU^BFd7z3um?h_bR9 z1XGc?xHu*j){gJH^gANVXJ0_v-oUyee92laT_FDRM7{%vld%V0ZQoo%iK~+`86e)lF>@T6Tw;pSpw+Xyt0#~?unILG4=ci@%4?FK)93)59%Hz3U0{r< zaeuYpQWX+x-)84hTwG{mBODYa$x{E<*tTbnmKHdhb9}Wjh;uYSQ9NorwOVCYLI><| zMAg^+KHmF0B(??LAXHF$h5l_z2Y;(_r(l>_QcsNBCP28_r^O{pK|lGSTBuL*dGXVx z$&m5#Y1)@Xm^`WSZoS`jq!dk0cJh!)kn3e#dizyg<-}kQ$VFvfool|0AN)}5?{@6M?iDzu|G01GI|%YuNt&9P;5P2p zJ9Y|`tGkAl{k=i=%v-l2r}RR3kIka(35kCd*?haWgU$&q?!Q0vr*|;0k78jFo%R(B zy<>?A_fjBqlySq{>Gi1MHWJclS$D?5p7Dh=FNHLa_eX{ zFzrve=!!-V&d8@-re3#Bv4RT|rTMr2S64^d(YJf;#IY-25ETjW>fXn7Gt*ew?3Gi$ zeo?gUCli8;N52ZiD-6WpQ0o%)vQ-d>+;3DhHa*=R@rtuyr?>Fscn-3ii;LpMw+(pf*YT;xa`(qIIWPJ4;o zjXnO`o~an=>@OM*xUW#3`IRFv{eT`iI)h4jZKBVwy{w)@59}uXpPIRx_X744MgN}jCG8lHQMNH z=TL^?_F!~ahGOFbeeu`C#+$Q}^L0?hgLxkUvew^Gb!B_ZeNC|V?p#JibR1I=WxsP} z_vEM}N7QlYTfGzT!gM$?jN^M0r&#O4U|pc6z_f!k#RC(k(0RM|bWkZXVa;yvf(C}! z^z`(hOwr8TD1orW#YNwn|IS$WF`H@4Bdf1ZF<)=Tdk2rTo&~KZ$S18hy7n4M7fgh2 zI>Ga+AFHd9a;o4Lv0wb;JY zZ#mtcb#z)_Ru90GCYk0rmz7GFw3LQ@yM`h)^PjzJX8U(2#%MZNC_Jh0syvEghq(|& zlC=6V3*pYN9HNCuKk{H5Mvb%M?dft#Ik!EW&IRYAt}%;P`>|+PzLb*md!N6vLc-~v zz+0#m>~#mhAU?X}8l={W|E1!J1OIcJsVaB$_EB*<{VY!6#pb7c3Gd#Gjmk$KT{K`u zZJAI<>CCrpAONs1QqPcZ+dAXo!!$S|hAFTgsSicU3Y{%J^?=uAPZEH%C^!1U%*3%fHeE?F<~w8f zSc1FR6oywy(RRQGn`p+z3CNwS&y%swH>*m7bKE$rR+~1Ers*6YrNzd&)YC~J2DMt) z%!uno$TA6+9#OR(>DJ{R%()_R7$RttLBob8IewV9^G^X6y>Qf1n=hv?mq+KIEPWhd zTP~sQ!|#X!aP-V>Hoy+txc?mOpMcvk6z-h>cSnqfh*);y_O)Zyu8TPIxomcC)q6M! z7G<-VV$LABrVi9{i7;sKI-m@UD4|Hz_|7Cn!ha}NIJNq&F%(d}l0zR}-Bj0B&2PPg4P_pYd26=Zp|cib5(_>&P&iok z(?08TzuzL0OMZ9)jFF{mTaKmRevur>m>2}i2KM&f;dVKX%UE|-aKpSt;PS3nWA+^Q zstLdYHX4EP$&!6Dy)ACl+;Tuo3huelJx7IgVeW4tq&0e5{Hl4fG&X&CwAqFh1y>~_ zFV-|4F_jScsNXFq#SSZkr0!35Jd3iYO_VF3SHI0kk*J0*H53=LUeE4%N>0^) zw%I@|?NAc?H|i*nB-*FE_B#!BI~DSuABk9L$PKQSS(9M5+}_s_<%%_4M}$JcSF&d3 z4}=KWN+F6vjTy+^OZ~ei4vzhjo}}AEdLI^|duIoYo6C*V4>hiiT1JmnzGSYHMtjFC z>D;2Sw$!C9rJa$qoy1qz!(66Y%T7W31USi3;YQd6t*z<6fj0*HsvW=w*+2iytFKiA z7y}$;@FnXs_znyW9V_DcK#d$;?G4wgHbH|2J89P2lEFh&;GOWGM%xK~n!={u#nlBn zNrpTe(-YfsZDt_`gCaXiZ#KlgPRiP`#DMWNk%Y8nf{gho(!Cz(TVVo^XiIUt1k>d4 zA3Its>JmMc^1kws!}xt6U&run9j|j1DM?OBY%SzNo~oHZB^tAq!f6KZWKC;5)e^S1 zE$I_1(1Xh`qv_C)@9<>ty%x=Sel6H1)CpwY>+x1osW7WqLvW%&DluwoSztu-*If+#3jAhrx>NI0Ae_1fSglyH`CFHfD_hmKZh(`h~F2w))60pDErUg4g(h*0!Fc-PgM3!7~{iwha%pW!7&?0|8)Kgb{1UeEL^i zgy2tG2RxPmo|9!ZoYH%G>^Yzo%>Pr80l}TN2G#GlZ4F^2Mz=3&FP44x?cVJZbbIAd z-~o?E`fz#NPGoC<^_WFCX;2ZwPlliK9=kU=;-S9q#m{XyCcMyR9regXRj|BGI!EIcZD(kiS@>w+R z(u?0Og)ZGA5AI8w41`)_%_b(eu>|@+eaXBD|NZX`&o_e>tCF5EtCtfBoNdxRC$2bl z?+m&q)xBp|Jxk6=4NULEhE3c z<<}cm9%(hmo-O0n*e$dAw!e}rtbR271kXzJxtYjpTX$*%KCSRsTC>Ud&C=t;CH!A7 zSFQt3VgENosQ|?WIQEP2*bIWVS`tO7t01@b-&2Y!1YTer1h-Iy+vzX4)7w4y-zf~t zm4F2RXGS$M2~zjC332;q33{KFovl!yBsMBaY7TW_PI~`-*QUenAZP&akOj1LssXS9 zAgB+fjtq`9iuFp&Dzo84s9G;7({=*#C;wt9hb#s)(&_;@Rg_6)Qux+-S_%Vys|&Ing4K~N zk^5%ImE=?`ZH|-8c}nG&6C1g9Y(jJMbrn#kb7lOWP^^ccqDyEsdZV5qp%FW!$5U`(J+cXr3%zTO28T@FI}K$7Yi&B_fr1?I(s=4yFB@EEHCX z9a$JB@{b_#HUqQor8|tJ#rE8jkr*uowx;!hXW_caB;^>U#oWYbV(x^zpb#vAW9-fq zI=?TGK-0QZsa&*oKJy>i6nvR#ao5Z06Jx?Lz&R?^{p<6@**ZuGM2MqP>^i)xaafar zM{^_SeH2z`%-9(FoeT-g_e;b_*&3`Em(*5U@J>mFJ-%rEc};FDw&a2e;cJv+TTFBJ zYto_oG}~FV|M=vc)SfwPi!5#76rb7urp*@gO5(kQTPNfrK3{P$Z&kl1@jftm&(?JN zb{L0av$l2(Wjq$JF2#E$M9lQF?-Onf>J76{GhzKb;7`gDk0evW3pEe&?SIlai$o@T z(=>=;*JW(CmNoYIrW(u^Ov!;wo(hsH$tz2akbl0zu9^$KZ^pq zend`_M~^^MObk39*$V_XtvhAjrWvNI%8}Q=*6PFL(#_3})IyCFKQAtsUTW~~a?o25 zO!Bp@i0CzEN%iw#fX>Ki$XZZwk;QS#nPi=cRKUCRbo#(A*Su&@+I7%cM~`~JaHS(R z<#3)5ru)-VlqoLCmW71u`)pl#l&E#KvDc|_GX4v-m>srF>R~i*tBhj9n$A!3+|ib)vR!;2$>Rq}W=`7$K7@-GLwPW7Y*`c^sd9n5 z3ODz8l%j?{Eh})55Zs&%qDKFWW?1#la>x5yq2x@H#MG@KEk>JWpCt@loD6s@q|2 zQR)CbFxk_|maxt=*bJoVGIkVrtog78yvG{Y)XnhDK!W4-+=;}XVR zBBYVKKR%d3QCC=UI}!&8&z+d?2Ajfp(ucq0^eNN_e)|x~w>47uX_+jH)?P5Z>2*23 z{~-Q2cIxx3)-ixfX?W(n`U<16?o)Y%CrSAd%cw>Zs<$14T`WvU+5SQ@;rSqFE0i_! zgkLJBk6t4_X-^D;aO>QLL1?uZcyj*Yw}v;xtI`ESEhS@3!Ltrj)=cO7KC-hL1|`0J zA3Ab@di;YCub}!1XdEBX09gOcNuzbtGT4QSlqP<_xL(uctELmew<;!qjseC|CN7gtMzrI0*20AI)LY+hdO9RbaB3zc?Wve(`YiRRfZkWs{B&a&Zl4&mE_%1};`;9K7Eko9+x&WV)F)ps z+RaSkJA++60>>H-l!Jnb6rEv&NSL*m26e9N4QK#=>P76&oW;w} z^r;LJWhO#eGEy+y?;>9J?~33|*|pWxElAVgdthFxo(vh&c#><^potX3 z{BfXVh0I)vcZ+O9wa2FDY;1Sxze*TQwdV$(xBiW{O7JmlrFKm~RID*0d*10sXXjq^ zvxZX*rld+=);6A*CYuRS()Jt%uWecVRgfxTIFT>Uto3Wln}pR4loWVXnc{oA1Fkpj z?C{*%E8x!6z0l~S+3s^MTC4c~H9{cgvVSdGM}4iF(I{S7Qv$E#8NGe2y0p59uq)Bz zhtHCqoGH;>YbfG)p=%M0|Nl7Wt1XVeJ?@7Bp&Rd(}*A>BW1dcU&)N;zw4j zBn+%f9Fb{c)DOQ-d}U^Omi?)jo~0mNLZF1Tf|s;%4oq;GQ%MWd3QI-j5ciKRAe3FGg4?L& zKcZ1q)ry-c8zDE8OQb@e^IgXl9l)(sC1MUPHmZSy#sKH2AuW$&lP^|DFgq@p)HM~l zTuM%^rrRH?8e~5v&7;w=?!4?O0G&BbSfd5Jza%9w+BR%{EUj=12d#*G*59;(#qwRG zWFn0~u3?aL9MYX-%a>jLo|m7!$O&h?K1ElMN6=bepxDpY0I`#|&YTmp>@raUhne8r zyVyMTeqJ~y(Ts;4cqu8j4~zuiaaj%LQ`*mQ-OBG5xfgDaGOs3#HBNtG_>{9A4px}D zJ@7Lsk6^E0RyI5qkiHbbZO;u)pDoyCE#2I@h{kqQ7@O7{b0Z6u8p)wcf zuj3L@X+3=QNHsW4&jK*j>O&;ryZje7&a0>TWmi? z#))hhnA8thY17j2P-XLqs(|nVLE&3%NPS`70!Aw0u9Md1fC2M97EHUk)KG!sP`O2HM#CDZaWLBqwuB%wu>WUgM zj^zDOlO*?!i~TLqH$L#4pNsh!Z~ST~c~t)iO4&Zu+P~IwSxcmO=7>z|^$gDgu(!_U z1oX0**P2oM2iWlJ`Xe7PJW2Gxu4ZA(xJw*(?u5NbrplNXaNNN7T zc9Gxc{kQbI;(kZEp0k0oyOk*t(S>Qg&c=A|*XJLDrhvnR+p=+*k9Y|tcJ}=D_~$*; zxS25sbW`??Iz!2v!N?c{E&JZd|BFI{lBCSC=GqKGY5j=5mS$}!EL)aI)nzs52x{fz z4?geT%D{rYugdGA zp(dVME56rzG3)gN898$rwsoR*8)c&&&e9WvC?dR3AOF5cu2w5o8K+8%`I)&LN4gbh zciANZQ<(qgu2?LMrHhOk#$n{R1aU!k)P!OGri%)5{F``56V3cGoR~dV{wl3fk4gI? zoouDju~~vXqt8B-I*By*3tVFL1vp~qtUoGeOB+aCy~ugRfrz4gdd6)&M0QGI7W6iW ztgEj4MN>P@67P+v3f>Z6P_{=K$pA|FrA4UtRDjq%dEq=4bxX}PRag!-`W4Y)?$a-9 z1k)9#%OCuKs;A1tBcjB^ovWxna4_t>G)=%y(}fGUjV5R$qBGXO#_V5JgK`uP2wfJ=ry~W z=#jLK>tN#ba+UmwRYOBES#mE!*e_`$PWXi?0gFpo&KxpG|*&nM!kZV0@qACu@6<@f<4>Uc!sT-;^s8=WVNyZQ8n@nH^L! z&#{&yy+x`1kt#-CS~RsvBNY2J>o2=wp^g~I?%;@3N5l6iv%FjRnvr9_edEkUFQ$pVNQgd@KzD~3Ql1>6m*92OKy5IR~)k5vO$nVCc;{K9=ZW5btZHlHyyWq)Uq7%OaA*>EAu33fG0lt-D}uM9M`BmaAPk zDKsuqpIj5?^cNtD`*J+n)Mm!ddwVkdmJ$~HtW2o#w ziSi$&XE?-JkN7gtwNninsqMM+0)ZrUqM2}begQ=G0w?AZipAVZbRg0)s=Df9N=f2^ z00U9W)~Zt(^gr*RylPOhU;5b+tHXXvvDGPZ)p zYr_dKc>MS}oCO4HNb5)4FP7-1|8L#u`FAhv=_v@0HZf0gcm_{s0g;J z{*g)Sa!sZ6eXBxxAC608;e5boeB%(8oF=vxN24}|ha`6RYnUfMzohqlt$Hpe@f7l# zGU{5j*U*c&52lr*D0h9cK1tY+V1~ zpVxNA1S~d|d@6(f=2^}7{ED@ZeJqw4=#9?F4N{>1D$Z6dH>jVj)KsfKTKr&sYH3oP zcOYJ>KEM?)32NlxfKJJj^QK*yI(_i6IMyP6gXss?8wlr?tzxl965sW*qC3M)##BmD$4D$sVVP`JM?865dz@vP>i|OT`)hw_SS&v8!9T z0M>3la>hc;4>UeP@syK}B3co@N*O#_elZUM@vzIL*RJ<;MdydKk&DU$*Msc{W#*FB zLHtkm2({;p(+BBG(`gc8JI|&)d~4lZtF(TqmJ`t#fL0GIhGQd{X!hCG6p@WRuebLM zudF~@o(n*Nm-57ksBT+o4Te?6xhlcEXY@5eGy@*LJgCxpZRs!HMqJ<T*BB6bgl9KV*@F{! zsOp&aVSsUS-?xc4l}9!>|DE~$2l0des}29720+wDzIJMQEU|YOtbCCgWOV@i5>JKI zY6YGZngZG7Tae<-%o1gXlUC8NihdzzPr3wx1oRIH?p5yY4}CWQZ4D3kUiZgfVQZu>Y~rsyZ?<#WpBa|B}v?<9LH)u@ZoZO{2w%gQ)^sqi#J zJ!z%Tj9Kxc*3KzERF>SlCM?;VF0S0EA_k0HH=%W#o$o47t;gz6&{nf;%#Fw!b{ft@ z(UktV^Vpv9L>h13Ec8AwE@~#e>ZeJ-*Mn&C@>PZN2SyHB-Hm2eYzE*eELRV*tfg3$ zuQVOPI&Qyy{AnZ*DZj`yU=q8T>6;PG(u7*k8VM$D4xNRel$p$PmwZ~;t10*r^olonJg2ZKEfRM37i9TF;JHn|m_+xSD z?!3Zx9oXrvftVclBCXHheCq#pSaihAFxzKAMp;D2RRHWk9IKT!-%(=z9Fq5pD2iUp zLQ2Rg$ChD>TAu+YGT`jDN$ug(R;)EFnt*Blydi{EH8Fp~-X=Q0IlH_$SKcMDE^vS_ z{6n>zZY)E%rNn`KZuqpBl@E44Lov234iYngEcKKonRFwSvIWjrkP%voEg?@yJec2> zHa|O~q4Y(Nyk4?TRJY<5$XN+O$bd?^mBlpz`kyqpDrkAb^%aa{r?JR?W%-9QQvbVxi%VLliJro z63faRUlUx|ns*T(^*D1@l-91XYg+fQx}VA&z|^Y`cP@WNW75J4z_G*$F`z4*J;CHU zOlMrZbgsjZ%|CbfXj)ZOuDuk!fkY8naO9x_*g~MRY|3ihJS~A8eqaE*A!^3uf;p2r zE*!InmM-~5A-3vxfe?Y58mj72Z+I zK*_GfX0~_cB#Aear=-`uqBayAszp`>kaq{_0rZ#z7p+%OC2N1y!?XMlAj|~11}q7T zKmOo@6_&uDZeOnCzBSjC(}toDwI0iwes@ru*PV$n3bG-HN}$CK$tiSmNdc|Y9>(Ge zg*7;<4A1SpX$RTAt`D~f`p5Il~;5oMzxXu(TQ(L*%@x;m*DejGn#;a>2J zk`=*}D@O^N- z?@Z>Q)@|Qr$??5SBx{ zn3cuqc{fU>5ZysJVck^O0xDv9>Q8ptMM*ae{(pQVtYt+c3+)&Oc>ie=QV+*Lbu+mW zQ}3ggWMnSc{k*nx=A7>F9AM=mmZ6R00A)Y1hE>YpMr`V_@l=_Q3#;W1)s4So>#lBB zOLi}ew+{{kDe`*458|vfF7tGHKwjYMp1Fttq&@@MDqX$)MW?eel*GkV)k*5br#LtNnWk^@o6Aawq&RTn`J^`R~^Fk)}4Ib0vE(cK30Pe{rZHh(Dr4%jAm0MtLJj6x4zZBh3OLQ0L7!5k4e`@72 zhy4ov=D9C9rIZ^H{vozB+`s#n27L^+WkOKG&1f`CG%NJ7|Imd5iTD2MxtKPtE}1Jy zH#?NSEYgK5`AJqQ< z$4)J7N*I&aELdvTO^)$CAD%G<+#8TV8sVrKPV95^&=qiexfie;?aLS+7iU-F(}4XZ zH<1)W5XK@=eNtP?D?obmiCGQcaa2rUEL#UMrQ;f4FeN$o3`M4p%>z4d42AcdDJps- z(J(X49yzN|e|H`h{-M))>G858szl1zU6Yqg?UB`MlIfx;%LZKqfFW8yTy(9*kJ86Oe9B^z zNqMX|KMKbWY>#$+5DG-NxyWkDXeH-8*Lq$sA@ft2J-rW5CAU8=zfD|$WY&Lor=uta zAZ*C%;52h(PG4u_l?u!BbkP7UkR$1vU3kT$C$9(jEETJosH|3`2|Q@$^@{LBEDo>4 zIBlJfW>?ezeGx#ASk^tB$rM%l8qhrre7^TbxIb;N0!#`|!@7_jof9y16lb_&sOLmX zX2_8gQpcRvM1VF13pH(MYBINpp%U~*dxU&n(H^#~WkmxgcFfy5LJ`1Mke)1M)PJ1?8KeC@7 ze_9Y(v#16D^+-tHmY-J^K1$1BrgapETx-%PW()xljQIsEfJ|~rO2UUzdEq1$_vhJSahA;@Pl#9GIG%?0GIc2yLybfv&$#AXg&h#x3#XW`366h z*5aRFUk$!1YRH*ZaIwmFuSX+E4+e%Z z?VktEF#t9%pY;WwN0Q*f{MqJoHIHj*o0r)r;jv&7CKX!;DEJ(mqLD^##ez2ty4!%f zU{q{n0_@gNBLL9 zWTj#DoUO^nqW+mNQ<@3Y2-yIp#)Kq{Yi%p;-k_OCXsh03hZxd&`n2P%+sFH%9y zUCLj+Ho!#?-us$i1HD+hnF_4;SsgUTgZ6LCIlE+h@#DnjxsV*&{nV+0`q!vHUiHxw z`pK#mNCS{`{g7%O>#2&TfHX-LnE>*tBc zY@|fqd%TLG#k`bx(rTsc!d<_e;|>rjD)rA+I7YQwR~aK4BELe>U1Yk}odHtCkiZnQ z_Uk36ZSzp~bD)&xBO22k9*YRl=+BFyjbnN(Q_T_qzT3^Yf~n*0YU_slWSuA34!$K& zMy%lO|4~M^OsDr%v|bR0<=D>{7B7Ffo@U*s(a7liX-<+Gn}~In#=ll>us&wz?4Y(* z)OP6fr-6xKah#n`^Evak1SN2DCq@Xuiv1Tf02_8hRXY0dQwZWr*GF5u%? zIq1o-{tGx51u8tcH1)?MECWCOwYG9rZC($QPnR@$1Iz}C0VPi$>}OUUt^Jk0%QUO& z8bmC%Mwb1w2qqwauqg|D2qjq2T;)!rHPgM}3eo$T`^WBQg{_B{wsJ-tRO1nEp!TaY zFtiMw!__J6{X*#=TCHqu+28*OuaVuS_qDs#CcdkJiQ|?kOWTzZ$lNC9v7UM7>-$Ku zP|G_CdLMoeuJ#2iD_nulOw@NGWtAI6#J16MhqpUsq0VxKzx2&OjhO4_7Ttl|Oczu7 ze7RpGeyDtxr-Gl#?RJuvt9E==)E*)y+Img?uSgL1Oet9P2(va7{JHv8%D_SDki%F` zz=6I?FvWL=M3Q}x&v$!rBf$NMa*E#R>X=8k>n474cd{7$yts?9zvvbM>Q$<^!9{Q+ z>^DoaALJ9Cbx`C!$u4F)ok~S#_Jno79Q0Jxq*nXOoRA=Sdse1fZI1mJv)~HWnzSfM z!c?W8nt5&Cs%P=<*s9bofZntID5&AP$q3Bt>lZ!1Hrja^tiz26uwY-9!0Mc_O_AsL z))8sfuMtJ6H?{0Jhp%wkqX&r%J}pI`nSvhl4sH5-30AfB5#h}VnCoy{D0=+c4yiv5 zZ#w$>KBEW6(ta{6phMDurm{eZ1lb_x(LE~4=Lh^<#!VSu8{rBM76nAF@x$Gu zf_?WbHP7PzfZ^%pt39DQZZMa-o9T8laQxY|h&}y(9Gyoi5G!?Mzg~BIrIa4jtNbbz zYPlsYZ+<#SJBvMB8Y&f|q;$wZMk+Hl8zN^LNjPQXeu?+=2`oOD^~aWF^cJj0tHs_< zL(q@>Qkzjtb(~{)0@mV>V6*ZXsulVEbLj*}hO%*tnugi%YsZota|u!U=HZWZN)qp2 zo4+k$52hW!Uhp5p8xgRAjuifmuTbhGXhZOj#er-%rq8lABD84)Kg>hXfibSzvgYNH zCkbLpv^%4$IhS0y(*I%Vt)rssqxEk^R6;;PIwV8{q&pP^>5%S{4(TpIx&%}}y1P?S zTBM~xTImjH`0aVlIq!OBE&jp5VKH;>dw=(JU7wwyT@t?j6*M)whNM#xuUgF@d)m1B z$g`3d5j;DF0q#1Fy0l9D_kVvf*>cTPQj5y7sY?sX4xQu1ShHJ#%DGKB@CfBpK$$+H z^)tPy<=ex%q?r5&7g4)n^D3c;H$}cod(ex_FX!l%E+6J)GzIc0woM)u>)cl?dIJhy zkt&2wXz+&azVkO~VLf}9NXm*MpHJ3TVa3C2JulV!0okleA7p{~&2%P7W%{K}IV+g3 zNmPW-Z&J=Y4CNfw6qaj$QXG-5jQ&N8!UXwj*C}@&7TU*i$!_0iUX<4}akc%FSbEi9kPOpiL-EyIb>36UNb?&kMiC`O zEzpfkob@nKiiocTFYe1_OmK_&Tum(s4=wY5g{Fm(3paZ3F%AdeQTE-NcgH4H{4`9{t0+| z(6k(SUXsh1uM_CvUc{y+x>|2>@%;{Z$k5dB>SiCAtm$p%?HU@2bXqm=Rz725UcY(v zn2_}6Ev*`z%e~PS+it1;!kKQRPRD~te+%tw#R_-S?o?|1-rglQ* zR+la!XGRQJv$H!Ym?;*SA64VoW(H#6I4@hZHZWa{ZW*5VHoHm93v0hGQ)Pqmh-pD3 zYHTgV%;1z$GV0TpjScUoq=v?At#-P+{?FAsyk{2|8A&ha`77Ovz9L_9A z^=K5ARes>N{VxJ1p-#kJedUh>*_OAY;RJq&MBIxy(IK*dTT%10_CM^n___+0bq^r8~t}iGmsxX&7Ym8R&;yh`_Vz*>4_)N-2_BUQI_A_Odh&&qrDet3Vt*RDOo)s zmrE5C{fD2T%v*f#_*ZJcy`*BvxB0gbIy=OO8ozpd&AWI!9)6kdzU|(v>Dl|Ng{v)u zOz)P)(j#txaR9M2AO_JFQCzLP_T0AATL!P{m6|sG|FlN2KDfe~kyUr5L_rF8lMq?+ai_Uz#_F`GApY~uKM!nVe64h^ z(sI=43vK4{)2+-nrqN zSq~Zs6Rel&iU=%=Q0aX!RcKC(UdhWIF{zEtEx~X}-gG8D&-D2?ubu+gY_|Q~+7Zd2 z-7zuKZgR5e&w+Ou0s1mA$wYb-+n2XwSFR0w8QeWio#7D2FZDD;B~9Zeu*KsngZQ-$ z*t3HJPp0RKWxc_UdfG@1yqEWZ?Ai|NwdC?&J~ufBqx-#JS;*AJhIavcO8&%Zy7X#h zSobA!9F82tiAW{4jF(=;yVCp5uf9~ovkzFgg0T_`{>zA(lChqZumWEzHQ`YR!Tp+o zOW!QILS3{-dC$EGp~P)5gmx}@Oq(}4Z)@hrCU)r?P}g3sV3%B%unM`Yf5#vt5{&Ba z37Qmk#o6BJIk%L)XR`o)%2(PKm5vLhj&70q(1ctaW9&EVPtWkuaUjetcItob_u6fA$tu1TlnetzpLZe#!@=3=D zmhYjhdIptsyfpUr$hCTRGujSm==pi*9ErKY=tG&nKF(wrOg%jy_Pq_J7?1T0+*fm9 zOdrA;cH!ffLDxrQiWkF=6&-KavCgPdfAsu!t4|-Ig#tpsGdwhzE#8>FSjy`lxBUwu z`-F~GNRa2(Pn7Do^pErI@@FiSsT&lAVe(_6C}?OVlactj#uB`m{X0)o_k-@{45-Yr zZ_Uvv{-CS(y@VDip^pBQKe4HGrRY3E47n4Ag|btf=t3~=S6u9lnCbVPeWd*UrBX8d zvVwt(`Q@X>#7R?Yg5LX6d_C9+>{{-@L3*)MXNJj!a*Ib%{Kq>hJ>8QMQS_wjxo1MF zJu|Pm$jrdLIePM`_gb}cJ#YZE-&RhH5SQTv5{HeTD@Sv{ePa1S83)ZhM1Ti&R|b^dQNH6LtQ z@+}aQL;^*;`O)+L?+h2xH9Zf?I}%|>R;c<|atQI5MTAv)@T|?q?2nMDX-aK+mAvX# zw1qnpO|SVO*vN`l!LH@98x8`aV=VS|en5xJ~kHH34ad3nVb)()@Q z>Q8M44o~lQmCM|E%4BsOXlpSvn|?0&jBn^vh;?cU@|aXD0dHc(MY1lFGGI2j@O74* z>6-$*QvUeEhaa=bk&dB_@uR1)l-nIYlz|o`=GFJz&UFRhB&&#$fj`QWz{pwi8o5NfD&WpJ z&4+ObCj~KwZH+gb=Qzp3EN6xZ?oD#}MRKHO5iEl+c^@Mtv+lP{g^H=k2=?qZ38cfe zA7$jOyv@lQrql$#DnAv5=RkbL0_YEkknR)Jv*XgsQY|dppMFlZ1egafJxwPhsd-cZ zCdf#8H0?NSb$!EZq~UvzcxSIGACfwNek;{2&LB|u| zEfS3piwyOS?+>{j%yW`C;qm}n*`s9N2fKIEyFKuxHf~XO5+v|nOL<-N_uPkoO2V`q zD&?x{Eu6n(FeIm3BYfRaO@Vtkq5EAK53 zeONTT1MY}cx^Xrn;gVpF!avH5a)#*c;!SJcM=E=^uSQjJh05naxXUj`p_QLvXt}|< zg^z`Mr~6I;OOu{Ht*!@&;9RB#Sh2WTd$6r3z_5_}4w1c+kQ#d!TO4sQ8auFbR-Nc} z_BU>Rp!hdKg3LW%qsWw2_9#`}Gwu7`DH)+Tpw?5N*mgZlsv6+_ePBh_Aia=gG&;YU zP}dh3fwLapN0e%Nu(ez;ROEygW~@RpcJ7NS%2sL}9nUowIrFs4DE1Z8GhB*~PUQp( zAzETvayQogeWj#R*57Ocux1GWy%DB6@jj)>PSL}T~m&#eDD`TnGuyKg2E;WnO;mQM7Z1oduN z)vL|O95(gm_|wFKx!)Rz^BE;^5K-5@0;GBMAiMlU@1pUIBgZO5x-Z8<%j2#ap5?FL zYI#mN)Swr;P73&{L-!-I#dPuccZIe&-zc7Fz%U*BAIrD`DqBjOI@NJf>#Am1@MB_RI45)? zq8uIN)Js=9c0G`Ka=fCLl}W?l++9bmpn6(Y?Dg$6n|GExO%aNJpwWXOOg2Bwm5lLH zY4a)O{j0!cbKk|`4?C~MNvq3Uf9}bc4T?cDCRwFai-VM-gL^gXi3*(y@3T+zQ0I8o zs8jle@G&>o_ku{|?85XZ69!-?eH4QAwXCTYb358K6e?!BU2*-pPKyZ7_C@P*6n7V9 zj@f#!nZIt=DU2w8PC6>4RD4nIg!CbhmIY$*^oEZb`g2@Fqd55DtG(h#SQp4^(W4qV z*V(^zY}y%=tN$R7@;%qKjGPyQq)ydq5%*LFf&8v24`N(oNC{mEF-x^SCr3T+uCG%f zx+1q6I3HKv0H|QimsD(EPsB)2_TtFp#=1AzbSeGk0f*|;s=p^lqNtOvY|BS>x$UE7 zKN5AF#XSG8CDFI?z5?Bt(!G}uchy7T^N0r#|B)!SQLStI+}X3MWB9Pq{N#@xcY6jS z?yQbkMB^3UCA`qF(_5~k4K+le=m~^z-8e6*b_7&weZH!jGbl|nKVF)q@C9R1PRIPD zuq{rfdkoCMJ0MV3hUV;L#Y}r6H_7{6cb@f@07VOlM*e#FXwOf%8dH4s!V?AhT^27MSzgF}eTU!89%e+n}@dS`g~&K27^wRn$6UM#}=>sFb?X-mRds+^BEh7v!M z0i06mieCQ18pxXQ_|QBHK@2Je9I*4G8u%o}Vl9;i@UDK6uW&!!@ZxJt(=DysNLXDv zl3y;8PDY*{JxP)OO>bEKT_cV-AT~W2N3>M3&kUtPx2uPxC{%li0%C2&aY54IF zQ@s|=w2{H(Th41fUdGD5Q}vWFnqTxg=L;uES&So-Er`Y$^Xyi#4F~dDRRnt81 z=NTqwT-rb;>Ev!S-K7Ph;w-se&&7x9YwxZoik68Fg(I|{L2)DK50-YocG+c&QO-~s zs>T-fPyMDrqT&b4oNz69LQcT%VV6u{3Wl?qoY5cN4W0ewxXry_)KA3mVc()&STni) zVEHf`=}a>m@rfx{m!OSBBu)_d;S~f|H-JEQfj#xCuetgE!YVgiNx<4F(yHBwQ0w?F znJF`;$T2)=QmI6T*BzYDzWK(Ap4MD68kB^48#lvazuQy` z_q#o+RjHF19`~~845N%`CTpkX@xS3Lf8%gdoyQohUUJlrM#%D3J)2 zS<&(Sp~D0j8L;{zpW9IZx;^I4#V=b$Z1E_h?AOX z&r{Z~63fSRlAK8$;LggeJX_TNh5)xgEchtWJZeU=S)pxLMQ3&>xZ8%OGw3xgjJB^d zFEcT>!K1ZNQ&grKIZNyO0e?@>p>(Tnmyec?hY!Y>9hCpDAv;8WtGxFhm(Y z7qW)t6TA8DnEehEp-+zFmYbY-!7+9!(8Zb;y?gtWV#iAh2+>VgS72sf-`AEe~NdRRCbo_`ID|?W}md5 zS((;WI!db_&1K&^gDU|1n4i$lz^(~!fE166t{w{e1xN0u`a*Vz5?ZwZcCkfOPN)Kh zW}hpW91c3HGng>%SgF6s{Ad$832C~;1GMme(t)~$5paG?ur2#MC)s}z6t+`>WUCow z*0dnUSs~Zpt(8(Za#JBw)!mkrS?swcO3`Lh%g(}5H{o%toAxF@jqc}vhqv4Q{{Ava zJo`?T(JQibU%}V!#@qBlkE?Pu>{ZeQtG4HR=zJe+4#N9Eswxa`K^d1b!!0oN%O&oS z-9sNd{Qcu*W7FdJ4z$I@4!?2hg}V_0bt$)Vw83Z@T8(OpM-)Jp_yz%kGsr#N45rkB zWe(w*yJ0mMOzDSY=3&k_=l^s++^sva4JSR`Th7NLbYWwG?IxGl0))~FRWleFV|!*e zF9;$=C5$=0)T#eWgOOfGA3E0_yeu)p}L%xam&^AAWu zG#cY^g-BWT^CIQQd2NRB?Zf$@rs4(`|M?h~4&7h8Pt$w!g27OQAg}tk?WHhaO2n~V zi`4tpaF5nmj-5>p<6G1Ph5e`ykvC1G@8I3Scv(ioDIoBbbh&2i3j}Ea(7N$+$ISAT zAAvdg{bQ!JR&}JP8DpNorp_l=QMBv9fs(sojxt#lpsf)P{1PcQ3@-jc(q>*5s~o2O zuTG_0IgE##G-TwFbxM?TOR`+hPF}Hcfe|fPR;3%GDmv^J1`g(HcDT31kVh1SocX1R zTg_~9%~mJ(7e=P1&G#xWz65HDGWPnm#Z3^|wB~LNoHLix?!MHg%W+25V#N2SE9fdL zwCnpCJV$I@EG8I5%jQ@WkT-4-xQb3}x8r90=GF9GxcTftG3zAtJ|2G93y$A-$pEAR z0;>WebuXJuE^b5dy5ancTRkbJeE<&BTFqjUc$8>JAS!rtSCbp}%a-s4E=yLwn>MJ8 zwcz=ul^Ulk-ZWS|IJqE^oGOf}1;z!TAS32&H;`0W{4W^%aBm9-P z^)27ehxri)H|M{q*bLSqv;7&LOr;MfSB?tETA}@cn8Zp}ipl7|5r1y5Z-0jTdUX5D zK&z4E$Z{=Mkw$g8E2zpL4JQ>At8R4s<2f>^;1-1dYVGrmfvu50h14CrrM8AAe;Hx4 z1jn*+lp1lL%Add9r!KoV{#q6cwb_k6KY*U>%c$si`E6PX$6JP8^SlvWLaEbQf?o7# z&bJ60&{n;@tqQsjA(ZR=m5a3QcaQdi#^N_LAL)9f5A2(AJ{4>(tB})*d$Ido)$1bV z>NzP_yLeO1KPrB0TktJe&u;8)Vn|zjIzVF|DBBca|DN-Rv+?%(uXziO=vzK@^@D2M z(s&MNFiq;?)q7+Hd0ykLb!VjL9Q{jZmSB)gHKt1o$04d&wkH?>qbl68=NL#vkA_diUs0h0?1kc`9TAjA58@(!VSALX&m1xkuqEIUx z^;5lQ&2GUo#f|oxRbw2Cd*4z_Io+60|RTOabMpvZq*KmS%V%Q$5cFT3I9GsZBomvn0Ik;>!j zxjp;L0N7x8Q3a{2%6dXqfH@%;5J1-h+X3%=GJC>9xx~S`IUDGG0Jz-cR2@6868$LQ zpRELisQl|sG9|v9hOL{<>^N%ODy0)J5XqdheYf)r6ElhyfaBh&K|pWjf67Uz>Cg?| zLOpAGTX7c;5B{Ic? zB>VP(5i+685tZcxdm?uiKUO5oS}2#!JXd119eMvnF|UORcUjrQq@|eRo||9+3HKIv zxB*SHp|)BBu^!o8ECb#?SJM1feT{;=wx$(l%+toVhhktiY7+H$ccS1#%MV2N2#-?T zhQ(nT{4kc4vwLe4p+t!+PP=rB=X6BiTlU;wlJy$1#>?>7U<4WS=>5Luf;DejFheS>MEnk~2 ziB}?Txb_biAoZdT2ZE!C!Z*>Y%P#Ei8mGq^AQA_zv6E9;e^jM9-{^O`QXF9tVn3*ox+T z`9#0@`ljt=x|^%j#^4d0L;swX3?7Iq-R2XxKe8BTs`62PoCq;{l_hpj$nlr#Z)f4*^7X?`Yv-fCs5pUe}z&RRs zpvK7B^XL938CBDL&3u(d8G=ZKO_st z9XUo=5WQiY8VW1g2fM|eGDR6(XP?jdqat+lL!63a?O#kc)vffS4O-fZirl0gWuOxU zunQUd-fg|#m$msZ`u|rr0XzDD%hDAoaz|X?=7hI-(*c=u_7?c3o=cyX=Bi7xG##?t z>N<;j@`rQ~-uiOGwFw|r7<;vJyLbf>;GDp_QRe<9%F!cgsqDv8AJK^uSI4di9Nr;9 zLo)f?1}wVNz;%J+J484$9o;b5!}ooMBqvF+*tfIV5uF|HZsXzF&W?9b2?C0UXzp6j z0r?9K1_a;}E|QzY`u`ot-@82R7S1WQ-esDs^#?@pwsWjp?rjzI8FoLeIOaF`s@)yq z5X}R6(Mzk|6=27U@N7Lz9jr8qo-PIM@Yz_YclNt^z@{1mkqy9+_B-!tO3;`Icb(4+ zH6?up5b1x-4tIT*AiX{22BRV>X|{yRNr`Q62h5tZn^ea1KYD zg5oINUhn(Qy#r^1lXxP3$AKor!tu`Qqu>qJHurYPEl0x4Gy4Fl7@A>x%uOVd!eAku zn4#o+-o6@8{5Ne;p0{LQE+D=|0qx)zx%ttS z`;R~9ul&5Lm%p)=xa&|Q_l(Ru8tdGUX?{V22Qi%dwk=)Ku7oZx%yTawgbP~m4X@?k zcW%V*0FDW%u+w5P6<@4a+*0dVmPt( zPmGljx3GX+SMj?=`NE0&4+81Z%@2bF(!qupVPY)d(lfclS31hczau1ZdmO*Kklu2G z)3X}m@C;G5s#@l^>Kz+}<@%G$IpXjqWi!Qrc~CYxO0HX3W1q~?Be1YQ`8942#6ZaV zMS3kMsJOH&P8-VpAGb~}=3Py}4tlxc(xUX#$gSy|BC5N6Z;u6$CGCzR>o8=6EGZPr zmiegpT{WK-PGo)AhvOZlse?g-tJDusz!3(3RUYD&lb?UtsHv&nR%-C^^WUuR6NHI1 z>N3ma9ttuLM*+B>eSqhVFXS!k%}yD)I(Ln)KkRO7hJUKrlXAbED|>1qO4=L*t}m7? z)2W4s^9s&#EuvlF$oFlBa5<3@Cf)g7*?!Z`Ghc2MKHr_SkniaB)vE=ru+@)yMD=wK z3aH({G1~ai)t5*oHAh{(f_LBdWuNXZAHnV#y$}6Mzi!B!<@Dhx=Pg+8E*|&%wEVPT z4yE^_^?GhzuL^RJqdR6MoW;}l{ID=n2VNKm-oIzM*)8$OWhhnewGMWI@FKC)F%3+6 zj2nH3vo%8@69*B)M9{`C6lWOj`N)=@5Z*)E?A9)Y_NgrO2Dz;yq~DTf&jFK2*xzq{ z?dyf<+Sb-vi09rCxKNM49V|DRT2`Bdk-GhYA{SM10ZRRzT(sotnaOIE2uoet++c1x z+dbuD>W@;A23Ik}rh4PS2HrJt_Ua~cK6y7lnTiHsF%J)tgSxXOfRve?p%E<5$o!-> zc`1{OIJr(f&v@CWb8Lq3V)wcC24-}Rl2u^*8swo7vBcgDK2x8cJxVUYr0g?tCd#zD zMJy*GPBRu_+mBQDCL((o_LeH0?9BtT-z-*zb&0=ude2x_Ly!BFt8h!B_+v%|CVEmn zok!glwbI}aQ!FmH8Y)aJGN>%fI8PBdLV#`(_!1`9hDhe?Tu`f*J$F^8XlaYXBO|FK zV<=Pj*47N>PIn>E-VHi(G8kxXA&kX{CLgrvb*Fnm9{h;mKZrpfVc!!>M_Pc}BM3Ss zWqSbXEw@?_UkSh9>#n)E&+ovXfw+qTMka6b1!&KRpR0JETFHKK6#pSuBX*u<)8YjGpQ=UxADhs^JGFYhZb z#laR0WlJZ2eeIIqKVimvnueB~@7dy*Gw4E@%mAV!^+?f$&Qj!P4zZwKuQ?om^hrJU zcLr-)&YXbNl^<558YULCr_vAt{sP!-W5-KoA#*_PX9r#_CP|AU`+OPsFkfyw>0j8W z2H40I5j^XL`F^nMgY*CriWnucf?yxWL$VVs8mUj7X?wX5^LsMa>LIyN2uF#yumgi} zxfE=gp(Zxrn6hng*LMF?xpLn3R~nA2FCUL4RTbOye0BZG@JT`8!9=zyVDa*!!vQ7E z)K`MmE;(x;mV-k@x%)eO)r9pj^E&_A@TSI?mVDfN)7O?;>!7?|xILDEI%B=Go$!szjE5X8EaK-x%<$x1Ki!kY+}Pn@LbE`8$mA>dF?)t_0?SHlV9Lc@m_&IVTmB^ zM|I>@mPHY~LW!6U-3B+g{1LEcSwUz&M1HSV?>YuZmR}9ae|VkDb~m~mPF7nI%%2Uo ztgWpr{BL$*8D`9H9UYHS`Euaiiy(;u^D;_-t^)zr7Ciag4)7u(GP1t_CtfKiELmF5 zO{IeFHp?;6 zOa=#kda_2$YpkY3*bXFucy{Ykw$Gw(J|Jk## zJ~E%r*P5EY`(aAQTTDsCOwt$)HL}&f{JK&5*Nsx7bqh*RXM68@09_~)$}EM5b+UE{ zHFjI04@2pzJ)y%yNhx~Q0@obvY3?SHe~tGLZ=}wd3MtXmLmSJFW7IxuC|!ktH;Ezd zmyahmAz9Q8IUx{9ZWX@1?%=smhKsVnVUVrz!v6fglJJ>zdc)$iAfr9ktxfUWv>6X% z7;7Pp>gTL9*1}2cJb-45TjZJ|NJ0tVU*NZ^p!&P0o#Y6(4zUn#0T>0y}&n zV($IO#&X&o@VO8w%+L>w@Li#gBI$doHuJ)-6!8!)f5148ey*9*$N%FT^aP_0`yf?hg3%Jeo@OMiHl3Kz4+{(H3Sdkf$L!otOUGRALe zf;HT9Wll~y1mQ2YO;Q9pr6yjX&y0^tM*YNNL+ef*3y&YqQ^iviR$)g0KQvu*$g__7 z#t<6V;f{NA_gy)Xsdrn?*IWyA62J15@8eNExj$|PY4NTN?F#^C=H1G869ep(LL?#a z_FX;0%^v7u7bS5mHb9=0^9X+sddVWch$mOVoJGR#E>$KOZSaq8Cd)kG?}@NB*VTm> z@iD!HR`bClZu=JtG?AO9$Ry&sZ%zNLf4S0oKR7plkUtojkn@^)VSC0{p<3+V>M`mY z=okKbyn7VZDS~h`m!4pTBfKDgym~Ir4$oS#yuYulCxs1B^(8c z$B$N}5oo`WMMyP1M)(BXWU}f4f30OYHzcVuS}>nNTFdr3H1?v3cvsCUTC{4DMfgp- zS`AUtmD#f6lk9%=&+&%3c?G_{y4`-_)x@-~dGnY&3LL+e(tHt70tXTZ0HF8Pre3A} zUn%Gw!Hi8Xk}JOoIB@-6S(2nam;0V44fDol2Y)XvPl4{nP40baz|^=WGCebcK*_{& zSrY?hPp{`K8z2l~5G*f*-ev+BI~b~C$64;SV4ZXU!{H~z&FO2VZ+~#)c>m&YSxf3VhS283_1)I-B>xidwN#&=s z?XLX+1>|@;4r}hi1i3LJH@!4`yq^=<`2zLJ`x+NGd94+#dW1=4iLpN2M(CT73)SUX zkicr9aFyOr+`=b_6?M=cxnM^S7BU+Bw-Y)^=Z}d%4eBbIE4@`FScxoI@+(!ZH_1+q zT{{&n_$M1FeXci-p8uSi^v=#s1=a69UA`Kb{Nf$s_FUv*BU%{8KGM|1oAZ!X+j?@| z^4sSuhK7WxD>1JrXt_o<`2&HKg}A}(cTjGDQ zKX)N(>mtqS&}%iCl*X&3V|Jj>05+~R$lyTer33Mdl}BoaCqUyB^EwfBU#}@iLA|z_ zQ{L&HJ%W0MNxSO!M_1p=8fQFco}PFIz^!p9{vY~Of>$7CxiGHXb~j8y%h?rDlxRi@hLQ_;|x{bq^aRdiibvbTjC3i zsGcoDCPr>HK7}Buyg_YYiF|5zzpX$B^t3}bFN^tVEFG&HMYv1ziKCO5zYr%kZ}8r| z)RlU~DKc$S8S@+uL(W{`uArH?JOC~7`zDdFp2YU7 zmEHhmMu4Nl{T@u=8luZhe(lIxE_E zYR^lrR2l&FJVp2dD>vEgF$R=VnYTm-ag?H{SN&NQJM~<^Pu>4*R}%_mK}cE9bqS>ZE; z*y4wvlPl2S)vcy@APUA(7&9$S2%>{+hMfmHl@4yg!CfG7ZcUaaL)QtdTuHNl%2em{ zWHB_RPi_rj-n?hp#XfS4`ZeJOTxoQCU_{fZxJAT`$lDp75KKO7k)F3nD|_Gk+u|~k z$--X4<+kO;Cj>AqeR2=luTcv^`9lX)7vEy*Eq>v|l(_s<}e4v^~=Wm%n7kSPD zDeK5hg_DsuJ<*KDWFvki6NnkbPhF*ge2$p5?VP?lJX&COy3MES9dLcAX<;8AN@Z&J z&Q5`ux&w`AOHGruBLrABL7YC^DrLVz6Cs0o|VbWv(2_djLjw;SbeBaAS8F5W|yJ)thC-o z9`v0dxrDnep?yO0t%2E`E1n&#Mzz7n)5w^KHtl~^I)CTh=o%m3RY$q%2X#O(BlY<* z?SDCqAEGQnS);{ubSmT&w4p4t`6g4pabuYNg3QGDZEbjeAh|}9^Gl3|JMNfe74zw; z;Hp`)MVCn!^PFcb(=nV-MU8u{4|#X-i?_(=q`V_txV0hmxP%J{L4}i6NF#EUhtpv= z4K?HY$jJ34l^Z&}#R6^(Kgx5b_wS84eA(FBTIPI9)PznnnzwG7`@KW7cx#%966K>j z1F2(pdFffeXD2kG09@fT8^E-Gv9>bT?_g!q{>9ZFd)`~B zQNP<@#ABqs3b8$0XE~O-U{go0uaHYcENSV~6w&jRIDG*8GYea^J9S03SLEvg!SlJk zM=CSKDkuLr9t11;V@5<`wUok19^uD%HoBCxwYh))vT?5sJ;`k);if>^;D6HZ2R^qN zX0N*#5Gh!A7$Hcvh*KRbtAy=%13b&&+*hK<{SVOsjs||I-ZMgNBzKXHGLL|*n>ufn zW7ww}cREOkWh1exAmjhYg0q_2`gmiNkbH|emLViXPmX@?8vuO|KdS}w=v!rL2TIXLP zqD72fFboS?UJkAK|*OB7_EOPc1M?XT-qkvuwi? z3A)-3_d!uz$EPi9g>ji2GOn{U7?-UgNHmTe=`3~GSt8l^Sfj5N{>pN!H z;ZIap{<;00EJ9kejenD0HJIGh8^^o~Kn2hC|O6WZ0Q8I$I4}2>u#Kwi2{GHuw>9~XHQm+WmOAbTiJ^f%z8^JyI z1IX_;hr&1Q_k~YifWPXi}$v&UJ zD~tJ@j0nZUW{-Ay9@i&L4~aAgxNg7sJVE%QbhU0k{!P9q!uWjHrPo=}O5#-RSXK33 zVFh_dT$*`P=gY0l*SnAP!t5-2BdFGYD2`v%U0OdLuUfFnRff-Tb8vk*rV$Sw~6x?3%o`jd)U5G18Y1fo| zH#M*qU%zpe$BQ&^$>+%i-$#O2EY^mT!U&<2W$3M6fkFHBYi>}g`gW}OVnN$Sek&fT z6+e7zzV&4s)esG)2tOcbt`B(I2Vd{(g2=UI>E!H;Gq_EwkaWPcv!Wgt6h!3n!uLf7 zVQb@2X#t7L@&U0{YN6e~+@vDcvY(%(xV~H!|D=ShzPf%(J6IA11r@3BRFWIZy%UND z$9UHW$MO@&KLbHq?pqomX)~Zj49g84+TE@xE~G|5YEJvU$0Qk5?(OD9R$jDlfCe&u z7dpcxLf4mjC#g;pJ=ItsH+a4Afjd$3F>gk_M;zkil&m2#>o9weG_56YB68o0S=wq| zJ=ap$si9q-a5GXH(rWif4YEC2Ry^XN4YuPk_-WO^l*0V26bws6bQSb?cj(ZnoU|V7 zPl3UE%}0!_F80nf3LO)sfTlaT%YO1p+z7ovUJJutQdH58AEK;f?1M{VvGTPB0IH;p zvzq6?g>DGZAN(V)R1w+Yx8N7W{)gW87Jx4ypKCTY1Cy3vtgWdlaT&3?d_o-Q0<5X% zNB4Vzy&zXB>cTJxsJ@H4a)<$ea#m9F^|R%Y7vUHCjs&)W7Nn3|zNxVMziPT0HEWZr zTiUYdzQS(~l71|{^X^=KcATyh2l3xDe@w&0wgAMg6{AFi=4%z9 z>wWpX{jeKY*5_Q#!&szM! z)akzTGz@4u+t#poWPLmI4ISRzSvc#jfWY2ylWH8aQ;AhHx;&j?oo(zfr*)@I8%qVT zZkdwrbxfI}gkR+}TB$xt{RNv(mr_GAW=zD+9LPDa|J7OR@Jf3oDZ=w-N11Y!THg734*+F^;C;2tDl7-CkE&q- ze!rd0_6Blj{o9ASodta5u*cniCWQtE?c)Ic+PalxM{ZpA5j+33b6e{QPgU@`3=?uL=P=^<>i%c@l$#q!vU=>Sh@@1jCx`TYxQWLO9rToJX3EOffyFR{r7dgmKjbVoZ>LR8-y6I7)A1$^qpe~U$P(wh^Y7c5xSWB zv_!G$GJIqWLHe?qsfmC{(@H9F+zJOi@*-$htA}o&XTfaOj6({};E<++2J*&M^XhGf z_c^*HhG6$CAlOG&{=2}wC69F#q(+l2rg7w<2UivTME$;l8fyz!)8rGqZ?Zxy_Wc6xq+lPy4k>(gmziI{ed z{tYG1z4E-&oz{t6KbAExn8!l2vQ}~e5<#mE>ji@DSO_1Y>ZBS0ySM%9_%Gz>F{q{P z0=YmS5f^Kz=I*nx*bhm|fKLfTON25@l%#sx)YH3u7q1}mQ??a#Tf>B{4On|#MY*6Q zlOGGN0#7Y}w9;Lzsw?L=*{BqLMTpnPdqEBk4ih9qznFA-n?Gz9JLI2wyi0VfDMjfp zf6vdu-i@RK#rwUx@5BXK$|^9C5F~L&853X_=-?(ItpjLR-I`@nAV?_07G|e5sT&<* zue5zn)B1MeXyo|R1iLii^1BJ=vYA}mk#kZkR$($L?kwH+Zv7q;0bYtnV;A<*Jl2sQ zx!Q`qQ+$YU%lbOsb@2D=*Lg<7o+?h%saMa&B%7R^l0_pc_czdE(l^zOKt{VR`~I?< zOL#?ND7f0$M>JpY>i!YtuyufZ7^wX0mp#dEX44u*#q0RNA0c^){qRp13@FmS7P~wTrcm>@M~~h z&XeMw-CFkSwpTDCx6ulQJ708Z1lu#7*Ykr2|2T;H2rn-}=~usde1i{%nx4olHU#er zn&87fJC^XyiXk>Dj5yGN$k@4Py}A)XxSr1EoE=y4?~>oe0Fc#}_#CH%He*CqE))uib8Fq2>h49=2A*2j1FE4Puz+RRB zk(W2ZJFshT@b-ODeHN7~exu5%Ipxn`&k*$d_$u2e<2bwPvsJhuf@$3}ObGR>UwyO087-R)1xdJNUp$9&Hr$-?uYvuBdPKNXAP| zVSU=%obFE3$sB8G*ckEcbjld6u5{iC55U}^yH`<8TPQAjY@S*_k{a6{QB+pFNuhVj*dp;V_WqXKK_V~mhFzV5of;+8#3cyRP?+& zwcD6)5tfDR*+SQ+Mi%qUSEXM(Zmv$&_ZHiaU}@aYh`g{Y;w7Cn#}&=ZpUu`gh4H!U z$RzW-Z9(RP7fE1|*zY_T)1_JbMYG0AlKyk(5kBu{iGM@i8m)NQV4z{Ld@hhzsLG;$45({E@?|db@PT zbYJaPSskx(_Q2C7G4UqD)SSL)=Jo45Qwigx*f#ZkdjCobHHFf=d`I!R5K%LR2+#WL zpV==;%`Aw9VzV%~;b}_R`pg9ry0lm>?|vm@JL>(xwX#~JcOpkUR_kf+57WGlku~YN zleGVH(MK$0XU>XowfC2Q7RLe4HV$@j$^46#*Nbo2JwT{CRq8H(mZ#k3U~lw`$00SO z{^&5vkA{6#xG#2T+HI)X0iTNH=j1-~*-*LcJz`=1Z>jV3hEEc{i&Li@(H7$uw5BGs)IcE8^I>TjJ#~&P+?pF&8$F!ZGsQPFFeJFvpJhTwxo7L2|>I zV_I%#gELJ!m+(NVqjETV#AJiJD=d_)PgYZ8zR~Q-XS}4zez$HJyomw-Q8b15;l5FM z#he5r(b=_?Ck#&hD;4uNYgDk`)SYf$BZLJnckX~`|EJHFiK*-xLS#J zdPW8npWPBNnSkpa_aPQBNARcx=l+@J!O2N(x@gcZSez&gXWqOpg-oIorXtusKRym# zP1mgjz?>Xzc!CYOZ9}|8WIfn@ZsaISr#lS8Llwy{b5o$+!2Cn-W5(X3VW*O z-(?Zq5a*3qNJ&X$#6Fe(m9LZwPf<+dvZmwVi4Pa2)_`~?8d^qL+T-bO4o_87Rb$i{ zb8Keo6{E-n^I%I^*2p^i(LT27r@?kIh26O(#Kk`fNA-p#4|3q?MWbV7jh0K|d6}J^jYmc%o9c7T3$H`Z#6+9@MlwA;-RE2K(2&BLd}U$(j1k=&`m8&>AP^0Y~8l&dV6lWY`h`+IPdfQ}epQW?;%H8(I!+K3k zDROdh4v&9U?w38Ld0vUCj>{{u1$7#dMEOtG!Jur=i+uO4VoS4^HVQWtt&P=22;Ow{ zp=m)uK}KEOmm>g!41YkyrQ@i$EA1R=j0Ufj-B!7}_G3}B%4c2|2Q7eUIZc7iY^*@# zXlSyW(T<#?pYGL|CfV&*cC68kY|oRKbn0KRB_vJP$Yn*6@&76_XqTW-QFvv>V!68M zt2v|oj3GTf&o)xz3-c$1UNiBg?bX#@N4lBJ(@M-9Pf1N1n=TVeKVS$?SQHMudWd~!;;-U?ke zwMf1$O=jg;?%MHlt-OMQ&^Br+^FKeLkB*e&-&|mlNfwVODx?XoAKi;|s%~z5k!%0| znELK`to!Ki3sJJkmc5efJu{OP%1rj&Dp1eH6Z!Gm>&^7x5Ul5g!pb_N#))z4aU4{u0h#g{Rg_|Z^rZ^O4i{p#{??1k1gIy(B`=0xfG$3#aB z{x)W{EB82MhIvo? zzgD6rzu<^ZDcRm2PYVlXb93{-f9pdK;3&GW|6M)fyJnu2$|w;TnL-@1%Gbwn%mf7L zy1L}%W@f);>%G^3w*S`Pd;S{@PwZjPO}uZ-K{uazR&BJzSnw^)MVy_zif2@P{5bvI z({FIan6AUx(r+;4&9Qs$FF$?-_XAn&Z19aUmUo2cv|FZ?Ron2+Kf+?vnWh*w4&zcdVTsuiVR_x*Swbu1^*)|JM25-$=Q^%a^( zcm8u3afY-niX+yy4<2WZHV1WCjc7E?Q@&z#rq0?U8Tu8!Q(}HPX|ibD|MY|5&39N; zX)2sgGv8;|;>|p>7)odzSpT|A3Oz4N)!1}g4B92%mo}vzE3Ot>pJZkEcos~Lywm=y zDLBSH&H8&MQt>3%Wn)zB?XTE7i+7c(UmBL?&O&qiKqeMY&>i*rGV*BPHg5`%eR{t! z9+9DU;OqM5zs=X_Y9#;Ot6`mZG)s+QR>8;h*Ydi5Q$jNji?vlWs!kj=$G%go-xC^m zOxHK_u!E#-yZ3z0pqY<_c0Oh`h^J=E(3FwUtCz>YnVTk9RHemN>xqA$`&-DnG3)}B z{I~0c61JC~p>;+Lr4~<8XnW(b4785n11Zk^UHaTw>#XOW$e0R?Q9gKg%2HFyGNhU@ zEAbp9bV_zcE>2cIwWG9fIdMr15fcw<^|>FJ+RbG3WSi@EC`s+}16o}uNo0N)|6bPALYGi_ob-J_ zfs^O26zP*h>(;p057lGK8Y~2{RipBPoX>^KT&x2_e9)#9)a)|`pKkT@t+AW^3gmN9 zJ3kjwP&3acKoiRmn)(>pKW_SU^o5wtOCAFCNe+hUYNAMAkvEw;k6-8qZKDe*EP2QN za%itdM`R9K>3?X)Z9VZ|xlXvncE{l>-z@zr+9Q?QyYCn|JtUD=g!!o7W1G?#jJ7)Y zSidc6r%lQaoF{Mg{|f#JdSF?6h#l$umLt?dqkfxkue^;<$eo#h`^G=GM(zbwM%28C z&bBRK21`{KrnwuNnxx{3|H7;`cgFpNS%Ti0DG);;V-oJluHIOO}6Xaf=`J)Ij zDG3RQ4=E{q9Km9)tJq%N-cKDIINI9VHQ>&D@S^5}#Ko=*41eS*i99Q5SE=&x=;>uYQbG#oOnfbQ<@KY&Fts;Y!O z3%fG+C2%h}>}+4(h?GC5b;XsH4Yq)}q|XWid88QB!>{&J6+-TtWKf}B2LvGQ5EB~@ zewDk0iCJxG?|Z&gnIP^hz^s~lqv2u81oPAHYAhcfI&!sGlm>F*qal_R`59fZlwLro zJ(0%z8>}&1<%}lV>i3Q_-lgGsmW7K8KPM+=9*YU~-z2-=zCG~O{~ARl{@~w!ORBiH z@MZF0e{JJLJiNb_mX=)@#M8BCEslrmDgroTzG%_5?=5u<7Mo@>_jI(jn)IL3eEIgR z-CT^P)O8TX5NsN5)VH`afrDly9iZhl8SzQc#!E4Nl*t%;n68FE#64j}iRNr-3Pkg- zyf*aV#;X&d%XDF~6PO0DjL!^AYc-WkfqTU=U7`ce6PnJ1g0;URYTNH_IQ zLtN_i;hX`J@gn0tQ-w6tzW>zg_Pg&3`Co}WmoIoGLvPxzl%({U2M~42FFZ+d{u~gUx>b%KZzN4HZfP;1ur+vOTh}*d3M#ZH_`VAb6 z4Bv}?^H_{rNe>WB4z3T`pTcMLrq9B32wQ$t2dkB57?bIj)ggYUdGcPB=N>(^=X~It z7&FQFj%?PT6l<(8K&Zx7xIz*j0LcL2Igt)iYCJ*Z{)dBcq+dX!Nc%Bwe9I!d?aF&0 z^(G?5XOR&@au*?Sx=aL{n}*HLB?F;J&}-7RmH`|_&GOw9D+q^0%YOIpN!}t zFkUv+*I%Qjrlykd+uYIG^dm6w6;3jspA9G6Gfc;AoA}sPy66;p**9%I`nQ#PS`=7tlzZy#xX*+W<8!32b(Vp*JC=`9=!rr$OsKke+1kDRdSUuvsO*Ld~(87mK19DK=nJbRo}QeT(0 zJm4_(R`R+ne8F!giC&`t&)kF~N5W!Q4rlK;!m{0>BCg)g2O^$-T{&J}k(*gq(9_YO z@bmL4Oj$qtZGq;wLgHR4ZTB;FhtkPJ^x*mY$LBGwFoHz%PBPLt7hC z5lVBAcCH9h$*ZxES~XyJUs-ik@!Nw^?YXhkD5v~01Lq?;^Sn}ntl>AAeHhb6tdT@h zIQS!av1Jp8pwuo?9)#9?Azzo}2lsho@=A)Q-V0C~3ACW&=p`w68HAv(*Ww5j_az$m zrabPMw3Ic^fuCUiVzpPI7&1)`R*}MCY~&_?D>OLXb>B;TA5HPyozu$tsRMcqoYQzR=;%Aqs`MuWeq^Sx)8D zyzDG^F~eGjnZu72+kE~ztDfid@$zn?G<@7v$|n64R96)G@zgE%u0)>Y+#dgzSHc+K zQ-LEUF3#g#w2GC=lLcS zx4*-fKi}eY01DcLV=3r)cnDqB`a_hNgW61_U~H^#oQne85flEinyPp(>$P?lu8S5s zfzp@ck@9vECFn0|+?r$PWQ1G|`l>4FVik%21tbf*Vt}UvPvV4jdU|>n)``zUKU0an z_uxSrI7&jeP1~D({zO~qi0(k1JGRnIj0jL*Z%)s6jO-c>CyQ7SWZb9?4GqX*Fg-J~ zceYVz3X55-2EU?{aOj^r+1|b@6;ATN3H&2-ePhZLg3x_V{G*`{q5#FxZd+LYMz_=g z1tA|z9SJ?_nWm=k$?n3Rk-V|HwvOjVTb_QCd=Yn#Z?U|{8!Ba_Z^BQu2uHWc?+^Kf zO4bs?8B-){VtwxuBZ*RGXn6R!Ozl0ukPJ zyWwF3gl^x=T#cJXr;n;CK#{d7d!?|fXpvm5$H>gwGBAKACnxuvIFt^*r|8w8jVh&| zpWlBy01S`8Z{NCBL{cB%vy)&M_#KwdZP~iG@Jx{g(D4$p;XpehIJ?J_LY34H(YV;MhR#24+aT(9$p zAlcsX(|%LhxGunuF$oFd%L$?8w_zVzMT1B>SsYE;e%=-fb4AO?-+3{eAJx9|eJ4`X z{i?|tE0g6+$VDH9jYYyCzt*3!u)ZD!l?BTDossgx7`A7d!p@@Ph8h~Li;9Zo#F>F) zuE9WqK2L^?AuB6e6){rfy!0#k2~EFa%iJ8z`4Q}*AN?z*!TOA%-|;!C^VRkP_m-+X z;pKoqv-W=RMkBZgJYlFP_Pc!bNz4NcJ~ACxZ~%K;@cjM#vB|g)H2xRds;Od_8O33V zJBH2h1s1{1M6tu+r=ii&k4#MHQ@vKnApk`Uf9k(P4Wo=d4k?Fz6CATX2(P2eZi29| z@k#_{J3szL)ohtpuS9@B2zFj-r{m+h0}ZbYvKtJ`orPXfCNYm{)50_wHz2i*`Y_h&rlNTV?cm(J@^1Dhd+;p# zBfm?J?`mcx>lATh*8JfyTBQk3vx68F@^x7 ze7uGTAl#FHN@Ox6sQ0ppUi>xI*B6L@&B>APV?0FUJ}TTLwvDf4{n2#G?+b64?Z?Yx za>{&8F;dOby%8iA2{o^C`N5-DD;5qVywX!*3~J~(U-f)3pkZ((sdD97_TU_Fie85a~d$D=5FR{-TuPPl7C0($c8zj&))0^@%K#RQvG1 zvG}_ars1LoTQ>U++5mg{6q#Hiv%Z*lOEn)+ zHqCl%BzC$jsuIQ=d!YaTxIdE$fKTVE1h8%%J-EnUC!Kd@I=_G-CPSM1(+-zT!*u&0%#A< zs2P-+M4jk{g0T{r@0g2F-p0b|h&D8^FL~lY1T(ZJl(E5j@8jB;j~3HQ-*a-L;0^x% z{hJH+cnOHd9W!w#`Z;%{L94Gjb5t`kbX#r`UJ&@?t-`#clz9-}Vl?P9^(;*eK0@fn zY8+3N>}uf)8hS|?qgjgzUrmyPjkIK0J!i0t-L-#!GV@RH82@VY_lH};2;yPmKrI#M zSEI&tc|G7s47Bfvzz>sUeegg|N(!Z>SKo@rV1zdoa~vk{@GZzv4f{n* z`^IedCP=YpKlQxWkN@yt9_DsA___&*h{7SMh~5!R^GHSpjmKqQ$ZM$dK);(t*bf!v zzP6=kN!%w-oDal_3iuyJul?GUO45(2xw(0~ z(O(ky3heN<*PEMmZrju3dwYA|zkd(N&Sv?hn)0e+66j3Yc5xnRz* z!xROJIX`|RAIwt`5QIU`mr7SNTH6-4nkZqawV!$8NE>_Kd9Pa&CIaWlBxrXKNYCDW z6??}g@P<55X-hOGe#SS|(9X_I=Fk>dSe!_(A>bEIYTy$##KOtRX*bt!*C+jkKvHCe zs^99wno;>AOxm>Cx#?=wI$1k_#m0To{EpD`iX5gY^5c|gM6gGEP_X6WE`<{JplZ=d z&;|DKp3fW$`Ql(=HeaBJ7nx0b`ZJkNnRKr-(u_v!) zFd&4)$izg)ED%3a;d}1hf3k;e@|{n_ZF6~g4tbt~MQNgD#*FGT(0y+%uX;Fp%|bg> z%P1n4Kg{K}1CkY90lyuW+aEQ!9wM14;Y@aCp2$Q6!Nk5zuRpZghF*s+D=xqlwF?v{ z|KmCd0cZMF9(3>2F-%!99yc^`$2{Km|Z(pICpxu%&p8}=v$XN z%M(8;=`bpntk0U&qG|>P{lYiF_q%skgM%N|#?fI0i`_RHu8XXRvLkzySq6=Yb)2_b z<6!+@=3d1om*Lv-OqL88kVSRBJ={u9z|2P0WrM?rdAVp zAP%o>^yTGcBiJWTCSXSf;|jOUFe|%m$qg^B0=$*RkazEHBaH*T!nbg7+kfIoMu0zs zj)4K~RsEIYWDxLf$IfT}tbJgA>YwRqUJ>`r(BnCOn(FFmmDT)$0^~5k1r+hWjWIl9 zV`JpVC~@|OflAor21IZXyMOZv=~-ECc^_}vm!vjQ0{Uz@JKWd6ZjUUyM)bUq zcR-5&`}1D)zU;=rcHibo_@^AW~0F(YXtO|TZ#-cpR(RM87#Z?a6QL? zZW91K!6?2uT|Kc4#$nWuz?-@4!!UGRong!ZM_BN4}oy3$FZO-IIy`oV(layiH#Ow=C?7IM*hncw_Vo?Bw{1?KA^oki3WKrQU<1do4{plNtes2O(Q7iyr$ z;AU^Rz0Y74eVkN17JA0UA@%&_$m?ETd3{O8_H}v2gNZn1MtgyWOfM_$tv+|Yy6`$W zNfj|^EqApr+vZ&yCB{g=$rGMz(aceRd$gO7z2&rkSH=i2dYIvk>;@hJau*TKa=J6F>ok?7k<9Y zaE-p{>|f{?8sUH=DJfZIGjdzzO?1ciK|Isl`DUPaI+nZQoEKWq6_y3;Bz`NC&26aV zJVOF^dIko7LvCi4mZlKnp*7L>jOTc9JF!>OZqE{C>%NAeO2;{B)IbDscmRg2A5g^2 zBM^P=v)#WxnDdmS4%T)4vMJXoHbMWIkpVaa^>o)I!C@SB#J>aHC*0|iAlJXuhru~8YT-fnlKU>LjG!@ zjhu5K+g>tAd&7;UQ$!Py-!JU{%&CCvPPltw*q*S z@bF1U+y_Dh{jkc*%WvV~UCVeSJzhc*W7m!_t38&LlcN(BCf}N>>>!!%$o0j5F=WETnXU}cWKrtO zo`Uj4l~GR19rtb7`yO(QA9ah&Z}8R#182PdJ$#81t@)t)j!owswoesra9_sT4t>LM zn65%W!0xsN5MQ#B)kN4(0}rSjcGFi+99}N=Q&YigqS;o~iY)tds-@zF9qW zB33{gx(DIhqEm{v-Pze$PpFm?J6Bg%zcwhzQ1$br=KNG`bAp3!n3)S}>e}Sv0;V*s zjHclKENn5#JwmWe!=^BPaAn% z*1@h21qFqs%j3CduY*;7Yg?^n&w_C%1?GX0kOO%YpNJ?3ny!#k~CN(>?fC~*IXdU*45V!0_Qwk=S75wc~Sc= z#mR#>0zCgugKgeL zX;KNctC$Piz9CsO7gA%}`Qt_4$lEkDf`z?o8aZai>iU5^^}K3C;P!2vEm`T5Bl<_> zo3t%MHY8G(?z6`LWin*zNLSvu>Emz7hIw*=z?1Ri$~$wEy%on?Ky}RC?0~mo1WV`8 z?WxoETotXm=k@n~iv=Asb*K^}q72ibJO6MYE04V2v_5Q63&bvgMd(Uy^=gMfy3A%hP z>&OAK#0+ycPGvs882-Q=_s()4)jaj1yhIRIVuZ-);yvDu+0%dO4BhXN?KKTci-H~Y zNzk70Q1)jgIBl_4O5!F@8$TS;+-9`s*0N~4W3pDx2$1SE+kMq5JEQ#XPwWIA>i5*o z>OU|$%I1#M_YRs^Yd39cv~Et{$+~sxmb8tHO`LCSQfF6JGYn(;3QaJCxWm|wEa6oE zjpN)`KQZ2n(JPH1(n28KwRy*AUq3(+kajN(g1U*d`nJ7 z^AA~@KGaSeeh&LioRQ7fcdOGWz;Ppn9B80%%;mgjWKq}BdIOvX#sj^wrh`IusnZ9T zU`+G7I<@Rg7ABhUp1w&lhhVJO3L@vVFzWlvub`;d@;>sDP`ZSfBBh_0nXGvVaS|@Y zppnsRHitoVn4rXT5D-0JvpKpIZWRVrCQnb#FJl2=pd#0Ji_S&N1;hF-FtDO%TH1sFk{8a*%%l|&p$G{HX*8@N_n>NPYeNp{$bSlz zr56LW9cHgkrOujbbc#p}~3m#fHpfD>Z1OQpbaQ{Bm&^MJJP$_0TmvE>=d~Qs5g;e>JV`~@i-!^qS zrEr+eHEG0>BV10aZ?a}#*;g#E5D{T6r>u;Ppk-vd5rl!$qQqJ6 z_K5QWBrY;OEBsWyGi$`d6t!-;Ix0r}rvhb&3@&?RGF}^{2e@~I+vz&l@@&UnncpTh zsP2;%mXJVcYhzLqBp4t@_|2Aggt%>hH)=MOpOSaZGcJRNnLhKIWid%xIV$&UEUdv& z%O{^ikjhNk1^V~O!r@%f$8_)6bUwb!8_l1n14&vQa2}(JdEAXxri&**c#_h@g;7KC z8d(-Uf+CrRZ?|p+*FB`2Z@uap7`TauDnB@R#PYDEjgHT8&6_EK+BMQ9TJ7 zuo%lU4H#D4ilgzT`owAYW{_G5hFv=spYy|=XkpjY`M%0od1YmG%jeLkyz9tAe%!!| z_70wz&DFdQ6}~iNCso;Br}35D?$#6;4pL@DV%aRXvZ)55SgZ z=qDkr9ZpP38)#>7pNFx1_rR5H2GCG3uzmWh)z}~P!aoeV-AQc>Sn6V#5Cf_S9l^-L zQd>5d?P%DW$a~Xq;B!bq0$G0r_;7UcBWh}%GiZ95ivAGsrm~k0d9Z-rdWONU=CK{R zwiRF+s8Fx)x5N_#!N|j8Yh4c>P~|jD z`3EwfmO$5N;@73Qiev8(%5hY4hBf08o9^c=?$;ZIZY%_r*Hl1~0s_ zF|%p!{`DFhg?s0O@Xk*_FHZV#J%H@6ZNJiGgx_5PYDiXc&20%D(aPL$A^b|8`tkDG zCY{`pLN|)pk-O@m6WAOICS#^qA?To^`gJDf5iQh91K zv$psGv=UvpZ}A8}XV=K0U>?4tHy^+;mA| zR;jbQyCsx>fy;Au{(m?zA@G4A)ydt{Uhk(*zduWn$JkLyCbe&TAdW{~_aCIwl_5?t z$dCZA5~4=tnOwJCmj$v}d)5{3iTHmMV_(!#Xq$ITxP|w%&T?8boKonO+Z?884&*&! z^MPp*M3I*Lm7dtNwCG3QdgTfY>S5-716~!Q)9%8(^><^aiUq=_ZHlKq2E$pnC0<*FN#H<*S$m!;Ro0p}Ok*eDrF7T6Rmc@+w`oqMba?-jW|5MQvDQV1v` z5M^RwVgkSuv+4L0@#(_{BhVB73})OkF*TiVMb3wPxl7&%FE6jq(9i${#@AhD@}uj# zLeHPQ1}uTpef~_8vaI^-67&-jGl+}gp~l|u3X$s+UD6+ZK&a%LJ5TkF^M!fX(v*D; zr$KdlaY2LUib%mn?#1ZjI_Xa0Uhlb&_qBJyf!Z`QLVI}PXCXaSNhm8rU&K|aC=A{cz2S2X1?WAed@K4(ZG$^W^SN!|!X z!=(=*VeV?{Xb7GKKJ5=S4q|)(>e^$1dYi@8L(IxaN2t(lqF=v8{d93HfSk`IFaHrC z>0PAS0mZ)+_G7$*ep9V8b4{RMU?hC0?!C^Ng;5wAaxZ^5C9!KI?!L$Q^RcqwLeI<&6Q4?pP8G-(@=U;EiAJ;8U>O4 zceMELI4G~r^017{tX{C^^LiRCe&e|r@;Xgu-NzmiB$9=a;4qnF&D!^-Qfz|H7*Z;4 zSd@~ps3tc(cdQR-HWf7;d@@8gs67w)xQrJ8Kqagp1o!iV3?hdcTYL!-XRrTlahi?{#Roj@~M3#MJKSFp_D2V-y#tykI z&SZC#_J$Dx*NMRJ_m|k!`Gd4zU9lalYJYrH`UkuoFqXdka7RApQ^`nxQg-z~XB1i8 zy{Tl9VGy{#vz0rse;;z^X0(!Fpr-=@iI~YEd3=>(?syrOIo`>1@mZdkf==SE+ocl` zSv1r?pT^BLWj>B`TJ9u3G;vDuhNGr_-bdB&;0bm+LRlrxIjadV&B5VnEi*9v*>x9V z*HJG~tTvLBm9;5>+f*oS2)&DWnPx3sGR>i7JtM-`eTkc3?}uuw4N;6oZ)je~bn(+r za;C#9dB(D?`LgPYNjSy&5IZc{^;amc-_!ormnV>8e$l?^z#uMuH{($>lI7iY*Z%Ar zZ3<3XPgr09`^|MmRF=?%aYGM@0YQX*$Q=+;#ln?yf)?IkFO^vv1zq6#n24seqKtjq z(G|iKkL^?T1aYsvi7>E(28KdZfE$cRtFK6r>jl9iCBXO#h*)mRek!-EN#X}4&rQ#I zXy`aO@!)G_dyqjTk-AYQ|DvF1u|r1w;qFjaC%3leS{u~kh1#8lpTK*BLD6q*YcmEd8p-$| zdD-ES5o6Fnz+_7dGGe0WSB}dJD=Vue*rAR_&+NVrXDPk`n=Ndhz4BYIxG?|l=@Zh% zIbQ3*iM(;(0gn=Q(;kA>4$5}t8ykkN{$-aIRVDz%kUPK-^BoUFxmLQe=QlROkv;*a z#xm=Nx9=FT$vH?YEhofjhxY1kQ|Z2Pr}n&frKPLybcA|?e$6LEmhI6iRcdAC)rOX( zF}lJSXK}Vh^{^i87du2)q#UwfE-(ORP*mxk`e4U~Qw$`6hw4&0*Z2Q`Lx{Nd5iM}b zp!VogI^2bR(673zrK#CGFc9r~;_B*()Tnl0hB4k-Y-`lqN|QD>XVR;5h+NB7SsMQZ zY)0qD2%b^nw^L-eOxZzVn&0{Gm570ais{y@^1BpyJB+lBD=Cj)X0Z<@CZ_bLr`Och znnGB@PW1Li4D^t-@vVHl@|M$swWJ884fp^O9UDSy8W0Gf_@Thbd7t0$tzyQ{KivHieZWa*RKFwIIe|e|I{zWIeTIbnOwEY7 zBsf$51{1QM?<)z|1NoQC?M(Hl-T513Tzi_IL^RP`t&E{k=n`}|544y`Ucy)OoCDBK zw(6QOXwE88J8x}6-kWHr+yrPGH>x#oiX(KBn`6#+ z?4}xmWUAnzq+D59Nj81|Id){38x*eQ+MW8F;cYD(`ykcpyiN@T>vUIpzWcv$&KRd# z#tjd9ulhp>g>89fJZpapehq3jGWp`#wf`|s%M^Oj7a%FB+}7VKCHjlP1s#02F}4JY zfF1!101skd<(wboOESJmpqWXansl%n{;sIUV<{Srt1yL&i3s>uYsoc0QK(|vHL;xg#Hfs`;C_^6E5ga33~g)$LMtiqe)Vyg zyt;>;UL2(g8xxg}9)4{M9R7^1MzIlNP*1?D!cL0ZnC9mg#SJay>^;jY z&Z{27JCc(W*Xl+{jQ(!}U^38rDse}j*TJ=KR44{O7K>i&;f;kOx{*;eG; zcD9==!z`@Zzl~U}pR*~Ko^%>&vqwcxu(JVUXI4;*kxlu&&q4|S8pullO7Ac2YJPV< z0wdOFesiGPEMaYr95KsfvJ)i|(`B(egfHB3DtYAzeSr)-Mh8Kc}Vp{lKHrKB4#a?MF+@R)+!#jD$!c)h=zjO^*AT9~dLpJw^y7Olj~bae?ju z6(i_H%z6`YK8MzUob`pk9acDebPUed1yI{x!#s3ucMSgi>5mN;{;!|0$$@~rJECo9 z41x);xAT%UM)U6ab)K=joaLZ-Ky>qun+TGe>~p(2h>H6~Ri@3IWdiEA%;Jov-yBWw&^ZhoB4}HR3V?F@{Rxuuz(vgn^Ci_JD+7 zq)Mq?KKOav^Q=>p$ltiRn$3}vz{D;#IGBj)z^IwIIQq};d1i_yYAar{j6J$KGuWd0kBO+Gr#Mv4IM0=PSd zzk;SwVAvRyo$B}bplAi0Nsf(k!6NCwrY|neC>^CUUY4)q7c3a}%9skrxd}RGO0hph zg7lS`g06cka9RN^N%$UwA6z@7J1n=(BEb=R0!E36GAL%KPVKZ%RW|*EcEJNyaykRf zH&`gwWKvutFdP}3)@*gdkQI!@i{nO(=iA<30ULQYU59EnTY`Q-$|0|zL3ro(>bL+| zguAUN(lab3v4W|3Z3xSh+)-XPG#hE>D~>NskiiGmcN%lP$nlW85NNM4D%V&4tFUGQnnsIqalB|N5kS*60z zJ%cdK_lP|E5+o^;&KSDQs`RLlq}!kqF%B)2g{9V{JqVIl2;Ux2a%E3ae8v_B@e zKz@SA%k6jJ2_^6gz%JzI74zZ4jl>`8&gTyx5|%9HMF?sCpHN#Cc6OrAcN%G6q+xd7 zp6<%+NZOl|ht>_t``|1^u%h`5+Y=nMMH&Ng21v$I<`xlE=|IhLL8%P zQT5Fu4CB@Ff%(YlC-dOQ2B&Fc8lHm@iDErBHiGDhu7eTtV`vMDB`L!NYGz;5c8QkX z{AIrL6RX&YJ0t3q$(fGXZoIVdJYBg33Y(@Ehws3T7bt<-Y#Beaszo%0tvTsHSmE*N zN$1;D&*x~(sX64}j&@{o)i;s2k5ewBTOUwvrkOW#3md!jz5URxHeBHZ)YMUMJg4c> zWd8St{_-2_{Q?V$4djoLQn({H%@Um?g6Lk`yU%|4I{y0KjQ;t`XCC>!q>?_}(ZF2W z2%eK=T(sbk{`0NwYF#ohE7rA>3add>WN?mz(;z~;7ME-9{ZoyHvM0MgqjjYxZ33}` zQyP-!nz5cFIIWi@GCAMXsd2j(daJPEjEzl^ox{co3|n!c*qxI*j8FEXDb-)KN0g-~ zW1CiYzI$harAlKz`AYQA(jf&TmI%3ulmN#cBJT4^jt$N51@0}M;OYOZoBEHN0@oyR zrGrQ9{k|$Vu=voX4$vneq3BuEF-PjEPh2Z^Q%nO;QlLh;{st5E5{T%kBl(s?*d+YG z_08u-oTs|gIG_OxtqB$be;$2YWpaI#t+Fy}b|j@BJ!1ZY#gFY%ye|6_ zE^2Rt!4srO8nQNs0OW-W(9PoqCk_q{@|S*yEdJM9J}UXWtr@m5F|eXi z)XvhwN)J|wx`BaFg2$=UsNdWwJ@U7iG}M^qxsFLJUCc|*!x8O?3s6mk;@wbDi?Q<> z&OWQq07urr@%+EQgA8qax!wXwm zID&$Ls(N`^{f}p^&@kY!nnic&2fmeq)FdC*MV*mQTxiiLvKDUp@+H5IkB`vd(fZI$ z04;c*I9|LSu`!R9GF{vSxU(2hnBfrK_Krg0m~P0V7@6r%$PG};Km#qlR^G-AtaHnx zJaNmo_qV~>#2lBHVdjynB8kNFv4U1{hG4tN=jQqnXyNINwx>GD95~btj-YY0*sDBQ# z6(6$R3}!rHpru8?8@TvcZ0e5#@*WafASEtFMn+lz0TQIe5BZzJ!^5i|Tdz^Z?F5pA zoKZnuqpoq^dIvTDWZdQidO9rBbo>HAS0@jTN5DNJtyhcCYaXeq58mB*3-=OOUywHj z2<+2)XV+|ykwHPKsxUKeN|$)!fWyFZv$#XZTJWPVxM!&oBZn)1Q!*}21gRCDjiK}w z>!-$eO=`4%MhW7n|49r%#<>&4BARI?{E_l4sD=c9sDx z0|xrUK)ii~0;Otbab}eU*mUg-=ZM zF6@bTaDgx3V1+5F$Wg@4D+bjKtDEXR24z3nEGPnI3SM!8B}<)693v~81nQzfo^ESy zTB5`Y``-rq&~7|!Sd2Mi74WH6qvNu{{CaMw21X?iBo|L(`KD(Y!fXqx-(8`kZdU)oI|=GW(bdUi zA;mfFMzHFamcF$t^yhBiQ>ZIOZtK6%kp`PsnGOh<1jLCZn>jo#z+BT%=@hkS4co%F zW&7iMURe$PPlT!Aj7}I1!detZVa{~9RZex%?%}=`8XuTUBrg9wt>ZFBnf1B!{q$-_ z;C)=&9#HM`gO6zHr(RgMi${Tf!gxWtT@z^p7fa>Ebq=9l(hgg)yVCB=yT)i+oJsy? zvPXUAmY>sqB2^1IfgshhoDbEU$pjajkCD>@TnKt`H`e);zVPku0oM+;MKmy7ULQ2f zVp(mLt>|Dg5QdF!)v6{GF@#l<=;b%n%*fx$>V4T^VcMV#T_SJvoj8k%-5E$6<8}{^ zym)yATrlVR-m07c?WJ|U=mdr4!HyFe(ZK9i~re z->h8UMP~PajWq%r7JhhJ<7ZF`@Q_&#xW?7BwU6&V(g6ay08%{{G5=Q}uW?&%G2qSo{cX+V^Il^{O$`MS=;@VN1)_(OFfg#B z5&c$YdrAvn1Se8o^Z+YdO>@cZW^C8T&aV5ipnQk6_HuViKEILPDeS1{oOgRIJY|8Y zM3wkqy4fnq0I{Ko9HW&`v{#x3M5E`5m0~Z zdoNkOjQ*Vm_%KmQ&Yz*5^F}($*mO_;YHLx)J97goWs5hrp4lF9-2M&Ox)e(B?{+7v zqTkC!v@q90v)S-_zEl*JmFTd0g6DIB$9B~IVg5r{)q@>H$a_C2Ecyr7y;NBPmD6AA zw72R(5JRX~^X?x=)<1s)eI7)Tw^(gVRUrIyJ7`JK;?K{g%Yd9@bg%{B2W-_cZ*U>9 zVi=t&uI{o&XiNIweRsNst+Uwa4Y186u`jKxL=xZE3V?H3SXo0=8eQ^}GpK-Ix09i>v7k;^Q*4x)HiJnzau&EOPxPCMjvj_+Hass$GaT^k#8`NtkeJNDKmF03(Q|R=s#Oosha`=`B;G7NsjN zz{JMKA0=%e0aVe19ky4VT6%8oZJ4?^wx;X^^s6=AL2tdZ@Qr%nCy{Yyyz&}O0-83`V73Qi+B|3={y$UsmQx;imjx_=%49ZSZLAiaf!^%J1l)o8_)HsJi5$goSb3lqeM zf5J`3OmOE;Fr1DTeaSI6_Yi^yTAJ8gHoyu50|RAM#5MMN%2dJNQ}!3NB67MV{OI25Y=skB zL03YRm3(-7zM-R5(KTr8Z$?|EMZvc&FE)u)f9_#vDkX4;d26@h>@Z?6AD(J#g#|1^ z5w0!YNTt4wvIn%Ng)7gx9Gye3Zr=9YgcvL(DILHy+ifsAZVs$plaO*#J~1?CEwdft z5WD<0)sP5CVi3Q%BLb-fny#Dqc$AE3ua~w5Adxgu10F}8-Gy>xV(q7bn;WTnAo15L? zeVs+m6)eY>DAT}P)ul^~{A#c%`2Gk+I~p0$8u+{8y#1HYT1@NRNbu>Pw{zU~M&UH- z$KF+GY_z7FpYo(52RRJ#W;=qAxfMqFR5ubZoPsetvpCD!5$S_M(`OI~iPf>J?b=9~ zw2fPWF%!fl_@@xpvbl{tUT! zdBS5wQ)?#sitJRi>uI=2PI;rGo*LpkeEpkxg5>FJJ2V-UlCRW(m*o;fiKu!WTqV?KwbGDum z?m{8&qs_uqYmh&{%1S^=>M;;qt(t8*5IvQK`FU@w$dTrNY6UR$=H@0`u7)9f7D#AM zKuH-6$Qpu$C2OtknUub0<@q(qibyR1vJY`B@nZ~hPR;ZR6 z1A0k#W#3PTu%i*NsJ1dhyO0XZsR+$(Tz5*B{okO7KMCAdNGBB1n*~lF$T~EG&sWsL zMf>db4b-aI+Kh_$>OcOcE1x_U!&rV**gr=g{geJ3ALYQK4_+)>NaaC#oieNR2*7=bO>Te1s{l+iEYl`w!>|$8QyC+tYwf z-3=jp_H5J$4vvAm9=v=Q8zF_t@bRC)pArrx3g#|8=bQ2P5CJp7aCqkT9HJVkxcBb~Ay0oK=2W&*6zk~IFL^X>YB%xkpPNMl zMQmflV%q7p6P8uoZbm`u%*&@h5^^-2zl!~^H3d%G03d+rK?s%}EG(S7x_#vgB7?Qa5D3IdaMlLW z94pFedtwx`fFVX$^e2ZMZB1oB@)zla2csSG>O^K8kqiJ2FK@a=b&c;u3K(}~fcYA% zbTrhjb>HDQLb|aOm6U|FQjaNMZvb){L@56-Y?nT>b@xu6XYuZx2`QRPIA4Ou_}d#R zIRgwy*Tmf`)~tkE^TUQ25}`!0mVU-j-s*=f)ow4&P?{#4dmq2l}cwe>YPX!yK*7#P0p0&lzo;V3CN zIdoM!$N89smmjr~FD3$V^RO{}FYBrwi`oUKGqB8jd&SYuv!MJA^OLIZdE!|qidRQO zm0wiebghPweB(JaGT*}ud-gw_?Fb;JbeIPgbFx!w{s)499j{3BF&HlF_kFMHtlrMB zO&(6vD5t*LoYX`DIJ(fbv)ck!&Ovfr-z>)pR@u!^6VRnq+7vMoLKKBB3X9C9_<&Mr z>gn0w+}fFWt;T9rXs+_0s7~FqZPXMy77Ot_>a){T?tw_dX(*-`U8%ig38q*euVMZ# zjF8DUeUAWKZz_o_r9MXS!vt`)ZB&OY&Q?@zb#uv3mcwcjW=$dZsb)sHw(Wu<=(||k)`W?L7>Nw>{8qoJuOi!I$c1pb#g!dPge`obwfCC~<-Y%=_WU2Z{oWL9bE-DV1**FVLs zC_u|htQ_{7gRy2xo(MxB%sP8}U#l@8yK*$>=+i;a(I*L$i*frn zo+hyw;TodgxQcg&11RkV zxbHZO75~nEu}%M@qod;IbAYUb-=|l{=czFr$5?2xj~FjSfBW*51ys&b#hj0GQ)fIR zY~I5@g2vst{TZUniMDX`&EG??OF?;^eK-d+H}eDmeQ`bYuYu4M_&P5XaU^qs$av4DIE#c0}sv}|Ef z5hU+bq4hPn8VMZ+b`M=(`yTrH#R1Fp>(@Uz?)@}7{5y%FyfCJDPI0tLOx%Trn>8jbWgc6Oq=~sj4jw5 zJD-pJu4hG&?;sg3!UooXPF^{rM?#=AYJK|}r8k9jE=G{_X6CIX7DXwbN!;=%4XK&6?(zHA^pscZ+B_g0QBbv6{ZTjIMJpK%sU;ozXVIB?J7y-0k zS0emb8U^Rdtu-dCO#ZmmT;m@p1SW>OYs8tmiHbW z9PL|Z6Y2dE#)1p)o`9Jk#}ft$fDH8NJ=p;U*_J2&GyfghW=Z5xvBf|$RwU(&pF==P zW5|~;JTQ3PHGR2*0f!(6(9BQy`5`!Q6A|K3pyXBhz;z8Mj^BZkgd$R+APf4$GS`r~ z9^Ibkv_6j**k#N8?0Rh6xYxKe^G2~oT|Y7+<)uK$FC%UT^`}(^F=?s6nc=!H`L z0%mP6iV}Vz{88$^Y+iAo4|ABoR7oQCQi#K(;Z~!`JLm*Z+fwDJU!9gbK*0U{Ob+Ol zK98VT74q7mI6d6FP4zAqs^;8|pCAFnV-@Mj&% zKcD3~bMfgnnb2XFu0~+ta4i-LmcD$dpbTJ8Q?|o8)VOuUMY>Dc3%S83YZ! zj%6K>PDw%eZ7iS|%#7*2*BaB**^2lK{!UeF>z0)GJG^wT-}t0uH-WOYl0fK6Fu2ZI~NN{8@jE0p*Q~1ntU)T5WfSYPrGbitv!zc19Q8rr&*s2Lj}OfCTx);sne9o^^Uf zs=Q10+B_Lvc?kfX{qX@B$*8|EEvs9{WwMg>;Z^%tE1T!QmySGU(6^*KXO9wEiagqE zBepR+&@}}FIIBzKZ&erp&ko}qsfBKf55R=uf~N2r7XNlDe?`* zT;RWVu$DCh0>O)xk?|%^(nerGedzA4tNZG}HP|O!Kr==iNISLwIuT#hsln@a%8d;S z!T_02Ns%v+4+N}Rvst8pq6bwCG;tzw_j8{&z?7sjeR87T_BJk#pmb%HYn#@hE5Kob&6D4kkZhc7q#; z)>vFz-lab4#079x49Qwo=7ZeyU%jApJ<%WcjEm+JbV2pa%tW8Z zebMqY?mqx^Bnu%a#{u7m1R(O-GQO|<`7Eh{6Y2p8ej_h~Q|R+DUmCnbmxh8~6jRG` z{woP1FE7FG9r!OkG@qxv!r>x|M9CsTZb94VIcY-s*?jK_!B+~a(peu(WuXsi4QY;| z)~z^a*NLmf-fngzz3j^CW2!Xm_)6t5rGx&h%23Dgp>?2KfA)p@#0_Kg$0R+QTiD_h zd|wtT&>scd7iWZ(nwfP;0p&qrJoi(|E&+?-cZMI@jAk((( zxJ~o`U?~{DuP$2q(9lpQ(eyOAsb3e{;i!ET3K%ZWrb`iXqcUgSGckQxK)5#ff@|jM zu?Vdob)rI9#e9#`OzWHQ9GF6a2gPJNib}rM-cC%ha^2PAnQ|<50SA6Tr{CRGuDR^! z)g>RNMDF6@py}uJOCSh;^V<{;cbdj(sD{lAlOS%@Y|REe`mcnPmLIwmGyX2$3ar3fLo>_G`R zrrdS_G=gj80{2O|3!}TcHv>$S+bOH~3X6AQUXt784hy=-)xpY`rRsfTZQ(Gv8fWYm zx|vNiyUKeCi_x7m3I;)?R6tbi*liWb=KSND5AD*a4 z6T^$F&Uy!864_^SHD%zKLht8*VGffIiL$lz9UM&JjhXg^)eytY<8<}g^(=oV%Hzp{ zj91OBHb`)jq%aCvwQILY1e=U;W{)Y{qq`zEoG5OSxU@Thx#72Jf2Zm*%vV5J38NhD zGpw#STUllOR*X-{?K(m&!cf+53QRXsv~NN;Ky!ch*-zq-$vtp(aU7ag~iX?lXH3BZsPP&~jKnHxB1Y8j_*8pc`Z}s<{Ww zI5*ROUZrnT>#+en7atOKKmtjG5>%{Cl+(go;R18Wf1D6O<@9WIfX1lS6$A6}@JgpY z=}R%{Jk@kKRULP$-2MVPX zQ4940*?afzW0dPfzz*bMnRj#+2`Ks9)Jw7)9;Xy!@Md1u6nOGo1EUY27rkSc09psO zwED*UEHuVSdz-Y1x526xZifapH0_x{n;w)3qgVDcBWY%y)5ruNG3;%u_x(!Y8Y2wNg3H#{)N_`M@*+TLcg)s*eXn^D}^ z8=He>q3v7G?gmzb`pqQIT!XLuG1#i84s!8`A7px(vW-^J_)N(>6dC9>Z}hq|Y$JbhoQ}mqdiq(DJ_QGHjMr9Y) zHj|Nyfr1h2ENcoDSwgYb{Ut=4v2IMUI)Z}aKFop{f~-O>aESQ9f|9nto}*W1xjHE| z6$ilW{M=xZr<&{5H9Op5~H%p&zA1Iev|;rv;k&jJOm^BTR7`~X1(Sw z#vgJCZ;c;iqIWc70L}I~-UaLEawN(rD~b0Mn#e8B#7lmZQ4~BuLU%u*EOr4~?J9r8 z(WDg?zwc;UIpY-(=+@S0mwm64I4nBZpC}Kz|C7oucXT@?Y@1Y(AWe*w2MOHfpP>DJ zPcgQ>4^|?t_WwRgtcBc%_WTrb;a4fFO|K2L`B5GtzCi)>@i`jI;F7Y_T~zja5X(!j zsT@d_ut)$!)mw1G+!~|{2{7JoZnuW|X0v|&{V6B0HaU544{BVuz4k0C; zf>dbWA9F+~$|8PMPUQ?ohQQ7Pb9d(uLGDOMgb)5@aMyxI?(}}y+*pQ)>jmn1INKA_@qE+FHwd?ZT=X*H2DP)n27@;^B>kTM?4s%fZi@nVn%0 z5DDxn8(Z(RE z%BtIF5*;BnlMqt~K0ZN# z>AcwAV|#b+p*0*rQX(mMt3Y2ElT40rdKS}SnYsFz>#H)n2KC>c=(O`Y=YIwhvar|I z*Q#qFnRedbhdH~8m0u9bGtY=cDOA0m>5P;Lye?lb7CUH}`|BIp?%shXGdd#7nNLS5 zG$ru*g?y=&rmflo`6U*6JW?6^lC)kT)~b*@ZUwX~kK$kBlS;o0cI|HgoC?t@)j6Ab zQW8Sv>gwi}mH$ueYeTnRaU=upB-*ZvS03t{?;N^LcEAYnrY0anMqEOZkYz`n!vLy65Mknp_9k} zG8jBM=pL&B0kDF43>&8S=x7O8e8LBpj*eg3Z#4?C3kzBL^Xlq8La8c%FazM>7eLV% z$yFxD#=`mpa=E%^3z#@mk3OJQOk@B~<4}skKN7tXE#p%aJE2ns#^9Xq(JWld;&-wJ z<$%=)**7Wz$(txqq=$zGA$a>^`QfTn|MPS7cA6bWF3RBkXp%SC>UBfO9p0SMsX;sp z5*goTs(fMCfXv$v9Po|N?ChY8W;bOz4>dopSC=v~y*`^EP!vb_wh7+Zw_Hj4!A@?YM0%ZSP+(w$*_ z9?JMIF1$dzFdt-qf%7Z=C#1+6$%?i40ck2J-0zr0^iCeIgF)XXJxU91O%R-5g@hfS8qPS|9-Rx?T@qL*E zOn>Aak)MRBvUe)au757)vabwt7kZ~PrF6o0B@#(U(wU+vh@JXV)J@z-38>J_R)4t% zj}u>pvvJdr ze7~M>c8l5~5ShVwXKKH(c&p^xZT5s*AS*^HYi6~HruSH133<|?ytt(NS8r5DFVz3L z0t5Qw*%ESHg;u(wUbCp5N=x&5-o%7nU5X72E5Di^pVy;p0kA8qt*;jp$K_w}g5ZX( z@x)p|0F5A!#j;xZ#3SR=v+bdRx+*wS(x`M9rsgO1U4(9gvcxgZ*R_ z_=j~*>U34BZ(*?Fg;`^jE_lH zrG0w`PAypU0O1es>D-ruz@Y1EsC*F=Vd(~fyX|YpCAnq>0a+V@nS>b#ODIbNH+)|hZTXyLgw|o;!NV&tZB7%rI(I`L z>}_oyKxu&cwtas08!;iFTvMbRB79Uqqc&D5C`d0*v&i{DP?wTUQNrVY`XvIfoe6AD z*kZexnZBhrG(M95G^Ex;6a$8}vGfbz>nOCegnb_#lDJ9pCs=BTn`>Rs{tEoOzpc1F zcFxYGo*r%~Uz{G&RLOME7?N)XDHSky2?~Zq-J*r&fhvv4vTBM+1QD-AMydZ)fafDS z+owV;hc0V$jN_9pNa1PKR04^rnOz(*R@Te)O3E0Vs-mwxTWqg6T*U7g5C8et8M2~C2I?2}KDmgRc#MGhJ7 z*%3zTx9;9~@J8WrOTaa{7*^)~rgsSDVaAT~1lCc@^2v!!Boch0*KlM|tz}&3$;Q%} zJiIv^XL3j_b#+N3a%Tr?q>5l~nm8ndu33UD3*H%ZDO)68pDEWnOKj~py84Aj&+Z*N zIc*V=Qb{3^=1H-SL}!`VgraYllcIVmITy6EhZrHy6P=u$s+qmo_>rqoLJuJflj9;W z@h^QFCf%l>vm2Qcte8{+y8q(Fh9XbSM}$;R^30X>IY01R)26DORZ&$uS;yE|nVz0r z%g9IxWvu{BNOYg|e;N-|92MAgj(|_?I$&Tgowz;HV8DQvp4$pxLj zjpY7p!;3gnXdRdeY8e_TUJrN$TUrTNm$N0!e9(%Dh{*2jl*7Jtt0v20hy>L>x(YKB zV)x53#HsuGijIztYZ@CumqI=t1HMpVWy!8 z0Aj2h_%9$1w&$;$H?G73U>6*j3zoRm^*edLKIk>;$}a$8#U305lOn7IN{nJutc%i4 zWmL1!IG05}8XoaO=cai@1pV|NX$vkYV@10_Rnt)d*Z%i>0=vJycXTnAdZLTwcW5@^T;I$R6!RZRqJcUtaGQ2CigzhNyS2j0DXBR6izdxj_ zvjTDof=8zs!eIP)K6)ULBlKq2U3U_+N;u4>FP|^lA6`XFS&fvn-Cn%Q8C&!ks_5s! z&@%SS9~lAhPLE(55uwyH74YZBiAqy055?o#iYez?J|jvghtxVPmWNUn_~Kc7khDj7 zg=e*+8(SvJRfXmEDzL#~q8w{KESKZ=g3waMNfDDk2BY+u7Xt!OqOX*geFmeJ<&2R{ z(?&8gLpJ+~Pn{U&C8#9M=I7l?G0(Q>h$n4-#PzR;-;7x&UVXX6F2|#} zQA(BlK1k%?w~D{zCNG`@-#Hfat&-r#uzN_qKJ+s#@cS&&)5I7KovoY(^85Qvz4c&T zBhA9RI%IcSbT<>1-kE!-D}{2TYM8$;zX5vJgnLL_4M)SX=Ns-?u?`H)5F|L_FZZe055oBVW93xM*US zAZe|sd`0u29(Q@mqTPhxg*SmHSBtPp!NbVKHEBocMJe)xwDRaa1-@ndtv}y*yQ~rS z4tzq^;?$Vv#LI{lx0dB&o*mCx9IIx5OceZ~{*%+VEa>vkP6@fVfv1G(|KNWtzTZ$O z3tx=J`5hWr+%d+|n6I*3LmMG8xgyX}p-GoR=`?UcUWBs5ll~Yd(oDTX07*|B8uUK@ zvXoqt6eV6F{VvO4jpSc93$zC$cK2@O-2Rg~!ft-O>Q2qOr#_Hs=dSPlipM@NHvP;6 zfq=QjZEc}31ae1Cft}IS(-YowvP|CZTwXjS^Cq!maao4XB9l0&7n!HXwx4-%s>O(@ zz5A5!LVejg&ZA!{w0vuXV=h&Il5fyKvtiAsAYmUfGZ>Tm=Lm7BVL@@X435k|(L0}E zoV3V747l$@s9p@jWqwaOYf0+S2@7?}nqXSFpjjw>V^Hm$he^vcFdunCW1!8IRncr4 zPDPo@R`$~eu>}$=?6yn91o>9?cz^qE?St9ltSzxrC8GzsK0RUje z04t(U=r4}QbFPd(2s%?Kr-~%Nl2aCXgWBoY6I~ji4@vbvU_LkA>N5Qfc%U7S`WBa# zq+!GV2`Dq^pmpy_bEY`6GsFe^E_QYW zF^5L@svKn8NoNI%OMnE7NrWfR{O{h6yq{bvnL)3eB2pSEM(VU5VN9)PkC3rz?;WGA z*~d4J^57Hh{_jc`pNKCXNO60t4Thh;JQ{$F$|{h$vIRCp)DufnuhfV1iblWOHh3&0 z1GYWf08h;-O^?Ixe#V4{~mQYajT(2!Nwg`xTm)Jek==FEkE zun(;5nVn0_2&Rx zzRf3IM?cr&spHOey$_wp-9ff{ij&?H>o&DednNn16In+Wm)0a(0(vh;L$d+lkpb{P zGcoAZ;oc5>H$e@%N(!l@wXBbn_1<`oNvcC`(UwQ}C@r$!fUi|7K(mS|(~cPv`7Zb& zRsaBXs#_A^brIS_+0M*B^XExs&(^YI=3t1WR0NEW7+)jTl8Q^}@pX6r<3w{ADw3ek zc0s&Xmu0SmxGChu+V0N8Fpn;7Zc59nfcs>>Bx9^fjl(UfXAB#lO}Bjnk%qXCOMdF~U_oX%NLB?9j# zyhrj61hKL1x)&HabH(b+f13o1LedEd?^t&(7QJiO*P|9mG9MqGxE1OL49U+{2I@Q< zQ1M&JX^)bW7xqrHG&H188Jei60BFz!m6h@I?|?1Rc>=MI7oWaTWP4~r9N3SQz0i&Y zy!4-WQ&R>2Dr5oC^v~W2U4jB+#^R%eK?_h*BXpPoFa!;lPyxh83$i|ezL>2K`Hts6 z#&)3z1~Gc)_wPYK2Ng>H_s=CPJUr{u<6sTg$}__t1>mBdc?~oJusl$P=6j^x%Xxja zUa;}va2oYznB0~qg;kEQFQ2~VMmtQY(UrHLdM6A0`CrSrZY|Z9Pw_m$KXevWD_F^P z+G-C!4&4h4$(M>zF`>G&3`)48gL4q(N|l)0?T9ZJKfL-{H8g|?D!vCcx#DHoZ_78K zD&u8YvW8`5dp`~N5(}M1)OVX+dV91RW|Qm?Fwgcd{b{*e-A zR(l^)E&xdIRo&iGZ5(j->|Wd?a+e_<rykeopg*XN#z}K#-~0SmWnIcM%_BJ9@LBS zf*c06`&wIne)PN`;vthjr#eIg-9;tCwi?!Gy3~-jdHu_Pyguwumpsa|L`@L4qcEym zQSsE)+4h0&KV^pMPRPQXICIuCE7q(Jq;s1y?~a^daVpfBIWIK45^0<26rV_z3j8jg z^M1IHGioSs$vGxbaPN$To!x2#^ZGT94X%*6bIV)P9dNJ^2n~DWNRGIh^ZdDo7#97g zmbz!Sl7-|&yEk6hi~=GpQbzPU{mXEkUQ2;j?TxE5M+DQ6i@)XhJjvp$9nn(dx#Q|@ z4t|qu=$syx&yOe8m*e>IEWX3;<~kEYE=HB(f;2zDFCcb)jCl7a-b+P> zq9}u#xYiqMq$fAtTmPI@pyJZl}(lsHv zfw23mwqS?g4wgpZi@p2Kt>8f=1CMnj_^}y){J84rG;qAuOap*Bdk=3ASRtK#oAV`8 zF!|vpxh?m80`q4z5W>NyBknWtf4`3+6{6A#-Ab6)*s|jZ7*`X6f(uGbkXSqA**Y6c zOkA&~D#KRdIoayuf=&-I(atJN0SXv7yWDbL>*m+l1iUxYlruZ#@y)@up$!;oWyvC= z>f9Y=VZj{(716_S{%uG|x3g9(-1>vxjj$E~R_L0Z4xg&>G->WPu0i7AJsByI)va?W zkN;!%HKWtTI<2%Y_dQr%p#TXmT#N^L<73YL=SiN9^HRZgscI_FPaf8!^63<4H77Oy z*P*1W9Z2PwgVSLNG65{#*}wj4d}Ij1W+$<(vlG;=sIZ*3tCtx_La7xMXsq* zBLQ`?YRK&n+_L^sy5_)7-9wXjs@Y93^@Su^N>W0m{rz9xTVS|l>8y7 zibm$r-D47Y6p!S=U@U_`T#$^nJ`R#c=xP7}!A}wa1z!$3;r;nb-_H4WMc+J_zR|Wy zg@#OsJfZGs_nf(#U*-KpuNWGiK9&V#2Ht{75o{c6I!JUE6Pvpss0J~=&~tE{E#`m! z*^zvQ=yp;s5jGAMoinHE(CP>2o9utXY&%Yd^prP+#_>WUCc0({a73JjH~!Fb}w+ z1E|1hBocCKGJ(*5s$s@;1~LZ+s5ucPYn`RP@RRl9A5=}}7gc$O-N9W-;f%5%wPx%= z0VQ}3W$G|xk|>D@q+zb7*}MkL)q~Wg0C#V6R6hZskXZBWpQjwN)e421Z{Nh4eNH2p zs;2OyMRM^e<6j8Ay@s3<6I14BwQUyIC#4dccgFF-n?~y`n6=3@&;<|WC zTT4p~gaBZyQ-7}uK&^946=KL5WDB@31jAcYZU3Ye4MEv)T}Xzw(Em@jM!{KGgPpla zN(|Q}ARUb@242l$_ydKx5FcMWEEmemE0Rj~&$V4ZrP(#m-1uTN;)O#ZOoCf2)o#|D_4TF-zOIQO8O*WSV!4y9~C0 z5Gnyts_yAB(7}K6?6b>6_D;zMtH)`Tg{e)J=q)Sl4P!QSIq zq+bK^MNMXuB?5=tv?;~*#qb46$+CvRF7M1~tOJjiGRv)UZ^A%?k3PAy6L)ANkvfOE z>GTcC49uY7w_;exovg5-wrn-7pRb?RRc5pqrsttunQ4%3*hz#ZGJng$K*4beIho<| zm~|AN0|Yo`VVeOS&3`J>TMQM@EljdBJvQ>)CceAtRK5`j7FFV3mmWfSLkIu69h&0RcZJ5SCr_x(oVpC*C zK^v%FqJuF1o0s)xrr=6%C9|`YkoS#DEzI%#(wd|`5fIs{%EJD6;5(-amW5_4I_?bl+C~ZJyeU^ANkLe4Zc`nJ)PlBc2)Gt28{y?GxL*o&Kt*6@m?Hn zvbyV2@D#S3wz5YHYythv#=kLqO8r5;1m~H=``TUGKfFNG#Lf z`(6tTNj{_=9(iJEgm?=t=VGuNwLJP1aFtx5-27rg9nL$jhhy zQbf)VB@Ffay{>}DMC}k@aUi07#9v@BYzOSsN}nSqk;;+#eOdk&G)7GW?Qw3$*OapHa%kZ=qL|tz0@H%80Caj>3y(1 z&$)j^0lCB|V|g-2b%4kE%*ksdY5`DjLOeW46thbdBwxinR=g=faNB^({wYY3h1TH5 zWas6H$+k$g5aLrPA(FqAktSe3fwSuSt#1kb~E{9_nrf~(y8 zxf`+V>aR;JooC@}*0-y+C!#ov8l*DhV&#_jEbr}==4ln6%Deh^%;8P&u#edHMz=A9 z27qv!=PjzrCyYM&GUYhZ^wa@?s55yMzKBf^&(Qi{6~V#kRH%+`)mw9O3npILNMM>v z1NWuT!s5hdV9mWR`x)qCp&R_Q^BiO{EU4X*e&Fi6Z}wF~K*U@F^G{fFzJD*{TAPw} z`7FAVdX5!AlWlpg)OyHQ}g!$F9+@!YP{}1mUD+|lf8ZI zK`Z!$4^QyD`gMYUkgQ`z$CEo;^H8(op6JGYb;Rppu}qd~lv6;$KzobFkPRm2kXvjm zOKNiMGTg7z+sx^cq?8wBDDXc&6Awa>6SOq-(3ivE$d0M+v#E&hQhKAV78=^l)na!m;*xqVh1lcnAJ#O(2x?go zfcQ>weOfZ*VSJv(n=q)Wa!(nxDLsVSf8x!PpmnTI4$209G0GW?X|vB6FnjybGlCf} zt4`aF#Joy{F6VRf1K@BhZY(HUhbN$Y{@R{$QSq)Mi#^!GqcKDn*KjV^-R94ejjmys z4gVfodr)p9KL}I<^?W9wfpjQ)ZOAvgw<(=yQ*jPOTK{&m^$Z7TACpBGkvkP7lruX! zoG$`=l$kE2peIE>aSw_OVbStbAi1E$GB-m)cgdA^3kU2U%UPzT)ipK0<97mr zg*qMZ;m5*GJ{W+EfU|$1Dlq5)IxNS%^d|&M7Qhytcjz3EN6HkImgYiyV>x$dQBl$4 z6*(DMK6Hj7C5D3QAT|A8*fvN{ys|H>&l_ES@G>CI8n@-0HSQU7@~Eh}fK{O>i

$=$Ec1_G^-qUn6QLZ0MtQjN2CZ8?U`n>TkDd`qWDYd;4}e*Mxh-KOT13Y7fG? zexkB>%8GE~XY?*Ry7yFv4c0+Oijuk?rF`D-EZ$JRyPQH|Av3#kpu>wQl6JLmA9Ao`q~=qee?|1m86jCVHw$Pj0ILGCQ^nM*HSb+v@b66 zw?3{VNe8~xUVhPd%0J70U+kb{eZ6pd&o@oh>`Er|Sw&>ZM4ZLiQ=+J?-ro+I?(|#Z z&6gg4rFOh)-XqL2_&PAUM4znugey!hf@8TixqJ5{-R94SHxVC-XXlN5`XV7XL6kzM z9NWMCeQk06m3)tEZ1Lpz&AZfN@yjwjm8HAlqhOd1_dzBGzNKj?KEi_1<#dVwxqY|j!$&!ZDRpLM?m{e2xWn+gN5jp4~vA@OUI)5(Mymt3A-AbxW+>DTo&Eb@2!@y5MgxE>}H}7V3 z)qAy~z_v`I!|WYMVEN8oJ=0p|f8pR%^BeOIbAotrEEg$;hF!w+dx!C@$9eoqTl^;} z2fiIv-3m8SCH$;Qe%Ltix$PHA@s#kq$c~XU>0Vd(ZLvalH&y3@c-yyJ-;W~4gT(X^ zm;o;|@7&4X^s|2XO!ID$zS;gSD!Sb%3NI7qKjy(wIvP7QH%e1-Mt29?@O6Lk<16P= z2r<(x{mPYQh`^li&}+ARg&+KFTQ~XmozK`#mNUol{ME&LEeNnINoV$c6p+`XRVDd;86>OE z=ls5@zwzc>)1lbY-jnd->yoqUWM&Irjqi{Re9O2N`MMR4``ohEcle?fX|K+1VcJ#8;pSR|GYxivMo!|a~a%Mq6C`6bw z??g2I8&1EB2l*hkUF>_WfAEh}IQ@3QrT1Qce}eL&^E>G=ZKAC@ltIK~^sIC&RsLR& zAt)*!(k|>4l_sjdNGKdNA1ihVzUn;yQf}~(#8)&?@PB(-_g|6 zSIOgiHEYeTVN$(vKeI}E>2=n#8Kwh9_iYIIS-?Eg|hv92VoYT*-Jw@g-mJA)!3sr|{gYf{hNQK=Gz;)l1Pf#<`_W%-s0wd6fi6dqM|xFw77qoiDp z^!XGeE3V`SJ$7q~!#{UBr24(=LgStz-mtL>`P^$TzPy)Um6JV08e70uSg4_EEUD9t zcT1Z2DBfuIY_ft9=j|57mXwaGm43mN&@Y_VTx35GZ$xd;(O7zV=J2d^#piqoLw-S| zCz0|`OP&-+pa#Hhy$tm5Nl{UEVF?X>M31$#!BP}bsIwtZHg+2tif2L;r#(*97Ug>2 zbqkwqI{Edz>VIM}a=(twc1~WN&X<;Qq*5GuH{PKTiUw#$Q&+d+TL60PR^0|lmJLR1 ze;I?X+6RZJ^a!PWfAuZBUS75a6N;VSX;N$QmVZO^r&j`B=VWU=?meEFLO+>W-hL)M z+e~Ohh;F$^;!{Uu5pN@`qIsVM`{6^szJs9c;lyMkkB1x;%cn631b_F$Bx^mmJS|9X zzh~dKbS^>^j_|CHPxx2SWx#8U9t%I1| z?^f=lieSrJ_RS{M9ZJf+Z7RQju{cDy6SteIX_V1eXI67FZVhhPxS~LETXd1-mABqG z95uf6}aA*S2KAIH^i@w9xlQ<8TC{xA20ZUc|rWYm_Nh-eLOw+IPjoUp##O4z7+c zJj*UvlJX_5jL#Dd{n~XYuVf#k3}UU$>3Ke_zBoTWt)JZXjipR&9=o~3=5((5)#E(V zw*HMnEn81rR{K&2ifaK60|cU1&w7#*iy#FyduYP0QyZ9NkK`CLUVoA>E9+V}>c&&z z`Y*ec(Ruajo2El-g7VkhrQbhJ9asOfI+@U(x>6Nn{VU9}7rLQzzAPLm=jY^mH|v^p z1TR-j)7i=Kgzy^)CYrS>-rP?}SN zEs=OcD*S3&M79ToDGm8t*^-vycu5+G-zcf|zrAb1N+P<4>!Gx zwn~C$Y-)P?hgv0&oZBDJ5) zJIL;PY)+k#-3_MH*0w7yYqPHS;;~{<$jbVG*8-c>)Gu^wOa~n5<=x!&e5^;y3Jd4T zm~OR_i;W+}wO_2tii^1%uc`VMPU|_ts%D|%U4?mDSmp~<{1seB^Hu8N0cN+Pb@q!5 z2VC9mOXqxPs0GUiyW!upI?*f*?bl528XA2Z<(JC<(xJ~Y8*>A_F5-ku&?dn}PrC`DT{Rev6m z`Kss1xO`Qh_kjjWkKSDAge{*L51Grv)6>iBA${{uI(m+qF}4jq#q5!FukuE6Rt`L$ zqmSq$3~%9>ZCP1t;=Fdx@kA6X2_BUeu-E}3pg6C7V;_SLAg8_|(R&%igyp)qN|Gfe z1LG?<$@fbM>Ue9X>(1lf3x#V{PjPRUco`;A=gsOM=R6n3H*jURLl8;DBQ|I)8InP$ zxTdS~pI5~XvG;%UCRG67hR~NE3f)uE>2dZHb_5guZlB3qV%H{zqVB5Rof^*>h20fd z!#rzV&S=I&ZRMw&tk`oBOwJ{jr%Asq&FGvlOjCHbwj@8?q~*5MUvn$PY*k09OAxLy zMqc$p!ZgZ`2UG=>wSO9Z1faeBmF4z~3lzb*9C!haLwASGhsl`A4v#!PRW_S&$ zfEB^Ky_GUO;wdgO*ba^TwTYq22ehKPU+OoW$Rrwv&+6&$IY;_ZL~=acKdyW4qd#)Y zk2Y34(=!o9&QyJOXk~uFlR9X2?_Zt@Z=s`b^0Z&K%TFFp&&#(~%JU5IcFUHwRVtY2 zw;HkYBleBj2ET>tMbf`KeeZ7x6NTPTK3>(J<4xJ?oU01+I3?nlO_pr%cROLpNZ&gw zEZ-G~uMMOLAq~&bs6SrW6&{j)FxXYq(jmHRj9-jcBNXtG+?f6FN}jo$6rNB-`kQKv%3= zFoO3GQ%U(3?}^&^eqwzELR;n7Ep@##z#M_p5^-6&l;s2aDfyPHJo@2cNsrS5ZS!?Bh! z#_l*;Jk9s&=;d>DU*T~&3wRTC8=LrD4_kesPZ~%8@bMBN{rhm^{ zG>NR2#~tFk>ciH2r1rGi9S3~7LcAA$=|lAwA|pr7OguG` zhAZ7A_y^jE^6c&j8bn_zi^bcJ^V9L><%;wRHjp1P#K<)yE1wGn_xGy+_bvVS2jg>4 z)ROX8cAgmz)kopc$_xw*wc3d7k)mGXVO+caHH?Iv>gJ10!4G@R zc^2n<-%q*Z0XCa8*Bs-XWA0aVg8siB>~1JU#SiMmcvvs@`p)vf;CzbK)XGvAL8}#c zVrgw~zj{v|68F#E=oK1@D3^C7R`p`fe?H-ESD#FvC);lgT#nCdu}4!x87Qac)`GK( zMp8=E6pX9UHJnfK+UlB2)FzIKP>FcIiK^jAzguD*ZIWK5W~4Rzi*~$d(;a_2l3Z38 zTz|SIZs^MG)EtgHS_%r{&*SD>MMcQqL?_%U)rxIN{_Ei7e@Dgw!=S_t)wf;KuOVNN zg|DOBecST7${`}6dU)6qFb3`_SQe@Eyuv!%@B_R<&i#lU{?jTXP8#|pQ-Bt5@_JN) zPDm`8>NZxF!%;v4;p+z}F;Y-OEYtY;tZXgIw0A|&oQ4Fr5}>5U#)JlWa;Y5RUXTZe~#FdC{2Na zD6eu`=lJgfypNsix2RDj6Q|xv5%dBr(xW08z8a@H(C2+lhJF)Ry)sD$-?73B{Za z@K_b(@*Os%M4PgsJ}9QW;y4PwVQBX{axW@@+xX3a*0DGr-*}Af?@i3fjTA9p}=k=`$biQw5zN1+&%S0{kkX~yy4JDJccQTfY%6Q0%{ zAe*m;b^Iy#_4A(hzY}loP$e1!lcGGGhfQyJOo)o{uq(u^lcJ@bMq_Bt1Luv7_2=9) zB15ZI<&@d5Zlj#l;=&r_9=AZw%;GW~Mhrgm@4p6(?zh91EtNex#6TfW8`GR(;7f_C z*uCT{Ri;gA`yBTLG3aXRl;C}!A6S*ZY%-C_nchhB=o0$J)WESZy03~395|so=8>;e zqT?{=Ebka!MIfmB#{$`q&Bz+!2yxpJ@mymcz13xpWog@_cy;4OWSnV^Eu%)&@7%-ZvKxSgT`tX^evZ%r@NWYYJ77rsc%Z5+K;Oek|AHTA`j-~EraL(ND&$d zb+=EVuZIeF#{au^uFn7W$!ISQ8j@^ayjSj+cN%|_!TNg}Nqy^HZidir#r~Wk@^Egk z(t=VPj*+E(W7xvt@vRE#{_@5g`m(U`1sC!``^drWY1Quy0oC<8(vqL(KN}L{^~;Ds zy{mZU&R1UbnU%|CT#n)z30QD~icSq~bo=Zv6C7!V9ZxMgY7O7orX1pFIaz`^EgRV&IQDOhzBVOa|=hGGo~oQnN%wGnJw_SN6ZG&mLakhdy@KNey4-tVY7 zHRsQ0IW-5ks2ZNC2h&BOTtdkwll1PhcDSro^pf0pG2=TFneG zh;X$~O_(y9<~4_eXY6oD$%h4uGA4yNQt`i6FiD~P&@X;!QS72;{*CWth$?YIin?u?DGffF$Z&!c3gRx5j-wvs^l=_Gw^ zwq#LL$1zY;MY;UG=xYyZuuUvXwK-M4?UK+=z~3NN#HIFQq8-aPM>x2{axV% zRBA>edOk$Jr<2AZ?(__#<$FMd;?vf7V=$;>rDoKlN82X$`j_%_Lp+R8q0$s&rMc5#G zj5xl_1y-==bl8J!ZO_akB7PD4QVY%Dtw~ZJqtytrTaW}@mR%hrYXnRxTQGA!{{4=Q z9n;~zvBb>pAy(bfW>yEp$X~y&-hkSJ9#P0wt-OgbpY*n0Nqd!;HB+oZ_s2KVT1Wo2 zuh~7+a+_$0OkcN?*tx%GxNdX>{J;)ob9Y)t89`;?;pP2ChVB7QT0^jH<&y%NnQ@5DWg>&k+JQjwp=S-9_N${342ycOUq)hDldcW#0Nym?t5PD zG1S;p=Ecp@>TG`3xkINv{s{JAiG|}9Wsn+=X=S!fBl~vL465@*&Dh*#F7>$dfocTr7v*=Hz<9d*?u0;O^X~3uLH{tF&EU2`XY(ueCHr zvR}7(-h=cakB2QHZxyqOWx8BNYAIJcuDG#Z68UEhTB1cGHf#~uMGac-)pa|CiP2;r;{CgOE-m@qd*|GJWbVZtZKnFY-KUtpxESe3X&v&#>JpEkvW8< z(&ZxB-{lkjUK2EVc}GWUa9T!9%Dj%(Ze+027S&MS zB#V+%gltS&)YG!DR7?})<`gQ zLSJK9bPE2XhqmPJ@Z)lH}-&HZDqBN6^(+-+^05pABFnM=(_#Pw7o2o)bXb+0lUqfw?JQGFd9!929xQD zT(%^~+U;JNJS{IW<>lO@C*(}a=a&qw`F3Mw2GNr)Jh64>+QuxRm{OcM%`vzEl)dt9 zMKL(O&&V&S5fM;+l7k1!%Y;g)mtzuYYTx^lfIicJ30IEz|z5s_Ko zxwj*O-@m2qFG#j5_T2*&nSgCpS>*zEZTqd}F8)*_RnAb^T!Pm6g)lt)?yUFQA9H{C z!AFI^<5cB63gK%hI*Q_&nC)ccg>r9uzf=u-;wC@Ux|p&a=6uL{@T^?d&pD#%Ol<%+ z81iyst9JVS^d^nA+=8A#cIVee`{wR(HJ*r5DJ+>Lm*R#P`Zom#(Ba>Yn@q-aXG1Nc-+zrlOSV2|Xq(F1Qh4&|2vUAIko& zjO;Wp%?8RKl%XfROdJnVm&$S|=(%W<9{{e{QE_z07l-^fi&A9MVr0vmGewc4pDW6C zv{AC-OlO^M7QHmi;5h<=C6~|Y9i@;>%*-|c0pLkDqwUJpePk|;>mFvU@d!n9ZWP?I zfqT~-!YlONA~pyfQ}dJ094pdy{`|s#Jx${--HCzSbNg^nvZH^bENj>S-;l#BJc);A zGb(Is?B6>#c63Z4DnBwP#^$tmxB7oJ5jm_eOlH68Gp_bL$`Pve{=`$+FlH|fmH9ZZ z?{BNu_VzL?btOf3tP@CmpzIB2G9#M3#(Ew*)ufi!re)DDa?*+YR(Rx+msk<;7ZPzg z7WM6hJj_oE3225=YWiQ<1j#b1tkZfR&Z?^S_b7@NLmd|Co%668HvY*?`YQ3mFjaJi z^Zvk(V9YY!GbNq|FSyolEBw^nE2a?Tcy>=@(J0l`2Q5qpqAq|S>9kuCz3&$o|4oE1 zWhPm0gnoXraO@P1@_@=$W~R*Pt)hb?SZY9Lf6&&^yTAaPtjPk|!lsY)dFhVy0-sY& zVKO_plSJP=cMkriZjra-w!FN&9#aRh=OJ+*F2(*o6aTj{dzXU*Bc5-z_BglE;1gMe znoj}@Xy~TtHy^X(<$Mr( zkGWVOE{67Ba%1nf-=Wjqz~I@rdF`Qcv#jAdbCS2J{$|!5IVM(FTEPysK+3qROg=<= zg^bxNQGyNuxxn;rrGYb$%A{zkF$RV2v8boNd~%N9`|pRN4bnR&)Q>z@Wwqz)Bu+;Q zFdeAgf4grRqi?anik#m?l3;5mNWAoDA8w>O-y(X({c-BC$9rl|e)>+6p;TuNEcOEb zNWcM_-ZfRR1BEaN`e(E^OeTL#hQ8zhS4t_~+_UT+^%F5#h}@>#l$?fNvxxb0xP~*c z(~J#lR)rPhoz24x9?j4&m&>-l3~t63?+*ZD`y~WNbs3voL_<)20)7ch zDm_)SGn3H_6V1>UkLzNc9iWcmSvgaKrKVHOlP#VTmu-}BlC4<>4ejEUM2~;^Un9Pl zB1s&U_^ifebJziNTWh2s8EySJ&HbctPpgzXVHj+?cbqL+-s!PAK4}RikIrRLTVW_L zy4BDUN?`P#)$cT6`$Lx2ZAFY8zSZW>QuFJhMYX!aS4cd&sG(0uD7gZ(8oj8f@FzKS zAb7~SAK7uRFMTx_0&c1C7nD%^w^V|i`d)+IS(PFt?l&DRLQRQQ=%WAPjEXA~zMhKO z2{C5yVTftua=@U;j70+N@P(PW$_JrDx?klX*&1(gFq;$@0%M1<)L@LO)Uwavor)2f zM?pUv*DIo(AmCp_xlUYy6A^M&OsDg{Z zDbmo(gTzVHkUOXo;?+|V3JG`zodzRO+vR@t_ zC^U^{^Mw168oldshP6BA>GPsS>3!__)k4qOp?!J^RS%qCaF8VHu8hRic3ptH#D8*l$T$e{igw*=(nmBsj!9nz0`p474XtKz{YD` z)_6OPDo`gx#5d9M;>9+j6K;Bhuy?VnfT=iMdQW1W(%ye{(99Iv%h7_z-3Pw7>eOGy z8~2RdKIWe>8?#|&bfl@KhMK+714TV^FWnK8;oaDv=4)@1)_pK6j=iR$`~7T}v1L2(Y|K(fm}7=>^>1CH7jPlm&N8H$NV(_91uS7{ zrag+Yo(v}%l=9QijsrAY?$Sg)t}0UTd)ELxjBrhm6#WOAwK!vh$*L@0O#56e*(WFT zt)_Kx*We>Jws+4+azT}*W8jOe)W#7o7%J6rC-RdzRB9nr-FP_3k7Z36LEgV}D@=(! zsKkSLgO_pX$%1P7+cI=Xuw2|B>@m?$u~~wvrNcgG^w*Qnw;PlVgayZd)3BX^BLyg@j!#$naW4iUlpC zu%(K=zrc#g_QJO-z$u`37uWR;@)o|q)Aii>vBwp`TF&r->X?%xCP1Z5a>cbZI8bK- z@z!Hok~A2TuT_0BhBNV7{>EZ0`egwwx2@1dU%(Q3-DvUDN+FADtPgb#3SYID{)q@_ z07I}qNxZ68zzxcMxV6WYCGRX)xg1wq))Zo1*7)^w;j=-%Br{c*938%b7Fcm+pzK{RA-#4AIH?+vbSB!e>6R;{o;!dAbCP z#eksYU7eN`E^i2Fzbv!$9GhqBhD0ob^^XV{MwOA1bg#<1v=A2A`q8W1V_Xyx)_j>G z#buzdp~3rbPZywtQZ6*vGx~n6_-q}wx;81wR5IJ(dT)h)V1~z15v+BuZSZ$OD3!+)C=Y>)^pp`uyS z#VuI<89V$zHb3Mj1liv%>lkU<)SAF}V&L9|MOv*eRF^AcR$oXI0jo;kef3~a@jSP@ z968QW1hQrN@Sr+H+2mMr+T3$-wB4Xto| z8piabxEfrQ+^l(52nlfH{QC}Zjn*S8Cog_WG zaD%0Le{N<{mOkm)3}Pv%)~>W%(+fbwMhh;~x>x6P+*{j8i^&q@AE`Z;e$Hjsh(5O4 zblkd^h;il&#UDQNe)7Om3aI-Fvpe&IL4(xX{Hf1~E@zF|N4?*Lqjq)tY$~tG%-It4 ze!q5f-9YA8z@G0f5(x?EWnC`tfBMjFVQDP8&nRSVn42)v;%t)4-WVs#&!k$ZX)R>D z#p0|kzSI+o(-JPsv(EpUU-w#d<_y9hQXdnavk;coAT^tcR66k6PW+8>UXGknPo1OuEn79P3xSw+H9}t7By_TngLl z(e7Rq3a)9U-ftgBr15VHCMF}JNTFcnxG&~pWsSpKw2?-M_Gx(3#dao zLOJhdP$o`@Tj`MsZuWi?=?e}Op`@5h1RXDmt+ZZvsR$%x_;i-c|NRip(yCVq{^p`by}YYwgpl_ z@!RFaRY601cQb?k?S<<2niMKk-iG`G;hm0A#_ij>x@T0yVtyGl{=K%kx_A(1?c4bo zhxEPyZB=YcV-$dA#)$6;2Li31@DDY-dXjga`%9L*LA8O^wcVOuTzV3)Fn>m{Kd4y` zS**=40M{HYY!5uO(W(DZed&gwF*I{#^M>EC(|MgEKsix$Rzy0aK~L}2 z@pSxU8BsnF+WuejZJHRc# z{GEB%JFYtE&mr5Wp=mh_SVHL~UNdDcIQqME%iRQ`nw z_cdeydTKJf?s)NzK*;8BcJKGWL6+~kO`ta`AoqDBnY}o2sG-1d z5iBMg_kOzh~L0DP~S(SWQ110J-SW3lv;~e zV_@dkH0kJ_f8}zCD}f0{8J`b+M4vnsmh2PmUtc{7W#kRjuawly<`Y38qeg1fB)gy% z`H09ESM^|gGou;$bM-oOV8=JjS=um3IcHdiBuNYc(R|N)4lN2QnL8Aj#rpkawmR)e zw~anDGAe(2jz`vQZfPHR)3zBL{|hD9cHDRg-C}{9aw1MQE2#CuLTu?Gmy&5|=FXqFl^PpU>f zR;$u+5x#$z#3syFL!qC&KtTQXER|*e%ggKC-SyLZeop|~BT0&3NbdG4(C%N=IX{GK zq?4zM2M~ofW)^tF=9t#$296>0teEP&{#xKf(^Pd!ybc@z%)8s#LGV;Qhar5NU zP6ND-29KZiFfYc#i)}s@v0jn8Pp}=!27adD?V(6J&<$a;d&Py93J@nBZ!ZbmYG0@x z9X-5L;|^F=^%>sL2Z|J+yp}ZhvR7YVL`s>|SDz^hKX)vst?ud4WJNV9H%|mg0h51s z>%z+yK`s5N`_7FKXmCnA0z4r;~1Ukhw`f9`q5`9)WujS1{v zy_hg14wKupWt}kvR=MqkGQzDreC|rl{((X z;j6b7hv{S4u+#C0HnEuNiWctFp`EW-%chp}W)1ML*DU%h0@!68jXBEQG^@9sMyJtL z=$NZd1!Iu|Lm(#!a7qwzaCPz10&kglVQcq;^jzi+-=F0b1&7Qp&`I`#qlM(`aq9bW z*U({o zUV&sL5P};ipCh(E-6|9(?QWPK&MvnhNDqhF0y5PA`)+6!Z@qoLbp!^TMK{y&4oM)B6q|f;C-0 zb5;At{xjO&&|Et}2!)4-D?uQwdB^P^zz#{|jvp{^Ym?Vc^c_2PAHv1d!2BQ>lH0$} zs}FrS#S5ydG2}7vK>XYb`tkQ{BaBGk$=ax5rv2%#9TqTqe{So4-J$`L38BH_^;3%$ zv|8g867brjahr~=vK+Jo?WZZ|@8hLv3FEnw~=u2Qj=AEULrSU~( zes(g4)v~A~1feKf7ftm2C;o`e&n%V~@Sf?G%{P@MYjkxdE8jgW*`?n();K$Tc=NNc zyf~|Rxc_KB=GqP5d%0O`ui|RhyX>L8Sfp>{<4<&SCx#LlJWhD9s68_7rwohBHA=<;B|N+Lik_C-+AtH za`MW_S@Z^!HL!lAn-zUn0yO*3N!pl)7nU?aGEURkR2B<_Tm6}7<`(( zCsrUJ0YRP8=ABCpZ~^rFs1Z53?A2m>Iu(?0e#iX1=q8{Pmv^X0#{4#@5OZLtDRmSI zi9&jqa8W?*qbibQBJ4D){Ljv0PdbnhyaI14MUb!{);iXL1Z=-#j4RbfJZIJrFfGc= zi|^|!vo4EDhcoKY8K3x#s-nwm5G`Y~0amfaAQYWD96hoB;xN^RriZi+LrgUlvbCNi zOJ=1?o8Tr`JmL1Me;0KDj*cmK>H6g`Vu>96Gp3O{h9V095$57(MAz_2X7qnr&kooz2-vIYnzGAkNip|!VaZm|TeIL7u6w(U@H}X! zg<$kZlRD~KG2Lk#=fe;wURguu2r!LM@QibkI zS_XPLN0UST{gIKTpHyr_sW6Wd;VHD6{kXb(SxZg0Kqb#B0UdBTc6T%}fA1&%-{#vS z1kR+zt3A5va^~u-xVFWC#hRg;1oHUn&(S)rWWu9@QCS$x?FHqQ@p6W7lhdKW&}*iUqj3D7tKomckV;;_hC>Z}FZt*RRi8b08Wx{n0$!0FGd!m#OjFw)SVxzs(lqxqOpnQFSa7M=IGw?{e1s%Xd+AaADEHp zu|Dy{s_q?lD5k73-opFj_Job9#dj0k(^ZH_BZ;8B6l*YK5>?w=^IS;7*G4jYJa5u-B0TWE>A@?T8K+8keV+8zkB-PbK& zkP`iSV^yE>^Cx0*axyNH?kk|&9SDQlw=#g4u6`EnYBD%~vvWONYXvgj#N3?N_wUGq z(CEHCu}5};zOeQoPwUwt2nrs{HXL{6qEYpSU=dt)Gfc34!&@<=jWZN1JwQVW0B`}T z@mPg^FOp0;-yq01!e!Qn2Z0|3VDonv0cQk=W0;tkx$2|!j6CnbkoCxXV~{CPa#jD+ zf&&D<8-oS5bjD+;pr0L2C&xtrXD&58eI5vAdQmQ)pH82>rVkl@^{qdiIKRl0rIu-l zqZA7JY{p|cGmzf$kNjea1|OYL;vyIrwfF_DYiIpQnODv5|C>p=gaS};n!;fC8N0UD z*{-cIj{Kh^fXrhiNds;F{`vOhu;E83?=4GaR{Gq{dqlQ5`^_u0QGK^<8$#FqKHz*Y znLgk;9rH>+47r67*`uWLxV#SNa(<%j{2$XUoKaudR`fqmXx%Vh(BwtsR z;qcnETh|5?_?=Bf>j&5ba{d_ACr%qYfX`sW6%w2X=1r?Xz8d9v_tOB>_UvQ81|6t0gPn}e8Q!puqPg6;`K69YP;@_^xc^R@nKl^`GXn|4^@H<}6` z@WUwwci4vW>}20Irk2pkAorvfQEfv-$0Lr_fK?moE_wAqLF?&PS@hgRO7m@2H-R7o zf^S9>OdeADONhV}>y79OxGpW2)pme{S}77OrQCIFhZfRv?ZyIlkCJ6co}{Ta(Bj^( z?BB5sgre-RO>CmIle!(I^;jP+Et?SP5;Js2k^MG)r#CO#KM#Mg0u2%1BmTk?3zB*Q zo?NuQRnC}fPP)boPLGg-mZR9{IlP11GO79E@YwTHqt^QtAEsHXPc2VO*X&q#+>0?s z0amnEbqoSl34TzToO;i|`UShj(&@;89!T!Z?CmU02b3*ZD)0VCM;4$Mn4(z}-ebKn z)Kf`eWnqkHz_28*TQwt28k0MwuG@~FRDgnt)pPRa!W?fc9liAiy#P*BULubb)Id*1 z1Rps};K7e$zS}(9MG7Qg29E<+g$kOp3taLN_p{vU+D<><^T`}xfVqeKGqhx$*Hg%h zc32)0;afCQl9WFQ^Xf$tcu$B6JmFs{9ch(GZh=3#sznR={%!5ktwd2uVM!dh^!lQk zd>LRty^S$3@X$6SNN7oov@-bZ4ufOBo3u`v!%2+bGJ4Bq^&RfLBDwI&C75*yp}Bvf zzsO)T;}H`7SDOc;Uu`0j-kI$T-DtA2VyYK#eIh*_zo!0`m^Z4_+qdDaep9)Czr3W` ze9<6l_VC9Y?YR3P)Z_t^qQjkO&p|iSZ$T+nU8}lKelI1vWMf}P=z%K;yF0b1gv(O>f_vR_zDZIS$2 zhsvi>T_y#QWM8qqPfN)LXR2`h&My#Oz#Ji507%|@bBg<_swq7QJ|UVAubD{;9?><&em(=N0NW!ET7ELl1OxI^&xbCk-bK_>IY zw<<{;yLCuma2zQ*`46MJp=0IAZlS8nA#9AG6%CS9yvK z{Nxs7?R~!-R?-kHxh!|%nxXr0k2z}dB|iCwVl&!9$D`_azD#HFmy-?=2RxU~^Z`*| z{nE9B0R~{|V9e%zR@3{#q)DtCRS(E*yf&3B8`*S1qDz|eTtAzkMJH+<8UTfg0tE+e z<+I&p%U3Z4wdU^|v_$jiQABS1!==Mg3v_4A3FN!7%x6P$gz4x$VE#7k=Jci&qX7}Cr^1rtYx}nV1E3g&|XR0D&nRfDyfL^M}S+9_(52S0# z8eW?i8JUfZN%8U6z}V^lhcdVK6W_;=AHlAvARoBb2hek%csoYlUJnT6O#U9fOaS_% ziK*#i{juGn<9IZP5;h!5oe3M|-E!YcgpJ*nLp6`;~3$ohQw&JV5;046U* z$eAp{(p!mPoXFwI=Z;EfG}&Sm&5j>By)rj8QW$g9wkpWfQ&_#1#OH;Q`Jv7ak78S(((e7bHm^tO!4T4zc7sG=v;T&0jS%m$ z^Re+zPbDFjgH*Zce4F3QtTL;K;BBpQe>&XEtkO$Cj;BlP2Gz9jG9vQQ@A-CpW^T?d zt1-?;+B2hijrwX?@qfETanymT5wNYH|2tzV{Ezv=9neU805=HgGT>|o6*uV9`iyGO5|;LR=ZINMjY7nb9LWLt|_y&ykUkbsUH zDQet8jAZfwfA&07zqk zfQY9w!nuEbkmG`7&*mV1u1kVI5LF1*F^86Cp_)~~#yfvB_rhp)bospeY$+A~TjiNL zcR1CNe3F!Pm-G$>I>k5@J)8qj9K4xIilOAx;#~OsI}X487`{6mrj_lUgy#rFkjD&B zD#gz_n!y@7UVG(~l8<@Ht+nhe{q|Ex4;8(w*?p?@-h1oJ@$(q>S-c3RVk=N_j&Zq; zJujS)i!O>n6g-p5P$K&ZOX&)_aVDxgzJE{koS)n&C@QIuc&BA@-<27V?qAq#ML%@S zun_b5tNNTBLqbm7#q})RhDmA|8NlX4gfK$~E__+q+M0Z!Ji>fYQ`MR78K_AmB5dgs z1?D|X6VjjE;lSmL&Isd47Z)pB6;*r-{B6vdQAA1UTr|VGlJL?3S@cSX5D<#hg6|JKJfn1QO2$cT2$xIPdZm*3c^Mp<*~ z@FiqeNx0E6$OmZUMLYu(9{mTi z;k+F~RX`XaV4PVCPBPvYGT5=yB$J2hyw2pVBk;aM0q}c1z1NSi9%gJdj{rydTbA-C z9hevYjiO3k<k=SIH?k(1b9N0pzMy%!&CJhbc_M} z(*@PwJR+HL0a&?66v+<5k9IA+as1#WArfQMK}}n~Bp20lTw2kc zkS$&n(E0V@p4FSSXKd}6IZcSg2#&sj0}KXsR2vWZ!lAJ_NjLK++EAW)n< zYBaUa2C&nqIG!i7)@b(VbB|51GMez0wO*kM$(`q%|J^_Mw^uSrHP^%97tGdz8I=#~~{ z?mws@_27SmQQ;_kzk19{xV#+Q-2Q$VDu7c_IkW&q~l~@k>5mfF$(Virdps3ElL^AK2p(0lBx$$ z)%6c4va%{@Qxq5jUee-Db>i|6+nwf2UTokLR}3E*vP7?^W8J8E%h&c*e+>$E=(O|F zWeUgLXx`0820I|~+ombs5~+NbBFI^kJLRF9#*+IRNKVlr6c>Yo#s7~O1570|A$@cD zSbv_@JbL8m$W5zK2JKlU3wMw9{t`8659ps~AN^b^xESYTBo0(LU8_ZEI)8ML~qBA?p6u;iK7L zvtb$4w_Inr^>lyQZ&kl#!*%yit4b&8ueH(o`HiM+FV;v3r|dIfu*gEQGb4C+=Ydr% zpjBkC-R$|!aPQ_#{*8q2y*;3!S0jK4vUToq^)$!X1=6@*B?7-u!z$A2!4bqRi2B=% ztI%Ojzy8dXCIbp@qvre2-Y%fb0?_$rtu+M{O5OV9>x5CG?qjFRs7Me06!0yFhyFnT zL4Azg|GvK`;t%V#>NjEgiY1DeS{o|QwXS|zCX)_T1*G7ovL%4v(kQyr%HmI%5V z8_ad-xgHFSo_|kNFtU|DqBsQNL8urnC)O5wgY^7D;AzDp(t1qTrj{nZDSCcP(-vST zzzFRf>Bx6`wZzjTK^8+55;nb|tHe*>+8v)JmByCdTK;irb|f$&y~C)M395Umo9-tr z`=1>T*ZRuP5njw?zqxJ~c zUIydxpU>;RV6yDF&8QetV(aT(et!B3!AR@L+XADcltviQgAOPaw)8*3=~x5OqO&E= z$o>jc=)$k~T)(oRh3GS<1c_j{QCKz>!0S|$=P~~fzy#zEu;Kv30eUYQxs34-H;8O` zRlOEnQRBZ%4lfK?@B#1PX;Fq;q@6PvG-}qTS3kA1pOcjztwiyit!x(Sjrjeq%}yK4 zoK97dCQuh$B(=yCsqMHMgI)osU!oD@WgrWo-Ki4en9*+&3fHIEc}kOWtIe(Bn9*gf z5>Qp?pCI|)q2;*UBf}OtASO(U@x=?}Fx5lOd3f zHr9OwjIkw8ab;i~RjA!ts1PKgWvy&6nB9MkaP&dBMd(fp>mz|!ea7t6e;FKu!9ru> zweHRnr>0ae1wDB>dN@=CQ#yQEPBCd*WNz(yw2^_f21Klxb7GHB5wkW3dV42KWp?X4 zK@|2m1ol%sZH73CBV?y%^V?Js-3dBn)_1f?1ZZCB%@=HVPj4{g0A7S|M%IGMMvvr} zcldgiNBqy1NGDylBf3mz!F@msP;yyGD9QT2GDji7;9fR*aGH5}2 zsnnBnyg8$z^#M57_D1tw5g$Lc8v!cw{^)*iq8PdZA7jSf zMf{v$lUyXRL((w94yrAMif0&b9wj7U04+p#CEz!zDj9A-2k;ubZ-AwkJg3D^jcLGL z9_)x-Obxck9pU+9Qx`WFM|526sdE63V84Noo|QYsm-z%cv!D3~@*t_K-dWH+OAsFi zQN!v5O}qkVd>T@-P8DS_WL)AX{MjZ%smp2t;CUJ z3w5kBys(y83<0|_$!0D)2ECH&t>)yhsUee8aso4ghOGm%kv9-HXLee`^%#Rr0f%)| zN)j3k?E%<7Nb6wy?Ah%tB*3s%BI}B`%N?(KP7`!2K(Jv~lvs-xE zgr0Tn*S&Oxvm`OtSG9;*w}Eus6!^a_(9Y{=Q@!X5>i5RtlGARJDOtnP3)R#)Gk!5A zvfUqjt1faiv;@^46EW{Nl;7xVr1MdgBf5BB~2# z*01PH{<474y$gYH^P$G{TpfVMax(lwk^RyDSi3sdRixcK>2)aRAq(pw^%a-lM#|e$ zke220Rn~if%8t9y7!6Z~2LR1zJ*q3yeWtGCF! zsKbfUzod!>SVmGxzSLvW*^x}Wm5|#t_q!Nw#25tws*dI zYY!`#7pw0*7Ty$9)or2_-CLiEt^#W|`+T*m(cA(W< z66-E#P*a_JY=sd(1XYVpeu;?gkz5oUC3LKg#^cD$@}ALR<|3|@7%TKGPcy8?LAN@^ zS&3S;gZ@#P|DuIaPtb?*?_$GMB^Uz;y=Rj-;%bd{c~5HbhWG>oV0`%v1Ol0voz=Ry zp3epjU-PPrsVOxW8uPxsdzzcD|7B=tdAw}phwAx9*yE2dksIZAxc|VK?W6Vf4B)+1 zf(*)=qs4AO|G!}JU|X~SAZetM+4euKGjs@}et|cUVIg8`&}7iB#2tj9H@B8|Z|QV_ zd9A^LR}`j$46S{nt?506Uy{a& z%z3;w7bA_Kxzl(cFpD(so!)Jm9|{4>uRHb8M~AT$+J!1Z+L-?K=DOp@o0%6W?Jg>i z?AwcvSfXBwG;yT$$>l$o2w*Vw=hi+0y{tDO+=HLRC zS>y_*P9ZH9TNRKHoZo?+SK#B9w z4@7d6{NCP9HQJA*tYeI;ey8r`z=htu*sa}S%XZib`>DH{7q?a$1KY2TgK0yg&6|P@@mllKNCkmJAZnBS;$JrY7nAi;?TqF;e$I}r>#0rGbN}(iZY~U~A?&rH>(Ag1 z1ud}n1yd+~+mi7;H~BEk`gE+?>y1h_Tat#5)Mz8ntHj2AbnBPc0{WKyUYhgN#6(Bu zD0*;E5k?t;zb6u}ilpjCSP+E)P?REkk@RQ7u^P zMccU1ar0aypG*j3;mX)@X;WjAa6PSCo@By9!DvK^UZa7Mb`bH^1e`=a&OZ}Gpu@+Ssi*EBJn?`rhcje=a|Ubl%MgEnwbyNEY(j!gyjt9&ZhFqO~~RgEM*PP39;e*Y5qF^<}&*2o_CS&#>B2GUX*HF*T! z%FG=i9q#+qeJ4&_6zfD=!f-KDE35q0R;}lIpdy|R(UwoFt;x8%^QNVxDJdy|ooa(d z)nT8$B)7WXw&ke#I7fT)F+_fElK>0A6%}`l2A9DycV`C3&fdIox zkp2+{oI)nOF8IDkqHzFqT&aBfro&lN?;A&}@-%z+S0i8$st;O3LVk}#JXWKYJVTHO z+i>1T0JgF8Y}ta9+65SO(PXwKE8JsgyyC*ba6lA#Dat6H_*A{$0i%+U=UodCkZL!8 z6j;#GlA54gYS6io)R4OR7}_tJme*9}JB5yO|Nc`}R$2T_1cO;IE%jyW!DZ=kO&UvN zU~SP}N1WKF?9SAOV<#U|r*mVP22EE&ic7 zw{upKYjwflrR8E}V9}5r8Fc~19HneVw*`mo$yelRV<}+GsxR~I|GC1ZUtU>~Nf?i8 z?g{9&+5$Er3IQLeXvfYTr#{`Wkh%XHc-{nj^>$?I(E0lRdN{A`^O%eWe~`$fKtz5E z61~nqs?{CinnXJem8vJ%wAvTH*H;sdm^Bc4eZ1`6rDuOy|8I;I6-vd8KGRB8x^Y=2(Y^7N!NiIEAC}GnsOzr#_DCxrNQ03VX1-uB4n?t;hs@7Dv*>k%zzr1&-K;A9OToq(FO$f;1Q+Vfw;FxwEfEV+4vCWlm9=psV>$8}oaD%@so-;hN!)?;Efw@CK` ze$whL{OS3_ocd~w%=r~oO?9bMEg3_sJ@NeMhfe=nTCW#317StQwcF+f+_vw_94s)N z6F&HxDjM3Pc;TQVg_iCjkdr4PT*30oLd#MQQ{M&=dV`H_Gqs}k|8{?lQ?bnLpd|xz zsoav1(4^JCA(?@&6zQYa6(B_s>KX-fs7l;gDN5}lbfADCCds1;KvP05B+E>mI8XHS<$7su{-*PGlVx`KiC-^H}xK?dll}JG>~ZJ z5ZNS^#^~5DIe;-34Sr=|P`0J={(o<}PatInj=C(s#+NsK8#e+WX-_6cPt)LhE5aomPzGcl0K;phEB;s~flEr!edE}Xf*bM5HBBN49845h)eak6>h2bLmR zcaJzc&ZwGWV&A&iz`XG`#(xMg4+GK{-fsZcc{dsn!Kz-T>V2uZg8Y8OS#+7I%;LD( zPotF(YG8+-F3B8xlev_9ouKzfexJ(ptpNC@uFV+{cr1)&?3SO*Ge+rcJijv_dKWKg zMrNAlR%^9vh>$7HzcPfHcrPs^(eI2n@5+Yfrg+8TXs7x%2=>bo%FU4<`CGnoTM)~Q zrBvRti(h!*@yqTEqmoQ2TU*fvjTmfB{rEa);4pJUb@mk3a<5`qNlpTdtAR>qXD|ZK zkcF^^%v2Dwv?5gq-8LFduV(2jf0k2O6d3Ffzq1`}UuGwQEw^7s)A^{;f`=LzP*{jJH$mn+&_L$5DS%A;B!39@_|_FX!!*H`cS z5Sz463!pPI1a167Ggt=4=yxT4f{u>IY9ES+dih*X6!Qjf0TQHUJt+ZM0TgHnQhC|* zL<9@gQ9GR{QXCg<`GTOYvYyeCe16=aN*1FFJG zbr=pP%B;d+vT{1oFhdxvMPk|APpb5DKg0{V&xjb=Rf)phWw?`s4w2JaRr}aBbT+TC zmtq#uqAddD<^0p1)VdbiMQZD(RwLstl7$xR_*Xztlh*j#`XH_pt=~(FjKQ-}#38x! z)-T1?^<~-{x!EKz#V-;YM!(!mchqxSIueY3@GoMo~$L z{+l2ZI2M491MDNTAD0(ix|ds`M|L6BF>bCmK7eTRfoLM(bw*PSP0g{g2FKT*Tvww^ zz#`M9*QHkl)Gocis!2D}MVN&W7Y2Y(q-k%^RF~4*|NU`vY-VD9z5^@^5UUOY1NUbQ?JwoX^oK4RS4JmAyw1VbZ-*2Q z3K*8YSrOcKsnmp3>9Kcw;J*F=V@1zQsu>}z2dH-zj+{3v@f^BoB6*^@(?B~ypZsBd z?wtiD0%lE7q-z6EVI+NPQ^nsPrVysbo8cX)=BLj!%vYl}(z}2Lgj<*Hn%u}d8z8*| zhOg=sFfa5cOu?<4-JKQLYc7?UDG=Er- zUmYeuDG7)n@jR3(&T{56609ZrrhY=X&3KOJf^$Eq6);NrB&ah^YSt9q)@uEFtVC4B zMhj_HkNi^hn<|ERc$*@nzvyYBoGIZKGx_3+tW~VwY#GnqUW~DRcf^pUG8O?FZf2W?^6{Yd0;Ook9?P(Ze#|Z zZ{rfcys;_GF+yQKk;1^mZIyOStsTPh2cMiWwy@s&>KFxq2RQyuj5zNLX0)!om;Zm4 z^@17K4*|k0dW~!j#B{v%yD+d}(4fSP-iOdsKo{W!=pZdxnsU-TO z=x_55(3SJ6t5Xacu+aK1Dvjbn=&NqsdbV=_WFZ02E{5j5`7ZN?}Ze&eZY^&qf+nPFR@g$#6a#mRhJO&S0ut%_1 zPDQPkCV{%Kh|C`s+NqM`hJk{!Y8ggI^7~fN{GRrWGwi!0w>F$m$v?nrFeeEs02P+3 z%Eh*Qf0ibcw14N-!=#ERB6yCVLInERqz%3cM#80sgG zpwKxNL-X^%_{omg!?HC=kEqgCV1$~ba5~XK;Rd0ZDAQ7PH$wPzO?vUp0_34|U-**6 z{LmsA!B}2t7duX5-up-g2+(YaOHq)TCjvqUMnGs)I^N&s44@m-85BnHxJ;<{8~%8{ z1DJigK=e%r@iy|Z+4e#mudOaReaUtRr)>%BB2m_1TvT1TaVb4Xh#d} z@g@(!4(}g{VRqz57xa1u26z^$RCWtN5>KA-yq zy>8>+Z`WweAj9nUlB%t`R^N47w|`4;!JP8MAMZQuV{bUL1T9Gyns} zAtI#GwfOCO_5k(7WE0R&0f;-1F$O_&%b?u;qWs*=XO7mZmDpro_Pm80;$7o0dA-9W|r2nW<2dvC<84zQnGYWLs-5))v9 z``a|(5s{xyDpxeCPS^xinoxi1FVkPY{wDuGBosEevRTmuQdLKR^I5mN?N%*;R&y9+ zQNQe+xjCM#iaKaQf$X~N#i|^^*A=@zCtU3-+!K8J3$Xo-vEXnlA{#4lXlA-2dv3L#)$Mn@Rt;e$^mm1%gCdjTBW9~BjG zIGiDn99S$sQ}wZ8PlwHJnHb#aMMWvdq%+(HR+`Axi@}l$=z{^1<{b5I&FR~Mh8+?b zsR=tIGS^>-H%8=}OCpoX?+3jA7pC!FZy}Eqq1?WUI5r2lk{+zy!%6}wkD(gKPkE4Y zF4+}2T`RGSb6GR6ppuQ;vxD9M1=EnocUG^NcUrTDAm!e_OQ zjlW|TxXUUK&NV-N{<01MK$J!Hn6fD3Hn-I?AB7q2Do1Z&`4_1%H9odzDLXU;$q)w) znFt8>o!UO{e>e^88sT?=X&)xzJ|MK)CAu~z&n|8hHM%0T~m7UpeAqo z;`kY}MpT*W4)3o_W$kTadQzBBI`c>W9O9Wo?I1bc2b-opEgR3Y%33bFaMeV|W!ZN+ zaU+UtWblv-+6~#D0TBoMs;|dC0QGjVpu_co69OzQKFc1oJAOkbhxpBHBSZEJNKC|( zB940xbzkioN9Pc!laV?By*)|x%5)T#4EWLh^RuJe&shO9DI=L7jUP2tU*$cy0V5=) z2_K9C!$ZQ>R3iHR5YhxC%UmFHRDB+=1vYNK(wOPNpKeZYaNZ%^K$$W%*YRd9m+etr zTD@h0coS_l?rf|4At6Fv*>Qz*^PRYej}GJqrts5Rx`Dh+Emej0bb*A@XErd zTS5SDE2atJt)!7(D<)V0mCC0KL%d7gk+f#7E${QYMM;b>nscRydnnI{wDlnj2W|H& zxC=TRFf-rlKF9<$v1?UY^SSl~_KvxZYRpj6pw$k=n9Wtgy&%(%<#9dWjlH3x-#ms=yY)w7CF z;P2DJ`URtyzc2cc0t)yWR8D<=25J6a#?3p`$S(}1#d@M~9KYLc6Sg8OFwVE3_wWI? z0%#qJe@MVzcoQL7YSotLzeA_?X3dBm4XDK7PQ!Q^1iNw<2-GUXjn`p);1*H@i&OZP zwQ}_?S}+tgni>MX5+U<*pcey}Feu1l^}5r~&}e>PBQ}cgxp~kaMbMt(xx98r+s`-M zC`*#aCZ6We@T_^gSD#K)gMCG0Gx{IkOC?OO$SRUnIwl(Z$8DG9Bdi^V%{%%-HMkSZ z?&b_3q09wgXEB}Bc|i}{1<^qBJv#P{1`?8OJHTQjz*mz}ehohx2k0xhe+F>^m}owH zr)9pJ6%%b3;Bi&js7qdI;<#VFA_LSfil03x(;E=P-CvEG`z z!edK?gmh!?=y)^(K)DkKpL!s|xwM$AFySKcj2EMP9=MKKEe8x3869WgpPh_BcY|nA zynk>ZKbp#&exvS4Z+Yn)0iNRIg1F%jFs=h6MJC-*a-e(wIOZfq8(=*@*X2WoE7tE-CB=DofX|c+U_iIBcu*iX-9bf>Tw%pb@zgT!InNW|v$vT|-s2$JPAv%Fn zqo}-8XD$EqZ8)bw-F44EWW#X+wqIzG17kpE!S1@ntnSoaC*u7*w82bz?f+bC_bvZ1 zChq{A{)}S(ruBWbEDjX$7DLRC|BJOE{oOJPZ2EsV8R2CB`an_zju-VE#22;!A!#mJJi0Td%@gvf)%_5(6GvI}{3kyMxi&@EV@Lo;uh?uxNBOPA2K5KFaqdOmw1PrufUnnc(#5cr! zj7)+PR`E}VYwLO3oePS}2gCar$8nc+32a}QQ`n;=z{HAzj%I4eLKbDnfMl3ECBTs2 zbF51jKR=*3h~RuXQx;R zNP=5{!iyTxTUbU}FynMMx-O+1HEdfZ)G`*5UFf_zE`Da`Rv3hZM&Q8Qt)G0}SOf7Ywg1j7|f_O-J@e z>Wy{q=4`u{swg3LFh=wX=&%G8LBU_NR06Z$7~<%mz1piQzh&`@kAR}%j?dJ{L%!JN z_Ps>8f)r47Hu5xX50;kV#qwyy`GHLxqlfe#)&-{g6Aq7Murx=3iB@F+ z5qAgU@|=J{2#zV4z3dE47sA>Q0J9>IOu=npuMd^UPX+)GC}28slPGP)XlRPa=}H8{ zKh&6S1y#PEGN|4yF2CS|aatjWCn?AUdJJI7e|@YbT?e-ILiqopBCbDLM*#vTFYIY1^Ree3%~Ynf{HW(2P+ zm#bnr1Dwt@J_$yHwfA8r(L1m!GG!fA_FU~Q>6oc@)_&Futf~nE45+3z;<|)3=kZ(Q z3E+4fXO1)ObJYAi1z1FZu^&*yAGTc$WXi#2X!1Sm^2E?A665c|Nn14|$BNcJ4qk(D zN;F55YiYTvoX9vwuG@aP|AnWwfFwp)Ata$8hHm?eD!|n4>tdP7YGueugFi7 zKjX3LQ@k}vAO>uv_?$S-6x=5;EROGaArI{YkJaX{X;SoCaQ0M3wS+y=RZ<;ktna(q{ac8a|Q4RMh8k*8votY6>V^NQp@e2b- z<};e3;#C`ktvO-NTGx94whCZ$r<0si?sscc^CK{eO3)WwR7f-YNtnX89%6d44`K0G zK}?s?yfak#yLc*6EM$p)SFZ0V3Tzi{0^B#LgVEi0DA6#PEO^QF=}lAz0icR)Bfs*z zany1|^6AVlxNjz0Wpr{=dC5b-2>7}8)i8kaztUCsP2aC7YIyZ7ne%Gp;6vdFFYQ%T zi3Ffs$@4e3DCY0NjRBG1J5h_Bc|)PEZlZ*^BjB-YbX|vw#fGx<*5ycl`4QXi*wQxf zh|B^0K&;P=&X3?EO9TQ=_SE|NOFxqr1kgbgm6c@`7k{Wwe*9wb6O>t>S zDZjQ z+c4m)U*7hCUWe-*5;r+<+(1^bPMRAA!@(3815kT^SSqsnR2q5B#G;&C20@fXC~$}~ z#-GJ5t?VM}HK^JP76H6eM1Sa@#hd&v8DSu+0K<+i-n=g9Q?K5Zru-hq1uUQ*1B-`w zv~6wKh_-?L+C1=7)uBX?7M?MD>BNHHu2g+5s2Ar3M0TlTzI_&?86sPb93it~DZ36% z=`e=&08A3}9q90zQR^DhS={VS?@d>LARSvE+JsQlLoy&?K|?-kpX*LYx1-eu#W1ft zyZ$_#Y;7Im_w4C(O+uY%%5p0e3O|ZJkK;P%?gR&i7#-%~9~eT&YJD(<>;s)WqXgM- zg?|7~+ozD;M?|v&%jC69Bz*XT%$1fWwl9tXvcw4@F|l<%p90QRb!CLDM(($63ZjTb zHynXAuSWGVe%6lWJHQdXMBNzpXa#Y3LmrZ_wmO3f;K13GxLKEK}_DO2Pk_WP^GS}?(Js!03`vGE;574EvroZv~b{ehE5sb3BU!W zz%UwkO+Wx-DR3i8$+N_k{G8bbiB&?(D_uLqk(-PQ2Q%v1k3ca8xE!-{EkGa#6bi?g zJ;_Q^0SgR3R_~o)%2oa(1ZH?9!KYndfwQDTR$%lts)KlJmHtzOs8i|o_4ksBA4K6> zakvHY{27HY_zAhT1P2;kz|P_Ve30#{nk;F64^oC5!-_%wc*PB?imL5-PXtXVzOPB$6Wgj*RhaIlM@A==j@yBisZV_EbDT)qCzKIPYt2YE_ z6IN08y$+-inG2O?4wWn5#Lg0fPj<$|Y?)C{55 zyybF#B>|Kuv3wYAdDE}2?4BQTro0!hbWRFCrEz&g$T__SSPkHkD*bi61`t#^LWqW7 zs0I#0Bk9aiX{tkloyAH+IA4g+y)nuZR`)@C{!RGpfw0*ENjzRyb1fkBGFqy7gaAt; zUEr)5tnS8OD)=N#%;va-2;R5(_aat7DQ;?3M%d1rbE!1f@M~tr4@Mm0t;Msh-gD*$ zV5xZ9`b_>B*cPPRHenuqi{E^6Lv(b-oPPmWGaxAp?aMaATq)&vS5!<1q}zhB4~xiH z)y4?JL&IF`3!tJ<*ys%z13mMnQcR|36O9J{a&kMgc4!w^++>oFHGgY93Uz@(0vV;{ zvNW2gQQDw3X(0=xX?Rst*}{qh;lh@t9ke%rlf;21EVX^@jPVwx}#Fd?>1xp-`DU$H)lBp@JUxD{0{GSauAR{_` z+#nj`#^EZ#P_Sl4NwKbwo7uNf;c<0W*jt>k_&Mv>?b^@+Sjo$49nA}K!}c&32oj`@ z93DgcV8(}s{P+kbuW1K-8X#$==i%m9wb7cO0mP`TXeq0z;(+PJT#2&X)!6rA=cU1+5qj{NJ=}5iQ^>DZBJti9;=-h+ApyHeg_(u zlrgpjsy(V?FO9dB6&g8|6*2;TFPsiYy849j2?d;5bZ@0hsB8F5Dp7ZGtX$oPT$;3)Yd&qb1`5{b{Z9O0ArCid6XhmeWgc$z_tvOIFDJ@O_&60 z-n6Zl?jIAspNXwt=H@qmb1k9g?Wbd%uE0gI9qan*LD@t-^Bn5Zdla6kMYW^J!RQWyxYh zi$Rn!5h{?iPX1O^K~C5C4J^s!MgdH4cDY+--*#*3X_%CRwGHpyDq3bq^muTg5Gvhs z5)3=siqYMjjqg>tOOelR{ouX&g#o1DHf>GJD_>t-dH)lkyBK> z;Ze4C`E7mrfX{<9pvRy=i!d`*zC>-YYL=cYezTjIdX356yv}&=U3l0P-&+jsXGFAle350 z|6qsh2{{AOU_8{QH_z;N<`E9M4YQJ#jvOF~<4eH696PR0U9Z+h2Ckx=>wlrmADlrJ z36Nd?vX!RzjU+~=qVbK_A;EyLF9yc$;QHb1|3#=s8Va5<<_n1K35=4+2k(MpDv;Cz z^D_~-gcPTLYExJ11qI56!uq;Tk+N zOg za>&zF4dK;7td5cJrw5nvs!C$a3FL%%2=()T5Z`BU^dC7Ebqs=mDWJk`?Z-utM%_7nBNyDK^L($f5m)+Y${M|JUrE*SQvQ?3e%G#t`|Z<_X8szgA1 z3i*KXwITl&29PRlwtwMk$PH=Dwy#6L`n`*6ftE!F91(J{?>?&u4RRf`5E0!@xj!-* z9R5+*a8#<+MW|cS2THJ}QrQwr9@Xrgc5U!<{c8y0JZtp(f!}G_u+|@fLA5^r>kQ;E z+Foo40>8u?%>~5WjHQ0RD$#yEvYbPY98j&)dG~rr_sVZ*NCL81X*TZrzdLURVDUsD zk8ocE(7F3qc^s>sAF-*9Hsrn>@Eavqd2!M1zvX9qY zlLFu+S{VskKKh6d`wndD&gS}@thHqVT`urQKW2rquU4+d;t#y99U%PSC9QE5p zR+b*WMAgy$B}NcQdv0HS<%xT5&(39`;VBC2R)7JNs08)RZUij8Uxs=(L_K(#F}#kb zXNq~>nP5)CcKOTJ($_x;eSCZkD;0D7$=JCxND477SYKJW$@_4!F+RQ_w~sBIV@C-j zd4nPoinEKtZ}K^y-ezb%QF~*_{uKB47I676|818kK9%jOJ4|9xigo_mpwBM43{d6u zJMJie9znv+9T}XyGschWv!XyK78BDmTJ&hlff@v#Kfwg-%a5Zi8+XL}4LFT2562A2 z62R%y;e4`AO)8Ppc{pS2vK5UXp=F}yp7D#VksS_l_9G?^t88lhK zyu^zJ3IjPxpkDirY|m0KatM&x3y=o|dk?45*$F+v;op!>R8|d@M^%V5j%d2b~nDL>t>SsAV$W zRaSxwBn;0vg41#-O5C>QHVm;@#(otP)9JRJ9a2LHe>TA@$7sD+DlIqCx^WKdYe=cX zJwMUfKbu2sDjlxPOZnnx#dQ;kmS)a97`KGP%)+91B-3{OKa$4S5A?olHzQeeV2-Kn zzxy=iXiPO;_z&H9B7G}D6R6M9n#&#DY5|Q2=NO?oM zJ;E6n(N2Q}hBxn}GlTCJRKq;Z(@FB$G+YVBaP zd;juter7?$7tZ_yq?0TXia&FzHufC%7fToB!-xtJTWl;^VUxj-z*(?55Wm!e5>rYb;k-L6&dTlEwAm zQo2P=vZT5bW1;pph}$o066asnRH}@i&;}w5p7T$OAMXA{>o;;~4UdSy2lOb!#oO+v z&;XZ{;c>)TZAm#kue8fTyZ#+db+rT@w^oo=g>iS%r<2v1Oba_qP0sJa!XUp3V3<1p z5f4%5$Icvo8@w^|Z2L`*erp|e`3yO=JOV&RxCh1WB+ z?SNNduCuimJ=Zo}ko{P=Mb;}Qt*dxVq-tYI)n1aX)$ zXO;@o$`S++9I|>&`wW<)5@#&TzaR=n=R$*z7vEUrj5)}`6wGY@myn8)5blTtnT1-T z)p40U@q3}jl5I7_!B!)NoN~}y(O7@Ezqt{mOiA2tj?d7vO78|QL`EShXa&ne5F4)N zQ}z^g%xtTdR3vij6fi+_ZX+1%+zaX1KxWZbr{&fb;u~ga#K{({?F!c7A5ZJ2Wtw;C zOzMlP(-jN~&T%~WH&1ULSi3MioFe!~p)$g@WeD&C&9q@{URb6ONvGyl-e3VpJ0(nx zvWQkp;^H_F0|Q9$U}*Lw)y6hYO0m#(IlgqBa?@&_vR@$lm>AVI9~FZHZXt=VK^=8c zDd=;$l=x36)-D-nyM*8*;;aRhZ;uI&=@K8Zs~WQdZy(|>{u_Fq5G$$}@EK9?SN)*B zQUjJKaHipQnoLqbKPVgg6^mhHb)mWRQ$KU_n-dijl?^=N;9lx<_SO46wU=A1fHPp7 zyELwrBm3|*yic)PwM4PB1uBakm)XELSwZ}&f^5sF1vMZT&~!u47K|@`Sme zDrTrSG{5fd)9kTnbqjrJ7-iKrksCV9iQ2!8jcvrJjEeEQ(0*Oth)ucNZY}1^)P-9d zPo4kmCU!8|DZ+PwE7MRPmbL^j>X1r3z+*n#oli9FXCQPYG3Ym5Nd=I*9u*b5&>wkPchw~cm>ClDT< z2L^^|cH``=_gkUvw@b#WPTEYOiJqnxUIgLFU}cpp80vglx9~FI!4rF-Sy>R#r?<1a zD?3Z%f2XroGqlw#e3LqcgdMn5@)%kI<`XDwzS83&34&maM#CVH*n|rU1d+dO__)9 z;FJ_rBjTr7Z-#!SFKSMnz3n?<-_s7I(;86y^3|lyiZd-UCT}E1cvF?S!YifxUu@wx z&S*&B{vbmfL+J@_Jv$E}g@NK>T7Pjr?+k?#dVIo*N>58=z@1Fo9FEfocC>$_n7G$R zm&BQt11?84Ufp3)_~WP@y_Rhv)FC#aVXGgQHXk3A7;8_=s_m2Y+j7qsT$79$&F86U zn-2)9sa;vWd(IY*ze2OKvm1OCU-U2_z(|ifpXmG6OF=y&qmrLD0l$l0;KD3m294`n6I}TB+{riX}q#<>BLAvDw$x6d>wTinevdxlQ}bVhmq-F6Tc4 zhQ@u~DXAf*O*QKL9i%EwqR9Rf8Gs7qYGSHfuKS`EoSA0+r_BX5Ht0CHgS0E0DYfJK z`lFnliq)6L2oeTdolEO4H3_0-<2G%Qm3JmFMLi@(pHmwD4S>4D-i0xPM6Fuw4=w>7 zXg!bVK2illyxPI^c5LIL;Pwr2iyx_nuE-s!~%VUCcZObSvr9L`7mUdEZ zcVK2^txty|)7k0`9?j&{R>Up13JyX#lNPlhkYwc~=@TqnX$&xp?XNC2f&%tg$UlQs zY8H-U+|3SqlhNB@xU5NID~^mVPbRpyWdek<%}$5V!Y~@O#XtMy-Q-gj!Rz=U~&Xic|As z%`1swTe#-+&nMh6N767HpOx+1$!PtsZ)Lr&r7ATUJ|1p0o-?fZlDH$|nk|Q`O;_K) z7A2F0L-{}~EiD~cJympfXe|e_)@C7X5WETNHC%S%ug}UB-*G>l)M$AA+dbN6g83K} z>*EJ&h|*-rxgXQ*e*LhQ$a{DqpRN=tRiOC3sFubqB~5&=T1`nKDRSUjXN+b4HbNTn z#QZTbYIvV%qFhlvMx~fr5Rw%6p$L~ImFyIQmi10jgN#2Tk!9wsYFp$e=k{!_?$N6O{kg7^ zQ!r`IAl0;zqmZ1!O!teW%w1NYpeAd9Ue?L=J>#!t8DcSj>o3Gt9!kU@{4Y7}YX@`J z!4+JvII0UfX7`i{mG8>lwM7u?*7T_OOnK!Z1V@;kkVS#@q8ZA)3gn}f@WNXitoP-i0z<_&=eGTio`AhFV zOZq~k4bzcb6%EsJ7iXKCu=AX^69C{bKz2q9d}z%Cgk8cdavdOpl9L7PkT%?Z#qb zY4v^cW_3rsVKQuD_;`Hq(I6#t^OBGRRUtkxDmFGJrr$@WF&?4w4yU5^@&g3ehHfaw zc=N-bHEw^@ay=vg_d#G5Od1>8A57Ht3`QaJlXy9`@)`mDxo+Us{|6*pLttFzIqV@u z8q4)qs>kO%LeoHRe;{LPXUA@TfO54zQIMY>_?D^u_7~(XWCBmQa(eX3f8Nw1#HjB9 z3wCMgHQmnP=4kHC7*VRxBu7I`* z@toSi)~q}?nN4^Q&NDy5l=XLZ#*C2D*BR@c`dw3}@!2TtN!;TeQ`rxG8EWxL9s^2U zqAm6T5klF*W1A1ce`l?``eK(_U6?{KX|lB&igwUV=u>n$*>+V;UX21Ec+UGr&leRl z`%D)!h1q_So6M+O>f#aT3}Xnf+$|oXpQxn&IRR;xT7gUIU<=16LvMk@jO0z~O=W-p>y3+||Mvplz200EMAjv3X<>*&PbUO_N+1R^c zCk~&a-5DW@X9V7IJH#olpf#27MiLoxRhpK?zrxj%Z_^*j01CVa3| zDWlijCv;u33GWKy_ML>>yddxK&pfd`zpo~XE>H3m(4ZQzo3oB(SC=C6oO)QOsfo;z9NB3P`6e?09G>6ueJ zPHbxk`4Cb1>rdKq!sPE%x7$kNod2O3 z9jH2j>Yqbz<{eR9^q%e9qi9hq7Bq?x-5N+%?gTcvW|w2x4)2FUnGp}l zw`&aFlOoO4xW`I3Bq!{qraVIYU8;uGEU8;u6^&s`@oW9a{;w#rzWT0-p4-|eF6$bO73+FT)dJ%9pzF)iX2wFJK`~BM+ z$cv_c1gj!#^65XnA%kq-2nI9N%(HU$V>Yln2Om}>L$Bqt`9w)Up}SD72`=ACRi9e! zz6eyzFgR?fD3z}L0FC6}Se12Yea3?j)qij|#%ujF66tISBvkW2qDa3L)7$PRoWvnp zmGIM~ddi?KHp&nXB4P(3yUpxmHYVeu9q7t&>Lqsj9X5H=FeAXdfc}r5KLSsT8B>2GVof5I z4O2#Bzj_+A{(OC;HdtS^!{zghLme|SGrhrJl$HHxxDXNO0NR9~Icq7xS0)2l-RV%^ zp&6^g{>z%gDPn}RZqc+qTC_OC4DWj{tdG94tHVL?t#N2ipkyk_d`O5G=3f)0yAMjR ze_GbMLR>b-r7d;u)-W=osF=>US-LTR+rtMEQSzPKK8O+<42K|0nX&Yeytl&t1rfja zF)*HM(*(i%B%zXH)r1-M(w4>`Im}F}Y{;Z@PwvF~v!ztXrJdp>7Jf2167t~*$xr1j&d*&+^HWVUrAZja zrra+-*y3|axH@~cP_?w9LZ-Wwmg>AYJP)gN$obiDd(sREQ9d*fhJ7ym5Pfz7GgN1Q z7Ag*p&pG11a$L|z&%3=@G;~JT5OAbKAaw-~-P5}r6Vv4esauNDteY!9sGmG`NI3QwCB6DA(fUs6nFoFg0+=hp$1SO%`(wxS|Brvpv@-E2qd@#QmF{-&Z$ccm; zM$XQBxh{B9nmoPSbr_#6*Ws~QdxM4)2cXi@*zK^{ZHZ|UkIm~YGzE!KWwo7OPKd2nA7uSZ_bpXk_sb0^SB0|52P#rA&~akxXy$WLtHd)EI|@ZgNXoc$y(fQ z1pYeU8MXk$GNY~UD@JP_^&BP>$7uO2jtIYfDBs>GbXS=ky&37?`UERw;po5e##PMgcjUq5qu3CSf zJhcmwH#$q6!fNa9?VP3>oX`AgfwC&*VU3_q>D7PKk3}(dCYt#4eir8Y4Epj=oa3Dy z!uf8Yl_XWSbAvgnck?+FZic>X%Z7~p^JmS=f$dP1-!s?@#|Ug+RVIF2rcRovO^|3S zJkq;LVtRy*7tF5u^*KaN(;Ab5TAeYuj&;By#G%@MbvZ(3cTX44nOSXZHdX*8cBSVR!W_v^iT2LQ;;S(=82xV84A<5^x-z&T9Mj*r%CwR zBtTVS=G{*u3M|`--Oi6<@9iS0pJR}>?T<=~&jX?VE5IG3QMgr4Ko6)NDjmX~Q`eh% zWKcSFP7c<5}3I{#iCN5y;s4Lg0^gJ;^ht359F*#Sc{mXv)?( z;yQ-JfejIY`8PsRIQnCRr1i}^V$nYlUkziQE$gIREiq>}pT3HItf}6F`SI^;ju>gM zI!vjA1pR{zsos%-q3?XP>5*4aHp>G$r!?oC&_I`zAlt^dXCP&C6ODmJI-sq1n zr^9ulYAHG@-@fb$;;8J^6;=2U{}R9$#mx0H~=%#HavyNJ$mHBo5gZ$WF#7PC_?hB0}LbA|CP= zhCePIof6uBSWFcm-qvqSG+|04VdxjA2c!e11GH5B%ShBi3hdlK0rM=QPF}b0Vu{whi$Ift8vZjO+C1}UkwP08U zUpcWsXF$xCYid<1W9<=Y>N-*wV$sI;S1K98-<jkk_j0BYCe)FCb_d&2OY$ z9W4GnhNgeu@K%@kfT`3w&tHNWb3*2t(x5Q$=tCbYZg^l!HsnoEmPokv@)l>z@Zy|V zoqzX~l7CBf%2?OgrHou$BrkS%OnLw{8JH+1STL#5kfP?+IM~sM@t)EmvnNo4K2(PI zgGsyZ$qf-Sj=w|70isZL945w}%X@M<3OSk#Y5%#+e0;hfYi6N@Arbxr#XPqW_swh1 zI%3tCkj*CzgsDdVI1ykJRKKYs!L^i{WIm4^z>4YEo7`}dDc zlQw(mzN2>XrTwjh0{3R6Zclp*#A>4r5i;)+u;~SY4lxmwCSz*k0DjO9qPA&GAbnqr zsyYEi`F<1TYTX{#daDi3;pgJQ!rzrLP0}l;w@M=|PUqvmGUEgyj(h<42Be`sJ)e8= z?mkQy;Bh%e(Y4?EE-$Oqo#Z8IZI#xHXG>@DfN?8qJ2SI&ABHl$(I~cdBUY6|?|iYW!xiNh|UZ_Nn?r8} z2Xi_~)D>%K+>wJF)0C?gmBTrlqAz zKuH1Vk`|CIDG`wF6p;A#Jn#3@!x?56=DPN|)?U$jkL&9&d8p7YDW$6M7lG^#3a3~Y zlBWb>3~UR5e`b7fUu*8<)lpQ^$ur*D;4=;5$Q@I!i__+V4vZ*@S7e_}=yjs+KKm$8 zK!{Tmfx%q=27-s!|3x;4iM@Dg$(bDqP;tDWBIYW7P1-{@Sz66wOgT`hk4_=qA}bj0 z9kwOu_va1B3S1$a;-}l-Df`Bz03{y>_IzTs>A;6w`J1@ zw!RU^AXFv%`3(+2m3d3vJ4^m|G7J(WMF!O^MZb5)|~8@&o{l?j|&%jaNpNzQ?G0_KLp3W$!c>(df+mXo$l$$ zeqq5-ve|k&O^O27`}av?%dWkf+ErA-deoiqK7%h)r5?KcAek3ze}H@1Mfo|?>nu)sl7|a>k@@6it@$oloy7atiC8N zZ)Qw!1pAc3Pm$q0eO8i35LflmaWSMlhn|Y|1H19VDyC}VV^~lUC5(m@Jf8*%P;Gmg zt|Zg6<}@)S>B)e!kH>b@!0zh7=!zUszTCaT%d1j@w(w`T65~J3@@re;9Q1ji{Ot>$ z-b3z{vSZa{O!@ip$5je59{D0e5g*`^o*IgP!vVndfv+2gw68vRFgHsOwq?9 zZGE+5eB78GwN+CR;28;A3GU^_TZddU1XGV{AgPNaSMs#r5w01JY^(Dvkwt2&Z@rws zOCRLC`$6NL?VIHeKR0?TOX~sCcfu9LIpe9d)_g`y17NwcGZ!xXsr?+uDebrX`omW< z`ek^{@be||??&=G7w_oin3&pZUDK}O_-~3ceEW#uY{I97xxh@Lr*}nl!o>O>CrKo( zoc7&1wUfesO3KB5#BJSqk6d3A`*5`=ZvKEyu5Y`CPgMf5KivRa{7@|?W8iGf_P8zu zX6{ux>zdlFF3z&G?*Zq`CLlfQng|kJB8iewMvcKnIbGqB&S;Gi`I*G$10r!&)h_(v zBbtf$<8C<5?W9U_F%FP}@YdD~@x}*JJ67rCOc-AL7=EG+KwAbKUxoh}J^u5PgskWz?&zyBZ9n+Mw$_O z9!LK1D-qP<{z&}DErORDXxcqXNAcz=Lo+9W+$zC`;p?gj26v+$Ju0(nXxm3cPTDyQ zOj^DA{SFges&2tUH*}UW8Zr8uxPN?ZvaCj3J??(!4 zZstdv(-1Yk(1kJS4oP(X{_bm5n<-e#tsQ!KLF|q_H)U?KKGk1*3$i8uue=&m}#k`Bqtwj z8{VeH3SRzqALa4yzyBurclJq#)W*C(EX$-Gu zn(lOL=7yq5inaNwaY85PMGWSLiM{Dk`%BW*8UGCpR$`@{tmEiOvf~P4AB+w!<6Y6H zvgS-L_O7dywj%x}OrNsw!jZ>?U)y5KVlYRoIHMTa3>MbDIMRxsR-Jge4UwjdDh4)( z51x5`n7Rp=(lzWgVJP$*^_J>GCAD{CZ&+}=)BVy%=s3l zSP?x3O=MCaFbc~kgqZ(5SK ztYDOB-}FfJ1D-i2tsZ+^PKqQB)TxDSobwE6WO+`rp8~^yd=g<<5&9=rb@(dv7Fi$T z>#do*B>T7(bQLKV=~X>n7k(0vWxspB_qqS4RMMgxwUPj2ts84SD$9x=rLK#GNYRq< z8LllVt+w{}GAUyAl2iS(T^F|>f1jXCB<2>5?aT0vO~C!Kbn#qCHk4 zpLn_zDvYibQny*G0+K(pP`A&#u={NIPtL>$ll;2TTKmh(w-Wc&3v@yY_w;+y<9B#7 zXr}Xwl@!TcHEBOFjIw{Mi_x)-zwS-Q;8TcY>I|wu;|e>^LPh1v<30XxN_owj8>?KgTL+!Suc}OCO||1Le?}ab`E{M2T>$;G zF;h){{qOIsM9<%*Q0%7?N~)^k5f3k0JHZU8to#<1kGi!xDA|q+IM$OFU#-VMy}+X2 z>DkH!QOOYD{{za3W*S`SuP;yeVBY3C!P4mdr@!ImCDUvHo(OoMz`H~Nz9|$pV zd!i|lN6-3eNU=m%pA{f^`1bn4D(X+cL{0Rwj{2$)mFm8|uAJ!v-+R~?gUP)*WubXjdR!H?rS-3A z=kB$$>O|9tzR1{%G$>Nt4WSYjBp_UN`=?U8w&w3%-)xpH-!H3?fNl6>LPmbZ%rh=C zDJgDs76y*8O0W=?h(iA#n6W*QmG_bvo~cMh1T(E1owIJ77V9*c>(ho@)bk5QN!-_d zX1m5YMm)rusnM6`;e&;WrQ{q$Tf>5N4{1rt)xp`^n$066Y&nP@lE9ERE6zB?|V*M}5)R#Jgg0hOKq?aL0#a1(s;Iqv95e{t| z&wv@tbbJ}bdh!8^s1#c2a-w?8_{!5%yR834%sh?zhC0kRe;*=aeKvpeG6c!9-XF+c zzazrfstmOu{C5WG!Md3aoFp$`?gClR{Psy7E}hWVdBr&DbS{!d@*XKNtr%3mu4`2d z(u64O){--`I7{1z6E(F2prc1Rag(7QPX5{ai{0Jb|I=ZG+9RoGrXdR?svtGBVb~h(uh~RyH=&EG)5E0>12kS79Xv(zOBRrYMiyS!bZA zkB9j7KLuZ|#0m|u1RP>OVyvQpfkDZc;sZ**gS#pE9>jOTFe%sIrOZcign?=1GxSzK zx_8&bm7~p<5O`(?R)5C8V07@i-EHS*+^-hj<39aT z;81JuT`*jZo@B4QOtG0+o|Pn4y>Qjw%uO^xzlzC)0gfrop`P@ZPV*l9C~_#d{kmzL!< zpQZab`F3fA2^-#c+xhZ@AbO8{&&83HWqfZ|)w&YD}Iecl6=3L4T?4$iUOdR^uyyaK1KB_8JZrnC%^q>}VtbW&Ow@aSdqm6E?IGfX#&M#p>F^~IT6k3RqJKJ0uwRg?!js(C{REyWRf}a> ziLW}JglNXyD%Ha)=euf)VQx5T-BM%HFThUo*OP6UV(ugKAGLy zS7pVz{`w~i5od0L;_+ex6%sMyPBM3uYR^87SBCMg>m3MtTQ|sR2z-|Q*RLHk>K2(VGRQUkUXITD?Csw;QEgbva;VNOMDqDWmBj?! zVg0y64@8D=1y^P1&9i&)^7er$O@?vraX&;nWwiDyWQ-mk{^QtsL3al?Yzr+IX*z4s zZ!xt-ZMl&k*qMR)00~JzWoo!0ka@JFC3(r;oyj2kHLBEiR^$I%7G1rN243|Mov_+} z%$l5oh%s|-kw3?PzmatnM`+Bfuayg(HN@v?soW%METtf2nWdk=N|^ZUXcdo!_21B^WS)y~fan%sMGrjOx+ ziOWmcJ_X~*Rc8jN2$0$Pl!P*!li3W4C}aU~Yu_a` zg)bsLj6(gumi|~{I`Pp*#>V(WTnjhTvAlcBpPO}PEww7|D@dU&$BSK{d9?pev-`f^ zNMlPLHz8Ll(TiGGS(rkn>}YLZ8#8az3N<7@5pVVR*$6#@3}8s}K=d@}H*T5E&xBc% z;^tlaI^r1GooG6TxaB9%#E?@Ltz6@>ZK=q@L7FX1$B|NmUtw+10=wjQ$Ym1wp8`(@ zSA4FcZ>&67pZzrQ;~V6#zNWdgFKq&qcD{C_^Ahuee=MD|(z zcrsh|_!Y>fU?%6lxlBn%Er((bcj*)*yn7iB!h)<#YKxo2XirM|W!crHtt0oN*c8ms zi=?#zSs}1aGqwff%Ajw0U!=!zFPCv1!^IUflzPK`Hen63WM$haF5Oi4qe}#7>Cl8( ztcjU+&*oXKu&A@J!E@W~lhbrl~RUpX9Y1JiSC}Cj4N)=)v<_+iv!C$`dO7;B8IGQ zw#N3Z*X{LRMc#mXiD;3c@k+-hy|Cp{pg#d1y-XYB6X8s`+9)JGR}(=;CO#`TA_GY?jyWoYk@U5E-!vZoA%f z^ak}X0yJy41qh5>>0iVKUcGxn`ts}SO$ebuj^kqMmR5!hB|=c_E))i3+wrurP7QiH z@Rr}11@5=}{Zvx2vLATMr)lWV)#7t{?_6i)LjC*j@H)(J3%H!;ojMZ&h4lqjqge168FD@gEJ3J#&00jpLP9^Rv^Jr=;^f(`VnPQG6lO3SR?c?W`V+ z-Y_rInyBLX5O!C^k7geLw@>b@aW40hK!!( zSEfzg#n1+%a!J{Kr@3REm zi``?PJW#xQ@l%l05`C(DXr_Ig0PVPFnG(bBWw|@jVubD=F(gTVvz=zpPBBwa%R;t0 zVtpFCVt^CgMO^Xmc40qpzn`asrd+V@LdWkIBOHU?Tu}7N)jB9O;dJY0OXU%6=$2OP zGhxoh%-O$#5@lux@%&o#L=-yI*NJ|*Gf3MfQKrbQfs@gb3j{mOczl~2bRJbWd%SxGzh9cj*O=)*L z_%eCj0r|E*iIZ}IlT84w=w+suvNATPoM=g+tE|Th#}_gkAGy1`t3kCa$kx^U7m*)m zYiqmh&L%}fpxfBmPI~3;^ZoK(Cl5GE$W}5mRD|-Pl24^Czrl6w3*rr_*mCmCp9ZN` z$_5%D+&JbdQ{)dfrXx?&>p1MkwA^%FAZDXzlMGCvm-Y*v<7=KzmG)Y(!gh7zXL;J# zGPTB}-ACUP1w4F$PrKxRsRN&UA9;H{vh5C?uCtGHKkzy;mDs#wrz&7HXmkO84x`K} zY=@PmN4~>ancGK?D(Wuml85`#9&Nqswb@3K%meFBv={$S=wQSfgr8>PlsAR~7Os!S z%Nye%8Wqt?5BJ)CWl^rWh5xtR#mn+uBX(%Luz))dIs*qkm=iH@cxFIUR6JpaRx%zfQSxVyk*AfS__>T4pV2!5$paeV zR_iJzf)usW%JCJ|S^OYK2KPvorll5LLXESEvf?lQ`Cu-iOfB7`2<3@r>h{t3_kMQc ziEZ@`)k-am9{NnaIhD8tK+f;eMQKO|o*dgNyjo*2*Mm9F#Qb?`D#whMRQFRnX=wzn zZpC!GZQs{Ag^%p%(GvIR^504^g+e0&7J7nhAJDVrJBC-m0q9g;ilmMu(oO6sqGj<2 zEoK`E%PQXGerX)ec^{{PSP%O_)�~c4G7V4?-+hq*5W}e#xV#z<;JtY&oLops2UBO6W3!nO;yy~@M9LO7{x!4u!u#w6qg1rxU08?T z6IS=OE?#ti9Ju72QVsO$lhcw>hUtvKP3wYsmLJ2kBPMxoI(KOaQgq~#mA-rB%I$pN zOz3HMLaMkGQq2#T23K~^$)*g!CQL}rv8&&Z7#jwHri(?3pHq{Q=34Yc#+6zVsoC@; zwag{r1zFXipIB+N<3 ztzQ8-(E6xky5UWa37eU=Mp6%4>wf1;VQ)lU@}G=|lJR*&U0(ViIgx|<^L*3rrNj35 zR*jm#9M;*2@AmwI$U6UO*uTSXJ|yqv(4=E<*`x56HHs#yXKt8VB8Tz?>qDeE2L3xD zW!qw6D)WsO6kj)bTY`%hlXYLN^n@4%ha34Nl;Eq&r z#jNEMqFdCGpT3d{=>Rzwk#hM@=L&}_4E5xj^2-MtrD>XeB$6~s^Q39nEoQ`1Kaw|o>PprL4+bA5 zo8uS z`t1p1RF*Df@nx^2w%tZ;JD!K9>G~8>r^*#C$^voN7GzK{?6o)0-!GDEj+}IZn($JR z_8qFAfR0Q2Js$B zwd7lxq+5#c4#`LwPKMgC)0YTCs%BlMDJ;sNpg=?M4npj2@ANcGu*r0upKSRsF@5#u33 zdBLpT2RjhE(zq88c`u#mshX2{jVn27?j4GVM?9bc3In-HxiZZPv0zK&%aS1ze2AGCiE zWQ^kyAEbv-i%?_JLZ3$Q)``}upBd11+%c*Z zn#uiUm=Xyo*NMhduk`UNn4-;k;^d&9W+OUzLnj*k5Ei|7G1PtRAs0Mee!`P^{8;Fg z?{A<})+8Onl1Jz&=VWN`r0*H)ax?%p)T zpUMYhBPU#o%oh{rqqKECXK47Mh+>=nP${;CJIi#~Q#PC`IgRA-Z<-l@tv?Rqhs~9& zQD8`E`#!%Osxf;Psv)!bmD&(h*jo9ngW(+zE-P$UeSPM!_=^Dd!B{YG>drlps9|#m z)XGdd5l%iiIf>A+y;UkPz4Y&abElMp12aWz80_z2y!f*!n?Y>fc6+@KOBk}pl09v1 zXFDJ|rwr2L=Qw--^q0%d(+Gj@fQm2n^zb^+ct${sF^wu|U^ zZN}i=fL_-TWH0E6e`E_$zUfOyikd_$R-v=yc(W)wUXJeBix;c$vdqe^Z(ldi1!H;f z_y0+YkH=|r-%cJ~t1=%@6F<_n%sj2RbUN%NthSwIW^TK30mC+eP==N7+S#0AQF-~^ z^Ip2^c#Y&$Rf=I%ap=wjvI>EBmto#YqPAcET^t@OkiAPQOQMw2v%0#vRU3G|e2QY8WQoIt7j9X>f$cYben{o#XeV1+f_Xa8u%4farLpVa=H z8zT3a9k-hVgdTaABb$c(I)Wwd+eA!Sp(ew?o^+F%{vF?i}a;myH8J^w=%pt z(YU;n^ufyz_x7N?xmtU7bynVc;j$-!lQ}h7ac;sCiUU3>%D}6`SB3R9i>(3(5@MxS zVR^MK9WBD26c~k~6CXs1sbU{Bz+(DrmUpAmyl112zo1d4TJmk+I{bSM)Dmi5(*8^lO#$uQM z;Mr`9DHg?l->iM3@ek?ChCgi>kNlN`KC|+a!9ZLlyZP?d6Y?_4O-W~qkEknFQmpHf zME(+t94Ngf;>$hW|{q7C0tyN6c1bt7iu0B9oAE` zB=)k52^PjARGg}|iHsV5?&K9e|Gu9(+1$(1fGppjGs@W~$hiHgCO%EQ8IY$!ZK$CB zS2BC4u|HUHB!yp3fANRIfKIJeSuTux^l|+tlsafQSz3=@e7JcsKFyKrAx5%&V#Ar= zefL^07Y$s|h|4+7K3zG)aLD-VcjMRY;B&gw&9~nyYG6PCGUA(8+*OP62#e>BRE{t> zU#qIBx*{Y|zhRREOj9gK8jBCT?X7ym3H7^xE9xn$`_&K8GSK_fySX>^Dk6-+=RY7= zQOYz1NFAHoL51mrAfK0^xgMb`QeKQ@G6Wwj6Yi*%+#nkGHvzGl!+I6uQyKGQ80NIy zJA7WjN(>_I>JLsI$v-5W&ldQHnLBR1nM!RXjP0~D>w71E6mW3nU9BpU7vQWl`M1{s zqgPc=?K>=hH-6P{bc;W*``JtmZvj|pZlC6u#!CsX2_>TCYeP~5`J5&^_CKZpsJ4(* z7huuX7L5UwwE{VVY|HrK7jIJJcUc~jNzG7u2>uz^=N^|esC}i0#=y|U7*Y@;wVnrS z>=Iw3WtMkn_U+ZW+g|DX4rD@T7b;}-L%F?z>u@Uxb4p2)ezp)1HC}kJSPCq2S%p$0 z(nZoX-TKRPP~Q%&N^5T>!!7gh{A$CU7bYgLjjYnnyBeLcv}s#UA1vz6IN_sO8Ac32 z$k8&4Fh`bEf}A*~WF}G3jgdw4lj&Sc59@7E>|A|xRtE|>QWN@c=P*^lYCFCwem}9` zzuu2n-1EN@TU;p~*S+ezA%21vgA*a9=1$Kb=K2J$u=%4Bt=2ffz^2{oSCg7mZu9hN z9dn6)_k@>KL~6PSM9i?3WC~ryXl0>r1`-6LK4$z-S`g`X;T{&DA$i(sLWdVqd~3@| zQYDufF+RGUm? zT>VO-Q#2l*U{43wJMze_&?!wVT5RR~er{AiFtu_=9FcS!&9n(D|52%0KdaqFXn zHMay?ww1HX$wRkBe^3~#(u!~SKUu&8Q6wkHF!zwU`Xo4%6QHin5<%JkWJSqT=)Jl) zPUUe00pZ6 zKTTt)q;U{RDd?L)JPH~&U9ZlaYg(cF6HIq3>iEy4RYEhXCT&*1blY&*P*rtgdwMs~ zuNX145xr;-Yyo)zULft-n5}tU_seHndN+NXg8seTr)ioSvsvcO;d5_!IWkfl%i@#=1<^RmPZ1xW;ws&)^0K;?Mx%b(Arze^v zyY<6<_xT1_ON2q#q2*LV(Azhzk?EP)6lY7oFAh z&%P#{HFQ2>vs9SX0^d$Pqk~1usk#qgmS_>bIl(npf+{y6H}u_@!GrSHCmGnty(s&? z-ULPy+8gmgSzPigah9cdo}&!_lF6`&(|nqaaotGnFC5V)INGEr_fjpo0KjIS=!Fe$ zAyX4!Qu2@WO@t>%(32taNaA?9j&h}yk78w@0V9sjF-P?m5#x*g<6XExSR%-FUbl^d zeazXw|KO07;`Cz>D29tp%9}U6zvT98bPsSKD?)inVc;8qc5BT)bB}UqCxwWEg<6q3 zQAKijNpp!ID+87dd~g$)W7U=js=7o^`#)n_MSn^XZ9>`NQQq}i`to7O!Q}%SUV(?1 zY2Jf=K=|glyttNl{%4{*4ar-aDEvG>jC z-1+Wq#qqY#53`>ed(9vdE^@I_;9EZQLfj%%MHCbidOYiy28bC?rFZv)E7&T(oKfNQ zeb6cAX0`g@SBX<|cxmhCG*X+z6R2w7X6Q5NeNsx)xzVp17b3=*%b!1ioW3IL5{(b1 z9+a1}iy|@-M0!57U=R{P<(cfsOic52>S~Lq{}tVC^g7ubY6a_tc#3d0Ef?LGWHY(? zZdRm0XXic4GyCBqw_#^dB<)w7ZU>(W3*r>R-$%3!Hyvw>SB`$zMWP{jWaZ402krNy z^&MQ%&dB6lhO`Q9Xi0I3dA(O0QpZFdMNaqWv9`Ys=Yn|MRX-BCJr?2z@xtw-^Hpn(Zk>bM_EfYM zQx7z@z zP&!h14N^q-#P20jIg(cqEAEf#1q`^>xua{gU*(UElp+~ucmBTXZ&fury6xt+JMk7s zWk9h2J?Q<{~dR-;E;~XxwPNPT*nmx7DtK+Vk z$z&qXH%H3R-#kjM!$l1x67u05wMwMV@(Mzz3jtcF=w)mcUdU6)lc3ytt7=pI|5KJS zgW*mEDN9S5ZI}2t%}>05L!TJ7WQ0gYfI3_2U@Lv{Q|{>e9@U8tIqW0zI{2eP*C@53 zUy5%g(}lxAl4-_*lSKZxc>ch)WO4#yE=YO*qsJ-kfXyemT(m^~EN}0!ij4eo%TMX! zUz(b~_kGln@Fcm~A4(Q3tN zX0|G$au30=<}sR6i*srgl%_{w9HWIkrP0Moex!kdUm(8qGGeeQ`87jE?#<(nr%0X3 zj01Qf)yVS5*jew*$s8!vSNpkn79#Mgg|JlAis*j^7gf9XLjXE$2xKeu%O>3(yHrVk zM?*hOm!3o}NjQoZN&4}H50UwS7TQ4*0=C_~{QxjaNr+@lhIG(X1MOdE^^f;3K& z30E%pWy|zxwfHGB&f={DGZnLweD5D*YDt9o^3mG=3sMp@hzT)c4|AmX&bN`Gl@GW*pU)=3~; z;!kGMzV~fwvgB?(D2`L1G*7qIZfQ91hJ&T;iXF5817EIXOY({ovk}~oq`p2m_*(x| z4-2YJOXGGxa~TyqJxca)FW0nQCp3jXu96l%`!*Bunqn%n%Y_OFach91bP?3q!HL`A z=WgNOb~s8PbwhqIkI%`*G6{~Xlm5A$|&9R>ynqm2n;-1 zXD|NUGs4`5`J9<+u36o9C@xdgI9jy9pFAam_U(^Hn{df!H6%GPR{x8$EzMCiDAYfJ zt+CDZcIw#8?Pqf?`3j%{iIrTeEey9Q3b;!!TF{4=VwwVmo(6g)UmO#}Z|I4xUpX_* z?7H!K!%?OlcIy{~htq90NwA#dfM5G&ktNG=Uc4uYY#Behr|-oC>s6`6*woAiv-LEj zl=*VG%Yz1_+D4k*v;GWs^};mIZlRT8cdl=HX-E>)q3?8Yqnf^dvOpaUq?c%N;GK21 zy(t&0mF$`P0A8O|cjlWX3d!P~|EcX2=2JCnsfndyB* z6_-p3%Gw;8WJhQolQdZOX8(?i0S9eP*}KUhqNv=E{BF*y_C*T!z69=b%kI7xVO>K#+;6`dKO4^D}5GhUz z(dQB+lU0jNZwAl)PPl^r9W%4%+t_7qIGvx(#hZB9`qQ;JHr*&BKWyd<9}pMVz0MzH zJuQyeaUXu#tJHJyI520k{^((1it2apUB8bX#3e1NxuvvQY9aeO&Dy%9Krkr~#FWSo z^-Fx6HRs%f)BEws8d4S0_{4-*lSgVo;1DMQIDs{Haldy9N6$B4yUzl&P#f-85gm#{ zTwmNwgI+8BK0{+hu+HP_;1}w7Jhp;?E`d36cxkE9bHt({Ra|yLnk}oBmiBSw02L_W zu??(~&AfuVj~9ZMvzE@~9~^!!G&q+xQG{AnLc zP@8E~sVT*N`?ejJt6@al;#}u>OFOyed^;Sp*|RF!DzHSp*i14W9UnJb9Sz)GBMZIW zaO2%;+-8Q)77ekWg#4%dw%eP{y{SNQVAkdw+cJf2FNQZxeI^Hs9~nl2fnlW8-xo%9 z*x&sQlfEIWNkqU>fWMCsJMrh4FCpP863duKGoA;m*Ow<|i*Gw&N)v^R4}w*e@ojof zJihIymC+ctI9jvcsn6xfmIO7`xLGxanURpk&aW^SJ9R_k+5mF}nmh|7r-vA!GC5WfhLb-tVg{i?tdJ13qd>Tx6-Wg- zmEXH{i)-{k7kzb{9sRzsT1=sf`$DwS@h|^R$?S=PP%avC@ zZBl9Rz~o61{#^XY*IE6X(NEhS(KH`L=sU`)Km90W`B}RqjZV56LK#f!`u6(2Tgd0P zrwxt1fM4O$u4ds*9hVh*X*8*w!jVf=oGQUkc<{VTjjgLU!}kGe@nX$nf|5U(SV2lO z#cP;A_a@@u=O_+7!3;(CO<~%+dlTKW40#o54qq@}_d82kva;?po?9$Oo_=zOBp>R} zm9w_zbA7jXKsM1|@!C*F?>`ozyyosoWxQ_h;>r46!q-LPRATYD1Q*klC$=t&aSC%@8fbXFR4lb(wM1* zd;KVU+V-yb`ZN1hoC-7~q6&;M2XsxcnLAE-ysfnNguCPUQt`BYXw%?S9ce45Kui_n zO=g&P{KXgZdp8)aP$Vn=aOC&NxsDtj8a4fcB`Gxl&v}DD#u8rF<^SNJIn&_aVh|EG zUodiA{xTOZLd+v{y{UKslH?$Xd}GiGfjakQQOBS_Jzb4y1`pT!rPlir;GrO z3<%x}ULRroz59X3b?yIvaN`(JB4sDSHG+?ipO~DChykfKnGSpStR>=}s;_&C`&3&3 z9y=oBpjvR(CEmz=C7X5oD0OVlz*QR?6&dMjQN``*zmMdfeS{cl@6 zLUsLj3^Z=>Aglt+zudW3)|YsO-z@&kMU`DO$mlq_`@N=F(s2DiI4v;SufiJ_RW*2l z4PM>>*yvhi^F?KajB8u^CyBB5*B3XjT2;@io&N`5`mk$@NXtcAg8a1o1-Og`zia+l zx=PG3GgWb4L;3;O*JO&Iawk&)uy|Ia zp*D9yEUCq>VN0?3=InT@>cs#J66@O4n&qa`>n{PVd00~gt&Ez1*!0+vHK__K19c`3 z(?0daSMdL-PmAZIh;!zmCFbeOS;J-VdL^Mz8~PSEt@8~zxk1Y=u&yNW!LJ+4xolKRz4ZGuPldtTMmU5#`5w8SeS7PRCg_^U3xzj$A= zBw6uOyPk2;t+kPN8U;HO)@Gfv0r8EJPT;iEJ^oYwAF;Z5i+0ns=u7+AgR<5imyS8| z2!JatJH{O+1vxRB>fHBRroN9gu7%@(CX4_wN&0&TQKeqi_&KdtH>@z5yZnmt@29E9 zPVjX(4emvwpD(R$m0`d$k(7UNhkVvu>|HWQ+Lxj@dlNu_mTD(T1$uM#+)F})Moldg z8>&dX{t{BT=v&qF2iG}{buYEO1tTZL8<6>@EcxpyDpfW;XqXxKE#Rj-qt`)Md>u%}}qo+6s%lh?*!0whs|iDxf<4Y-@Fo z^JHU+_?6$){%J zZ8tup8qWz0yq4rydrlFH8!pEF%hr!p3=wbf(yw=9WaP=w(UF+GekPX4e_s5| zkcZNh{6ZJF;~N3y`rr2sJ1|)Olj5Y?N+%;8*h=5&u<=AQeDXmV)VblvC8EJkm8pWU z6k-(tGb%d{&Ksao1?5Ufw8UA-acYrM<3xkCMI-E!ZB?Da+!BnWPwJU(m9!>a@qjc5 zjVk!y@_xtO2`W`EJOV>o0ljJ9DVc*~i9Knl!YI4|;JDEMP6FKM+KDQq<0^@CRS3;B z4^;IR!akLM3{gnz2(>eIj9AFRDmI1TYbh0KQ!)}e1?-2CI?)5+ra&rA_*d~Pbga46 z@lwE3uu~E%4^TutudLW{6^fuPVIYMJZeqL3d&J4nO`eV2X4lK-52mtdv@3}LbOg>k zqLHr2slK`FOdpkMtE~USSAOI^GnEOhnMt;jQ&0Zw9)Fli6r z^lF*72pIL=YINooEvDs8$h?@nhSNWNutfev%3OaHlSBOyGJ^7j?}Y7CDK<&8#GU20 zC=}2mS~dBy-AHL?wni3Uu``H^G4e+l?kMZ)zo78`gN==fBxY=!dq>RQ-@m68L{!)b zbn-aJ?}Q&x3hMD+pNv!Eg#QN-7;KPO;XGrW0;m@pKF$z9Tx^<@l0pwQCOSHW3Z_hdMWWpOC2o|cRg9~N=OayFk*#h(nR<|utb3R6|wu;O6$Qbs6MSbc|h+xQAcMw zw@^M~(te1dTT}h(GVSNO0RsV9ppul!@!x6W?Q|3A1>CK5So?Mv{7osVHT1fo!j?bp z4qCxPmpnHrF5^KwRs4wqE`04Hx&8x{bB3D`Y0z9Z_?0;NAN!=#Y{g=U{8+fnso!iR zMtneswAE$G$SlHt3mDybvy#DlKK!N&z@5p=d#yy5&HgJq1fwAud^Ck}snq55%jnA0 z^}`GZ4niy@_Kh;p=NhV zZvcJrgdj6~vgRkfB(qNmrUe=ODtn^#V}J&UfQhMjvcG6(RWs{o!f>924a&!Itg!OoH0;ey+a03VJ^bpN7MaIrf|(&F<|!l+bSGea>uGzV`2CeSoIY zAx!0ON%eBal63KQ+l9E^gf_?iI{a9@D^rECJ6$Lsg~)z9iisZhZ*CFkNxPt3zQhhi;HwvXhf@+eoo z1_sQaUi4^eI?a4DX@jp*rh!l)E`#_1Uj)_pe8agiAb~vq7-A@kDWj%H>B=)N5og6C zfEG>yUC$kC5!6bnnJo=>X2TW*GC?0byV=64n7MlQu%Euap@|x2>k>A!Cf#YL%NvKh zl}yNy8qR)YPE5Srl%HgG+b=8fCb$8l{6)ft&p#%GJ&# z+%BaW{D&v5d0)S(fuKuG&t=nmtEtHm*P>b*0T@gGZ`^w5M`PKKv;HqhvF-p2{Uoo| znFTKXBp^Jw5dHJl&0{-})ni8sp_+CYrPVak8UTaNSDi;bnF@o1$WGo@U^<%5uyd&z z4DP_bmy3#~U8s-2XRd@-1vB(TN`iE$2jN8n&mXNHHhe17!s^r0nUOc#| zVIOhVj3<0@Nu*>l9`scOA(@UXlfv4{cbBZ9f|HE~lgH#}VP-hPOhEF8h=61l;m z5{PGF4wFzG+kd#J1xkw|C--hf3=8`JW5BapNJc^ynSSPXL4?XLN z+D(~ZVO^qYpr@EzT|qBMG06N$oeWd+G|w5i4r`*U&QPR3u16PMP%k|>ux8G8m{D)HQ_UOwD#$M# z==|~R=bG~DuTYCxg9Q0-DOoj;x9Ikh%0H632(DA2{ahDZDz`5l`~tp(BV09jJZ{|3 z)~NV8oo@Y=GqB3UW=K)|bPHqLdq2b$X$z}8L75yv#MuG@C8Zi`1QHdo_dpb_Qq&(= z2r{y>g)7jLzyJ%JS-t(m_S}H8t-o7G3(!xm7P(*3N^Ot#C#L$#sB1t{$L;a*)lbs2<{d{_=A2f{$%_odvfagL z^R`xwB}YKK%B7af+_-7pdvTY*Q)O}~sTHQwe|i@MDxSR%W^UlmOsuD`|CU%d@TXqq zYv!QDiDTnW$)N{73{I&Cf|+4OrhfBR+8AeNWjL_tFT9`n;7X>;cYT!D5i@j(*mlD? zAq|pq_{-?rKr9AoKq!az-i4#=hBKaP_X@UxZk!Fyw3SwebAC*VruQi2yUU95UUh-a z_}#b7jX~LO6e2{N*4}-zGaTFMxz0WS{W`mLLp2XX52m0lln76J35D-J4vPE=j2-7E zx}3b^VK80QS?|N*^rk~gZ)6qfq(nsV$hmU7d=wp{i-FN1yZ4`nfy7tTC9zvs@RYz;a(Q1W&aa%77kCL;R1;%#3Bp#OKgXxw8&t*cM!0P+s zCqd@j0>la>5SLNESC3LG-i}S}pra&nLjZ`@2@E?Ie~atzc(j1plE3)Xo+-H53}odU zfF|?EdB@+)JbHzom~=CP@0kiAT64WPLIm=d&Nn^>B?V{+IY-F6VFm~oBe-FV&jC{* z%KlrGLwcZNmia9`3+Dr!hnwqTyfr+orHW}P&h&N9UgA$`2&MCrQr?#p`l(ud1q!H_ z)eC(07N}ibYb<8Qb{|fa20w<|ppyW>#*j2hJy5+GGxZ<3XPZCC@J6pXBqZ{aN?v96 z{Z>z#j!j4K5xsh08zWBeK8A6qkx%s76BK@DC`b|;$UPV-9G1uLfj&bcKCk%|UpNIA z^dEg|RV*00J+mK;4T8s9LIM>Eo0ZdH^Yi#$aE6HR^WVFLq9=w~?+LRJD0LiWfmX|wKvg+y*K&tXWdbY4_!>Bo2E&ba% z9K^y*PG_13uoU#YUERzj_P1nIA{lLZsEUz!Vq2B#vY+ro(ON*xG(_r}z`0z@uVz`TfD# zhHipL0)WZAZ6z=K1$FY3n(#d|H@!=836~I3=8JpfMY)=j2iPV zK}3ok*lryzeYPs4rX~)U0*CFAPCRz=nI_fKx=WLk|FJVLj*pMIPiJiYjE?@-9e4t7 zP86)d6IYWG3DSnB7c4<-iRw)bA}p3y(+ezD9%yhihqQwTE*7v9$KUSouJ}7vmIfVq zCyZ0o1tyr5eZq5g`43dKN;De(VWly@j`mebv1AC$G6kEqPCrQY=^yL3lYW$W zR8BxYl}uPqI<7aE+%or(krXOwh$YMG6vh?YIYhp@^0ngA5-PMsVtoA+u@T z^Bk91$T8Zl_U^e+hPWB6!7OfdTV6c=^nIl`%bId%+iKohJ z3V2<~`3`o=z+0Zwr6B8psRH<DmdsDMLTAD!SzI}7j#mD94<$oZa-UpI0c=m6?4-Tx8T&%I12H){Fxn1M28e@gE zB-+wE|7hpoSG$k|^_;U-=?Uy!gg!g=vXPAyW#YVN(t)cPaN1rM*3c<|L zq?K+ezN3ue>JGr(K#0P0iMoi6Z}AtqG<@Yz$s_k)qNJ;BA5bagn+;hOPp14{YOjC9 zDZMgyigd4(&P(w7pSIV82SXd`soB|nX~Qlooa%2spHkt!|Fzl&6E&NcUY&M&!$Iiv z7u$4GRi8~8FuSN5o)`?0eeN<>g{&C_@){1j6-33895qZ;&IetHe#<};oxBSe>^)#Z@r(9UOoT57jaM6ZQ~`rwQEi3N_UOE zyBLVI#n!|q_6wI6M|xQ|L6J_74o7swIsSJpQ_k}17C&{9pFg?3iBD6m)S?sIRp(U? zi(=)9G;k$jx!>TqO+;_uLQIL_Pi^Md+O~-P^W)R9$$X4nr-RpcD0rn5Ki4Q0_c?go zi9`ZkuwV>ZEJ6%LZzlJXfP=wX3u5`5!&h|Oohf|+PgZ0m8Is)#4@yAX4ke@r9Zq`M zPuf-_iKX3Jdwf9O3rT!fl8!x@ZJLgX3#X_+=QQJ>gf9-Xr!-BxDiU*|={mdArM8aS zmN%H6j~)SDMAy2`SqWzQfyN}O`KX`XuSmLI$Rhf}yWMRm$7AE*7kY*ukyOpNp`diC zCOA8@?ygHaVTrZo)BgQxkN$$zTwmb2400=cMRwc@D!ITAr1R}iHb@y6KBUB~MUZ!^ z*4r|jZZ((xetJf37#2&eKrI(#{2fEeLezScvu}VnnIWv2DU$uF*~LF^Kt8HqnrsQ#V?6n%^TUEqcufFP z288mKU3U=vgc#o5ySShIxqq(vRsE&!*;dzknmyF@JsCrP`TG=o_+FXx{l;%X3m;|x^H zAd|e+T>c$~2>XLjtuv3N- zH7$YSv-5}SuR05!F7R(07hrbWdGO#t7M-b~EW!VrK@tnmO;?nVTa=aL^yDl2D**7c ziEMhUW9g&U}Rj}Eg>P*+fw z@g|F`o7S`e-0z?IB`J;o9=vX;AezmQ`&J?s0WK;JIipMuY$9F$BE?78VQ0X(->XE` zt0ZVDAxYZ~l4DQ|Lo1!zjV;507lQUw01Q0nPlUo&etxn>YDEWZ)g8wgocbHm41ovu zN`*I?eji?k^9Bj(Sk0*OKpujFD=!cYOTg(Wy}SQPSH&o>cMT~X(7r)AA4O;jUyTN}fsBR%p{{81+!kYSr!se#0WNHz<++!oQjE-(Zcw zpCrtZ6uXrmb1{|h$*yc13{ADD_glTwnIMH=36Dcn9i-RL>f64hSEvi5i+CK&CY$;M zEu`b$x!3P5iqREH4jH9UNNj)?nv$)_OQx0ak(|FTLE+oC7IbU601Rd6FUbjhUmUMK zYBPx;dF6T*RK5LkGp_O+Y`EF6lkht$Lg}XgIo-F>Na-W>XXVv{2Cx?q`O7FNh=KeH zO-fxx7;f8VzQ5TF&;vz{=Z?DuHNQxOkvoqpHwdPuLgQT^=ry~NQT=-Fr>>i_M}A0N z4|3e<<7dYT4+Xu5BZh?C3Vl9ML)pN(#x}c#ofs1njo?&|hGVARLCPY9GLWD#iq7A!OEUeXq zBNj4&a+oQj5_1ciyRju+;tr>l5|U*_foy7X)5OTs6!!2h+Og$xx^(&SB((=)(fI)B zgU(0V#f0`mYeF>%%7mCcD!R#lMY;5$zbG93f{ShB7=Yi0V>P8GoVuIw1y7YCeJd83 z(bMk{@#1^k{($!gi;wSFnBaDblt`tk%FF#QyXY3EPB~)ud62}b`RkBj+so0iz7r-L z+Fgn;1P|M=6v9$&^OoD3NgtNM{OuW}-?nU24pkU91K3^|T>kCt?O0e?Xit$47gJ{_ zi2=M_wj0Af2s;)O4r+;tmQ?25XD`5Ur)dX1+$8bIJ=N7&*M zEkQv`i(EDt6h#-Z`}*~20(V8PDO;IUV@Q&h^*U>tXO}Ll2}%x<2Q#W-=U&7m%NBHo zn`%&2bTcJgIt2}_;ZxIC^i$n{R5$T*eVw)5@ePWg=NX4L1&tR6dY+6{UsJQNWT{Su8encvegYxuLAJbtr(5?T{kfc^Q& zX+Cfz2HGMMv;-;1C0WvT@({g!6%wxem|Z_LyS<0#R4PKWye@6le_MeSYTH@!Nc@MuNw=Nu@62-R~c8tJ>?VoL{ZGupjcWaw_=n& z>m!jz#zh-AgZZe&_Xo!45AyK%?ZT&#rnQZxWg_y-LTOgqZtVvrABS;emUFwZ!>vK) z3*89JS6P(l16Y0&W$iyq_sOcGgEgVS0xO1BD4r0(sV>B)k2~b9@E;dYm!B>n%41jik zOt^q22Uyx@%#Ws6auVMQ!o94$U}_Vd7%3plONIWwJDFh6sA5^Uw1o3rKzsQJzm^+n z|NHG;F>0_3}POq7j__yVUgEoiQ?Q=XZJe!tfrvsxXGx zE~0gmAs`L=4jHGMxE^!%TCYFE zF2H)YoS@)sY)bRy&-|Zvz`9J`_CHzlBcL3IdeiCHI51No|HM9|$6kZIH1p6eLsBCD zMYzaexh&50BAOH)5t+=ce;^-KyyR_xp1zo??7grRVQk8hMGocgb*cxPLP_K znqzkMf+xMOFa+vFowCW-@qN&aXqPBbn`WhoGzBq5^1SFA%zvw)cuDFSz(7hn#VhYy z`2L0_qtjK+|xT>2KYY-g6P$M3KrOHPFp4bdJtQyFEKs`ln%m`noD*H%KU~a`nw)JVA0zwTF~%+N7~!=Dq8GrG3l!sq-BV7Lqv?+mm&S{tMZvkV201$2#^sH~qjO!c`rf2CCt)4`?1MflWU?ny} zvPlvFw&dmKZ?z9lG-=&v)(L$lNQ(A%NQfi@ijGwNZ^MHVzksp39>?Qfc#b*xMaTLp zyeW4H02b(@qyf?>+%SOz|IsFHqib5iq%2Sm_pfVp_cnkL2-So@gQ8Qh12Xd-3$q!= zsp~s^n_mhd(hgjDcY&+HmyzKv_|ElPRk})12;G$~GR*zs35Jfav0X@IWX%@6bTH>Btj4b~)^EnWKX?tahGuBpz4aR`>duSY?%B@sVY3Fo* zEoj6hpQ-DhkNKRFAR&pON;S>&5Q0r1eC^Nl1NjUB!6VtJgKzgq=zmn7YriP_OhMf{ zUVLj9zHn_T7j1ObdDnls(9nE*1G`t5YCob36G75M zZBx(Yzxb>R+dMua8<0*$wyB#@jwwed@!aDBLPLK)qN@#Yp&YN; zVw1i9plP8>N6(w>%GUDwV-}sS*gD92ZRPCZQ2)NxcK6rVPF*as7E6594h1{=H)1ue zGTnCNa%Fl5ZR<94-alXsB6*Vl;DuN=E4rHx-ytY9XNw;0DBf#lS;g-I04hRUKVdo* z;AEjm4{2=DZs2;0@{XDJbom*Yd|o+u0J54uMt(u+djtqcT4q8w~2LEeI*Ae8rIGj6+O zLoxr(?ixF~Mu2i>C};zGK|L=Bp#ZIiq9DqY%jep zy)EJR-Z>Y?-G6vps?IzMPZP#x90@)Nzq%bUr^#YB~__oMjGan$59iBo+U=@WNj+ zR*O&rNW^g&Bbw=`1iU$(IVPmU(SI62RHJ26>%+)O48ow}iECPfv?qqZ>jO37%>uoR zPbV%KAN5+2wfbA{AHTU7M~<_K3czSHr0z8uRuZ~lB}P)lGQ9H1Id1oF8t!+ z(FGis#z%s|F%KaQX_{$byDI)`UDF1b9RbKdm{l`swZ}qe(AwS%aKG?IBj@5$@#ae% znMt?6NisCcD{XCzY(eLo$ekg6|9zkHm2OG%#j40waL-p=dZd>*BQAaHMoh|arQWao6};QsuV$u9}`q zRlK05pz}>ncqy96T8E^v1xrHj*>+`@_AO4%sMC9cQu#@b0)GX)JcG`*)ScLsQo6SO zir6oLts?t35nAX+dShFrKO}1SJ@W8w~@(Yd5L*JHJit zirmpjJ9TvUXKWi162fxKHGfLza9TESoio|N^C4fIxD6i~T@ zy%K0UVjT|%<e&gC#p?wjz^^vqr>JeF+v&X~sI*-eye|X7)NthP-*Io|nmQ80I zSLL9BlO!MT=iJJK4<;OWcd>$XeEqU%&L-WV?v|LJ1lWrhQ2o_^*HeP^$-Uc?b91^~ zCuW8_|M~GrW4kA+W9+%T{db{2{n%8H#nhGs zb3wpma*^A!fV{=IwNKg`1OnVw+9LTVhGt2LN;ro+g?{7n^nn`|gc|H5F$x#~G;Kll z-HR1fJ&PVjl+x1P)KT7NJ@Y}y(U&iB;UiL_heWoA2@$_?J1I-{f78#wkHFMu+bb)db01;<|7jCZqxV{SyDDZlcBsT>Q?DiVk0?e+TBMZV5n;-2@AjgqYL*%f)S* zer%Os{1@+4j%xfq>S*`Z+Y4}qVOaDg7VRYu;I;)RS@_#h(FU}q#c&!l{>FWx-LYK- zCy7h-U)lW)@|~(u_SHuEx!cZ{ULoff+m7TBeF2Cc>DJq6(CNspI_jLiw)9Y}QNAp! zHr!_}`8zUz5{gXL*2{U=9=<}`p3gpy1Bz<2H4RIT7^X}5eYm3i$@qQ!nr@DZH19f_UQD~9fCYcsWCAl#63szKHC61-3mtcccyYjU&&^g{<$!^MMeGQ1#`%- z9$e~38ZT`#=X+WSC@4-rmTaF>RC7|YwWZ1}-KjrCOlOSqh-WfmiBj?>H+VcMrPv4= zQsxo~-*=2JQ4F%US5ezsF)^n}A(-ANkfFR*HM^I=gOxF~)UxsVE&1KlY~p*W?pOV6 zBp$*~>t=nYSN@o6o_tF$KUWdfFRdm*r3ui^-ca2tYoGQWefu|j9a(U{ie&^hV0d=! zQ;sTGZ2$1&>WDE))^KeEHqs`gL8%Y#bWZ@E%5eh4<16~UivTToY;T>%vue}0L4Pfb z$T398MV?yk3Szz@*k-f#AvbelY-oP8^X2xLx6Z{rwm=?kL!-^-sB&u2jI@;OC+1l? zgWRVNUvLm?!#}kz>e8s&FpaiHh1aB5LOOTIqh9w5Tz@H1rekqZRV=_+6zOx!>zn_I zTqQ*pCT6f+XtOeA{7{7sN3|bmF^46b0)J-u7%PAhV}tHd>M3DR+wnzXm}03@2u6(y zZ`0>XPxL7lb=L>hRM9Wv-_*8wg>jfiV^)oLWRH@XABOc|Q)7uL>c(aYR@sp#m|%|A zMUB723SP2%3qKAoPyS&s4%epHfg!4Hasbovuem|R{sTB2$}aU^RfeQotJ=# zpmN5#hmo5n;`TjQZ2&~tW`289+bz}z-Am=f1XYt@APy+I*PmK z@9n9Q1?5S$Nk#VV>DRq~weyAyNrsMzr)qDoge~(8HS$XourQUgb5g(w`2ZwT@yu3qH}*aIcMS< z^R{}v=O$-}i#0y2nr>*0>oAW`em>>KS6&>>?wmz1B}U{%S*ok2b@V_XYqiXz^OYhc zm&NgHNwbdW7Ei#Avb$K?#698q*;|9kLy~sC%mxS7onV%!w-b3=Pcu&q^**8!*}Y<$ zY~Fgu{+(XvzoS1-krwo^H{@O%`E@UP6dka1E6|GkgT$wSl+x~s9DB< z0#)E`qW3o?pfl)}7Do|Cf$-wiLRw?-27^>n^Zs=O*|s`M_P$xA2Xmzu|PQ zhpCguD~&qAc37lZwQx^wcGn^vD!$0Rs)_Jv^Xq#CiLR)aCp=%1lTUc+S#RH+%$l&j zKU>EwL-&-+#t!j%(3yNVar35$m|;@*C!&T`!SwU-aN~gy8oP+RW=O z>+?_Q3F+ht73d#u?YXy^bA`4VS1{Y(IGZ*VtUgjNF-9q+cw=3r<^6)GWeBfJF5r1* z@aUz(ZMq~x0#BH+71nQSSK@O;wT7HAH447$BzC4lSW4l)+g0rTgnu=1j7%HTVq;=V zF=m{$3RE2ye4L&)z`|0zh_Lh($bFPRQ=e+6{oP>{H)v3^YRO_~(G^>sGOU}_8&A%; zLg8~Yj(lPtt}uXN6jyGfu+WguYZ$TG^~)BmxGdiWk@GPpX2{WOwW!mDwdhx9S11bW zv7pluiE36&B8}$nAPWKOPH^vl0&_m%hcv!8KfT?uShOl z)+tUA^|K;F3HQTgt295!q>G13sxZG7PSLg{3*M@UrbHS`ty*@jG1|nv1w4hkp!sPz z>|{Ne*lnZ-!|(|p)deHfiH8ml4^IZ95v&!ZQGZ+#QFgt@SDR+6UDY%6l}=RG#)tPf zc`N6bEOnG5O0ULAL&@-YUPSK5pVQ zzK-IZM_z$k*Y}AcYH)8uIe1l$b(LGAU}W3SaPYv&BCM zN@Fq%^b~Y{_CY{vbt?Yp4xAT{?E>RjH|&*&+=CORB-_mQU)${T5pql9>0mv4))Xh~ z#@UC$AKS32;LE<-!Lal4P5Ro~?oUbkY6hDv`~5pFOCt9k_U_&CL{aL)a70QTTO=Hh z%WfMfbYD$xt$;oVpX`mxgw$B!sP~eqy*X*@ADmpCGrm}oZN2tM(*0cnY=T(~;;gn4 z71S%a>yAt84s)`R?XVcOx|ZE(CF;n9{m(sD+mEg=nB$;GPBlUV?km1_=t@_~=4*OXW>z;gOdV^!#KI$Qp?@@U?{)bx{YA<;FP!y3i9 zT^LC&2O@5;^0x$K`_SBbsm7LNjWz$5Nhui_KK__54*+UrzP)gzas^b*RiKNeDbiVw zDH)W&64HE|%$1P?1oE-(oZ0vu!NJrHCT^u-&E^i`F;aM3nGjOce+UyY=+&j^a5xvd zF@-Z=UUjzQdx{z1{#-ylmeYXP{k$hxq-|-&^7_lnwYxpD@sYySLZ*qMUI*RD#22AE z0DJO|%yZ_jY-#}Mze}klQ>(8X(_Z4UNxjaUv*oh(A3b_g(^tI-T*-YxPj%Yqis z+$D?eD>+^dyIQVZny~t|bH2~isS_B0+$}zpmzN{-4^jcXJ06x#<3~T`6*+3EP(Q)&GA8!K z;uq9jLvH`vT;IwCMGONkHOxwGK_=#S+V5Sfw)>2UebNk{H)nDSC}i;Eef;A~Z z`i$FGY}X6dEa^OYHb}$7mJZp%`;1x$Qvw|pvZ-=?bo_bKbY5eOtCg{a%fB?I`uuCn ztj)Y^n9y&%f8^L_BJPEPw7i#WB~(Vex*NOyBE(F1uqMpnzAC2|ZBj?-#Y)NLJq#Xf z>pAXJrj8Oeu=&xA?Q3gT5xK)gn_>L`p2tS1Hrch}T@>#P!{d#+)-_Joaq;DEJLvnw zea~}KYrZ|#wSF*l{l$DYWtD2r@EA(hF5Cbt(G^~D?vy&G2cA)@9h+Ab>*E=DyEAte z@cM#@3AT^qo^2o;0tps=uO#eh)p%(`-l89P<$jcFs zzg#>0_@sADoG+c1wOuqOxr;0aN0HUrZRJG$$8V=yF3zcBVtGSmox(ThJ_$7MSby(d zHY}b-dt|1?`=!KtWGH2HfguWMv;=-*@w6$&a})>FxRfn3^EUm8%@&73Wz1SBI&{%Y zqvGj@$CyEFc*B#QxMMrCEAW!C;O|r<{GTIX#RDofmE%1sgxeDWUhrXfh^vl_3B|no zL$Z&@6-9<`$Hc|N*dB`wO)Te0F%r-QFdIt$`PVVKVQJmXq_e3Ea;`c)yxLxy{rP6J=E(@L9{p1s58eobydZltUyw$-024?66O4 zKT0w|&whHi-O|Y4Z}7$V{ca(5K$__Rc^f3)JVx^RdXj#eK2Se^s?z;*>$~H+4>P7+ zNonopm50tK2?pGhiiBv}Y-26qqckf43dv_V8+vYb)rswSa!)9m2T-f_yO<-t@J2qb zEPZ{e;}b!JLcxFJ^I*%9CYH#hRBYvlWOC!5(R}-!b7QZTv&z16MN{-~cCSlOi$~p$ zvt#_uQrO;PipO_=MBu{Ntq zb~O_QlJq_=!o<)eDK?&YPZ%Gq(kYh`)y*P>x8<^$;4`_blQy5VK1d!K+tgXvB<={A z49Br}Q{vz$#nD}Pe8>6 z-@9l<;=v_YY^-%^jvm(c(WIh=so$*bx5=YAu9ftgU1fL=?(bD(FYn(D%mp=XC`E7` zO!!Pw6@L!xkjDvcEPkOT=v=Y=-o5;ft>&@7A`~JIk8^Tq3cf7$;s)p<3(y4cr3(eJ z?miRPXLBxAqCY}vu^oF1cuDlY!L&+~L#Z6<+7Gd`hIjLFpLjXoKI()A6CNW!N*TD7 zCBAv-Qi(dNsWKbV{!QDb$VGd)V@es(BTExqGHvz)dhE`JkK;CHi|P9&HWw?shB(Y_ z^cVfH)Xdzp{g+dOH{zT*61xbAk-c?DS+Xnyr2SrwJW6Sw&gN<~p%|HZ_nk4y@X&E< zbs(3kX=StK7>T?(F)qDA(BjKByaB#QT#*5MbxrNBNsi%V$H`W2#`MD{RXX)fjwV{< zC$nwk`Qwk;7Y)&UD;3V=G)SMZe118F4R4r+sLnRv1+`hROc)(iucHXw*>RNRn%B+= z>U_r`(I5zITr}h0<|&3hrIBqNbb(l+dTPHSDf|ViL__xrvv{!DrN5|;ljpx1{k^R! zkH;9*ZEIgCO&96kAV>26hrE1bvi{`i7?qpY|D`lm27%J^X8GWy!eb`Tb63te znFt0G2+%~#{&kFtG2Lv_W+HZ4CwmVN&)|1IZa>`+rscp{BPMp^Jn?;mLqV|*PKK@@ zt#^WqIK3st*;o__VyD?79jt=q+*G%8eI4v5WEA^h{xngrBzF}OR^NDnB$LsILTsbD z6^g&bb64O;%#h9V6#p)tPvOt_o|W03PS)367~dhFBc?)a!U$ADlgC-Gt7D&7%lzo< z%+;P&>2Q39(vD6j-34Rtu~PnVBx_W3?TkdxQ?s#f=#sw5YNdx|vk z3OX!7O;oZKGF^J^=P}Gj!f=KaIxQ}wLfiAXFtwpi@--H34)vKUy(yFRn^r3vdZiEw zQJ7^52G0bAoG5ssT`ev7Y_q%B-|y0modGpA@UiHOSgBfUxIMvnd!a^ zopCI@#8B50uCJJqoRJTA8{)b4t~5Y@*!e2UwAy8;1KmB9llzo$D=;tDh9t% z)gt244}Yvv6=5PKTL`76dM?eQb_^%eB4upeIxgirR4cA8@*4+BzTLbmtatgJWBk)Ek!3aY#QUoJ(sJ7VSax~x zp>iP|5z68kS^T(fP`-rnOaC-hwcBa+y>mAVZ#L1)2}M(09>sEgpG;$v9>F_@<71#kpS_>%+q7jTFZ0 zcE4$}xZg9lL7HVsPxr^KQ>o{uo|QNRtWrVRI=V%fkVV;h*XBn!fj-tsdH8CDaz_{4 ze2?gp8qy;hH5HpqyCa|Y|M_sg0?qRSN5{WikFz4xMq!$FfBix!W$wU>mH$s|aCSEf zYO}tT5=A;EtP@}AD3<-&yje;2*Lc<(aZP!BLT~fve;F2Zs1`ABW_t{EwGdUYn; z>zhWfK9FBR+52o#$|UXKXD7=L zO=$2ZTY9^4+FUH=wFj7tTvte@#2X2~oJn>c?2E4vLyL#R4u9=8og>nyA!GMD7@I>u zHVy0L79dIXK`&ubI=L-aBz`gf3k1}ka^(I;hXrIiOiK_LOLxcCqF4w7D1qO(bb1NM zI8+c-LpDS#KPu<{!ni?DHLEIhfO0UoxPnmqBqb4>H!djOoJz{Z7SF%mhEww5`rp1m za@o1x0(EYSLOQ#_*&u^Lk6WI-bjo zeTNzI_cv!cCosgau*z&Cl4W6ou6yn^XipF@vnx08{FTjgNXepB_p&_Z04whtlxalKITeH91Ca^ zgg!EB)PGl5#rV@d3dSzaZ*X*#sPO!sVZ@8BJn6C>L-SC>)K-&t7?3MdNbNAA>^dHI zIXR@6E=Q9NGi3v>=4&OI&wg3VULBJpA&YjL$HFVNR6+tn#uvt4^xp}RvEeoAS1ekg zwCUYDdcfi#leRgiRy1dmOQ1J9oZoJ=cJN?IMP(gDL9L2miO>YbJQQ7LxrKAeW-C%G z683e;niPen8iCb?dpap&=J^Eol5%TX9~L^-$Y2sbc>X2u!SE!}IiGYW#a>>>5P6T+ zCh2FIsW0fcc8xiDZKb)a;k3mPRPpQOA|r6Ospbv#$H4rLmHflWKnpQSv{p< z6iGk3mNHdeEgcd(Z(zj|ut_&rtf>6=!3Nu_DI?ZefGn|VXmT~LBa)pkDMER>oa(oI zc_o@ZB|2dA#}|JN6&aCY@JFqnjJ-UqPMp|bb8Qw3R#jK=UN~fv-{R&HF*xuY4}vHh zhzvVFDjPs7XUHUGmZuFshlW7*=c_lw!Bu3l8f$O>(&PzJw>THh0iIiKTs2#CU@zA+ znjLSB-q|i6ksvgZyG?|Sk0$a-hD;Y$@c(bA{_ZXyN65 zc%$`COmD4Gc@}6kZ+MnO3Z< zIzjQIFo=u3U0>l7ibHFAh8P#y4pqtdevs2U;AKlnl3w+@&gW*&>)qffl_l{!u8P%D zZot5>G)~lXxjoc7nVxLXSDTVtl&n`?-8qf^*Y^CO&Bg82572Wk!$uK@QN0&o(gO>> z(G$LT(xQeF0;ol(5m9My$AFvhT7`t46ixHuXk*51+AP6t$_Q~Dv9PlrrjuOzDV6E( zgZV?NY8o*Og$av>Werp08XMHETDO#6Itq{E=BeCGVM1<&T^Z~B4^jy?s z_SJra=;xKKE4IbjqXZzPec}7;zrQ3TjJm@1FLCmGcKhwDYq$^RuD=^3I|aRxHP?Yo z?`kPMwE7~Tr+TAFMb+4NbMROE=1#j*StLhW_`LrmoYO%01N`#agY3 zW%?*UH@yFvgVhJE4da!XcfK@CIVj#BF}=bN)h@~3LkcGCDJ2`LVZ`IT>-AeMogJ@v z`cHPC7_6tv-ZOd>uR4{>n;=zHvgj!7^)X-tzZktZ9yU^miu}Z#k31Ui4r<-?Mw7;2Fk&VmF;)!fuIz znVtLL5wAf-^4zbl@zqrQ(0W2`HH$s{x#9F^OFp=#ux5}}KH6NIVgNVbCv)6VHO}L? zb*qs<%`Enm{*1#-#UpveEOPD}pHn5@MBh;b+^LqyHl>K?f`fn-Ol%A6UE(ER;^W)}LZ;aZ+AIYij z)p%GH2#r8lW2l8q;!Awp!eqL3i>{-Lsxwy0lj4Vg)79_M%8<%)`ifZPp^a0PW`#07 ztMQ*x@+2o$Sp124qail_pq717G#=Qx@OP|kel^CQ&HF2KQDGDu`^Enaoo?Xk!9uCS zbs(D4+B4XwjJ3HsNl2XrQ<9&Q=Bd@_sU)qD3DqAGv ztn9kO_sDM6D&gi7kp&H@gB-9c+t}DxuyDU5L?m6Y&ssi*)pF55QspKhQ=!ez#T^R;U3{Hv8tXK^9p zFLN$4$r8oIP}!Td2TeEFJ}ybJ5}ZCH*Xu%bYL!7yp>3j{%nOKRA=<1{PuSigmT@Cf zxsPA5M(>-~?No*do$Qr?-KNxbLk1jF0He{!7`QcCwW5qN_x7PabAf>JI2X=k zE>qVQR~YL{(}>;GeD~fZ>rGD+Qlj+9C8{XlA-UkXvBe*1Uf#WPU#UKhBiFMwbB6W* zwHTVyoxW11!Ol=luZsM@;X5$>k?r&AKNf|@lT*W{ez}GIsjO95Xr7jIy!Jx0Lk<}6 zeYxXLkRE+b_r#WGtrj(DDNAow|AMEJEv{N9=l61Wl-k_Kkh}eR$e8i9p_b+B-H({B zOcergBGs@j>-&hwOYYCsOkGa<9Gpw`ASKJY_;zU%rxZ^&QQEO7jxM zwDeISca8vosD}cy<9?LGFc_jx8rJUAQ^NHxz!9G@Kd2VTEUQ*Sy-sKux=&m+LP~n6 z5_j|fSRQxLcjiiz>E*C|EXAB9{t)fi)ddSy$!b(f)k!w`#YMc264ol4I(iSs2Dkak zYc263$qaZBRtbExb2i5m@Q0-_7Y}Xi4Zp;~QK^vkAbP}OdgUiOnxH4|5}esTx|Mep z=QO?Tl0O%{LQyU-2^7ebkuILf*Dd92J^oL;KI!DN$!4(4e;crX8j2cI%hTyuSq}&2 ziV&IA@6Z0;gX@_~zb~#G!q-}ZX^3vY>bLwuRzfS@tg?B>Ze(ms5aubfv$Kfjsi1Ny z05Nxfis>69mnMB6)ii8^w6?NVRIYVTh7aUnf1yHJ0zmeaa;LJhdw#(pF5kbdm&v3( z%;2zN-zI$GGhtHHPz^+KrE1HC@wcd$>4v6n|5rpk_}^T*IMrd(P%A?#`Q2N|&hcXW zvhi#5uMDSl+f!CgCO>(X#;rB{z2|%V47TVJD##NIdXa!keq6 zb+)}ph*nVhlu2|~bD&>iaFs}0sV}vzh6y+ z5g$C|A&-3XEX?VD5|=)GrD{Hbs8s04{<#*^zs}M|3-UpmIR*=(?(?lNI^FWU=ZL(- zo1-iqtBE1p!{PKS=5xQeg#sv`>KQ%SQHfDP@K0Grc z#ul%4b9H`ayYX^YIJ^yF=8r5uHq2M7-$#I8D!JWW&f8q=cLE(04-knhP;!a9##@fj z3DE-H@JDiV$QJ1OlOubM$d`xbxdSkb%?x$hExZq(w*h|q=!icexCf@x-_Sg1i|WH6 zL4-KIuLJg9z&}!K#K$jo zo7T*!#NP%%Q&|c*y>7RiudYrOdp5SWuaFFuc(SUhPH7p-CoGxx5KE2VUMS`fW!A>d z&S;f*EVB?c~s-a7udDsq@DFxW=1Bdp<*#_cn{{1f`nn2+VG{@z^+ zIQ43v*15K;EMY(!>%{PD!<(8E)S&)kB5EUl$Aa*WK}09DL}!FiK2EG06+879M_6~t z6JC|XOZKqtJDyig$KNkonN&v&Hx$W((X^$^7BwmQ2m9INa@ZgC3Fh-lsr;yp;4FO!PC%Vg=Ajqf!XsDx+tMkG(=cq!d z{$8+tuA@SFHggYKK&6dr@Z;Y&wE5jra*>Kl1(lScne|L|G~TZZHJq=T;$&-aO4QvY zjrq#FXU+NL<*d(JkW2zzjwsDJI=@e^!fS`e=F(@)^k7HYd0@))4gI2Fn;$yDz$KQz z5e9vR6+qYy5VDmrhDIj4*)N|#b%OHYee&(_@9h^M9G-A|5f8R4z~S|b>^|hhP`Sec z_utv{{7nZU#tQKba&;yxnywq8O583svVQ&IW`M7Mpo&A;?$@sUi0q$JgB%TZ$h#B8 zSs&AbW1R}+S2%%kG?9B5JzZ^W%a=Y%Bd}w?Dl zvBuVa6H_qOeuw!@gV&~23bE;bGd|b*dY}-FvKBY~{_#_i`7cz-0?7ujo7)Dgt!~#R z_Y+zD!iKPW-_$CbOIRlBVmYp!%xzz4bX=es_!$VXSq>{&);CtJ4Y)sZHH#tXUe@YIfyi$>bnP^c&_$ z4#a8lhzx4x`QfA61X@8j+=JIf4r&i9;8ouaOaKTkF*?O{_OGFFq+&xUmyi7FE8 zp6~^TdD?Kx=;%=L^YcRk344lE`v5}+s(u`*88-}g6_ETq4pN!}?$d)zWdInKaD(?P^AlWMDWmXb^Nq*Yw+iMsW>x{?emivcjFoMpW2VD6fqpLIDWz^ySAj)l8jxV;7l75#*uitN$ zN>>M8BfpI{X;F-g5RkrPR&#%fd+B0&^a?w?Xg-z3=4oQTMiEyDpo>7g#sJtiQ#9Ie zwF&`YkT@r|tuTJ6H^bkP5>|;!nb5%GH<3*026wF^>GT)^&HOP@nenuuX|)|6|Lpgs z<9?V`>VKByN%mcmLvle{I}{ZduveS8UZ*)cPUZ%CH!)iAo+seW5C_ihafUORgbi1l z6<20CSH0GDqK|DbDxI_3g|RRcwK^%!#xi)~F^xm{StdP3p}{*Z5gocX(X2MCQ8UJB z3r@Qw3|Top78D@V@l5sImNhMPMR=#EN)4-l-dUtDj34mdqiig-^$?DRSb5-ksVm-B_^W-M_Ut zBjkNJLX@M5`A?Nwb?Z(>Qx4LC1{&132)^MU~M3?&Snpv(K+%F;&&*ZJs3Q(TjIamRl z{yWCU@=jR+omB12zVY>^75-1h`7$j+)2~j|kRtWkQrM~31P)H{Muyb zM0pe1jEW0$ujt~^M;a1bDB2(XN_Kup+)Nn?7+ zG4HrtP!fSSarO3CCh*{HQQBZy zxH^AFNI5{1<*)|oGV;F3q8$%HhQT_C4%pk?4R+H?NVVq%-n71R@aYLD^p56vmA=Sz z_opvv%%w<}dE~1Pdvz_PW~bci6I<4>uIZ7atZ7eAbVp3ly*mq)b2aunyB+rr9@hvM zQ=n~feXHE7kCD2_kA$qCq^xzdiP4%pz(?!sL0WDP_ZwNfDB{(+OxgL-<9@c;4jo$? zq&^%kM#_`GNUFYgkQ{JbQ`_6h@~Ek@eu*yK*HJne2y9?C7H%sGsQc zi7AwKNyA`PTk2!++P*!HRg7*>1JX8!gFRq$;RHIn~RDu>pm( zG5jHPVBk;xjV7V5p(B9j|6s-C^#A)E-3>-o+5iBCS|e1`$5$+AzR3)ZFQ>4Q=t>!r z1tzcNZM*||u*Vk_BMlGqm#58z!dv|gDlPd0gkE3CL+L)zItUBk2Ed{)OXfDIg;Vn- z2lhEz9&}%Q_IFdD<`!~psLZ2h2sayYtR>^e4b=@dB%NmQ#$DVK_rk3Frk5$hmln_i z@cR}F#QUY5^=UqKl_E4z2&Bgw_tbyxx4HYD!KS z6%P;xud+C)s02KGS@j0wj>5NGXXq?O4w8RokMhU#m{@i4K6GJA&OJ6iG7n<*#w}4b zR~yYhZ!4mGyjf|?v-eBK7GyJVatqtHbV|%>OG~4mJR$dnbqVDKt@J=b^#LTejpV)R zf|a9JK8$O`%$liJS}H1+5Qv6+4mNgdzX#M_F@RYfpyyjED(F5UIa=kRNGB%_%1uLs z3q6hKyO16S$+-yd1A{MF_HbaUUXnV~Wmh{F%riSfQO|G2bssKaBN_$HCjimzs+gLZ zQd5LQb}czP)^%RkV&}qIm~P<5+WFI>o=8XTSXI_I;Pl`2jV(j-SzWAyKP`Za8Cq>C z^S;uLnX$hIJzS`A?tk1hwra7P)4#T>gAyj?^! z?6x9A1;p?`aR-=6o<8ciRD>9nQTuO;9u}`$p#K2g8L_zjVbaTxvLW^q*zV={Jj_3K|%w12?lmZm!y(c%JekWxyz&{y0 zk!AIkAVxPU4tY)fxc{lBxa{{!o*d+R zbR)@5aP6j9@>Ad-IcktyeA?5F8E5fzQ0`d>av`S|vJf;!yD|dEw@^U(Lc#8v==PAj zLD!Gsxp6k}2YF)VM!mu!+0q}jG~tW6C)R`G>r1}G-;8^vDr+9w)2YHD$G4LS_ucpP z4=1ls+#QZ}$40cI3fg;~DX^&7xVFFGfu;yK0l=?cH*on6iQi4ohvZ>of8QzQc;e;$ zLxAXToFIWPgs&*y4;`Nl$;hyQ$$JkrAJ`1aa^0oKN6$O3EuG-hN5-|#$u+hL0sx4W zRK{`;X8acxw+u(QKAv@?pRxr09(yAvztkT@n`D2Ji71A1`@;)%P(VrJ30S8r?-FSK zvSGt(UU1Sa)(G%f7_%VKmeW5k88LSM#C<<-n~^ofKvYz;^J z?!dT&HI;}l2c8E0%@F%BvS27LkD`CFyJ(B)p--%@GmPdb^2Ui0HqF_{rat-1KzK26 zaPvNBp!xDX@-}e5_y(eF)Hp)xNXeiVWg4aq&kmf|KiUT({t&ZAN2oMoIjYdL!CwH$ zGj~&XnP`YAa`abL_KPxw>-ke`@Z-AO3vSy{^Eo&*A0i{SahAk?WDsP5D-Y$zKx1Tj z3#5zX*+fsji72YLg~Gp_VtKd(#vXocd78A!qBzp&8zPF(*vR>*;=hwJX3Pv$T^OGo zkO67}JSn(W@4W6X5n)<=xt^Z#(T4qF8Uxj6o6zt_3Mk+{8G$=q8z0Nhv^o$5SK^@y zQxcCd89z%?VM_x?-uoH&?I?|t1CF1_s@VY_r(>Ycu|pLiLqwT~^ha_uu9UfJ*M7ut zF9bhbT_?@xitJs1I8StFr@(}2T`(QXVO{~-FHRts1pp-EgpY?Q5dree?^e@F>r_qb zQ(hhW{oY1y?45;nMAC32*$#Qdag{Ko*8klB4;CWJopC#QWuRd+oTUQE-th2zX5(I2c!hZ^_$)1A@X zcfHsk%=(-#AvfpM*SyGx4g)3rnM2}J6Pb|1+lT9?gnMV(BIv}51U||P2RgsmFO8XA zrOcZY6a(5fW}0rRQ!$%^Lr_LS_c%wt+0%xTbrl<3F4n;P$G9@UH;d_l7mtuFCf7F4 zhUQy?G=Sv*Gl)*72}{=@<@H*oO7v2OTW-fe3a($evz-vBzkt!j_vnl$9r| z^M@x>SG?xVEaD@!6Dr&Wy~4SpvvJqG?s3!f1Q^T7-Zs$bFiMi!-axL# zH=h7fLUKqoUEPi9cX%m1_NnXZHGZV0`*m}Yee4|-snmuQp~y}YQZlk}yDsK&=uYmI zjZ0mg?ms<0&bY5pSmo!y+f|VC%Ik(cN?5yWrMB=zsK;eIXv`tlqxVbnqio%}7qK9~ zPQT?NA_Hg?47970R`nN$`6av-FLj%|(Giymk`sQuDk=EHzfIFkr@WE*E--G@@xGRz z(m6bcF9uexf$GSN*{b3-))r2?iKXd?zMXlyhK@zvuti!=Wlq44*o-A7UTFGTB20Z9 zO7LV;3p8@!GYu%uPDQ*jLF*jVFnOM|uVXn`=rCC@Ag?oK-M?eir^iP`bl86O}?%vQ*Asnu1jA$Iz!q@d1 z8vbipFd1%$9yWR0EX;g4VHOTHy~u8C?k=>p2WxxaV)^?XZJp}?o`!ww(}S30iE{S~ z?H+88kL#f4tca%_yvsXz0dTusK#6<8yX$v&&zZfG(Y?kfgIi1vP44sog0G@znvK72 zZZ#R3EdZ2hv8JijKIqjuiP_W3XosifL2>1^Au`%GVe;Q*l@JL~q8cOXhB%uR_jTwa zI*C@d{^HA;^v|`eXcAO+2X>pKf$%h@M+vl2XTsU{a+$TI4DCKMMzQ9g%?$hz{%^iB zSXXW!h8%R@!)BmMSg_+p(XD9n(F-u(^(wCnH(+QkS-JezS|c4kT(yn%TqlRf&Y4Qs zzjaV=UA@V9oA0ksTj;&65iRSG-j!y(!qxttgX!`sRU3H=ppXGY$Tc=Ve&zTNlo7&M zn`QGdK#Ydmp-&hNW9{uT*oLyOmeLzRZ4#j0Z2kIJAC6$0V;C%vv@vezDUg{_%cAfm zoR8@&IrRY4`?{MI&gm@fbDkV!2DB~NjXNMl0}DKm@O7%CfgwYkiBpO_@3HMMA;4C8 z{@I5-5(UPitS($Olsio@?wvk(R+`xGb1^QpOL%6hK5kPhENPkp6$L#epgG?Fn8tyn znNT;s`1ZKkmv;y4=%2pJ({Jj-EbMIC#-`9Bd;d1ERm<(|Kv~Q^wTehFnjK4-&bLGw zzUbAiCo44<^#+vXFXvOGmseu8@#WGZ@4cN0hs#zUr*-j}Kk%ziNsw!HO!V;y(c0?j zoHxGQ_uBVQ^D1`DYPv&UG>Lqk2F_e?N&=i08yBZhX>?D&d?pa!ZD|j-UroJV=J#C2 zO{dgR%1C&)b1g-sbn3b!mSCgO+=uJ8+0M)Md)bP)p^>b6IR$ki-nQ-TDNgg-X-8rt z?C1P=hs?!5hZ2+k7BJ(aDm|*Is@jibN;1(7s?Nm^obO~WV!BKI`M3Ii9}cUI*WS?H z_QYJ078{( zfI^^36n1K8(0*(AZ9)~*`QMvT^#V-;QEcN)?mM&^Kho;TY{dIh@`drM4W)l1(|fJ! zGz%FxO2)|t`N$csnH}Mfs_D=N((3y2;;1Yn-I?3tGz-GBC==dG>h+-VEKd5Uo#fv; z>8-q(LPv6>)_VMANWGXR{HMw2_$F;aCkQ04JZ)*ei767iLIL}Q3Yramnw>_FzV+do za*cq8gY09QV)&h`1i_#)&-~TE=d9ghwph08o5vgopZ3Vk$wS4phq}vaBhoJ$6Hi;N{W}l1MN*&eD65EzG%Z=`L;@=by)4;MAsY&f;Ap5v3hiZsyAeIJH56x z0h!=ISrm`(XO!o2Xb5Kg`S}9~ed>WO6MEOw&<;=f0g^E zD+tJs_@=9;ltLu`3P0qmII)oKB2PE*!Z%*N+TNTzI1%-5Z5xeui{=$T$Fci_r0*mLPz!UTMELaTJ5jm7x-&f=RR<3tuYoSeqMwwJMygGR&mIzK9W z&I?tY&ehZ@FJJy+VY<6&HLX-BQ_{n64+Qxtx)~eV_VZ5VXO7_|{R>&4wI*MU`^fW> z8ImU~RB3FP2Y91;23BohlJn6+Pb`;?Yd$c$p8iBbS>?eOMk?>`fD(Xjs{Wh*3?d#L zw*l49x;dtrkEurVBLc8i-h95a`(1N*!ZD9G>lvzElAlA#ZBjKbjb}93Tn!P?feTC1 zt=l`fw5n=GxIHsu)U5P724VTjAoDFuaYwGV-MMj6Y1mB;bqYhhf?y!)S;7tjUM#Tn zSGGgeEGn%1H;HE1EB@c1|lGKxx^ZJ&p8)>@s zeP|rsmMF29A2e>XzsxQM#ggd($A;D4JSR)nn5@X^lv1C-WZW02*x?fgD-TqN-N8_? zG5lb3g8CyU50aE=;#f%z6_uleXxl#ZjF{z=hk+9I1LTSB{Ce7J)1ez#r@#)U^jCIM zBm;L?jm;X8AIRUWCRqSt1=@VD10l(#eiZNM5!wFtivl;;wNnJ5uG(mD9d31jz_tNA z@$!}0BQH6TJyD1_S&nCk0NF7P8*iKVkERf)mE)~dlET>B0?-*aD3Hss&?kzs)Gl~g zc=;8UEW=rj@&;1Vu6ERkm^E9ptZN}0Xc6d(-|%BD#@m;ywm~q%9uOb{V`JR($%S+@ z@A=4Huj=h?ztp0A^FJn?5d1rB6rP=QO{dAWU*2n}(yjb;nq&37a58xo-5qyG{psSf z#s(|`)HgRy+|s%3xyC=Lq&(nkVF^DY`X@4k8!;qW^jx* zwcQ)zY6$Z<*Qf9}w+-c4CK0kv8y;n0R%4)5lISm0W{J$JqBDo!1iO=DmYLy~*Hz51&>Sijigy>-obcP>M)|q$L}qU0!?t!X3cgSl*S{x~bY8+qnc|jSP!5 zZk#Ya6aPlQAjr%DlfP&Er!$-|u9o|lJYys}H=k8IdwfcrI4*~t;kSuMn2wxbkxF|; z``l?Rx^=%8#*0#4t4gKJkx7m~t4@dLz~nhw^TN5cEPQ1Z6$?(rEi=pMJSV|$bvp%e z+d2iBNM;2FH56gV>@X9@t9RhdmP^nh55)f<=`1Y*3Y zEZkGqB}{R=%wXy#d{Lbd-NgHa!&Bv@C+pZ^CIvX$iRO-8lNVk$9TpC(1~-(`pSl2G z$8IXOH1kGGM?n#P+#Do{AYuN(y59ToQX!RL{)I9FMyImsTO0GzijXZ!65MDlt=R0| z_T_{1h<0`^S4l?bV^mboTDzdF058>j@)v!pPT)=1Z{@%dIL3|4&Znj$gHX5pbUh>t zxCTf*nCBkC;iH9xB@YloEk*ZhuIjkR*15otj;Q$UOXeWC z4IZ6tP)Xb{0kN*x+3auF=Vk!wYQV1xyd(hoB+$BjVKLHy)N%w^1V+R|fk8r+{n9%B z=ty?XfwyyI@5kU+%i?>HP5wDSm)gs`mn`4}#FgQ^!>K&ze3eA3m_Zu@#BP?tzMCZb zW@pc)U|P=d*BN{4fO+2d5u(FC0|G(aYqbdqR*w}NA6B>ZC7uNzdE$_SR@$FQm++J? zhLmzW&9NmuBzq_D;6czVPvBkt(eOup*k{!VvII?5(M*@U#4C_CfP7hy=mfph$N5dR z0DfQtzzmT02uj8YW9j4i6T}q!P>L=RxN!a&Kp*~h%pq|uEsAdTh>)#f+6obqApS)} zE(<~+QU2wpBmqwyYHzJ5u8GGVIxTmam8>c&B})27DFn>2Ohv-+EYUwx zpYkv(Rr2~OsHn@OKJkjL-bp1A)U>~~u|gU;FfkC^ZI;`Hw!$%L^@@2$llJJ8!=aS7 zM@HKjt=BFZ=@Ts7F@OlT zs*y#4`}Yu)rh^DG#Ok5bq_1Gx5P2;qNWp8IJ5oIC zIj9RBM-#^%*`>wiM-m|hyrSZY^j`?w;^8r}A=i5jg=i)0Q&ivA{1bkOyf0J+KKMy$ zii!@~Z-+5#{TraA@9$BJK&|Y9KptK#H&Bu&tMo0}BEi7}{r!!T+cjO(eiT1ho^2I0 z^019_4~cczIM1O8YDE9TO#nNAyZZDnQjXVD;k}AVr@@mChjkhB6Jb%<*w|i&7it{_ zPrxA?YQfnB@~&TiZAl`^v5T)L5JK0VB1ty!%5hkQAV_@5ipx0O78L~?JBz?G&4KZ4 z8IG$T(E1)8I>EYHOk$Sv@{NF8o%FGVodT%#$RhB-R+{>YO=}p>NQUH8RQNR~2{`KA z>*SLQv$B3AJo^IKv0tHnd;pO?K86q0*|DntFE8&H{Me?Jmg3QdOhiGK9*=e?M#^^D zHj_VCShHJ1(n{wO=4EAFTBm&#ikfb-BpZ`g`AMI3c&dosK-;PQ=*3kviI|;()p6m$ zWYmI`(<~}wQ2PS%k$2_`rIog*T#80SZ}8&wyeQ8<{P^*)2tDozvL0xr#)V0%EchSY zIdY|gb;Abd-gA)2-Me?s!NH+#W=O2vD6W5Cz_V+qY02)l1Mh(c*)mr+O8yaMST1yA zA|JJ%NY^P*C-%OiC+Jw&gX(8jNj`3&dX=xtpooF`sB#+Gbx6=D%iXHCs|?P{x<#$9 z=i)sb+NyV#eI$xeZ5P;)!NEaMd;6#GY>>!|2<8uGJvh zShGxPuKH%5yr0emXXQHt(4&Yutr3l_A-H@F>5x5M<*gJj4cOeTZ{raj}@Si_b; zMYn~ICy~aFVPk?=k^UQUl3rVE%wHJ6b4S}SJK#NSG+dTeQU zLR=9;;Ov&3`zX`3e+pF{x!F`TM(p2Gputq}O74&2o#@i9o9`Yg-AbzVH4hyWWAL&F z2Q^kKoA%1oaGMGc;w0;T4pE!H?FJaaJ69X8X*v680mO_afzzrNAX1o$nO^&`QE@zW zkK1e7YWU_Tge5@Ui6d?K0A(Psi`0pu^U?=6(lGV~5Ku25{MZf_#8L z&0OluS)o_vmfDX%{UFLasZM4Q2V^1X3d7fk9og{O``<`LKX|3AlL(p#P7loACY}=Ikzx!X z&HW#4KqWlobJ3O)3Fk)anUId>Vs}Cln6)0nNmEFB8$_>1AN;qScugFt9fUnDZc=37 z0t?~mPzL&W)XEd^3N2<;eCs6!tt|VPWfeVkL7t#YoQ^%`!>rz6k)(#DFFHM&9;Z4i zd{?d%An(TR`PkW}4}Lck_#|3*g-vccK>!s*i+y?hE@kq@wutN*G~Y-f z!ZWAup>li_v)|vGs(>!%G;9?hH)9yJSD;ERS1g3;@91?CzO?5Ubp|G@5=irF$B9=b zl$x5TGhdJo7k?nx+;^|r^I(Kqfgk9yTzmnT{I3AUY30kGgXg3OQUX>5=SRr<*1_jYi(w7)Rsxt10gQP9l?l&KM~KblN*(x0{oHmuKLYeb{n zSea6S)Wql?xu`dc9(??KLhky0V_sLF&{tcJLwr)IsFDV93B+iGX^3c}9)T<^cywd< z(*6;9wep1I*3IvfVPkRX6LhN{FCx4NJjg0d8JU_wXr(6(f2~M;E-d*24~J-1^wM|< zhMW|^S9(AJ@&R{74p>_}2oJ~|hZN(q|5Jb$3L>lpr1S$qMinl_28*limQ6dgVBUmX zW{pB08e%!T2Te(%neO6hTXJe=1^yWnc(%Of0wu6gJvcrqqIU%_|5!H0u8Au*0c914 zJ)&mB*s=NQT56f5l$4u`{UEM&?O@)9*OnI+t5J|XxjVJFS#jZ>qnhm`cyN&P`FY>8 zY?1;=h2%(H+T6}+vOUX}`;nb}15+|VvtjTu($a3`C(k)@n~YgKLr_?ly8K^6OXsT# z|J+C`FaN?d>lz93+$UJ{>Ib^oa3qon*?v4Sf}P%DfdSCSKmH}9y!`Da6D&5}Ri7Go zbotp;)1hU1-_}&tsecRL9Bfc;#dco=5UCbuwP*h6Ke3-dpAt!_&l=H$Z$YUIxbS-|_Vz6o zi05F{Y&=*zcR;XsI2lI;idaMxD~lMXk#wA0xY=ySbp70xoc`xO>hClvMuse4W3^_T z;GVZ{#^yp=z&!-vKqSD3XcAIni$Fv4w?Q<_=GWs$?At%z(8@vT8*TywqcottY67Uz zd-x4|s3HGRI#IhhLDloLC!I1Qb9O2!DnSycZdb9G@2H~HlUYexmuyg)s7Ru_G=CkQ z&O3k^vE{>iPPJUc$v5Rm^jW&+fS4=O1pzHz+7mGTkDNRxJoICy&U-DEKTIv*>A;eY zIpn7D@bJ_z9?`xBwa%GsnjG8-YyoBy(ocRgT7k_MFLQ|Mt*JGYQf~Q_)RP3^h}0~4 zp1-2+E#+=BQE~ow!t;D7&|NlVhd>Iin}XP>ddKfnj6pP0xos9{G5huF;^J9eO`+!I zSu=(Is5Gs;iI$YXd6jVe_7Zlt76o!16$wA6x0 zfqt<@wmr8^-I5(x6IA%o%u5q*C8*rtOXyx}zuCRX(uD_FlN7_|!XbcNFQe&TinB`h zGT?oYGAUlDoIE`%!VoF0NO;c79D6ox*1xR=`aJug(XB{jiMY|8^{`f$rm|JE6mSDu zN^c#h&VIMuM@w-iJCjBziNK`WjsP(==AqAGV{>|jV>x9<(e*aqs5DAxhdmtC-kf;M zQoFAN-M~MP@lQ<`%b(KGZ)EXkks<9BwhJh^oZG%YB*laB5&W_T$e)P8x}>xe35mQD z)_x%H+I2_VOUnfYFP+N6U%p^zMtpt0zVXMwPQDmu1|gZX8LpJ=NEte8OS6?Xh3edp z>3eB<1s@vMZ2Bu|a7qMrL`v-rVHP+95XwVB+X z@taS!?lt-CcDhaWI>1#B`!o-zqj;7+zeLbah|4B-{o9BvO9eCLT$RfD&nt0)!_N?Y1-m*s1%2Z5e*D@ z6%%ip9;MlGmxvK)S>4II4}T39L&b;1s|Z~z*^U?e-5tz(&G2N5i&$3oO#vpVv=LWlF^`fctHFnT7s zy?ZBgizz)mc{5%vg>MflqSw!LF+7$&t~c^kk@uf?9)jCIKoy$?UE6hG$gf2e^`&YH z8JwzxAb)F$h`|8IFRx5YOuj2Kgl}x5uvt_1>?~mS%+1GHCj-kvlxW=$11F?96OqNF zbr@888ra$@et1^6Ri1eGwcB@atwq3`C@T&`uZ}gN2my-Mnw4JGkm?gBq+6giLKa?`r-XB_kn(^m@FWODp?nl+aK}1|P!IA{D zQE_qP=0}Oy!{wB9ngabm>`s-9xvyjsI)u7SSPP7JAKlr;CVh`|oA8E2AO#Y`)BNwh>|^ z1E=SIM|Kp`25yX1UWh?fh^Juymx5SKjJb8o#8*ib6(mOs-f!c7VK@K4s~CvXh)6tN ze8>j^VqZH`oj@d}ZM;JpaDjP`%N39PdP;~9dkmaz2zRn|7*J)B$W1A7i~8HU)4PxL zTBO2zR}R|oGlL8_5d@0EXzOoLM+ND}h8MLW~q@ghmszK=NAUy6yR&zOp7Ugyu3+Mq4SbhW1dbCQW z3ofWlz8eE%!4@_G@Didr1064@74wwEsL1ixWaPjv3H2O%Pnql`e~k9){hRlbl`^ra za0tYaS4R^`Rxvtyvx!EB9)sGOo~YFR5g5{E)07{H0|0&X_^tib z0tpi)hF`Tmt80xiL%{6$GFNhSwHC^MxT@JIAbe+;2S_TY!>Dw0Lz&@$rl#f%su`G8 z^DSLr%tLOh`HYlMyw~|^D9GPDsX2lS<|4x`@7@)B=wR2r9Nxw7@ov?-6G0mQUYzqu zeV{y=Z&3B{A8Av2bE1711E#UQK-StiAH}=d2!GAt6!Ylm$&oVQbNdIZ42iGl#UA~zRuX+G zwzQY+Lke5`>Wcm+Y1%nTV+KsI$S+3xB>WOMDYMIQ1jqj67{y1B%c`Xxio_>bqlFWjOQTPkcv-OR2zPR`&v z{AzpN^(E*h#2!LQHUkbH)zHjq{WDl$Ps8!oWe@xg_|G;=BDvvsYLR=kNekj$u8hR( zCkR{D!U^K(LOC_U`MH&<`-TFK|9pAm(~87$F9nQy zovO#UzWk+mCb=^aI~!n7D4+U5__1ZHPqN@y%-6bQO+`hty0NVg-v0&5tp(G( z)!aPqPVvZ$-WWv(3&F$0x8H3lEat&3kEdl-nTBkiscXB7)G+uu5kMSeJ7fA`0T6n< zuVco=?KOaqY+9)b^hhK|kKmQ7+vH~X>}6Wsk8Mm)`|`q33azH?uf>!Y_BA$Suh8|? z1DB2I7I1OPl(BV8f9)bJ*>vPE2j4lw<>Ys3O^z(u-qo`aws+QKCh~ew&YDqabiphO zg0bWP0${uXlqG^5II)9FLy;iLV#e9xjo0Hz=yj4Xx|6gWSwQv$s?vd&sqX^NcS6J% z2;TxArtOj_x9oN9h*fPI52D>5G%?*Z3GYw=@okha7y!a&+^D-VRqa>q zcmZZA&M%{FjHdQl+35EIOy#zedCe;%0+T;U86g-YWhi!5`xInU|FK=j8%f9;CduPT z5G}j`!a$1jdx3PgB3O1#qls$vDRJ6)AG*yzQUlS{bm2XH!&{>(71ho1mB~D{<_jtW zQ8w(?uv*G7sc(tnpYHr5^ns2NjH|A~K@Z+OyXPA_`F&c4GmbEBIGLwzW0@9UQDtV{ z8lcFem;xG{<)HJ1ELQUuPj?JRLGn_)o>~uH<;vrM7bnf#T(;AWqs8N(AGKkT;S{i6 z=~)8t#8p$fB_~<=lpSVmZ7s0OAkcjQ`M~g5R^CN)&Cm{w>?=uHnvb z$NioURk7{syIR-K|4f{XXDFsPFIAo??$v@<(--m6!sarM-|_BbNM;V?1%(2D9|P+V ze57cbI7YQ0RAVkHJ8vN6BC4H2bjHI&j4zbkwPKmg{nWHz{8QL7xovqSolstDiUFfI zYLyqaQRu@20*CrOXsaEmxvf3;MBB{iE#z`)n%d1JNy=&Q{X##C8k@{lSvC~~m(tB! zy$q`>;Gr4WPs7h>6eudKux##b&r7!xL;5d;;HGaV8!x_-gd2@7N3!}9S15rC=j6k6 ztj#D%3gH>6XQ^9T9}lhT(8rnBZ@bR$+0DASTpfb!D+Z*Jd(D)WpTMp^pZMBq8xO(? zB0YQ{Iv;e%Aj=C4z3uDbBpl7eB>ih{aMk|Q-AaoQMq+8Ia{tQOajHt?D$+2Dm|}iHOyKJQ1e-MozFn9{Rw< z)T#xNh9?8loMe9N?EkL45S6Rf6Ok!H-A+?o;>A zm=R3D^OMS}`n-ATcu50cjH@`3*n5zvtXHb(pj*o+r$PwEi-L|P5O1iL$!8+){hj=S zf#aHxThtxPfzH};Ufl7|%@upbwcYzlAvMwA?J-0ouUZ|EC1V3~!N>A2WY`BQNs9bS zv2fVkZJ;R;ys(VOa0ap}%=U2j4sHcUswJ~fk)1hzM z1jO)EB-uJJpK;&>SnB&>{D)s(r=QH~B61W!`ciW(-{|&$!NwvZv4$GSGyV%(jOS+s z@;ZrMvXWXe>oa#;f1wW(OJ0W(QtXDIfamNk$PF(?Qjj> zqWz-}!G=M5NlEtKHc!z; zd8RzSk3l)vLdqAG;rWy>e$~lOGRb_W^eEB1cgTcl&hqHwtZO-xiI4q{W+nIc-<Dr!Q-C_ z8OT&^`+ji*6}dXsbM<30CyjGe86gh2cANYi`hy2v-YtI|%SjI@XAc;sI0eC~Mjfy7 z!`j->suAqFl7xi+jf*qAsA+^q-94k(L;xI@TnXGgRWi%>FCN~m|4+Jz4IFWJt)(d3 zxvy`NI`J@Iz;mP}6QNjABS0Szn*WzYMH=XpN4AiL&0?WmWup&unj2 z$I%3>>2d9CHj`v}JXGN)pyg%wVrpp$)0oLON_B#&Pn3TzxlmAqlST6LqYVY3cK9t5wM6ub$4rEDCon zVp!$W4@dJbK$n=WAa`O2^vufYC<-1<9(K+PF@HrRQ{5W`kV^f=)HaO8%Ri1d*F@YQ zKZyBPMpgWKuRG^6bgApdv>GGkuSao*3qi~Uwea8!o&B0Xcu#UhmcS1wCgI}t+)=x{ zd*$6L;)(?Go!1`j@2?G3{%#|S;XQD={0k`&d$L^Ev1s^_+y50Y*+W{6O~bn_&PZ9Z z^g-z)h40!d)=%UXkX(o7X-$YC8ORH3XBn}Tq`bP`vG3%PuO6$|pN>>o536{fmU{D@ za+QwIzp%b`WA|$|>Y>*8GX=Z0iG~kFfAKz%eJZapm^%(1SHPeyrwo>I^{xWGF7LSfczp)To%Ym-C7I${Jt9qm|561cUs z8QV2peC~V|)BZLjOXJ4E(cw7Hu(Yoy62fen!Kg-)te86OG*|}yH1xV>Ten_Y9|&d9 zd2_Lo+^~k{#!bW9#qr?oTlpE4fl2fxlkz2-|LMRBRKLBe%k8TO^+Hdbd-ho6Gw&Ed z4SFtE%`}S+sN@=?IQ;~<8L;R;CZE22BpngMw$BCWlt$*WNq}wnI_rkafVm@aU~4MY zr;r$k+l;!}#(|E;=6o8hfH#r%Tyb5!-Ku(jAC!(3_kOZp?Q(DO{Ht?taYT8l75it} zmE>KpVL%K@M?@5GzdkBOdu>&MpO_A=O}yWElpg;t2NGNqZHR!xQYP&G3eIz|>U+TC z_s`?C33*1W*y=DEe)c{L_szXYqLCc*ljY$&=XDTqpkt zobJCkGvV#0kmkX%3|lUE?{I(+SdjsCWyJ1{S{1J?$Q#;ah&)K>CZlmRcgq$b@h+EP zk*A9rY*lBr;`V8`48FAm731i=#w2BO@vtA0^t&OOxZW2DYu+-P@+>;3&9k0NYBxts z;$iIs@muJooZ_O8DA&A=$(5i7v<;9!%}4;!NU!*gP6s0D8zO|hE}7yCqf@!S4r7b6 zfgeiUKqJiOc6&qxIX0LxG~Hgm>VS3+EE#O+;H1*769Nah@A`3bp|?BpXu2SI>w*(_ zCjMzx*2xGu?*-sgL~kVUHTo}~hO|YNl$WhGnQu-Yp4OGvy_&aKdc<7ED|B)z-*ed!$s-%)kMm7h=%3|*R1DR*~nkP7rUbvmltsp`><*Yc@ac3;d( z`81`46|B>llvmkKf4hBF6uUmO<|JQuV8&0};I^3o3%4Il={@$o^WvaSv1P;au((IZ zoiRD$$v%1((IEe>%&USKn0PDJf9<#y(Esenby|Y9e_>Kcksy?BSIQTgtnC^ac>_{D zG2!aqn!S;)W{)>2aWQ4zq2Epua@Px-5*>nn_Ad=)mi835z=0_uryscoSy9pU*F4y>Eo7P;rV|0^=EW=8BdK&tmgVt91Ds3 zuDhH&r)Evw`Cy81We@k`hu_CEN`4>YAh_$0r~ZEns-CYOlL`OrxIYlw^TFIBtS51Bi&PW_xs!{x8^emO%V}Jmwcbu6&Anc!MlKhTA8yp6~g;sf@rI zmn_H76_5n07DY=wiC~g{*&gy9r!NIA_Fn?BK!Dh2zzI8_Q{3|dW+*!@5MIU3X&f!c zK}Kjq+JPCFaWlkUfsptLFl zIB;@xxJ*%of{O7#ptR~2&rd@ir~1)3v5>Zx^npZzukdGJzz9%oqtZle>Nm?ZdOKZ4 z(oMnj+aw}XHIXIt@aBC~+p-a3hR9Y3u26kQ!wHLTRBX$MxN?FW<;pZPMe{v^2#k?7Jn*nJ-Q~+QS&t8O}<8 z&9b5d&hMDUB?qh)LA3j^pQ^64tAk^^D`{BxwzobzS0%cHp6{7hb{Es264%>ObK~L( zFg2B&)Ocx^z-BgV)R zAm%J1D=T1?DECLfun`n6ybKBT(Uv(dpP>Kc+7c->T!qBc2PWUHDi_B8 z85cL@O@ppk6XGj%2>lbRf#c-TD<3cMUst;8(Itev26$AVa@*u$5*XA=)9y+~tZr^D z6*SY?POPQxoXn?*i3kIe#JhTojP0|u9R1EOZAq`z!_OdG zWJVQu&w$A)UUU*(9r~Xi5q9n&zn_$=NyhaYmb2^M&8%Jr9SS6|)ZQh(CpD8$%G*)u zrgn9gmdx%bY@fHoA7lMzCKW;I4_p7Chu;Z4NnYZ0)}9DWs=U3=aCdWTctIgBIj;u4 z5@}v2dM^>A>ie&VVjg6;d5CVsdoPkES-xNv6EmWgezI$nKAdpYDuulnJNN3@ z<%A@z4@z`y9vj69R3blezs0=lo$6IPmdleF?y{U%kGl; z`gEVqf9pZ4G$*c7!vjLe`8Y5S7Rr3X!oH6AxOIIv0|KKrc~)#-EZhUB4FohBG`}t1 zrs0h11~b|Di=m*lHKh;#B1eU;!)Ri4ycnMQs1?%9O^b1Rqc~ zS|z08{bX%hin6KHKHqvznjum&`a#56FpF69 zPs5_h{|Lp?($>XLE$;|%N0To`47Z4k<3<6zgd!W9P0SLxmP3*B3ZI;jFgMh?)?^%9 z9I!#UNsBLLguF54&-aNEST7+;>yXCC@+bnD)0Un|%UF#&a$E?1SdIfUXs{4#*|c$rv`rHTOH@&MsOaADYFKz7gX z1Q}yRWgpi<|N1oa_X$Eeunr`QFh4&;n?X$~^{Hx>&U*)^_rsGQ9Vjb(W@4FNf`{84xx1A zT?JG^=3gK`;8XV!TDP<{mFKWtQvcHe{#YsOccG9SSv7G18z8}Q4kmC2lV}d(_C!cu z^&HNv>XqIpkE;u>`krj(3XX1z{xi4(Wt%T(`Il82(Lw1tdF;DH; z+kV21X+c!UFJ(!%La*FdmG%y$F5OBj)9B55y>315<6AascK$@Ro?I?`Lq|N7_`da_ zpzpEyQt>QJ`91%kqPW6Z&9p9H&pB_5B)RY3+5Lz{n*0Mm z4hc+Wh)|LaR13(owCGH*`+qp;O3U!&H!K=x*8 zY8u{YaeK4*A}Hn=D2PM-spfPZzKw^O1PL5ifVPdDgf1Whn{oB`$X#A>gF>T3& z{2!=(00*>Xzrin|2Wn_NzA)&SsYB1qLN=c_kv$+@L6wefyPLVmACnOK^58m0lY2v= zrf%`m@%(MftTt|G(gqwUVE;y)`|;sRTAf}|lBi`*(f6hI3&2vzSl78cGstWz&KP{o zsURH!hwaJyC0%rvpjUM+ZTLDuBJ3z_lD>A|6-fn;^qkrlP`pqn(PJGXGnrIpL2x~$Jd)(n&roeqx?AvTN zHXp|*%%_9_zF)2pxybid7+d!O*nf802@CtWx0^PeW1F1+r?1tpISnKVw?s%9dktrc zZv2AXL-R}@5_k-#a==M>W)|a9UWL9!8cl*q9#p#hm;TV;4MpK1ooG2?kgpl&j4Kty zUAO4TjVf>ly~(KYVr>N+C9TIOq5G>$u`eJhg042^_fxP-Vy!zMq!j+-#GlPoUwemE zU6K@$mAg;pO*$Tvor6*25aeLI^K9}WA}w+Dxt{p+z#Y~miTsV^8ZRFhFDp!iN9QEn z0sw&Q%zjZwg#=MyXht*t+bs?~3NF!+-;}jusrIt@A=0?@lwGM#N)hR=e}h-_o(}1+ z`Hy4JNyg4rj!}%{#ZR7QQq8JZWSE5nGlAe&1@LvNe9}bb>QCER@c9+(7$YX1kX)Af z2MR`#r8q#Ph(S1vNIu{tY~)e4QpuV8mX-1CfPPi7<=V{3`rqVei&lOi$CK;_c9i2^d`0wD~Q);AuSXLEv#T2BIc{4aDEYdhRvQc4dpEu1lVc zhA-~;a3s7S5~BM@94Jt^IH3V?neP0!9V*<GaczIW&$n;-5eCegVEwo>k|(=@OjyJAOcM> zS|trslFT54@$kNRg`HQ$-9>cWFTvZO1aWG(AlsuNp>vbEzsn+(&0=}~ww0uhcan)t z+@EPqXuq-lFzC(?;_E9fj(jU z|3ft`9w2~>4+P+Nc~;!EJ+tZ|xVpMR=^0w`)`M_1GrbreXS$OqF?c)xC27HWKb%0M z7^(+B;%Mea!xCN^wL!5C0W=Cc3;ESmwF0)RUwYS{<0b>HfhAmKjzDpR`MFE|qm5In zA8qDODjt0_ea*2`AOvu5V>uRay=xV~rMue-@HiP>3g zkpZ4bKtg91jg_JFI~>Jgh4F4q(r#FA%!M>IsMNG5y(*~LeEGH-+r-1A%WpO6$dro`uvK90V*iGb7g4>BC<5OXojKxRdh1HRNMqr$g|nICMj>{ z1x?+aRlH!*^forcFU{d8Rh9@|5G z4mi2xFyhj<$T@@^yGN#Z#+z7Le+v6IdF5;;`vW#qA>9rnKheSVTrlEzG{l{8{g@lT z!VqiPx69o|ug*3$C-=Q53KguHS1d`O=M~D|pC~)v5bS{|HIyk?Iy}{|b-%3K&6HiJ zt?0AZ*VC$wl8TW61D=0sqZ1RIz%JVj(pi<+$!@Bf8R&`ffSGTGlTiso?3lJusHOzZ zm*drZezaX~3<%one60RM;3cf-wcEUy)HYBT!&{GfTuJX&s^{iCOh4~oESLF$N#$Wz zvF^v-&LPt19zOX>T40 z**IP7Ub%p{H}q8N!z(zrMJc=A;0_sCVVq45Wx)Fa@$n?MJ zooQ4863~2vFjc-vJu*j$l;g#AIDoKN_}%ZX=xyLbor4ukJ&=l)-%{6mogljrWH11| zURZr0M_M8R`u#5@SQ1tKAqcesI&hOB>K!a#H|_hBhnJ+QpG5hql*pAl;s?O?5i&fb zu=l$kT8;c`NEAcV5~4lD36uJp!F3GtzotEyOs);c)NsXT$7@$RU9>U2gm0P!w{lPgb7{_-nU&i6kF{=XSeIL zMe_lObyr{{kpP9z-zNhj3#l|9NAORWYK(O`Egm9i*|{7akN1g{;({V_MHnz%{{g!* zla}nM54F@**>0EjzL|ViOo%~=fCPvCIR3JL8k|&}VYAg-H(%WfD*20{n$yPCJh#tZ z?k|+I7Jv&GxE#*$;Q(ys3VWr0ZI0duhScmDix~&4oo>N6ND0s^*Kt>>%jK}H_6f5x z&Bi<8EBLX~bOusD>Ca}|G>Ak`G;8M_jlt`y^~U!|!L#u0$s7vJuvK!3#M%7*+Kze_ z(RFiPt?A{IU*T)Xi5Fru$5%C6G1|^y3*KM@OZCKATuT&f09_7c(yrYlSC(1SoD^w) z1x$KM@H}iTSUE8EZTc=@uYSt>y~2X=B{0f`=9>ZE0AJfeFGc}G{dpE23nF_^fAzcWa)6Y1YBf;!7A%U|O z%ztJRBQg!(w<@<;&Ce1NiW{!)uD_=TEcV~CF#D8^i_76yxb(3;#{(2NJCte67ECu( zbT>%U>Kd0#8yR9@Qef#kKqsXjB@H2AmX;N&KK%CT-NyIh>(y6}XhJDW?w@%rRKS}~ zzKfn`xQ!4Bd>%pix1`|85!|TN)#9{QlN!F*j_x`;Z+?e>{`T>}LMc;JQ+SpmmVoVf zIhwnC3FxL)Hw(T;+c_whILl}<+4O(P-gs44w_oQc5<9gqr1>VkxMV`zJSV7Z<@fhE3W^V3MXDf# z_*9npdCVUP#_pZ%OhrB$mJU2pNI>*~nUzyUxU)-Y)hla2J$!Q+Nvqfxp0kGUak_3ZMs=9M%l%ZYul{v6>Hi~P)8dok?2cFuLN`}lVITkDYN&noxMSa7 zcnj&*q)x|_ET!UZdw=sRJw&6S$dzL0Owi@kz(e~&-*4JagSrRCX=<)%G2F7!PU*&f z{Ohami%FOc=mTLY4{A0dm;%3ETzKEd#}@^`UN8`sTU1hV-F`k`EPWMn z-2evlyJ0Juu^DrL8>LI2f$IkckI4E1;I{*Y#=V8+bPzXkgj=zUot3?IhEkQ}0 z1jwB>Na?02$m#+5R+JyT9z;UHK+s&}7xD+HCWhKf5AX<$POw75ON4?-oxR4YTZ*s- zt3y{RX6G5|B^MYSSAM1lRpL$#V(m*fh-DKw7N|lyS04&;ve=qtz8-UVVrsQlQO+jz zl$B(cJ3p_b>f2hExBZx4kdH2J4hxICVmMZAh`7WXQ>JsYb^~}HG}IC*-aR#$hZ$b! zO#f2bE!|9>n9|@}AI|dSevLkSI!r5%NkAjB^`ncWKY_)3M(cISZs^L7bR(lv2^`$v z10rIzAqSbHIVhnT_~H}O2Si3e3c9?nliE+yc4^aH=xNx8^>n5_`2#EB=s9Bb2PeL+kayB(XLCnH;~{WrLrYHps8V5vwUWr)zu z>)+u$_Or_C#xmsl(y#3#Q^K0F1_xI_c4$G|KqxO?cYKION}~h*&YL8U_{m;-d2MEb zxk4TbnWDu*dZtNu^p1wgupr9HfHLhhU0(TTl(ixoj8Dr9p>`A~q=8goI{LL5k53jM z)9BMJy6Mg^l7>Y}3ewqiJH0xwZ5?|C2IT_(z0?e_gjr%){xzR$@Q7SZ=pE>5^EuTG zaIDJ;&^3shYhkdbbD487*xtUn`GAj3l4`afxKaIfw2t3f!T^|5Qu%vZmhP?LFr65* za`7r`Y+Aj2`~J1KFcr+5({hw~4U~rPO`X^6w)3or`ahx8?8U#_PKE*)dh5Bx)(zkc{|=@k16^l&*@AME|fpdf88)-54g zuSNE5-}tGh_=M|QUQ}tCnEZCrXqE3*!c$(i@;RD+?wNcqkz^s_<)3q{b;PjWLt^c9 zgu2Qa^YB1mu-cRx-mf1t)IqdN$vbONAx8yc{EiZkf$x}0Uz1g-GX#=!---?2mF``; z`0Tzv7u1?=hdOyONyU#|k)8E8K>vWES*^*^V5>Zr7k^8wB&FvhHcpz+ZMeD2Vi_o;8a zTq8))8mm|8s>OKNvq@$|xR*gs%ob@njDIC^60;heviUz{wa ztm97jnsixacykAV-61O%ia67UV@*wC@sfP}@R5X)R@T3?RVY(ua$5LPN=ES3BR){| ziv@e1qxyG`aTiQCoRe6^@*+rA(o^QpgNpLb>Tt@j#-WMvosP|8QU(edcK;OpsZYry zR9}1Nmvf*)YQ8FeC8U(Z%|4Sgu)>THCL-~9^Yf%Nx5Wr-Svk2dli8s%K89Zb((D%@ z>dvE~t&PTuDYc%`STHy73SAju(&k^-XCVM9SUqE4SYRZboDUi{GTpCX`g>g8+7M4N zdq7%nAo2;}%lodbcH4V@I7?D#Ab7AYr|1JoP@bnlfYs6)ABi`#X-1Jz@dRY>L2gKB zpgNdHi7Hs(8_{+QR)N{U`=?jV(8l*(UYK~t@oY%qcbIn>hhPRv@rfx{I(d0r{HUPo zTm1LXswdW>ci6P_%`uDWf_E4poz&(rA>F+rBz9Q4b6!+zth@@%iPP>F$l(21Q{BN-C+;Ky=A+)eIR>+p;LXMTlu3MN<602XtjRuA}51Leq)k3 zqH^v?0O$U+E1c7Szo$Dq+rPawzo36fT_ue=DYWF|0GjfMw zGi6um_KPi=DE;II^KY3rk zvgpUpp=IPSi@m>J`14WUK*zw))RUot3^JWV{V%C(FOUzo#pza7tKvl!N6y=EI8wQ7 zU%`9@L+f@SUqwh^RyZqjZb=ZhtTnsqY;Ev_ zA;p92`}nU331{X_2L4Uy%iwlx7s%D${(cO;gTIuHMUK?sevi*CZ&y{XB_RkuYDP3r z=$g+wGmey{r=VNDXad9+C z&rNfF`%O5D?@>qm`f;)0WX<9IlW#Mdjnh_(dXC!!waV{A2r30glOU@?VEJ-P%!D_C*14&#vZV0+gOC-};o8 zmEIS` zppozvINluD1Yt6$Wu-MhI{x(sa zx~JG=Va2uI!!OE>s?j{+)WSc8C(fu<@>Uc-l;C;Zr?z*o+rqfD!CP?xS3P(na(LdT zL!4ojKu5XuyzICUfCCfm;+@FYNL;cTF)UdY`NTf(RHt-Z7<4qiWWcGvy@eZ}IGr>= z=`uezX@pxFN;iTVGnPhy9<#1mM(*vK!L1B(@FJ77D=m?)I|#GnRMCqayqH^zC|hw^?{Ql^&|8@) zho=6Pzf7w-{=Q{5i=;{};Opz>i7^YVe6BO`t}EEM0fC<-|I#T@&SxC&Ta8fiv+N+D ztRUT+xbuDI!qU<%5I=JPgEHO_$xm2z!gKcet?R$)+wv19N6!hHg&rA<6+O%OQ{DJ+%G@cH@O zX^>LqJ*Q{gR|wPsqoyWXMWwPptrHY`?Uj^^)teKD08OA-fS~J<v95emQSzI?kjpQQqAkL)|kGLWYMYQ4F0+;maK3-85#)@taRJho-__p$l@xui1<1n# zNIMKv0-(~@>1!%blZoKt9V5zz4k9wuU%w_r5$_;W8?1uQ#x92z!Tn3$(xhiG4n$g2 z1|ogcMV2v#LAmIi+b;i~N(&4SCcVzDAvsh)C-mu4Mpw=I36$?#;|22h?0wHO=O_R; z$f(FEbi!(ziYaGn!&XWlJ9Je%B@yY~L23cSq~1Xu3>FrW3>`RbMF6s}J!khW2Ujo& zMkmdoxwI=qd*j69(T4<$i1~VP3FUlcFsJl&rpU+{DAouZj^Wq`QbAU~0%gUyo=Pw~ zdet&)y>oo;v3+rn%pOkS-6Vwok8RSN7MC4e8ZI%u=k4^QAQ>a&v|MGZXj>Ek;t~PG z51XglxQN}2X(2Hp=j|(hBV{#Jb%Gn&)`_^;c|iiidg+h!#woq>BBh+rv!bb~_}NvK zhny^ZYNp1E;aWS{p;exuIPjkMm?$E$-z7V=q0T# z^mv?EzI$+9R0X3*fK#2=D5<=ZS$1R$SR;<5@!`=@xxJVnmpGm@P_86>r9L<}+zTXnBnJj^Ohh`A*i!(e6?TFUFu zj^Qy|qY8y?aSu|3{Ch9#DjZQ+Z1}-$fl`OxkWwB4f>>|pZ@Xr_#B&#JJAy2O^lm*P%+lMY)-6t)sc)GYUGin^ z8NG5jE(p0L_q~f((U&rO!ggtWcbr`3_)}kp&*tq5>G!*e9s45F$w<6lXYrI@)rXrg zJFw&$`)rLp>JIq_Z_CorM0j`Wz#Nj=e$?v7*_`Y8H93I){cE{DfBrIlh-0esIev$l z94adJU%r%Ns@%5*>0N7UYtF!=^B1^w!obBUw5dq~#BH8*I%fahz!C8~KUfDYZd4UO zUO~Y#2gLKh>)FFPiP5p^|IM5u1f+tLlCtMPkXlt7o$!C5D-B4yw`!6};C>|- z4%wltVrKpwLP;pe=tL~+0vMH|mv}NqaDYqDWbPRTSjBi2{lv~P(=9P=dEakp`L4ul zZl7;b5L&Z*dKD_Kku{Q&Q$%4L$(-r{zUcC@g^aAD4a3}l>>Gev!1a!$mVtz9o`KP1 zp8?P|IC!-P*eK8ohJfqR@=e)aakJHi zveu?*N__6Vq}hXm?0jf*gt?iav4)ncefGy{t>}iH4aU6#e-Ib84HV6D8!X^*sE0rog4 ziPYV@2|cOyhIz$aL6@QVCe%gI$W?ljqoVX77psJFS4FWyTkU>Zz)tBG21Z)i`tvW) zQ$Prv8)&$d;B&que(Vc=d==6fxC5WC1D}`XY{v(D(VzjWSVk8DjtErH&!=%MM&G}I z2Tj+Gw^w_|u}oePsQL8#)7P(&t0HE*u43c3EQwlVfHalNGv=v2VLq229DuJEdYFia zGj`qAkYmq`{PG;QCeKNWEihOwtt0!o{sc1V4Q!h$$sBu{R?H;WT@a+T%EHcA8iqFa zGD0M`gsjJo%|9?-nS7w|2sSqnBks^@BQw~(6>AhM$$5*zdU_HpE;bspsEQz*w@JIZ z(XTO>*9zY9vVzYw)@a%jw`eCAHj^Rl=ll@!-Y;uR(miCwzBTGK@QVNQy2$Eh=fn-< z3e5=rwZjK=FQJS1(eeNpM{C_Xyy_JXk{pxEReyDwzM=h2UJe1qotM~ z4{)PLR3Mt^U@mNW?$71rnO)s54$Q9+5U97`?Ip?7sX!K(QkLWFyj*4;Upy~_gmHGR zuus#UV!=Ee+LB`6CA-X<>lUZ`C2&ohJS=1BSf+$u{-P^+pEV)V%!=`oL<^36a5E*a z*|OU=Vt_dq_7O`-$I{{G>PCJ(4ZAnITLF3O{_c_9oA2Kt*Pi>2%8O)7T%#B(VKgvb zXvSZrLSu--@q)I+86(m()wXwKt)}$FKZFQ2URw>ouBFr_V7TA zh$@^$jRRCL1CL?#TBGCV{|zCwS1vA3P-q7QVNk`dfFVMmBXO`y^ULC1^J5wV=!mEz zgl`oTSoYxdEQaLev7@&6sJ9e;vtiT@24$Sq($)q$HMg(8hv&BOyXWZg>x&>MkkUhh ziYcHLaV8;ul+goh)YRghJVnh<|9S_=@M$XzFp_p;vz1v{<%WiLq0ooKLUv2Jr|>=l z10vtfOETTPvfj!oBI4&pZT6+dkePl0?$Q9lSJqywSbfOTgl8)_e@sg;S)aQE))?&IW42$K3n_p7(R_<>i_}l&o)b_c>IL+ z-?E~|duS2W@j*oK(q~uxlY&AM*TIsU;Q40}dc?uim|M)JoD7Fo6c#%$^NK!U_7#HG ztcZ*sY95fiEV0{-ii^Kb_R-jMzaB9U?qMH)QX*!E=fJxNYZLDMw6 z)4&L=7O!~0-lh1G!rL~x`X{4KC8W9d=l6?xTQ0~>zp^TkNG^A=9Ud|9K50OC^3LlD zLtGix+ynT`Ty$lrw7th;sotFYA%z=s^^>cfcc!}yvuP4ehg`LPy z%s6ZDrQbJ7PH zurI#vobQUg!XN_rd|~lOv3HH>w!8MNbbGe)Jy;}Q83mZ_Ymz+SLZDkq<$Lj&0239t z){rp0^k9yz$!jsBxR^_?;>{2^GZ&!WDY!|%*2(?(ztsf5;+oG!9*@9Z|4udiX_Zyc z`X;8OAoJrF#Ho)$3sQ*nG0tT>&J5hNU^T1Gai8pi_&J z87ZzNgL#%we(dbfyu6Iek$dfTwk6h&GfA4l6o?xYRfsU!nx}*Gkz}NrO~-Fx;oo~Q z4N%n8Trt-^F1C&#&A%A_woUvu6Oam)VQ~$({dXZkn_K_VzCK1)5!?2ef8xcNFgMa( zrUXIzukoX2;zcszIZ0&K#U*IePb@JT@$vCULn9+28CMzKqlOP29;gYR-_EBFWbf+k zeyy$DIhrGf;lRSe(sXgcsxESg$9RFft2!=udJwR6PVjP6aJ=^iv_gha#M-l=tcMtEx{(KR)_CG~FO9^61hMO)>w? z%RuB)E<_9alU(BWNuBpL{i}}Pd)$2MfXI=`gM0&_=YZE)0ZXbCkkqw_qRl}_;eQF8 zKoR;5&^ac)+xf9fFK(~rWIKasvZO#1ES~ zQMN`Pq4qhS_cCk^lmzf>j`(Q-X&fPTC-E?NP*6(v$G4BCkB^THfy5fXwiVJhe}ZsO zI}4ymLh}dec^=b-WH9u;TwVG8`}dDp(C$I%tIUBo;YHa(%~G@0?;s<>1!Ui?9{GwM zg_EN$g1t;Z5&)*LKYi2^kzU_Y^kFnHF&ohV)>^z}?JMPM4++(H*}HnNqXyN6%G|vr zWO9o}HZ^BQAFzjeDoQeO7=X*3B|hl`!Z|=<2apJ#05(VKGg&BoPNuB2SE@4rZY;HG zD&Q8O6J6YZ{fT@9_Z&Z*w~UXM=!|5HuB|$BdhOuv=p=%2@_bkRFXHM^cxC7YF>X?G z=(XOI&`IR0@7A&HZcYx1j2~h$m&tzX)=kUs2OCA>VNrZPQA=-j_x9JVKIX^15M%Lu2F7aN zd`&Ym4GRmF4|WPi8mS(dHVE|RKOKxpbYNP0|_($T1$T+9Zkb#;uWmzY<}9! z#_^;R56G-Zg?t7~g37tEBpl2;4 zpJ(+jc3SYuAQi;Rqb|@l^L$I>;KJ(kE0hU*ri?tL9)f+oA4+W4f+mm2y84(TDQA<@ z7@Uzfi!_lHdb)>paHzW#G3=SZ$O-qz>M$?UuNdzB#We;Zh6?W9xtO8xp_$N3{NU= zXUj@-`I*mg4aQB=vAdl1=Xf8fTkpwoQ?Uh6DaNKZiDWM0FJ)GXd5yEHNFi-#fVVD( zO1Qm)Hz3_af!)3*0WKI((WI3%H=fH@<3P83cSF1c|F3#3o zMCZ|~uWf*27Ws3ycQ{O#iu^v&qhX0S)pwrdXIi?u-N2t@loDFk0wZeqA4pG{;!!`G zKh+%k7I3h(iVWHuO#2Y~YAWd>jC?rTX-`C|_<}qBAEvxja8urKRHfC7*0mV<^&DXv zN`xOyB5`+fOh7HLR9U_)+L%dz0xPl||g^av9 zz2xNNN{3k?R39Ab?bGMaFiCyx zRI}g2+}v-FAu3#J|7YM9NkKsKH2mbmeSBi#rHVQW`}5?;%F(t|;R?8_wkr2>4oiqt z)`*fSgf3gUNZK!=S58R@C-W}ZBM|LK^iyP{v;AF_!>o4z5h=KEGSN2yPu^R19d=pV zJ3pa#9n>5R6n#KTtKt`tT0dh8sBDaVhkXWc(fpK_p=@i*k(88FQwxQ(3-f-MzeU(s zSXfL7LlZN;2KP?Mf)Cd^XfIAG`Wih2-sq58faUz-$=Qj-eu=x-t4hwi0SmJ#`w_ z#iZ#}|FeY@>=BlBl=l(pkum$*0-j!&ra^8cF+>{9&Fey`>G2pQ@P>F7Kd@UYucE0K zVPWBWr0LX{KQzU5)d02{Wu`%}ft@d87l}Hi1@J{=d$69qIB%H)dv~v7Y=njR3*(i$sksNQcVKwV{P z$f&54931vvbJJpExDIfl_r~xTqYcpUP*X`{GxVsiUfK4tGwAl^Qb~tf5nq*2>RuxW zxP3`L^vyk@#Ww0oudSKw+t%PwOi6fr1WEI-!WaH&Xi|RIR9T<^+CU7CFC|aoAZ)j0 zb%BasnBnifriawr%9GRZ zpw{?R^>5iD4gfYHrBZOWqhV*zr9C9K-U(W1Bu}ZlsgTTpS*YEW$~nHqX~njr&V;_X z66^S*?LG%-Sakhbz2)sE!9)#skh;2Ke%UJGMTe>bQj-U4>JdR^$N#R1purfX5;9`0u~?@<>~ zkyD*Dzr$_iok-mMa7GbLC7WwF-V)E7*e@BeQ+51$Y4PN#F;r}d3DdcU$(((H5KT2T z^g7p3kAN%A;#xiN%FPjY=vB5%p;~%dEk?X8*fXR!Sj90d>Q?w{s;_+{r1u(s=`9yO zZ1&J2@UcK{)v_iG_>aE{Slk0O85`A(#LDhZIUwwiR41$7^+|JX*G#r$zR79-C{TT>kCk(ku6gSdIVF zJ@aIBYqpLbJfEP@B&pTr#T}1WmWIC5ThPyVvuE>S6$~kfzT(zzH8sy~|3I92UY~Wf60%Jl*OG|5OvGoO5G_pT@NJ>urqcw(!dCOPg zJEOzm$vyN##BiOz(?(CZL^~nW% zxVT$D@>$Ujq!D0aptD@g;ts!F((PPqc=mYV*F_C3y(NxR#`C(`v4w+2EPU9#F5v7t@+~TZFhAmA-HL$($dNBwuy~7! z*+Lw;L5tZO;$uOloNXko|C3WuA~m(4nU4Ecth;tjN@8TaM_7pYgZON#3OK&Bfw#!( zTjaL?9~PFbtBQhOY8IqaRI0Ds-LrYwpA%%Uy?m*{EFqCmDZu-w&dd9y>dQw@pR$fF z%pufWwIs^aPQxR>4u~;+a35WD^zko!xTJ^I3Fz$Dr;ks-r;Ok(cIRh9;$FqapYNI&i7B~8 zr#D>3mRt4buWMl@-ja;q^}slP_?##9>y|B2T22WEd=(kAs(&Fn0IIJ2EPY<>A8*dwcfa9IxVu0aPkBA#Ilem1$QbDVwr7Q413qsYQ+|pS9k|;slC-sUR6A4#FIS@ zGcC?X`%_PEbw9mdYxY0MklbriQU*?$LAvhQM3$hV| z7#(%RzBp6$G)XS#da?12x=<$8U8ylYu6a|B9Mf(fmt%R#zg#mR$HBuR5i5RixP6m> zC&u?@XOYQfS$0pAuidc8pwT%!E%EXafexyH`go0*A7Mj&f6cC1`?qdBI|F}H(wibXNw~Ixx`r=H&M`qvZ06UfP^pvq30?MU(~# zp|bo<&A}~4L2>G=hEdodJ2{*=llmsw|5R6Aok-7-ulT3p$hujDNi}A5b=zhq?Nj-z zMh|XRa5VHKxtE&s|8YKWfqox-qEVv$q zKMr34Q=1@i1TtM}alA2vx*!8CCNCK*JhZ{(E-+B4ySv-k#^!T+x~huG$b+9@qT;Pl zL7SBO4>%iIP5AVg?QW%IF4Nxt@&X6>5D7=F+@^P}1;KAk5n^T$J7>?R3GNKVOludh z@+U1~G8ZR_L+7U!d+T4wn{mEn7R^1bzYZdv_7pkvxJCX!wq`!c3TyUS3L=K?U++v@ z-em?A7^i#ys0VpL5S+bm*t#FJG__bm?i^&W%+(ymo#!2iA3s82Q$t+WiOdL>Z16 zTwZ}9%F4=Jo5$EKrL(vEHm{SC(BU!ws@{*8`MSj%QQf;s!bqtC)OAZtt_k`|E{Y0@ z?tVMd9rMd9QhblLVoU(EECjgO7_()s7B)9i%M6q+Bu=XbTiMM;-y$nDJ>hWd@?TXj ztdRS-GXVG2Kw*C`Yr-0}3zN0R#}UyZ{NCWiY~$w;Q&_L86zJ&eB<*;_${O0&7b#6= zpqyX%f#RZV{q%Cn*~@&EbqnvWXC2-4owWDT=XG%J8LYnNXE)0N#xPcZMen#DyjhJa zGB%dEqjh6+R5LHRrWQt$@&Ue!_j>}daqq6#2?D`yLd?!aMMtx8aYfs_qi#Q|I=>V0DKji$=!xpJdYD-_y8~x~ z=&_FTOY1hrQJ)1fwm~&wCSE=^f0(H(*U4i)t0?FMG`x4i&^T@`E|mVn=khIDX*gzA zI39u!uva9cIlJdS^UU8`OB>ToPP6`0CtO$zLgLHHHFk}Goypp|#akPkE3hyLo-xpQ zUlcfU=VQ1(A2at0$K`5!sbYN>tYHI|KZsPc6hrVT401l~Da80j*Q&h7IRSV3z*~0}AI(!!w zqtXMEA*zWA;#c`)eu`ufcG5m59x2Rl2i&In>+2@}NV zXd7S%4O{$0otjT69qZB;n*EN)Rr#NI|6Mvn8a)QY1kky(oV&F0suUgWSs&c~yYyeA zRx4v7^17Qf*pYjG$LBELogn9eeM>S{6X7~{8Gvk5oVy+E#R=rUYdGA#JGj1H{_a-4 zA7NB->3ro6f6{W$?Bq6_Bys*ivcO@se*VDefnI)c#F~MmiT>{L$&2KcoSYoQl56BD zLzwcdSpL8;n=zg>(Sc8Wen17JKHEp$9A$;9EJ9*SjAz~wK&oqr z4ggUI|GPI831G`(`GYO2rE*GG@N){~-1LUsW3H#HnshVOJYo`WI@mfeJe=z zlD;kI@*}Ew(u_^lx`1}SAm#1y@%tyj4YQZ=$@M0VE6AkDOtW~SSwvKPJbT_O&U1Y~ z9I%J(l2^i#Dm6?|C>c~Ud8MP7=k4JjSv8gy7N;oq z>A@h)CUyft-<4HMTzo=8vN=i6^fUH4q(z0(XjXjd%clpC@9+o-yB||jjg866hm!*c z444>3B{B*c1wF;o&W54&pGD9zCDUEWhZF?AhY9RT0IACNh^tV@C89TGYDbKxz5W{c z93?ekij0ZDpR5-*a)Fs7Qyz9Pp6Fz&fhSj}w3jR?GRF%Ck?BxpMSQ`O5x{R@$P$_S ztdgvk*-P@H^0#1$7p$ZDDXsT8O2fxY$A-TQwwuO(@%R=WR%~$RB9)O}Na5mpRYa*WVljB;6!&086!%|$SZ$dZY< zPh-n>oo0ed?;bQxsPW8;d`JkR__!7H>;gYRWcE94yE-iL^s&p z+^y~5Px|6yL41|jy6~&j$q&!D&0+xiktp)2 ztCO+R;{`u4U$JM}y*deuaa^1sZ<-UC0%zpR(bV^XpL?~jcc{7~{bHEd-> z%(CzhjE~>T<#KT7gF;AQFoS^Wod5vSp1dam8bg$v#iWV)I<*xHz4hQ2m^fIm7-@cg z!Q?W)Svam$SSW4c98zv25zydy6L`gm8zQpH6B$3fYgB{lvR7q9a`52C?oLg~CDofa9TRG!!Hh5}HzR)g zmV_+yt3XI@E-OgZ>c(pUM)k7N93;_B~xeC4Pao>z)=_+^dsqIi|Zs zrNu6v%+nEKSzsGRWyQeh-v*`JY8#|bTrk>R_{Qrz=L=pvDc!$0WOYpFaw%wo_lNXG z{Cl%`%4m1j>G0y~M2j}sUBmh9yidfx+d78T#^2X+8Gbcoo6@Ami_HWd<#LYw&c;}& z!~VzW^#Noycsl>RItWZpRAYSD_i@3S`q3x7?a)HDj!xe$Rt!l@tKc`BQIb3Lh+15- zSed&ppKNmi%&4JE_J>_Eq2EiMHH@yFK6O_fdHm~(mL%p2uZYKm3x}{{SEbLteEYuhTTNB5JU-K?z7|%r|c#{8O z1fCH9CW!=fngfEfkTPy|c9yfkJ{b3I@oqpS<#)$@RT%={VJ$o37rDWR@lU0}qS*EB@JH(UMNAK7H0Xoh9$YI49j~juo+r^w*(mc z`2*~FU7Q8(`}07Zng7%B(rP&aWVWsL{p34~6Zpq9NZ;(jpv{MrWAS1Y_F%}!m3FuSY&aNqA zPru%MyPlhOS;!2d2_fE;q65EIIWhJ1B5CRA7#=n8iU3@KBR|B-=6c@XHL}t2+gNTH z7_D>X{H0qmB(w=%q_A{h1pY&jEiVX9bR87>2HqNsVURW>Wvy1J`gYE%+-#QC7W2iU zD}yU(Me2O$sx2f%9QQ2FcVRJiA^#?ER`SBG^qXVs5V4XsFHPm@m)%O=*!(D7z;k6E zT%oFzwXE3{{eNuz1yI!c_dkx)ARUs@D2SAZgmj9E5=u*lD4k2UgfyaHPyz}9(%sUE zARw?vOLuqe|E%}k_xCgN`!n7dxw|m;&g;47dCsHGlcI*y*@XDl8zu`#gw-LcTGPgf9e=L;ng-^E?KrSY@t`*OJmQQY$>sAO-Jo*wAouM*{;@g zxcRtGUXpvBG13+1or;+skWiD*{(VhRkxS0wCq70a3E)Ej4Z$ksJJw)DXu$)@=T~`H zX-Qm~e~j&3zy5PNL9B9jf#+f75W*wm-w^W|U42KM zi5q#(-_M_@XS+ekBcPU6>nG(=udW>`wse_EE_$-u=TC?}YvaEDnisP)qs?M3vo{xH z?yy98?FG$0c|N(7gIEP?XlvG8z$#5pew$w9>5TL*Mye>TT?0^4&b$0<5BbdA?#hal z%*lVEK&T{tWxX4rp7+0jewS4etd?WWU3`j4N|+;-K9Ci1{b_GHN8OEo-iX)op`V*U zBE}^vX}cU?@<)#zVF&Pfk@AR$>R#u6iyQyD^WA~lX&HgiZUg1uCVj~tm)539R}xN% zIHjLvAqVoiznb4zkH6qGex4$sK=koEJ7IN9If!=JW{A{^2dD3Zkl=-ac3r6TxSQkG zswEza?t81IWQVvj(Yn8p>^u`tWwrLar(EPb7_^t)*6=&+%3ADse+?3xT0O;P-+38e zvNAgv(pg2We}IEyuP4HyC|0t>qd26-%C69%r^AoEwDNAWo~8Qa=dfoiT00+qlbHuW zSJf5oohpyK?Z^JquNw8>oqqkz>V~wiegLC3k9=A?I(`idWQFwS6}AKGmjC0vrjy|G z?k^`0Kc(Kd(m9tWjbc3NQ#OSEjJk92eI;7A2jE~ZsAmxL5z8&Qe&>4PMjf`1Lz4#2 zq!-cgUhu1nv$V#t2$9y-^!?Soz1WijvI(L_6}rmDL}gT5)?W{E^IQ%8tlH+EpE=F< zv@wVfOER~9<60@FrOAkhV7+;M){O@}iNnzL=7#w8U%W#4gaP36qc6A^}DoIc|9bq2J) z@0@CNOlIG}Rhb&(z8RC53p|NUO;=j;qZ z!no0j4?fZ`CX-WT{(LJ+<|BN8x%OqCSkM!~wJkv3E*jtr1;B((Wfx;Za0Q;FLQd zTvMGuaqjne%kJ{VPXS!pDaNAj{((W1Ep#I6kt54~5q{9|w|am4k8lVS1`F`q>2vFe z`xY1oRwCpP70|m-4$g5-oN*~m>slOP!98+zQjDv2KmljS(Lk4;Hsf6dE6CGV33gZN zn7tRzj>VUUmfp27y%tLmj`B9fr1)z8Nbi&kj_Nd`_L;S{e079|8w<NCe$Yg1pD?;!mfr*JM-0C{8~sJA;PmISwc;YYLp^ z@3+1Y>^${prPje2(XzaP#y6`MK&jX(Q`_sL9yVV^mg{V9SwyZf+wX0CjXi%`x!11tkS)n@L+6 zTPSJUGE%J&dLO<3P<2YUuV0pskSJ2iO`jT_7-5M5tQ6hH{CfjU{?pWKr1U&pRs~DR zEx)}yW2Udv^#2_u5-3PzWo3+esxyaI9dBg23m@ypUlD>9Am|sVoi~av4{QwW zonolOu*~!Swi!uVw7N#7{@^_Yye!y%W*-_iEGxr>;;;Wyr>wyz5jVIIOdWAYQm(_+ zy~p@aEFG)3zr=#`c{bb1kQrIc1NZF#c9nV767=^V17;X}8O!U9`-;qaGsWZrs?HtU z5QJ=fozt{##O}%@s(%u6Dd@vni)(H>izc5+6yu-M^v3r`Lw8Z{-*!)W>An4ABW!wo z0Wk=YmX|C8V zJ1E`#oLy8{eY^HmX-Sc@i_?S$HfWADMl0mpgmcSolKYbw?nTUfe;tU>(t815zs>BO z*;Fa8AaGRLlSGWKSD1<)i}MgwKcm{$(s46HE)VwBifYSa!ErRi_X3aAd;5O*oemc8 zm@n0QhogiP@~$Wt^_hRGSyu7#^=mXd?AE#`v0*Yd>yvkKtYDVh{_h#VYpLuAhj0FP zLk4^rw-F;@ByH{)v_S(a+iy4d{Td*o)9z(OVMT z@CH%WLD>mE0rnN={2V{ABX>|SQAzFr$N_~@&5(TaxRh#vV{+F)^XLu1qt;5RI(}a&AjYu&*uacO38a4igj?2Ub0Wn?JWs^3I#r zkeVbmPq(JO&Y2rTWB299{rs6&uSjk#;n7@DP9n|fN=?SV93+)&%{Wc?l-d|dv!JbY z(0&x?1xr~*q}GmlV!mqjJafzMyrX&dZtpR$IO>mr#m+sFuc#Ddt0A4n6S!=UTg!tG zPVjW0LS}vBt<)tRNdt+jlid%1M*jG5AG~vpM!s4S#j)~HzaaSIpF8JppAMY@;#0`@ z!*cBGzkj_s!+z`5QI0!<3O7ab#w$J#*~m8!-fI-R40t>Y;@$W?s{GE?rfO@{rJRcT z;f~WQz4Nn^FBbiYo@cWkNCs0&k!zv!Vi%tdZr-}-fbISk!Sc95T6Bgz|IG)fA?~yC z04)zZj4%7F7JWqINnx*joRL)^QX88K_mc^Z)8&)5GK{f4#OgyW7@l2A`e5LVo|Tzi zwW;Oot%Yg$h?nY4@>1JZIW$ z(<|QChr%HE5Q1d#iGf?q$j&Xe?YH;3z} zKAg`h94deQ>(|$krnTGl6H&fv8tlIj)GaRVoa>_CxGnpP+wA*AsiD>1y4>6lRssAT#y?XU zWsN)fJ8iGyuZd2rf~t_j@XrzWGiv@fmZ4tI|J(;U*<0hrNQgX~Xs)p1R7*W2u>yDP z-}lC7gW@5^t8dte5!ir4f#!%V1!23Pa`)0gT#6W8A4>FGVCp(0i8gAio3H*+!x>3l zbLa3fenBmT0RaGY(K-7M71Q#9GcuQQhROrXL*;hK!Jym8M8^J7m6$R>BHofno$0mi ziw=&*2N+Y&xbd=REF`uD*z+cCXQhNt3*BTj>55PqzD0)kQg4b4gOXx1oo_uKZfEYC z?%kQMk77IfgK{5jVzBQ#gkG4uzEIKgeVmejiIRocm8kZ6{abw{hiT+O7kT#X!|SY1 z2()eadwRksu>C^DP@L1O8P(S3go5cSm>Oy>f-j0qhyPrd?D`o6^@5!pi~FO|!t&tA zhtUU)T1Q7YOcLt+w9_SBHT$3fh6%J7kGS_t#17`lHA!jK5k0@%cxlvkp06uydNmhM zYF2h{efa`1mH0~{>F?gV{U|3m*NT9LYPM|wfJ#wMFIJ5e$`9kGLHKRD+*GHVik+&uQ2^kY4e;4q&!0;oCKCIyO&bXEo3`zoVGOG^k;afs+ z-l&L_1KrI}-j~2L9>iXc=~J8bX^^kj?!=oSS|X+W9`|r&+x~h`;Wb`G`0zW|29GuN zto|ma3Ehahl`stG{hWK46mg^5gB_1g&$=CO1A>URg#L;?+r(i)SS271q>mx3-?MxcvAq*~Agx z^ndzK=D(oYM##a0cWh!J%_r4C)fd2>X=rM;mO za6eJDE*e(6$rw;uNS24M+Na(1{nDAS5cFKVdPT5N0bae8U%_WE#Ah z3el~r3v<8QbKfrjpW&Kx#4w>(TLH!I0vi>!S5;IzTn6peeWnD$O%G=5uVJxr?qsDq z&BcTjvEP7V&~HCEY+y0>co^G%)NakLW}^6tZY5U&Y|Bc6CGt8@R@kBakT{^g(42cn z{y~?mZ^}+FD>-KD>@xeD>!v10N%o2#%;AtxvcU-+eTx<=NIUA_uA=oGRuIEIfTp9i zBUM7eiWTov&GGS9Y`?J5k4ol+Cq?TsUl&~&uycU9Tu8w;H9fA%zmF8$_bBq;PAKpN z=>=o93Zwl~S-(Rz*`MOAe0-g;GVKlOk!Ns%YP1WHc{T+9xb5=?G+WcU!zy#vLpcETIwn=co<%b1-?$V_%a z#yD~c_6~90??o=68aO9?sBETWVgACqRe?GC{k5j17MqJR0SU=qwDMB$BIp) zmH)Uh1~yuUmD?ckjt}m|Ok2sM+|KcqXyH1=^Wk%S#gAFWapSuEZ>(3l=-Gp)p#)oh zo8oPftc7-ExeUft^w7!M$3H19Oq@N12{;Mr|h)tA1k%%$46{sjBk4P>q!#CLAX(BYc>|_(Kd!t|GqRG z+fu@7+ZASoOf&72X*%Bjnd>ed{ocmr1&a_~o{qp-Xjh0q^+TfKg;b)xJ})M?E8xRv zXom68F_9NcfXt-ky?x!&vk7e67r@2ST7?d%5)){Q0!2{HPZ=b?3HMP|X0W%a1mEE~ zC#N$6w^2%)V0dz+MH!S4IH1lk6lrPAWp>G7>UCrU{SQRm8M-v?Gwi?O7VApFH|Vk6 zFoMwp4ZI#~cVP?+4dCF+t#`VuxVF08A-K$C4)HM_YI6$;-V(VLG=g3?we=Z#GL!(i z=P>rA2E&l^p7rWVA=3Pd0nDB?L#2&;*Fo8zM)8fpBSFcD*Gk3%f zb8~%Q5Rp|(EHF1Hrjyv%N^ej--++7lf~mNfZekBKD4p9#m+nTQc##WwXGTrsqc3<< zh&0Xs_y$Fn+>w6#T;yT88N0iI@JqO$PHA=N+T!!2n8zF}j* zyq@&&H!yh$RKjuMBv1ivCIjwsH#EI3AbMDu*6^+rYw_H$?02V(`k->Nz`^dyQq@e& zzS$pnCR!;3ueMi>k+L3rZV`E&gE7Fb;|@c-1SC>bCX(*_+VJ3~zJi3TJi%j8JsCtA zCmE6I!orUAdt2aL7xc{;)59;d2fi!c`p3#%Iy z*bun85K5F`Y1K{Kdtot4pBEXphzv6a-_N+uC%JaeKo&189%sHYPfwFV*7As7YhS;%k9G z<1=6}V=utk%flC)JESeiOeP3i?ZHG#!|8UW>UBIw1^GJX%(P_rACg8abMLQ~szC^3 z3bt{1Ri%ketTq1xlY!hdvu)0qf`!FDD`~47uxjAkL+nO7w{_rx87lrhzD$h`tn_UQ z)wYO!RrPWd-(rp&@v$++Y-%>#C2@l+CS?BXiw1zHmL-|AF$Fv87f&k#+)C$tn0EMk zwgKD_7}rgkao}EJ=FBQoY$a0`_6?9aRr5MOl_1BqZ+pCXYYL_-h?lA;&|58#O%#Be zCeV63u=7+8IYPvoo}ubAnudI&UH`6`$yj4FJ*0R$mu9}chPPLglQUE*O)^?)iJC9y zpcd2W=c7Qksn5zZQY^zPTqx4jK-3Ct)Q0R(0iJ~T9XhSx{7pX;tzFl2?fEnDo!K>_orHws@|9;DrO-lZ`$%e|&f$(ZqUCa9bg{@nkY*~G zHn$WP)|GTrIYYAxqMr*~i;K&hck%_z%SIu780yi{c9+TM*iCo)-n06X&`V-W5uajzhGRLp1&9A(B#$93|f>PzTwl4*UlEgF`P$=2KB$e;{g?uOH7)lDW3aAV#RE@s zMRoT4Bfxz_=G0QLCGt%VhvT13K4!B0coB5wAbV%iDQu6342C783+tyxCVB*obk~f% zNqAQvw&i%5NYk8Se^4h=#38gsy!A>Fq$ZLL@F7mwzrbuf%ctAx<{ufkWjm5JlR-sQ z3|PLF7F**frT5592p@W&RP)4b7hp0^PeYC0i*$?ouP!1QjyS)~bo@f17QVlgdjGq; zJaG2Yy6eQG(@eXQ0qT5rvW8m4;eISbH=2n~X4F)`20D2*1AdjWcm`p369bPwqDrlgN5YY^S(md1Ler^#){r2;3Cv3CNo2+|IlJ)*&04m3X7j z|8+E(v$d8=^q{rbSjsnKymaZ3Yx2Zd66wG5QYZ_bviE=uI1`FtC|`b^(;hAEC!Y1V ze#+3lHR5~wy)>)s%W8k1F?ABw-Q(+p$4l{29&jiD?>{xMUgeZBiKmI*7 z9-4!T@PEN>qs?(VJ}|iWS&0Tk6_4NqGa`&wte5cy>#!v*C&$IbZG0rHup7O@{USQL z4H{i7Oqk3ZXD$%rYzLF6+aJhreAe>31=u_v9W6E6&Q7_>=~6amUxc3! zJ9jfkr5Zbu)-<-YC8LrWuP=kOcOq5FW88j?4ouDB;&D7W{fSBVvbMIo(=S!9TnZ0Q zCU33jG_o%xyU_U}4R3WkdI}1QtWVC{y&oj~R=%9V_7eQvGvDoR zT-B>DRp^S`coOYwQDWF)2pWxVHIRzC(=8Waro*%+$tjZb~cO2gL8L;m99 z#d78nNA4z&qH48*G9x=?a_&&jePLeoQkCGHKde)+)jy*BWb8vs@rTW)yfoCt;gA~p z#B%faL;^cV*!4ww@dbL_Z(M!FYs>|njyzUr{h_%#KjdPMJTI_O z^nPE|&&mD~U?|2kW#s^WeB04>Cq{sv&`J^z&C|(9Z#DpI=E!LHOBBRX&!}=UrqhE2 zrPB)9BA z(0q!F>gLjJd_UjZzZ~sI2PMYVO zKfDs}?>f|nBXu{D7RGQ)QGo#7#{tf33dGeCz4I3(1Ef|z6wI8~f{X+p5ObwHA_krH zOynQg;!A1FtH{o_w|0z%tRJL2uVoGUhX#{Qh1~24$@Z3j##!$2#eg#$poA zh=3<<@9|ECP7y<2AB6JlYu>@^)Z@61F0#)}L$ZUMv37n}M7tkMvaZ+q z3V=>rkz47B>&>Ae({L)-*R}%#RFD&RBD85>3R+`N6f8RPZV*xG;7$cFTp%bG^FGwT0wWV$#aV`!{!<`AQ`{c;6n?C7_(_Cl00U zse=PP!DYeMG&Chse0-_y6O+@^X7?sN!}egQ_2&+zj`n8xp3^pg&6^_Zrv>vpC!=Fy z`M5Lryy4MGkS{t|XpEHB)B(0gX>iEQ*!v4Y-C;ZexNF1OTl}PgwsbJYP3`;v9R-Yr z@gX6QIziUZ1E?Y%aez^M5hSGB18SWAL4iVkYR?O_rXitt;|EnU4Ra}Hf7I{ct_S{i zxtWMb{?j!JL)Xk(Ip(a=TkJ&t?|gRZPC-d2@8-6L?s(9gy`qnZ_VKLj`&^?J)~8q! z2k~flFCa#jw9hPg>_}PG`Pq^n6lEq4tHo!mK{y?`cxXFpl zYvs^PtCUbYm!iHfAm*CIZ>;=9O!Cyw!4*j~6GcyJdNfmkeNukLOl}%3*SKcg)kqXK zTpgRf--wy&1=+Ll!g=J_{vi(uyGqEJu82KfDfhvLNQ`ICp0QQ9Z8oq+G-!q^e2VTXu}Mqi!X`6L(8ui#T*T@ zjnlU)Sy@?*$Q+G}4SU*C=Mx9ZdHMP^7quQKoHPXn6zGw}l_2a)^67=1I8OQ=$N`PL z^Q7I#m4e5X--|0GEJV);tU)TfN-D;hDoy5g{x1rLcjFdzDer`9$YZL|#P z%+wuUgAmBL~cNhgUOdYTj*bZgSKU|Bpvde(_fYS~(4SQLUgtMs1_8%zBbA&PuR#OBoSiHP zqj>%trf+GUWlxO+zZN3}i0FBHra{|qwUE~jDfgcH>qp&cIc{y{Z7gB0oyA=TSXG`- zN^LtC`5jrt$QZ?<+N|>8s0%sFV{p^I*b8V|a3(tAx7W@u^Ee_<7kF~XAonUiPRyrG z=-J*=iXZo+so8HdjVgM~oOmT4n<=8wIPwx_9wfIm3H~}9G)64hJR6P9*Ly*C>Wz`y z%tg+f9h8mPXT<<0Gb@5l`(Y9*5vx1uf(1UpK1o3zGXK6sUAl$EIUz`df{Ke>=7d<-Px4E5cTKPzzM+Kd`&XSQ5*ppz zhn)9KngTG#yjH#(*I0E2WnEsQtFTKJoor2&HYA!-2n|3+f>C(fy@#>&*21DRe)x*> z(!)C{3c!=aR8G`T@2>^-s<&PckB|c7*RhcHK*!h=1TD{>Gk)hgjfJM?7xf3be`hJk zfr`xBdDQULL~I#q=>j@;mr=|QB2lVW3Tk7fBL1bIh-GlVg$Bt0j7ye2d@!|D8Swi~ zm2s#nn30`JrLSD?dZ|bLTa-K{|D8xCg9zFc*v4{6@=Yyewf_ag#0(m|%Ss8(H=^K; zK+MGB+nXzENR>{Ew3Q<#e|Rojp?q}4-mv=kH)`l{zFui4>>#2HNPfN+K42X14p+8Z z98a${G2E0a;ngnP-rgSX*+Tc&c9(Ic!Id}!&HJRW&W%()OVs&U@=eaLEi1!90ha}q zMVJ1TCdq3nQetf=c8gwAk$s=x#n-|m?i)t3Y>}xQ%;#CsFtkI%-+tP4jD|`GXraJW^oDe#U90=*A6K&Arwa zU68U z{c=?QLn&2IMPvrYtrRXK#7bx6mq(KEPhl7Fl(h{^H{lfq{Xz z#6(0^lIed(>|Y(8m@PylDuN)Mic1w|3R0We7j~RrX{2Av+y8anm(a-#pu$5RGB73H z6c)y;!7J=+AZ_->d}>;j$sl-$82@b;!$Rnwr9!!{)JD@Xyg?3e~`YJ2Li9T`4EpV3pw}gTcb<`y%cB z31I+w#~ixCuSC6;L@m?wdyy@4MOXIq<{Nf)kak)8_D&-Qt_>1qfX}}BML&?72C{vA zOk96+sD)~&|7jDp1wP&93I5r}l1P5j_*>L^TX=|x%$%W=&Jkg`t#OQ81W6eUs}~BV z`Q=`R5J?5Cz69dB+s+gDpHfMWBO=i)B{U#J z{w(6JUU8+GxL95RkP3s*d$pMNaYa;T=N&0jl5xKZ_jAj>fr%d5!lYKwi79(j9qs(A zti@$^Au5|*R_Nj&Fx^wAwS%M5w$EuW;B=8AJXRS*og58RRM;zt4vi#&g24HM8Z?vP zk{c>6PS(y6LrWtNEd4M$_w_kdRc^`Lb8{2sP>zx8@A~gF(fPgltD9gH!5`umDUX8K zGjtDi?hOHJ*~P_McLZi^mM%sE_e4?O^EE;O4`Xg5EZ;CBF7)Lj-YJMH0 zDr|VCWX{N$aH9owl?C?xpU6FbGFi9)!6EHFPX6=R0_tT55p9WAo4hpx)5y5db8-kC z67<@AYX6q9n||h)74Mu7rv9o~EB`<7{>h!AX|76slit6N{*tIahiIkG`RK~4P;nJ9i?gz&=GF78b4oj(g7#~f$61B=l%HZ`@+^1C(U&A)(w zQy3$#ILaGwTdj&~6ToBaOvy>a0`;)&Rwh>bc*)`hc_JPkE?Q?CncC9hja7Z31`9&~ z3terP6?<4yA`{N>MOs`yK1-)G1S~;>PBeS?Ydd1Vs zjt^J~5No;F8Fe*_pL@(M57xAA6n6wY6a%LF&5bJ_EZ@J!y%!05HCFqSf@XATa^hd!-WxP-3pML%|^4tXIuMlOcRn29kDwcOeV<6o)=7LQQE$wp(^{@UnU zRG+f#cr{f3`REhZM!zwigK+~mzKTbN-$987=D`YN7k5kOmZ==3=-AlWIkP*XXtD>p zqVkH3eifQXE31%K!u`HR+NzevB$1wh^)lgu8Vi%&nsUQQ0pd1th&@IE#NFDpRUu6c z$IAt0kg45pz|77Y1r1ZC{!Ij9SZHWylhJ>=3gXa~mcJ6MqQlJ=GPFvV>pZ7)Zw2{C zUA<`!UoyqvKg4l^m7Dtokf;!*5M&kk7ZS78hnQZL2C9h z6Vpj2u_0rHJ}jHH%NTcQ?pvc|SC)&vym{n%jm+DEzQ?Zi$;kl(xIE7!bDg=bvgG

g?+S;KX>JV5)Pil(5YYjH9j z@g-c;I(4~sgiBUF_+y-xy>>1xRS(^B5?=P)tsk=-sFR}o(cf@yejryaKizQ>c}_|V z@z8TybBIq>RV4S{AuT(z@;J*rh}xMqxNrg=7h)ouzPoXo{0dC3GyI_RlfMZo#f)w- z=gOSg+RzgNf%Q|ptYwo+F?o!qw+dGZSe^%2gb0M?gc$Um2pEHgE71l=7yCSO*Tjua zkc!3$L?Z+VATcrV`ZXF-Y&hIdt+E(F1sY?r!VGuL``kv*CONvRi=+uIlA^%+K^8=m z!Jf&ax&HG)*L|vCL(;wDT4|COk23sB)=x7FS47}LG~-R^U_fOzbyxY1iZbkIRCAYHyywzJyI8sOSh=F}`hz-Wd-##>TN zU40*tB;JWWxfp}_Gf4H3b$N;6PIr-}p>ca-BXmCdg?r3wY@Lht3kCV`*5QzE<0^Z!nV)F;<@)K1>;Ktfsa3biCJD+T@YD`AvQO3c|>!rVKW&KW|?$) z{qLzn7r(?Y^}B48-nIit2p-{lg9lg3L>Zqe15v68%&Dm<3Q5N?<^SPWgSh{JFdTvkH^gA-%tc%GY|z0(-024i2>31%kR%yH^Z7=rS86x! zrC^lV3_P@f5PwV&q)^RhVVyOSvUf&qp!5xJUo4PMA181d|2nm6>241Qw(q5Jd3Tr= z8|x2NcmHMo^&}(B=Y2w7ZS04+b!zbN`a@~YQT_G*S*EGZEa323?|F_a-JWD6SVYXGIS&j;&n*<5k zz{O44>z^TTi=PV$&MqI6AbK6(pW2bl61X&wNr?co+?y!TL(YGV;lg-Lv=0eL2zv-Y zjGz>{fjpr=P~88@6m2+|EcCemC#V!mN*w@L?JoonEaj!xBJ-J$^Jt!yQ&g}5L;lgZ zIxV$y1_4<3$~6tF0!J6!T}~EP-FIYX@~5xh7g}XjGRP#k-W zb-U^|=kw`+H$misTdJm47^#G9JTG?Jb_j;9lfx!djHm97JMp_)7kLw~9pM@tzfk-o z-iifXB!SyAn5JSQ_QqefJB?D&Nd`B6v+E|r zP+Gr(fH_I@Bv$`9lA|kYA_yGYZSNfKHGC@ows0Cej>D-cqwntAi*>f#cWutg6ps1u zzxW{ONafAkF9OqMCJQFp(A5!`(|_1=Ko^|kzNVgDHCZq#MwY+?>Clg>JW9{e5r(Cf zjQCdGH(r})s2WV7A$e-TTW|z#zvuVCtey9kfJp`uCE^L)Z$D-3X7XCDznw;8WrtKe zht+OCH%3-^PBwQCgqO5jgemG5rLF|b>-gY@KdS&#u{||fD%SNFu~2nG^E{hC`*t+V z!%Sy`RZYqfs#qeo1WpBuq>?{n2uS5tVtqkUEU@&_j3YrkmGzq#m(j zH6kM4)l1ROS19Mt=Z$Puu?vzT9E9TIrI=plb4T>Ub;+JjZCR%|A0609-?$N@lt1c; zO&^WZk=W7Dtt=D>e`q9Ax7PzC)dZ+Mqy!FCIQe>5=2bI>@^z+<>dP5NT#pdyeaNJ=?YOQQ-T50oii( zEjvaDHgY7x@prX9b7IR#Sf}h?0@q}nlgLc^Li$R_s+H<7XGNViQ*%lK9zI*z=&M}1Ugy>inxhbOF8v&zS(SA)U7K0qt5K)`gYSDJ&r$1s zDE@JmhjtBW^WXmuk3>kHE};GA26+3M2noT$2%VkeD7}(E>hL{9Z;vTI10y3UjMxF* z=_Gz8=`h%P0B0{KWVXw%9Q^PRzzQnYK8zsnVOg)aVRj)))uT4-nhM&@jkuUL?j6Q+G-YQ|5ImHK=A_Bshnn_#t(hWW-knP~S z($ApN4!8|&hz>wwd998msYP9}-+UyyQB7-GN+9iX?+x-)d*#^N+cLc@51=}0_)Xv{ ziTchO=EKIe014bvK>HyC^bdJhf9d9+u9}*^PU9n~VUEHtpH#N^j)Li!yRHO=TM?}? z8)?qJf3N_~D)o{j{8_3b1`z(C#K1y;W4~J(GFt9hJ~tG@$LmxY>ilRklqjY?Adqac z_s{^^klhX{<|WhPC@pI?CGl1|hfaU&ANdRr);FEMlc_TuqB`qHDF(troq9|!8Y*WL|T_$gU^&@})< zqS2-9##>ilBbZmNN98lR-#?^_8@Qa_ZDKn^*gX2s+tu|bO0ihL-$-j9?W9t3Ett*+ zo$&eQ=f~6aj%*0}Hr`2#4#eV1 zGy{dGZtyLpgl@R&h$tp)Y*_5eGt0=(-G6s_iatU)w)kfGYdNuItGAR!v36i&N^QF} z4&^b$6AjlU@8^AEhFbV=rW|3SFZar4j=-1^^h+RU`Exax_c3Il4Q0`nfsx`zexah9 z5qBl^Wk5RytAnQSbGivSL*xK*lh~=Q^%^mdDn<+j`Kt^eL>-ZLD;GQ*21D@fma7GP<}% zZW8+TkpoHAWWFfW0IjVJnD8iQDVL!+KRsK|*oQ`VVMt>4zJ>-_MP=nafU71zR|DJm zJ>!O*0}FJeBjqL!q1vuDeDJvZou9!AUK$upZftC{dWg1$ilg~LS%nf+!aO{fsG=!A z^su^8#R!)&=eJ~n{Syf+=^?-EzkIwCJul}NuI1#SJT!5=3*51C$F~_`zrhzVx&s`Z zNKoDTt^%!KFrlT;B&gug`YuihP5pol?C*u^?e7do?1QZq_yJ^xDou6m{2j3n{t$s! z$Ky~nBWNK%OedlqL7!kvZ8P*5(*YB}pXZ)9#f{S5Cccg)@L%fI=NbV$IXPHYwUG&7 zJiz>;{4OZ>XXF>=;b6zbAzO>2M`MpoC5qP~b+MqoQL^W~ch1ZQcQ% z^U=37APJLn0%)VeXG)(VwT9vzSUFKvh)r z^zRPa_2exGa8oYyh42=8LodV&g(H*6<{!~OKrT3px0ZtCUE|rl(H5!kB6=YMdl+wurh*#Y9urZale@>MQHcm0Rxfs7d1FL zC7>%E{>My%9HR#{11HN%Mr1JIv%w8mxg5&%=F4dVp1u324;<^W92b+V8oz@2Sq=Np z>BcXg^Zd3O-i-1V~`=7<2S#f)MHN!-26+86XQ(#8c^l1@vFQ4fipoV*WvnvO^;k|ivLNydq z0ZmUK(X6ykyN4v|z|_kQ)^RwREr1lrUz^FPA_W#FNC`;C`li1;k~hxB)>IyCut< z&a@_cRyPBP6;daAllAaexeu2;Z?kQm%!uJpixS>{=kJe74G?u{S|YDp8L)&%C=(`b zu4k(Wa`e5++O60yjQh+awBEkpG?Wx(l$_a7z#UCZ3aFPgujP}lSey<#dY?w4dZp`NMW|Fu1+S${r?&6X^_1v z63ti6?}Q!)3}~U1=353tz=J|lkDNF9esa;$)ul!lg4@s)sKt_BJFNjoJ?H*Aliuj? z*BXt)uL^1I7B7BD-v(y>#hb`SSCyRLwN;_&*W0}91Ta=OU5v==TVfb|eSO~WsjF!W z-^ce>pIdP|UdpT=fRC~j45I7n7=Zt)=xTHCr)m@my)Qgsy#4#F$0wmgqiP9&;f);Y z?hpSBV%`PB^1fN%Cgp9VLXxsc|981EZJ!<#v=@mTAW`|y%iLTc|8~Q7Y6Tx$Tg=VO zjG#wBY7^iOL^8CWo@GcgtXv*%OF-oWtKbtd4qJ6Q&5j*bCLD&%x3RaA;>sre( zO6PoBCK+m$yzbuK05dJnc0WAtyM@iWfPr$OTAz8sZ{bz<$4kfzBVK0n47d!&hrSUI zj3g!8^!3c~6vEG!$=5(*c8kMZ+(V_rz%FQXJ8;P2R^PlhQYRL~v^ZpiHZ0I1u2!Zn@RZHPs_DNDGw?GndqGY>R95y5H0 z`~pL?Y_>qh@3&bVE_W99HVaETwr+kq5I(#|Eokvx6tj8=JwpZ&Hx?!TFe=;(+}POl zN^GwMppAiE_QAm#ms}5@i|udwWt@+Q3DUgqZJ;YS%hiwbi{y70J`FC zAX3{Z{_Kj7^<1OFOw>mXAFi0EQMz`zoIB7Mr{EiFua{ygrJg8@2%dTi)^^(A4;rEpyo za>(~(J6OdOKZtm*Oao?jjP4%kKX~4a>%D`y-3J{P#hyw7Asp6c)srqD!+c-gr}}=L zo^J7j8%4@}&Tsu{>ZO|b<3aX13N&aP4Vmdf}jFaU!@RYD<#B8=|QJq;S;fqup}I=L*@{0&5ganzocbczqUOsn~x%+sEWz%B-`q_@z9X{WX_`9)&7=Id9VlwuETI1>d;zH0rr}z5pChZBd zMd@mG76|lCny&5~{jstI_ja&Z%Z>NJnh5il^mP2VJ`!U{RqxEz6*iC}JK*kqsi`mh z#M9tsW{)=o2ms3oUtE~yE-6nGz@_=@d1XN-$`J{8JR*S1W$^l=s(i2BhCq2^ZTW-r z=SL7u{O;@5NB18*$O3vYXUK;GM8b#%H-A#-R9glH;*y8Wqjrx;KDgYFPxDjBOd2Lq5S{Z6>XTxCG3j;UWKye5PODTNzV$76U9(rOkzmXn|0cz#{pMQ!59EazjFvJk5thFjXvrCv?8_P~=lZ z1rZ?4a=;NFy?*c>N7z4Q#+p-H-!?TjXOfblIXP@5Fz}I3n5iu3goe?fAV>FNe(|kQ z-1YoZ-(rmE50jF1S^`nYrK8R+o(G#vtf%#MKUBH(O1oOg)1`CV z?dhvLvbq&F#YkBJ9(TT~t|Q8&LV#o-&g4FHlULubj3*KF$Tc|2FAe#VknSU-We*5N zfmZ|ffWI3`i~H6u`EMf|*|8wd$#WnTM#0kZE@ELd!JsF)dah@J2uN;>`^?(e4%86d zPjR@@%9!q+=HeRoge&%1a#+4tphJDDewGk2`!BKZicbFg_QZug;Q|HoZW<)KhN8Fh zSqArSlSVf)gq{z1Fc}F>LuF!K@wmpqWF*A)Cj5tn2dxEAEv9$VC?4zP{ZI_CN4Wm& zW6SN@nji+o86&PQ!ZMIjNa<$81112$S?|0w*Am^?D!u0Ao31c>3^xw_LvMWEC%o3a zPq<@BTmL3Nm4`T&PeAF$wyd|^iuiX*toUo6IA2ALg*?>p<9J9LX?&%02Ax6FIVLz$ zUT*V1C)>Q!go~VL#TRRaI+AmT(Y>uSAC945(dBGF2bM5<3bRq2604S*tO@$P?lo}O zcLC=|X?^&>aO4Lmn*z`oO6CX01aL)$Q<@(r_d>jddr_|V=pRYUoQpy zD6fu9R^~mpr=)U=0LStE{rkN_6QGQHe^s~N^oCZwaWoP%jnMMUO>Yo~5I|9(`XSQY z-8bxLUv?U(s($ZRwc^dzLhB3w9sSW)pT9K6i5)osXWxE`hA#0)?|8Z0pf=n}0b=Do1X%~yWS;2-hqez(gwnT$PW)GB& z#lq4SIa^z9U@k{sklA#)6pnm=eOB~mnmt62vwps|(uba4kwyAyd=2YZqZ++&B>pp& zb5LfY$o_S!HI@Xk)Z?#zk6zeyti3dbV3UZs1Y7HrSC?TJEA)8D%g>z2LoTFY`a3fx$Gj>>rF z^cGM49c(I27GF?0F`x^NdD0U}vcwm&ms|Ud=tP04+Es2nRe=#L?I_bQJ9Z1k#jR)Q zCF;MHFz}J>-BQA0kr5K@3#JCsgT#u#y&cIYuj(%-2?JCPk1m;nU-;19DReUE0M7%% zejZPfh;i`J_9tG5??wN=!bq{;HG_5 zZeR4%un!91sTB`MW*JO{e!V~Jnl$;_Irja9lAeS_KgAV$IYG6FcLzAZ@>UOaPkCS)L(WwgO0|yQ0=BEb}_d3s} zjrQUdzj5f9mwj?2gbzlrWd88&mrG#lP~Ie8h%id(`rNW}zST;zKAiCo6uJ;>gNs}1GU&kZ!>onj z|2hp|pO8=Mk9q8Pv;BWtrA#2uKq^{0Tpy$15e^T+UEB-v;ewL`t|-AU%8}e_Lj#W; z30bKwL3X|VYQnPUfKJ7YyOGMVHQuAnsugsI=agJZ-#8prx^8bAK0x?tn71u1=d*YX z8Pt%mI;xCytsj{5&IK6H6*!W23qQ-?k#o>$mZJs@uJvuTCOLMC$rj!pFyMw;@iI-2 zAzuvStsqUzMqA^>tvXl#u9U#|2z)_`imIv~`0kO`EXiRLptryF^~t)q)#s#+{=Wa_ zO~ok#3=EZ#<61BgB76ZYxnkj$m-+c^WFk{3??JU3G2T~XExf-~>NcHl46UI)$EOqD+} zFc9EIB0F5@DC`^@@>}C-ax$PwBViDj7l3UJFeZRkBGrO_C}9+7SUqZ5%y>1V5GnVc zKN4;w2tq*%NJ9bK1|(%#yFG2tEGR^1q*Eu<+5TC4EaFe7ST;Of?x zVl$pGX*38dSZ`4LCGqKs*7C#+dNEWUj*p&zNOm>#qqXlzBT(@_=Z6KMQ};4(0I z0g`ne+EL|Avc(9aJ2m|Q59owjZ#)zfXv4v)`TAgS@FX}OWbbH`!B_^s8^dFa&chED zCnkgSBj1zEG&R8sb@2BM%S3%$ykbOmWUX*}q63|APsyU;S%b$MtYf(TlYM%J@R|7}>-iJ4q1)e$c5+dO#4Hgi;?Im=O&8A8icxW5 z(>e;_EzpJccjZ{)z;Ghfh48(J;9&D8_YV4b-eey|P+^Rp;#1<2=aGuaiszEq;CE3$ zkMxi;aIBT>Rog>g7uKg$aMU#YtuSW-@k)Q3D!3EZtWjMs}9ig**3z= z@Z$3q?7NHriIGxc6cE!dYA8HnLokd9MZzGu?a9x0s-n*~`hZ%Io%y>I(D12LMsG z{zxI2fA_QIw}{Ft!^o-ff@@UkmL!u`Ya4cMB9Mm;BT?jdC-(BZ?Cg=Pa0(nIx3d0q z;&R4+lnxjTVtH9?U^?eZYR0qh**T9n4ZPtHaRabZK)d?2;DbLj%LGbl-WK=%UDnky?Be0`0o(F?)*iIN*c3Aj! zbjTr{0!2lCMEY&OTmz4X>wY)+bzutuG!pgh+g0HP?%tXd{Cs_ z4aopV4lf4~_@CRtcto?&xp*U$tfv}f_&^j^_S!ZdX;fK`-GDE_7JAJl%Yd$a8A3x| zyFSZ0UA5jUw1>)%Lc9{DBWXMcvc5d<3-4|>*YY(gGudTOY`&S|vh?QlW$jkbpoD^u zdShoOasDtGA}Uo9EmwQmunDzh=T~$$$3*bn9XJeNQ=9lU8`IB7; zUe*{ti&FnLKFgo%q2|Y+6+=MGoY!?gVeCok@z!jk2r3HZs*sYRr=u*PJ?yNch8f$I zFv-%)8E%<`F7KGF#b(4SUTa=&ceiRH05nT}w3}83Q#(TC_)D4lXdHu-+`m^}HJhk; z5@i9TGSCt*(><;*_}=-q8wZ@Svjd@tlmIr<>d$%U`g4oAVwV7W!AVCD( z$G3g;Mv?{lXr???U%lmQgHTkuIBxCq%^)Lsb<50AD*Mw79M8X@&sCQu_m?-mmz#V) z(B~#6rP*8{;Qjm=$9;6q9|5b(`4CHPv|~4JayqkdzU2FZf{;<4e?S8tfNx(&maZW$ zvj>Kzpr&I4$~l-38(m-myc9%k;Hi4J6b<7faPLHbRMlo7c%?(d4;EGArxe2eViFR9 zKYSnoaV;A**`2~%jy|E^s9ba5Ku_<8A>x`EuAT$01Rzix>3@B|d;u1BV8DV{pC%hZ zB&}wdp`XRuV;4v~e3HbE0Z!nkzGGB<5vG|gRkyc_JYVN;V=+q#c}MTX zqQIoQ5XwfEP&HPfCqXUaO$`%K%L$CCdKM~pwU7FrbhWj6#Y{+U7|m`kKAF7uMp&_4 zQBSy9qqkR~L9Jpi+mB2t(v6e$71$vM2kIe&g6#N-*u74S1$5-fQwpYrG>!6i0%+O+Qo{( zAoE)7F3|g7uTYHN$ z=l*hV?QB_+tL9rtvPj(7&&O`);DNxXteKmPLV~XKd|o+J*4y5^#MpN^t#wv{r&;wn ze?>^@(^Qc`zUzdO9Rg{SA-t6i0kQveg9)9>`yq??rGf|x3u+eM-inHKdzDN1kYH&9 zMHAdS{<4ohZ^4XDAmRlf(?n~^k>TC-%wseaGZ}V8;x-fL^q+`LkiN=z+u8$BZ!q;6 znB0%PzKw=}D`M6s#&ER)yuNi;!*4$j`u{p`qj25-v`)s#&#thnQ1WAH{JTYZO0Ynb%!=<0NdF!|;HCtiOg| zMMwp6!Od`)mg9#2KP2aoYD6~%LE!-k=L1EIJ`QkeFvLct_lZJiW@&t5I;;={NGF`zuzuF ztfMSYX^_4pfz0EBW_{AJ4z7y-6 z?b$~|3Lf~>ftBMUiiExa50#C7ZyI!6KtTkTT&T@}b{RF?Nud$s)yeT1|N2q0=2bRx zaZVAD*~c;7+`Cq`6UM&5jH){}*4ahu*4qPH`xq3VpNW~Wg|src`uVA* zBdR9#tao@>2s1Q-QzD-Z$|Tpx>4?p|H@wt}Z!(^JVIcg47yqv6k_(gTU4RUs+U#Df%ieGE3=<=x zOy{|1=_`@##4PsxDuLb$9WQD3LyI6LzQ|OKM>{qaCMGE{6-CS;3Xjxc&1&9$wz0Jo zaB4^paon#UOK8e;a~j{MS2^`Zr%!5UYGz$4(H>70|6r9%{x<0g)EUB=Si_jxaj2di>jfW3 zk7lJHKIvVVG9oc1X{eFyFq^rze&Do%Z2IhHIMyx1wzEWbYz9?P#$J+n`dR*N1qedk zjSs7PYJ*PflNQZNDm}Z_=Uo#KuH7#Z-u!6U`W@}k8a#cmRo^WgquSSd=cWi<6^~Ro zzW1mn&(^Ht>+sM1?`}VJaj4(l#VP-6rmz&Af+fJrJnT3>=U}P!YyIe@efgqW39QF~ zaWR(EacdaJgz!{~%Oc9&T1Ld_D^-=vmMeYAlwHdqCmuET$%!44D&O*m@em11J=J}e zdGD`)*F}bW>XH739#*H+S&gL98A_$k>q+w|VxC_e6iu8rM?}=(&03rcLKs>*#4@^0 zw*|+R)vm)vC!5rZN0;}R-E_tGMoFCuaASNs9(s>{!9mC|I91wRlZ6gnb3ZE0_PE-= zOck9KPU0}{IhzYG5|7|lkX`&RfzNGCb)CTb^l50iGixi+>hep>sy!@bF&o_vkMp}G zTX@`xkN$GWN?#8YHi7SV5@Lsig;>I$5RgOuPNMIoZ6kT}=`+aB5;O@;M@6WH9&FR( zv$`oK&ApW{7n=9@-oTg@Txu;A%VSO1R+q~DbISBQm3XsM8QVoZmTtE`L}%N8b*y^0 z@_oji`owKNQ%1+N$}jTBF=!Cou+8>d0McdOB_*IcKxwx>|8|Yk6LpEVFn1 za_0KH>%NB1CgYJ`ZKdj(OW{wc$a<^$J62CU*9P^N{$3983gDM^tjk345i!wu3Vhj? zl00#UNI7PXk2Culwf=&yV<0!w>K{Q~@6^bhj?Ek9wdTXLJ2o+5HY1B1-!E7e02Ek< zz0Lm!d_*WCpC~FWLTw?iveT5oM7pJf=l^s+i@rzS1{R=nu)zm6+_~nnw6Z$eGPq8L zp@rN9BdV)*gh6i$1=Vk#+AWJh6IBWckk&2CyuSnP0ji5T?En@1NZWCeb}iN z>rSMz+f9|(cMq^mzvfe8<@5W*P#czCKqOztYe$hTrJS|GvpknOQCyX1V zEj>thP{`xiDJp|-l{0Y<^KXKn-}@Mouc}yc`YTcmjkuh=$yq`l6b8g{3Q&fRj@%~5 zA2fW*_>Ng|DLqI-xHd&|{)0u#_S^Ff+|SIlqrXjUY_?nPTT8={1x3Qz8q8g}eQTyE zBXX794}v?}*W!hDel=!D9RD$4C;9PfV#YRaWJxq+5ao7c_lCXO?QmQ{?LW^+EK9}| ze^@y2b7NPDNeODliumj4%cM*{#VI-GawZR0#HO*OI}xsCJ5mpx>WRk}AXPBZCB#>H zM8)MST+MQWJ$d?RvVuD=b>{4E_yY0Q*Rq6~^FMoB39NdAvp??FwdH5{gt|9hJv?!P zU95GjnSAYXO$CEpdgnUqN<ZO_!$q^zm6 zn%6t4yXZlAHO{}+wvB7=BEKc8Hxk^TT2qyB8_&hgog=Jc4c{?8F*hKKxjeV?iDE3g zLHxVgI0`?@!+MqkwW;v}v#U-*&Z}C3&9T%-aRqBAV@vYk#W~2Xw5;Ka&q~*%SYtWO zdtYCkSa~&>uAaPjjcKNlIU`z0B^fi7Xr* z?VDu6HE?a~sp9)=XK5}-o8P@AzR}Fcal>WT?kQozdsF#B28J$#;_=o*G>5np&EZnM zJTFB~xA7vFK6>Jm#o>FC4i?j*qM{X+1l^9lIJWDDC8zgQb70FX{Mm3PKZU7_1MCU= z`TOm_&=B&oXJuvO1|M7pVDf@z{$3VTT8)x7b(&v;+6(_d1gZm{Q+-yQ5onubLAw3}lMV>mMUP5kEa18YNny z>&-#P%sDdT3(uqya=0mP$?MwCmU@W&EmgcQyD9EQ52d7U{k&9?{PUK6xGzdgU)7I!znEa7iejJ+@&R+uroe0Lrfl8bwr zD7IP4v;Mjkr17&Z* zywKK-Me)ayg|pY3j|-|4EkUnLLk79t+nD$(VEJO0%|fYNo`nQy(4nZ5PK@rJjGC#Q z{;>CQdhltAJB33peL(0KW{OKkAR9ln;Mb>2YQ zU!i@a_vkQ+}CQkJm(f&%WqevF^6|8T@ICZ_&(J zzprl3F+YD~Wn1A@lB=kcI!bA`O4Ny~=oF|R0-p`q1S$i?NAuv`RR8golY?h1M-hwQ zmSdfKEOXTD>>KB>_d^RI8@ua6v+iMlTh&N<_r_VTIg_=3iwuX%73+|)2YwI6-_qru zDr|GU9nt1EA4p$dmo1!ktuM#kWV}AskI>VREPWO>N~Lm|Fp>4==eDC=_<#soZ(IbO zbb4+O&goViduN(Dw<`1eDgTh6=3tX`-2FYpod5W-m8`7&5pesJ4u#uzKOT@70)-JOCF4eG)^S&WC}6W8}Kv6U=K?k zJfU^^sVcc$HHu_41=!fu%SflM)p*T^s9%|yf_9b!m8#wFJ!B+XTiLDu)N-8qQL;mN zeu2_*&_`Zm_vpCf;%w!%U~=P0xQN}nLg8RLPwQ2DIjs?8e9@`?^p>D@k#3D8jm!3n z?pZDy8XO}`%48j_`N;vEydh)J*rv5>iw+WTf<5dq!{16fR@Q4lgVV2hwd#!e@s*PW zLod%y%wN5F6{~rsvIrx96(F^T5)8bP;EgU5tnJRGZ7cA&?_bjbvm^z)Yah&l;+T6Q zsd|@|j9Y+eyZ+uC-oI)L`kju>*kIfExQ52_=ih(7eFNbmf2M1BXz;O)+`ZJEG-aUb zp&}YbQw~#PC|)h2=%W}r_>na}cIuxC=u#(9AVq&POMLw=&$$O9=2c2uj75Zl>`&~y zAdTz?-;eL7w11PjX>(U~Hh}O1OYW_ELoq^^Rz`jio19x~=wPUI;nVmzERN!wcUz~~ zf&C5eQPub1ZSVK%U(O#}{Gt2rVV0%miAF%+n`lyV=y_}&SjlOVn<`|EEPbEp><{ZO zWY3Ce<&Mr+d=znYFNAA*jhS(t4zIGlGbiSAj*?~}EyhpR4J>!F4c_S&a@z5?cS&fd zu?sr~^PJWF?{ylxyteGolq>=6PT}dJ?}5&7T$xg@8sq%wD+Voc*%jZ4ry*0r0aQ)> zxeEt<6NCxZd7izgsjU?t3$N@n9YrS>&QFOfegN;j81r$5(D&6HgIF^sPbO7LPG=`c z(?I9^V88F}v z0_dv7!F}<9L0QZrrcP=fm?;OGgBVo@$6CocKJ_*eVFyl4zIsAIRiTqQVp;Zm(|*+2 zk>%E^Ufo)<6}B%%U|_NcZtUS%0M32)l=cqT3AG2wc~b0)J_CHcdvF(Af{Ie-IPAUc ziW#og>@D8=P?coI?)Uc{#md3Hc>GG%FYlR?p4xm#2tN)uR6P+(7V5So7ayoWFGnwM zmt`;gyZ|teI|f2VpktAD`q!^I?@M>&AX#$tgL-0od=K2VMj;Dq?D9?XfvXy#bSY=`{54f^Tf!f zd7|V6&aa4wh)(xx6+(05bS6F)7vEDvWFD(*4Hx*1wA`JS+%WKGRPcJ+vk3{GI&~Vbb39yY4sswy#eg z{G!d~_|*Xg_dPRt&2?m>-u#=iRkUN%Z>_dXkv$9JIe&4dGnB&An}6;Z=ex?0Dj{1s zuYWK1J7?GV32Rx$_Dc=V{;q=N|A`f$c@@dwcj2OCI^@arEjr^7bK!|Cq1gUe$XY}C zb1ok97OK@C3@CIgGA79`Ul>r1i(dLvR`p%{D`s!?6q92piND?}=as-O^G;y?+dr{= zu~~TMSK=#kbH_OnC%wa?5B2D6kGbpL<%rY$zJoYj+xloJqx_=y7({>{xsQ`z?Q*mJ zv+R+tCv-TKZVC5W=QHRLu9qb${zBgUwlVX=QI$r8UkgefRv%ofU9tyY)&0qr^WHap zx3!Lo!x&oaxOA})V=s4gmeE^h?5;cLfspnQ?r5g^xjX|_MbC@mi@FZovOYdO?x}XqLLHywBQ}!=jAuq?nr+i04Hin2 zd3Y+GNS~`lWPbiP;9+i=S}r}5 z?|sho(pJoKhU+?Z4wD!w#p&JICXXPq=bYMhyy-c3f^y=zl?%|5qaN_KZb*5Kh<$)(L65yAiSWG zy`Nu7pH3!O^ds~!kH0L37tEJs*6Ucz8VNcc9M9@DI^bUV*+$npl5v~ls5kA}H`Kd! zQ!1aoqE-B{b>z5uLRBqgo!{T7(Erl$+khBeaNI-){MzM~p8E_PACerArEtf}B5!(Y zd5P5O(U;*NlPIwuc$p?nPj3#sa#ZuWwuzvTh=+{OmHZ6vZ$M|CVjZ093>{}d)9({~ zeC!Mv8`_*?`tHATC6_{&yuS<0Sd}*SK+2~j`kc^&ZuV1Q_<{Dr$kGv}D9}HjX+5vT zCFe3WGBQ$gXD2d~FO;n$V|$TR*_orAe)5Fv#hdzkjPsT?=bj}8?LXZAP#MBvyM(hX zpU&7;@KGx|f3$LHn1WN=u$-;VgdmWqs)3YkVIbZ*Ay`~&&W#KEs! zB~iIHTDkTa>8D3TURU&K=>c(m55o1BN&+%N#;>z-8a|3l$8kllC`PK6>c`Fg7S0cb z(hmO*{2gu`#PSrJ$LM~z1-AEVUGyg~!_(lxff+8Z!#u4QdPx@9Gxgo_3FC3&H|Inf zYmq67>PF(8#e1osZJ@Vhb_Q8FDH!jgKO-bdd0e?xJE@2FkWshQP2yx*eK#h3=Svzb zU1!?*?_&)3mPyG@{d!epIT78zY#uz77nu6qZ0uZ4knGtFoFBx_3Bh@2O>7ig_%$g+ zCQKQ+^V!a~qybE_2ET9n{KcAxIO^+}ZQ&yF+J${rgVPr(r|gl$ z_buI582Q3;avpUaXt`spL2)PZUlVh}mi_%B9j9qC<&p%FcOF$V=o)!`1kuuBK3`}? z&YBNX*Qop?nH{^TDrHy>jHD3_B$)4(Hh}wy&JrJGtC++B5}2r`gtu$M>oB=15d86 ztWXzTywp>OohiF#sPW(y%3jp;4wGWZneb3hap-OUcg$wCk7jQ5)cV0|#>Ir)Dhmb- zJWMpZhTp71r*)Cr=gi9PcRI{Qe>x`dNh>|p`Acco9Uu>My3OZ3>*^urp6zm5X4*q+8WMG5=eH83pHGQs%KF3EPL-(qt=vi)#@e5ct2{_rWTb|m^%UDYCC2vseS6B&ZKf(C z=lV)UHkKJ4Gv?4X8)2$vqKoLrSRC^8=B-{vPzk{LV{K&x4ft7xZ=b2LySq576Y}_+ z@;7b~9JA(ndNg$I8w^Nn7QGG766(!Ym&W_B2fnSpvyv>QVkx2Ps3f8?!BS9p=@OpjF;OfRoijBV99 zY~kBnJI%$CQnCU0XYO759?TCJnE@2pHdEy(923sCG9(I%f6g9@xs1FKuC7s!WrjbF zzr7BI_Jo$#4dB;ntmfm z3)}*{fo_<|sH&*!5gat?ayK3Rc&u6fhUL&*3f5U5C(&R^Mnk}X0Fg6%UjN=A=_}~Y zW!zG5{~p97C!bFzEm>5^{v?)|?r-^@DBI0DP=v(GQ? zbDhK7FQ3vDdNTgp#LW5f9SuR?04P&@%0v2O`Pv@c~v^}T5P!^2M&KL1=%8T~y8A$uyMzXC5O$1y51 z3zyQ2{ANnM-NGp=s|w9I0re%JDfrL-Zv1)`R3?Vs#hO$l6>Wde4ymHLOzk0-fO%4Z zrziuHvQg#(Z$@-nT7aDZrYtNvAtAvI8w(2y;@vP$27ahitm$mpZXJ^M(2k^}C&&22 z_{zmiOiCO|0;9LF9JV|-W0ul=kU*n}PD}DpV8f1@i7GgGYhrGS2r0r&aHHNbqaz8W zAq%4uX2lPxig`m73MnJGeq^Cq|>i4L4A2C1YpD#iV0E+t%04&8-*ljFvI< zEOE0cBm^d;X!6FrJ60;3o}NG)X9$R0xTu$->7Nv2m$$UOo)qLvbOC*Cxk(>h6Y!Y( z=hjTZZG zwpUCd>h=O2hej7Af(LFF;tQ^K{PunH32@gTSxwEkOq>4J01zSuD9Z!kKIz$HuSft# zL32rcQ_7}}35Sm^Mfxj)5FvNIq^~hY*5}*O>FDTjV0~er2G|+4CU7cADVR5d5^`A~C+YC3xsUeCuh~Oe&7dGA3W2B*b9b`d~pU=|y;~ekpgcFH->W%vc0u(T9 zfw%({NUEoe7X`nZhZ*QyA*x4H?>On6M5+xmolk4S^SMrVQ6(@$f>Yfohi{aG;*|*P z29qrFcN6NKT*v)V?4F-PEU6R5|L!r>*-Q>a^fWjF3n+TX2ux!YzbtpxiEn&%J-Z6(LZ=_ z;VVf1W&~}7wf0O$pnS14#Uf%=Sx;?fmCT!1&*@(%>k!1kztoLb#z`rm)JGK^E^>w3 zikxy~KD-M5MulkirS9cUgAAb?f?_vwUBt(v->oq~wtZ=LE6n_j3T?3Crn zw*4_f@oLod=n)i=@ksF{=TJ@mwddU5`(`Gyh6q9<=k1cV8@u9rp{=UC~RjwLR?G7zDPJu3f3;x{wceg4cv zFD+7(org?_V9`xhX#A~@U?Ft!{Vq(qq=VQ&mI-Rp$U~Ws#zUEx>PE(9hX3d0Hr@Va zTPW3WTv6PG|8WG2iBhr9$w|l3Wo<1jyJ5ktm6HL^9=KV!I7k)8WlXh0jOWychY?T% zv&#NVn_*6}7zJd;G)KILo0(Q4a4!pL5ELhcJO1?dzuG=`gA6CTn6jOln{3Y|bwuGh~ea?gNW)Wu6hIyVTUE{0W%GMzV5d_>wKJp_sWnx)!cxomir*FTU_XatS~6GgxBMT$RMQL|kFpAI-z0KfY~R>tJ1 zy8smv;e#}#)M_UX4vqqwfT0~I9~~oZ*t)p%=*4oZPtJ?w+_wDn;t3P2Ikq#Ton(K- zW9N5T%ee_vRPtmpVazM>eIp>0Ky|_|sj7ouClGg`TnEdnkM2av{Jln`z4A5OhWy<$ zq2-ic(up@AH` zxq>$MIfV`N+cURfy|^KC2opPtgxprx95rUg!Fjfq?)Qt6BSb|FN*lF!!VVLsjG6*W zAqY_;xzqa%Tv>EIFEm0^SqrO|Mza|$=zm+8UV$zTlzMsV?+tUztQc(0#5a>_8Py%W z?Y<%egpP47<#mwCrGVs69uw1_@Rj{vH)3R%RAXOYLqCs+69y`>q(2CBi!`p9g}v9q zNazX5^3^$T2W$2$dxY<4CazF2zbqz&5e4ZNWb%qI{yxwTt%9B>Isqv_21+Jt{r>U^ zBBY9pE&msrwAM!!WSH;Vp_Hc{cE$uP9Xpob+|rD|BjvrNiZsMyqo z!r0!b^R4Q;Bqk0>M&wsdD$tAymtu*7{xO3ILkmkiCpS0C*4h*i{Z}-X7Uk&98R%ku zYM+G%8Eb%PU`88B&i=YOt_{y&y4q6#9Syi>}Ro8Fe%`98ow^8}z93 z?#A{A>UrdTw^JW2?IpdZcG|qf-Y4fj$HnNN#xOQW|Hz)+u?o;n}z7_wV0~ zjEt3xbn53k3I%_|M^IarOQ#lO=ks!N;lYq#iOIylAqNiVK_>yz(SZXCt2uGXz%=G% zK|z74ni_Ick6&Qm6F4*qM`*!^pWEO!1e25p8Az@>vG^Cg@s$M-fWww}L#e0bcJl4r z+h=IVFYM#>O2v!%ya5meb;k+8F5+qq)t?a!Y*>V8-Fyp7v4cH+h)&l-NkUXl)ayu` zM=c)`M9EWsM6b78I=Z=7ahv?QwcgN{js&raX%1HOJR2g%f@pkl#X&kzk#?QNg?LMm z`qDrkKRnOgH{HsSDvLT%`TkPMAEXG;4P+cGmmd9NTiW^{1dQX$CIl&e&9SW3eLk)2 z70JRfzP-ch%Dw(qhj_J4ZPW9Vv&mak4rx#wpUCFrDiEI`D5{+eb=V$>;IKcu7s%f9 zbVTM~@m2h|bE)!|%NWU&FOGS4GH0*zb zc<0M-6<(__y(SGA929-r#ib?yUX5@^E6@u^Cntw$umytVvM8DPxA8l>;d9bn&K*x% zuPY7YnLp_vSto~rdy(()o{s3jND8m z;QWT{RpI$;m4bgyiNWNYkh{=-=3Q|69OK9vLYCK267O4I>@Sc>sLlANwSM4N0s=!C zk{*LWYf_C`&E_5}Gx1Fis*Z>#Q$wYD-7BNbQ@HQM3jg=@;ULe5rN;GHfJL|5oxFw{ z%0m7vW~Z+=ecG+M07uhVIb}pP>F5kcWRRKPHt0s1XV(yKOjjD7iuW*?dj|CH9_J)q z)Yiy;kJ1=l&)DDXoF85$E)*C%_zQ5{81r8r@vzP|O8P#>PxuoPaz^FfvaIxmoVeru z=x@d+*OhW+3)?fpeXRGrh2rjgUXW-gxd7 z^Io?d6Shj*xCx3Cw<3SQa-(*t@5u)CtR<9pq?@(kk(6dmrGFP}TcBtasa~F{QZ9AeOt-Qr+ zd#;XNE-*Tpxe_97_L~?QZyOK3PX}{PPDaBU4>T5xf3=s!U$!rIwhigL>pwwtzS@vk zaY*GMT~ssM-3BkGW*-_3>M>_}Rby8Zf(t`i#tp-+<9ereWQ>+z$1mmRo}yILiCbaK z_bJD3;BsUK+$--Fb??P};^T?b)EvWZ7Zw)mr!~$0%Tkv^g8lk+3>hY zc*Yipd>+ZP8Rx_c%>oF+4+$t<+w#Ljt@KiAI>FS`(pPntMSE1ZZ7aOF0yyDIU_A4n z^6Das5o}dB>i^=ogI5lHuMFIp+yz~E#IF82AWIvjdUP`cEG)iL8@8twEufz+SQD$92;J-x`SA6`@-5L1B9cK&L@c^|IMg@2+! z@R)V+!^kKr1ioB4e#FjVG8)U0h3EIGYHEh;OysQ(@{ZiWOrOQab)Q}EVC<+V+g}kLNQ4xKylH zi%v($FsxeC%R8j{G28lJHV76!C`s2wyR`$ccQ5&i0X zu1GS7eOCO-Td&lQjkQd;IiHEuw`~nf>_+R*uM1r{H!>4hpRqLwb+ad* z9V&1!{;j1Ne9ki4Qzy1@W7u6}M>T1EMsccoGMd&gf#O`g;>UmkXM2%0Y zzi-f>XLq4%#&DZRoJb9=o=Cw6Cf~bonxVjaIdNq)?P+!N#6vMoz{@gLT7w6=#avmG zWc~n-x+c$WL>qJjku>7`s8ss6rt|?IuSQk}cNHhF`WVF0bnuuz;-iNWQMmH;vDSPc*;D9% zKBN2iKGSDvRSxVE-F>2W16BcQU+P;A;*SnfO5cYrIPKod zy)2Wzpw&%}*iFr@77{6UX5(%0U&czofCMY(v2lw;kq3IBKxQp&kvHa~s~aG{arFZf zE0;UzygaGmo%vIh8G%`B%CLST`qA0hi@WO)uQnjnMRKqZzVk)t!4$(bjEi>UiJdW*h+y1Eounj zrmqvp6o0_)T(>76Bm@fs$-+abH@{E} zk&*fwGUTocye?kOF4E-)uCCqG%9ibc^0 zQ`g;H3?hGe`0_20TP&2EYFO&k)+ag&ns{YGnmrGJgS(G!t}cRlE>yDykR;5^O1UbD z#VH$*A>gJjG571rEExplZ~zUWV3zyisQz=lYjpm8HuoR9@socTJ%RUMDpF5PEXx;%8T( zL2Q(;>98*=-I$>$V-wR|OrCf-~>ms?I1r;1o=)=1Vd& zaZb~cA(^oe9-kqZ)8UN3`-tCg7WAF6APVpRCME6rD&MRCj&%39%)&p;H2_O^_(%R~ zVfi7j^=^YRsLoVeJW}*RL+$e71kWkJo#Q`(o)IPcIii zbHe}ThX^UJ`EVy(2^JtHjt3l@x`A!Ni_$pVFCfm$4m7~4Iru8x0|#9$PvfO%?RT!t z8TT7s)KnREY#ve^olZv{|Ze^?8DC!d`= zD7O14{vg!|`d&!y4QA^x-aoF=RK|i<%X|r)m19PG!E$kCVBN}O%yU)^2`&}2S zU571)yrCqOmT))|GqaRaZojaAE|t}61tp+c1RiJ1A&^6mItXHH$w?f#cdq`fN5LI$ z@BKw@*TLRoe0&`6wb$9mkY)s}uK-lLYwZBukh35#Za9f|OP}Kw(6c9CjWgXM4%mSv zaO=i9X}UWZfFJb31!&Z@=hTMxH|G?G;ql)O>v7;8@Xv&I?ux8K_T5lbwAS-O{u*zd|={9$9+qV0ALWX=EF)vH@@$eR29%D)Y++qV6?BB|b*s!3+PZ3n(dCVD;=j z=$u*E_-NA6Z-+ROq$icS2S}A^_L3&=8biPn6>JA7#-buQeW;WKeEmpunsD6VwJ9g(>ULl^L#u*EJy_a;mLkOgU&FkvT zr%@FvE5_rOn7)m$v~_}ZSB7_?)R3}=cdvmL$XozVi3GKAX&9*a{b!dUS7#2K<@vlN z><|u}id5f>ZpHAY&le&-E`T*jggUD9D$@|3`M6+j#UOTN@(0+Yn1$6|S}l#iXg~2 zx0x;T0rRoBygVwP$B=wLOMSQMcSlSU`uyHgyjYzTY;Eg+x7e+2Zb@=n%i(QAy>qCbQ;S$h2|e^h6BKQUKPpU1CFheZX~Hv?}6;aof%O#mkrW zHVbqBA0%Ya`Q#b}E^UbnFjzOaFo)0HZ~CDtS_Nblm=yHK1S*vUg6j( z^5;45IR~QtZ@ejwGLn&H<(AZ?jYkA}7b&M-)<`>C59_y13O;eJvX7*8+G%(k1?hFs4UiN+gS?QY?S5_wfja6afi*O2DxTmiJ@Htj$Z8#H zw=n!Yua4$JK_KMxpC(cy($q^x`)4!M+q)9M|0i*DGs9quXPq@1V0;!{u@%ohdoHXO zxr``ni@&oVXPngkKDk9Z)-_W@)nR>c*u>5J8;C~iLRMYnWSo95v*L-*IXsJKD~9#~ z$KD@XK<(CsdZOQX+31TyxlqncI99 zTt8sSYa%lhDWK4J>Egk8QfWSU*;+5V#(yL`*YII&lYZB|s9109N}RE$-%%6#aNvX9 zwDD8ewY~S$by(2Yb-t8iQ;C1E2gFy~54`nJ5^hL8m3p(fBF4ze&lsh$)U2Qs7Dp{k zy7sBGr4PWh_WxGeC`7(+d7T&PA7>vs2p<06+CcglwV;lcdSUhr*!(CYYrp^Y{NgFn;oPAggG=I8rCx%>GsHzHCkIk zmWQ>ayfVJ(&mWxIPt8X_`&`35jBeq8huvI(la03_lDo=kQ&4-^*x5}z-hIWh+&#OR1#0={S03wMdl1^BmeEvHW+A$^F_0Ygm+w(_WdJ4|ce<~!uj-Cp~EBQy=_ z2;K7Fr_Eatf{^i@3s|hF5^m;nsRh(-=JH1FT*%^(>yuPSNzMMJ!;D2i8={o37z?)5 z#4(Ljb#oUlEeadJ8@}}E^|7$vw(tk1AI1Rw`>BjKZ{qOr-fe~34R7Bp<=vi2(7*uY z`4Bp?7qR;TO^Wc~*%iJl>1_%YOPe*9i{87BUjEtK^Gknn++L_mBMZ0Wpj zxX-ZcT_T-g>~v3P)~H1}FX&?q6bN)3Gk-1RprgX+a<@fRU8si+?u%okk*`CWX8~Xj}vs z+!WZnWDz{zK@`Rf9#m1hR8jA6kq_{rdPpPWNJC@*M^1wIWasjB=k(g5cA*|fH(G$s zscE;M5Plg0%;0BkE;lc494G^9Y;1qw`pwqI`y(Lp+{!%c*1H=F?pdGQJ8wlpkYFR4 z_(Ohver|P9tZ=nNG2eD9ten{(sv?v%A`zV#`r*@ zZ!Dp@oYuXfZd2`*q}q))-xYA6;t;U&5+n9zIC1KZu_3!Sb?LLX;#+kYT=+3 z+orUN=Wo@a!=Ju}Lw7=J?EDViv8OZj2(%vb4aYO-n7IeEB*s)~Q?1M>)!H|o(<$3F z`1vl1MlEe=_UdldvvCZVdwNm=H_jq=aBm2rB=&- zbrjGafTg%jUz0%iy&FJ@{#RVFX6AM3iI&l=tk2zMGgE?c$i!sqou{*=NsF6#aOj)l z6nf8F$#0&u?OkzGu7;z9|GsMjDJFgt;ZxSsbDQu`DL7N-5cVcdb8~WLsy@6}db2w~ zly8pkbN3aKOA(&HmT|R>Eba1Frc^b4MXd#shx-~D*WW5w3sBD+wVcYPcE%rHd^Jg3 zDFh6-sP8*4I8unr2W&$0f(CJ@l!(}Sd9IJ+3~OdwE(5Zn*B=OemY2&Xzwx}Z|MsaR zgz@h|dtd;fdGm~@OT6hxhkOU-HQ}(ka6gD}ym1$U1AC<+WwgKUR^ie)TnG^7A4o5K z)4OZuhPxElcauWnP#{p!VD0P6VtLS9wMh+HH=W@>X3yQT5;P3Y&fn~RC&1gFsXNz?&O7z8YmG_|z{;M3k+2layl z?D_H?&v4M6aG00`uef#0Z`*j6I5t4!ePDDx(rw~rSN41VvnZN+NNwV>x?iWfzH#tG z3GahVOr1w-r|g~#3)Gwgzy5P&FUlu;bP7#|BQMeZ^F}wcUos~hq#u_Ug4$DlgpQgV zF+75bH(?Mrb(%31p)HqAsZacTfM^mVt-euF$y+`$ z=X?+Z3DMifUo<{^=mBhV1pZiXXjN^4eF%;2ELd4TNK~wTH7Ef=}Z=5BugS2AVIp z%s2q@+{(gcYJAP!XOG`{y`Y-h(!r5jQn6wXPVGSjy^b-S2wie*vFo)t`-PD6*&YqE z_PO77pr2jGMO*_hWsc>tU0>*-Qwme)on8Bf`AZ<yKfZmuR|9Eht3XRSJ3ZYEgkRKy%l})4kA+c( zfTleWU2;&e&r7*ZAJ z$F9_^p1wq3>f^b5SxsHnn%A~#=b-=Y(!X$EKtdpkV;dAnFa<4yhs2FJ#~ozIy!;i4 z^hTZ0!v{AlO%GLvs+A#fyW(@L(Bccq&b9dstg_stZxYIlB4Rgwh8S7By0^)33aZ&t zZGq}jJPiZV665vrOGUP?2qJ-QAW&$5Fr?d}Mvz&t!c22HfMWzncblv*kVrbVp(3}V z9X!9KTkV~btGJRd4H0F(+!W}=wO;|fC*v>AmFgu^CtDXc5=>d49%=d!QL(9p2Z_#g zL~S?Rd7D!GJ(W`N-|(*%3!Vv3crgryw1W~E7%%|OhW0HkyS5xKu+nQz-|S0tBKv@1 zi~%;3v=In0U#z|&rm(T~9rcF7N5iJD{GRH{qNY@@>jC{(k?gazM`9DrpEPLU zT}GHe9hk_7aPMW{jXEe#Y zmgQcMnF6~aKgsuy__(+@2`Ghfu*`FGMDHJV^4iybN-CLHmrkR=*z`7YP|zBOK8IE9 zYC1BmZbtOO>xCoD-e#Mrr}2!_ZBsvw_xoA^^R4`;>AUuR+$@o5G1G^XKRJN20yGvE z%P;bOK&d}dB%pY*q-)L%&3E`ASd244Xr0S%wz40OiZ}mgeVSg)XXEr{Q}0rzrFPFs z`HvSJag}J_$c3JqS5wes@~92)OOb#@9;a-!stS$1NoD!Q5Ji6V!9dtUGP$}xQs`*$)>yKdlRSc(50J&*PAA?Zj4=hcJo5RZN<>o=3@Bd$L8%hC`kgm{&x zi6cD0hY?yE4H~;kt(aq`>y3dJC{H+)v6*0Xcq(MtD_w+8x$KUy56;gSNQEzaV9>_k zvm(6i3VX&Px|{x>{;j;MeaBXAr+i@lbAhJjpPx-0ku87u+!U=fqT4ybh{o?%u^dQhW5XzP_&QqT3Fk$9qv?@l=1o z*-9$8NFjC3oxgSO%<=R!0R$fV=MEMCaC&9<#)cjglkVepYa*kDdSS554qMrm(wMTe z)Rfc{EhBo&6)m#tFFZne@2z=?l>^d84*xW2Ho(>{7wE~XF_!POl$NXH*T%7)w`$S+ z`@M6LaEdH;Y_nGRSRZAyKc9RvF#x-fUJB5=g36_6-#-9kE+v9zIc?hG4!dx0BZXe< z);_$LA&Gh@e8S-UDOk!>0)r|Lo~jtg-wad&>xN(u1KAS@ptF4_x{V29-5gFE0MZX1 zfbeiVUiFAS*!k!||Fd2JUP54AJJt`F*qOY*0>N7s;8?eV3fK#O9M@Z}a}hTI4l60z zTGxN_;_3|gUe!25ho7n!X^to@INNt!E*Z^%Dnh!v5~JL7TzR?tv`u$PLTKR5MvTGBBA6$r<6 zfzlX=9z40c;PL^ur-~;O=p)*JsdXIGLyWqzvT|;GJoZ~D2*Gg&me`)tHSRA>Szf<> zjX%EToA~!ll6Y$i?ci^K!RI*+p(>3yyRCa0Fp!#4GQ3D zqS%=?u>RVP`sf^W2gl!{rV;E#Egc$Ox@CDs zjDUWcznT%9YmkpIMOdxSQ86}dajWot<7mI*kgw2YZ)&}Bjm41t)mz`ME^p~W)^{iasU z*f>VlpMsnF*ydST`B10Ai|HM8@wu^BFZ&Vt>wt+#uBH^QT@h@@}Px=tZF!rrE50l555+(coCLV_-R=gS(6+#xfn8N=?%JRy;M~Pm4laoPQ0ih~jd|`6>6Q|o_30AVPCS+_EetH9d z#iLsU%goDKN9dvh*tjY!ju}>zPZ^9!>3+GF1p0kI6KYYS;>1W6td|3SE~%?P39dm< zi`##ygtfiVr*~Es%F$D3@zvq@CGbN@lb6 zwj+$}mo=cV$tpi0?1v=*zFVi=e*JACAi8_yj^5goD?L}_yHzi1t4xHG+O}`f64+ig zDSW6}a5Hw^Ln327ZNAp6xl-q4YY?{X1u_`v!`I4^ZoWEDN-F+TePM9xSDqx=tVRN!wjEvd%(=@W#l zTJg5Jq8927-xS*e0p`r3hO)AIOOC@Yk%r~BqSw{~>GjK%!SDijV=H&6t7Gh%f|^{o z{CR*~WHpP&#^sGt!L5E5xouP19>9z<5FxKq1}~|w--#F`KTH+EN|Fh;;!ROqTyAfc zXoZG&762%fMwU17>)*hK=kvE7cOHhx$@sT?M6j!id&BN*3}P?B?D`3%vwa+3S+=Dn zuyHQKiA=s(*!lRg3(=+YU3$mfX4eiEeN@WKFS|ui+Q7RI57HB6_tO^K?A60FYgS}F ztM6BTIt=-2@azAweqX!+bDMF@-|DW^C}VVv+s653C)c$`!#y6KWlxfg2GOUa2eR#+{|%r zsY^QA7fA~f{-L6v-{{Pq4g$RYoc^YE-NgDnFT~8s8V16O)*{muu1CY^>MfQMtgNhZ z#w>D{N=jl7?ODa=J;Wl~*ik74IzQA#PPSJ?7oZu0-MH+?ZI>UBzHh|`zq8Vn|hu^{D#|^^ge>&N9;sBTog+<{Y!1iiq};!)1@;8 zwn>t^sZ{vI@Uq_R^I^Wq!I_re@=r-5A+fgZq+99w5##&O^)^B972*h3Ioswc=VdGB z30pPuQF3sRV8We-`uYq7QrQUZ9=*Ym6^ui`@*y5*EyX5>u4N5@s0LUOJ$RYks}zhZ zARs-l8%*Q|bJvVc`{mk7-C8K?75J`}%jQFxo$GQDYXIFFNdh$p;9_`1r;4Aa7iFt7 z4s@!lUgi#aOlq(V1H%%q(OS7AuYr=)JCW3z_SlpxH>F)DmqK)nyuo3f)6BD|)3m78 zm0hP4?D^@Jd$o4+{HIl-j;-RoE@C&n>e{tE6H|M9mVsX)hO8HXSld)~6(HRp=ei>X zjZx{A!O+&BZTvgM>uNF)f7J9d6 z#jXS+s3Kv5Lr=$bXpIA5zIPPcjuJp847ATc@+=@K1~ehlRXfyNa?#6#X2g;QigTS zo|uRFPt%l9OT~i$u%!qS1LA6$P&^6_YN)K_qu6`&E!IAs0@l}=GmNdnVC4p+*>t$! z8BIB8F%}dt7MQtRDF0BTdj}N)paVd~lV8XZq&&04CmCztavlBsyHC^bL%57FkbN`t zkH|U%k>2kWL{A%2i2Q-H$B^t6^dlg`;j5YV^i|9xk-yiUW_VeDpWf8a@uqpzJ2Di= zHo&Dx2}iv8c7J;}d@L?%TLT&@+Hv8Y?t`ar63@qggkJUR?bn5^L8r5V7GQAAYqr-K2N z4^T>VkfUWbT+pOK)iQD@vL?3 z+PEZuVk@hv4!33aRQa`1z|fAIwW;P?(I=IZ?>BPG1mTixfAG5Neor0&)E2++(>cwg z-NZ1wa3WM7qxWg)sWvDRP}S0bG%h?vR8UfqRfI`OJyPt~Wnhnpo)RP{C;w|)ex&lM zyC!1OtUv~XQ`xNUw*Chcpt=A@w7UhY1{H!ZnASH(c3?OPt^h#cNfFQ|S}79{|NV5C z3Vyb-GZA=e4a*x5AOI!n9qN$_+aiqU zGIVg|Dzn9EXy`C;$l?;mL2(sC-;jk#IRv32-8kx(+DrQPZQccWOoGL@44B728f0`_ zoF7P>ghO9VO%y;`lwVXN0|d_h;+)Yx(=)H z#(5o?3Kc18EfN#Yj0jpin4St=FL^L`hg?#JtXxK!((-C6*)B5BZqouYvg4Fn9$2el zchUz_gNH5qR41h;JQa97MoT&fJDR`Iw;V6K`biOaDjWrD-dr|jy5=EE`-QK47FBMb zp+sGaW2CFQDAm-u2U?Wy@3f zF>-49s9wuyRBUF{)DUfKYyjA__jPHflQe@-(lO!W*UfVQb*w{?kIZ6!zS4%bu|puu z94OiSUSZ%}Yx~x{j0AatO%V*Hw?}oCyM<^|QUF61P9MrGOERvGX;;|qNB^zf^0V#! zH?wR`lXnVRG*ZJQ^!C-E^nGS@&w*}Ek&c*`FssG6o=D4lS zOPx_8b-Q`{-Fi2qPN0gHzymY{Faq>czsw@%NO%gh2jCg4u<&ir(>t8!AONqcs|zH1 z`HpOL0P7?ir#MgUL;?HfAj(+;NPh0^HXmuheHuQ7YXDz6$Zz18wnjwx2aa!Zv|JgT zKOXT*M4~*ycmjhh;Mhw7+kYWBYA;NP1TFVLimSl+haThGL$_2BHv*t72f}h9Hr;>$ z4gplw>7pJ*MvB7XKIv7i3;t?f3>Zw^_k_o3iSO&fTmSZFsh;EL3W|!7A{+*rE>B^4 zTD}v?LbSiq$ZHRQdZ;ZaFH+9Ujf_H>_@{?3@m7>5{&&LNAO9jn0kW(Yn7YsQ;zjjy ziV%ZIZn`q}b2o~YR%kM?_9j_YyK_@wnrer1$A{SO0()n1$35CMl2AO!j~~f^sr{LQ z%U3)CSxJzmOCWmma_1=yPmgiG*~wEhSpqF+Ffht_ zFxO92h|RfI!2<_>-w25ra2=qAHT9#VKk-0G=18aB$PGaAe~yccE{o!-pz>OlhQLQJ zPaZJmHqLyXEAaUP-Fp%9#oTEfQ!zwA?!~;3L((dqzVq8`V(oImhv$w#_IA z+LzD5DmRZHgQNJOHU*}sj+wykU2tO5iSq2h0PVYJ7n*!lu-tlYf3Nq{;VV1_g;>nW zDfDK&yqI}VQD&&6Toq~O|D%6sc(cET`>irjP*JlCgaJSwF>c65>?I-k0kGx-B@pmZ z6E{G|3yA0lmoXrBADZIGC zRiU9pgO~tw|96FyU;GN%Q>=_r(Et88D2oZ4Zf2`Ep&(|Wy}~$+naXZacoAs#0=(C6 z>O?&MnwYjzW72XmlGh8)l{Gfy|58|dg9@n}3J4v44pKNIVofwCIEpDFtmRB{@-iE; zpqBe!@7fz04k-SEhtO`dV>y6$y?u8EFgnn^^$zs;E^SCE$!f;dT zI9)xvQq_L{1XOs77m*q|qn(tYSs5KBvbstEeC0@xJOG*9>0BYv0+)gDiHYbR@yT2l zsO><1O%iDYO8B;u5FBtEX})rCKRNCJoD#hA11e`gtQ(ZqtA5bM4q6Bs&qB<}qqkY? zL3u{yh1(Ud+nw0ADWj(x&#;N+-yYn#28es#9%6qfxI`WJ6i7qKq?ssWEn&2Se8j+F z07r=Xn<+~1lcy}Kq1E;YCZuwONN|ep)9B6xt!$~hy`-L$Yk*8+Unt!T0JCDD(%>lB z3TWEEr5=RTK-8-{(%zv7f#j3F1@U*d$Ow?K`4ghu6XoBdS*H-l&r$VnW>`{TUN7b1 z^&8dM)b@YuB92iOj*YLNUcE4JOtfHjCZq5B=Q27=Z4Xo1h@FHt7))g^Xg>bBW zj_hBZ>!O7y7ge^PrYT2F6dXxvP8;l>4@w%!QARR}JcW#IYh(E4_fkE1{e{st3kx&} zoh+vU;U!I^n!f@d-tT|%+o>-}GY#(tB~tvBTVPZe%gYum!}d7!3V68A%M!ZO_>Y^8 zW;Te}uuj#zL{m@F@5x2#p#}UeQu*+WSRheaSkWY65qGO!Kl`Nws;JyIolxI1 zG8ri6PyVGyhZ~WQHRUlWV)djy65xqtRZ!EtE=7jyUvfy8Uf6+pvU-**PA5u_%KE1C zIRmPl&2nDcyg}H$4|f&ecR_iE`->w3;8c!E2zlc{7dUVc89ykvB+8UB_IlZGsmxIgiJcY`O{XUJv|fV6o-Ef=Ab)&A!UN zl>_9nf6_VDHeXM~WT65ft^EV@uf!&a&Y^@{$vmW*{qL*C_~zlSx`KF7!UbtjVo12Y z2Kd{BUrkhpjL1nM{z+n3xqJ7HJEzob@VwCk^DtXpFA^JnD|mFrscyUKg#r(k<&WT} zpd84CU>_;t+jXG)g0~LKsVRY}!{(d(1G0a&cK^+?CowUP8g}D7`{3NdEA?z`NKyIj zywpopT|NF43A@HW+shqLzRQ6F<^G>$a**YzSITdwa^MC2CI;R9m(>7bApXH-#=yW> zxxBRpE-uC(ryP*it!QYz-f+4aIA!kvcLv%v;4*|Fq5Jjv{y};%xr4Y|1GCKl_x`qU z?-#GiJt0Wl#a?vYS^}eE@4!&k|A^`yltKL_MB)faq`0m?LqF8#{PIg)y0w<%%0kBi zY#KEf33^<7J|E`lobvi#|24_$|mn>iH^zebdysv05kvyNJ4v6^OV6dStKH5)EE3UGp|fWm19IJIx}-ZaQBf0!vH2PDJH9&PAa z6p!Z)A>_aHKo^cdyFWT;h}t#Hf2E{3Yz|5Sb0W}T#zh|i9yM^zs=Lm9-SdKP&T990 zA{4OH1cZM9&=Zqb+1SLy&LIN@9x@b?)~2LI*f@?5FxmlElgwx9q@8v~*6ToAAR7o~+StL0IFv5w1Sq4I%p~>S$bgOD=g-bADQbt2rAPwyP zp9=emaOywb5ylyJ%114irO>Q8yso>>R z)ta<6Bo2F8Ca42Pn)Q_VQf)X{UWMVab;>Hr0l4}k_PL4qFLGP({S9D>>R?L|HxvS# zY5=DMe0VJMNPvCL_{7_o89~j^E)>Ap004;&7c;x6u91CJ0h3A}65v`NjE?xcT;N8H z5#t#fHu0-4jOh4ej99Sf2f$H~l&q+7)F%mC`Y(5q|Dodc85K<=*tGx{#qN^5E)&vG zgo;5D4>+cT#MuceihdipWR?_AK+9UYyw9p$oc;Qxkhtws9RU5vI`Cqxz7XrWehy7G z3#>!rUyw-~dU6c%9zCRF+-WwGjkq>F#c%OHxuAB&A!ryQRBB z=}zhHMrn}l2I+2iH~05^KNtf+F0SHSYtJ>;F`t*)Utg&8_W`vmsFZNd4cMR;(tPI3 z#Z26$9Q;T74_54paWM_(Cc$AF+dUmw>>bJ)|N0s9+BNitEcrweDVthe1!YC}Hl26D zL*Yw@3`Qn-JS$Ef91&W(-jY z82q2Z;8DU6PNBQOQ|>>%Bf7c}_B~E3Or-Xwm87(+n4nPAF}OCSE>K;mpG9zARxGl-evc|fdU@VXJCd<7RWW@2TJW5 z2vJIA&IJK6XLD3=L1oEWb1WbBJ`pKv@D2e>IK{Cq2W7;1l4t5Mo$dz`+@;sC0hr6HBVW8}G|go?ah1)goV=wgv5yjI^FR*5)P0*uCaHmr2~lGD zUAg4Am+6LqK-n4Td@8;?mUCY4ldwDEXV!NHz-+5E_hLM3){g!OejrDF1#U0dhjqgq z7F3W<7US70`IS((?b(!pT<#iThHOIawjBWoUzS?vUuJBYkEI_<9$~if5)C4aKU&7} zzpjx;S)puw01b-o+@^jC9ja^1LUd4~Icx#gO;3Ejka%TgIh*zmdSEXjmByfZ=; zglS>lg(PXS%HjNkX%Y5Y6+JMi`Qca|b?MTGb7lnHcER}D=^VzP z2J^)%J{- zt;#Jl9ko;down6U;8H9+Bz~(4iuIrPOkgzafyu@P$U-jG@>lvzW?0QAC0p zzN3lqmMG90q^bLagv9z$6}nI>Eo%9IBF!a!VHP>pzn=eJqiN!TqUH~VApW%R?Kks0OA#5#0{{)6LWJR0J};3^t?<5H$h|P^j@BtE$MUP@e+IQM&|?D+mDwX zDvwv-z>J;tBTQq6@^O9g6_ZkF)e#~7-9KDTzHfn$Rh%f+znYA&U57!W;+=InJ7Ma( zkL`j#X!X=zFti+h+*_8}sEk`G#H3~e4hl34yzOS-H%Zwy1x#pm_eB_g6AsRr3-ScNE)h%x0+4=fxYm zy*^(3ResG=2bHH|y0BEOq94$b`~fTtEmtC1JX63cCN|a&?63n#`u*!(pZC*&?q>7B zub#G>m2iMVMn36y5il7}@dMCCL!fYj0IHXns+X@Rtj6$QZ7;^zSF!7bYinnxd-viM zRM4Ho_!Zg7!5cLNxcKj{kHX{Sc=rLsH6|-hba^+h71?5pENW|NPbehW-7jN zF}V!}ZGj_XZ7#MaHZc{hWhzkRfO9>^VN65|g+#hqe8*QnFvDm(fW|=lJ`o}O z^q>zW)!Y{e6R*7kO;}2Y5%epa#iRZHyD%c59sBj+O5}l$2NVEpvORK57!Uq$dw3!| z6Lw7Cf#Fq0f_UAie$A-^;AtvSc|$|;6|WoZ*X?0oj((}DpV!tg%FCxjne;U;|M@dL zuraGSxq^<_uggFJti1x7b!_raB&noVVu+>|Y%+Ds8X3_ZnGO^XjyMvHe@xfV{I+$B zTk>trHmWL38Ws;Hq)=4Sf{;og4T;+!&7uZu4zN)R$a6-KbMcDXhI6}w0#UA%nO9yb;#aIziK;jwiSb zP}rq%M)ZWg*=36p(TDv@7skYe!20?5574ah3Jbq=zn-!s848*_uqknwDu}^s0W38a z9Z>|g^bc9O3jb4(nQo?eoRNb+zkWC<23*cl+ zhNOWcKG9GQPgVMPTum*Zd;J>R0L^dAcCacdY>f<1DZ*#~7?PNNWX6#aJ4ySCyE1L) zAXum3ld(o96?-fuG-ohAK-Qezey5?Q4FE6{V>dW}hWU(az(EWfsN!-L7XdazMGft? zT;S=9H2grn)UoiH(MO`tw?JC&P0>~}GmbcT0l}=}&*v}_Fh=pR^eXqrTaEZ|PM@h-k{cbquT4DUff*oxxEr2dEwE7MmuamO{C5%B%e z##=!a;xzM~C2vpb&{1|^-YJ?yB{LLwv9cHrU}-SM!CJxXhWOzPL+9Bx7o@`S#fh^| zxWmIoB&kqCxdUB0;STKa!>@j2wIYZN+)xnXhq|FDTeI!VoF9XQ#Y_zgEnwuzTA>Tp zuzK@+#XF%o>87~BeZv5J5SlT(xV8B#1ljg&2q*69ErYTKuMp+Z6X7;fWEji{VTD>l z*TTxR#iR#?cn8EN+6P&YyX9jWg&0KymolW>Qfsj=rdM#D zc2}(RxaSITvEB_-yT-W*BMA1a?2ww7dCUFm91ko?Gk z9tYw1lJ4v9OTL~x^U50Wd$e>Pi>c#{@yE~LkcKgkks78?U4*YL%v|N&9+LB!AAZOQ z)?K9nk}2GOn1C=SGo^6?6B#8_0J9J3WZTTE%BCg&2|-lrMz3KPR_u*QD<%}?S~VYQ zY;3CQ%dAbp8v77;+Z2EOz^^R*(f}YR1ZN8eEz;$h_!5x{Rv9$=PBAH8T;Y=vUI$25 zyz|w0)Z%`=-K?_P6hHFlc#OBT&Magp-;b^gaU8htDt+)s3}6$_-vA1I5yOQ3 zf&nTuVWPjciu+j;I3VT8%*JLQM=bpG zdeLSRxCHu(3p2ui)w{O7{*__jx|{A|3o@tHKq8D7*uIGcmERBQ-3CbK8UW95@H-+` z2%FY3nD+aB^uw8g#0N9wpFVwpB0wJSneGt+;hq5HY&VDS^#z}Xkuh-IQXGu*z&GV} zIv9p*Tb4UQOn*oB0c&vB5L3$`2(gRF9{OJK7x%ap`&}>z?1E=Qdpl94_dTt^<0(0) z^*QoZ9Ofl-<8zZi=`A<UOZl$zuLFJ?C%D@frN|tT@Fc3eKTvems~PFAr(Rq4fjzV`uO+~ zeC}^ZjU>B2dy3}iVg8A$HoagL5dI{~2pdR-<8%oN(V4SddRDp_-S_8&YfWu{IJ|X< zAS8VSra!}}*<_6ZmQ1x`IRLbQ>k%v+{UwR?ug~vG1iz2%-qn2o%bn>SQq25+m(Z8D zx?We9zz1#nhqe$xq5K6zu6f_ItbTJ7RqO?hq@Z`io+zO2dakG)TDKKS!i3EU5SR{q zo4JTDq^U^d)tedsl;_L(_wb zVx+t>s+Sd|L2`vqQtG^h{rC*d35ghQC?=JZ^j(OTI6g|T*QY9|s7AD=Sq@f10_JN1 zMw_@OSV0}60NMw8#amEG!2y{!UG4&996j6i?L;GlVqC2|q6uFKy-LFgXD|Ywpqg>0 z1JsAo1EF)v1S5E){oA9PH`ANcj zQM&|i5Sj!1VjDmi(NTg0hj$=1KRyzKp)4>qHswIMB+f+nz~Sr`Xt~9e_&#Xz~BTlv2v&5cj?==L)D~DQM_G z0xoJvH5%wQz~*``XhOii-i>Eo>engwN$_xE7IA?Nc9u&c_#Wsppq9oMK|h(($1_~F zXdRlG8dIrvf3;*vNW$noQVzt22`Bu1KhwE?O?Y_=WQ2(n-)wS5Lbed8`b8}XObZZ< z)KUVK0CY?xCL1N65*m0VLPQN;HHhVHS(Kj;CDLBuJ%Rt^s)T{614Q;!yt-8*G2!6f zLCy0$u* zrW_5A5aU?4M9}*xa|fy4Ys`s7tTl1LaauxAoH%cf6fd$%Ckt?m%4ad>fkky(C6 zgATz~2TV6y9IO%LZLlkATkG+GlT5aR(*{OjTP`q6(k|Naxy2VH`>eUzBYc^6S} zFl{4J2mUn&gr4_QcUdkm@kt-wsoOlkdR@r-bFI8-O9{?7XYj1u^<3iISanfEgaB)h zz=2Kzr-(7(qfR8AIono6-X99$y|SnFPg|&KY}a&Nb|}dJzk;ci5^8ESY<*_8y2|;N z@3TZfPDuyR2p^@TCe21YHu`Sq3@5bCJG<_RsY**tg5>2`I1R{x$nf?4f$A!+{E!@Y z{(EGVUE6BJ;RhKo63^3rf<(oBB!$AA=LPbAn6%B1j3u^TzB2UI*1=inMa!622r66u zKumf?&u{Ev2VPRSvJ}#++RxDuGK8$GPN(Xae~GetqEyJdy|l^v2*Xh_1=0J`eNF_3 z4Ra%)(Ki~e0xd__{Bs~K1R{meLQ7AEM4fdu+HA~|Udz2;+pigPIJ8m|xW>m(8321t z$|rlJr-~rnZrss4-+FS+jA6#a#rXqpuMkaFT*byaOaTJqy@m1PE3i@df&mb~eSOPh z7vKBvn8%3x58lc5ZUy0gD7Y3zr2B}#y9Yiat-V(>1O~)LgEdzS<4Z9)xhN_Uz_NTj zrdJsbAVc^LY zK{eCtx#s!`g8}Y+gmx$3S1(tK^KB@Abv^RF(lDx|3+W9de^=5mrb)G)(nY(c zRz5J}j@zf<9utrQieK&2>G=WrY2kxzS4GtQ4gZKnx$3LUhMpb)%;5oMupf-dJrJdz5YQYJLVefM0EigfZZ$o@D&Hm>qv8caRJiY z3%$Il4`hmp5@q3|KvwYFd8p1#Tg}Iv0^2iN!n-1Z)Xj*vAmZxPN9cZrzP93V(g(V`FKVaN z5wFXfQg*RK0O0@HEH?t2!i&Wb{9DNJ11G7i4rUa85$GyO0%*7YTwVY@ip(1lSEdoL zm_~$wz0rf4_{hd9f1(kL>#~(BrUX zF%^%h%lHoA3xUHg9d8dBPI<}DeeGWi9EK8nX8^_maMUdV6kbqaNwKgMv=S=dp#Gx3 zLj#&A;Mjj~6v`y>%0qj|q0m=&(G)c_m$`#t>;=dU3^A!wytT7;u~Yk$Bp6{Nj4F|| zCyo!b=Q$Kz1}=tr@9k&ws&Y$O->L@l{Vr)~FGiBMtOx`gEP!bT>H~WPobP9a@VJO` zIp+7E`;cpR5A~p$yKE%SjhuzH2@Hb(!S;&c@7eq$=S*c^Qv;U6V{>N7x%gPHY#5lp zV1Fb&V6iYt0(`gC55<2S7KjY!RB{lY`mV#hjg3u8vMtvj1Ul0;;F@LER}+}Dpg)t7s7N!G!wT=aX6#QcAA$w?#xk>)Gt49>}P3j20T>ROVG&&FbyxtFEoqMu6N zU>f%U{JhkSr2MOMk&_Jw2(=#wR)ip$<~4w)1DGv=G6LKVer7Vhf+!*?Vzi*sAei{1 zFnQ4`9S;BZ6v`iD%tbbru;w}GB*VlL0^lt$*9n8EN<@UCtmwEtcvY=151E*O#UUoc z`}CU*3C6_nJX0KC8-2nM{W6Ula;(XF58aV4;8;HiW zP80}=_g|%pBO(wZAz*wbWu}pG$QOK7U+~^S<69H($__EpDGposAY#t<*lFF_yn)E` z-ydn%vh%(bPTxJ{5NZ?RN<gNR|i_li-~)!Xtiv~A(?_{TeOem2#8&v@n8 zglRf=C_L$kF#Hp!Hho#L`pFh=b}v0(T!RTmU7E^QlUL7VZp6W+IdEC;0ILwh+U@V< z6MHc~72^K01>;{{SvNvcGux0Yvj}Mp%J4P<1Uvl$Wrv#7F|oP%?kUI;;pXu^12w-# zKTTA2tIkT{Uk3@nc%Vi`I6AgH)4IT3ECA52S4+@yKT5s1SbdCP8|8*d9_kB( z($SAUqHMQxiy+k~bkKSZ++&AZlm)OpG=Dyn=B$H-=Gf}tmdP6JQd=|Ok-cwSZ?|!xmkf6H@9bby&0;aRy%o+~(grF{ zkJ0>ApL<1}gMu0%Dtzc{?_K$AJn2pkEM%9?%@PJqK3ptNL&nSfRaD6_ewsydC&bt` zR|gwyB~98aai=XEeYd~o|E=nDbhDknJ)W6d%{RH0OR5wHb*X4N6U5j$(d6O(ZU3oS zx}7$3ym?R}OH!`auTVAYdGa_8CaM9Cy5Udjo>Z)8YeErST9J0581A zAKbn`gRM~j;s-53@YKF&T;7Wr`#NAP4ucOwUyyPQIuXm2*4IqZ?@;fMQNsH`lmrkU z?E7&|UxQQu$`Bax=s=gV`sdffLofKlku)}@1i4qY*+0*}XBUz@s`ClH!26Sc9|a9Q z0IOf0td1?zS!jT551^7j{cYC;=TxfrP@vPpAmRua8*jT%m4p2v@Fid*jRWfs^4{m@ zNfnx7dz%uKp3}Th_$&z!rfEqjHUq!fE=x-~x3AG)18}?^VL{-`!L_yww;OxujMYXSdNet(&ub)Uw7U~h z6L6)~5*_ypwQCK$-bt$gYZF<%#D#k*Iqr2)@KSsRIQ;zlwgJ}{{5>>?UV|!|>CC&; z=8}i;VDJ~^(F-1^2(T{<5V{aRyvjV;3L#)4i0NB9cv6U5*ZfY}yi_gD*R(0{$g$92 z19}oLw+&hGz_U3sGVAZx9D%U?T~Iiu5d3gR-_x8Vcyy4lG-7V^?6q@omUCaNeo-j3_w8hH3kHM0*6!BB z00^ixA#Jwr4=PHFe?P0bmwb-8u_XoY8s}i@eJ$|F+46iSkr?ZBwmPPc$oZ!~Q{eSj za-tziNbOUQ$%vB^K9dAWz!GlQy(ujGUOj4vf+z-ZQ`3eW0ZfH6_-9(#rB0p~1JjzV z+Z!6DBht82Xi4chJ%=LBTPtMjK(~@MuVBFxiXw=_Mw^-zAix;;Sdo8v_hn-8SSAe} z!~u#obC3#vTP#rJ17D^pFU;4bd~qQzqvrcmK2H2a`&ph|^k$kfHdQ~Lgqi-^Y7H9$ zjgWHb_kZVPMjBoFXT_tCt3uCw%L0?-SW3fGqKX@~pXkB!k>{ui&0e?gTe2Z8!~7`X z@`I9AN;u4%Ph+gNO1`US$!%wx{pDv!VT>vs{w)(ziZr~xl2M#NmnT#_$Q-p4TJNGn ztqgzSRZX~QeQbxoi{vD&-}4v5k>hr~%zp2+*YiG-oRQF2<2f1`_L+Ku5`ueDz`67Z ziGrtJ3`$q}$8$84TFg(So^O~P@-mB-K8Y6(9i72-4XmXb-jfl_Pl#$;AHEXW#8{PV zB;*`a;@MXP-_)$uq_7*lL6^3M($imNZZz4W4&B}%ZvDYVN8FVsZEUs3hWJ|m3JEV` znJ4vU^w;;k{=+|J3emTGBw7=LI^CU{1b1?tE6&BHh+|??RPKU%-E) zQ)hB4^?zNL`5)50@r5rmR;ppI-oh==qYsFs37v$Fk;*Gwx^taCCn8k~XcFV+3BG1g|>Nmxm z`7ZgV$C>0yw<(wsH8O+msYfPrX-+q6y0u;;q$~bCb_fz||646Q9Yt3hZBaQAJX~f_ zVi64{>$z4TE7YBqwLVd&9QTwoxbLGvK8m3^Dm1*~vy=N#E{=F9y6GM2^;Oi`Yw}0! zNc3}o>ZY!PE?4NqdW~X9(7*A29BJHroe%P2JiFp-W6PL3^@Sc1K36vkbn{}q{#}kr`h=!ifyHy8>cD(X46Sott~s$9I73!icBZioP2_ zYyM*QyovhDLm~a7?jcvpF&J0MHz`Tg-l1cj_Mf^?CX^8I^Wv%4PxZM|>0dnN zIJG9$I88seZnPyYy2`cK*Ek&UpyqeztO=Tec3sBSZXcz&n@)K(H8uaUu9tRb$?HoY zk6r#0@t6`-E!P}j8~fM;qRU3+x-Doc5e;P*b?3LAW9WlCePNKWys&1YZ{r3Uh{wi) znFn&zI!5FMJ1{KJ}&X_4BTc9+?L8fX7Nz}^*gpZm2kSHm8;*-554#=N zi5mCPUdlpZ)KYAv+bd-j>yBYbog{aGm-A?xngLu_i>l6(UuIX+$yvk*15e_gxk3}-2bn3(9$`e`PCeb-@^EE$F?j`SB5k)B~5 z(3ldbbTXK;Cd|~U#0pyNxq&7OY=eAK1ZUccYHII_631j^WdUwDbbBb-0gNY5kc0Ua zOF?!8sMSWmHwYqv{D8m%Mj-LGFvqlXPgG#XS6N&8Nw-05wb~frl}iSuJ1^5mj}@=! zR-RYWYA@GUljGx^AVoFl7Ql=*&s@VBHQ)fsD z(91elja6xPZ_7A5JPgBy<9Jk=e$CO{97+e=<}3HG zyu93u9S^b#Mc1<_$zgYttS&zH^q#lK#!L?7ikRUs4Pkdge|>i-h0j8B@?N!Kb=da@ zU4PoR7gGKFVcBwHB0V$jJ74~X_kbThu-+^#9*yT$Jn--pLoCDVicyZ=7!jNh2}&&T zib$frTvpw3P@m<_!SVyH&t#}NZ(#M1T7_wEjPRC; z3wEs4QuFuCX#9RtX;s_Nv{ZtRS`yr7+W^SB^@qN~2t3*}pXa;PpFLUPxuj^Jo~5o8 z0&j$f??E3x06k%rfWXw~i-gZf0TT>%AMLE0@?AJ4L4xD67gsb$3d@sA2hcq}NDu-? z$0x-noL~m(b;T65{@r|ZU_=uFaCLM@j|BctWyr135=0ngdDF>&R zAlaa4_RVFoA}0V8rD@|niE_w#FfXZ&E3&t#YeIP&3I=BEcu4*vlLiiCA!twZEuqiO zEeLILH<4I}jGWZPSZtcino|mur zTSkqK9?;BeDa7@4I6dBTC}`^>6t%(j0iT&Ea>WCa)xm-#zDPTDG z9DZ(QS*NJ@Lko!*`|Sb&6CGb+cus&IPEI2UHZX3CtdsIj$Axh2`2#I*Ne9uL(3pP# zfYM`);3}9!?1Y)DD>5V4OYy$i1;k_{SaD)w<_EIZ&*7*mkFE#J1Qhk|zR#I;2`76i zY~-Vj-UCDNezKvKkCt&bw>>}oOE{e`u7XMuMLjp#12PwQB56j|Dn@K6K9gW%(+J{g zsB$aJ_GaNt^W^mH_pzf`B@cgSqrd7; z^foyO@QUO8y$${mh6bKR@$cEm<7E5l^UHZDkz3C2`fr7|guEOk&x)+f$(i}+X``|c zBFml6?OD;8@ca}~qJ%1wrTu%3pzqJ8{(Cd>-RjyyjGNbijF%L^@S`j!w|=4*?)ts- zZy;_b-Dl4{LKA_F{#ah7Ch#U}_J{h+BP{r*G}((+l}Wp$S6a#CnbD6wl3XPxLr6zs z`W!nfZI3PxuX4cD(1|QVL(#gL4E+NpZh_LZhX8fY7ZLA2LOQE1L|lTEy-AftokZrIVd_7E&2U+L$0%IMEi7Bd8yuofP>XH)vpZZG5VJcX+@ZBfBA z1Fp`qz(`(?liuH%Q=rS%axR?2$?w}2(*t*x3|To=6VjF;-{d|x@Y(~sBc5LFr-WKr z8+pREa~_$3@KpDp&qZJ|6}#xs;$Bm^b`$8qx5)e8gkxMhK%7sg_*`#62ZB%n>&KDp zmG4P zBTBtZN!L_L@8w%$2r=9N$rF1uRYX(-AFw1wYDq8oulv(zqu5rdb>= z-%#hGyl7*s#P)Oc5t(iUYKm84YF%a#)}04eXu|t(McV);$FJ$bpw2fPeWT|e=w9*p zO!+d_mfw*0!{Z5mcKL;9h#yZCnlnK>!7fvy|)gF{0Hb;pKqUXZ>5j}5V=xSgSQ$!FJH zrA=m?rJ?(H>{^_z0@C;=n>KvZ4!^S+23b|WJL&fBbo;eu%j9~_xVzzmD1kC`Yr21O zycp!@J^3+IqGAhJV)kGhpEI>zBqJE=XoxEP<9K`OUf14!UH-SMV!FLG&KV{1N$Swu zlP(FfvF`#*Bo;>G{rkk;p+BFWIr)=-v3|bF@ZH7ENG{L{eo^N>UQ<(vU!HhV13D;+ zX%0~&*33~%<2Az-VXJGpq<2)%7lbr+k>H%pRcyG&!;emg>sNu0Y9kz#dQF{a%2%x| zU$%C_qeG^{JFnhLG@Y`4@-OqbO~j#&9N~yY@>>DuaGuRxnNOGaJh-EZ#R`$c+%8&jve~W)0s54by!?q%Mx;17RycI&^pD%wi#GZp zR(jPkz30a}2Hp0L>z7xL7h?|h`^%i$v62A_G@>*ZpGBn|M;4)lQP2$DR?H3o835cR zK!XHq#f~e1nVA_@iRrLv!k1sL!95IL>mxvmQ)50&x3IVvPw*)U zqt?{a)Q}wyeF7R#Vhwgj(_eG5AVOg=rQ-hjH>MGQkmE<21{(kx1*mf#RA`FNWZ3C! zT*7~(<9g3k4tPwDWdQSS#6@5$t*J)6;TwH!G;yqG6c>BFo=Y>j$}@yqa`26y#&F+m z(X6dSOSKyUd* zFYiY`g`9u$+h?Gv($fFwF=7X>16eC)QKB+Pu9Mc*$Y%dZ14y;R-t zx@_MHV}NDF?(KaN^6AeqC~$9kq%g0uoFUUPkR!yQc+zoCs~UY57lozti9l@w$tx5Z z`wx%8kA5hDcRXesc*rwc%E~%%e?vn>4QuDqFDnk6g<~+GlMBDcb!%<-hm7nOlok1Y z0!nl#jqLF3(s5RngtZ3=j<_N_2xlw9azG;myNu)C>ITC^z#VFjg6QKHMKe!qi7Yv{ zK7Y~kV%zS!OGic7mm2s&dE8@^UkIW}Bx^KJ@4lyV-USkrS!DY-(eek&K`Z9uFQ^GVFEp z(dK=tlgD{9g71D#m1i{cbHX_4n667EC{TQs+P#KW+z|3&`*g2Dm0q*sOwE(}Z7z*S zRG5sMG4XJi9Z{kf82`Pb`NDD(=4tgpU3B>J#qQuBee|4g@gjrzvGDWz?rM3owurVh zktK3^p-D;O%<6bp@GcWDO47gje2`(FQqYN$|G6AkMx#DLRfff8<$UEjumKrI5uf-p zqTztV$jOdJvwz4g9XGt*Ka=Mges7`zF@b41jt*=t>AzYg1#!Xv)~<+2-Z>U5pKckM z#AGaZbhZP<;1jVojU!Toql%Q{(@RyDMvV#OTZqv?Vdv(R-9zs&y%{Gp`!eeX`V}+s zt78N>0sN5Dy5N?pK5-2TO{_jKY+QL=im!cCeL&xxr zcNVe{SPmvreNO3izC4AF)T_b5F~Jzgkq>BzDeJL;U(+W$P%f=0A$nO=`OL($_kyO& zvufhJ{Yf3E$#edmJNKMvE6W9!8uKZ^{e_nx7D}`aVpG~0(p?vy{>RH;;`dckv7y}I z@_Te*g7ypsW!g{hg;zgrUi^W~moq`5S&o%1C*jf7d+uVO^<0V(ArWzQs7_gFSJgSs zl~?IdK8QQ}1NPQv#uLNz*y*2}+f_U4<}OY#-okQ)BF637w?r-7Q<;RR3eMQr(!P_V z`AV>NNiWm-kPMsrvJgjdqHV=Gv@_6jip&X0Re(_XyWtZSIjVmB(5Xc(kImB$Rw6J*Y;j?jt;=2C5b zsm0^JxPMB7%AoYDzC?up;aeH%a)m7C7iTB7T&W=A<%bRNKOm%C=V-$WnDaSsA(HX*k`puzPk5PyH8n z{O+4}cm!A0hka4Pu@2(he>D+ib+(m?CW`{0>rajmf^uHJ_&Eee<;PfZX1l(IzwJky zu5xP-waZ8n?7-aZ!XLT(+ak^S@!2FoIeOMke7EK-F&aAPW=e6f%=Uni>c@id!ZPbG z}tBn%ymSyp3%(|h*VwQ>1dT9`F%{yPCuhkh|C4j5?K2RN*qk$=s z)%8>l1_@78Q4tmJn?Y{&Q|8;^$jHd8?d_*AqfwvC*N{{Yoc+|0d@@@>44|mDK9yp& zeVA&mAAl19f~+2Qf5 zO{_vtb?fX-06xg@i`xIARd5jij4MmlV`gDreU;T)KZ!y@8cU}qF<(yw^a5pfO41_d@$w?-ePY ztA88g$6i-E(NPfQ|M-hVFScRjDWteCDz`X4JzgSOBzI!%;I!uGK>JH#GHUn$EuF&e zXDdEuMWru7Vj3=r2FlWWIyUcAO9Q*09`qf!Cl0OrUR;|E@g@D&}&4ctMn-3B! zQ2CU$du2jo{dXmg68eAx>_gUiUH;H;Kl5K20d0)J6N=Zj4hNS1G z=8>6`<|HrUM6{@3DPj10K-nO$>6Kt!`+X3D=DGcDPT68UwkoXhO>vT>BZsE8o=-S8 zn}(TsT}aQRSneI5;zoS2P#X36fYEhyL_$6i#?c3EDE-f5P9FT3x>i>4h_HsDKoHTq zBIA+VXL$Q*ueuf*#?gTtM`aDo6!rGv5jiL)ZqOZ|Axgx>G$jQrUDM95Z$jZt4VD}OK>(SP(5o(yIRE>J&C3it}*|@ojl2%MO&=#1f z(ZQ0mnBRGvr>2!$34K_c0iltYl=`JhEbUb~?xjD?#N!K4x=>z_ z%ApSvF_h+eqWc^-n%Z+j$v+K*ek!zZ5FRTYHXm{Z>n9^Wm!<<=GBBY4Z(z_=wG2ty z=@a2a7O+JLISm(CnCI5g=+^DXOYJ*FhvWn-VrSJVx9U5%gC5|EOEYIkoaoQxKHm~0 zFiNN{6K63pE!aH29h6b42#*p9+3>efrJLob(;q*e86FZYoz9{8Oy-_*;B8~=kTq>S zIQILn&a-u@&OoT}s~80qNWY_ii)fCj5$_Rnk^_AOkRpg7wTb^^A+He~;6?+zZ#$Iw z3;vTBQip;2oegchvne^zDU)%b#T#s7S7-}rMP^#NnOXs<3}cce+NU^%eM}0~y604H z+<4efCONuCxN5Sqo#${#E-cpE4VS$-D!j*oSj^BjsdR8Pi%%@jNu&t{MA7v2Da}#S zGIVY6byo6h1#AC$>sR?536+PA`*X&q=_a`q&x-HC! z{^Xp~SAPn4iH(iB-Nmf=gy=5eqS zM?Qbf&a~?pRKclR-e8joz4+Fn{n{i%4Zhh>N+EFI@L0-JFv68H)Fa>iW_Tv{)9kc@ zn(p3%hrv<5bW;|_c+5!uDg9Mf7bt6kh7P1PcDW$p0`J!6W#wfA0K1|I%h4PP!@~NA zzggX)T$x9G^WC`|d%jjw@19cp+6J9x8xiV^rtJB{XsRH`cgPo41`#Pi#vVDT_LW)2 z2sXQn*vRlr8SWuTVbjFj5@<;DUEI69I~wez4A4|;vbY{Zmyqa9?Ls;uj^^_WD{r8Po9!ZByZg(!xU5)R*#kayu%p()|eqf%&AhuY!h$=-EBGBFii zWFJNr`y9W&!z&Mmma%`IpdfVuU0%@g8#RY|-F|zOL63>z@=bNZDSO=P*O#Nf-eQa% z)n@)$rp%raR?J`c#X$@=+`cea-LQ%HQCQxZ`mX-8k489Cq$Ae7f0R|BDKAc)K^&~% zz4U23=}%BY?DhB{0KQYPh_mNJq^FY0JL1{p(c!R%_>2Q;>?e$PrbyZPLNA6si}_r|mHt_o9HwB-dVI-(I+i6u%EJG6(J9ZggOI{zdMgsd5Z7E^|? z1;3Ou`Qn~`jQYw2C0RtsI}U2RiV1>-jxGnd`4!bha~pn0${%?n^LoMSTB0=_TY?Nd z`q8mb4n20?snXNQU_0^m({?)){MC@rZA7~T;Snhj<|SE@6-Nc<2S> zMA5*DpvPkI>M^*utl=QTE&k`s%-7>v_-UVFr*3Q4jE4E|w`E4lia!shUK^5g+|CN4eEkCHO*zTaeW zvdqZTxbm*FB9+C>)1dvn086z@bIf>mOj-tAG&a$KD<(B#6gUDvsO4OxF&`V1{MvVU zEQg(awz#OFd)Y-C)*``DE=$rrBZJ#<;oGM-V5_dU_#cz4W}7!#VHEz5UyqYasx7N# zd5{@H|3c(fT;3uKB+4fGYL z?@mB_VAO}VlkUYi_H?9Sr188mn#nK(qO_Hip0C1B=ZLQUca4RRPGaaXsG|7mq3iR+ z0Rk#WkN5qwdDln%ocMga^&Mfw{r zZ&0IE@UQh!TGM3y{ah?!VwNK#H`XYqnno59vlGxPhtWv-=F&{dG${L}?Zy!?q;CT- zA3BPx|sT}$&@b9yE9Y(?ylcS%P{ zhN@OG(ws1J#S3Y$T1NTj_dk;&c#DS^ySwAoE{3eQza{G?*Wbs+C&p5kLDYS2BVopG zlS}3<*rtRwtHb+;zreCL4S!7@y0*jlj<+pJ#==F;OAY_Lnhj1Y2Xs#cE_XDH!}(yc zL&DXfRu%y6ntMs29*zIJ5d}(E%+KZJQO=h5>YU@W`vT5)&~EcmZ(9X%L@H*{!w~?| zVseVX_gtQSmY?vA77_7x^FI?#&W8(!Oz;;DKqBE@5~!Y3X2H2+Z{PuLPEsQ5ar0Uu z0y{?fupDs~$1waQ0YT8KsTc?D)oM~mnfO?t1Jwd8N;I?=WWw_TdkJ6ZnNvzckPU{U zW7qn1_O$u$KcLIds{u>hVGkg8KsRypfVA+;l<4Srq?{4d~?NOCq+VhG6* zGAu<(UlF3Pw|rl61uO}-b~%MmAZz6yG2g9Bf2q+9TqItr`tL*soRn#8zZs&1k?Nh` z6txR%`w@=N{e#5}^68Xdjb!nT2q~$92T+UYCi^$SIqW`9jJ=uO(XHUH6Q3q#F;lh` zN1i}i>c2Ip`_WWy$U)lGSDDr%nuk17_Y$dr`Mz@g8s7LDb=*&-@BYKsvOapAXGmKq zrLmZ?D&_JqR=*0mI-)M_1;<+6Rpe7$e7QGuR{LY)GD@iy{%y)|8_TBF3(zisYWw+ zXxrYipZI_BYj-S`!hedK^EzZ? zcdv62#03lhv?{KsFm1W8vLAV(mF7EZNk1glV@*wA|AzK@5w_vS+gw=9kI#sIv|i~D zGg|KEE*xc>Y?L=yJ99^*LK=ET-HfpK`#jw&4{apXlW-v`_0)T$3vaY-@vStu2&XL9 z(syA%^IkO~-UN59Sw?SWn@Xu{8G+x&CB=;^Wxh&N@0OOAT~qzpIikipNrpzkAO7Tt z^3T%2u+f%tpA|4Ai2oj*QqU?lU4t{%?KO+Y)&N`VyN!na1}OiH*Ht;j&hJ;FMks+ zRY^xewnR_$%3q*%KVeoJZJg=-2W|HU<{!A{M48RnWWg?Z@d$w)ZX;`-!z1uw;HXLd@G)F{ z!BLQn*U%I=`hksZ{n+E!=oaoNmGcZyh;ejJM5c1Pqg_X$5jC{!BY^TR`rs2P^kUEG-WH??<#Y=Pf8s-G|8yt5qD%FmR$N*? z3P#aCSul*vqcsDoWjO0IBU5H~a|Puuo+01m+)7&~uDjL0OjE~kJf`cBCWaV!-crJT({#l=xVH4UM4yICL~0HC0tsU`+(+Gc+_bS(TMBmzS45 z)j@{D(#cHQ;Am?!ku#Pe@Xd45W(8SLry^RpKLdEcw}22Y_megDJrO4l9?)1f z{`*&1DP)g`?6UXDO15l5$R;6McJ?MCdy_4DlS1|$$qw1sd;j0}^Zorh=Q*9z!(Be< zey{6#U2j!!mjbGf!i(oG2}bF{+s2G6ouCH*CR|6j5JYtiBw@Zz;I_hr%fC@aY_QDI z@au0=GJ4inuWNQ_p!oj&a|;R#aQW*NW_K=9-A-cSWBR}OR`(@2S-=5pE1={)1;?j= zTnzz$?SV*0@IXVB^!%{+VhZKfKY3}>o?NyE24WWYDb-rO?VO|w~ zPwvI{`VXf(hglZ)Y#v96d80Q31NX08pa@`R=tx6BuQ#*s-~~={FQlXAXsh5ZeuU6V z2HcmQZXoaLw?!T$??c7FbhURFS(FInSykW8%WGI6m+-vGYUO)GWNuy#{=PWMqb(Lz zpM34F&Nxxlr4-r2A5puyH)h2!{HPc(fES5zP!UB#5;!eq>#1P&xgv7WM+b|y3cQvn zW6GZh{GnorW<7tXHvB@&nkS24!R8{m%S$=dyu6-`^bmi=Wa$WveeFn7rJ*Ple+=W| zUtiUWWCDqj*1_KX){Kc^Q|FlY&oY$wFl*BNySzeN!iB{ILZkg=mouTvCaab#?vPf6 zzZfsvjBb6#ZKQh#lm^{@TYBY^#-k}4JgzB!*u++i#p+FbgiXSf4)A|Rh77Z-*svQ% zLw{s+nh)UxcMe-{%lM#vbZSg)`q$@&)VjWo7|NR!I=vfA!%&5OqVP8_Qa&U143iXk zLWf5pR2&67Xnrk{x+ZdHZiX%Fl~q3Z&Mwn33RS<-kpLc`FDgEuh-rIXO5S;58lycq zH>04X?74VZQza-Ji#=sqv@ zQeop@IJ9ZwCJ09dO_jd7W^!Yb+{{t)J+!vwugfgMFC_9dC7|2d+RA9+qsab|F?GOO z#g?B-CiYRL##a(?qHcIA%-%cETZR4WXF;WoPI-V~ifRV30o$$Wob4}d-l3G)q?@b|<0 zHPu5%J{SH&T41E?WBc2-iEPWL2bwiGNE|TIW6WcozE5>mF%!G}nXL%Cc>Rfz1=L z#N3gb z@3fIjM(VvYHXqPw8kR&z8&i_<*f+=D`=+FjJsU>-=H#*KMR(I^0|3}tHk~nN#w)Zf zOW2lpTG!Sod-mZ8Or&;lRk35{XP4o`oGk1HY(GF4AUJ`c^C6JLBU0{acLH~~%x4bC zknG%f>cx9)DkYU{qaC`aC_GGK0*8B-APj09yBF?Ihp)oP_4l)d*1l<&)+?G5Y!i~} zJ@eT0_ypX)%}j04yhdO*h)T;gIpuS7yA|2m8gFR4@(xcKB~&j|Mf2;==v;%f?(S78 zQk4%EdkjQ3BVkV@#F?^oyARmIOkWzFu@s;2MLbYXa5dPuRc9l*;G36~Vjn|NDIxy7 zsBGQdd-7G%ClmF2wk<=v=@$HM#$k?p)#5+Sj7?XXv5QZcS7-DkrwinSmSa_sY{&~U z>~>F*&n|U`mOt)$+?PFG=nT!s@HBgU$Pc47gxy0irikde&N-7@N$U-ose~@g|EMJg z6eH=^hxAB`?7=Q(gG=kY+eVOUIs+LDA2-4{zHMMc^frFP#+nzEDyA(L4wvlx-CFdd z3X67D^ZR1QJZ{YGoHfqGR|hoV*Dr7Ev5-VnzLxgp?G9sjr>7Yi|G@BmIKcZUCUbjG zj^gL~tFOyhiBHdNW-e^Mrkc_8XDv{KqFD4)8RuA-o5vS0jQHHX>WyhRiz_D5s#af& zk56Vx)?XD%EODc?WbtQQC1qP%nEgWa(}k3ZJwZTHiXm(FU!(0;k@>C~4_p^5Qb|me zbl-cgg5i`f*uzYwf}bE_5i*DwUw!=xs9}UgfRdRRQyITyv|-koSCne-sxz!Vnb#h&f+}D=chhwa zBL44MW~o1+78A4Sf4_NlMiKl%v(}ni{R$Nq!{7Dlo70kp0x@?A1miS>zfR;oQiJiM zvpq&pzOLxe*LV5{uLMbcx6pMPdJR{wKH09Ct<`2K)~*d!?F|3&tsLC*(KN5Gw%+Xg zFy|QgJrdGwZoK>5Gbblu(yJ^Erwx^UrRcq7?(9m=m;R1_>oK@HH=p0}uw}d<2;3wE zL2={kkIZw_h|O~MK#}hgjVHGIYs%moh+1rQ2P1_q+St$nb{R~~eRO#IlUnl6Oypa& zc5iUYcXfwokxrr5BIGRp38lRuJ;#V65hC8fn(MSZ(+yccH+O28NQBcP)!>x={my5b z+Que0XX|eq+THV~Miw`4zebWc>!xXDkb&A_K9n*1DW5Xp9@Sg~p!O{2m2O$Gu)0i= zAm-4O zdO+LX6PrhC&*2Sz6&p4VPgn=g!UG7CW3rmaXtZ*n0A}TU^ay+i^YXTYsGyFXXl5N{ zE!x=*9V0bniKMt{GIbOM_J+PzynY6oVmM)t#8R`)UF|9IgqIyO`+OkiH{>y3Pd0b1 z>+X@Jr9*Ew-1AHI0!spJIPE}oF+sQ@3?d$}!Nz-JhG|gx+V%6Tfhrgxb zYw!4qr9U77e9u1?Y!e9=*gL~S8#A@FK~q4^!ukb%h3wIYiYm%co;%=2dxT}fF*R6I01B>J|F zo=6FpE%8Cs#U&CdGJO2}wJ_+9d5H0T2Ny~^~MVUi9kgTVI>2v*qm*`SiP?Xz| z*=--q>WP%si5h&H$`xghFY1?hX=2+toM9#iAcc|QI6w59n)1(!mnrQo5SSd!9Mrk~ zO4&y;djY(}kPYx>-+?k@gtw*?q1Shog5{WaAePD5{<4UJ>h(WnVV%*wESpE zgT1$H_jlf5^D(oV=#Gm7o%4GRq|`OLkG?rIcfyWTN$Irenk6@GR@|!7f<{YUb086E z%&GXcM5cgv$p@{;efltM#y5iVO9=nn^c7A8m5ta6SBpgA3I+0s+WrFjvHzqef9K`8aecR6U^ko*^yHj zu7>r?R==ogSM`v|mARwpUbrSFSJf|B_g(q0&1-!bsvE%>jKCJcCkzwAWm1NR4o6aE zMPW>Zzpj>j9q?Z{N8eOfOuu25mQwyg=-k-1_7+v+TWJScLIm;q^9ykt#9EqLO=zOx z{zG9r28PiQccsOg#Es)D1ApwBT!GsuF1oyX!k%AoT3*Xk72Vs*Z;h$-uUo&%_(TIU z8a=%kS3HQZuc0-zD)J>7EtX|W!o}#^I9mm>;_Rc9{Wq_r2Q?&+d*>q9uMF`R%Bt~b-fnp@pvzj z=952%J#Zq&X~vqndj387_Aqe@r@A5+>R=PK&EPm)ie92qUSC!DhHJbP?i0qbvOJB& zW4UnRL&QdYAk|d#)1Rf!wFoScjIDsF5)ju^ITn>z=v{ZczCar=rjs}E{ctdv+MR>Jr)oyns~ng zZZwc$RXAQKD3E$Q(|dzPevRZ(zd^kDW6Bk4oLfchQ)j}`xU`^F+?nONtb#BUe`{=U zG)$cMpiW}Ce0id_~`WJ-o#9q-)e>*h@6;D@%zs-w~3qpRL$p%O4{<;_i{hu3{ z0H=#ij2igvsa`crW;yvS1~2kqMDLQx$Y~5v+2E2cri{P1$A^YJF+4MMJug5Y`v1?- z*H)m(5_g)E6s?H9X&Oksi8|k1y*|Byw0e5lILI4PkkG!q4{RX-rPqHza*+2~*`L zXvTIK#z!i$Z0f%G!$`_xidRt_y@p{iE|hOX(gJU zCN24h+9%%;djQAo=SAZq|I>GsXdp_5i?Z-h|5hx~>Iqebkg!O30z!6|WjDEXWb{(z zhdGzh{D^|O7DwXL*ovsKaZktr&Wk9xzKf+e{Qw9(Iu$L-?1VHQE-i5g91%i#N4>|R zeH8Mji81ISte6NpxA*BJn5w2{!Qr)Nb=Czj)hcfkB|8CZ3>$p^*Kae?32+P~o^|U7#wU5vX|?sl~3*d~qNW;U|jFHi1m; z*ZS0vEO*f1gUyXbM!4`3ZMm*Y*4Kj6%ame2O~_@6GUPFmVRvr0+YTpVDq@w=t-dg| zjpV8P0tU&VE#d9EW>tc>KrO{JQKr5gYU!YC{0>!+}_&V&iDD%9}UY?~vC`u?)pcjdFZ81(vY1KdG0zZzC3wJVwYd+XZo9%|D}vtmeB9_9%f!#g@h z@*+GRzch0!>z^e2T?sx#=;%(rYHz|_xV;R(@h7mj)mP+#3x`l_tT%(UF01tIXu7{w zwZg_f$vJi)J2ePr3<-qE8$AIL_bL?necM^`6qQ3xjhbhYarMzm_9(Okoktc@I#`uW z>yPdF#?6ORl3Z`Ur)UZpWu%~D{qkm{sYt^km|g9RdGKGYuVnTQeYT;wIiZgZe;A__ z^}^UMo^twjuDNV^vw{A6Z@pKUZh$lTb1Cgu=I$?wY#rBl65U9-XOGlR_pkV+B20CV$B;{W_R-yNLKPM659A#54jOSixbFMzVXsLn{jqhkZJ=4ycv@2g`H7UuNl*gIP1>1v9UZS2<~qFk1<`O52BuS8xMsl>gG+_AAGbFEn@cbRMkoim+!+ zw2mGu`DNP-RBfZ3LUf)@C_QYn9|cN1#*JCuMN8ygxG0~vdGoQ`Shj)wVkrE>vor^a zqsJuRy9O{XMFUAEexIqm%Aq`k8gPI?NA2R`0#GBHk#c?=V^d?eRrddAJ2K8S1JrxZ z#!+%2mnT!k)OBt3G^_^YNlU@0RMD50UaCKpW6Pvfo}}7AHh8S-_Jpa6T9DH>{I#>q zdDxTStwtd|sKcnu-VNuH3dHPTK9@R#EP4Y3M5!?hrntyatU$!kAF`hU;q!9s&Xk;t zf$*EHnYMzZ4nq!-Jv5(Uy_Q7#Rj1GmFtQ^|Xb`9Udw*O3mdONPXn~uyH$Kt$wt6!0 z!@fC>)%1_Qd-RWGI1ujjLz0+V@R1XKRLVWF4cU>`57+RJ{^$N}304$7Lxx@Yq**#9 z9Nmnr&X085_kO}PI2w-hG1nUS{z7OQ1q|zSHxLIqyC7LQp{%ML{Y=C{qn4>xvo9Cf zpqK#z?V&cVdUwXuRws&ggnokOXIie>$pR+E_Jw`rPsyLvJm&m;9s6VPR}88Xl<}(t z^ps;?m^3pno(lMwS8Ipt3v}<+i9*R?=)PafB1a|P821)7z{%li@kH6f=H}G}zcwX( zy5!aq~D3q7xrnSP(jVNg~cmz#t)B!OB?g;FMLY{?4PMrx5OU2HLs>;2CPrF zeC^`iS$A)bbYaKbt7&PIw+3d_%{bu!G(^Bh+(YsC>)5~;1k&jM4U4CtZ_R7i^K+jC zUt$MPgWU`Y6KeAOMMzH; z9F=G-5k~+wS^qN&$8Vjhd2Fo7JdL4AA<=qq=hkn~-WJSa{pUE#d5yO^uyE`xDX!;4 z_p-!^$&^$5i^Q4(EC?RKLH@0MUH5vJ&S@@BE&oiy%K*PRPKZBop3Cc%p0#9J$nSxrxFf{ED;h#^W4VK{)*c&aoLqNJAARaM|1O<6D^mO$0YVuh6I@ww zrkJ98fIran7|S)@)~O{Atn%PwJG$R5?E82!kDhraw0a2Vy}^rZ{;B|{KjGZNB1Rm& zNfaxGg=j90JI`21PMr}JdNHlM$6W>>Q{NoNLP_-c*W$Nd#g1;5V8Cy-Z?MH2#!?r3 zOKUV{<;u8?6Y!@a@Z+7(4UxlLq=tPh*eajxYJL8_!~49G=q70IKtx;XP&@Xn8EH53 zFDkj+YgJ{s^mjQ`Ud|L^%703vvceKG&93rnS(?OYeHx!G21brj-(@S7Qa8VP6g*DR zQGd9NKX)pgQ2Cy%(m6YbTU_p_?C*7Fk|HwGTT;ovwQqUH)@paNg&s;+Ul-9DB;E|$ zD>JQ=uIQH5{P@MNbix2)bq2Gme+*H#y)Zae>FMs%Ro~RDjT2WO(<^YYeUNu{%YN+N z=}GvP9y^z2Wvl*%aXysOI4o`7+d@2UFp|?L!293x@ze~bycaDvt~{&+a2!sid<`4! z01sTE87F+HTUslXgM?8Q-H%I#CSf~t*8_HXr;3lcxj#$OM4w*#jZrl1j35tgF?%Ka zLS{o+xZnoGk^EhLcLI9I^P8|IAB@7u4n3l(zFo%N3S<11R<>ra_HoJO{YC2O9bo{! zMM=kk^UmmWnde_rb3Qf6Q;wG=Is&`Qh}RL+y%+sU3wj_tS?kOnE#5DA)ct-PUGpZ& z-<{qxBDjNkA1N9IDGkfNR)nbTT>glEdWi1epPTJ=y>@dc?{;iZD28}p^~{Ys;__=) zZT`V6_&dzOAGU?OL(M22m*Fm)4F0U!kiT=)Q}ufmqm&sRs^y2%T!l`-c%LHi9~6GQ zgKU)DyB2x+q7N3#uqErys2trFH+?z47SG;R)l{kGNVOY?Z)i)bl)0SfyrknCfs+ag zm6lrhKgHdy_$Z$6d-LbP+y9t~UjDJ8_yYQ)$iNm6{+RhIX2~S=Hj4CVab*KxjdymL z-y=yrfBsxIYh9&nHDCL9!(;TQx$6DAuw(?;8O*)ECDQq_c)^;!=h`gdz!|}>S@NsB zv8Apq0sgrgYnhJLY=sdEeEI^m)&Upw7mvJ>~B%3TUVd_DvYXpCjagNsI=T+ zS%Br)YFLhwXVd9Dswf6LMUt3=U#9jMqZa%+^Id*1k}+v1|po9XTYyZ%PfezhNCi-nTe|b)V8_qwW zlTPCy)fp0_KMqBSd!{i_9dS+=b`^ULrwAEW)892X1qX~SA*SHzbsmU(h0{5|);d?! z6hhO}6X<+#3DaZ+#|T^=xoW1K2xVFMP>zfl8Y#7ew4+H9?P;qj`!L8t+?pN7Xn_?% zR9tF5&GB*z3so)NFwXF}sr}WY0$ckd6 z`I)o_L`Y-K)`MF|s=8^#&+uPxFFqcRe$Mr%(}I0r9er^NeWm+|MXsJ{GD-4O@357;lg`z`G+eea<)C6IqqnsjNZIA zP9C#JU-?5g5_a}C=^c{hhyz95D`8|&U#~mbT=?1nWs}!W0SAMcK|R-VUb8A7aeKFV zQ>db!fa{!B#Zo9@0jihn)fQApu6m)T%Pn@-zA+f3ChD8%{3Th_Rz>vR%+K${2&Q5P zzdI!`Qxo!G{nd^Z`2Nq>$ZMHlll{5O&3tK!ep?qJ8s>*M9>wpx0^rk8^lLZ8p=$T+ z$BEcIzorGd_rnRy*RA&&&1_8j)8E4AEdOI%%Bs^X(6a|V7bnIC%BRmy%of^SwSjK` z3EF=89(KJqyL_Bx13A^zNBNuQ&chNKLI-`E|E8%b)~bnlg$%xwsRU+Atv-24X%oL$!7#5POLzEr!zPJ z#LzTx0y-FSb78jTja|B|nr`Y_UdOUw-t@%zYkjeL2!YjYX4j$$fS88$vBs{-P~W0J~w_|4!rzL<&v;$?+R+N2aTPIIe7q0 z=cKvYleeCg!ZE6H+kb32F*;u9^+x~NW=||fdBtA-m*-ctP#egnBQ#l3A4(P*hAv@# zd$w5ny!XLCv8PYEUv}z;!hmonEpWCq+Hu_$BHr{a#=4$Q9M;v^yGz z#4Kl?>;Ze`pD+{7k(FYYFASZ4a`X7yn~$RU<%`AMFaX=|P%u2!;Ku5;Mn{IXCV&NQ zq?m4pzRvc`2W8qN!G$1IzNj=Cd^G32DsHMEMDTI&Rk7hlyXpgpD9qP2syd$(#Z~Wp zi@@^9WK*R}leexBh-fxl(l~$hYVpVs0NY`pUIr=}v5*Dwo1Hl+M-!Ll`K+?IJD!Im zcP`W`Jx6(R<)0<)j~HqcPky4oMbfM^xTlw%4i+W4FQZ32JJ>e)HEH?~V4+OiwM{mv z*I>sDt|!{JXYq6Hf%Yo{?b{CJhDWXg>COT!_5AL;P#2TAB7plncVUi`H<%zV8kbtV#_+Y}`rNDh z?+JEf=be1@kKk~&tK^WbbQ0MG#nG^NiqA~3o^cEqY{#__0oz5Dooa> z(gv3E>?y6~8F%Gnmyw>E2AZIDWo|+fAwaoi0K9ux?nxG zt?C`ez(*q6#6sGfC^Z8ZG$O4$&#*w^I~F%j`9d(uaqZJc#D{H7=5Wx48k%B+#6DL-4BPPCR?TArb!T z@tL0yn)lF}dx@ik&_4PfIdfDl-U4QsRc7hiyVluWD_n6rvxA5G^GKwNDnSU7Q|-Ae zmpS5knnkvh@t!{m*Xa3PI1`^co_OUd2irTzte0u#LRn-+%59~Ide@tNl$rN@$X4Jt z1SK`K`OvP+ENgHuM+2<`@%ueV5~5}7HM`ni zq>__r3J2lrNz%#LEa$EztKn4D+nc?M{2~<~NR4Zoewo@*#^K(jeB<&PfT^N&suFhaertOaz$_X~ z4_BLB_r+rJogW+ziwJf6Wq%d(#_1n+QFp}*!qjvEmKwVF>*$O>ro*FK_sP}!KgUok z{l&?%9XNfn8&i+X;W5yC`Oi#bX`!ZlrK&NBv=qN2EoGn7pnz~oRPYSm- z95nXM+~*H)&aCr`i{S;O2G)pGSGZ^VlVq(Axfn_dYzF47OCo+<0}>)iuGyK_XhC(_O2ADrLQK-aMi@E(5?e7|IvN@h2ZXlcg2 zI?*jX)b?atvH_QLVkZ{;h5idK@)IGV<;>)MCM{Om_4R=EgkC{8|tr9Unk6ScpxQP0;AGJX0mM`oPv-GE{r7O zkwi-;P9%`r@0_3PD8dF6>NNs1?p|*}0Gtwk5G-8`G?VdKqQj_&V2r(+qJbmrpX2i~ zO=h)Mu~F$o5uNXUSEOcLK4Xf*Ptu;s|H`!597)9f_EOW*iQb1nMK_bs?yk@G&G%m4 z$Aa_9{`ES_r*owtD)!geu20@qY{$gCLTVGD&<9lJrphYh4SkCdnuMTS`NwV zxoNH#khfhmO`Cu^zm{*CbDfuv*+yKke*RA$b^2%*-VqU? z9=b=rcxP!><$tuZQI2DlxF&a2>nrY#pLpqXe)^#YLO z=4dl_RrpMA<}ETfEm=8%`8}Y5>hquIhOYh!DuLT#hI=2V=E=r-%n&l5Zu}KfFyZo9 z_}g0)$GBYU2Wd0~C9#qd7VA2i28zp`HFXL8;WQ=j(ynld(pvz<;Fjo-B z)^dI*x)OP#xDg%F)j9ey>w1z8B*3)O6WZWUaFl9$cUbd&V6S*cH)woI+!d>;1VKl& zcGN(l+?E+YsG{1J@KREq?hcD0nhU=cEQa0c36P+g!od~3LJYe~xEia)5jr+J4yt=b ze%BUK+{rA=P4pe#j?}~d((I#i_pGK)PmGc^Z8&?HJoP*uP0wP$a0b~yi_oWa z2LZgDSST^J07#4hm!pCL79PFA-yN@;i^u@7>sKE(3i9eU3X4wLD6V(k%tlb6H=V5{ zhmmliR!;Uk$xCTXU^n`(PFt=0mC@`(Bx?B*)G_w6juU(5o257D6Gtr{si~179ML|8 zKjh_bJv0wUNeL~trjpb>>GS~f2*{mm*1y-{B8z;aw)+^q#Y*+mc3|tIw*7tj*z6BD z)~%)U6MPh%>DZ{DPt?xU(DKxA7R>v6^T_vC7^mH*tGPjvG<%++>FZW-6lJfI@o3In zqC)`|E9Ks+J`j2~?5gc6xZ63+*SvwFQF~32`U}D8qMyRY5!4(#rr9=U2Dlsj>~Cgm zsP0@A-;WSdp4Mc4ZM9jb<-$&~JkaI>2Pmq|shDMt1v8%()Sg)!p1kQg?Q=V|=(UEVERLs)jFXMdJI=y`^T^?n z;w_$qzR&ypu80L0r+HZj86M_S*J_H3)lcPicp%uOd?#?#BWP|`ro&Z>PjsojYm~?B zxg>notf=pp6$Pw~C_%O#4l>aZqxT2=Z+grec3Ej&mrr^Cg>pzXTEX4pzN;3yNwM{f zW%ES3Zps*#ImKhNL3_ujiL6BJFpgVCUx(HjYN{ZBYfJ^*S5YOs$hY+pN#gUbsyvb< z{1W;=tF1b~JHn~Eenp=h#@{b3xvl)7J&;y|Imot~Ro(DSP@A{aZP<1{2>sr3V$@eR z*0;#JnBrBG_~@JzA>Gcg54$f^;5eAWf7?7}y)n6U;lxz|*``w)9{;9m(`R3kY96mkrg>evAi^-LTdtf~ znh#Y5Qr$>+`(3;q-DqOM? zkbpA0gq$~j#)Is*_Fe<2=fEOOwh02ft`HK$b%!;F2EF-P_u@h=ry1=>B_u~wxw|;r z>r^Q5{lnDw-c?xC-z?ttZY<@<9RGgNi)>1EC=l4^pW@*o_FXY7LSDkJ;edKx+&xSLc;V^_&# zoKsh>sd_e9FX?$8A0}rC(t@|toThyl*t2ZJ_u)c$=;}d{7<%sUS|c~&4Lr<9K&1hE zOQ6OU?BNs*cSZObkdHmCa^44M6q!IQdDGSLG~nrzxFHI5i)@vg+Yo*E%TEZWm<0fm z!@`jhf*^M6BuHRlQU3dDGg_ClBCQB(AYzle7t7vs_2aqe%Va-zt<|(EG7&?D48rG3 zIu$mCi4^bayw?uJcV{ko^ty^ z&w%@DQr1p{Ac84%-Z5$)`oX04EvXw%VX>&RhW16kJQ@r<(KI;WlI5D;>P3t1{hvx& z#MS+DArL9W@s}17f=mz4GWb@@q$pFL5BrM!Yd>V4M$Q_}teZ3mqmlU!j?vmc1YFpI zVS91;ci9+Rp|&pWU`2{9%t)Rl2aXo;p}>d=bCsYF{_^sWslB5{D?Rqe*`=7k<&PJ% z4h&>)kIv1ZWEB$-m=(s?%7pP*tgq&GN`ky3vNB&GW_>NS=rfj2X%@7M)Zrg=pef1& z9+wxJpI*^VV=jGrmwblxuVp*W#X~WbqZyv`y=1ts#Ns8O^=~({{rr1d%%@}VHbBlL zlo=QYI+qS$q>g)m+Z*!SLsgJ}l0;?{_7DXc_fVVDpa}p)(=+m0;{z=R#8p$NVB4iX zeiLmB*HVN?dYFV@$X}n~xfJCKINt(x#L)KfTM?vxaCb(jzdJ0entVqYw_EyE(N-kB z#Ta+Ba%_C~=vHec{h#N_G)e#jhd)Q=utk@2rJ=LylP6t2S9X=OW8?W(`|&kj~K&60Plmt;-Qk6-zc3V@o9rr@jUiH^HfDQj{QC0gPJLV85CKZwc0j zD)Ze1F~sAE1$Q!k;k2ZP!9`k<8f=g z+E?@3NcI<7wKuApHvs{t2vlO#+tCl}U*a_z9G-vyej!3|o069|9{9V5+L9n!y_mld z-aAQt|LyF3b!D$Is{UBMau~TFVbOR<=aNBx61R@iqzR-#*)4_hUaxV|HXh$V{=q4c z{k7ZgZytj2Z#?+{gRJ9uwrn|Bfuf9$VD*9d2@;($+&)r?+^j|M&!gz7KL-Nyg-Ge9 z*R7j(I7r^2AHf~9-L$RfzWW0$cpa)FVr+gF=$Rz%*O_!$amFs+pl13?I7qJu%d*(Y zEre>*29RiXMCCCmiq9S^6rIAHzSK6AoS=I~DVa*()s)(Q{E8%vGZKDEP~Q_Zyh6aS z3cl1D{Q6SacE&QWCU)Bv4ct_d+=`>Pzh&Lbc{v#$Nmntl1ZrvC+$$WVXQSD4z+mdG zX@>DWo;lK9qyNqD+i<4*=HQrnr9ek(3yBknwr0|QzSmzB=v4mWnCigdI_bqs=Q11g z96jL1${r>DZR{2t{Im7}0Z6q1KM^Ea+7oOo%;IYY?nxZn=_~QP?^{zFGp;viR+n83 z2XBu0!}wHXv_U(`@$3YL?#0Gxt3kGBtCo5mi}nNjt7LZe_EAR$-)cM#-=}S5FF~v% z+Mn?WwTbwtR4OY+T)N5tG5W%*?hp8Lo=iP3>6hg=w!EVf^H-<%Sujt{u@t>MKeNU0 zI{xkpAB@!*V+Ff9S4cw{W#NkYT)I&CDy#3?Nc$%C3%(0iyz@u3fGS(bUd__+NT(HB ztL2p@n|o}~uk?!VpK-MZq^-P;d~te4O?W?b6ISHk=V(-Qva`!L>P0Gm*_K&KYpPqI z*Ajj~W8qwIvvT!J>rdIgEn<@TB)xKEB-jNkP8g=R$zixsx3^;Kh>$FYz3Vr3EIZs$ z&eb(H-0HMl(J_(@!TPZO>w&tsW0>&Pn(v2q!XXC(`>*SF|2?!BUoG?6F`${x$R2Pj z&gfOn0>}LO%ps20>qJ!-lO3%p)Y?0rl9M}uw&`L&oc-)(LCC#sjoI^}N9)Zj->fu9Nge>Pa}Uv)A{eU(Ir2gJP0Ev1tmW3wfu$xnh1J$gdW1o@<9>Pv33QU9 zq*}B4M5K2f#!Ru*9}>mpg)ak6ALnxedC5q)|J_?3f9UhA%zlO1AekMN3JDy!{corq zggm{UKcu2sD9lUY!YxUIN;Js9=9-VC7})wc{qBrI)Onf5y14f}Q{uI^A5H*;I#Nvf zH6dHUeSeE@fdK~ls4xZgU#~-#a`G(dP6Y7YYGBIo293qO<@fv)afQioozV(A0~(U; ztJh(h13daXtx|6=Qut*C7agg-oM{Vdvh4OU$_Mw{w7ujAOrnLWz-qN3GE15jB^b8W zoflJt#XKT*O5^uX@pB`FEC2PD5(~Q0+??wC={N{S7D#~@Sq0&a5l>Yu%a0MXiqtf`OHq=VvGYh&j4#IHBwz;doTMVqs23s8^@`8?Jf=*i|8|^VN z)!LdB?%a3^k5Y-}=gO8D709?#Xc;v)E!vd|8GHG4CW3~|^_6at+IisWoh|W(E#C{1 z`|~oAA2rT5o!Th&&Pd~vYu~<9Zp@F%MZY-RyF~5m$JnQ(X?hk-ium3gJHSAw=k6iA zn(in7;wg=i>2^?!8OX?Fe$BiqhW}&(pBy1N!;5%05dV&s!{t(DjJKxDdRBmow7<5n z<3>@mXq+l!vDOKRuOX%Wf1czQh=jHJoo^SovLzyy3)ngi!xC#poh#HMn@6|V&QF)3 zb9t+NclCD5_lTjP`)7lPMzAPoDMyPLd_Zv4)Gp!aZf&gZ_Z~Ntj5fBbQ0qNvow3xm z9LU!AjV6whVoh;Xq~^9;m!WY#9)E@Q*XHrr-A+S9^2OcT5c5&4nX>3J$MZK+I`%(l zn7o39fDgl>%>@H<4>hj3F4Dez1>cNTaOsYiC}^f?rT|0%>%3L+63x;7>AmgC08=TB- zJH(il(-dyfYUKjTyNWM;MZ--6#e)yGZF^ruT|*km7z`~k271xrz7SE;jE1vq;}YCq z2!eFH@Q&`C1&g60H?Tot#mnjl3JH%C6AIjZjBzk_@LDU{-O~!8S(|-P4n2)z5&gf#sQy3pEH6alL>$a2Mt`oA2 zivaH6M+iW;U8vcyuz%b@=yhKS-~ZwJCY+CE=^i{{GmkLKHN%v%C29S1sih)}`x4ke zBDD*<_&*0{2v`PIVD`ix2?K-2wR5hFUT3{;>Hwo$zu&_sBX@C35Oi^}W7g_-7h-8Q zHaGFKg5i1r#N|2HO=iS@5NPoMd>xw2uG}y@$@ZNXiN7L#WvF#OE7; zLWzrq2ee#1qwCP3t7>U=XNYz3WFF}bb4w2|BiR&7A$=qrwYl8u#KZ6hy9wLm7XgHq z%KkzmP;C0@+r7IiK%oeIX782z`R1wE^^Q%moK6JWx6|DAriWAMMk?#%3?c1Vx%0NB zM{fYa+P?hYP0h28V3hdE(Ia8V;kpMIEv?`wt3G65I=6JmJ`+Y?b!MF1?68h@BOK@4otz_{9kM3Uprh(6n!2s4RM|Zdj_xmj-cXOyK&6g8~K8}=EwocGbP+2wJ?6sHgg@uZLl{#M``~d zPkQ-*E-RzgYv1-2+F6BLUwINsCqiD_<}jPrR|vWLdsmArQdgm3YbV(GhRk5w(UilE z3!VLR?TmPJ&Qh|-Kk}N;Wa=l~FE7&7oJrJ?3W3>b8bGz-6q9JL=p_I9Fy#MBfFwqu z?NwvE?a%sm=4GYyLfrA3rO#^S!*%aWgR{^J4~VueG+xAU%_@TOO`f4X*mkLdJ;K(q zyd^@*%v5D6rudy(fLOz=91@d(!3&=+;n-~1m$eB;vmUb}E2AuemZWAK6PvX`w{o_Cz0N)kvNR9;D)WfD27i?;ee}~q15&$z z{S(VsM%#7EJ0Z6ZKz-+w<{hSnS?bATGxfxELz_VRABPY6M0#lrO2+J$-DRHOP}2<; zxAO1ncn*Kf-kwf_NUu&lb#Pnxcb(i~2tO)(JO%4i``b}^&I}zn<+K1uV+*bv5GaaF z5yXHzFn))%2S6Z&e0dd>5y7;*&fB%qFCpXv&RBAMw+7hEQ=ne^aq*DYazp$t&(3tuPvuL3urh%TN+wVfI+YYzz76Cyc= z_qU^XdtVi5)>x2_Dv(R5$~^?)b9#JlN69n^&j-r5f$}-4H#>B^^vRA7Yg9B;+RGl_ zd0Q*E>v+me3A|4gF@CAsYx9q5r1@@bU&sEODwr=Xzc`R=!`C z56Zm{@Yd$_C(eVjNskrP_p$O-UWUeY?os~X)^jexyl?g4J<-o0&+R|c@6T0ist`jJ ziE?0I{jbCYc8tzysahMIbPHyVb0go3HhFwjs`4 zOPOKYmI>@@`V6v8rUE?i@&Ae)jAbDFgC$?L6Kg-PJ@G0@MzZH*nEzSz*FXyD0{ zreHX^immi1MgoXop_lKS^ZlHOl*#T(J&;HQR%;;Xn7t*}hFkspcH~=cD5U}I#!%Xb zD~uoU2itEqur-!o)%mN)9f(VrG;X)^3lhbrC3{3c7@Al|`N`sCCiPJBd*&@HpuOO= zVncL&)+Yy{PLx}4wukUIV3q0RNo0FhWWA@m@y%odIeExEyE5&urFZrX-O`o%JS6Mr zexPSLRJ(^2A3yWY0f^=Bz)|Y8WL9OA@kzBUV~L31@u?fghK9ojCB7{Po)0~+2SyBV z%b@XT{eL`NWmuDMxJLy91QevC#nIi}-Q6ijx0JMkh%}6DB$e*b`~{@DyF(hJV{o3G zb6w|yUto@1+urwn?qA+JJpEBg?Ek)Qbk-`bx)#hW{>(K6E<6%=P& zNu>{k{P`CMqkA-!^7R!21q2|B*X1_~;r)a+%y}h?^05N6U?VcB5h2Or1&Xh|&w%B! zu&^+XCGZh`l{c{qN)1Z&YqkaJIA5V5?a7XV2gat9kJ%83`0>RE{X+}ICR>jQynkm^ zlWN@T$sTZjO_5l)xz-sv2Wp$Zc4pEMw)WqbX9*-g@Y&KLjA5+~;Ad8s~gH$o|o3;v!t zD8y(1NA*%e81`on`=AES6rgP~mDicP#%gH2c0%QDcwcmS%q^CRIEwsisuKLv{BXVw z36nx3qkxgH`sW(25_4;Wky}jvbN-H?`Sf1J-I4R~6`J11XUnX!8@FHk3fkoco568_ zJ2jqdp98@{xG_8?fVUQ8w8?B4_mW=~DbY8RAnD`iFP<3E;H?0!Sq~_Vb?-6^Qy|MH z!FZD)oDQ7GtcN1Opm*-`)aa7z{D#SQR4gnq`!|{{9u@F~*@vf{ABU+XwEFv}MO}f+ z56@6`oOH1t_|AcnEF*SxGq`icZ7TN4m$|(ht`8E~APYue!YwhvSs|Ds{PCa%?gy14 z6ERmpesr#IOT_=W$IXFd-KVUGy8!{FR1yE<13529CJVkiMY_UD(TxJOCaSexKvvCa z#}n4Gju))vJn3VT4n20gl6WZR^N;u3+6D&Ubm5D>13l*y1s4CjGoLA9Fae!_cxn>P zhL4ms`pL~c`?wo-WTvH3f060!6_JF}`)=w!{NsV09{G`Jpu_tD64*BJY(WCsFVft1 zum)dm`LA9Nq?Z4@X)qWkci*-nT3=IL3d6eL8bS9jZ}68NG{4#()j+=A+33CROSem> z%}5jyBfzFZB&@t}QhiIU{eMv`Px8=Jgcqa~UgC6=dr*NemQ*kJSl^@W5fMTWTvmcFidzYV1xQond-weHe%Or%-t&?5ykIYlES_&q2h5VVAD@V0 zTZ|WQ)EzigE8zp=2zVm7eqqsWn|6cr_YF`Z@D}n@UDp^Jkvb#?`3EP94SWU&UJxD% zBsd*+15#G1qnassYyFL=^TrtM0W{r{>^>|)%G8hV*gtK(`=Y3-%X}}5p5@Iuyg9aX z{H=gQ3V6uDp#mVG@TCuGVGp^(Xo;*m;Rp7yMBI8kATjNtjN^{v188-79o->lik2a2 z?BDG?&pcLPq^SQRsoOKPS_12h4i+O%1R9kQ|4fPtIoelz6A->jU4zA>_x9tu8d(8~ z0ncF)kyi$GgubNG=VXycw+GB;u1`etsc}7>sVf|wT7;Ic#_ePWW|dSNEoJ|GhWUo$ zD@acpZx0Ma_{t@Dcb-cc8X8J)wFB1j@wjX7(KclW;i7rW!_j$+_ylgjLW2c|9J)yU2R@zhx&JoNi!~L~hrTQvT8@6UODZ$y z$$#uw)()N1HeTT_f1^sn@)d%NflQ}i6r^bspcf|*k}GXc`^NzC>FHa!udiOZdc8g3 z5gA#6x7zPt^<*}3@;V3Aj*6vR_@7Z9FS`vd?HCDO9n{T;++m{Set+hHb$ub<)46|^ zM=46tYB-&LMgB9GE_}A89Xli>gzva?0w+wW%5ladL#UKF^SOK}+=r8cgTtlMnRO!Q zZrZOEr3mv5zy5eW)Gg4B0SQrB&&jE&_zd)sR|0XP(Wl`zy~D%Zk#jFx=cIRXt8+;Q zX8fWwRRx|<8;9e72U!P3ZQq+Vzkd32*YE(uiXi$R zZzEpvRE#R_F9gXw_jTsqF4)?SQ1T8#=hlL%+U0;dM3`ejW@al{Hf#oC&9a8!Uv#9*UAxQ?mI!<7 ze{xJu)DuAL99sNPNcmR5yogv7>4~umejEyP?i`j;+CLHoo3aNNgvLS#<(qYVtcy86{HQP?%HV7MiTZPbSt7)>qS{S5Br#fP3Q- z$zgcWwJvWgfGMqRR^6ngZxU|CR~*&H%l{Zp|73{ZIYYk!7u6R~QXAkz9**(im1@3L zu5_ZEu8Ioq9J~OU19hEw>Y*DTUYVN2V;=sPJ`sF>FLWs4aTi3_J0fYDOr^d1mh)L_ zvsVb^ceD_TpT$hUe|Edo{e8$e{+X(4r9Slk({QI}uu;E613DMm89O#yX_;^R*kyEz zf#uQ`=%1dwJwKix(@#yUY;2($9GMtaq)H8%*a(F%{#|K)Ud^$9_ja+3GWw_@hK28l zkK0;zWRGE~?oKhNXXDAq62XEh{}vuI^v*f(F8#;|R#@=FMeYX9V%%g?{GtLVY_C2_ zyd+{1y9?CUe*;1ph)GC1diqC3dNfsqLlD+bB#4#;Vk~Sx2|sex{s=$APH4<(~pDlM=3^Z2x!%Br{}! zo;K|j%|`;@Y_isRwJbT3CHzvKHC5|uUPq@;mOYRJk!A===!k&$QULLEK!~XiYp+vn zhaWm4)DwZnmw|g~c+$SS1scDy{kRUicQQkj)3v-ez>(_J3xeT4`ofQq0n~>D&SAC} zS9uJ9lz_X>rHOHlIQmau!M6dP64q;Tvnjf2`gw6s!hMM1>(n$(L+)eAyhrowwUA0_ z6V4s{%tbxB{oS)vH9Ct{e~8MFr(pltt$C<52lOLRn-nL*zZePa-<0PpB5xuhN8W#0 zDH#4$%9JwL3+(7%V7_#Ixq(mj79wBix5a>Ks6L?)*tS1d8L-N@In{s6V}%t>FymD1=zw^(Y2QGT*GGP z=56!I&W5g7+!yOCZ`8m~aIW*CfA-g}hg{M+z$Y|&B|Nke4rnZ*@LV&+A;O01`g*VM ziPF{oXwZ`ve*}&==rd3X@;-xyTKyQwsrSC@w|&}zjgq{Q84B&M-_ah@L1&4GRCF4V z{IBdkd)kj@Uu@)lug0I-*-R|Uug?%}nVCg!%E%QDKa!9S_sr`*9c~wIE&7VbwgHE* zz>TvCagmT}WT@lPpBuTvFI%yrw49k?243sFZA!2flfxaLDHtvDqPe9gQGLBy&!ER` z{NG_9>!=1sZBN$>F5ga*(YcU9S&eVm_aiu``W)wLH&K;!*wv0jj{*+-o7In=&jdT| zXiV2aVb2U9Lg$`uKc0}OwhmK~dq_H})A)pXf8c2ectr2VFX z@>%7c)l~r|I9l>oDq-iU(|khoeazug@eDPOXNA#=U#+vf$rVLC->c1qd`DAZ;t~lS zxN8sRuJ8wZY)Tk>Pjmd@`=Qf4W<^@&f{+RjGIoPD(eG*1&#-G<s6M*Z>-)wW zCJ&Gh`;n1u{m#jqN~V(NGegz^aqqC(gZUQS?qFE0^osK`t;L1s8`YKchy=725uT86 zjn#3^nlYH{Hyx`9@;Ac89Ii)w`jx)lo+6#xkUoJK7vf@Lb1|~b;$=LPFv)SS zx&HWmrqb6PI)RKvrhdm2v9n|{gi3D zvc!0~42|gUGyCm>LrV_oDYm-ZUSd5%0-d$!;koy8$8+zV$dY;BT^=n(0K@N-65R@s z#cfg^P(O1>;&rBQ$e#5c4YdNY*E6e8w#pooUTzaGlmZ`ar zvF-D}{oSwS(+*jVN_f@O@o1p*0*fNIPv4>%?(Hum=`YxrQ>9YLGlWazP5rj2F=l+% z$ALxgoY@mrul!1SnmgUr<5LQ5SFI<$^!wo7{r(=sWDr}|25>s0hmi(Dm_2!0zo;3& z^@M)Q>e7`88!DE?1+D;E8#!X;KVO3UqQ$#6k-d~mosQkpn!XPc!>Yb_BvCr^M&#j#x6}g|>4TmI2YY<^$Z<~2pTiP-qCCFR z)MNsyqm8SY4qn=P)O$YSB2gkIlK3S1Ju^eQ^g%qSIGK|8AuB=fUZJqV)9gba145%_ z`9GGpGn6pSuY=|T;+0!HsjB!g$pa&V*)xUrPUX`lN}9WumXWXsLN6C6Fzsh*=8kg5 z*3CGm)u0fA11Inylypk!Psg17hJ*J`37v|J#e*q$4ecct(HljGJQQ^Adcu3P?gb8E&6e|9g@)WW~e7^KUEJ)C$x=J*fol-XF0KH=rrQ^EaXL)GsPe0 zPbRGZ073q5MD$U}5uD05a-}1#fpXvs%sVql$>8$i9&%rYQ2+kn;ld%SJMr6d&F*M2 z!PzoHUg#!}bMyg%=9bOydp3ws(@cty(sB+c+){IszUY5w2dLxFh`sfx<1WAQet3_-o%K30%H@o>5<5`~A+W=>=$_42TA zws(-#mFO88FRi%-paI+o2T8ZgKpGk;i`V8S`xTnc3bStQm3QuL^e@O+X!B3}PhGX! z#(7#)Gt(;78BVRBPemPs)%x0Fc6mW_GBPh--A9(p@P-)2mFqjZfurq+1!UdtjCs$}+`4GXH? zwfbDxWqHoqW=!1cVpKmYRCl+>8fI=ZOb931J05FlAU~YO1VWIOPXcw-)gd$mXgw4S zA=wAb;lC3SuSEP>d35`Fdnu6!@CZ%@o_S5g+fd36+xldVf~_ zG!kzpy58EHE%zi*HGbYXSb1Brf|^btpAt%;@|w@S;6KN^1rthwH^4N^`?#{~+F z>uW>D<1c*?$ixnN>_Pz{6)}#+q|Tzxi1YdbY4BL8H99@G|UjG%plqADEJvDriN zW0U6Ae1$GuU+Z~9QZmf>il-O!8Qq5M;#A?jf_LY;k{a-7B0d+&BIWeU70g|nsEhrX zvlNTywFG)iKG5~v={Wqp6P)JzNt_zK1W~dvGDcMC9k`^Q$9DrjlONC;1o-&=J0215 zuz1p=q@|@HecB~DrvQEw93+6ByMh*%6A(kYx?j;gEXC3HhjAJTAxVpLu+f>xyZ=JD z!(=q9rLekVs$$KQm6Ji_WRT84}m=}pDtYJwCnX8 zINnfr#VJhzHgy`yc`dV6D@VhJfg(Cp-E_<2Y z=B}t>0jNjBxnT{JCZ7JNg4bQIaLi_kG``JD&uyCG07(^{iUgBjnk%K&ktRJJM7=nf z=5*Fy&5&?CeHG_JeVabr8e}Q0G$twEm&WkuxcG)of;Gb9Ku?`(Be0I^z)ge)Wbt(lVZ{LGB z@hsH!eWnaA<^4-01tb>afw5rn?&~sw(HHsWH5jcRh{A!ZK+xQv=d)JkoyU;XG*_BU zeY1djF9{QNH?DkB%LBp3kdQdNASct$V7^C5;Y85xUhKo_vLu};rGo$bhJ!zu`$XR0 zfCW_o2TYrQ)KZnSL#ILwDirnc;V*Mp0od|WuH|%_52(*9^{w*y3UR2{!}6n%e#9>7 z&SXr>l;pM6_jUOMhbYsKZ1lPB*!fP9FgbA5M>WzZ2>kn2RWPI{Bp)G3T_;zqls+*K z3=I2k!s?gPDs_W}7MV)qlxuAJEdSds!yBgegsE|~$Mc{%k3QW|x z%VcK5ZZL$s{=RaC4}f7cRotG1EWY~2I%Dkp4UjXyW#Ke^@hSpgNT3v%)w3ZGwE2-i zsYLD$YOoz`Jw_q%eYO6e7>pFuAOd-#Ronf)0RuxYRJXtZa^!TQM>EMX`VHqdq<|CY z4ezsUTjYr8_Wphar`gIMonGT0vqBargVQ>?UOCB91b#k2b}0-F4)HX~RPrEgD+$pm zO{)noiP{hUyPQ2v?V&3pteOW(IBI^QhjXwCX=LjElVzRIumWaYPsnD5z$QW6vY2n^^P zT@0H`y5P^0>UaK-4E6j}O7Rr763{F&s)6@$kdqzN=sEwHGRI0FSE1YI_z(Y;xomE* z&!=4+J7!C6Gqy)mdSLJ$E+Gr+q;kp@3O>S^Rb7XBZV%5yl{pS{ub1Dk*JpnlzGeG8 zq!`_ zhlO<30S8tlPETNt3!c3`qSOc-1>)aJ*qcngr!H(y%&v}>NP*ZjFDR7as7>vP<*mG- zJK@ayvV{hQs{*{0l>e6Oso6-Ic7~QOA%5{lNY!1%j_-c?!}B}%0r2Ta)( zs122eVpnt|i;jsFwWY9*+RVgUO%^zgnCjCw2ht3G((qqy6{Y_(S0YtDyI5cUmT~jv zFZwoKd4Ha8+?(-~`i+C~@u)!+R7wKVHcu;c+kn$(Z`4rjKP?g6K$@f2C*x z;!?Qn31S3~_zs^e&(8K%S64fGRNv#Bq_5n6dns~l7K+o5Pb|K)*Q=dbD8c|C!O~fZ z(1(8DNp)}l6Gv{Ni6&x?&%Ka!9{NE!PkyJx(;IF8@nf_jQ7(0^|4X$!#6#FQ*09Ya zN(An>STgBWp`x#>_;6?T3X$W+^n$R+_&D*4qaS4Qv|&j46E?Y!K6jscaFeHpExZZ) zv-{#J=#$k{#{;`o_xkROtmk`C#NN61U^(s63fB1xS>aFfblX(r!B4Dov)9qIJ zHIkJ2BmpZxx3XlCu31ieS`9v2SHACT<4y~>K3PLPX%*3HHL0-L3CCx=2DG-vU$Nn# zp{?&Y?^1-mCr}M=u(q5?lr1vD7CYW>TG>biMU!rYNd*BtJ{4=j>Qj))oCyY|HSpqk z0}tO>&4|cEe_~P+E)s?JW*oL7D=VwDj!x21=F;-b3fT9C)3_tR0+6_k?Cju>zvMJ? z0U8cWEU4lh4vx{>wg=xLk^7&It^B&b66Z+W{I@kQS7WW3xzU!J8-#_0)l>}*>qJjn zKz8DMJ-M*Y-&<}_io6WwW|QO9N?_o;N3H9I%%NU8GH?cj0InS(9}RSVuqDrh_w|1l zT=&ah{{T}D`6Y9Lhz)@B=_|*$Xe#0b_(m?fRk7Tv|LBNo9w|IV%+Pjxm72kr z2Xc$M-7aXDeA+%l#h>=~hnB;}>dgp9vhw%>0T#XA@z_t6j&}c>Db9I^@&%TU$1_wr z(ei3e)*pp(PYML77OS-=1wWDT93}CsX~X@TFJ;jcpnLaM{MtK@kyo#qZsTPk4VGd5 zomPw8nY{{psAbH?*GJ;e#t=qJLX&731rP&FjroSKW+CfabmHRr{ ze>0|mnLIuA_{rR|^`3H=XVh3m=00(`8Jqd7f#xbsJWozCV@zz-OVxj>t$7n>Nh6W+ zHBhAck?_NI(U(E)b(8lN!4BWi)lgCCccTw};#X}rTgc!O><+7gO^`3;Ij+=Lzl>_^YJ(rJ;T4+2Nd{ zxlR8ryYP0(arnKW9t4l3MxcI9S{f1mJ-XP1mkm(lN=0Wf+SoH4tE#XQsR#1;WAuz@ z$9U|xxz4}z=?>&EG=mhnjCb%n7yZTXW_EG!_(G7nd}VI>SD;_Q=0G;&O*Fk9xQ0G~ zHg|P(JOmu+2i!RAE35FQuANxCC!pHXOkTckjc2QJ4cHKN8eBI%fd-t@UqxBBml*9X zK$a4XSc7Gwh%X|d{6zPD=J5dt7z#SPw|E&?Jpt)V=Lh7rXrIS^1{`v{!gM?R$BikQ zdvCXFKE!I~*3e+ttZ-jHpSf6H;+WB%A(GCy!?GRO81eVw*M9##kHh=x91nHt_E zo5cpp(E75%wm1*0A*DhMqdKQ|VC!4N{k_tJZ4I8u3U1}d{NVwbI3G|UniGdA_iuk_ z!M3Un^)vPLpGo};A-61*AFhMf^zOcsV#OBG{NF%;ELZAzS0@Dt0`8tEGwcC`!2IE* zGxLf(5*UaArxB7|#FpOP{ORcnxuHNU?<8_QD!HPzUlfeUho20k@TALIaj3Uf@l;Qk zom^k6aO5cBe>&vJtIU(UEyZ2i8w*p2rHG`7&-@_=oPv~i?^}OMRAssid~kCOZ2j`f z6SuBxxIb9)Wo^gdi^{gS+&Qpk{k=praBycQj^HF#wTT;k-U+6bnd@J)=U`LP43Z0^ z9E4w>X1*4sMynMNF12;#^1a{1)|FQOl=HjyulTSvO3s*#g>taGAPnsN zh%f%%i_o?@QCSACTWMytJY4+&(%^) zmpAs4;IPhpcig#l!9izh>GE))$gz1NT99k*4jiU!;rhTD2cVO?~KFuMgKKn;Swr}x8e!qbpxYj*!h8E!B>hTxTJCD0tH`s zooeF+CWLv>j6zwr<(pf2cK+!4Nw(Tq;Ca`@2#v)j&Z4Q6ktVTQ|NZD0*Sal7hVU-^ zsF+dwlCewWU(LD^f8D$7_zI3p6siM;$j(%{h?$eCt!E8IIQANqm7ZqPQEHNUfZ2o zg2%$e%8RP+9aj_)9FxOuYg~+=uT5i$_;)2mLMoOB$M`qADhM0wqk8|CLf-tVOI&G_ zSOxPcVV;a75wdb`k>ERB6j)OVj1n21>JLFVE@Rh_qV{AnhOy~#rZ@}JaJQorWWn_g z?&EmABTKlS^}p|Y8|oovBJ$&wNpG+i?$=FkIRJ$_szrMWm$itL6`+~B$>6Y#QK|IK z?%HSd4u|`Y^)L@WNf(cKTL7FOsI+pX`3IOM`vgI5uS@z^*gK>ZDJ051gGbzewnqW7 zsz^vMJtHA;Rsm&?_to+7C@9eh3f;h7?6u+O1+eGSRmpcOtBc#TyWfwtgdcAcd30?v zW}&y?oK7nr{$Ir*g{#ES5Z70pSS@rUYHDhu8Y?USMRo$d^r(k8 zivllCCd#}$06tz%VFTq4EK9jm%4iVFp@afk`9fCo#8Pe(hq**}jZRobYw@8z)A!BS z&a`~y5(@EGj}G->x&v;)S&-~EbC)KmO;W2zJ{Oj+qT)wl6xgU^+XLCl6~#yGWNWmy zGqkDpl^pN3EASf+#T0Lqg9j_-3rbR3^j23^R7Y1N^cvQ_k??)olq(1M#$Y1YN^Pp- zTl6Q;3*BtS1K3Y0wDa^3KUxft7Ly_5&2e^i7FcC*;t%wh1O5H8lapQI!Kki(!tp~F z@*n9D!gz4DjMbVU0+reAZZ7|^k^wghfORt1-Urn)lZLr$FaB9FkWP$n`cO#0cif7-hxXYd2A!X~6L1GNt_k^$x54uOSrTG|a^& zN%*DX@rh~E$4r`)cI!Z9*wJiWD=Q(3VL{*B-7|4!6eRiSY2K=_9D#rM(rZBlB@Je~ z4Bhi(B?0`7JXcdc?PXcp?>WGy;ijjh8!d8cgTS)^tZPdzASglmBnKg!V#Nm7(K^V< z7^hh=uc~Zny7k>s5y|+w)XWP;Wpe$}2e;i1cV#Ls0XiuYue_5_s~zSVKlF>=!~e@G zKyO7&M_ZnhVaJpof^N=vtD51l~yHDLWC4?Ky59{-Sw1mI(tN-;HtiMm{J z&xjgdabJelW(M1WwyafkrsnvecIH%^O~wUkMAAx2oqx?zSzl4;yDD%moUU^BzQ8#I ztQw2+=Uh5ljFk1+k_xN}XNF8_t-Qj9?JJUGAOrE@l zP837fyqQPmA$H9Jih*ponX<{eIntSm)k+)U$j`*_Jos1Sr@SdJW#6bm^qnh53DL$j z6k~EC0Cd{iVjGSE;9Q#Yh-Ct*@XkZ~uq-dTj}M19F|@lbcU*?=z>dqMgD0&`W#e^} zKFAa0G8PUlgGLAKdK5FGGX=9BpT%(L~wgfJKZanV{_0rS6wNE$sK+Nk08c zkWan@f7@5QE`I0xqrPwOg92a~E9V?$hFd!iH77*7*2$vXd}|B0UA*rjyKxgBQ#VeP zdKzX+uYHfLWovvfCtSDvHgS6~5Ik&R546!S({dx}Xlb4Bw%=d@ zKm~cIG2m>7H(m?h6I+GV<>6-M45YD5^-Wt_T7ph30a9!biC-*K&1ApfGp%oJyI}YZ z{-iP|=QC}lJMM_knGyjW-YcY%k`fPoN?vfOfm0kHa^vGW6H^ZCmM*E_3Bm^SvDC0^cj3KO!^@AgIILN{x_ ziQ10cbCOWx3pa(9>tPj5uk`RM$7m+R<82ZZ@!p3S8gI;zDrD0}e1v9GG(_8%9^1t| zLim4dCvkB+lLtKLh~>8KPqo56^VQ0Dl~u4kE2^uw>TKlArx0VJdhU*I$&PT6#PoQB z{2d&mE@3{=>`$mXD^mee5IMjka+)Zm;T`yof9huZmc^`1bRVhdger#YXc^ z%G#wohCe9=cVV4r8u2s#(!^(*Qn)|)l~T?m&vGr)C;fUnlc#w)JY$@ty*8GLwuN}- zT6aAX`NEMYHDxo|*#vv>Aj!|OnSRTHQqh#0|tGF=86g%j=jBC_jB}b zei%J%csmcuyPL>RCF)*%3TnL??aeaXw%**9^85QJ;>+?|?h#-|@7`D`WxLSqw6AC_ zY<3bD3wy+S{u&8T=!r>BEi1fudS(_&1k+4)Od+nbBhe}D+g}<4ENk{9wpJIq+r7E~R@ETeU8+tdzXB<|~`CsU-%nqjA(m+2LOLBT0%#8%IdyRbz;0_Gip5;v` zyZ1TsRDgDcO<38V1zQ3~G@g#OKCk0a-!l)#@rM`58w6WEzVR$TlAQd3_K4zGzVJ6XP;hPY?$VUjnk>;U*-F|3bF zO-&8OoVATnc`8N=z8b^hV129Q@l}R}!mC%JAtD08H=br}_wtv!0$!)_gWQ^Oc)F9me(A-IJ zObh{Ih^#GkJq)4zFi7yUPw=oe4rG(iPnt8UbSLt6a`Txrt7Myw{_zG=`tQqcAb=i# zpWL)uzZj@e9E#nzaHBX=akawkgXb@(uz*;0%Hp3l07 zVZg~+D9(X?#xq|HQrtJln|=uNJZbNmTFDM4a8nFPo2{B7u)TbaZp7$1z)+ALyL#<>rz=g*QG-#x91C3~8%S&v6vRyXKL*cqp1Ab?uG zP+=0k=V4bZP!~}Dfc^EIz&|NA(yg|p?OI>8BugNvnm5YgD)<7nrwHfPvj?xV@8o?z zFjHDwJWkqhnAc!<3Q&D^M#lYE_0zK9UGV)mB6(+ed%hdL-TTN>8=q%{PkRl>Tfvkc1;f+<8}+?qm-5>ULcd4Z!Vz>lC4P$RF^q$&LJDKBgf~goz5>bc8L* zAPIPP+~42VIv%ovM?GDw2)JNex_#cE&3rm zESsvl_0j+Vc^XxjqaZ{+zT|@resn{p@1q^ICT~}O491puJQE^(imFoHoL*9Ti^5`E z`i+sfV&b*)W=3Jp8-k3|m!+G413`}G#Wj8r>3qQ4@lO+WZ$ksRd77Z2ZDd4lHJ6qT=r&J@zBEm`2Ut22@3-hHX5BZXN1 zH206KH86nCL#3=eyYR#c+>jOR0Wu1tJGA9>{yAIeG=0A=uBt?0$q#Fnd7x+M*@gLN zi}j8qh;;83I~n+)9Mq%)@ z<9{X+wqZK`8Z>FLjsiFpOv3Y8soL_4r11xe7`hC84P`?%YbB^#H|cGw02snvrMTA% zFUT!p%Aa{ZdTh+qxuYTGcQ*C=JwV|e06WC`^WWP;>HHshUmafUO_c~HL@#e;b)Ry> z)P&^5sA+EbU~9(*^ODq1axXj6Z&2*0^J`d9whjHx*5Y>0&A$f2y2fz9`JeU!_!VpB z8=~NovcHdL=qwv~?-+5k4iM&lB_79BwY3S4w=?a0?te$Km+soN*BD>v@O@P1>3_Ge zjd31=CvqkZU9g8RKx9?qU2m-7C`RO}F5(TKeX7>zi2f4AQAX8e_VKyYC1^R;FJEK&*u^LU)=&V%L3%1y5JU}s@Pge>Qicc zWlgpKlZD{sIfgUVN0mx=(e`My~KeM zPEk=Y&^7Mh5ew+c1^|B<()R%hCV2)+?wwpWhya*_Ih$exi2+M6(hB`a?rTRxhyXCT ze7f06cI(_eY?8atDb=g_Ql50Nk7ejNdZ^uw;&mweU+NhFC)oRL_AV0eLwG_X(qV_= zQy|2mhLC9^nR9!5W<3%nXlw*i_n2nYZ`5(SW;L#h#!e1}LR^=+%W7w}Z2bTN>U@e( z4hol*Wwgn>`@-we)}5*1HV;q+YOS|{)V0xO&Te`VHpCCqv7+}5xk*7lf>*vCuPV}q zKq@cxrbH)W1FSOu!b{5QMEc^zi+0VM{~vpRpX~v)mJY?>ti{`4d7_r6gYUdV{RArpvxSfyr zlpz|WjvV_`-!}U2z)iml+bi@JhDK0tnoK-_KdXE-jU+`BZLQp2nfR!&Pff_0l%((T z`nt^o1#{UGPevIwoY-LeU$&CuECcD3LKkEk|9CS{Y`S0}1Rr~D%`oX#F1-b7pEW!2SYoNkG!gQ$Qcg^mfGRXSMl#Ee09|W|c`* zA-mw$Ko_fk9@#bd_g;){T%r(dzxPd9+{NKY)L!J3hB_WFiyTrq1}l)di~|jBMu(k= zHzP?DPk%L-f(kHJF=xtr&G$C;vShlZYQnU+lZB;X9A*C6)27a z_w~s9UiODeM06?_^;BqzP)^-#<;Sb8HMs7o=d??4@$%LHWp%`MAc)7}lC|eZouq=j zz`)=G@s?Qux4ZLH*2yf!?$MweR>aV)7$%Wq833~45V}fF&w3Vy`-N9gGIjN!8s)qe z4#c|A0Z1(e)dRKPdl14gr0ZFWm^g^n@$wLe=7>Pt1{%!&U}U;|0B}lLun2A#kT916 z$^aB2PhC{NeH*mdSAm9X1JLibT*Noyimbc+SNk(DxpW`byTXujq_aaL-U6{_(ul|% z6RmRE=vNV3R z&h#JvIyfT{&>*Ar(UrFyVx1*TpX+jor7=PQH}3YX^n0 zwm=>O2tg$U90Bt?KHxMlvh&er+)*UrtGeV-2!CkOvwZU#Y@7LX`z!c3zY7X*QkKqB zqDd_fg!HZz-0C#&5<47<4h%F)@- z2G~{sMU@qv>m}D4-iwg1wmRlJ-I;H^SUdoRs1zW6<rX#8hLzUQEjXhxWN)8sQV&A zSm3u>#KFZkl%VPIB}%v3X7qmCMjxy2ysU6`oWqn){Aa<)JAt;V&MGZ&t-hB#yR;^q z|Ni}xEH1q#JsIR2Iq_=0TlzmuDTs0azVTwX6EzZ_#q)-ti;(gIo*6>Y=0|}-2ppD^ z|J~2&Yq3dUdUB_D5drL%hCdw7-Vg$p=I@hz@QA8+_MFIbKr5ZB!zX-}F>$VYi#t?$ z!Mcrog;%fY_!_^(&vBTW_(_gz4Ec&8`@V|>2ud&g`Ezvp3mH?{=%Fs3ciex;-4K4? z#66owzv%yzeIXC~HIR+I`qPwVgUx9;t|>`LG1?^8fYG^!VV*CErHVs1r)O0&5~@IRHa{1pJNJN7*Nz7#D6*# z-4Ngo7SmhH&RY57v{bBbVlvEZSbxvkc@@zKF0<6k%ptJ%pa!p$vV6)1HK4DMz^S#X z)3JX98zGh~)-GvqnBzcLTfg6-3K4;nyx`|Aw*gf`H^AZC4!_Kf;uuuL!pFyFqN7{w zp$N>VA2F-8!G#aWaC#kTPdV%nr}1Bu+~7&aO<1w!L*y}$C5>85DDHpIVdz$@bk&i- zD%<+Rpy#*uM$O{>tOs+Zit*CM6o&8%8e%1aZgW=_leTYJ1i9i0ulT@tF6;yJooK39vMl__@8Fb{&t{Yoq2a5H zn`CLf|JVkkc?Zj_cU}zt&j5<^k#2u}@~pwvYa#d^oDC1vmamI2BL3vQLjtb9v<2y% z2&h#0MBaenO86(LKAHHsq1|QN-}^7V5=9qsUQksvZ2xqU_OFrN;lh&+i;^i}F>m7@ zUtas?Np!&s+pKe@#_SE301TWuG)1n^#=*8^&f%CX)K@6(L(QCuv!Jzu{T`COPUbK0 zUx(@KfqBC&t}q#2+GCkw6B0!WEw$w;ARGDMq47$3NS3DN*<`TQ%b9C?MjEM z`0fn^C?T;3pv4)@Q)`7KWUfJ4Q+V@&vBt|t2(<`iu^W8#=R5I;CS9^^dhz_B-S>RO z^%KNzSIBijWh0)_PVE=gk4XVViB0WZ_nsLM(_U5X;cRAgZspv;Xt!`G?qBzkvAyS4 z74U|72F%`%Bu~C-WlkMd0JyQkD&APPJ@Gqe=CyJAWK%0Jov10mP}(C3oJ;Uy{& z=C2ujpl6Xo(nU`SG#=&w>VNF-!k+CSg{rcP7Sv{=M zKkv*`n3e})d$BK@?UE zbTyOaihRTy52zMt9MX*5kXPaEKe5#(%+}Rqu{J?4H1=GPDKX&NdVs9b@$?@!K~c$L z1m#FOQQUihBndFRjzFsHjFDSZQfKRwIxE^!FqFE0g5)Yz{eA_dtVdL@sZN-H#V93- zVHMU6XMW-WWHKv0kqGJV1z-+5emw&*Gw762jfk(32Oke;B5wiIUhQcLfFecj4mh8h zfm(6tDhv9ZOSu9KaN(E%%O{x0#hN_8;2`vHy}@1Ndv{r?Q&NaqX*obJJvC+f@gBgV z<1liVFrM_79Nx;Mu9jB5;XT3-js6)CkX7rLZ0wn;gV5}NjHRB5`Ns{4$CS$ln~JAZ zc2f+~)6;h0*zGq%&TYEc0oaIR?emx9N>IPQKSBdM|NR7rFqj6}GQ{(Z_Rn?o&$0Uw z@V4@sn#VKl(dm1~d|9`M7kq(s`FCH{^K$F5K=zsXg9tPhgo^$J)`*53VI#lQ@fIfJ z;tV2b^GtLhkQsk(#o|Xo6!2wE;LiO{je7a_vaGgc*s`$x{8{tjM8muQRav{oL&5#L zNE*<6qI|PXfoeTn0CtIchsUY&uBo4xUMR1nkqJ>&1K`Viyn1n2pd(YL_?7O?`pQK!F{K*n-+X{4W{lRgbi9UHs)9hwpR* zS^w4udFJNK*+Q1b>Guu*cV7Q#-w;aC7I>_Iz?f&^g9sy)KkjpU+25`;ekl>2acy8! zgeq1Xj+FOEp0`raP6NU$kA-ok9AUgAmb@&J6eGvbn>9^#gT09zF6=x5T5>1OMJ-0x z@pN>R0L7>y+P>RNsu{n6`Gp3<^+~VwdG-~5K85Ue{zued01*9lGg2gCp`!B^VAy)& z1qp=4pX3b`(y~cFE;{-$!c@BusY3dM5AVrm2q5j&7zWTX$NN!xhya;qKm!qe%Snn| zlbb8StJ}C+J|M?n1PnLEN4CIQR+Qz#@TBs1l>xz;P7pujD3VDQD{$4MljprX=iYxh z&@ja@c43rLN! zFEz`9!DhL@|1k9xKvDKx-wV^Z5YQKS~0*=s&>lLZ1QUf=gB3N!lDty37cmiLKit z?UtfIfi`GlA>IXLv9bW$Zr3WsQx)3@%gVQJ8DS|4yU)y4i@^a(_>>_LYR4Em&qyaTy+|?7T z5|(qz&8BPwTgc4?BA?4)xE_v1GlNNjp|0WmYt6*VJX-yx3R)FL9PNGz8iN}ZrrwQ{T8X@^+Yd0FRkg+(_Wk zA%EEtnUl*0+Gsr4l8muTdz%UEKy-GAU*T`F$_9SGhqMIl6mX(K~l5PQU%jX|U zu6c*atZflAI7E8WZQEVd!IR$f!P*^~&ueU;jF{qPCy$BiTviXTd_XsvoLhk7pS6EF zJd6XuOXGL$zd^p@N;N1Q`x@fRI$~>0zetBCvr6Rmwk)^*z-NSvS<~39)Q>Jrp7dE4 zuGz1e)9+0`e8{dRbDzv>L-UXnYPpKzJTdHNU1t;SJ0I<8@v_rkLS;hzm64hXJndW zu8YiR)B0za3>)e2uRBsIL&--C1_>DyZ$GF+55*8`(H_;55zewvD_RbY&6++{8!%NY z9!z}o+Ol4g(6zm|No?C~JZK+z{ELsl5A!|26;xuP*_Y1P%$%?N``xs)=lskZQ5 z4+{(XH&t6%(Nt|7&tX|%+;uM$_+)go33$jaV@M2W>q_F z=@o3i6S*Hb)C-FhXcoUcmzUvLL2XLGXp_Cz^iYt##IzoJ<1_2;i)C z)I5Lx?GSUST{ru`@p4Xp84{C{dL}1zC@3gOYHB{U`d?QF`TLedq-?3g+HxI;YRYP1 zZ1C>_ng`wJ4?@epj(IK~9;J0PcXzRPUj4tXQqu0j9GCY6Q3b=^2^16>Q0H%0jX7P7KkBY(-v(jIZB({`P({)WgM7oo$Z>ys-~ttp$|m|vhb!xUHD zwEaNMZv8%Tw(Zlzs@%$#c%V_}PdIW(ve)wkv9StPp4e{MDJzB~=)*YI{?)^@v2T_; z`I)_f7X5|XjrgRizQ;o?KqxhvsH|?7$(f?O%UGoq9=lMt#40qseMHz{mU!~QN?7)G zGu3!gF}BsOuH%mjbQ7<|9ys1bc}JiM7mUoHffG#fHW#Wqk<%TMWo|)LKj1gI1$12K z&j4v}NHr711IeGJo?#v680s!K;b?EaF^*Fm{Is7gWu|R(czHF?kS2b?8wAeNFnO75@Lzm*8r3zrIsE9#?{iBr?+>B*YxPX*vCoco?iWA6M2{w@ zA_rYyOtEACRirYv`0Pl7__V=Y=?QJs>?Z*d<-S7W=d z1Kr=4adtj#56F>jOJMuu54Pa9-kSm9E&k)@xhbk2ix6I8L+M`Kyk*6XwY;~KRa34E zm+>Zt8TR}lqAln{id~WbYeV*Hu@4!1ycy2%VId4dTYUIIfJaZ3`-FQ??b>+in02T^ z@o$E`i0@@1x8;0vXXEiPwRHGcD_kq&Nl(VOymow_$T40C33oky`HBze&d=GdqNX-k zx0QZmewT<(4P4f-~<>=b8~ZBQsH8%Hjr?#3nQ$^r17~03eEK>KGx32)4s#rz=`z{bLLs|EyuRT0aoGh>OH)QQg=jwTS zM=iP3wE7l~afO)~8h`CaI0myP@<<7_8bj2)oBVpz?}Hv;{GZ?2Y2H{jZS7Ii^HJsB zmVe|cWy){U1lf($geC8ee3+`)h1pE#?GGA`;Fi@=ilVZz&>MH28oW?zivBLpF1PeJpQnF#GM9$0g^Tcd#@zS*A1uh6-aC+- zeqL(=(OJIzU)K+4!cz@|v}M4wdF-Q7nGQel$LP0=LFJN`b;O;IVa+yD_PKNfS$%Ro zUkYd=c$;jPrRAa@u3pG;rGi3L{BV@I497*)K^S=d^3hTuy0OgGX@^bo8nMLBaXk_V zD|xx&t25cO!VeF(LB!sEBbUZQRyV(>QtctJR1kl>`?=PaBTK@Fey?mcJo8(MPnj{p zR-R3Kb~5YXjv6TlY`^jD{RQ{XVfce(tGv%29>k@8Z-1pXYVnmJ@WI6M63J4eNB^%K z)3jXAPs0^=CZCjc%thnXnK%$q{99k0 zeh}+VxaY%~Y=e$g1c(>&$mP~#wc7H}@b=fey}d@ZwmTlN*}L>t#{tF);QncDW=4hQ z2QXSSNaeV5X%W=a)NjkRAmhf`*x1;S${`IFB^vr}4>ttyDS3XaEi5gSD5r_#fBSY4 zv2m#kL^1jg+Y(}sM@zd2sp9ZpvcfAon)J8}hjPKeBVOJJ)IzlQD1!OJ#la{9m+;Cx zlvx|l?0*z1zq8xJ%Zebp^&k*{X+se@6G?;E!sI7@^c%GO7kU?rdh)?`Kp{bGo3PZ@ zYE@K>P#D}W^=0ShUrR)JJ^C5$i*aa-W5*ysOOO#os(CT_|-yq$ws7pL|4^(gFj8#!rNz0- zt(%NofNUhn!he5Yb6 z!(uAYew=BSU;Vs)-|-RSn0iphyBs%9|J!$|e^)!TfAWC<1O0s0l6ACqZ>1Xx9j)Oc zG9?V#TdNkxcf&F#$0t3vVURyN+qRY{Or%wqpd*z1aMF)+W5ehQ6!6<-$@JpGhnq2P z>?V6`zeeLNyL~dcP&%M#&%)EM66#%4i{@T5`tkZ6FrD@QZ~aY@9xi_A9DZzm-TV>5 z?Q@--W$~*-kPx2zcaO+Lfe*k^*)*_CWDn|hcK#}`l_^{=Z5I1vcr=}EKR&VH7#AO3 znx7x4tbF!RlAYReP#2(492Vryz4-Kj6fjKfpx|;8KwE;}`wDQjzFOb2?^+}((K4xWSzch(C`L29m zSYp}O`&~V7QiaKO&?O3vvmK@A4U8K?=3hFm^{wel{);*3(y%lq`bdeJf4}Q|T^slJ z&53W*C*yRTMtd^~GVeP+88Ft9t?at0lh9O}fA%V8O4oXCJaqWEl##Bs9%y-o~leJE~@C8_L zbgKJ$WZw=IT=V729Jh7%!!Z*Pkg5@tz7Y2y0Ad=%Ixhe}r2|#olN+nJUkvzjKcd3% zf2lUzb|f8PC=!)EX0}|B;`)tk;h$emf}rU|bwx%{?a_#hW{9^ZSI@b{+yjtkpPozb zysQakUbSPW_ZZDM2_SIMe1wGV=TYj$aT4-bc#KX#$$qd6000`Jdz_kRgd& zo$vjB7#oIk*sXRz|6iWbMsz^?st_=rwC8vA>n=)+D{FrU?j^7S>(Tv-r>O!CE9oCu zh-6-u)z)jh+im^>XBX1&<7aWISuZS~!qBP3H>iDLk!_V5kofpDI{KNjJ32l&_D=$t zW047HCeO(p;<npoa0gU&+LPm?$3QmnvAIlE& zh*M*QPp_Ydq<)5}LnQSV_z_)3E|HN--LpJbW%mM-egz@T%EPcbVFm_e!ebW2^!ZiY zOO9`GK~@baJEmemMQ%3{D)j)46Km_+w-fS}UTFROm(JW(RwgxC$w(7sCgr96M^_JY zeo=w)HQSgip>0Eyev-3n*Jfq4(%?^+9zR`c>@vz?lcOaSl;tA};-d;LDI9w$7q^P< zUv3{wDs_w7rQ`I{#b;BCSuUwnQ7(GSsN_CbP;s;A#&%tW3=mWRU;LWRP#MXp^{Pma zO`g<{n{Y?{*TbY?=7WL_G6t>U%0f;1CDv&64dGbkhvi<`i!~3lXv|zNl&M!|pV`#7 zK|k}Zsvg#~88$Ke*)mM(;CmOAKP$M>z&f%6v$De)E~xk(@=1zBeQw|UI1;d5MR}c!3D~6p{SE^3L@D=O5?M8S=?rK0dts2*>OK5q8V$ zU6jl+WnP4nQz1mw^5Sd+$QWT8I(A(w7)_{rJMju++?y0;$2@NAN5=rrQQ=^e2@?kR zKwC7k<$m-Ctn3RuK(hzd3^CRsGVYCO=D+h_@g<%|1W}4V1UE!bg?j}_%>}Ni$^5CZ z&61X_M(I>9&$j4c&~fDg@)K~KKixvYcJXRITih1B>zTWVj&x*QEg}lTpZI-T-<(~k zkAJ2Ux?X&Ug8a&k<<>sTi)x7?D^FkiWAz53qU}@Q_q4(bJ%6avs-id+r<=)~l zY69B^0pX%w>%ijA`<%}Ge%ZSK2aa6QyLxGe{dhpj-cHz;>?Ou)GCL`6RQ~I{BYyI4T5mR* zMWM=ZLrctUzzoPC5|CI3A3A1naS>i|@N|3qWB(#7s>|ytCAMZ=8!xGm=>D1;skeGx`)8jwRPAxA|b zOH2EQOVzzDZKF;L9R~Xn(5od`@=d2E7@ZB6_$bbXE}~!D_G7w}V|d)WK=#EkqRKe0 zr8;k9FXm~~K$yIl_4-8^>8Mrw@p*t8k7fLB`>I;mbM)^KiX|7Ew!?ZsdD<}sf0JTf zGZ+3`i#z#EDJLv~H%*h)P_fh0IRD^}Q7SMOFt$szv(LVP1s2LI$?bGsX!7g%3Ivw} zAaw_EhjhB`$54>l3 zMEs+XUwe;rnG}s*I2@Gne!WMbG0G0oY$a$)4^OL%7PWYEDuU_K+(qAQW-dPcqMCfq zi3C(^%sn)ZY-=hg%u4>C6mb|}P-XZ-dG<%aqnlRQp0^@ckALS3P$@H41C9SL)CZAS z-y0CLpnp1=lwVkQ89^rK!Bz)uIAFK5Xzh_ChP{S!lNr2Az=ce(Wf1&lKLAHX5S{`A zB0}txO-iK=E=5#Slx;%6*=7R2VK7hljrJ=q)MQeXwBDyi^o1BG6x$$=XE-W#}DE>`&OINvrZXGzvOq&`%X|d zHM!T9*@49;0>GcaMr9%zk<3d2D#8f*4!S8KcsaT_^m!i z0?`_MEw`kekmvG`ul`O&B_`q`Y|hZxxH@GR*&nNb9L4vqXbn07L&ckyO4`bi-_EWC z_4cGV{0hn6)Hs5~91nk2<%LSFjf_08UBZY~V0 zkEpw15Z}dp3uE8F53uoaN+91-rc4o55>;0V!6#Z7&Y~8%WC+H`pTYN4QTdJ8Y(w^i zr5#eyuo19X1!O9>R^8OH%1vTnafyLf0Nc2h+RgPHk~vkDI?G7j6D&*xZ*@dwotln5 z6^Zg~-;}P*wTp>&^qQcQdM|e`RGl?^Z9;9d@wX$aH{n05!}Xx&@=|;(>A9KDY1L|} zWezN{ zAvs5)v^`wn!ZmNybILC_n9qpe8r-b<8VxiAh{`#76r$pjvdZ5$y;sp2$HY8&KVAXk zR+_0s%3V8qkk{P`Ro9(>E3l%eiy6V|>0KuxTS9nEJV)_z=^cPDf^`@5fjH!287*Zz@VkT_NnoxHDR zW9N~2B_XEj?cBNEx4D~X{ziz#iZ?fpi8B|V`Z}?zcZ)qhG5AETpC4k)l(#mS{_5_uh;T9HCHRP!I z55i+mF||7OeGgGQ%<36Jr!Yf^1$7T`rm- zm{ArJSIE~ppO2Xo7lDw#c|uyU-={zVpYMs>+k2BiTHswtmz`{f2SyGE#6k07PPh0i z@{T_~%Omv6o1$4G4li$sKk4j1k zxuGmVqLh7uA_NmG3=@Rx?>PSZF&BtJXK5) zt}r_rulyMK%0a)?s#O0CPtVG&XwfC$&z_7Auuj8 zjB!5euVeHEZGJhpYBkz|Y{}$tv={FZe$o3t9>1H;+gEl`vABOG+a@0}qF#hIOm4Pa zeEGJv+r#in`t3>kr3g`L|4nqEV-z4V;QuHLl7cd;8NxmS>I@ zlNH!6Vi!C6x+JzzMeo`?*5}eLr=v`^$68AY?d3I^OF?VjS)ZyRv z$mXwRHj{t=wpzodi1qI^WbfZsL+I4X!imrWO`IQa=fc*j$OCD5xldic95=YEfWw2DolkV*>|o-RaYW z6Db0;4N5dSs0&yOd_3^3L3q~4{>oOSuhQWG*Fajm`36>4nu1A&AWYs82t{e?-Cet% zwY7gUtexK94kSH2+AOrZKcUOu^>^xk&@?)?wcJFY?`^W|R|5yQ%lH$&0C7m&+EeVi_jKR0kijIeN=2L1g_6)AOFm&!Q+po)aC4sOic8; z^O4r)^k|5}vE04NP4=tm%2YkIopu9R>tI67%H!By#0t!VwXOo-vHm!Y{ErUZ^VoRC zTgsUVwznb0+{H2PSNZx5W)vxMKeuq`2RBQgjm!DUR2FAGp}^^3zO+2?EWYp0JXej* z5_C`}GBF}N+h>2%jDSOV%?{7HYG^bM^6tb`BnvLI3YqB)${Js_-w5sQ*}=z#>=c)G z1)hUzQt_Ej_60jPAG4uLkVQQWUYW?F&fLL}Qb9+9Ogl++GPci4ZLvl-l`4tqUEd9l z;~-y0GT-#Wd|xCuDOjV=`$y+*nK>IvjuDPZ9DMjZ{>y0QtS$?gH=4M)9P%TFMmL+E zk-51zBCV9d`;8^YW)HD4B6A8kWWb*YBHUcmH6&DUN~gly>ucT`j+>Tcn7_FxUrSbA zDugFNOhO=YFI`#rB@Nxeg3)UKbn{)UWLZ4+cLi)RZ{7#3F+6x=^L#qBv$xl?gHp=f zod?eBSN87*8#Nw+K|ge7a9e_Skd{g}0ww}7y+6d1{gFZkRpylFut8C#{+1b>#Ti{j zGFD3brw6&vpFj>hSoKSnTw|1X1*dvuHO3k;at7jHIF{Ly>BOO#NTbue&4~w{1Kb_& zk#B#~uv^z`n3$Mu#4fUcEk4p`FgSJO1$0I5Qbj`bz%P(?*CXtfuX!B(29^gg@7U!_ z`xdXKf36}fc1HKZ#ALH@+5jKim)e^fP>pdieM<*~@j>q=VQ9#NMM&e^>Fzk!Dal14 z$r$z6*0y-b;*>}B&c~|5ar0uME>bcwvZh&^^HC)i31VWxfT;NR065SkLd7$ZSy?vMcZ^CA)u=7v^nzW9a|_W7Sj#1aQeZ+fq0JI1n_4V>J&@zrL(1;M{n583*4{XYxoWg|>(LN;-bhELq)6fG@qD zJz5p#Ovu94V{pWM9sDtZ74s>~f znjR6gZ=ZQkq-VNL)ct$po23ZR@!6*4i|F}iAV5kP((=7 z1V8feupVv_Wza45bn>(drtlE!KFMjr@ujmMs-3^Bwj%dxaK<*iMlK283}OAeU2B3L zssHDYCT&-p)GeZ`Em5Q59uB${<_&#?nO`CHwrlM|oo4rJFLcnX^1RpSe>TQ9d|>j* zO-&TMI@MVqr#smS1lB(oI~eXRbOlVr0C8p|5V1YhV)ATW-!lnomDVOB=6V)jt(b19 zEpJDT5D2qtAw}&%6phrv1+9l0#iD-}6NBwtMZL=_oNQInOd{i7ggR6lspvi+^XcJ{ zRwXq2vgnXJu})%SvSetrcZ02GJxt-WA>CZzIVsisbs>?nvjLkSIo6D?U$K<3_hhJI z>FIZzX~6USjz57YrcXbeUlMOu^KKtsp^!MWd1X)4T(ZS;jWVID=}!qQOqENo{^_8_ zpWmT&kj6xw+!S11s94?-m&xAlw5(?1?P_W{AkF2xZ`1vgtBte``y*zuATxeQW8C(d zaAR$pU$o3$ECVAAFqSi6y-5|mL6z*pIZuFQ;yPcxmY4<6MQ z7w-fzOK2CU6VM8i4Vscc`%HD$psDgV36cabj&`vB_nf}ibh^66Nkk9Jp#W494g#l^ z>gCk4l(n*G1VliGf$i`dEQ62^4hI8{moo=0wk^1DRtkG@LP`iJ~aP517G9TTKe$jZ>%XeA${TY?)DrIo9a6sK?}ix zgBE2oy~9es*S^23MZ0!-DBUPlU*z1br6YDOVlh!pbnWa&;tf&=#`bUp9zkqxhM7D} zg~6;E>VEO{2OCLPh$Su^|I&fuhmKo$?cZ2wBY1R!i{GDF5eT^;y5&b4PFXmA&v}Ej zIAn!bj91`K)xOB6$iR)){UvRHQYUhBE5PXOg}Jvh*BZ5DI2QZ@$CvSV+1^7YP0Vc@ z8dIj_bcd}gmZkob5c9%nERW+>`fc!l^lSvDofI0sDJPNCc!L@WQPWT&e4UGX8F)Z_d_sgOU z3}C=5J<&D9s)`N{08GUg0QkZG_{AL}wc* z^rIqe2Vk=xX~$+F4H=$295i)MjW)vb26q{736%f`Gv`e1H)!AFYmzKnG{MeXB3_`$ zNZwRP;7LIAMP^R;m(~phFodwEfM5Q$%#@hfxB6vxeruZsFH`noAF`6S>7~w;b&1f! zRlB^_f)w#+aA(`ceBzR%y|7t8DDaW};rIIHIoU_wH(r$pYfcztwtgA^meLq!=*M&q z{7>aep2DkqLt@{H10K)*%J}Xv`aXS(ShJ}u-!CP`TyT;pL&kL**cJ_%THd7h0>jc` zDMoH&NArJ$9Zzx53%I`#hN@#H^?~@~siwMVMKxP+^y&8)_W?~>D=iwU4XsjP(Hb&J zirOS#wbC`Z)CY4P1@^5fDk#hzVp0W5-3)G95nW0CFWGnFJ!yCwSiqISNQ9BzmG6{! zV;U3E&R|K%^Sx>g@AP8&LmF&{uGmp5xZq&C=mmJZy4cmeqIfVzMaYBq5#$`_&Rbu| z7WVhWK~(T|V(FoHyFvG-3b%OITP&L58aQ7pCRY=GgHMk{n%Z-a+YRqm-PxH8UcK&- zDOAE;aOrXuK=!7-cU^DGqOhbv(a&Y|@W+f9$Q*{oaqyMCer{!*I&PHSsMr_s>9{jo zB<@!GMItj&KFttVDY_M!vbh^Hq2y}AJ<*@b@6}su@JF>ssCE9BB5gd@5V5QsxNW~K zQJn>TC`^|$U%n|#Q_3-G^sW2}_dM&2KEC2dg{#$_EG7NN0P~5bzD~A9P>8Ja)Q9GFYSQ@rovp0dg&)mh`Z#%hi?8}@O z!-&G;ykh*A-#1ly0%fu@CtX?Y7@Pu7lDDZEhj|bTU+5oRZoghXyatMyg++lzIpl4o zdw#$fc~Is`n2oWUmi6)J=T6ZBzEO+#s!EC9Mwi;fZyMV~ubb5y=k;oGj$DBG$>@(e zZyf-H3&dDgG`ks~F&i+fO5ji4uqzjxVR}|hX15SSW{B&jz#o`gCOgx~&R%n2mgv-I zapuAKm51c`gvJXD`(A+qI;eEi9M^w`f%kd|uNDjL?X7=57mtIP%rpI_Z2vy!{^^RcGN z@km`e_=bhDlG2Del2&evCprFuacXGO#uhZ+WsgS6_pdpGO=_r3Zwhi!1?9R&)MKwz zy*ab$PpEx4+Mnw75akkWNytj?{tp-BI~Wqrv)t`KeX>8SY~LAqTRf*;SF{)bLU5+*SH;UkJXeK=vnSF8}vL#u@DLHM%72p7#}vdT7;B|6_yen4XZ-}mLH z#o(g9qwBSN^NM#X>E+dR!U1{U-*3)u9{yIz(t5@e1{T^45*@qN$j4VI1wF zZ|_G(M2>hO^n31ug9lZ3+t8ZKYg>p`5T0O``48?n%^EU&GjjTVk(&uGWpd?ioJ9!#ppeW6C%rW&$g)+oh3b&^IZ1w|(~ zvkl9~uh_XnQl@5&%V}`-a5eDu=e0V4m(Qd97y`?LC-<>ig-x6}KU1jsO;8#!(0pn6 zt+Q)sIG(CAbBOuYVI>8udqaIX{chTpi6fK>n#)MNP;{KaDpA0F-bPehp zx8tZlbhxVb<2*H%gfiiyFm-H@@s(d4!E#V`yx;FS1{%w`PFXHQU?-W)(!Zgdh* z)VkDHL`z|)7-bF0f8upz5JIq36X2IGO7O#MnArnf~Y-^AC`_}khkIc_;I~ZhmnQJiuLS#ys z`K(GT(F;lQYR}13qdST2WU~btab;{5;Ia$X<6T|qMVD%PQ+ko5yHuzcJvnd=&ID&*u2ZM|&_xc3CbM!raq%>utIg9oO*S{4FY*Gf z?jmz1zVX}-A^3iG0dzz}SSn=ip#*J%2+$CT#Dx3_o6Zv8h>`{n(HCmdvvfg!7>e@P zkFZ>O<;B%Q194UQ9v25!xha_V6h)su`+`Lok?Xh0z1krQ4BKh zJgx@Cx^>PNcz=kcMZj1y=jL&IUsR7Cn~<&r#V;Zzup7&D2<^C`Q}vFV<(uV&L-SDV zKIX6TkyGhwP8)d{%|^4I4wB#zS?1FD@l3D7wdmR7T&3!zEp#_HY{Le2-71hc9B-8D$4qjK!re;B6 z!jLbiA|WQ*vU!;4I*Rv4CpGSTo9^9~R>l`z#B-r@SougsZ03FRbv#2~QGJf1&k(w) z%L#j4G)TGA6a96*suV&0hNZpJJicQbqUdsp5VNvR1$d%NX)d2pUAQ`FuzzygK$ZSE!@eYihW27O7?}z}zgDu#5Xzt0&e^&cua^kZPIXC_V=j8ob z0wmb@X5HIg`x`f)7j5MH0fqder!x?B#pHY~Um1bSI9labPok*2GjK{u_#npbAz&pl z2LaC_(*jz&KLA(-mn3y656m)E0Gdkv#|4Y5?9uNEvBb*Ew{b{0dL}lBtQj!m;#6Mf z<%Ye`vm)$PsBNMrV4?}jaB@1Rxu=x^uoNe18}shbyIgeGz`?uI)kK4v4&Mfwst;_P zJ(88(e85|SB*M(xOai#ffabhCN^aedw6fwyWSS#yf)fbiV3p%_xX)x16pzV}@A9|; z*yM`-OtEqF%1rwdRzALq?uQ30Y6~UK7u|?Hibpl-aT!Gn^nfH4SA9~hQjj3b9Axt+ z7+l8R1ar;7pfvPxaUZ0LNPdZ1%5mZG=Fq=_=&KY#zHPVrl?#-#Cw!Mz*U6CZ*rQ}F z=jhu>n;uO+cYBJOjvplNxQ!ns_oAO4=8&-Ut^0KyI59E59Bf<=INngPo|&(YfBc;E zt>qhIqxprx$_)MZ`wZ;t7VknO?44<`;!;CC)=q_k+XTst`r{2xf#RRXT<3DvPTvWU zR}#`ilpi`34;RR*n?R~#+#5*5Yxcpu8u&36AT2?+-PBht2u4`N@Ar(s$Q3_lVwWH5 z$lUdkdzyLHTdh<@^OM)n9o#BgXF_8hO4|)TOM5ywEU|>>7X7VGq%_uD_td~7o_;Up z8)^NdwiAm&FZauR|5?oC+LuN>MN^9phTr60DvYve9~;Xlw0N|1{GQQ2Ej7$I&--Q% za?(YU(5g`c8_(-z&xo7Mft9Pwz)PN%&D)pO&KNKuilD}4@S}UR*d3cPbR~{19ZnOJ zS6G9Gf@#dm%m^!m#BreilLV$aIuJmrcDp(6fm4HIoEs2qQBeT@fC*W7(I+t71`6L- z{7D`b_E*`F%(&18-FMkv@d6gY^N9v9N~VF{Z#u7lv#<&sm3{__=*W*BeLxvb24>3Q zQYzBy{qkP^d!2}7d%tKHD*sbx?!?5$HZSgEnnaip ztc+qunG&3>-?F(JGeIEo9a%ZL2bR!XF`n>0Ht(->cK1rXsTQM51e{eX1r<`c8}-q* zbx{)FR*=Z-ul~Ir1V797aG-vI`Y~*#U+-XW$Sch3V5&giW^!@`&6nWJCR5wjkGD2& z385p-n{i4?ONc{QQ8@SeYN2hcmK7}(|9|;hRc>fxj3=7n=$WyB#kI22%M?1;_CJBpQh9%y_fQuz%sVXXGnEIdipHlPJ`^cW zrt&=UI~dKta2y$*gpd3h{)afYEmHRD-6Du!}KN=k1B zGNJ%!fbKQbz5o5=0+S+ei_0u}%#sL;ZlGC1Y-#D#2_qMw0N^8D1dP{i0?T`4xSqPC zL^!oU$`Ds@QV;gHnfl*1;g^0fQiy7#(M#m&IUWw0QZMo4IkL!y0gWD1{M z0vw5kNd$lv&=N)EQW;4Gm;#1`6r5s#ZFAel-Vml54*ZHg$$IBgR zu;AIxDfI53mwNFYcd`M2hqnvQUKpGff}&zE@f4DAZvsm`Po)*WUI{k!-VA{`g}IOi z1Hg!2?i-P7892LpyKv)x>VPH}_sYRJG9?94V>u5DsgxhbFvA6JS;PfUr+#nR!af6* z@QYt`wNh`BO36I|PR2bL{18px4E^Vo{-gi&DK;4ZF z7RTja4Kr&H2Z4j-^R~KuVC{r*f5T3gUr5}Gz2xRvu6OXPTWWjyP{`fV9o}u$uoZP1 zf=wD>Oz4l%RR*VKcU&mbLf5ex0(wKaEZ%(x^`DV{zHrd7c%k2XXPI^T&Mn)CNZgl3 zE_9clNvbuGRSw(E&e7KsOq@x3BX0-KDgS+Y;AYd!o^1dQD9%R(lshn%(iOiXNIL9> zH0&B&nWIf4%lblbvO z?TLd)U2hrHoj(>U;qy`f1od)UCx47*zW0^p%4pU`ZN*@Iy2k(kMr&L02&vxtyF{!+ zedxIP;~W|K5l?9ZiVDu#{1|oI=PE*&P|>TG+X3P;5WCWZq}25#VUqrkO|zMDe3@|1 z5DVqXi7K67x*sr+e1!*_oOS>n{DFD~-!Q`#U)<@J4S=$zd0Yzd)bFddy}oC8hO%@z zzdUN}0H>QX!=!WZb^!H@K_>hG{PjC6aSejoU&A)>a+{vS;`C@MLNSSdHcs0L?DE?H zjt@o^^Dd*^0R9W?=Ff5;ZGeqFKYev6u-SO!4*iXg{n88+a#2`JPE@NOFl82x`989) z$K_PdZ?Yc{yPn?k=pL5o3_73s|+j zH%Y}p;9yApsWuT07d0kE7mr8iC(ByI!YD;rj5xfQ|3n6lamv`og_e7-SB@-I7}2MTU0nRzg@Tk`49E^ zv-0!*ZQnf5&vYoRX=tDm8nf)T)KvLtR5F{>AX*;i#m-_vQjE%*ts5cj@6=>GX7T`;0+CQA^!H@Gs9CJkm2h=MV$&GQqO!2O_$1B<_3C$NkP_}3r`)$t@ND3w2<++AL z9620(R3hmS72@nT+gOqSeRD#`3%oCw*a5x%UCts;$~cGto%A;|lBxX;JD2lkIi!*R zv+_lvNN>POLp;;mhrq^W5IYi%P5;lT!qkmYP0?rce{Jm1y8e@vYrw|I=`+2BjR21e zYQSUsa+rxJ{%Ixdxrq9E=i|N1nkZ@&y=g>^wwSC(G(V{q@fxhzilm(~*=k7v1S5y4 zO!`-9{&s|G>mYrmNBsSraziq+bNtAUnw~wU^%!c-Hz;nVwOdNwF$g+A`eI@uW}+K- z4NTi(kA)9tFeE$~a5}9j{dessvz03N@!V*YmTueVj5-mEVz_lXahEpGk`x`q=irO;zF1F62dKO9M=6IybCwu*$Tws(WB@r zRUpJh0Xbl@brUv&pcg3wB5h23{CtYj-$6?cpnRdG1TOuC^Nzbm9^oih3t*fMRI8kp z_T3U?!Vq{{(jQe*FFYm>!KTzdj9TDzlPB8Ch7Usi1UJ1OGi*yHT=)R-u@3$*=NT_- z6cY^ve-R@|=-u{7KrzenYjkve5=^bFhkxq|KGTk< zwW>!$ayR=GY&&24X>g@K-Cy%Et_n{u5aI$piW8*9t0XJW3t|%1s@rojwHeDEZsv%uz?Ga5FOaNIPTNQTE%#txu zpE2(By`OgJoohi?q!Xo3W{8r_ta=vfu{EI(8}3wd@oBh76bP59-hpUtuBK)vLL^WT zaeOmZUg-K6m(ogxulWEO0UmX@2Dim|DfK*XSlaz?kcMoWBQmm&M8#mBUQwlud^i2&%xYWDutW zg01Z5Xf_^I{Zp&&pSod1NcSw3uI5Et^D@zGCZ=ofMxn(N&csevN|^6relnP_If<8< z6RmK%F9k6K(2l|>L5zbO`aFyFf?piIJjGJzKy5+7`)A7x|CY%H1$z8I0&yJ+AB+A= zylsiF2#OA?N;X(fLa;%#o=p8_*ks$R-|1^Y#u;Ag%<>*26 zCpyoyF zdLVv=EE!ySH`mZ6l46T>vMF5SgI*ER^V5??5WjvB$U zencAgNq>;}QhD8-2vW;&j30{mYGb*1vn+WqvdCzM+l%?f*ZK6)Fx;2Xy(=DV19*DV z))5oH_SR{12_T}x+YOX(E4_QZZ}B{~yw45C;830~SbPWb&L7(9 z6|GdSlXWB+xzWtOkIxT0r}g7~FxQ!r-9J#k7UZlj)@Ww+=;PJE-}f?dDqk$mzpV*o zupT^B4mUz0Z>yX8RQei2SEadZsqBXX`BQ^7uG2&$33+J_6$z3UQ)0KRUud~-+uh^W z(>}86wOC}$rzx#My&Kw32w%APi@zuavXlISsw_y{h=lw1T@NNxJA z-AoAfytRQ%CF=43MJ)t|ARTD=QN&(=p2oy2`Y&@=4qQ_U zqP3`OAi){WYX%Q)?Ldj1hxCinF+&@OEhH)hT`O z#Kw=~o%x*>d|o#BzYTyjzAQ5((Q}^lo34uha$xb1 zL{wDp>v$ZzVWZy;PYHwtZ}Epkv-1mUSKwB?o=g~YUcc~TyL-!K{}nbn-x>PjJ4x*t z3Jt9duHjyNvn_PB5WfE5a-{N)_4i`7jj-UW-J~~y8*PRNgqH!%Qm`GDe)X-hO(E%V z0Z%9KQwV1uXl^+B7Y5o8yuH89*z_Ovae2f%@GvBrK$>#tId|O*u;#k&~12 z?;X&rQJj!qYE;zM48UqKNThr2E^Qc?DFcFd;x9@S{%YZ;ItM=fp7EP z$26w^ofpSW_{htCc^C851F8kGZ8wXRvdn^S6kI(m^FW{St*hRus8zB2hG@NuBmMg4 zdKueYs-lW|>{#+IY=85mB-uD(?*y`{LccE|{85tsqmOzPTE`9>$6W|B&#Dmcb7i{~ zFqjkH!-G&g#0-=6dCDJPBjR;B&T;HBTPH+{BvH?%3^J1Dc*|h&khr+GC;W#n{ySlA z3>7K9j}puj1Jqg8dO*K`sPbFp+%wK}bRIYd4`0l@N8^uIo{cQ|)^q1g0Tn|F#uF82 zOc>&1Gy7F>L0*BLC84L0#1yF7R@<&am`eGEHG(bx8^+CgL+Q^J=yjfOgTBdeXh(G| zk$LRZd0N#fP@FFAO}uI^;$J;$-VqUYp`j8gBPZ|-({)@a)B2^!0Y6F{#F{>AmOk<} zW7N97p$rxj1?{0U9ZX4mUX++C4qD8^Hy?~>M)uj__#5W%#b;!71jU`+I&nplRU(2C zD-_6eB^bNZ;)GA5jafti82{^j1Us4Uhh6y`kYcV}hDaB0<9Z>{If z;`%<*RL^9)9@TF=yNz+zP8J^)BJzsufh?+~ba$0R=mHpUqD{E+)N~@AU+N@72ml)g z#}lZ%{)Du6Y1Tcf>Mz`H=BXF7fMLn(fy-ir1%C)wJrm+na6Ojk_{}u_{L=Y3HiDcy zwvp5f3lVE6r=Wm8J~6R3&P#Q0+kLy<`%P;0V#N(?QQz`rr+eKZX!P&&zpv=$e(yM8 zqMYgL@7a#syG^#9YqQyTRFuAa=bUV^rb z4>h43p$cKn`y(`8FNOo>o?v<5%$B}Ex2kUPBx;;3zAC&`R~y(d?Ko=i%*QVOB+WL5 z%WzPAU@rImSsJzYnbPBMY{Icx@!hNFn>Wub^Rd!;lf1#w6AesYuAwv!|G+#6<-xFn z@?PpM&H8^VT?aUo{rf+*Y_c6AR7gm6vR6n}g=B^7O-A^-wdLRMyG_K31V z=Kp@)-~YW_*PC!U=W(9o4%p|XAk&+Jj9b?^nc1avFz;!IIYcj+6pY8XJDUp1p#1Qqj5pZUnh z2yCX$1UO`Gbs)6Uqz#k@M%8+?kxzYHee^3y4yQ$lB{E1~%(>zry)mK-K{|atNC87C zyj)OM##^kWN0E2CHY0W`G(Ruin|R7ZWM}gD>*XRxb&`@tk-kWgDGon zlu_qKHSH}MKJLWtCoA^JrLA9$H79+g$;#Z<58>`>h(YrtQyeWtmd?9+IeeWXZQY*l zDBt)fcr?&r-)dWX%783E1VP3&Uf)=0hY?>M2?`Dg-=~P^3jqAA{CgCe;!LCv*c_T- z_sR0klN#H+;zaE2(w1|2lsfpE$fiufDJdY!=$+LqDPC-p&1QwMgSGFHXUyWD*$b8U zjV5n=O;e|5X*lx1-(!n8D1_Xm;+3BdX+PJH8^U;{&P*gvm0c@LrjV9P!DCNmd*77> zbFBK%=yT?^vZCG9F6Ic9M75mUECuR}N>P$>ISN+tCqzMBMFl;uywgy^Z*Ck2SW3jvoARoB`9o`xAuphl^pKsq*gR?&M{6-sLMf%Q#q z;?PmIHg3KPd_0QzLK>0-?IuJChSq0Os@Y5R_s6N4vEXAWN1WJiXsr|R|KQ#iyIDlS z8zLawy0M*E@=4S2{QWLD5AD8~z{n$M$<~I7u661AW;n_bky3JH4KKf%i;g^cyN5k1 zaXmm=Eb@z)^gOq}xCJKy?`x$$xsF!h9iQP2mrYyis05~?g;+a3-uqPX{g{_X_js9dC)?5f~ZK##M?YZpEq-Kv2E2h5hN?pcH*w9Pp ztb0kdGq|Z=41A^`0eNU6Dd1I%=Rth96e#z>c@7TWOp z6Cr|C{%(kAUDk5z_#A$DS(VtIBddeQWZ#IonasG`3geGY=i<9cqBXB?FPCLWv|`5R zr!uhDs#@QvVJs?G4Sssd*4ALSt24#fu8lqYvRT?^b^rXaIFZ=TOh@nBOY*8IY5vb= z0_)cHaf=L^2*Tws6`4HPW5jor_V@RHf~EPaV?{|Gp4YU?@O`9T1pA7ZXzs~@UsxG< zGye0IK??D$mq!jTVNg<08LxbMODkwad45paa^-dRPjJX|m3J|nq-Z+MId(B3*BX|2 zvDKDyM}h~93gM(!^!+f4gJhYWl785p)(4K2qzQ-?c z3jX(AOHyJFKqdir`Tj4Ks5Qqr4O^bj{5%vCN$^;j<#swL`=DUP=isN;(AV*k z?n0+HTkuqmQ`+eB4TiqFl0KGaCZY9ki zcYlr8o(N4!+C*Wx0FLab-~KH47Dbg(g`-M>ONHN9UZIt7#fLQDVFh9nqA0qndZ z2j>1PP&#KSZex?TZ4qZ?eNet7neu7Vq9~Dd1A{YC7N!Uny_*5^v`{MFd8IwlZ)O4Y zHQ9srHSeIHqD2L`BV}@!4rae2kGo?^Urk+^YMGri7NEn5Q><3_#@;d-*H~af1AaMb zAA@;m{S8W!wJhtsrOUVjxtv9-^pI#;cWF6i+31C9k0@a=42|vmYE2>R7{z;z7b@lG zuAHgg3T7Uhukr;52&3>rvORt&QOb)$*d1-dxCAZiV^>x_*7D3htw#dKu%)FS*$6g5 zK19EmcKm9}kIba!+SQ~V@27;UX81DM)t)?#jqfwB&62-Jk$8glLFyFJ?!c4*GESFI zH=kqIFTGOPUdxF+KgTWLP-jGGYipk?mvI-Gqtt(xLsC1BPQBAr6o|93v$8JkqOfpJ zMh$|l4bQ%*oF}S3`}yV9yy|IK(kt(s@+~2`3X*@fn?B%Q&D{c- z+fv(l|E+@CZO;1zVujVIv7vj_&5lbNcT$SnHmYN*b;#ePTshDh^u0peJYg^H0Oq#JgTm9)zOWNwq zMi0dte-IBWw52G{ZzbCYJ?b#Y@IGjR^^Oq-2X3c8|AqW0;(nOF+ttQLbTDNvX)|3% z4mDMhf$f3)uu~Jz0m*mC zhCXyiR3~`!Zg%di0HTYBLN*cljGz4heE?F{t1O54jN7r%4y_mf{_^^adzTnA(Jywz zcEK`=a!B5WuJ>mcTTJVv>#2T7t%o#*F0cZeHB4q!xN`OM^7*thEwzARvD{%^CKYTw zB9O-j9>pkNxhNoXkvc}?R2|k+y{z2yLLVHqw5{)C*X7~MPU`J`IWzx3%12KOx26;Q zg%L-!+RV|3R%{9Xo5ih5UM04tDrPb1G6eG-?XAb;1Xa5(bVs*>C8pc?iD1iwM! zE4J|yfhUs+^A_rTrKrfX1d(O_<0uWShZy!OM3MEB&TO|{r@$RA<7kXw63z~dY1R}stG8A%qpThS zCGU}JyvV*-6Z&mGU(}MEx(Q})8BlYY`^SEcXgf*T5dWdf$0G9J@qZIRpfSm?v$cb# zVZx>P?I~B9yiixprwVkw3w5^(?r7pNAOqW>?X(hky2j3M{tzli$ypS=#@@0^Vdx_k zbnwK?tA0a{FR!7kgKAPEOsDxt95kps&ReqAh%)4Z*QHM}%TNIRKlSJN+H!|jP4R0u z3S{xD8&g}aduaE~q#e8hId}!73x_!RFQdYs&GjDqRh?JF5hz_6*s+MPb5-=zY7bD+ z)zrFtZYKeJYpxLAt^%R_xD1HMMTZ#2@+gJtB*~ekBas`Ryez{r??Jbohsf@|)!9*7yQa zjVy&8O!)esjgmxTW#PFE~x50FM@bEBEACns^I(Ag57VdGQZ%P<)T+(4@}m z7w1j>aoVr^QHCr<9K@}Jm_=%&5s?I$4D7*VR6Q*dYw2uuF)t38@oE>aNJa#IxGMXi~{jz7G#FcrKpm z=@F!Efu`#>HQRQ{}M0pPJN$mS%p{bO&! zNBbFJ|DiBkh6e7h5S8DSTZ4@~Gh#MJ{T!4f?fW&bCr%=9WW;UINDI>ASM(Tv@dj77 zM;{YDaYd%NS<#-DiXt?$IkQ#vDE zY7mFv&H7$=SJl=&n|wO(b9A+&EAoMU)R&|gqY|b7dSe>)yE%NG`xdZ~WN|dsscwZE zi%Xd{ff_3q*XYprY~Up{t{E>c2hUK;er1&U#pC!bjX?@7RS1dXn5{#{S1|~%gpWAo zTt26L1NV)csasYf98t8lWSW^D|Jws|V=N+9KoA6AhxVn9%@_wV5cxNJ~qbfPGnfZksx)Z}>1E6t^9Q_dT?!_ z-8Mu~b68FD>Q-P^n)kK86SS_e+_)$<7NB>8q~3W1G1(iCjO~7X{TZAmbx=VaA?4HV zcGr#Csc7i*F9Y+~vHmfGB(gK$-s@{uXo(gek**!SQaNny@ndMns=5x)>|5~c^+72O zJmK?H4+G*c8Z@$L~vW252`#c@HHMNteVb>`Doa`|0+QdacD&vt|XROG^ zU`y{=^V4PuyaO?2b}i1LsR@j98ov2A9fLoH{D0Zkkux`5FrhLB^SYo!$4=Qb5bqR) z?N)qe+<0M?d3pFE=)dw#0pS;;M)&r!{~ahI{OEUy-?PX*)0}nvZWda0kx+>H2*N?#qU&z z#t6Co;7nOu5`rD$;im6^5YN4>%gRa+#A^D&qqfdlu{N;LKW_S31nb|4nFOT|-SY>p zIVgTSm4NDYDM0B_XsEk%rmB#lonf5#Ihnl4rOrCAq8z&wHeN_Ts+s7tPlecBrzac~ zIq=VBbG({a)j={=GAs0W{z__%3v=>u4yuq%)-ec;# zyZY892EY`uuAPYU02cMynfo(twKkzsH1A!msX(Ok_aEHgmj6K=v)a9)-CxKNVe0D1 z`Br)O1J50t4vmP<6qMmyt6@rFbF)$$_EARc3JeXKyJCQvF71SVoj+KPmX=%E!GqVN z#huR1nk^-w9B-j-?91%^>mH?iee|Mxg;hVsu@366{gf#L!@>D~TyDd?>|POn%p-f2qcHqD><%^Es&o!L6$Cc5jj zE#Vy9$1dJduk7>c5#bcb%NFxYttj?)NPuhRF?Z429BvxnZjfmg`ea9U?y&4ns>{?2 z@~|uS#Ln^iJV8Yb*2a5rwh~yiY09}(P!`>4e7cW%>OrraNxj7?~E@$LH`U z=9OTtD9@4f43RrhT*e(*C0*|5CsmV;j;jK#$|ne^zNK>G0}-*H`K%>B8r+=i_S^V`$m=s5JhUQL`I$uVy#vqCRCCUOa9PV>2Au z`=yp;F?O{!_aDd1_U4t(72M_Ah3>w)kEJNX(_gZwK$dpNCpTO{A=18vSaaQ^eNVQ; zq@-6rY_go3v8t=9HNd<5R`r?*Z3q!AR$@Xze11_9Y_b&=gAf~wo`gM%BZK6jwoo+J zTtjVj_0alj-mHu126r#!Pd%lGaL(n2S@KZa71=N|T`}O|En!XJKBNv0!eGsFks@r* zgyekhw_=eFQa#p}M%IFE0RgIQX!rm;4M5mSgB2qaRLBubK1asK^P$Wf$-TbGd~tHD z(y%lU7c>tsaF~+n)j7=naZ43SiH5mA4w~7=Rgw?`T?OSugDY{eC&`xy)-VYQ3lD=1 z?t?|F9IGk++ck*%pH@pEzxw*#CupBxW*UNhy=$<1HRoaBBk}?JGw&b1IfaFh2+jiK zh7~0M&K% zafipdq4DC!-M7j^p&2}y`zJ0#!jwe8Cl}|t*2mk!@>M=(xSt?kdwDJ&vN^>dV#MM{ zg_)dOmCrN0e}Px@sueu$9^HOcI~8tBdwjGQ>-*?}v8z}bHg7NcZUdU0U0$xCMx3ae zyA3$|ruNVUstgGg1$T4@OcjwHdn?8pBl$}2`Jt=h%&rT^2Xyoyl^F**PG%bDsE?yx z*V{4j4PLUKI|q|+MKys;gKe4c_Z%A^u)P1y-pwmrFQv;+`e4krM*iS~`u7wyiNt;u z3KbJ|=cnffB%~1p(PeK=hI3B>$=~#XvlMS=Ob5@9FA7vo7J~G3UO82CA^s2}<@!)m zBt|hme=a8xzpUXbuUwm>f@;YL4b39HRCFgdGxGo;Fg$Hrgzfqo1z~Y&ANYTINGslE3oBLB{&L%5A>A+qYq`tut8h}!133i6>pIy zX+I1rt{a)-mgLb_jRZEJ!2`V+5pf4=YE;gcNeE!7habY3!+IHzm@MD{Vc6zd`x|!c zRbC}WQvu)?PNF2)6NcuQ_G$p!zVlL1`b@$3o}bPE&CC2p3Pf>EqD){w;-jKrP3sPj zld;sj)#%R$nCY&E();h*xE-{U-&wp+k=gjkyr^gey;yqUNY|~q7A1xKz0^aBAnD*r z^9LqPnoL7q+s{FOB*_^5fMN~kTb7qi!%?ecb_Hlbo`w1WG|XQ)|EA5NV633j<-^S4 zkYoLq>UDa>qM9BqIdrL;Je(c!FnaL+K+9>!T}zR7HItte-O4FdmGySbts3i5h3CT| zNF|$N3>|P#*L5@9LCb0f_m>{4V_SqWr3h=L{WvL*1{*<@Z`J8aynyQsiVr?u04Y>h zD55(wbxV%Uv84DY0w3Mz3mm>IrQ#j!WOz;{Vj!i}q-_UNAW9)n`ql%#1V%Z4E6%@m zxG#ivIZhadW|gS@d;0jF^H4H%3q+&YJ)^}Ho^@yQ^z_^w(Rwi7hQ~ZzqCtubsX&>> zb(=!7oi)6kIcjBGJ9W!$S_IV!HM-1c8TLYVkV1mZXUKERr(FaTApQapQwJ6U2zfx0 zuTw5#cL81U4rrsiQzlZ?KJEa2F8y&B`kicmn|)f4M*y#JQEdKwGAI?)>=4ItHKB{WrDi`l$j(%UvXg>2hAfcU)<2d#t*HwI9zhn`zR}9urPSJBk$to#+uxVffAr6Mt~SVZ?RCo zibf!@`vB)#SX-aS=RSsHcp?=i0g@dO2O_8^-mFU)?;CAYWBd;?Di z^4NfM2SU?s1Xf0n6tu#nh=nW`D)UM@*dfN3(g$$utF73s#V~~&rH{kgo=xsMoisau zsY~Y!W!Bfz!5#<%)4%1o@Y(hqoy>A9lBWXZT{@n+pO~r{5{$6-`U^nDp|Ca{7e+W< z^ZwLuBq<4Y+tO6&F&cRN!Xjdvj`03+&+?-32b!8!H>TsfxkcPuv6hP1RS-aIEL4$R z6E32>1D8t718N_lCWF{_lb)&G2d4U--B-(W-X9i=4?NrY;+87DuVniDt>jA{pPO*? z*u&PI&rKEnNAummf$3)WAzNEpNDi4JY!*f; zJ_nBgXwx{~eDap2q@v{5D7vm`X<8v#C9qHE^8+2qlfu9jTJyENh$UhhR|ZCW56-U` z`P81aiI73`o%k^zy#nz;K&rv;4RSY!MpS55Y*ul!$JCQFrZpEmnB#kF0sf>f#^o_Y z1*R}Sy-ye0&@%G%ovUC`TB6AJz(2x)P|PU0@%VL(4!JD zWa*q$Tuv>A{qee=SR@_RZOeKu9Mf7}6g<&_2u+bDfAM*V*UNEAxw-sUSpCH;_-i!{ zxAN+lJkzM*dZ`QG;(nvLWK6o>z!Y zq)Ke^zi(N>zyL5tQ`dh%hLU!Ft?HA}t>moFRm%cqAfwK*Ac7wQ^+#SYKUF~*cDP=`M_WL6UeLWh;_6tw$it9L2GucWJXcQHJ8AJ?}s zIIWJQ!46&zN@`kTp4X`uFw;a35E{*X3pZkEWJ$ENC@KsAioh{zsSMZxnQOgSjU(8Q zvISN|N?Kxvlfbe!y=^2S_02}uxJ@MRGW80u|O);86Q0noXJ;`lzRFF zu6FiNTI|RXpsy$9I(TAv%w4f(iGr&P@{*TraPZ-f$@MfeAIXw~xZ~x;A{RS_qfYUo zP1q>R!NU{MlaVaZHf-_jMVSYnxjV2fG+?n#aCZt9Nf882A}o)})r_J5;# z4MZQN{NXS<8d6^fHZ3b_pVQ(JoLgP`pA>_rLyEF<%QmD(O-5SUpBwknj9pbt4JOL1 zP#h;I!1ztWjtzRwjoB*FEwQ%VVdT2I%9!}mD*g8&z~J@8H=V~-cVes? zXiDwgMQDYdGIQg|8dvK{%b~WDUCs;hH+@S+ECePX47?2BW2BPR&Ce6jtBkWQpP&I) zD0|@@>P3ALul6xsP2~@fix{I*Beh;JiP=>#DD4Y?+$eq0Ui^{lXS2&U$eM4e3uPj$NQh{<-505J(x63Wzq$%FV@j_z=WD20mZ-X^94Z z{r$S^RB_ppHQsOT&wpS90{kLjg0iw$A*9SPi8mgb3}#CEPV78|+27YeV`a91kiw8 zs@t#zw(Zy7^YIgkz&1ZPA&_;l?hsGsuoSOY5OARu(q_~Ol|2|CWNT;K7|i?L;XnWv za_}HZxQNI|$FIhUMyGS6U*g7g0K$+9deCx=i}$;M1`ew*{COqp-WEguA+`{s{!~%q zu-S%P?O9b-l-1R)Z=8PP)E#`X#^9s9rR@<7c!Q6Z&pyuwJ!79}@lt+%BNE*t$hagb=Tq`IKUJ?zB0gUBs4Ule(A#6gsji zvRkOjk$dWtc5IBjHat7*zzz?$cI3teG;IcD^yZNZzFxIh)zr5mBH!C!iN^dXeqiXY znj{OAQ7%ZW@Nyu89{O+DF@JxH4k6vBWrHR>U5Pbik*}(nJevc0(5x{{*kOVjVr<04 z6$B;K`#u#gVz{+J9ABcpPylXBQ2s8=$kjt#TYZqh($mHB9>cHe{;taH-3s ztcsAhC6lJMKKq>M(7}_d` ziqv?4=ZlEKLH#7Xr!lg!s|mFzuGbMBGp|snV~a35Hw*W!8|k}W)}Vk$on-+5h$hUpHx5!hf#W_=aB-FzU(&`{nxNDY;Q?B1C`k$I%x7DzNe zV+`CJgt)Tcvx<445o$5s9TWPt_S;>AYh_}$tSkympn{@8jMa6F8oNf)jiA)mNa8K@i*-AWj-n@R_f$UwcOiAbkrXfWR5V6De0plHmN`%12cgfSM&$5ZXe($ z-pxi~$pyJGJ_2cWcC#M+z%p^W{-b~h{@3kH24v(9a@C*5^OI{QBrO%^=&%MKTiphZ z5Hu^V@`*OA6Rd#>&B<#GQ{whFo8xJlJ3kPf1|-BC`rTR)Qm!fHn8i4?kIOA=@k$2F zAZCledY_);!nf_hG=S`GAtndB}~PU z>FG@0G9;N1g3Py^k*g)9?>6fHMNCv9L@2H)ir3vistOSa*7o+vZ0o4Z8ViAb`W>{X^O0y=j|SzN80C6_!(`mYThBY_5UAO(o~a$f_3_uC zv#ZB?slvN=#mkS$RGDtkTn-fvOcqM@$J*fOJee6^oEUpMXubRAwl%;~peix$KMsE* zwtLsYE5jb+wBX&!o!eqJ1AS8ksLejyJ-2QdNZPR|em*o-=88u^Oo%=czS*NX^+v?FdO+i71l#n#JNp7cFjYJ-T|c>Q6l&>Ox9inVHZ!sCO;gg2}uVn zf6f3Bae!9#8DIrdL!U1C5dj6_+V-RTdN_|&88J5YDAncrJrQW< zYb&d{eA#QKod}TtZ;;te1KbRqGJv`|+vg4LtHuB7Rw7g~hb%zZtf8Ud4=$uN7qL^A ztbTehc{U**^wGG{eZ@ycZHgktd2Dfg%bCW4ItLxG6TW|nFZ#`#>lrs@{KkJJ@~QXriuiccE!$ z=HUwe=U1HDsR?Sn$yfRaLWAYFh--!YuB4bTtU0MJb)Zrw1cGJatEvtVC)iS4dr5#ixpx#JGd~X%iz0(d)St7uQT}y+}i<1UPzbK+ooYCJi(ob@sRf=mO?^UOT5L z3$UIuZo;^=0-3!lp9gEMl@Vbz^j@EeQ+`e!c|CJm8xm+hcibs%Mpu)`X*gNsm&oxaN^_-YBusrZbq@RTI_&Bz+G%dlbrEr&HCHhm z&y&9*;!3%<0jvPY2xN}5TkvI;Or}29itS)Cdw{Q)HEKoG&jG@Zb0ci|Uxzg&JsP*R`e!e1%V!Kcw4b|pv+Rt(MIyqxgRJs~*L_LADR(;# z$Dl2$TX|=VAQoU=;v)(KwLa2G(DIvxW!rO5%DB3^8i?tD92>9=EI==h`nn*7P(zE& z%=DjqEH41Ut$;F8+J-WDf`CIAf$n;2k)Rg7=l!XMrdR`e9waZY>Vdr3`N|gS7q)~S zjJVw(Xo`jPhjHnj!(Xpdc-j_YC1Vgz8E957g&=v z%*QCUzOJn5CHp5;x8r`d68?G+_&2y^K0Y*8$^0)>-78T-kyMW=`ts7TMl>i!$2 zF-hn?bdox)?Rouj3ot{Y<}Q5yn1b>N(){?gP`W4(VIWu&NdFd~_ZEGBL?vkL@3w_2~S4K%j<-j$uMH4iX znVcrL$-|6jNF7dw)UBnjCCiJY1=lxfQm}XlIUS)zpm% z=*#{t>^h~PZ6p_-Px~R;r;7-j1u{7r{nW-UP5{V66uqqV({t-Mw7AjL4?)7lV@Ni- z!jFPI)|sq&jZIzt?|KM=`^x1A;&sppO-xBWPM=>}0sst-kg0_ZBcU#1ty;!n;%cZD zc_rMTKqpkN3TOu!(}Fs!5hFhVi#LofIq~b%p~y5SC5#7ukw++5w%x#enclrJxVUX( zQCQ{UgpNJ1yW>X3n)n%hDUeb@`(2YX$qZx_P*o63fxhAE=e#V%xjYRF#l(V2&z+P$ z+ym z#+I2db0E}7l8)`biVl&Ny`N}OjL{059!v&o>8jFXO7Vr3k{^gsjwaE5v> z7Dr4>tkdXzOC%;xA*apqV_FET<5?DF?2y7kNt4wa@2(-eA`Ii0D8zKI^D~eXChA6- zW<+-H1DUUeHi|4{QX=S(AqzQ(8H|XCK!`9Tqg|)gju8kSe+Lon=LE=HMiwsT0D^?2 zCMhv+*X}@_H0ep;bO$A=Z~0BG79xn@fezF5#y3FH8%Aj{l&i-QJ~S0 z1eqfJdMB@x?25}j4^NOSm$2RlUU}BXk9dFo{`EKulY6=9O>*OG&tYX-lw<5(%oL_0 z2!`jvmc>P$>G*nFdvbb*`%r>3@JkS=jQbh>{5>F4f5=CKC!`-uoJBf+zf|$r4%1qc zPjE+y-H+_z<`LX!i3<*K&qt&LgoJw)a%YKGFh!Uql&{_a7YH`2H2`$fpatBPnlz*OnP<h`jy^ zhz2mD8vlLwS7fR6?A*HbOMfc@T|(LUmMVJLuJ%d%B~$CX$2ed-VNBPpG@pk7H}L{l zj)pM%0)n=p@7$jofutd@&pdF*;&Ywj;x?$KuwZy5*tPOJ93^1pI6slqa6p%?r7XH1 z1(1)7Yp;Y~47pR}yC_^a2WFR&M&<9flh3~J&T9_{5Z{q}-Pg87nqA+lKz)JHeM!1- zG_$M~mD7guWr<4qLeFOeeo&3)w}6>}ns@K3aPp1Mz9>+|502$g#)97pg?Q_^lYNM= z<~gEp?}quQv!~P4Z|&k@pzi-2jVKp#nS@G3Sge2*Lo;amA_s!QuAE;VrgIRF-POE2 zEx{@8Yj_tJ>d=+rmpgOw2QbKdUGR`gXx=_Rj~F&V;)uvWEGPWb+EuC2BmQBcr)OE=m9p$aNCEcc<-x6w5>6^2*bWy1_K$WXYy5{UQ&m# zwr%WZ=Dr|mfYKeQd*$2<%NRgkBHQ@S7tj;CT)X}}Ug-mS*Q?SRlH|K^cA>+}Dn6I5aHmm~jR{)9#DWsN-bM6py*`?>57k=k;j zwPLfB%R48#;wmg#OkH7EYf<#_ndw$?^2gZXJ}42TQ52es7@y>AC7A0)p@?%j;`?fI zrU!#S`J@oAhlP5l>NUUp;Yr-jm!N#*{Qgp>d`>G=9EZ5D{ka`TIu)}4vV+N^T#JdFmxgm%DzUx`_@iBIHLBwg^AD3%{psLHe z{)bQ@b9|jF_M_>JBm!coW>sX%m3zl5f{{gC*Px)R18(H0Ct9Dt%2V!6p;wFxqX9`N zv+Jn&J=23wII=yhi*NXTyW^Q^kyp);N>Cw60t7Lj*YuGB3gYpY1B8fKk^#S4aPb6E ztbV#1Xb3Z*gk0|os%wq@5|JuvXoOr&#tCBmZqmmlL4MIWx1OcX~O4cLBPN%LX9`DXHif(o89D>;F0~nD7k`ylC2A@ z6zq8PNoEKjh!>!ia+w&5bxSlRxyPL>E58|+p7_RdthLC^ffW&vyBNU0%j6gjS-N7f zjEr-!9bmws2c>=-?m#J#0*(O|e9dmuqa0}!S>n`LSjyFt$+ztV65(7I9eH>qP0*C> ztY6QN0)7tm3Uzi~Lwo)_D((;OIrs>l!>S<84{B)yC`3z%$f{pnoE^GD=#JxFpD=`D z9O0=+p1Ev-9D3Ica?_-rKfjLbHh~&~I0NM#Kc+dDc76bB%k+W3)2TF%p1$~e`9Xqj zV3Yz|4D9YMdQwaBsPL~ab4Tl3jX&>m+zQY^>uyQ{&um<-gXjSN?Ik5qqkWF%2Q36( zbCAE3QRL#OzJdEZP872wQFu^LZ1Y1FKjlFgY%H zf2i1^TSH07S#?t&K2aj?7B4QYseoo8b?@BV&-@lPy zyrg=45|t$GHsesngUZ^eU4|Dx#U?TaK@8x3Ie%46CZkX6n}V}sL$hJO;hiT3C%0ku zFQSh}{tfuC5D1pi(-{6ilhA`nL>c6aCavYoQ*z{`$eQ{c0FImTJWy4p#}EGdg%#e@Di z+%A-%&sCGM8D<&jQY>ElvNwt3Cyzq$R(2`s?Y!sC^(F2NT9y=IOx;Mjrb(y$D53oq zX-~ilW;;Xmh@u+puOtL=L12GQ0@uo+!ncM-X$t{JeHsh}If<+-6$IT8dESNlk0lI% z>Qq{xJdG<~BeE07!P3&>215b>+e=R31VrBtz(@&X_}9M*0m9>tbL=Tu10YsN!x#e& z-)o7ofMp?f%^6DREg6XBV2ayA4T&Z8w=zeVs8?#wL8uGdaTcz6q71E45bq?Xk$|)6 zaxGF-2+!Gr6VPvZs}r`1vn+gk)`A-+Q|{7EolGD_JS~yntQ?DbSQGbSErK48P@e5+{wUM~7?EzqL^-_X)!Xyy;cl zoLB!<;|5F|1*|aqt-ET_FlOZq;D+pKH;D z_WTlNSd)~pvT6uQW;dzzqHz%-U2t4K5IMp=s-@-~F%_J8+B|-)f;s;5 z1tU=t-F@7c_-<2mb@j{%o;0<^>_{F=6fkxspj|-uL4{&KoYH_h>>!v@;iwFJkYOXJ zdQofXd_iG}jL0-|+QC-s+zO*g#&Bm6%nztc_xPOMlUwTBH1dugT~TZ6>(Ve+0bAF# zD1Gga)ocZl* z99!UP^!}fFy|3#$zAAcGfO>yAYWxn*=lJ`>8g@2cH2P{6XG*Ngc=jb;r84K-cn~11 z6%|NdB5&4a)HvlqtVVyz0o_o~vn>Nbas}H<=>}eCHRfY+68mG)G2A(<>(|J8T;`v= z!29wr1BbD&@v%TkpH7pfAUy-a#hZID>HWCEM@#gdVgx3|pCBWYGt1q#TZb<3OHQmr z1dV~abo~5Q?B`{tc?1H3YgkgJNREy1`r7?=@XZ`X21y|`DH(Dy(1`eunnynj-0A0= zTmO&xBq8q%ShR;&zELPB>5xj@0cE`aQ(`ZB`Om2lz7I@kZAd=^S{v{$ zvun}Gk4Uh{3)F%SHrwm&IoTTnV>gz1G#-p8S8kpyw6Kp{Z`;F!8x0&0+W`qe-p)H2 zETgY^x%sk7KFu)*CdFlk(X*<0d7gi`R(n`2Xv7h z?zijYM$R6C>d&NMFOxz?Py8OZIYv{=F2kZvUVII9fzf+zrj?=qO(!J3i&J~9VW1Zv z#SuZFl|{+<`%-4EG^d&emNDGB4ru6?!2co*PP>{9Vnk7A{pQZVp~3WYOCjc0G?RV7 z*8Qvpnv^2FnfYU{lE2(VM+2sjlP}DlzKWACE_?rKYC6j5rn#wS+;E(tQ0+8&cDI_GKek2&AbROZ_to}fIdTQwDb39kw-WMyPq@m)d&>>d{$ zDkqha4yoF|Gt0*UOu4Yq6at}o!IC+8;JXY8vSto}Qp)%ww%!1MP3`2z2NvTN0?>$i zWZ(r#BQ61yA1osljqrKmq5lA6K!D^TAPAz)QiV^!We_1#2b$Ih+c12X&jQAO;jkp) zZd@LP_3|(!u;K@xDt{1ZP-$WTSU?_e*%Hu38Ynn)+IS4 zck+9#$_^;|jON>eFDh}}9zLYv;(BlMn7h{mMhz7E^@`>>a+HOgT~b=rK0Wcb<>gkt zU%Emm@*I>s3@wm?qtA`|p-lY~ueJT-m$O4dSsX(Pf8EK}Zwl^tY3b^QBN#Yhs2~yF zmeQS=9*S3fdbV33%*FMj!=(IcOcWO9ZGnY^wg*3}@2|2YR4@2_>RlR8-ggzT+-5ua z?#L`rF&oe_`ZPjeIQnQK-+XwNij6BK zsf*sghcd!-6o$83F}6(%Ip3}uIsA}@N0Z#Q!H`(1Ctv(kA{SgPKf8il3)nlv(jtkV zQtQt^u+U-GuoVJr4;4hy)1o+4rVw%uu%JaifKEHoLgh6+OTT$jfh#x=5C0G~IBTnD>v}bU@Z|1% zPC4v&_;95W-zb?EKW33C ze)R_$&ya;EUJi7)fVD#U!HD)FP=aMQR?#X1VgRtgLJ4q1M4nk@>4As#6|2kS(42sX z3lPtj&+MH4K~4fmtYi@AkB%|&S8|W{AZ76#rQlFtJT3p*BK-jg`Uj94Z=$YMEKVb* zx$og>CaSqku`>_5RY_LDaFg86_Y2!PO}5PHtsCYJpVsijaUL0oVZLNEB=azp%& zgU6{wk=BQIhd68~a*^~)$Nay?{xMltTve!Z7by%iB_FE!X*_C*G@4R0n3 zt`+{9)i(_6Q>p8sxYzA(()wu~<|S=IVlW9h>xOe1eXP{opSTY;K(o(7qNs$ULLh2PONJtb82O`jmGddu} zOrA_$rnVy*Rt_hSIE+F61L`Y`xbFwowREcl@AWhJJ5c`M#w&i5eZih8R|z~@ zo`9+hrEYx-#>Ecc8Ja&)$OreVB{e${LK+a_BP=Y@Hq8cU+t(F$h`@mEITMV@F^haf z#rcx5ejGhNZaSe!lOH=Nz`@$ESwU!icOUV3P_G9D~ZNth{V%YrDIOq%^>CnJX8JfJNvRahvhS{STJ% zxPCAP+pJ+me6n@eGErrD8DeF~M9?ItS9znrjRyo^n)1_ueAuLr1Q+O1Q~oQ7%JcDx zOl`Tz2FHb4>i;}~WUhc7f)ya=ni^^VrjG8@vD66v8Y})UwGAv=46s51^rc}6SD#nG zQt&9Ws3x^2qFJSlRv@yj^fMI0=#DQnc^1lu{18k}Wp?ZOLAS4lJr@8*2|iGl@D)vV z&p}}|d*gs2B=N9yLI4Bq%h?zZyWw-P&|nj=&o?$SFqG?D9GIDzeYYpN#K(7uGHDE> zh>55pv#5v&B{lSr!*YUDEN#ExH73U^1=ShvQg&vaa z5zd)(jtuI9L2#Hr%v0b30}FQs_Ewu@&l&$)qA_20PE4Gx?5UNGp2Qo7j=IgRw!^OW z+4uMN>cFf}4u$@YgRU5+KWePhFcy$KyaDk&M4N_73jXZE0=|ErFH0*j69Xj&o(g{m zOZt>})(U@ZPMLbJ)>iVca`nj=Oa&O^t2Q4eRRF z)>fJoIVle8{SO6H^5nor$omPH&--aU5+{{oHM&V{CPc4+ovLP`r?=z=f|hxCc_~)^ zwl2odY}W2@=c6*?LGRs5F0fBd(8H<4+kjU{{7e8T&6CyE<&cSq1lq&94fSViw?B;K)SEJIKD?P|?OC`-P?fNWWp9N=VpT_t8|}iY$iDD5 zCUQ`}g650vHQ5d#4*NHICI%-SBJX!_;al-=iKe?V%BD}mWpo+fI|B8KC!l5Bfw<0j zNLy|NIOE0IoG-MmbD`Q?!621Gde8}2TDyDzy)QkoGY>R8HAf`B4w033oiMb=fX9gw zoowi_OoarlK`UL|eoG6Ejn&+yCdOx7F`d)(&Pd2o=Esk#tA?>k{~akyuoycV<{@Qv z3OTuiK6A_MpKVp{I}4!JU{(=9UkhxLf^|?~VepI;U-UP|%4ultJA0ljk;+rrc58=> z*=$^sRzIMUj~8;C;*3{atwwBhy^c-|y$qSH=8F{M{{qP;CoI{F{TR_*!=DbA^rmK8 z9vJee?#M+z{(}7CrOsxE_3Cj3rBD0;>|ab!zEKOIKa?u(Inw&#$C)=If2#*=+8L&M`>Q7a zvjFC_uU!+*UR)j^tSTQ?$mXCJAOxEm*x@Sb8$DL}D8u8sn8BH*7_Somst8eW9|~;g zp$G6~F>!DVa;9kY&wb%xxT zEY+>8U^lShv_BkD(g>0(p~N$A*^}9wnb_csglGVl&|HJ1Oqf z_pM)w$a|%G-k622EljFV-c63@~^HfLQSj6YUYT~U?hwmUBAk@-Y{(G z`rW)Y8U%)B&S>z%G2SJqKTG%eoi(hMqQu*BET+qyRNNog106A0UR~WAlsfpatXlAx z-2G@nt|)cuQ(i#I12L2M8#IaKQ`I!Fhv(cB-4}O{1Sf2SJYn~d#nJZs`{DIn8^|@# z-Z!kZOFR7mH{bc{kxz@Egha#4x#Y=(Fp>^|DaED`#lp%OvuOGxRc-ZAZSS&^sQ~>Y z)LmG*(bA2Zn4t*o)-*N6Zqru~F5%HMyJh>mG+@=k6L4{*((z$%_^!bWo(t}4F~qC} znyM#r>R%~pmuM+G(;wBWwtx2K4+sNLk3~ZMR5m8J z`(N3nSUL-+DEF@mlS-E$Asqse3ew$zNDI=bpmcW$QqrY#3I^TMp&&?i zD@cQsbj)|={=d7HYh96Hf|>K4-#L3fd!LHS%h^M}*7p4+4>e9g`Cmz9SMdo+djX|6 zav!4B-vWjBjB1yHiV5r9FZgB|yZi)15PA=cm&oawrYX?&Rn(Y=Pa-OXt(g8M{yrNW z-Y44sT-rmI9u8N@9he&wjKBN{!D$i~+k8u2;1)*UVFgP8Wn{ts!fpm+#8@a`*c)O9 zqh=Jp6Tyh>vPR;u94$9(yQt9BdUn?d{C3sTyFvA?>`sWYSZ7vh#mjk*TZj(aUia9r z`j1)>BR29?_1`=v93$lk|gaCiG;H6})ExcLG`ezPY2c-P`KH0&9EE`_VL_Pg@j_Ucoo`cWS3R z6n}%9CHKvnH~%TJyZ&vA+AMWNbn)U9YO<%*ZHPIuPOcsqz@atke8ajI`|Koh(Z?ij z(3Wq#`R`W5t%6rQ=`^VtKBq3;C08=Nd-vrLEPq)yR`3(q`&dbLgc;;xOGM&Y$`@6F zrvggcyL3kqKgZ@6EQ$r948gJFG1R<52)llD=O>acHP8c+6E(1Puv%4J?P>shlM?R+ z&W+ar?`V(?O?Wx~h@Z^txQ|%#S0fiKKW{_%-5G?C$S5P>3h zYCf1$(gYo{<^&+FX$Hs{wfQ=;v7la#g1i;gJT^_nc>4U|Vj_O%V%`3+$-}{0zp8>m zZR#%L%Uvh75G63)_9+s?nlO<1pffmM95@8t`oVuyUuwj#a1wF8HS1$(CYqv@AW4x6 zZsP^1*6@~(vrk~5XH_Zutsd&>CVp?a3FD)#g)zG#y{5%gJ;q1Fdcj~2U>WTt5Aut_ zN}q+$r{d;DlXX)0ydxU%Bqeqe>$?dy2bdjQhru#pF)W&1hmV=DHxh11x6;7pn*<|% zqSm9Y<5j9EiCGCjY{!q_aPSI4kz`KqQ-QJpCv~LNdbYsH(9+F|tV2BB)%a#4qQWx) zoY!peJpq)E&}+9|evg`2EQoi-nHoI6--N_?hBl9AyevS3roT8GFh@N&3fWv<1gw4# zprK#&(IBnLG7>Ep%SlX4;6+q zhSeZ^S+s2Q7j2RewWsA1SWth>lr2+W3&?dhsI5y5)_#$aNneracSm2Ex&JO{a( zG_FLH(F!2AhdQ+ZJQn)8*^>1WSfvv8RYKA8ClBHz#TkY(_7m_KL(KvXxP8>s-Byu)=`woOFqUIRbH-v(Vm^4~JmMZ#12rw|NG@Z9UDa`b)jmflyY?28 z&85j}pTm4c|BXk?t`bep_|)c#8i`f}TsTP@xUkL{v9}yp%$ALMR}DkzW78h}v|S}# z_f0&tN;lz=TWXaI>pJyrSb;zxW_UIz*5C+9>RVYK457yw8y=Q^eUtDm3KWX>{m;G6 zZ3+{^_O`2-IrZf_USvPQ&r|A%;!#Gsdj8~nM*>Siew&nB{OQJMMB^$|VO8LyN<-B^ zUVk&zw^HujtHW%sHH{CV0eq=FWA@Y;F5xDsHZ=|nOwlYM3e|z+`{8yW3~=_+U|_V z9qkP4QL$05gQeKHWnxRI)b7a8#sBLXQ42mC^^@Fw((dZ9AoTZ-{&|l#NB{K(ukxRQ z#T7wOZdNsC{N`bQ;{CjXuXlR&Hm=Eh)af%3#=m|%R2=p(noLyc-I_rX^LF<^JhBcW z&`Z_s(!9J%7ht!&Yx1C(G<3nIart>apyygRCLh+|=O7Cv+o{ld3*wt~kcM2zZX-b; zE?@=$mo^NH8B?s-srm_TTsZtslr2gP+z2KV=YNvK!#(j#Hpo$q)#uDq?R)ax&Bg=% zdz+ujG+({qAb!NScJBUaK_K(#i!7(pY5Ss5e0CmN6`OxkV_ z+}Z^LgWI$r&i>@@V-3!D?Qq0`R5(x}3xV$1~Y0 zL9CIl>$y_aeS-K<@#LSshO+rvDFf}db(Txd*W)ZP#RMfzX=pA=IVzlR2 zGG%-ZCG@G+HFa~IiwU!6{68uii_~bbYxRY590haIi0={0o-4`5i*iH9=SX6}ML|-r;oRZ$N$_ z40MHae0g&(O0=gA>q&K6PM!x6yPb5AJp(-HFY1Woo zpqOl4zrtlFVIO1T-iS)qqPv0dSj*n@i!LK4%m{+8yv*$TuR{bE=$-5k`o!Y2OQ2`H zev9U})BFT=>4R$T{nY`M-_e3e18Bigb2_^5^%V%;$S17(d@o24>-lFfn4fiTd5o$u z&)8l?M_{ZB2#0lze@^z({X!4umvDh}!t3^%k4~3x z5TI?)O>J)c7*zVoMJyk$udlxjK%1XF$C;qAh7*8iSQ8;qU~>dp@;DIlJr1LBr!ew6 z8gKQyxJF}y>E}IB6mY-7K~};z;L5F`b{XcI;Dkn+%k5rl)gl))r=Tcuf@1?CHGu^* zl1R_+@gl+jbo;#3CDOx-5akFCbt{(M^B%}Q1H?Bk*~w)6+q((ii(=#E3>-F%n-#yN zPZRj8#uLdX;Hoz~p^f|6STM%{ORPITw^q4Sc42uhr>PpF!V^b_Ow;HHoR5Yvz*Pn3 zC#i+Uj3I0NsJ z{_6$Y4eR7!g-8>K)H)0oOdOJyF##eA3+HwS%4}#soC>;6lOBy;Y;UbQwHa#Mn}w|e zu&h9Wuc51jonrQjP+ncvC_;%d=7HAz$Qx6f;f!#)gK1*g<|i|cGj+(=aTWMnj*rqY zEF>fy2)M_>57#btqFDg~3u;^MHR(@2CIBmJsI#Mo_7Q-+pMH*b{dn#Y&N^!PkW&h7 z5#R?@0MKP+UYH`s|w&VtERHrQA7bbKFkd%<}?Y>z!D(^ zqgSA3QN6skB7Afj%*Jyc=-liRy0CR(3uoix?^r!Bb98Ao_f7A+nPLUcfjv_(rW?6i zA+%RmoZeh_$|Q#zcLZdJ{G)VD|zTSE$?r{5NKC{muu- zQGwUIekLuvbkd`ZZ!4;rX9KLZ~N5A-QNtym(rP>2~w)N-1}LXe2ZH}V>})wB?p z8}yq=!F*8s`%=)&zera$M*r=BqTXz}6e|!3BRsl|K@0>G!>5CSoxDiRI3$(SK5+VM z+m{xlRrEw2lqoPeL!^Ia2R{%@_DkMn!>pycy83d^E-Er|Xky}nf9f^8UJB+Tf<1c1 zZ{pu#+CV}Pg5){Dhe1tjAEV{!*x=0gv{gDr7lr#qKetCPyKTyB=y+;Si6|xS&d6fz z0a{#K$n(jt8+dK65vy3%H;k}9m}g(RGhq7vxNn+Xg!X;r`dFJVL?<%B53x)5O@!z7)C=xgN?PKrRuC@wOr^WNNCH?(mYvN4GZ;^$eC8lkE%Wg0z z!zFk6P)FXxYN~$)aWx zZU@~&)hQnUnJ zh|g}Q`*%F4#Q?%^OOg(YSZSAuXQprPHs~xiG~1q|i<+VF?iruOE4E!6Hetzga-t|#g05ZP_bcj>cuI$G%uXz4A zH)K22UWNzBu7f%olX1maC|fCqT~(^%P!fksi8 z4u*?7$m0tV+2Cvd+HG3zb%w`j{MdPH*Y+342n*OG2E%4cXMg(wmhBep!I-Bo9d@sO zL4h1neutyI6X?7xfbv{_bvzref4oU13$w?tC2-wfbUO+C4!QNG7Vd@+bQALd(=XRv zfTpV(;5+vaL|}!Zv9`BqZ!K%pcXFY4+l7LXV(e5$7{G?Z3-rdv&Ib1~LqSse$SGci z*>KTD)*xT=Ex9}~Qj4Oad#~%lkw3=sL8!6|rE4?Kudh4ypqs;blMltLomYw*MEQ|{ zW`Phu7HS7F*TQe|IK$*9U$}X2aPcaMxFZ>Dx#@ncuqdrnvTdgg));(ELb^9*5zVVs<`=+ucbXnKPyxBKM%gk-`z6y4?4)zu*HDV9t0< zTPLTBxxZDAD0n!h&qW9|&{g0j=oI0Oto4u-+L^S?p=HN>0G%6KT&rW476?-cJrU?; z$o!GlrH$kyq`Z0gO*Ucl4%k6*Kj7dMEN@hcz+NGW^V(y24um*X-BI8}1p}){6s32j zb3bZAXL#q1qKY3rywVWIz|g%99xK2kxHWHmC&$aL_<#dcO~{)dD`YwF00%SD7GgS- zUn!&CO5NSLl1Qy$sYqHx4!T6ZN`m!5$?5?h3}%+lO?XMGcT=+PJh_ZaRHtaWFNHLe?)NQ6gOzYkgB-K;mW4vqLQ_vx>ambX&O z2Yx7=Bo`K*7hg*M6GlOqVxU3cZoPCSB_~I|Ro{uYyfv z?TTZA(Uj-Sw$OHhi@LeHrOl{GPEw5*S}3zVJl*o`JffJ7f3Wc$72W%Od*bTOV`-)- zGeN>RrjVo2Hf>szS~JEj{LyxbmqAHn+u<9^!+uu*Qt@O%+xvOWb5=!xc3FAlx-*OY zL)C^@-+iSJ3x*Cd+x{eJev<0PM*cV6Rf@L@WR5RVm=K!%_&r2hU5PkF2~U5H^26QQ z-Omx%Jp=c{CJ~aGQV*Fk`@=qEd5^cg>*ykIXI}#qfeslY3oH64$R-98aNCCK(x2!% zX6u`p|HvdSiGEI1n5L!C`(JRqi)Ui+KV#{7_^K_-_ksF!KsE=r9!lkO8?`6>JMrUn z2gE-u-iuwreZ)0k8c3ut&O8lCCd(RbuyJT@Yk96x^18X164L&1t$2>#m1>f5PUz@P z0+JjOaIYC}*7*t-H`8=E4P}4|ZI2dGxAra%>C4EsQNkR4pRKi%l1lM8nIo>zz)c+>8+mKm1c z{Z`t-OE?^<72QCHnFQPgO;562bj4&)ag*WMV=t*Ivh{G^h^tpT-|Mni_nRGJF5ky+ zb=k;f-$3=OS|)LolSq~17)x{N&FPN&)zpK7?fbkH7nb{htThBH-=D8fq4{okDgSkV z%G)JPnEQA!+$()RUTiI1a<=3^n|j*!j`1r=x`D1(lmef?fa}9-WIaT3hnA7a6#Y+f zkR2@#QV##L)Gz{Ypz8LG;QIsI5t8MDNaYF;H}T!aM51Pp5tMIGpF|+R>#zk11Mt8f zBNrWLvF%$B*FLs0)z}a_(uRzA@95}2G!H-D)^iYt7W{eymYRle#PdAgD9YXPW9Rqy zYlw8W73asy3{2PsAg9IPLLo<5+rSuCH7FLnl*gFiSNn%78)8{vx-?8mNNX+jnPG>V zC*Hje-~xcQ$RsyNCe(wOVfE`v>zXT`y>KjnBJ#p-p_S{nO7Zi+5m^PnFu8Jk?r&=} z@Y8~0-Z&^Vh&~*|njm)?oBXG^iY~=Qg5JqnsWby_2mBjI_7^UaKmyXsuBAty8(=$! z13VCR`5jVfOf6FJfsbA3A_K?(q1*pa)U!wLNI{G38-zvmlCx537RW9DQXd|oyJC(E z0b!>+rox!^BX_LeKq;lj+oLZZLW7UOg9S(eq<#m3xG-JbTy)N#2#X8926|Scp8-VZ zwKHFc8oZ`f~{!h^!Xk1?n988|B)Y=nU~--9nyP&+Q$o2b^Gij zGY0BtsGugC$EUKU6f4(AnRW)m0j0>FgW*D3B;CUp)h7BG4J|T4qHL0ZG@{*{`}O>a zd}g7h1S{CsHXai&%VZ6<2p`;#L7o9t@c%u3x87Q^e?tZe8UKVb@op16GLNF>IRra-HWdX0cZe4x40~&Qbf)id%Z@V0?!-% zhXUVwx`7F4*1>JF9?Exb+joTn3%G?_4WJs|Xr_gHKe)I0CgAg)4ED8otyeZei~k0$ zteJ%YkaPrXF%HhZ;s?phn(!ZT>lz(yff~z+}$zCebxPn)n)88tOpoVm9`N zP1ADxB98B5nGPtFg73ci-!#SX#m+r6V?QvN-OXXUbR?i(Z6j!?ZFQ);by=C1EAH4} z9$>!KN#vI;#rMHP8x7?*JctTvoZ4Iyy_WN(N3f)7``1YO?UtH2VQOEp2d%#v7~7eU z;X%j@oC+W+Kx?1#IUhZRlCO={onlJ-BN4egyq6w zWN2+K^w9VVp&yVy!1#MtE$c1FJkb#6ldZ^vUWC~pEc+@gVCoPAgL6z6+BW1}A(4ql zi~?l5sh+Iudpd#~{uelBq+DDAzL%=Ag#bJEH!#uLK^xMbdh!u)o@;BcOp*iPEn|Ee zu-E;^DgJ*0)H!JqfXuf5O`WWLrGe?Ah9_}z!;}NEgl5CBSAUN z4b;f^X4~!Ryuu2F`?}yP0k7HXw9P-Y64Hy>hV)T^fy_!zlah1FrP$w|2NRc5-F}Fk z7r}Bf80DU_Dj0*~Pm%|o6XSKOn;5Yu{siM!pmdf}hSCRsHXbtWXBtZ?C5_}%HS(X~41 z(Zl4*d8igF^2$XwQ5-MOK%Imm3TI>hK!Tnd>V9}07Br2Uz0ME<^G?vk5sb57P*HUo zr&qF>zV(9?aEzoj|1dQ|`Zu8dTNiJUAGuJ++K1h=>%Uri2o9OVesGhB@Ra!RQp*aY z%eu%SsJ*Tjw3FULk2lQwYc3>zLh%Npqdol>LV$p`Oo3=YBK-)2>PdrP)k16?cT-OvQJ?A zh3nm?VC6?NVTh}sh0v7%!q;5$G^AG= zP*>>f)39+k(L}v(Q&{Z!;b0Rz&ARzr54U!QQ9dBDENtqGh~FX`)q7ro{Qy10txk$> z)0j7utC_|Ni{nGEGwQ}E|A+%w;TIx*Ar)r<3iVG?t}Qo)_wQBvwBVFCf82W%QhzOJ ziucge;39LLZ71a8#mlCHZ9k!t5gVc9w|tIpa<#uIu(JSA3K-hNE*BA3*%|md@29I6 zjQxfAbq!IU2Wo}aTVls41x~v_bSsJtzv-t>^gq)ukM}y;@p^tS`JWYP-!L7$EAQ=J zIW6^@_Z@?*R0IZSlcY%77u`~m76RI$Z1%)L!f4T)g@0(|)Nx{Bq7l)0W6ju%l72i2 zoiu)3@60mQ$Bq8aJABcXX}ZDe^&cX2F?tKqZs~V?H@90Zf-Cge{ypa+g3?^BpZi{n zl5NB7x@V;X?6FQCG`>*J(C}Zuma9D9rn7S-d zkU^3f^nBgKiqt{yK0twLBM#CB9?|!+*eO=+PIJcsvR15lJR&u@(&zJfWbeXun-jF4 zFYX{#9WC#1lI(rkPoX1>id$00(0+!Z)N$1wE!dW*A623A;@7QNnp5_@+Zao24@7|D zP8NZ?poI-Y23{=sorI2VbO8EZzw^C9n4hAc%|eBL-HVMD8W-is{W@FH3Y^`3En!)-)BzP_+qF_g94Sdk~+*) z1OmdMMp~5cep};}M7ve~VxL}906ME-W7e`4P)(}&C!_?WD3bH);N-hhyrY^;p!6Qi znVOZ(gs#Pu!m!M7F|2b()++*=nx|(4U+kI}MiYCOOc0b~nu7sB;XdCMl_4<^_=7-d z0r!vVS5%%~^1=CaEq*GpCEIum$ufI+4W;^7qqa}d20wh=Y{EK#upv{SNNFy7XUisN z_B5nX^Y)WuE>krp!-lmdB?pn@F*hG)}4B+;9-TqFf{WCjL3O# z$HAwtf*XjMnEGh}f)~D4;=yGC+?w*uv({n}QbIF#Hh6YsVFI7`lNom%^4Q>N5{(|y z3C66?R)-Vni}$@@;tl#(>WFr2L!_Ap5-dT~SgKDlCTuLlMx_XrDxjp1{HXRFfDV0r zB}}?3vC599pb{72jnHlTI(BdYM=T!B+}_e=E%hyMgaCce$Vnq82{re_y+Lfo?tQpI zS$!}+Yf2#{|KWnCG{6Xtq3{DEDEqZ_lvYF-29gi<8^0bi+Xuh?_C6nJZzhxaMTyN)jc zgGI_a9d^hOnPoDrO=#}x_@<8=mHf^yPRov8!$=kesoXV%c0?14Xm7^hK)@Z!^#_5M zcOuK1!qlF0BA;=qtCY1Z?F2oDc`fjhEd865)hNVqD*d@6x~Q*yD}I-(UH~rJ@(oev z8@#_t!yni~8I=+BcM34bqv_b>Xsb0Eb3n%?5||_SgEkc7LphXuG!S}csrN94QUd?+ zhW(X=TniLsUjF4(ls7Gle#B7X!}pI&!xk&w<7>4EWJROLHdDxj0J+s1MtbG5$t>jl zJP<}>nBjW4j}Bkk4`6hs@k zX{;#0c5Ya&-#3RAREir*2`oc~{$2sqKFnzQRfBDRTP3^B4s(S!m{Cw9?Wm@-)O5B9 zsc(Xcs11)6?|4YoDu}j%qN|(sHFUCVV@IO-kx}HYQ@CEQULLk<>quXozoJja6r(LK z%ADX?JvljTu^ydkVv zfvZVcKs8yav?2;PMNL!TMt$5I!~1GN%JWFadbdgInLqvQ*FYy6sYfA@ymUm;U3?`n zfT{t>4XZq2>9aSZ70>~J@!!VHj}m2OW(EbNyKX*=4*hViX6rsymvk9>2ew^R`Q1|i zFheQU&c%Ei2pcB5!a>Qgh!(OL%GX=UioSY@z)ozrfmO)8rynEY8MQrk&X0TUF+4(B%$2@_H$9Ux$+={l6?lb z-UGkY!js!q4GnPb#UNEPHfdE(cLaV#s{T8i76o-bTCi z>eZuYR;}t>rq<@ZRggb%Ai&r(ZD{O#5u7w&dC{#}XyaFuY#w2`K=Ep(kv(zz+Myt-~zQ?tC2xEoN^t?m^7iokB8+9Rfx~Rijkd;E30;yVo z5{N$={YX`q*{P#B&LbG(J9qL`=g-_-XXi0cC!_+>ff1}^Izumvt zsWCHT#$sP(vDLY8r1c<~A(lsJzrfWV=CNS&w7S5Mrcc*OtFn#Bk4B#erw<=nu+#PB z2@e;$riX~KGl!Tp8^17}>#Puq{icVDiuW=ZK-vY}vO^5KEI`~Ev)eVXm^O?N5foK} zh>P)!_+zqYE#|u=cd`c+kqItF;H!qsuo|eCtf6ceWZe1B{Tl(MY5kg)WHvSWKy*g< zyxw^%kX{H2uQYy}gadR2mCH|kkKUFr{T&^aYgkwuGJJQs%bGZJ#v5_X^XIR*>HYma zgmlLn(Ig9B7YytP^7C0|vn5?)rH-^MB|I*jZOzB-w-C$Un7>@aR*GMuV^6$@K&&b) zr9Pp*U;XxUTz7>gisC%xboU8U`}t*{LQu#+Sop;z3ggUnT}%}5$%+K?hl^b+_GN~@ zTb+-rvkNl_W}~`zBs06F&QZHb63%B<*>;pf!!OJoNG!IJc48*IcPOL;q~-(_iCd5F zHf+EugBkK38e?s9k>QQILUj)zlmO&cZXO<5ATaQQNPsS*0C5erf^CW@V#H6rgY2bP z$%Lt4Y0xbZBM$l^OQA3_`(6w0$4WlwBCHSVu`H<7jyy$Nv@Q%VItk86E!8dFJLx{` zYW+tN(U%g>yXVE3$ndvzR_Sz4c12=#HX>U%G2~lRM7WY(wbFT=BLhqg*J$eYzL7I? zKa;r}RXeFGF;?x^DocbwnJXIM82H(eOWD4fUYs9E_v;p)Tt@NS+u8FBZ(XA$xy`ld zV9=UX|G8|SDhMO!^gUnlaReh*{TBC2b}p8f|DyrVPUfov~X6%5(-rTEVzOpnB5a{OtSTiIbUk z{xS)uL=AiQM)-b#3hHA)f!zM%8VEiZ27}ro@V(^$OW5H88|f9bc_&~;g^ZVA#=6># zy55fs7y==&|KSuEU>V3rNnx1Ldtv{1Tp6@>5}^NGY$8OOLt`CU&fcI50ci0$y?f?< z2Q6}xH9|y;%r-`rR_9q=6TeU(76MPrS5*Ia%?iu`NJd;aXu&$3CpN7^Bm2@9=3PRN z)9$#jqMPeqm9z=W>|A)uM%OrRC;#W8>YPa<^~02gL-Ds?{gpTCecKT)cAGj2KD0<) z-hB^`+sK=|;L$B&A6*PGGEyH|jf$X1e03tS3Nv(Kn|uAim=)@U9w~*kyUjmS-am2< z2(a|%<;FRe)oAs~W*r=uo14q<;cHC)A-v|I{xDOnj&n0_1gD7<_qs*mR6x0LOD4|5 zP(fRdO&ax@)?!;SNjmK5BiiY|0*$ZppPf}di99TJ$TH9i-W&>#ifl9bp*!kUc@Slt zNWGh+bCa#4SWbrVUI$AUVcFDypv@dB1ODuqb8pIy&vEw?3=abIr+#hw6HoPYkt%f>8anpS{{Y z+J{>$urn;md09O?5)du9CEnfB%fZaa9R8~fWsS+ z2*M&u?z_wzj+9RU23OkNK=v$N$H~HZM+P$)eRM_)#;PRpQ52%V*NW|YAwc# z`X*!UTRqy65o~Ur*D=JOITvg$Qtzh#51rvhLiet`jw+Yik$`!AN9N=`zM{DLbu3@M zri$rpQqtt%k9N+u?_U~`ziDc%6U=Rw?w=?Q8i{-fFQv}%nk<&IzBd6oc2;(3!hy1R zK4ZBk^%M<(OfXuxehD50Z=iugV>BdMJuv-+lI3^&jN3hN^x^&c_s{A`C@CM=*le_Z zz-R-TkE)dw3#jQGAL5YN4GJYaNgNU|4qw?eY3IOjy)$#&9iPXeaG)v|DK@)qlKnbs zpCG~Hfpa(Oo$jHbk-JQx|Ky&u{IDL~9Clv}Bb@=Q7enZgOQzOm#YF^$bxluZw#R!- z7ozw^lCq1*@dx1pm8i(`s;MB&zXuJ4y@Y5A?`NvJ&L4vCPYR+(@MqpykI zAtmNJ99G-F4kKKmLO!6PBHOS^h2_GbTky#m#23z6CEtB}`-XMP6b?=l5r0KV>ke$p zJ8y*azhPsSroE3gr=rq)!QESKW^Ljh;oFW$k1;drz?AJ%s`wr?L<(`BQZP|6+^BZQ|JCv2!;+!<($9EfH4v&1$i@~p<^;#l8qO1GP6{^#p z@c7?|UJkAa<8<*mHov4&{bX{=^xD0`2(@;1`WKpg8a6+D2GS~Fj0dHfuxcdhr%zet zDvz%@rgq%pOx=d%_|0z8&lQp7i*j`+Vp|RbnN7LnE4ZKZeBPjP5;n0CYtK2oz&eh! zP#w^A&1F=)UO|ski6)c z>kxde=3>@c^<^;N^~**MzR5w7-o07}^Z2XN%O0cRuhCz=aK1Rpz6*Jo-(hXm1Av8y zvN9g<;a~WW4|R3@G3k^g6z_s3aRwzi!wwmajzklK9xlLp4k)rvbP155Jgxqv7xp?j z!jP;}4};kL7YldqB3`e5MdoVCp8TCZLncTf(63u%u^z*J*XEF2!3GwwwYO7Gb#y{P zLc)Pjm95`-jrsFEpW^BHdF0QZk6)yJ`m}VwQA*0r&OY##Oh@Nij9-`-qkbsS(y!q! zt7m#LSPaL7g+7)OxD0Yu$vWF2j$KTZEtOvtELTaxFY`2GTRtjQ>Q`ek$az0)DW4Ya z9(Q9nrt2jd?|<-5E7meXc0-tU`b-8Z_hs5tdJ6Y;e`Q`D$LGxH0o6Z$$ZN`FNBJud z>TjSH9`t;fa5LH#O&aca9`-3>{-dJUMCbFmsDBy06)p&~*dtJel05U0=bCsn&L5wW zwexeX?6U*W`-adZ$LID@hsjC0k?3K3@^+ao-uM}7d}OmvkmLrX!eP&PJhmwUAQ*@MGh#GLBiYrd|jEe+Ld z&wWcBJGj(R?W&<=;PJT8`5;Qp$}O6$?8~wBB)^g6>g~v%hH7`YMwb%o)cxI3=GFm>n{pk}e0tSbr zTOdSKMloM(@f&b=8tEYk`{o6gsbHQEi=AgO>s;d=W6jHieJZ81YaWDB##35^1|d z{VCbtTg2@&N;+B|w7K|&5P9#v{C(OGfGau{O^)k!@$UAa(-#`X)&4BT^P??R5)u;R z2ghJ4;6X%mG#OxE$y-`lmfCKqVcG33-S(p%l;Dv2xJ4`Dw?@Oz+ZzIav=D2@`7N8) zOUCM&SE0w9gfg=YqZW8}zh!cZ3bFmJymL!T9I8w77^;O`^fS~9n)TQ32w-*d4Pu?# zkvqzd4H0qqpqO4W1K8}C(jz9fseNlO&tvrCq@FQGd)a687p zo;YvXgODNQ{@;;8TD#5dX_?Fi@>uDr$jo$ea1fYx$Iu`mw|~Pp14jx{aH(6XB7R`2 z>*%nM$`ebzXyyLkuyywf!Qk!~i<@GX%IsIH^wo2Td|Xvu=RL4{`bZ@ST1M2JWNbP8 z{PeR(F3YX1YImg5LjKXb*>ZDf1mTGZgTU(G?M7t@zGFH{j^54K$ZY55;g{c+LuRM_ zf4=8lrLNb1OZ8CM)5j<3tH|K)Q!p$ONlQ!T<>W{e6cmJq;*w_-j3Y0}WffL^w{zJd zi@&&vyTDSgsH@zp^ZMVr7nyD;^-GZ)`fL(0U@*p`;$MHK9CsJ0nIB9>x#Kf3IA6Xp zxOa9<`4|)B)~&<hf%; zBJ;|fzd7IL)RUU6wG##gfs%Nd>dAne_~!5DYvi|Rh!qYm>#P4!>$80KvYZf{+WtdI z>Y}eW_;YiqQrXwfPwZv=+U=BITgy8?eeLrKnn+Ac9Gd30?9nKNF}vhKZ~aNhg|ZTy znkr-H8p0<^_xABoHPdvpLvmoM!Bbax9sKbLk%VQ73_IwUv{fC<6phW;bK8Tu6heJ0 zpVIaAA{4XeA|YDBehBKy>JqHEc)dn)@VUKxou zgcepi?0gk?SaUR{w@HN5%&t&>Bo90A$j^F_ z=lSBMj4tM~cN!*G2?PLg&sf|0QI{&AxS`WZ;MD#i&Riy-`KvOND3^ z`h-UoERmgaJ%-yiv{NkV&^_YAP5b^b#(4~XEhrzS5LAO^Q7!T+>A@>+S?IltZ|2csiMwTRA{s_JTL1B}#mIXo3(kH#b9I)IwtOc$m#@_r(~ET3lLMWL=#g(Bb&J zuSXW^BN38-LxHKW8CXaSmN$;? zTOaYOR1+zUw_kizWhJg$Cq=fWg2q1JPuYg2QT>sAJrrixvvz33XJ^;D`y@n4cKp#7gRtU{(o;J<_V?t^m%fE}hH3n58nsfC zMc+(fSK8m#`=E**g{h$zem>{IhLkzC_=JyX|NFw3S{d{yulek*rXh82v|w zfc9@g=Gn)|3dSo`?PgT$_gkVrSz6r%Z*sI)Jzg8sbG61pW8y<{W$viV!tHDI72$C__z3N~I_>f;EBpzGV;5!QtT} z6_tF@iEl5q(mhI`;I}4v_Uu{4eMjCapUm&4VRYZ0^D$4T^nKo|WaFZKyD@#%B_Hb3 z(>S|B@W2IkV@hCp*~}Y0_HfAWlKaZ6{JEqCYK>A%>~a13nu&3Xl9CfFQWR(n8QT^o z*(nLtHH=<=`q-tQKt%D&Iqv1xf!}WpNtW~X_T7cO;I`UN4Ia6>{1o8t{agfR;wcSn z%^1!;jGH%p?K}JxGFrK*o&2RqhOtvtEaTtT z(nJ&ONX|ayE(Nb7dZ^&^t;I>TKla(8U)fd7{@~#()Jb@CrOfe(NJ`tY_;3nZ+oopD zE(Px!fq@KzJQztyy2Q6A-?U~EPJH!N?!8B)!bu@$_UV&Z$!As}wz#=q66As2Jt;}Y zyrd-GNExkimf8h3(k78)x4Puhhm=og`Uy+<@hlN9U(wXK;dA+ux3RgH>PFwx9VX7D zy?Ee^BZWmWKT!?y6YunVdF%){KIAcV^suwt9h!`i@bj*6nY2INk!+iFX#RGpNQ=y; zMq(bgYkJJyYKQU|JC&Z}xqx|`XI z!fH>U}buIQG{ftqse4&nDW8;hnE@}%DY(D^S#qy2HY_|jZS5Rn+E&CZ@$-izVaF2&df%h@lyS6m^ZT=+ z#XNbZcH+!^pF#|#&1@~#om5UUQ^Ykg(j{Ph=6W7O;Kq}8s3<_tgdwPs64uql$%{)C z72@RhZFZyY6M0?(62_H0b?!z%Asv(NZ%u1`#d~&;_hL*b0nO)7b%MFn4k2OkA+|`m z1nt+`cLt8@ZkPI}sFjwv9{-}5F^^y(kh+&}RbP8`p-1!0y5swx7vK6Vzgs&pgU?N- zahRC}$XeFKT0=ri?0iUhmM^BW#C^n8=^v4RPlRt!{DfNE=h$+0q1hKwa-eVherW`6 z>wT#NW(mZli^S!g?D6g*p_HQ|FLDF`A!NIBR9iq0T>|^`DiBc)9QPpF*cn(4m3|lQ zOUugvFEF4;BdB0NkIP*jh-5afsH6-*ZBG=ar9?3`H$RAd4km~>P!kV?-EQ>Q{ZL%o zQ+4~(52uKjnCF0_hPcHTz?UL_WZu)_M5A1;=LcUOSU|PU(Vz#v{y< z#3iu-T|G?#CAmgSBE45z>>OoxIhm)Ju%xkiezG%HzPD(E@04~D{;|)k*!t-u^}x@3 zi&^H_Fht>s&e+84-r?as7H7{7p1b_aJxnTwa`sI^S|WDNu#D!w`eF=<16svk`0vf9O-}Cw-XWQ ztf(t+)6CYdntKSrgp`Ga;rqs0dq%#xIGD!9w#C4Cp7{}@z`c&nePignV^a&|iF%;g z>$7x;pi{aUu`)FV9+%uQD3;eAAJf#(kxl>piX-rChOf3N3v}i-*&l2QUaZ|B{k7Qf z%t+lZ1_QeP<(1n>8jGe((CAJ?luBVVc1k~glG|NVQu-0vVKsBxq}dAh$i92}7QVObovRbxTUsQFgX_EG+kH<>^B|Y(bT0Wbncm$D$W!rDMQ| za2g&s92cBoXX7qZtz6sHfj-^8rG@%OkNN7`RH2<}`#I-X8nKg0TtmleFH5VnN;uRZ zGgFbBb3in7&JYx!Rzt~(Jctv^4AnW9Sog@P`Q0u&{a!fc}vgS76V zB%d$$C84rfs+Q8&w&JWCopb6BC5<%!0{z(9BZY$UYqoh4SsX`k*WuB-Od)UP%z~@z zAhp%L_a+W^cHQYHm%erwiVBcBCpq4c%?&${^#BcY@s3vkxHqY9){iB+;p7t6HH;(!dnShGOzYJjN@s< z^Xt29d9s^bScv!UWbNXYX$K>duCaj2mkW{?JzfpVV9&Vi^sx0$vvMKV z%|Lr%UE%^utDNBM7-!8tuW4Q_`B1FAw+9UUvczcs-b z%CphRGd)8HTs>BODO_?23J*-%P!=+TU-w2jo12@zdtikl>b}iP&-?C9>791SjsEdT z=X3e&PLj^k{FiBpcO@uduIr0o$zn8_U+{&bQIueN}OfyZUJ00;f(jPGLH)s z1LYEL%^BU#wNX|1Kce0Pp6dSnAGax)DcQS_QTE=u$dNrOWbeIa$W}_(o5;!vAt^+5 z9NDtT-t+vgbAP_S$NxNTckbNY4&LW_UDtD6-JCXWD3;wnE#IWbTTdjtzUUrGv%!XQ zd_u>+@_o~p^!km>oQDHXs$aj+Q%KAAw$cplonAP9QdjOL9SfKmsSp4B#htCA_0i4I=LH*7>N@=g!C$cCKZ)_&{>EaR^SK7gWZ34- zt@ZPC@i%y>ZxV~k?6M0BBfOrswl7ke1rK<%oE-WAq}7qw7WUcqdo5_VQdsY>)rz$K zS9L?Wx>TU*?(FP{1};B-2x1frNO6*za=ELvC8|PZC3>`e(gVdzuAZenpsTC0qF+&b z_o|kR-NJX8L@a8$bVJGUCA&m%i3o}@O#2_!ew}&gm&D`o6Np9_k=N-T^`Fs}95Jo! zB~C~yJs$m%c`BgwO)%=)8_i}0#Or9+Q*rvv&^dR?1JSPgDa^Q@0@Ukn19elulQ!h8 z_WY3zx{;^mUh*8-_w{AOkFj2{-8v^v`gYpbxY*7h*sTcP>#0}SmNymsqPyO=;rpp^>1E~;cPuX!l zCuvH(aV)d2_uSz9d3jSVF(su_9Hs^q^UlA+6Vhn;8bCtIFb zN(i#k1ZnDc4;@9RGyA+LTg8CI8_9lKindeZmJ{Dzs5nf>$pj;KX49NBm11Smg!{Y3 zyi%$uupCZY7xwy<|IH*!RxoI3k&h1*bcNyPp^jO)%j&5E?s5*ao?bdNpkQvf}HvozJd6+J=U4sL{iYkDnhk`QkFoTC2~FC&clYE1&mD zPUJBxA{uKbNj+2kl`F&>#o%VTyW(a#q#!WI_^pC&#zIOJF@<@~#blf)@RC}YsYQ)| zhHyIP+gl=;qDn$ET{dG$7- zCveH=ZR3xsEK!ai?D+vp)B1o&edy@O1FMam@2?D^H#28{o zkBTBdD=7Gg85D*EmPP*fC_;nmYf&paX0NSnM3a%^NId$X=zcoZ z`GvnBjt0dWUB*NejVPDCiVsQwd$SQ=zG>Ne;$Nvk^zmY)7QTLCq~I!7qdXe$+%e~= ze6ZEV@ou-R_Q-IKncj>HLREAwC$k%8bllh0c3mQUNsa(r0tv4iY6b(1)sF1!ocFf= zCB@6hFNGf`%rhcb3+S{;RNQ{o0xxG71}jufi4Jy7ZZEaDCVYjlf5h?67YBLR+C<~0 zo>~mm+hweLqU!o}mz6`RWA&C(sKeRIhV6N z0A!wlxn^8ZiTvY}V4g>9((|m)~ow?=pnE#6In?ekXm;%e}5$U+XC&<@ zMUBsWZn?JHYY~K+00g^FuYc%gJh{WdO4TruQSS3E=-NQiUu8yIkNc-fzm$_d%C3sx zo*T;s=ia`-y`ji;q5Bz&!k#V8&gG_Ssd)PQnzEVl?gOP)Ts&!*7hR=MpK949XT1L0 z)vhr)VqB|kIG9$&_-N8}=vBUuB~K92`#dkVB%08v<&Sw5Y~|{zpI&pnTnAYJJEt79 zOWz0cy0Gz?0n$Ys=q=nWmq`<+TwF?P>sOz9dSJWofJYWJi_MZK0}ZMV{$jj%dc4P1 zXV3okVW2YrM0O#DS7@AudSE($@`Kf*eA8y*I&Rb0$vk{*J(i=b1lW7jeQ|__16%Gp79zX5mE6Y={SF+$+gc*)BtY&{Q0x(h#?R~@!MN{s+ie&HUqX#Ea>!Jyl^4A#(hTf_-4dmqOd@RiNWO&6oEa{44!cw8|mh0nB zpO>6|^cbok0U~#IWLs1i9{I@MZq9X_eey$itcDuHS8+yWB9UofWb>KT4zo-mtvjBu z`*{>b1wmkU!27JA%lFC26knQ~6KEg-Y5U30@UTIhJryiHw}1to|E8K!fFa(4g#XOR zAN&BQA$Kp&w)XzU@M=NU!&d+al=by*fmm#5i(c5gO;}j4K!>jS`%CaP?~?@Pe@oV*0&((G4K=Fqdx;gi9Hk0x7jMS*#-?NC(yW&X=h#x)tcun= z(AX|M$TTyK$nDB!^&XAp<|&myzEEEJOd5I-$xl>^DSXH2sj4VuU<|18wIS$i(ros;^%bp4DWo z*55SIy}Kp0#dV|f@|fS8?eBCb);eFJIYqZ7-M%Uv9hTDjG-jSL$-pkEmwvxRvKS0V zhQI%pbPcbg19mPBWnGKE&x~#HKt^$V>g|V<0r16N%PLJnQ$_duo)f2P@vbiJdDF%pI-UKA+lS`B}eBe)vVBO#@y4rm0RIo6AHnNM&sK+sjk{vjztPeDQD zUdUzO!s6l%=m?!hboA~!54=yld1D7cvt1zKiO+uQo)n~P-sCp?Ra!k==yg=it)}!2 z9_L7FPRwmR|0=0Jpzy-Dw-)ovvsZ$o+faaW~J*Zzlf53E>E`q}C>0_rF zx_YYj=Nt4ww6iw6x$b^cl0l-b`!6DxWv&yQtgLR`+C81J5=}dpR8%pQ%|6byY9VQH z>7u(69Arn5rnu;-a3HlECyaIdYuDzCH;3!_NLN`J?hc5>X1o{bNSIIR`TE|7UE%O& z%;h007_Kfa|1ra25Sl1=#w;whsyJ=6f_pdVZ-wktue&DZyz&zs!T@5lRooXDK)kHm zz+9I&J!K2y=@xsY{b{C+8a#5?6m`q8Mf}=Z(f|o(R9U`zo96N{X=t*L;6ySA7 zBkZW~JD+|z`tp7_I@J5st5?<&Re7ZWoy7UNWr^Ru#r``Hbt64Ffu=T2zv<>;^j%Wr zgxm_l7p^_r5n5l>g?&V(f5g*i8pT{5AlVb7u1-%E^EQXO?)9p?EPtYAFI4lMziQHh z68g-p_6Vwt)CeIj9&76ET4dwszt@iv&-P`dy-9t>?$xk8(A;e5?>cW?O1;?^RAon* zoILR!gF*%f~zvs#F7=*_`o5%?xsT<7NEFP($j*D z&u?pM%gBczc4d$Vt6sUa3CfdS{6-&Zfx0Ip4VJJVV3wZIZ-VOjZ}JKzX1Nxor}$b7 zFLRd9ZCCT#F8!I35Xs_{PJ9@|S%?K7!fh_DYiYd^X8#=i@uIp#r)|jER~X!A{JP&JnN*MbkrKJY1Z<%T*e{g_59VP% zt|Zez*1LDNbSq53a%|cKNU5rZ#&bx*`Eq%>&U)t#ztgnD?(VKeH+3r9*Wy$_cR`M| zrT=_$WI_T-3?=V#bl!W}s5yN#z-XC)r#9&Hg9N__7UlSD(64=h#TN6KGoam(?ghcv z3bKL$DO?g`z;)BqZ$oezR$0x{o-Cg%7fGRCH;>T3gHk-Ls#-LYQwxB?p8v+$Am<$2 z%t~TmW4{O3;Bn!@r|`3sIQv)q1%9BA3c?RZ1AWlBOGi9i4-n(PCZrUq!!UJ>Y+A*Qu>DD3^1x{!Mdj9i;EaNHXmwci0B2og7TCp+>sf zzy3Fvsi&`&-D*XJ1289G8J=b2MzKj|+G=-%4g5_A`1WG6@sEPB)rUn>-#jX+Q>A}p z4q}Y8^^F=eS6s8=5+zjwwhbe{%tuJvNIqAa8L`1t1*C_pXCPTs(o>v96Q{H(jX(qy zU0PWZS|Gtult+IP_^A{QvSM>5!0&{wUozi z7zyz?j0+IcOQ^sJ>NVv-={4;^`P$4o1cFiAlv)G1F6~9J@zZ5>JRArQaId4u0#2}s z!vfih`D%}}M9cS{8^*OOgKL9c({U6Y^lJ4fCcS#;qx+$IJaND)-GC zzdQ=KTF_|}^(srIR5@AlRvB@3!jSG<(1|KJIYV&F=~B(l0VBcYkt1<1!830qjrq#KMe0knmhKy z9N!_6V+@}Kj>V&PaWSXDBxZLu3*BY(_&55{r%+%Ly)~i zYvkuQie24us71&yKwQg5X2pcoRUb(cnEgS zCx|!MJ$TCqW$hHEb+5^gx+j?3@9snNy_3jpp2U_Q6w?)>y6jl`I3W+t?!A7N!wr5l z5&=$kl?-v3HoUzhCK85t5%(^(otc4PAt(CdV@9(%U9E4nil`H-{CrmPm>7AG#d1hY z65v@!T+Sz~h>Ggd2Y^NjP+Vx0_C ztFO%M_3c-twQtpoi1!{>EKhIRSzqmL%pcJ{d{eft*M8&-DKE$DAu6wa={P)prMp9C z6tgrpK(j(7ilxXGxHlf2uK3RBY7BAQT&m|5`Jxq0-Qx1{>-XTT=caPR$25VK?+Fbdx!-p9ccGUOG-SlHI%+k%E-IsQB8NP&oUl5EZE& zeZBSQQsMMG>F7>%Mbk5->i`X`G{%!+sSlINWv@57o#k!xo_%hWniiw8GaIhDGj!Dd zPyGFSQ;=zDBI}1AOVfcLrJhU%G1#@9odL}O#h1vj^}=)6ZGC0M8p6Lj26E&jPbW4! zO#@FX(Vzs-eC_ZyPrD4>jU2S*VZ1iRq{)+Rcxk5k;OJDiSN^SK0y3y(BjI1QK3Au< zq4?XW-k?nD35E@nUe;d1CjSG8bElkjF<6Z8(X2HSx?jqssXkE6b#-^=wzOoz#Rd!2 z)8R_;H{s9{IT51N>Uvi zgw7kcKFO4q01}}CAvQL8A%d^HYrUc<|MS=h+X8HAdp1`2RN*`8J@;)4d!e+w!pia% zOcR2HUv5k|kweusP}!N(4FDN(?dt%{RY`en zZeg)JzdSFWZZr_agL5`Cv^ri@t30bvTl*H8`0-j-(bBl=NLsPFgKG~-%=#*#Fd>t#gR_JDH@OqGKTA` z$)otiAFH12Xi0=V5HPt@pB>(}@z}a=@Spz|uA8ssP2w-%5acYeUHRQNVn!gVYJ>ZV zHHnmF^;e^?R8z$6i>7fIIK~7=4aZrVM$22;lt3J4GVT{2LyB`;@2*7G*HsWuio^Qz zz-QDW<99n1oOTM}WJnWo_zPz9MfiPwopT08d8*OF9K@jLZW|^h{=jo@e}8|pJ*f43 zm$3FgC|a#&P;koqG7a4&kkQj?1O!N4US4)?`9~ute!Q=>V;u0DAqi2T3B}`RjHSA&2;ctpl{!u zsFDo!W+}dD246DKry{4*JM0&eteP~tai6d(X{yqzdu}y8qhYp^@>XW{dN+H&ym3w< zncj)}(1Vzg{=mHki+E2_3qvzjSY4UvjnmC*A zQr7>7Zs?63^{dNAk}q0pZw&ph@)JP~dhGMIV5olE9gtTVh5~NV$e7`odiHbBG>C37 zCsGq)W_oID25xD8aQYJp#aGE_xN^onY^a_Wz}YV0Y3pRn&&SUcY&ijiXyCYo5fy!P z->FrRO8ln7C|5G2DbQ;!vlzSEb4uG^dxdE{{ZBt2)SmQGCr>!K{W-ckn_YY{*(9uG z-XP*WrOEdhJhO)zD$D6%htqyhr@ma~opr|Zl70iNDEgiUyi{#B?&{@AYzN=Jq;dL; zMe9ztadAXOjQ=)eaxK-4I_xT;5BACqP3;d9hrk6l&|yVmu=m0W>^|O+!F2lcgvHBuARr?vMuQ!m#JLVJ49c=Qz?$gKL<3^W0LR0 z#Zh0E;wBovWu^iRG>FlId^6Fm2Uuun1@^ORv?7VXzM*Swqt{m7>qk;QmwMBZSYO8Q zG)BQIAp=wk%sqV3xi!wOFzGdVZeNxoOG>nijADA9?~PGYQ(MDh(DUOFY?af|aS@@V zuK;Uk5=v&TQY?_+a% zE-*7Gt=Brz8NhXPiOM3fzTa0hu5ozWo6!?waaNWcGDGNYJ)W9rN6+hw4%B%H5;|k( z!@55GLQ{134E{0}7ObF+?yCZQUKqGO9L8&&qLuemLJs5Mz0X;Ac;b80gtIgU+8L$- zdG9S))C*o{pl)_v4L#xC2twWuLY@Q!1RyAILF$^z-dD!)P7ctHinJr?BxEN|wawn{ zuIr`4?&0B~_so8y55rMbYi)lg;Ick@FV7_}tAjdG-ShDyqvuEJ!vequ_?&IjpP<8` zKYYl6ya{v`CYV7YaS3m3gP__CGLPzRomI$?w;ufz0C9>aF60hofT*w<)(in|58@SL z-_?Trq9W8?+qz3i_`&w^27v(l$zt~#`-!?<17$q~MfkmZwX`4`yFh-X9==bx-oMM8 z8V%f2`0(5Ba}Vxjm6oq_pI+)V@r3WNW$!ezQXxL17S<+W+?2~EoXdH3&w7CIW9PCk#ewJt#$DQK!(4lC~+;8%UtS|X#ER_V+(L#!aaD59^W}pC|4I-xP&UZW0 zrT;UPTIjZIEtE0A=J%`MV{Y^*UQ=YdE)w^Q*m9su{%%4Ge7ty2b&Dq^OHcoqg!9Y2 zkH{vJ^K-(H@bN~x-Dou$x`FchLql#LhTe5<#HJN_rP?`~X%3z^;nDIViJ7*%Kdha~ zezBm;!jHEYDhRlnaNt#3lCw4-^ds-ux1?VZV2YEVy*MBeawSwVWWSzYusk5f(f2`Z z;5!_7{$H9^;}bmls-XNUX~;t3pML&KK>dy=Q%8F|I9SlfEFwOq91aKBD;tJvm9QNe z`6a)%+? zRFhX8`_;zs`NhQ_z@YS@SA*0$P7x3gI0q_$f8^ygk{yCi|4Hl2vKt#y2L?2mSy&*& zS88PGoJWoTc<1BY9^O@$L4DVTv&9pe8fg>9=crU#sx6bSg2TcZSVvc+38*{5NV~%Dd}2Q|?-7sLvmZ@aS?p2X$U(d7h98AZG-ng``k2-zEH@$NN1;o|G zx*YQDpLdjOFjV)e6F90Xz{-PcLkArCoBB<42gov{c)CrY?u#pbfQ2!Nb{iI=5Zk-tj*C1Cu9BC zjFMUr>*lT(L1M|kgwxH_@@_0g{=hxngC;jnmH zPEPI^lGHN_Xwvd*sKH;BtV=Ola2%{DL7zy76a7)7Rh}~%d=!+FChlag?nel=-7+;a zG@L_6Lo4B(2-iz|{W`9kkU|l?{MHNabACZVlHR))n~8%Sbkfq&!0demU7WAhQfUb% z?5_C&3*8_IG6g(y&<(9jH3_e-u3A`FVAOaY4laZ{Y);yp`g{b%?UB{l-RRJ)ECwkl zDQ+RSQc>8_QQhq#4YxZ9XX}MWy+N1LRqE0g=f-Ter?O;e*AI3MQG@q24#!%L#ur*c z%GZsZ+i>^SkqI1+3n4*lqyDj(S$`-lAqL9_Ti7fIU(-98=>)d;dc)-Mr`n3->C>m3 zw5ZnaPPpuJTNlzfTV_7fg%}(rNXg8-hE}KXz^#D2fZmBm7QYHH%FS^-1aWNzNhVLr zvMbG-ovf{$`_B5$15njL6R7Bx&RxW3{BHme!Z#U}kf8k5l8=v%R1T&Qe5s55$71d) z+x49I@Bhblvcoz59)jpMfoDYW@LGV-+6@Iys~*K-c! z=;)>6g1N5lO*S?*jE^&wWWlLxoS&+zF~@4{Qk0*gb_lLrQ$XT=;+osoP(Rx2XkP~% zNh7S)NpKNqlclBHaFgobf5f!O&l)2F*%%Bwn@G zmZ_DHI+$g^8Pk^*;xJK%sp(WbzpUHSlARrVetv#)59ELHZ*RzWPio@e;pN(<4nHkr zzY0C1Q@xu;Lv?SD187IT8*SnCzJ^~7kwO-)k+dFO2Di_?rb ziK4>Qg&S`N@{Fv;Yg|4sSuEcmqb6TL^wY;aJpcaLI^o)Jx9!~CUKBvnvQP@~5>V4bug&cSf}Vwd;75{K{Ief60Lx9`E4 zBJ>bfu3cjVtrApi{wLkw_RFV&mjI9@w3gK|m99eXrIJnHd{8_H zru9i!Xy34BWO|ByJNR`ZdqrV^TWPia8r0EbsVv6uh?XEMwIe-QPuy9MM3V`A=yKRp zzPd#cT->iDz0IoDFKV3@8R8|PbxyGG`L5wu@y6#H9BUG+TleyBvEZAE-y`F!%icG_ zy!oi8*qU8tLV~s@+o_E_HuHe#l^(0_s_lZ%d?JW%O`i0R`6)l5+cn#~&bb z8B#<2$4luKiE1QC80%Z`9Z>W7r`#`;e%c_8iQsF1P3Bkfs?)2tZ#zKMb>7Ldy_J}e zp{}NORk!rU%+ZXp?G&}kKKrh^aL7pXSQYlkS}EETt)`&_C{HpXkCcikYTtdKhxh2# zBR^CC)XBFl^NjaM5!&H^*1UbsFPVX22#TGo=RF-QV^1+7RwpSCBs!?-)Kp4P61#9W zCp!EmYhfiXhb2O*5*P?_J3#XX0P6>c_2%Btv@hWzDD+ZzmDi?dIPQ~eET$kbeel2p zo$NB7$jm}!RL}gWdT@9sRtq=oNZ|C5G?gu;q9S{qvN+ut8p&VIj1h9X-EE$L#<0ff zh%C>&MU07PJYrn5>{LyS2k~$wqykL+RlV_6+SfW~Z zz`OY53FF1-0j%C#>Nih^s3;dw9>5pN(!hU2^AHU`_^>E*qHu!z0;#2jf}3| zUN&y-IA}kC1*H=AtgeF8wGa;7-ZMkKi8+KR@>;W{~Sl|dP+&knvvNesS&R9BgSEBHut0*njn@qkq$-HZp&6py8j zfBEtyy$!eb2&(K=dguOUlA~wlwG5^GRpU@T4MjX3J`79%T{gPwB&HKnFgB+3@bG}= zu?@79=<85)MNN1(153~v-B-VX_EiLc^hX&xDDu?IYC+Mwsf_VXTelgnkI3GMz^%_S zXFgx*<~Qp9*mGwx24+`l9Ch@6woG(3J6i2g@scV@atrcmh6Y+|OVt)QdWuzV6vyA!2^1HUSTVGj%8D}49T_oIIhnh8fr-P%2P z#Y#5yvBM^;H$Elh2TQ35MoY^jy@qq}EBJOiBb6ong3m*-zDzoX!(a~c7=1K8OmR9o z9v>~GR&nt0=ckg1ZY!AMMLbpZ_7?fXWnL`~J|TJ%nsCyw4PyVv(ef`}Xw`=e*?EJ> zqBIBp8D^-#{h0t%%gvY*3P6qU+%z^ z3u)5>FJ_&U90?0B0U^Ksx-}l%yU5F5At5O$E?&`2b2A35z0X=3*`$XjlNuPD9M5Nr z+(M~k-{Pm?vGnzP}DBf3N$uDhjumDG}EU?@{rqmfE< z6))jFX!DqdW|TI!Z+X)kdFLME*KFiz;#J6BwFd@%QeH_3Z-y&?CP@&#p=UwbyTE7{ zP4xnw_xPR*5=gsYWJHD1){Q>Ka)=L6pZjaF$(2V{AdeD3hkdNRp#l4`wPc zFCrN#C5{=x!x`cKzWu0UxBKpFAzprkKBkLH)Ia-$z>9PZ=X;<27V!5g(DQxQ56e>~ zo2X-gE^)j>r%TWD<>e1bwCcI#8E*GRy?Dl9-+fR;`|MkPA1x!Lj-D(^Kw!`4A86JYNH@22h^ZYe{$v%!V%%(WIJMl@%tQk3x=2Q!xTW!F#H zkA^RxhbN^_aI0C)QWit`=STggo7OWUdKKRFOX1&iod1s-8ai0Hxj@Y(@_rc*8Jm;I zxR2cs;I}Z>?sL?91i|~92Fmf-&3o|(xmkwjAI~7(A@E)X(d|1K0Bwwx8G)5q9L$Ax zpu~X?0tnxg$%eiA)sxV&6_=J`!!G1s=2@++tveY;2!bnDaz1{Hg?bC_gM0T90gwi< zZ-_md`JlA;@zi07%A5~nC+n#G7HM+h9V2O!iqY`N z)bg=kMbIO$GgN8nHcE6;za%>de{=UpjzGw1`XP92pi5CEXamhE0WopM^J;{io^pk0 zvqbtwsl)y;)Fr`}H*4_zLOPcH6G znW09P%Ex!D>I?VIM@4!k0#b252))@lZk{cfu|2B}NF7g0p>_$H`m&(1C#CM0cFuqQ zaXZNWF7HEaLR%b*_84Yi>Gj`KAZIdACERL{PCkqQp#zs|DRo}Ip z!e57A`_5&NGm1s%%?!vDwtV@NI*`anV&H$zVkXV%l0h_;SmpSnF@1zMvU_fyD->Je z>}qz)bVNe^3pf?5{B%vpR9z6vUcbww^8}{+XS>M1 zm$QGfG={?tG9uRz9nGt*m%h;#H*7gu=d<^TA zDViQ!%3JR@b?<^oEJC5BJ>+sw8JxpikXBw2eGFnY8mIkR$`;RaEA&GaKNK;t;?NT3 zrGITF@e@NKDLPD|j%Zy})V?UnX zUJiQ7blMR&Fz^HiS3nenEPoMqxYapOBUe#;%TD2y0c~g|F7%c|?)Ae8X-x zYReM!VCUfAqTOKh_U${N&<&!OmciTQ6^8k(tu5^eH&jayS8i@@lX+KC2R#=bbctW)l1j=_uSjb}iFep%I{24t zLs1iZRMEJ`Y3el@FG1ZrJl@;^E5&ewn-w}Dg2Zae2{-1P z&Y^X2?1I~BtIA$#`p_tTdB#-%7>{&`|LcaXNnh+=2g19?(Lp6%Pe*6x2c_9I?C!{X z*7NX^Gq5Lwj-J~8m*VdDS2befiTY;!M6G2wTIT>$X*+Ze-n_P<}Qsdr|Y+zdI#| zjGSJfUbngeS48&^jSul2U;v$-G!bXFDj>a$qh&p#gzsa1CHvwLV|@?~A}u8I^5$>L zXH<R{-q39QX4kpRdNCb_Vj|wNI5=A zqanDW=B4Kovitu|RPIwU-kG^($PGP+N|F^5t|zQ#*jzKp3ah(#&J`BuT(|mui|Isx zY3);@_(OhbIvwjX21U^e?wSpA&C_4_sN5*e}B3t z&0>p-?^*2AtWWzrsm*QPz54dUXF|Z z#D;f%H;b>eljMEb4lQ*L8782+$V>R@s7xc&8Hp9bz&&J z%lXO-26}@&Aq2#n0YUc+Lw}g`Mr*EnAg2F3Iy@rXoki=3GBUXC*?#kU8!v9b**esg zgNp=~`UVEoL)iyu*W%R{hG#IH`hPt{WW|^sv%PZSa&3x~JyiN_l?O!>Si}5a&)(;Z za5>Bnx}@t$jB2AY8&C6(O9%H-d__tn zvT~l&pT9L}vrmAWsQD2?v@D$sps$uS`))9%iaMv3hle$2J3!IwKWdPe_$1ayD|D=d zZt;9C&g1jB>+eWo%FeL+Wm%|MOjSe>b_2C`X5NMcicWu z%rDHH^kz9ezSW01*=y^>Uu?tA=nLxZ5v>W5PJ$Il2ODQrJbN?IAJCTI|Bm~6AOS-w z7LIr38^HYF1ODIbgM&q22=LMFUt|BBAO7H4E8IgX(VP}%_7>XzlcYf~MriqZy3f-4 zU_}2pV6V?W&Q>%ASrPwxyhdT6gqB;yeJ}a;yE6dlU$)gQv!3W=tfz~~Y`ylBS8??o zrhhIOn>5SGcV@+FyZ>631GnRTfGfY+K>Qx&hY>oJz$@0Efu0JX6{HYddiID#`>yY} zc7Eb;TPu;4vv6ff?&Sa#%s4<5)KnT#RLJrg#0{<5D2!+*{J6QGTc5{}v6%A=Bio_% z3W(w76=bp*e26%T33KhSUDul?@pK03roz)*3ANtMdEd}!BD;}BR1P$xIJMIha%kwhbiIdxnlkYc)n(XKzpj=EF(8^NU zeW|p0oV@J8q5raY-0d0V4#~j6F@E;vTby^ngzqeds(mDl=6g4$ybz8pe-e+JazBwD zBq?wE9+`j(ZS2#J#GkE)hDg4*$=SScJcHe4IwGvu#_aj+@usb3K6AM3pv=ty-3v(7 zSEMhaB&`9f<3vyz(?MW5M;J zvXP3PEr7@O(fFn1q(}`FzvgB`=ez6KPLoj-*RfSqqhKamU4^z~fs7^gNkl4j%?rY| z8HSU>G~+N`ZRgpY&JUG_svo(NXt6H{5^&k0zk#F@?YBU0na)#+`wQgb zaxKy{ucjs$LPz5v+-m5^jPxGbkva2s=4AiB+wVpB`oj#dH$|DQlA3-1WUmYtkpRhs z%3wiThW_`^hy|O*(P=K~D(&Km4eyiH`rjkBwSRk9nXfk8y7>Aydla0+B$s8RZ^TI# zX%fkm8xN2Jo|9P73~K!(snlq1@0|qMtJaZ?>yC+ElFnZzP4Vwx4j}5|bT1Jr6iq$aX4Y3p1?g2lW?~J!7lX)0KNVNWq z%sgQ_?qG^;ap2;gm7DM&(ee&jIG*bm6#TG4wZ&{-z&>Rq_TVf%lnjvc%Y+k& zTne6tLFvx`O;nQSHsg4%B0z+8B4@Bm45sd$yaa03%RFWI&!xW;i$5yBsFZx=x^dXZ zXl`%T)~_1;So=##Q<;_p^2$Gn{o?(#6@_>Q4a=Gx3BsQiJm{dOP1+$V(W9=E|2g_B&$;HjQER=o%yP65Gg)mF<3R!XW+2TVlP9-j z8|!U~sVkZs6Z$(Vn;(KrW?-;lhhzq7>u5EMf^LJ0NahLSuFo=);2WXhQu%-r%Q}He zI-3&n%SXV6cWR^6Qt&t-B$eC#e*FkA>ebCCJ5$prsR)++@_W2>~x6MC(G1u7wQarS<)^%p=8RJU z(oGaET@`vUxx!lTw^a5)@IIZRBox%zFBvg1To|0s=;b{rQYS=r@%44r+(qJJM@%Oc zLX-XzoS(d8`*^Hd&iMmU*%O+==_IwQ`o1kBH(BgV7q~EH^cRZVG?7^`mfQW=$4s^f zxkatG(`3NB*1XF-eIU(+-NJnLZVXHSGmYUvw{&zsoB>>fZOa)kGxI*_FaZ*wU zoTcDSjIeKRinX_78xOz2o~OiAkS~%(2k9StHsk_tsy? zzGLH+4-Q827qL|AN7ICsmQkTt`P0pVlw2}WDF5Oz#kJ91H!M%_(@+t&jhQ331e*37-ER z3$04__WxSdFwsO(!G!zvOP9J;aBr=i)2<~HF|ozUXne-ws1X*R$ll6|8rlm;xWCU* zA!kCYR^byohcoK7#NwmO4EeUnPJGk&o(dZD*q?sv9sf&60(26(MK$CqhP59%;Koh+TRMu4gXgD?2u>L!8lA@81K>&NSnhmh`jLodXAl@kK;v+Ps0bPEPYF=aA?!BN5fm z7`rBeHwGlxFexgEp4qxB=T16fKkFo3=;7HDsRv0F*m;?W=xo1aNi!_IHTsG~CZ4vQ z+RDETg4l$LR%SDI1mRh2gPV=C+wEpCk8ivC!x3~OlHr#B^c7UEJVWxd`ve`|g95cL z>8IHd>rys+bnIds2}*H=M?u#`Z78%VjeII1uxiSXOh3$sLg=n>poVIr)-Jj@>cjLZ zd*X*`t(!WjiGBaj(B81d&yt6|OujoLU1eK*rz3?dYNL#b8IhS6WH88lMT9gi8s@L; zC5cPNPz-de7@bRwT}^uJq||?QO|%jlYPdZcNK3Gv0#FW^ljKqngo_khcta>ixsvjY z+*4p5T*#A@VEA#D#= zhz0;9;2CKaDK=Xs#!!D_x-QPbSeZy@t;cjUgF&r=bxjb*ps=QfA{awQ5iY^CX-Rn* z0=dGeY#9I7j4^{_+x9nmdGnzmj{}#Ufb*9H zouK$X;O22R>`nrk{B$$`PZ*$b`Z_U@#$(c$4!F(#dtBOq-6qTS`lY}(UsA*uJjCT) zHKYyDEOFz9!Jf0K3&hvI{#_EN7>}z%UE#WT8_L#YefNaSmS0oD54XmbcBcM^>uD2# z-xctFW!|Yt$+7w+q;`!pLFwumnyk*fJ@V$o%r%G3TMNlXOf@7JFNm&0I_(2Ftqig*r<%lP$ZJ3W?s#+6Cm$wDEb<9d}SIiylkB&;i(}?CP%E z{5!GJxz#SZEIH{@PPLR28EEes@kI~G7oyr!WIMdBp%I_g5pYR`1NvfBK`gQw!~Ne@ ziC7aXie}*yb4%8p1jRDQ#Ahb&_k4VOv<+x9`}AF(QSK%TysPE$+*E+9#f!6~sq}Nm zlF9PN`V{NhNvHABFW|tIb-QaDnZk8Hef!TJ7+U8=D>M=a#o&YkkDq%>27&A~@r|^H z>o6dFp46LD3HSK`gRh1WrQb41Ku-WW0#XjQI{p*HLyITc@ax_Eu+UH}NI^77@GKpP zZlB8rtHAe;4nDV~hoIa)tRR#Vf@Al{91lH{r1OE*b?mbXi`V2+mB?{>q0N>vM|2(+ zSZNItGxusxT__ z4Im!xh=Ke!;xBQAi@C1izA&5~CIo9D>ZhU>_y4Rk5F*AUVM4lxX?Tc?+&lHE~`Z~EpIdLU|z ze}~5M+T@Z`{e(@W+b_Wrevlios&>W2y9f|nEN96gjVz|@82mAah`NNQa4<1D&OJBe z%<8ULe#0qpGcjwXep*XSwbPr9N>0uOL0Nb65%Y^kmO955xy@2vKBUI)Mf>JE)8R)!eDt~y6pKt=&aOYQ2ZG6R#DFURpo4!1GEr&#~94f^*d z5DyBL8I~X484nz_wb*&lx%PSckS0h&e1nqg-!!&EkJ-T+V`w04BG&*lWLFl=q{hv=S02b$ISa|i={{4R^Ir=aU zN(!>W2=-i249~+W7#@zAWb2ZQJwgb{9B-s{SpLD^0e7fIFh&Nu=MP+fA53AcIlXP6pj!X5m_Yc*G1^oi8kqg%Cy|g@N#+ zXqaj*zdvkx4-2Qpw>@XZhFpYm@K7Rr^2A4BkW>D2hR*29T7@1O8)wyf&*!ie7IgUP z^0vCcxH`$}+I}f{X!DHy%1U3CXf zJ?iGN8*Gj3W=UXFRA3}#Py+14f1`Fq8WjuaTPg^|3XGuLJxRQ+v~u#8AS0heJ1>6T z$y%t>Get*Is<^nQNcw8(=|y@0c@m5nCFz&g=q_tJj0fgdXAuc|rT>qr_l~Ff5C4bH zaqK$9F}~|L^Tf2^T9a=L*N+rgX@D3k6-G#ADi2 z|HF%M7jAJ&NW=r{k{-BbK&@{9M-}r^5!|Wxpoq2hKg)}@mM5j(p7aa;$u}Wmd=`hJ zTSB88!TTY3ndyHKKRP;(*tocEi4`~$L%WLqp$`f`_d$NAl@~lN*Q3uvwM#&t=&7VS4Oeh?;h&UhOwsClya#r@y`6qIe<%ZW;d-Y-b_q zgoQA$Kw8@>M3g|^laiV!%|ggK6@rOCAYRb z#l^Ba2lY=RzYU7iK<+|YVNnk8Ns?o<#iJM)I4{W!XI(c0^43Mr9~!;FdWi$<2h<5P zahCRnF6-YxVZ<|J!sx3St^z_4=P&7B@!GWfs-Npx9`z>=yg7K3IS?FXW>?*%4kOvr z|DFB_(b2`NZ($RZu88Q*K6*rl#f4|kdj-97jc{I3tdQ}Euw3XRT+C^e~w$Wy#r0sO- z7;-DC=E`g8+#d6o_;>Tp*u|#sjlG@h9tVCdu1PAhKX(HQ3JU(WFLt3V2Qz*gnTkpB zNqli0h})1T!Ygn;SGuo|*5O=uAiJP(u;`KO)-Um*p{F=I4V7TRlRsVm9kb`}BL8bt zu2}s%CtS5GHe6;;RtFRaS(h2&>E;OerniazfsHu4(cnN@5D|8qEr6;exUK@ohpazi zh6NEq_iK{N+b7pXrI@pqB(j=+1+c;#tU1#y(c<+zEQ#WhUshOPD@ntLS>*e!;yE4y zk865Q)f~=bh}m}LumUFrI;Lx#5VWWU><0#8C#0F*&iPpUizMkIO8<=H%3fNn@2nxk zb`n$ZsA(ppurM1uxd1&-9Y6g^S9oiiG}9AQZ~*?$_ikc&7z?WC>v&_Ae#PGqiBi*B z3r^G4On-5;I;DzH)}nAS%u(R#s~UXhzd;Y%9gclQsF)_N^L;E#Y~qp*UKiDEjT?-C zGc4htz`e1Tm?OW|C}v65nv&hR9o)&)LZv~vJoBXFd`Rt>l{3X5Az0iV{jV|ml z?QDuqbOv*A0~Z_{SX1ZGZRbUuuO5?lh@)U18%4X4u6EGAE2(QW@U4kxijI6 zySDy*BbW^&GF}4VEcko#Uf_PVIOBzlV393!;j{nyy(RsN`hr{)+a2R${`~dv(EE6> ztl*hB>jdL$uy>jKEp4)gdH>NF=V;-(?*!tc>8CJP?#1=S6`(7=b1npDhLYrnO*}k2 zylY+o_w*SzSQfzV&XSBu8#Vp1s0@mcws5rnmrS3;m*S^I}Mf$80)u-m^6l zh4>jO)oeohwN%l=$gn-HV{v0xN|PvV__2q%d? zM4?;1`mnlwAs>HCrUnDus?BT$*J%0iuus>Xa$t`K4Zz;f4@M*X*9kI>kvNl0rLgQX z(nt4Q62Om0jdTeLONaPX$Fvn6a;-BFqp!98+s~rUL7O!~f1_c;H6W7}!!lfu@Q|r; zOU7V3D%}|F@})n*KG^*qt*ZrJRI$rq21hL%!5^76cMwu6kpShfUqQ?=ju$oNn?NnM zDc;GbLI7jp%|^PgO&5Uv?m&j;9UY9Relx$%?s`|vp6>no3kqWc?cqhiL|dD4sVpj_ z(xa9FqyYL&{Y#p}9v7i5*If3td$Fy3RF;F^lE_ql?hViot@)j?1mVnD3(ev=I&jzIr)CUXfT z(VRr#7!k`6Ilm#=izps3pg^gl_1H?$Y~^W7VD?&}%j^v?@gC%0`Qu9l_QQ)0_g>hC zR%HG5hKQ~VHs21Jyx6EPmC(xY1v3z$$=RPR! z0j%qA2Xv+zNZE-pMMartzXiS-tlyp0n(wpK;PRRzP1SB!fJ{@`=LC;cNX|0*h7Z7a zS~V-o?>x6!Y1oavHF`Y_rDXG*gZWpj6`4PPBxSZSTdZK6Vo`D8UeY{%DYaEICkG7> zCU{mUh%Gdoo`n5S)wvhX;sC?Qrm>pi^2c4N5pUBKUdK42I>onpoZwkz%!5Wsu#V#_^Xh;|a$rCFj>!W7# zL?t$e&WVSF2!;FCNrr<6TER?eGLr48wA@iYe+nAr>^fHaM7_XV$~pgk?&af)!sQcq zZ-q^6@PGGt4_4h#oaAXq6|l;sfMw;lD1 zJErzr#c#ttKPr|EduiM)Nl@}k_KXI|W%YKg1?D99=usC67fD%zf(e>gmUJb$i{&h;xoWSN7r${WsqtENR6my0aIp(;wa>3yR>)^( zMR#glpu4-t{Wfq7jEkaJU5nE3yS7Z$BP5#@4&`(W&DOQ>^4Jxp?c*H^GVH5pDz;qb zo5r~ao?RtfoN}h@*1^0zJogsbEf=eJyT17Nh*)GqM*w6h#mDe)T_9Y_xQ6ij z`&6CtN1BS}$zjFQj-RT_gUIXGHOI!f_SYZi>5J`bw0Ircbv1!fu07{Yrp#LP zR+vfg3;piy*7|zj`p!%qf6AvksanHfMSXposm_GMU+;SYx}WityvUIQ1lAr));Sb8 zg2N?`J)y=OJ_fU-kxB9!2%sR(htsQA^6H+QE+JK`q$}gunmG*)_SfZ?oz_z|M%+e0n$2x2Lbbi zy>ky-EyBK0Q1>wj*^9kRaZOHKL3lUIgDGH1MQ6XeEwf7~e_hqZqf;r~JodonJ&$RH zEBsLQOPcY$HksK<;XPKQI9>gFJt!?>0zt@o#iEfIqyjkV+jVCyW1s6DDYZT%^}r%q zbJ`9PsZ%DnxM+;{55U-SM9&B}i&^qyC^1ORiCY(t%=Hpvem?KKfABlgpuqHIAC4kuG5wEK~ z4_1kpomh<+Uy4sY;DZmh*7J~%6Xkq2aTbZ+*@hb%Et(fEwhoxgElxP40rco{cB&bn zyK~53hQ1aO(=^V?rDNtpF&m>%*GUDbf-|CzI0Z>pBBY}Wu*}CN-3gafas)wSIjwzb z8qnn_kjz1R9>H8{XI-!M^bn5*GfQx`H~CWxdwKM^b<*y zw4?RU4<4~x?O{E-8I{kbahKD#c40NMa_R~*lS4*X zG%tz$s&QBpk)xyL&F5jnouv;-h$}oCuil}-(Lw``gf9ASQ%{@!k_>sirok`!>% zdJ58;EL7fF@m@a+us6q-lsZv}c?2HAL#1{MHQqZ@FZolPBDVm{0yBvY(54yI`ul}% zjEv)i#$ek@&Bhi{TdTYVyAEFU#Dsa2e~XV_b=f$}^wq=U)}`qP9*}}x&J=jnVt4IM zG;iBMQ8=7jOvf_fI1)=8aL>6*qf=Y$)@Y$b1L`zbz>(lt~zc;O8g(hl`LpT)u< zcdNtDbybbOqJ$M&mJsL^6Q{u=SjPPdt0LQQ@BI=kL= zzAqJz#<17nB#gI9A*e$>>M<#TxXqijJKp&=xOcnz?EMh0otfBHF?X}FY;av2Ivxxx-9EfZIlZ8}kGMLkZ!3VP zy_yv!9;52Rd^dOSRUx~&!x3kWe#r+93*5c3XP9{$I2A6iPf*dDMQ@j%&+)3~kyLEJ zC14SphO9dWQ z*?(7nL`jH5!X<4pSnj|EIUc|QiIn|#_=8Nqz&P+6OZaI66<}PPe7rbDl5dA>^OBjr z3e=Qf7zTzE1qlpCduww*7n^GW3*JxV?@FuOojdYPwvRS6n#VkO23ndkz{4KMmC9~m zm18x?_3PT|*bQ~W;?hzD)Ru|L0ZQ&wqgHwj!Vq;Vba8U>^2y>Qc5tZNUa#69ofZ!5vMQUnF zuQ~WQSy)g2yv;7d?;=?A>Ntn}sqbYfWpW3qeZ{k3RnKej~|163S8nVCW6RL#hTc-%uf!VJsm6z3?`xiJx~Lj z9ZU-!<2P$DA0H1sm13m~b-jCU=Uu_sGWO%Zs{O5i&ZOa_a-X-3gLsy zz>Bx)saF3wmZr9zK3cIOEpalt8{RZ!{P*L@@A0z$Q)&BKDr~i-FikpHRf2wDZutGN z=>N>%Y9l}u2Oy_LXG(K$1+W#Az%Coi>W=>Zc=Q?GDj#e`x0OETJ2f2Hb3PLZ^G_N6 zsM#K+*17zVi>Ba1DIW1epZoL`n==v!W&wEe=CUVjh4@~t^X7!F{3Ptl;=V>nh!3s; z+Wg<;!PREEVeW<4!}4Hh`qkNDj&K1%LEKe~^QaiG!TuZ+=9E@?0NPo^)ek2m#1ST` zlyk2_2QrqL8*0nTWpLik77aqa2SYx;9$?J*FW}{)f8R57HT7v7D z(RlJW)fT|#o!3bz!0%=zt2<(;doa279H2RdFC3(UF2J;gRhhhI-7dcD5zQ2TW~Nbi z&Mgg6+pQLnZPBi0Y*r`UIbT%0)>RT74NszzB+1g33|A|NF4Kbx?Nox9PKxIbwMs0B zeFy7|X-#y8OoLO{B~sl4;V^JM8UC)ZINa|v5pY&>M~+jX&+9AbiSo5Sl*7eB|JATQ zxliv;fmtUYxj2+`E%h0FDi!n9eHh^ajZIlj zPCh6;iq2^#8Bb^hn8YlJ;ZDlnM7dGK%6k(M^xY%-lUL$8IsaB{I0;+IU40|wm`p%e z_+v&!(Dt_5Siq6E#AtK20VTM`P=bim)OX_=_Hgl%5?GaB|(e*gR6>8;4^@nL_vS{N#Vn9=c>i?poy2%7?U2AJxv z6^;4$_mN~^B&e|!V%C`R@q z*YD>bK!yL%VK z$hQt6NiY#e1HyWV<9G_dti72UR8VDaeJ2w8AC_DI+53+hgO8#%eqJh&as2w}R#J1C zO7Jfhw*MUz#MOp;B%1v%msy(}#@s!#&$8+$^VJ`44S&=sO!lDWoxW5faEgTHr6*nV zs45-tZ0pJlMoPesKFFYOt~~p!W$b=v%N;+zV|nqU2)nq-swx$&2)t|pqZ^}ICOIs9 zssuMiO$5#_hA{VOjY_VrrhM%j**X<%bC7!A-}*o^@Ic`S)R#%%St%i!IMjEH*b4m{48OWP4m=rms87I>(ImwOx840% zU*FA?sS!n}`TT+qF>6bGcP@RF4ITR4r@$f(BmyjtTbjZd?o=ep^PY8XY-|SZFD6Oa z{>c}~H?9aWS;Gb=_Os)?&~rK;s*){7iRgMI%xR5qIjMi+OLcitX1~XzK4TAUTYXPo z?^p1)g`D1REJ0oHwKasm*Z9h$Et|&VbeW@vU0q zMcbHUn!xWwWG0l@Z6fpgRP_l5#OJk2R9sxd6ciP?u06$nvwV1i6%j_9A@D;7=$II4LP~s~RrU1L z0TqdnJX*>==!z|MIqCVY&q%0Rjv$-v@pd*Pcyg7Cw|mA~&m8dWw@5q!Hj^4f9V z=k?RYONh4uGTeAo9#lHHDojimb=1QdO*=&%9%K#sIcCzy;!gDE&*o7aYTzB*gY}yj z_h(uDd7k)D%g;ySHJA1Vp41QIAoHu6>Ir3K&8yr4bMMm$+`%)BU`}his(DEtlWW3a z{N&3Hvl*7$d!PEp@=(Bp;8~3TC4w}7;6fY;92rf}`H5QpVuTKt$lu?85bGz;tt*WF z?UUgj_g=gT5Crypxdv2#G+qM&HJBuFaPsjH0AOA2dmt7R6qMHgh`ZaT2>&C#llB3yC@}rEpxJ6!3A)@#Oou*I90UMbM90rt7g+ zz|y<#>;2xJ`>D~<7?7Oc5{Cl@_o)y*U}X@+uGj@4EEp-WfmK%vkjgemCC~QK<5ByL z>pnM1tG_C=JMNq)o{+M#-n{Mwo;XSd^TM+F7_7*_?* z6C6TtUkc$Oi-Em}3$t$enh3A@zvo9lNPrOu;wUx`_R6Ej)UKRk-V9$l0i6;&*+4!4o)QGW?}POkto_eITsom>lFer$&6MEZDU~xSj76 zciL@4%4{*%x*;$uJ3m^7SEM?`<^U!n3H@;bTXzT|#yb}IpK3wDY{VlttA^piW3Z06eDXRASRE0{EE;TG>(y22RR+i1_F%IsToBp1!upUY|0Rs10cAEM}}5uMPXOcp$6OrrG`aRp`m z6W`2vouj<6He$5#rOSt-pogoYC^GW+i18rA%&dx0=?(3pwRPH-h(W`ylOtH)gx56M zV^$kp1X?_*_dg{0k#&8uGS7tU^n*uTf1aIe0&(TO=iqobmkk%A1o8SN-pvbr-iPNj z)G&`) z{5_$VsN^x~#`PF2VNY*WMoCgH2I0E4w(Ea3j=iF5A?x>;D4rHUTEL#!9i|?^oPPBs zEu^5f0K=oZ@S&QFj7*xx+L&gi<~5_QirNlh=D!9a2R)qlVSpOK)3U2}?wT=EUA@W* zsgDLF_@kjZw*>BeIk)Vpr6zPw{jbO3Lomk3(a&QBvw@<=vlmAKoLMEm3^a$Sn3*?^ z)Klx#!RDKkcVv?Cm#15iD{0QS7JdboC0`ATrryXPu4-sJf_xM(7~<&4x=sgPqhw&C za}5kE*cH8>uK^F6LD(4G7Rz_%<4vX#`@i(K`6OU#xs09X&z;t@%DUk1t&Xbc=tNgU zh`xwYK4k>Yhz776-TO2H$Tc(AFEt(?Y!v|$^y`D~l#ukQ20jvIZx%^O&oo(0H-)!u ze{vkhB4+l!h}(3SZ%j4q@BVcx0vBpePfwg5+&=isP23SU8om^I4xUrO7nE!_uvqNB zTIr{QXz>0JQjfrW;J@e84rJ``vnCwRtz%3@u2M zWA`mC(1`XB6=V}iR+Ati7DorgJ}fi1d1(=CRoqNKPsNEg-eeY>e0X~nrYVN3@Z~tg ztXt=-lsHArkwsalny&^`ufERel5BJde)_LXY&CY_iR+U3UCNgpm2r#$wY7!inT=DC zGX>+Ex%r43>uU_&n-d8M1qU3wii)EO(jNs*!Xk%TU+sj0?T zN1vXlP^g!k3E;otz^+v=kTk&U)x%3$d-B%MlK0^7kQM=%Ip<^awLZeIw8T?1 zMqSNPBDNg<%JlYf8(nN9J6~_L*V~s1ZoCfT+Ly&*8}0vo<^Rq2GMs@<>jt;;fN+J7 ze9}!U*7Cjlz(^j%_JIA7EFOC5a7V=KY~b~)^Ot%i>Vv}{P=o}KycT5^w zOyLK^teMqS?Z1DLd1~H5fZpxc`P|8h_(X0RnmbUx-U6l2lm0K)#>SK^9p}_{cPTBw zhIZ$6p34Orerdt`-*_R>?bSbjY*?hD7GL+(a~#6m4rUY!l5dOt-0H~mUlU1>onZaWSg)6n!J~;E%7bVQQphWzX}l zFi?04;h|5XMA|#>uHs%h!gi`9g_kddyEjkx>eX|*=E@t@R>@t_i2F!C+Qj_YYv6(T z?~yFSUz+QLmBj_}$g~@81lzV=4r*pa#B_?!#_U_j6~$}uhU7;N@XZQdcW!ygpU9xy zENw1bc%}6%hQS7a&SD}f!fGBy`Ku9}~7e{e$ zz%uu7sQ6`1uhm=-Y=t?ks~_(dhi!Y)es?HMEPLF|ol&hoJJ<1MAW~E;&(xkL6SpNm zK4RcKdiG>wbo4QZhnwG`ndvZRpiWrJ>dSi|{p$7W z;|8(waKOt#vGxiAFdz53@2#3ld8JJG|K8Z7`sO^X{^ZYBCK{U3r$e!2j^n2{W&{NU zj>SPg@DbR!@5h6OOYJmbEsv77&t_A%$}(^}!3uf;3Gg-E8i!jala}uWw98{>GFq^b z{^yzOZ`e{WsdW2X-oJnAXZ%IAfdVtIA-w2$@jBP|Ik}&j)uV4TZC?Jee)*}*vpGDg z@vg`lekC@ItlQidU5JWi$as~Qcr=QO6C5Sp<(FkS(nprL;O&1DvYT&w>(fn;JF-nvC-C2Shf|@3VH|FuCW%x#FSYxT(^}|vh8D(+A ztVY%=EcWf$c-imeb7*D?rcg>g+G|T&d@mdOb1ypoen0+f1=%&5>Mxe+zxD4H9@?s= z=4Df3wA&`{UM5M$m3xWDX!Ex}dfiAs z7S_{yeWWn3)waYGXnJ8TuGfuAe=ZwxIgAQw>@&2s+Bki8*#5mfsaL7@Ip8_h70{6( zx@Owq!XKQ6W4_9LC1bHa_bzyq=785-)}?JGQRcA7{NC;t3(UEx3?}TEzr`ibIgMNL ztnJ?Cu|G0avq)Eqz3R3Q9oVrp&|5l9=vvjL>2?+rqy4mpCNAuFEpdKsZL@ZOvHOFo z`mo#swYknze(2X+oSoAz+1+<^fP@K$6OrW6z%HqGBBH&{P zH4pN_?Ub7^lZ@;af2p6#$H2hg4Y?neSXwN_x*s^!T)SkDpG8E)4oCamrMNs;!ku;S z5r}~PwWbEgX=ZYAvVk1o+23n;xPs?Y0o*fXoF)*yetyW-omh2%%OU!+erTw|^UYnI z|MTp3rt9a@aB!H7JUJHbdm?GrryXsHFnjd2;={G4c;n_m6FP(lblV&k!LLvqKhpgk zj^Wmar^WRN9EhJ}1iV55($w)g)T&!G2%9P)PDOSS?Do;Su}{3<^I8a&L2gTZ)TOxz zxM&hMk3;01Et>DIPnNsPlIHt2HB7INVSch*ek$qhTS<#>T37vc|KD!YXp!r0%f2sD zA;}6nJw3%SadE+r=ri%-tw&7KKI0B^>D!jQRD$$Xi^4{60h2HItShpJx?e+ER{|PL zW)^rQ97Y(&e{mq+9@Fi=YM9wtbtf`&mNDFvU6=fak%%#;N#g7?-QSeeXQ#m4j5;P% z%v!QXejr9%j6rm1$9yE6y@dU;To17{HciDU5qJ}^S_&e zd1h3^K|2RWd#L<;f&OCt5+kA2v6}EsrIT}cU+$?5eY2XJUWp)0N%73JfAID3D07>? z|9oF|_`&hN-|y3tb1%{M^f=is>8~q#*R>5XL1o>5`S-j2vz})AudC{&?0N|L>Jurz z@(XU4=wi0 zIGKb>S6BMbog|d=kT^%tD_QQ{M{I!!77u+$#_GIgLzmAcZPezInagH`wFxbalWe{5 zM)|G+mD_S2VGGj1l$A2?k6Hm=&N7C6ZE$i0yIbzmh3PYVVeR<$;h7lQYeLnfy_I$M z03QX#A!4w51z-{Vl9mq_9DSwvbgmkEvikraPjbOGqz}rTs=#RgT3rl88l*c(kv&FoHt%IckxpWNa=!P(Dd-OqYBH3y9C$)V)Kb%|SZodH&x za7UR2=n0;OJLd(Q04T(dhK6p)$%G09TPbRq$V)R!n~7&z1c=uHUB{=O&lS(!GS zDgPI~KqD_NU#znHEF=uP_#tSgdqk04)xP}&XSOjOzJ`;qBm!~ypM2rFnu(FcpaRjq z>h$49?Caiig?q>*f2#67cx3kp`yUJ-2AR+9uhU|9qu&vwpWe_T)M?{mMh)MRWEmFv zutUA(OPt;#;biKF_7ljEnMfaw^?S|j6d8n5!Pp4}BXo`g zYqXm=Nefq>2`POMO|{Qmr^xf0A_x?jDQU-NQkn0B2&wkpUg`hrHqQd5Ep0-f)e0W1 zBuKx_;dE=JluKE;u-GQusP;5_?#6@fL3V%0jSn`XBUuBxYL&~ytT&RNhID2v|^CYeXU{DklDx5f~J4vm$Zt(eG4+z(OzaG<~(7 z_ulzxIru>)@B#mA?&V)h)fMJ}6*YfEYR%8m)dsz0N!`|Jh${WrgptU?RB0+Fmv1R8 zfv5=L&Hp(uYy{4tVUJ_a2PN2Onfm2ErI{J zH*jr;ZhbR$?%5hU9GrFj`^aMq67%VkRfkPyP{DmV&07RF;t>Yfq*)OxDv62Igye|o z8X-T&5yZh!aPHj}@5u;tO@3|k{J@%r@nmgk(3*U~@s5tp{+WdYb5X#V|5wu*=9;R6 z=2I)V>h60HFYIUStxXY&rTX=`5>eAEZkn%R*E`TYo(Z$v0hK6i2g!+jbw#B_Tj%`8 zBV&}>`QQ_r*%Zpdrc+dgNFkEj``!}bN7!fo`A4TugzXksb^Y9Ox>P?|qq3_RY3=FO z_Ppl}KI`AK7Tl-Q#9&0jwWj7HRtHOVeSz%Y;U>SfF%;Y}7ws;6yt1HYAjNkwY zz0k|e+4eJ`rO#<;I>+D{`4IS9gEcs4#t(y@87?IKovS95{Uy7)C^d>mOG}&5pO%i{C6W7}9cenZ>9_Yx z)cpFVIs;~|Jugv;%PwRx@*c!HSfpp^8}gsNh<7gaJ~#xOqN7U4a9@u2_a4$c_=tIgfC&EOS?+z(;BdD;YSrUJ$$r ze5Tm?ANol5)rlKeR7VAKGTooLx!?UxQis7}t3j7`QZ9>~UXh8i#;7qB?yuE<*20L4 zvPB3kM6-DlLxBJbSrF=ARA~7Q^`buk21RB@mi~{mFlUr{QG^IzL-tASn8CjszpYB8 zB;uZywmBCC&c|Fe@9?pGM-y5#WF4QyvZAf1pNQaV$q-^t27<~zJ|sdNK#28KMz2^s zCgN0NKIAXhZZ9e>MvZ@rXxdMGOIYT}p%?OcQGZL$o!rR;V5z+4~!24-{l)4$15cVBUqf~yHV`v>FdN~!QOoW zEYr&JUb=t&c2XsWld)<1yV^O^h2E-2L=pY>0_47zysIT?=l}QjUaX(`ReP8wNDAw0LM|gpTL4oc<^43Q>T} zWjsC4{s`-$YW(gwkKGO^N-5Nny?CIHtkC>2|sNvcTqe{>zC?!SiT^l(XvQ+w{$H%@#QKYfcjK^vs z!}_2=LtQMmN;*`8p6bW0{rFTurUJeSL7x6m&jD~7(_jQ(J6v)R2cZK7aj`joAmogZ z!*?ACQrpNVrNh>dGS@lnV`yY${0_G-NuM%Y@!qxtV_i@HovZQLYtFhJcN*&q8^+WT zW)>UA2d-{VnJ|c$HbP%!_R_csi3T`BWEv`T95R2jyMk<;MF6T1>b3Rne6ck+_5ROu zc5iuk`4?f)08>)mlm%s0v?Hg4G1WUUF2y7AWGx}|EI#(|TwV9Dp-M%d%eMFIWHcT_ zP=O4wZXMpbq;;^Xovpm-qE+F1&4lq2)tp*YhrXuPg?+o^DMEMMV!iZV@XaA}Q6H|pBP0666V@wP zK_UmjhU)_p$+@!$GUjQ^Hcnsr=*AAZ2QP24te+3X+s&?{)yAWoNP6B1ecKps zBY|Ca-&&JcK#JZs#?}LedB^gpbfyHZ{bgvq{pMDBI_Gsf#Q@6FQIEiFw+hE3DQljZ z`^{0xcqu6`&Z|4v$K)EFH@+_$eiecO)5I7#SB44lpWa{R2q3?41Mg^-!D|eOOj~za zH>omWWfT*u%Qr#6MveYJp>$Sr4l&pijxnOZT4}*u&h?rJMM{nz@*J}^NMjsM{||eP zk$9nzB!^xY-;*7X1XX~pZo~@&?t^em+L1ESixG z$!}tpZrx7YA1i#p0h;v<8KOQ1;fY3dRO}%zJ&kbf=u>%An|{rmR~ zbz0jtp_IYVaQyctiOQ?0&a9@ZDk>hrh>Q_f3E)&63iBxLD_7PSBt_k>t9baYUHm21JX6$kF1 zZd>~i^M`g^dZ~nPL2Yah6F;L5;`=~gmLJtLVT#Ej@DU|Yja!Mv^Q%u-Bck7wERS8o zh#VEIcvh%Hi1Pl7JF8z>3BnIZ*Krl@soy!v6yv9tdYH+`OC!ougPIio(w!* zi51Jx6gt+zhhGRU7QO9Y=kp@}uogCtesN;9l=eF&prDGfSi=p2PsC4Inw0Bg{Rv6< zVaM@+WwdWzvJI*1(BBZ4i-9Wc;jXmfqrUl=_4Pujse^-q^!vruK-DI6UmK&4m#;D? zQH2e2c5ZdFDim&|N%#z>XU_;Y6HttDZUQhQkANM1JS$gX+l}XUwA)njxc3SqnBGhj zBghaqot*|$u-YFkSmR1J5;zCmDgG_19_P3Q_b$}KTEwKJs-B+Zvd?q@-p9$SVJgq? z|G#~#03q4-Rs`mC_=7j&6yeHv<$Z|kaXhUW{Cn-0)NAfUHglxd&3D`%BBE1nzgYk0 zRX9I7Q6sNPnhZTzpw67ugUMz+O)4Uc>lb|Nt}lA!gUBOVy}a9~pv=~TV~O~u0Kg8G5vN8McJO3#hRxW*0;s+qUUCztBqcylCc-O@ZYL)(^ItSvxJ z)>k7)&z*q5iSt&di=v5AXuu8fe}>%Nt>{8ZCt7vtF1+QWb#z(-t&GhR~SBWV4F zKVB}HWm1(U`2M$$2-WiZ9gL>1q)x@Rk(4;p<2?slC5CmkcRDYKfksX{@5L3{=+f6E z5;{*WN53b<-n*ouiuWArM|LZo(|!o=#`VA}{?*5Od*f=1qIj(xFJwG~_nRZ?J(g`3 z&Q`upjQdc*MMo@dV|4Z4sD0XPiSY57DKYq4SX!xU?hu$^754YWzrRk3LplFu&Uq_T zto!|O??>WFFFL+&#Wb_#E38{`u`w}7gutDQtD<%p>yGIFb^-OkJVfaDK&Hx>v8Itk zf#@q^TXAr(m%7qLh2XVCrj5y53&q5T_~b+Yk< zK2q$y9|))eyilsa{q$@>zV!yogd0tIIdX~w*(Z2|N0_arT=M*SN#r>rGes${gT74r z6_y82+^8n_xf7~mLz9IO;gtC6$dsHo^OR81x;2~N#~)PW`MQC3Bu>!WpckZ}bF%)se9vVK!3iPvD!mJXVXhoO=Vfh$eg+ZaUeKfQ5)n1U9l$DsVtfG&7+^H_L^AgN1aWE{1JvMVSFo__U@yQwvM-%dx{Em z{rME`JA%*C)$%F}?|wpWY`Bp6GcahfNTKI@HKeu+9uQ%#k*yy*Lb_Z;Vol@=Q~iT< z%HKNZ(h6`QeD_w(L#f*NNE8$}^Y5ebeZAI0w0Pd#q@L?26+K0v^BS-2^YPW6a>NQM z$jUOYwxk2H-jw(1M=gMXM4GYN%Hj)d^Lfp!P1=JM=unIgH)-Wz{4@?xuu$cdc$)je z!AvL&DgIq<`jNT7wsjlQ`0$meMI92wDGR!J z_(zQxN>QOS;Q$ceIl7n=;fu=XeOzLy>3Sbm3NT>?*T=0Q6uYn6DlNwB2Dv z;J|fSFbjIFauCCoAz%T9Ye49>5w*((DB!3S1@4F5I-~%!2 zKZ%=_I7JvjOS0+r)bg&WfZ@E$I^QrxWKT3J6vb7&LDxuxN`K5*Uj;vn z5pmYMLk{``iyV>2v}|JDp*BoK)=L?TI`2Sz;qU0A-Ot$(68J@rv zR~`9;!h3Y}5Xl#MB4jJgNxU0j^tQ`&{L=6D?jtw6A1p277evF5{pZ$5DVb`4smq~X zuGhjtW>Gx%r2&+-Hm(Ie>(+4LJ$f%m4FcSZxgdfGKxB+h{T!77mJOXESc1=H=(Kzs zga;=ZsMRB4*}z>H*9|x?>Jb-PBM&-85r}3^;rn>~K?G?f61gP@Jt>M4RnJglq&%(mw$aJSuK}Av-?J2kPDGOL#@4&)4kjD- z1!v2Mu@MKG&WByB8Y6P{S3fE38cJUeu73Js&@p+!qvg4owqZiuve)k)FQ&&(lKxTr4&jk$r76g@4_`Jhmyi%xlX!Svi0mm@*Ae zD4qQ{4`;Ne2~MEf)7!fNQk%rTaPGf2*qWP$NcN`9Bw5_H`Sc0b8oU+-R@u7a0E|K> zcP8NiXaP1}9`@6z!2eJMF4@4cpC2mbhu4IUz)=DL@9zTLj>Vsw|L*{`R$E1=6Xd-N z#TU+4xbfc6_?BLze(!f_AbEgarSgpA`C{C!$Me5W?h50vQjL{T^47{uwEGhCeh6Ot zy@m0+$a2uL0)@?cvW`qyna+O*1br!)lOujdre>bkbrgLWvy%W3_79${8akKGyjz~L z?`ZgPDqjlzvI2VO_N=YX9kK-w@!NHvnmAB%`W(hDzt3Qu$59>kkYkjWy@>=S@t?m~ z;!y9F_3qZN(qQC`W~EG zcTfHPUO!-p=*Yu18J>AIa$uWl3C6R@pzr?5vbD*n{XvlB%*3R=Bb8{pOh`sV1d1c{ z>=N1frhZh3B?%7w^;ocw9k&cB_m+fHfA-vA$WI2%P|7x%Ekab|weU2f65^?3S?cB} zq6FpJuH$R9p;!DB=BKX72BWd83x6H!;mz`1N5EhTb8;+Rez;TL|Ahb+@sihNGs=gH zdWBjEm}J(9Un*QSM6X`qoovS~k4*d{bl>FNr(~i;OWm+1Tb}D z+iQJehqPBToyj>fb>`_7KPBR{MIz6x92@SsFrtc!ycdlzooM^_1 z7Hjrd&6``T^ab8xiQeR(x8m@go>~DxK3nl06&<9yV#BoAjC))gzL}Ysr?)rm%9!x7 zvNKkhtZfE7(0O@KbOe~eL7eaY5>c{-frV1W!-f}484NAX<$gy})?ng9LU+L&G$gI< z?U(3HaIi3khK9yvcrRdzbkZvp|A`gKCW8*>>qWs#WNRDei&oAo2V=CAeuhQ0FmkHTZOt zPB&QHp7KpIAcB6j6S+@vmrhk;?EFT`EJ57;dGDfm=d~W2XXp0k3=w+-q5*F}1h@(n zKiOdvcC_lijN!G{1B4sWUm0Be%X`M2O3&S5wQu?*nP4?qCt`*T1AA z8V?Ii8V;%wrh)Sq;DUEA`jgjEn z!D2#gLW~%B+_tr^ezw{PIkBi5lKtOLhKMCF)Lv!;?7W?Q+xsz_m^fl}8Gi(In*%CT2qk!5O<%6nL z-_G_kcD;fyr>}pe?!*uLJwQ$06?~ND{+GXl>c^J|kFCp)jq~-9;BsC)JNDCnONFQ? ztLM8RIdXOGN>*yGLISV-s|(ebk9LWW$Z;CqP%jrBfzt>>o+<%|82a?j&G~@PfiVPl zSrmEqf){PXD(YHkLAx_I$Zyy_6VFQ_uNJXiNy03^ezu4RLqSu}*I%oV+P(0N5K@0K z6;x%(Xp#8!DLFOH$>m=?J~Py&!D4>Rbbf-{I)tQ#229%7j;R{Io-Tn~Mz72WP$X6N z_XT?bZ2bJgTe~N6lMZK<^dG)Xu?S>3{ge3Z5J99W-Em!sQ}WzA_UMKquES{>eeDE} zelTgyf^hy-CU=YcSB62=m5e&*?CWSFUa8I&I{Kq15~)~7S}8;c%l^CbyleG~YLkyg z>6Aiv43_N~n4K0S2i1;Xd~kDfQ^>vLzqn+5e@hk4%ECxVZa_4!JWJ4r0f6-rsXVo? zuvpyQJ~y|)0N9Iw;g(+{D7}%;FRa=Wa7jg8Em0sj%@4TzX9}W8uu^|nr@WSU12cX* zrarll%bX;Txic0#j}O;|s6hPlWFPTALrGqqLD}N!Du#)PiLB32Y18D{eLWWh2!62R z8mH-!Q7MC}ZlM{B0<%F8bT)*(Zc3t*-CeC2MHHkGh_`ZzV$C1ME`#EhSFUDNbKYq} zZO`|bnputpQ!)k1@~Lc|F1kAE?oS3@kC+`Ab6Vkcw*}B0i%-6Wkk``fIiLb=+YuYYP7dycG^y<+TQt80jl8gAU{Zo=8T#X4 zuj{vG$UC8#B1(to6m?WJ7&x6Q&!^Qzb$M9`H``mNcn?t9{)wui9Ks93<2vO5G{IDg zSUo_b(1;?{04Vmyfwy1q9dYV=o5!Ty`|3881YuTsx*7WUnSJ%A=gis})_1?> zfbyUc@TjL-LD^EQYCqJJgIHLZ@#!6Hf4tpr&~xAfMg!89q0;Pf=sHT{8fyVMQqrvip-3p3_4-B;>D31%*>DVV*WMHX2sQ=P#WuKT)se zeS^lZA9v$o=LXELJGW_)7v}c5OtxxjL;x z$nG4qMXQe;H2_@+ortUDR~40LH`$dy)bP3bVePWYE!Ree;L+Dq4ufeJ57w&p-z21^ zVL}&U1g5g4yZWb;q@d59Fs>`Zy|kBHnpH7Z{x|DlWKtoJH$txF;EAm^wUJ1}LHk$B zWNGZrBN-bu5jGVe*(L|2;u%y`{%4?sToDkA| z^mlxf&`X+RP30(l_tz4N)bDmS4XfCPAc-n9tzm&k)Vmu_e~cD}-cPgr4uiHUwYMq6)F&kXT83 zx{Q_;^I*rB7h>I*(ybB|dNg4Em@)n-m9{cjN{|&r+-4Fr#iD2;R{D)$C3KEh=SEVL z2bnneXDC69$q{(3vD!{htAB(ddPA4dI~po?y7ZiDZT*CZcx|}CFnMfS8}#$K42fwf z(Y$@DOS6N72rT-aSEO@WqcN=o@tpBX<~AypnFP_vPv>1bu|U7q8A1_UC5X;z&3Y=0 zhZ0en)+|f&A1RYi1(YIT=?bQWluVe9%RzMBZ>wb>34c|D-@%dH-Rm`x5Ul58284q-D z(=E>XO^g#EAOT>n;YsKNSgNp>E5%B?ikF2S$>e{|hvw{kDVJr19@H4v9vZ4!r5ppte%WDbnJ$X$}0(w?>(dcdWBw`*kx&))5 ztsZl&H0eBkobbU3DI9=XiuX4)wL^LqzJhpld*y3wc1S+x93IvI;^6e&{@yP^$0#y> zdg0I4l@6YBKQnLt6*?K+#B#w=$ZEg_s0ZAHD=pRiVe~ zfJ2rgJF??y?*v;-lyTWQ04-XHw ztBbRAX$w&FUr0F2e&CD521CyI7c<2F>$gxJ;X^am0Xt~FG9*Jf(ibe)8)s0gd86(b zz`_6T7X@s!9R+g|P$*?hcVBSP^(qz{`jC&Hu14cwuSfq?-sRIB{{r>P6y*YHR4t-i?QyHe=6gr1ywAZ_w;6YkxCW3919BL7ugWC^z`T2!>gwtpU@jJf0D7cP6nwl$vD*LAafXD5 zAsusN2E}f0E^X}XDI2B5-RQ6si51Gy5f2C|KX)jp;qaa?oszl+pN$0Txl+TBfOizG z=PD(6o;3AY+brRaKXf);LDX<}psR-*?TD0`(_yOC zXfbCx!`O@Fx;jyeA87niYpj9~nZAKhwhB|L6}4^3qPp+*qLx4Qu)a=)&c0elsy7)E z@cr0SB21!l=$qYM*no2VGJv6pQgZhMQT39%3E@`GNxSj_OG-qciybsC_qNM%(6pop z#yNtkZV?FD2CMuI8nr5(t|szl;XLH$e8w2|Z^1P7nyI>v51>4Bu0=el!JyWij<2Ai zexs)_9#my5L>2XYR>`3j4>~vyD(}e>4fvo(N?liNKTtT{nk2b#0|PnVgQ}}CYUe_+ zrpD^|pCFi)zrVMS3WL5(<{2VBp7^b9Blh?r{|agzTatcu`i?rUY=Fm3-j;Gqk}0K3ib)fB|6t#dpRS z3*{MEFLQ;|-x~)ris8CF$3g-D$6JfsMhE~CcqOSmN0LyqqM|W7|KzT^{V-Ow`u-$gz?tN3VpU?4u&;1O1$G-L2Jcp0!j^TlOH5SR_p%tzou z0NN1~1klEFTfn-IlgW-`YNcnv?xR=P=O&y*aBdcmr)^`tRVM7>Up$hMgjAN)ddF*P z-0_Y3E#4%^Mh#}3v6@F}q9ELb1cd47={a#2ZXiVChJ?$UVD8`cD=y9p%2T;qjMc+@ zfgk1Fvqt8#k??nC0ebO_ zc^+u%KNbI^pQ0_AMKJ36MoP$pit4w!+XgZs#ZuPREJlZGW(H?u!SvE>-z)we z)qktPVsf6vbZ8*Hb^etlOz1Vcc97t6eo#x7uf)J*&|pW<;hw;6iMde6(N(O)to_>R z?m7da=;tRSs@$fC_P!Psn0mM2Vd9>63=pkAEA?Iv<>*2xEQ*W|75>*}N_0+^`HZXtUkuKs>vN`Vj0s=WyyH|-Upyeg39$fTAfW3^wFJ%hrQ z-yY|G-V4|HA!`_OPFc=(iEPc1c_Mbx94)!4dL^-J*t8sG`UxYxZzQDiwI6!@aV1i- z?{Hwc@c4|4*btc{DdyRi{P5tFdf>Kb6Jvb6+NW2OlAqLw+V6R62E(~RAbTvmI=eea zoeI!1p>VCcjFyjCs9ulG?Bn&wbZgdgh zc$8Q9hvbxtChA^ZqR8wdi1fz*8ne419x@D6jg2Xc%S|7a8RA%h?Af$^Ui`B3vwY@z zld2`i3+w?L8v-2iCpH5nnaU;|-G9%BAq1}*IUO5l=z6E?DZedq$1-78^Q5rUL+2o~ z*HR&WZ{qD8OYV{&cQYu# zQMugOhxOb*e@@rKj9qH zshUV(HMoBAKppQ5yCX;6-pNc4(1EFqoV*+f`2m472A*`ND>P7*xLs16h|xQDEbbvz zTM*!cNbw_lmYsvbZKD4CQz)>lZA(l~P`a2Nl;cchQg$eryop=p_22l0*X%lBj{DYb#aj3J?aogW z#oSr)CKQp#Kvz_PpMU4yjfbhVj0jLq*VfFAkyuQ;O*=VUSKQxU^gTj+a|F!~D57Cq zW1Cx;xej4bwhVJD4aVj-F6_(m%xnH2207G{@1Z-tJH;H)-H#ms$v};dM+T!<`eWQp z{HytRf*pC3e>S5%JaubE8k|_wP;Y7mp`rzcp_{d@*r@ZEC^fjV<{7W!B`|@DD&U7; zrj!<>Okng}hdjqGBw6~JQTymN^X+TeLiMDumKIr{bP66Y+mnE9 zh4e2eot@Lu&dwX=bLah$BU7f1j?Z^0RKv|>#fL|&mozytJ!Ot-dcSC~e{+2IdG!mX zsP_cn-EBOF{doEi#whQA+g+(&RK2#|tNs-t1a)=Tr}bF%NWJ`_h_t`C|6EJ;3PfL_ z4L&7SnMTRWd+-Nhk86&_zD~jYwD;X@>l+txj~3)St$8$LO1Snc_rBT(V8MFo&)+)u z5J0Mi=?NQSH(5rW!l-pkKu1rXZ%~GGD1ds$=$+B#^Xw8yM2iw^=8fBhK=V6qL^}ak zgo_T|TO7zc^I=p2L|0-Dz;ZlFQ7BsPW#n4hswGm^KRt9_jBFa@91c zw4erWt0GK*`jE5;LenopLj?T%{A+i%3v^)jYUr=E9RwlrBXnpmx3RNj7G*=%&5;dO zEmQ=hZAfS+)D9;v4^Pj~l61;viCDS0aWgV9YTKG= zN++MWRhwkm!cwJWKxD-+{q&QVqDkp?Cc*6frK%F|5x&5KmeVgJ(~b=@7fbv0ZeCs& zy|muPK9|>`eb)?-n)Q`dsyn={PQ>wBDN3?NNJ$A6xPDO-+El@4k4XaQ7Y<3eAJW%f zU~|jZtUBnm#S#Iaia>zRi`3oC(_MIhQJIaFoGEZV5@`#A;RIfXz~0fO;atlLvXNYQ zH6tVPcnCOP_%b+%my?sj2_AmPFNg&#N4}5Z#|Qs$bqJ3zf5^juGKI8^1xG&MDY?}{ zB}x-FO^lC7p+u!?TOjn2I>vOe()n67Yb#RovGVc6oKSQ^p;R> zlpJ2*v#kZ)%&J!2$gE|)wLvsVYtQua59;u`vl|1SLUZt97Wb?TPr{!7{QT~K=~K&x z#GHSKPhW1YJh~ELu9tQEU2mn1=_^D~Er4@(xdO3^*nDP!J@hOzDG>GB&5qP9ObB;D zC`=wiHtfsqlXI?1fen;y7csm|&@6XV`l2EX5Tj0J%ZO+pf{iEXE;sUs%Cf|81X#_GxHIdD!odNgVhB%0VAbYvf^ik*LS^<&kbd|C2ieDk&oaQ)`k;P7K_QqmEypO$&JmmdPA>3zD# zthr+UuNmyP6#N#EP(m)-Pgbv7-iSU90gYcC47AwT*xrwcQ;-A$Ex^G|A3uV#7tP(> zJ+KuGIs3XTb>Xy}t!1wL<7)2#aXPY(aGGzKc0^lVPz}z1-?3Pfw`chf8P;+zC@>TV z6@(fRS|BRwdYL!;dj^C|LCh*LCwDF7#qjVjaOk&TIv4}nKQdSfqAO0nQ0sm!B-5+$ zGxPpJ2GdlTEJ_8DwaJ=t^4!r7Yr^gE^yR9o(R*ipSGNLg4x zi?+YNFX_L=YlQkLA%PeX1eh)i2A?80p!9HNv?rR>+GtSSVK=HUC#d8fu5N5pgL1;8 zNop`IJXZA3KZWy`?Np)Bw8!WsKBvh{@0_YKHts9M)q~<+8&fPSG2Z7|$m_3kgeAOy0?PK_jfS+CBR@6f!dxjEnO z?4YlIqGPCgn)`j|%sdH0&I8V0o0!lJdixAKxsjXo@?(SJ>!;7)4lu7!wXq;|G%$ z>|y-dJe#3m>)V&oi#{n9B`Cy@Sp>l9Z3`F(!KbeTiflkIt%NhG;jRSP$hR^>EWlQ} z;eU=kAe1m_a$bCY416<0VCyxy+(%joA;J+l!>@up(CPBEqx@4=AL$A8EJMqm7gtzC z)jY>Y@xXZ}p8i#J3$>^>7m#@3qBu||NJ_46<$XQRvyk7LESZg`Y(iu`GV5r@lsq3# z`1~E1WvZptdA){|I@FHW>}r!H+1M24^q{5->ZZrE3?Pt#3e532-g$) zNw6Ovu$QWaX3fAPiTXkjcrhMP&{oLO0hkFoaSTgaUL*~7+@AO}W(K3p>$#gwD&8GE zG_F2Mw=%-EB|(MX82dUBNZ*1-sKQ&`y;N1?g;BiFr+aS6X|2gfl;*(zN~MkzyvigC z%ABO(;4TXoUiT(B&`u7dps8fq>@;c6xt+V*PVy?}U*W@_(Fb&vLU1$h3DYQowC%~a zz8R!54=Hf~vk zYoOe3Cj`U$#`h7Yxwm_6;4=tuoi%<&C78if`G5e}W<>Qy{@82QgYKu>d2gg|I9&(q z?Y7~q$S4>%@%|Xfgm|qmvpO5rfY}coWeCLh)}%+5fBrfXE)kY82|1&H4C#;Q;QP&} zfZ$J0gHJ)`DK3y!K}-rWwJOhy`qFXwdIH_(kQ(g=xuA#e^=-yiWbW12{IrW;WZVlM z-!L>H2P;vNqj0`mgCp2`FI&@(Tw#3z>}kRUGmUCG|9!a5_hvZ&Q;=7Wv~qL9Y|-pr zxbe(!F;t(h>7o^dNFbB=24}v%t{D0aq%=a&ldk&lTTBpOcTNt@oK-5)-ay@ri6l(X zt4Iv0`$VuIJ;wkP@&+ikOP+7JZ~CHvEmrMV74;jTBi%w?@0G3jQRW|PEDp1r_ZEHgrmuH;-sJn2AGvo6VmJ{8?02E%lfLn8vvhBS zSw@C-!S9d&61*Ir*}VfD2U6idNH3iS-oW<338NAT7Je=aG46Ygk-{v>J!}PGCA6}0 z7GHm37Lo|zaFv>VC`#SW6?)nCL)oapXnw#uHzy;MTWl*&GhEoXXKUlsCS}hnG$y%w z@!y;Pf~JBYNxxEzik21+(R97RrTM}aGMg!;%`(7detNckC`c9O=HbQ3OUYRxCT& z5hm-L1#cM%3KJhIt*zF1o?kvqIZZYLW+;gn^2pvOzZt{X}AYFhcXFV|~ z4M#wL$f(Tbx8B1L+|+^85KUqvjSul4HmfVlA~KYBxlO+}$HnW0WX9B`PDIT+C~Nx( zzcc2=yPJPB5XH4wI*Z^m9REn+q^ylf^R2@6Y2f3hK1KF+g^nuQ`$+~9zjijPY%lC% znc;3)9Xus>KN)4_FnLtyX<@((87$@NywQ=mI=5Fn((DD|mlywR#tmol`~ulRHzj~9 zajK!9XIlo9g{0r9U`tDjUaQYjWZdM~*qFn35e|*GQzYcYEy9K(WARRy->>StVAm{2 z=C~{cWDJkw6QIw}r zyDx7|7WVRq!e*nw8@;{Nb0r^HQl-!vL7D+QkABgD+#i$T6OmHfv{m9grf+8WT-I`7 zK7$oK_Bg$rCUhv#(*!V6>s{zWTaLzUQr*Rf5S(>FQH<#*n?u<>LGrY?N2%FFZ1kpE zcd;8NgA4>hK;|~OL@ZgNL_*4@h-;2o>1DzsfbVx)=Ha<@Q}2&w0zZyl@x#Tr6;6Gm zO)h{|DJIld8qfxd%T;lzeZkMmukJ4{1T9NqhXlcrh-y}~73@oJb9o8UF!Naww%O}$ zf4i_BL5($&!bKp$!w^zZP}stcF8>m%#v;I3$s|Pbd9GRE*Ny=-b;8LXjJY|c*bkCo zdYZUJa=smnPB?8^lu|JrAiDl{jP}VLO-a!z@&VB%xT(=F`&Kd8?;wqxNKJ#-ijzi4 z0^-?fbe3b_S1Qmv1cMHB)pcAeQ#|;|EQQ4{dv$$u8GjnLXGkG;P-fW_w~$ake(e&dCyN?drS$b z7y#!FN-L{|+;7AG$&tG_ywd+#uyclMhsO< ztx_h@WVIn$2%<%1l;N+yn*`E9H^)y5+5bv^{TcdF$M#CM-RpCjnK{_fuyZ}8eDR-} zzhnA=g7Q~c)OVIp?fcgLJ~~#)yS~D6Dy*!za*(^lZOmE5MK_@1U(J!o``dSvIb(w2 zl&F#P;JmpIFT$tdZqt5>noB%|2*r>di#Oxc9#Z3ys3+rgNCMyW?N3{Ypec0y%O7Ju( zqGQt!vIz##j$S4gsu1ZX5tC#y(MBD{eSmmqZw6xg=ad{xA0=6NdGVFeRnf0ItRhv> zvhX$&vy-u)59MhF3v1s%e`sb5nRku^?RNrIR(w^mIg6~sl$0|MT49o*NZ~Qj znKDaC$bKK39zp9RU19btM-?k)eQ{;ODosln9^p1}reAU@Yb=$Z${*X(`)nvqhx`m> zM_4&@@v?PZ8Qk)qKxbrRgjef*@KIPzN{F%IsTMli48_Kdk5p`kwyh8*cBuC$va>Oh%0|>GT2_3h z6TZq+CLtNaDZSeeGC64Is7evbKE{5iq^%?jZ>i#sKr$x_ci?=uMB=!Jrgd#37OmE#PsPl$uF2q3RTSQC}e-~qX0gg+-pd#>B? z)XfKwb)oPrs7jP>JQRi4oDMZM10({>qTB{Ek!$xc41=Q zX%9cLDYDQEYHSfOT=Qp|C0cPrU7sf9T_!v=*afRHx@J3pW;@p)zie+%^P;2-C@qnH zpyHfvcsP5m;ByN@sZI`b$wiA*Orr8ub*kfs8jFJ`=(cic^aKdKt-E>$DjhX+$RzF5 z1g#AHffvB-K#oSp(AjAa#@ONtsvhY-n!0Zbd#8LAi@Y_&eD*kB$0bjzoL0-|JtKFNd$ zJ>vZ>OtnpH5YD4re9y4h*3B{xganh}co@ z;J=S*@`1NWGZkc(3ZT-gyuW&=n+4fSsJIL+!O|%|0edfRCbj8@>_)6=m}>Y1X^VMl+-rg4kMVT-#E@d zhsm7{S%o;yPsiTRtL6L#_BTYjQ5LGcg;ECT_!g`(>^00C6%!?RiQ;dBP!T1FtSXZRWZP@(=Ov7zn^XBE=!k7F;z|-N*Xg$3;a&dN2ZTa&bjPLiC-d z1~IzkojZJXL*!T3u*BnNzn75=@y@(L+8LlBh{!%`(je#3Ev%Cg(0yN+w@39}{7yio z2KkV45}Q7>gtm68=6@$Qxg=R$)_NQ`+4SCgYoKn!E7bGVS*n=^m+%#Hg^nw-e8pIk z8boj8m*nj^@W}D{Cs)i+JtAkwR*-r@Lo0`>RM2N7dSgqAJU2zNzl>VkpDbyDQ&xKZ z9iMJtAwU|LZOt#tapPj^9LZFo03FdNkud!`>A+PUX%Qt99ebzfE6bCTG2o1OXSQV0 zg;dAfHOT26w&s{;=Y)JNaQ2$mBp@fJxMQ8zFtK$+ZmRrY%pqwOxBn)Krp}P}W()G! zZpsb58?qABcS}4kwc<~#i86MM6^f43$jh?Weo`z!<@`s5r68qV8~0Vs1DRxvB1*+y z;?gCycC+Q4!NcwvQwxs^ILBfZ)!!ep4YV8UPkm2~jZUcm4EDSiwwuL{*0{$=ZpFd)4c=Bx|q|C1W->Z`LyPw@!eB} zMY=^2^WLGuIdWP)t@CRZfF_3r@&I|B4~E0BFdQ<6#g1T@Q9#+Ac z_AI&R2O6ZM@@=SFclcSG>1z0iXP7>aOyr8_vQM_5M!}0*#Wd@(o8u62z^Un)U<-Fm zIFF(sWh7t~CO`9^=uY;oqO3W($EMhhzfw!~6LT=0&B$?{CX8hOm zfw4tRx0QReNnO2IlDyyx;rp(-yyq%obs9cGai;(7kY28tnkNq@o_e9ygpqYE>6)%3 z9l!3%bS2YId~T+%^;U*4@#HMsXnZ1`(En4~*0*=;uP&pBRT3e}u++6s^#HAO6%myw zp4jv-^sY=yBvW!#RRvQ;T31yYOIn)h(oBPLMn+diX#eSENmo)uZj?rn`dm{Y==q{z zRiF5$@!#B^b>+LiJIB14Zs~NC5O9(BI+sxci-<6g?nB&<<88M4 z%}P7nc%+TabyJ8c5EKG4l}4 zc&p1X@U=b?8W6eJrf~^?uDSz?*Kw zgdk^R)J;i#^y#EKj5FE@;42zv7w?uCbg!8iLUn{e_VWFCu1P$V!!9Y`eaXC%7N@~W z${S4M&2l~;J-QSC>@D5X`r5NegB!p*)5pT18I`>8RY&fzC-r61UHi0*^dX1eE6_8} z{5#+3b(Ws_WTdtK>)`owkcAJOr%+RW*j2y20jHOmS_8SpEsjnq337z{$Ho$oIj6{= zVOYUo2+4RGzZ)0c{)Oe@uryf{@U~(Y_Vv1>ZgnJA?Ac~dvFNSgs>5OmTfIRyf_J6 z7TWG0B9|pSo3I|ir{rCxFLQ1{(K&z&l)a| z`Y4v)vQ3+=SHC+t)!}y!UrJ7f^Q4-qU{YXdMg9a^G&vo~ZKzkCnX2zu;e;m@DP;yF zIaLq!)y|VoI3APJP-~T`#1<$kmu;GnQ`~U1^&Xk*!#wOF-)59(ows`AHVH+Fn8x-{ zCOI+n(4f#uS-T~U4E2$+;={2bf!ec-*GW1xQ7Yo2QJh?Sp)8NcD(G(3OUs=_s7{w> zcRCaVjdIdb=O@;rm~?f#=Mvnbm|wJ29^3CvNy>RV`Te!PcYqx_IN5Jyd}diL$DfyO z4MtTeEU&zg=tM_^8^hk(=`)X6nD_u)&D1|4rU3b@#^0kNp+X)O*NS+0aB&a&BrBw_ z;l@Ya%~s*-R}ZUDl$VlAPKusGW7jVV61->sYR7`06+4mZY-_8U1OkjLNV27IU^(yA<>f6GFW+L9aW-^yI`}K?w3;(#;AUf z$w&JyLw#vtzsn1!Yl)CV>)$lXw*;M~!~U*u*W^<_?wY1AT?YL9!?-V52tOCZciHFm z!*bba<+o0xxv>MeeiTt7?3CHe8V|SCqvh1C^IWgYFO&^79H*HAt4f-4H+^KRGl-l= z0);|N$E8Sj|8gX)ELf$TMRt8+B-3gJ!m`tAADMpLmvT=zR{3q_ZUT92Y?|&9t?YZL zMelvF^uIX9ca6LJC_l`FC0-Df%ml5+dwcWEqsPL051c9t?ASBxzE%s(kUe69-1OqN zNss$UrPodDR4n|>MBsM*vtZ9-I9^zZ+OG;CH8$}j%LwJsiq-N17DV~u7 z{A|t491&eVKjBeCgR+`~B766nqhP}0EhFoinzX}5kNP2{K%+#*OjlAOy{qD$?S0&# z3^^a=rHivJTxxMsr}EU$?)c3K90c;tG__*N03*1aTucna|0R}`aJ=yG83&UxFx}yS zT-SprL6Lpel~KdH6BCdba8p&&PCg%EsR=7b4YaG()uy)8s~TGDozIoD1YBDEs-!qV z$2PNlgsG!+oTd`l)ozWH+SdoXdx`qTEcfphWBT8(OY&b6kL@6!=bJVWJPc%j1tDIm z12?w7LI7W8(1$%}sg~@IDo{suR}fz21A@1kluwCS&z^6B0PY)Lr0?|dQIXlcuZ13Q zN=lxZIHTiUGr+EpQb?g;!$w7JJp$qyB@cWH21-h(H*en5H!vVwU0X}yj84hSd~#Lt z1DV=;CBNY%t*NOA0o6>P!DBq$UC4#sbsr)#?}&TV=B=^P0Zg6tYS z2)Ot_lk0`wh`cd}uRRv>^6lv-uuO11;N01>y?_t_sjq^x(T}X1B>JzhLGHvJzpP$i zOxEkZCggj(rD9%j(ui?^5es(!ViYL67~6{_Cg?3e@!AJyK+pLh1fe1?DM&qC+djNH z-@WEqCkp>g)I&8{aWCCvE89hlgJ$R9<_rad@hvKQ`S=bAT6Bze!0!ue+%ddkqz=U@6Hd_DosXJUckaK%{(l z3`!_T_TD(>X%GvhUWJZs=dTZ#J3Di-nUTIM1$|A*nmkKK3T*aFoLl$+lWHW^d>}-p zo^DQglc<8jmF8>dDmV&lTV`8qe-CUJrd%2Alg#AB^Jxvfh$-hH(Fb_*CfmKhVYaBm zK=qW#7HX1iHa0_xg5#H;#%x&H^RR_8oT{EEIFLV%v0)-siORuvX(r_>*aN$Qde~Xy zh zDq8Y3dYFo#OXAbvf&OAjMUoGP(*h6wh#J)m4sVyB3Iy%@F@-wk1XyN|Cr}?aY!Csm!Mlj2CGu&#^y-ttEd4 zJqmhM+^ZS7RRLtf=n-zZC6>e@PD$#>vOlcWB|7FI>i>^EDWsj9>kntmw5&?G`+8hXl495+VV z-m1;^?2&`VPTn`T?^$$fq9^gxJH_dOXu;HBzrjY<^6qe+Rk-rfaw40`ZzjmEcz3Tm zN+(NF(L}eR1Qx7J$DUmW3#me`4I9FamEhCU7;QXC^XH-&eWC%f`eXP`vHhy&a~4uG zw`A^#_nrSsIwCEsJ~|w+p%ITQtsaXIN@}~#(Kak0U6ub-d*nqb;lc((a!mvU2Ob5D zCe%(g=%#Aa4*qG$Uswrs#S}BJ?OwTvC`i~TAtfr<|mtx#-B=0 zb+FC@f%aqPCAsBU)X0a%!>`94-RwK+rex#4XIJO4^!83|eE4}uablC7l*ZrPdtU0g z9PXP2gA3EsMq`tykf^x1-Jjeq>ilNNhk|3dC$13Luz@Ldb#>a#+)&T^#zh($f<)hi zqEIfT{^;UuqR#KpOtN5yL#p5Bnp!W+V#Rd^Swn{pD(h!Ww4MhFCk@5iAlW>ix~#sd z3#sQ$q4A<;83?v0havkH@zSh zP$2ZECm|*V4>UwEz)-#0`Fj-ukDZd5ssg%!Z}pE2A@(0)xS-f36c(~h&(5}s((YAV zz6T--*~Wr}OA+=ZrDSK{cRHaBZ^vwXpbI;zy!I9qx5f&wz~%J{ETVT{{e@AR--|8C zDUPEO3W3V61p7==l9O+%7Q!^nlC7i0V_cpG`Rt>=zC=8Rmi1eSF5|zA!dDL9^oJdF zw=}i1471{{!Z(h6kOdo})yJUtm%@O5^yky=_@~`AEmqaoJR0^|#W;QNHuR=MVnAIAY`;Djw5M?RPB{IpY`#K1=U6gX-- zXg~c)h=AO0Rz#8U`+hx}qe!0h%^$xxT{4 zl6{dh?{|3TFnt3o-Tkw(=azqq|GBOJorqk>0q^%_dk3+m=AA~Nayz^ABulrjIOc2^ znAsBxmPm(sg@x-Z9ar#UqU)kAzHbtwyiPh9=ky63Q-{IcIB`L>FsfiViBp`pTeZ#7 z4V@Zy6NY~-HkOm(M-*{O>@1q|TYLK32;Gv`-cMZpG?F-(LNSYbno?^%~IWu`ha_Yde>}dxr|0k# zLxiNzDHt_4-bJpD317EI8{Izs*6_pxxkmt$yE9o;NUgi8^-@=qC5#S?TCqp+Pj!5dlrr%tk_jFo)d!+<>g={{~Bt`bO zH@^KOw<3Bua*Ngato`1MBjPq;)Z6xkR6m{Ceo-kZlwGl0x*?HS zl#t36UQy_*nf=o5bbi1`=m^qE)Jmhe7x5DuUdf>yOp)HTOxlRFF+SXh&M z&?y=J^+g$$RrJQO4h*$R7+0`>rXjQK>&UCvP^^0@LuCCVmpz@K6~cIt_dXE&Ud>6r zd61+InGQ*n55#DPr(-DD?l)Q8a-k8^jGwdo5UXC2tc68R6&=>Wm*Yb_u|{^{P#5>FUWY^P=_$; z$@iqA!eco=2dtzP9Ew~&dgarIIqHG@YU9~LTiS~jOhEP$%q!l)Dhz!n6aV$QLU3D) z6CyUFKAiih%8ey4H>V``JIqqj45+7_tFw%CrkY!mjuGASBBy14Gy9i+a{SC+;JMP& znvisjh8k3FkGlCCbcI-l?RA=Hf`%*Ew>LG7dg^dcz+FEp7a7R&cX#3cZ>j)Jc3S!N z%P?3DH{aSnIXOvZ)ay<+S!Ni4NyMNHkY*vM#>W8}K-RDS!@8mNhDjlarIl8eoa(OF!v1wbAItHzbq8A|edIlMFzu zhnpJ{F!x*Fe);|L)At^dKe_6P375ZT8~<+$Uk#t3=f7*qWuVt@BH)Sw!zO~6-B z?T3r{^#jPkyn`r(@IK`f#{RWkT}S{I4UF3PQM0T&cr9|YH3@r8$_mO>AKYfsC;;XC z80_0}fEF-tMnqOXV8l5ndQ&5^Gk%!xFT z`UE7$%b)yuaT=j&;OCLpCzMo2fKF-lDnb2TZ+v-{1@5bOj_f;|2r1I`jeouDY_m2m znOJb+tM9~AA#W$)sWtP~uhU-~Na3bR1A$=04C^B@E~%k^^P@1VP|&+Bhgk{I*xCoP z)IK6hvfL6RC8d6{+yR6lr=)BTyRWLb>Xg=?-YJIKkMcxgSMSFuf!_7b{n4n@D!Vcj zI@bnRA*D@hnj`f&xt!7UE8D9pTU(mj->W!S&T^VT%~oc_ZEh;Mq=&@SEW02@xC{tL z8$qhb^%p+C;kv(9R#t}NcN2j`_IS6sYJP-@`)gA=5>NE)D)DY|Bw3%3Xs z@orN`>GmXeHKu;&GDT6leHY>87r z8FI9(n>#&3+qragu5$s(UH`Bf!F$ZD@oQVd@2D*61=Bux4coK>&#g>l-t*?C*I5d% z8jpFRL1M@ZrK@BaCWokiy2#bO_-pG9)T9U6tSs9LhG~?FOJ5Y(=C6YKY+i`p$c<&@ z`qk^V9C#)2vXpgVMYnOrd$i}?I^qDs2K{>~tdFN+TOf01BXD>|yrW8;1^1C~O4kk{{ z|Cv+&gB`E^0B9rswAOyq6fM0l zukqp%J33+K`abpIgXDC>6G}0M(I9XIa7oWhp_l+pOhQZhWEIgtg@iYx>jtMH?L~>{ z&E6MwsYg}P2_-DhPxtGHl3X_3*PVVG3vToAI7WiCE3EYGjg5cluH)-Ag3`4*YV53I zoAf>dL$!X-cP!r)FZNF9_(7HC$RXogzd8KpTD-QdZZnkV|B!UufmrunTT+?`rHqul z$;`+e*?Z6I&24YtL3VbwkUg?Dk&!JkJA3b)^`86p{`cgGZr|}a=Njiayl`}y0;bIR ztv0L32;?d39jjt41YR}-CTgh|8F#(!n%@xvUnn`8{|JE|W|O z_ijGG4iqIYq9$@X7f(d?j*rJXZB2-Eby4GDV++ETCk)uLH<XYFuy<+@!fGUta0U*SEAt~e zDXB-&sXPh1|JF!hH{HDC@JPeytUKCAAk{B(R+YcN>NxY6I<{Cw-BieL5`prW0hu-O zZgp0ZmLwT)jRs*#)D%4E6|jFK>DSYa=R{l%pCEKo2&TqWbd%w!x)^qQ=3?e~JgN$z z=}h@bdDvie?CW;l1*^;m?9@T!2Mniz`SlJN8QK3VH%3EIth`|E~e0Bbp zN`*6Bh=!1dF!p;Wj@y%AsDiEHHcWDcg~zuwz?FALtnJTp)*!>%RWdf|zy6>_kp(FX zu|P2y>#%+C?5SAZi!sa_jiS^AY(-u3U$JCCBnHdBHnhT)>)C+oFS9?SVw=%WvDANH zgzM%yj^xLp^^~JNzMuVX4>PFZ4VB6AQrC-H!M(CM9CEh*Rk8@lq{e|TVEL#vgG%1e z5_SFPZo#L$!C!ykjiq64R29PGS66wMIi5(6Z)>JiH#WG*saToEBsf4F1w@pj@SyZQ zbxdTeQ0e`+dx5@mw?tC7fTns9y*D&;7p#TIlJ_)>_(hT)i;jxSOEA4m33o_hXQs!G z)XKh>VB`LbOTop^wbBHduP>>-<)bMPNja>p%_l$${uW^-6K*G;$Qm3cSYpQA?=n+4 z^=UD6GwbmCZ^?n6Rv4*wuw$SBGYE3ARkBK%Jsa*bqwV)3COV(KtCOb8f|H;s90!a? z>=q^rGwJ31{OQ94RBPW77>6v2zmka`k-T>j+&etXgo(u=u(9-Z{%mc{HSCH)LaKHa zU1Sjr(kY(*2nlgdc=k==^IsaE^u@^>EmXz|N>Czh2>N+P+UlOEyu#8WSH= z&%d$(N6#FL+;C z#ir^?oJFb~43N!U{0D~w_x&s_^8NW&BwhbF{MHrUd%Q_$%fb2dXua*A_5D#2eNWJ| zDBTQKu+Q>cP){2O&(n0f`8}9E{`xGs5R$Oci>C00i+RRn%Ber#5$vcX+i&3~Nm)*i zyyQD+;%W#tzj8})8DgD%ugF2AQiw}+MAFV^-A=@ywIzsf4jx{h$JNM1eY&BuF_N1V z)`N&Ao73g?S!H?W-_JU4a^lkHk{J2GiSdt^x&rY?ToV)bi)EtL#FqLjz`_f$}vsX8n64wgCHw1D)QcoFq#FF_>iH~ImOv>vGYRh zuD9z+HdGCJf|m%XwBmb&8j}(a@f560D#vveKQHKeT^)}j5YB8G<<31l zJ@s5MXxA%iR4AtHm_7Q*XG)X?} z{~1KQivx{f1>EY`3A5-_V}~^Ye!t$px^~R8>1+An&mG=Bi{v%Y7QrxgRO^avp**&qfqwOLKGIFUMOq z?04`je#SYwX@h^V;~_)M$H13vRQ7guXIte1%=GkOn$^2Y&LOIM86gb%*47NVuFFsR zHEUwn0}rls#|PUc-HOs3e;=@2A|gmo;>m(QZCq8=^=v=v=Z=7xz%BLJkep;EuGRjV zaLuZ#_hORFe;~`i1y%vuLojH=u1~4Q_4z6?CpR3QZG&;$4KS;BvHEanug>b(DC)=c z;hX|;6fd*=Q&=TMj`GrBP$0$Yx|_BH3k8yT&UEW9-u|yr{vfcfdA(D9GO>bEQ-Y)1 z(5^rK!X)|Y-D3p^!SOoxfJE2=9Y7hwaM5$mk6pUmBR961fy0324wm7gq!T_Em3p_mD?3 zJZ_`(IM#kGKU)0IJHy(4_}q4d)veAN&UDy8dvTi9K{(v7S1)vc=P)z?w{*gi@SPhg zBP0=}H8eqLLvAol#&4>%*b|_jr0i@Toam_<-(3C{uY*5+ z{ly@hGu@nY#`fpqM9V?SR2SdsSvQOEJ25{7^w-Ol>FDUlP*PpTCJJ-&GhYR>suShk zsiEsBv|uj%tru3VL>Vh}qemrWWB<`d?+ZqCJlkW@$8TmQ1H1D3qATguuXSjQwi0F}-_%vQtxEeJV6#YI`zzw>+iq`c+b>N-({!eSsvuNd`vw=tLgX&&5Mp_M7eEJLP>{ z9Ty!be^>F+tRrRyjv$>)PmL2~E5%5TdRfY{D#E)*9h3 z^M52wp_#mwQ8^+k6!;cRzz{w_@ugqSwt;qZP*hgf#Lm&+SPOy#TY3667xvayTe4yv zU$U;k{NYZ{cWwH5B?CQ)_7^|)~;Zj7QPCmH^^ypnwd3TCajvG5D#9<^3)7z zZ-xBZ{R7%!Q|ByG!3}LC&g(`Mt|v5MT`;oD*7D5FEWS$HAlY0 zFWxo53*nl$%sP9tR|?sd+QNC(4cSG6mP*qPs^+G#(+Uag4|#)Q8ReJ@;@5y_;^Si9a5=r6oIm5l#IfTmC`f z+mpoWEoBwqfx1m`YKiKpKCr;@g{tAh;d*P&--)cY!JWfjRVS$DOFXz4&h|Sc0z5=XRi$>hTdvbR--cbAc9ZfatUYDD=qR~JI7p>mb}1z zW`vT-t^LD+r$fB>-_E)yW9XlCi;{j*FQ=0E!e2`y0DI6R=V#p)|GR$>M~_SF9%>i1 z(DrI~@>|Wr&oKl9)Y3^~D~5la6kXO{hp7n^7iaG+gv4;st#rl4#yn$GDZA7>ovsa2|fCot{I#@KL$nU;1_)r68_kS({QM3v>VAU~g#rwty^UKo)t zujQPr-H+gBdcFix^_-9p1v5=C02*I1)-GYMEHCfR2NLtX`mgrJ!KG1$ckjpT-oCzN zP(v2aaO!9Joo8JGJBgeLS(uq!P9{tae`4}2fezzUmV7oKfQ5Dq^*R}rI@#3B-;Ill zE7NWE;yInR=Y%W|wl1rYT=^ij{M*j~X&Hk1Iyx%q46yI8sZw`g!Ot_gJ}~=CI&=$#!bhp#;s;M*fy&j>e9maELUUJb?t|i7?HDJQax2aI*Elk7zYY%&VFXI3+>A+;;pw>epq%Y`-_#Ee zh{!8#_G1#TO9>@|i59E`s_#;J80tg8TCZ@E$}U~159Z(HBzBC zvOJ}VEiHgxza%8Q5W`(vUA4D!zi4o(0Ys@>f9by47!Rm(nR~Y(^Cb>Sv3hv`LwpEa z>SWO$ws55jvfMX&&cLQtJaG!nFt7N)9J(I-&0LF;#wtYx9yYevo9ol&MIBah@dd|m z?VKv^`TuZSmSa`(4S3<|M_KtsjJ4UNjp%QyP} zX$1cTBbrLJ^9nW-1bCy$1ziCMl(F*eB9FHn$`Vh0iy-L?eUTY;3d``QcI$U}E49Jb zhRL~vb-Wu+j^o?%D)2EuwwD*M8hORoC>T_f5FD+4tmVe@@??ED(9M%4A%TC~00?vQ zzNAIjw~?dA(lKNS&jKYYedEazr0La%W>?PIC9B6}BT5p==*nC4zD}*0Nc+7GzqDc1 zSo}5io=hToR6J>z$mE?Y4cqpW4;B=d%Cx(F{=QTSHf3gLw~MMaiavFqK~0lz7a)){ zO5qLnz!1rpRvpd_emR+$-I8lY94>sm*fc4_vT|%(FG!MPY0mDfO{sE_B}lV7saueT zIk&+&_(WQ?Kosq_TPWX7!RUxPKu1SuiVbeVECRX*=Qd?rO3WM-RILI1`sJK6Q1Cj! z#Fz*_Wd6>7LQ=!iS7^m*)^GLddB&_^=5K7-YmVo=h-Zf_a$jN{|H-u9B@%a5;G`t# zLR42Lj2>&W_|o;TYHSsbC$TMNaZITVr#t3;x{pc__Kbb4a;cfD%aVEGkKT9F_io;= z+BPu8xh6CY_-WZc-4_f&p_on3lU07_w_WVL>{Z&9f+w#)h{+XfJKbcw(YKF*Mo~ao zJnbVT5<0a14>N$I+r`25l_`_j!Ut|AvAoE0wjn`5q&t>`yrGnI~Mz6DcG{;Tp3n`jd3nzWJK2&Bh# zo|W0}tba;yo$I%dT8ewhCstc4NZ?B+Kkelg6-2C5_BLw+5p$a@CS^`>`r><)=g8NJ zn|N-08a#X?-=E`)(+#Qo6!uu)CR5A$}QHy0$eSg9@qKG6J1AgM_ctqNv>x(+QrQ$h` zNi>tCF|4aF3=+Jq#`!MEVl4M>9z7U`B3BR%abnqB?@YUNvzn$jw)r&vBVrL|pQLIt zw(CA`Zcy>;p)6ytAD6Z8LtjBv*+wz79lVJmKDpg}t$`Ow`XrdKA$>jG2gQFiHIvo3 z_~c89OmIoF%zjZdt8g=`K>&%AsdGusvycVT!AavCttR``6uO@N?9GFog!q)cTB3_x z{ZARcgfQR5m?R_a!fX4v7=zNs8^ucog+GcnzE?K&p55x?b8d}YWh=ZI4~_p`*0*sI zi;fI%c;GGnJD0Ds!fw9ee|;n(@Z^@1H)*dLN>H#>FHwBiI_VRwNka%sDxLSWt*TD}r3mkJ+^EvjJmKbf(1Pp1t;`aK5 z993zwQ8qGRWn}^jow{N#aNftOuC&=KcHOLe?7bV^)_pf9%>!q-#(C8a7UDvh?FtB9 zhk=C7sLjNXAl+V8#?1C&7fRArnHrkFUqED>c1!Q!fR0BJk&Ja~Bylk@$SqIfd))Ug zT_KR=HuCV&vFjp77=3~9=x%^{-wO*vS5w^?K$5#IPwuk^S1vz zj4K{Gs`MP!VbH3o8@{5`ZA=|*J`b-yq3K77WpljrTEx_ zzsUFho${SAVREg#0;AjZ(YXg?L2r%F=&*`y+gG>zp((k|%5y4Wn^uH^DinZow!nfIY7a3FL7{uOM+Q zHcKCUPU6ez%8DI`iyJ3Mtc#*_@foZn1)c62c{%xfIz~%eK{qds6s;#Cv;51>2^S;{DMb2~IROgdFe`GEUi|h_ol5EBdzk8oA$Mc!F-T?B= z->;6^ETb!!g7yP9Bdb*Y@aHUym@sHmP!;zI&|qmwAorp0bHE>pI`*K*Yo=IUo;KfX zA6=U|euk_Kd2if10)NFRQ;y0{httA<;NL$Qo}y!KK3jO0>NIJl{ahZE&mRbLCQ?$r zPD1-fk`j{Y-q`xl~_jUGrO4%Q##S;(H2{7-#5>2)YPX$6CY;$niR!Jl2Q?! zGGgc{>lO7QFK+T+rTCt&LoBuyZJ602Xi^__=eKF-Q?{^_4omyG7mxfWwlIfQRe3qS zSXFQ>=vtGDuUt=3gym+i)z_1LwPb>dV>)7(06#i7Q8dg}(44c$tuaBNevYPmMkI~z zeZfpSzZ_@Bmsj7jTlh68oh~rq<13{*YW7XZe@%rvZ0Pe+;V|`=|7sg^x4cnNVbavG z%_zT9AN6#!L$lSt^~o4}M>Or@yOPt#Ylqb_)@yq>R;AoX@Yz{pY2O}ZFNFnvF#nIB zh)JD@^@~;F{ynjbY27K@RgEVP#MIPtv+VSLyA}Mp&LKN|F z&$Q}};D<5R=ZmS*PVqUzWoTid7FwJEYEpk^Hv(!5-e@fJym=qhwsH6N&q%5(FFvs_ z{BJdj74-w*TpLug_5<&Nx}1@4uKFi5Reylw2+sHgz1$XN-sZLD-sRyyfqiD);%;`$ zszBfB0;a0MG$M&2TGuB_?XL}2T4)6i3kiEL->gvHKgaH!s`kedjS2egnYgDsBUOFX zfeW)@u~DeoT{mAM7W${Cm77H;D%FENY_i4B=w?}(+2>oV`z!Qm;&SK;*Q!71mJE0i z6Uq{E?d)Lt1m`DUMJcF^N(VYlR#{3H3`U!R5^n7xvQEW0-&b5D{<%a}D7_0`$Uda5 zudZsBJAL`>A13uqQ$)-Z=YdkAgxO6w!t+;)=mQJc3`^Y>HM$PxS+^7sS2Ig3mjn&= zSA-blKx}E%vCeY-Jr(}&;WY~j%lSGwvD{iO#`t{tS?fdGsnH4(GQs(iDQZ;}^Qi}@ z?ap)GMy#KNX9*_rojhN@x)_x%>l58yj(>X4tQ{r=`kq9ADs941kre?x6-Uv8-#n3O`B1o^Ks7FraS_+wl_8;ySd) zbq4}V>Vm<<#rJpCrz3)R0Ps8PeZNj-F_BRXjQ+ugLUae^R*B$PwHTd**FULGArxX% z4u3p2FlogNX#y`f#jDN32HdmtjEw10upKJ1DYm2ArnJzPW7{TjAhW_aJ4{F8iy=a?E ztz*Am^zIAZ^;*0M5n-5zeR)q9w8l5E#QU|$Z7;ln2E4I9Pzj3Ov1<*bx=pXRa`m`7 zStIIKbyH|r3mM{|;W?fSmYT)qv#}UsKwEJ@U3mbRIf7P*hCwF<3e}4jFL-up7TGsfi;FjJHU zQsGyggI}cQlNKev%`2~^7@PSl*$H&rx7l9W_r8HjYiw>#1H|59vuAC5^8+&R1@d^U zR!BRw1%$I*)uUDA_70CuyB*l@M9O4a)Kld#+bNm$SSef)2AxtBMViX8x^Uh8M1s@n z?PE6k*^Fha29L+m#>@X$)Gf6dy;$IQU0Akw9@hRRm&0ystK@w5kWtUfj5>ke(D<2Mi|?IBfj=8g>IlBX$9GOy*N0I=$F08oYzqbDpsa`CLfOi@66GVA z^X_a>m)kY9LUk-*<3@e~*HuF!%HQhQ7~lit49=;OFxBIfr#4jt9anl2Hclab-vv~- zLnH|f{O$rFibj!5cTW!+BaKsi%4)JbMvly=@qI$qrAV}N-cMQ#@sq0}c-mR739aka zDUQp52-PnCP8`xcGQdTvJBSSEHxFM15WfhEC-b95iAHNw3Sv87y3+yfWzF~xRJe}F z*UitI>b18w!3LvTEt|rnTsb@N>(`otJQ>!#Tt`g}mIGaneG%o6!F~J#7oG+nrP&sI zAr#6EW|I(%RMj(1Nt4+b85xVkU@KJ?XoImNWfOlOFnEiLIv3?5mdnBSHeuZU$Gl|p zg@yZgmweROlWX7nhtV~?|2c@o0RmasI?woJkcFr`(wbZY&UZ30O=H{8n z4nyMjPgPOT?;;m9Bl&?~+G0H!83qdi+yfr02YD}k7A-Y*uIBvQ4%;Ms{-C^+oP`@V ze9+R~&fCgBOWUqs=Be0qNJg7zX6(x}zW$-6N}Hr%q4*qia`_OJGYlE=u@I)MWxBoMiJ@NNQEJ0_pJ;LPD!5{}PPcUoW2!{MT=*&SI40C)^ac&LxwUlu zT?+iW{W|vSsqi;pev{Hl<3ne%KSt`tj(!5s;&X)e>~zE8Y9c!Hp(IOGd7widHr46Y z$lg;}i6$V{rjJ^%@t6olaLv}$tjuij1YKxaQJvVt%*L_Ctm4o5LB+ew6l-0rm1e47 z)-T%?obM2eHdXbeWn8RTu4MR&#aE7qPRe^EwA`F=cQgHgvF+e`D?h@HZ&qnhLY_1{ zlV{tiSJfUn@JZNUT2r~?iTmeZ3&BqJaHSoQa_Ol+Dv9Yz#-OcLrp?7 zjg_%C9PMGN@`Wyfi`e|s5{U74F(-Ut;y2{t3Hqb7G4v0qM?2%CKWh_E6>^7_<@kq2 z{K0K~r~mbAF$J;Ro^m5!c>QiF_OTDwgCvIZtr_vWDx;mJ?6ety!JUhzt6E}+p9wK3 zn^RTlcnZ@OpX@yBrY?L8zlb8-DJz9C86FgbX8=tJY?LHJ?SDdUCS@PymlKTv3zXB; z+>q`A2UW0OZ&JVI5%mCZ^Vct!-isE^2lkJ|a9D?KN+vS<>(}4Wipru=a9Mg{#raY0 z(?UUsLNafPUA+{e*$kJ&;Y#I`L;5$La44u2d}Vp%nQz1;>{sM8%7iO-DaAYVaZ^3x z7F2cH+bJ5(NZQwjr&<%aV#{WVkDy02yikxhTvfb08m?mOVQ%k=rslbF&`j`+y8`Tz z5%b2hx{AbRQh2^{oQhuSkF_mX@M0(J@-4EUQpTpg1vlrIheJa(C*8E17uOf+a&mjo zn|j**@5x#r>)9Ss*LlF3UC)(_cKzDa(sHDb(^}*Cawvz;BWJQTBfI&fdwkxnvr5+} zyFTk8?%QgR$#{|rM>Mu;+f@ImoD+Rx)j{LF3cm%;W=o%8OUjK}*%UfiFU$ zpaz1L2Vfv8vpLEC3|w)Y2M2cS748gbMJls)J@iuIoBcPJr>_G9UyB+np_gY|I>>Ym z1uT_GqwczK_L?_3g_(Faw)t<~;oaByujM;)+;Du`AH685o}~b761);sujsjyXWxF3 z9*z!>RB#zJanpC7np71H_onfsnVFgKY~*HoDW;sET|d9)n_hP(;swhJhkC`$3)@;P zR1z*ztW3WwuPn`n&h833esT`$hVgM$S$?$}Ys0x2N@0WlmGh z+sa)YQLKrD$!G140=US&Cloo!TyI!-z4H#z;HsJ+{u|7;Ni7cUshRhI!|mIAJM(ed zd>p774sakH>G0u}A+FfTqn{s`Xpx%}kf!a}8> z-{B$5Wpu;aPKVH?j#oq)saL8oGB6keYu!Qe5{0qxNFmCKF=)H}cJqH@P4x7~2S^7G zN^gJv7|(lJ8XDFMDC;WCLNt}mntT_v@tN>430KDL7PkEWbi_q|QVqBl^eNG&A5i|j z4kT<%jKg=o!K=XIbm;wDs!^0!cm1*J9G~&>{n#MJdFP?GzQRrgpoK49y?3 z8QE52clm(k*p{e4t}q`o5ZT=8?nqw@{7D zseM%5NA+&F32CXY<$azSJ-#bFH7RP4vs75k$sgDwsUA04xM@q)J(=BKG1{;ki1xG7&MP?wReJ2=C<{rtZ(pYEZ zW3-JoR9I*TRxIc3 z;A(A||CJ@$MWk>bfKDVVk9&SHWn({>E`W|UkB1THy7o2jpbbZMskHND4@8S0iST)4 zN_i*8@6<6O67C%eYV2QPCOPkKy9X<@FUEv!Z`_fP2auEg{n~+f>_CR96E~{)n)Gp` zOq9ZQJ%FgJ$-SNyQ3+*6T5?n_e(U1&?+0=cRVt`z+25BHm^ZR&uHtG=7RCywu5Fc* zteSDZ)@+M*H$g>vH(F)YvyAI}!|3%{|K)>%h)hcx(qyA?(FX)K(uncK$toZG2b%+- zyFw2X)wE<<&Tr$wsabXS`7#RZwrr{hj( z-VP!0M}H|-wl(J#VkcR9%e52ZTSNj2+XXhIOlX8a9JS~E9shA%n-IrEQ3(R}&%des zRdq+%=gI?H%QeRDy3X+iGd35J--rj78_UBufCQw|T0KIel1UT>d+GLBp>9bG9~N4G zj%RFjsyDCX!9i~Gp$tRgzNoRWaambrX6vnw5b}DEY~TOE)s@qvFL88qo@@`ZrOET+ z-p7bad-A+#n!T%1fdKB(FLRALe-Fpc@=At^Z#{f-Fl^%Mpibs>BJ~L)n%RbxJ`DhJRqs+H~q8@|fGsDlnZC9`axC+H88PI1Lha)!lGz+1zY&YX1P{i=r< zw*06(!^O>gQ4Pm9Y^&C{vt#bL2>ZLDZDm1zIR3z6$6&j(KFR2MH;gIX61N5sf0DdL zk3Dxz)15PmS*|S9WEuTq`$9wlyY{cU&!*F-7^8zl8~vkyMl#yzpKdp)Q|~RyP6^*G zLNaP23YA5h`S%@&pvijp@L@u7axeJw9UR{_KLDwow!FMt9sC^eI&MLbZ*t_q1LbAz zSsC3S^7;U#E>fX2QeT0MMsHpA^e_n6w`2!SZkqr4?V`xz zx&r!8%o%P+=(7h#6MJyQDzkNK)D?3@sv+O4Ltw-gUAue4FKJili;?njXL zNb9@9)12Nfnbsk~nOY5ku1%J+C#1-)mwT}!Jrec$w`>WGa)v>3q@Qy-~`c0glCdD zVqWS!WmVnosfvXL@Dc}w*lq1=&44$3>-={KJw}a+Pi1t9QDhoLD$+9!C=JId?J9$& ztoe*yWppghLiT6fca6lw?=u35sfnJTaWIv2xhBNm-eN+PQ(1sV6j-PEZc4pos--FU@7)KFEs)p)CD>!O`e-;9hj^u3y7i zM##u>l)XinXJ}sci&j6J?aim`l(5w|Wxh`z{qRA*r|kkz+A`#DLx!I%%uALk?hAGj zZ;26mDlM80Ci);i{d@A!e176(6j`1T4GQw(>;I=BhN2!v9h}jTck81@eg5$--1$w) z&To!d_V)JPP8m-y0+ru%gMRotr2YTp$hjt-i>-3+11+aJJ53kXyi?YtM$o)@fE-uH zl!3-27-s}sOJ2zMP2)2 zUw0*nk{Y~uT`cNpL3m+7Fp9rMwr(L-?A*&fo@ABJ?E|(DGgfA-N58j6u@f00N)&0& zyX&%R61TEQ%>Ss%XTG*4IEs_G09auzTGzFqM@PYg9aff6XYZq8QaZBUXWsQu;(`>RO{FM-`7pM0dM1-EGu(cg)4(p}!J>MNFZ>|dZiji>k^?l6K(Mev7 zlBw{xlWYFZG7++*V|(a=AenV+mlTV?uH@ztI!nCmaB-OkW0gTqM;|9fVHM@(k9XFh ze<{RBEbOn@|IRgy7`J=%TwmF&+tFYLKY3SqcP6=S3b?;4<0^Lt}gCjY)&56vX{ma z%X)%&T9q4m_DkH$Q2OjfUr21fcGFp$8S54Ux43Ucx* zid$DUBJehG{cnOG_R(>YO?HEG$q5#_b=KLW1!6A#8GXjttUF@L-z4?x(ND}os|7q= zxnz!rp?^*bnCjVJ%)p&<6`MLKAHaR`!yuw^VpEe2_ob`wCN{r3Q&QqFI;k?yN`jB4`|Wd3oq$=foI+39+&5nc-Z)@mhA`|Ks!S6`_8F zZKq~cQ%BfojgzuDoL=t+K17{$c%x3&QZ&V^W^_GaR}Q+|ns{YJKe48h)4P9)F0M{} z>u(kH6m4~o$L0>dyAa;v>?Z7tT7d=CML}=LTR*^$A($eQQ>-{XI=cT^2m@K>(!AYo zJ8!9aNm?-^R5xR-9LxIWR#!Y$i56M0NE4&(TB>NDtR(c7TgluNJ^Qz6WnlDOBDq${ zjKfSOv|dA)@KB}G6!B|XxFp3kTHow^Wpb#+MG_c-k1n1BE~w9*T7;J9@3AX1@G$EfH825_@wau$!AWTerV zr6_QaVisz!#R>eKFI12BD3KPW*B@niM7&Q%=~PB(mb{s8SE6F)3eqMBWf(LYWv-^y zPxg8e4*yJlp!e)7Zk(=gq~uGPq;QKhvOCf8C!Vcz<|{DYIGc%zeH73NuA~sn%_Ey8 zMb1!D)()E$#t$Ojn8DxH9-k1MRAA=?N4swn+Mf|&xPJ3toA?XSW!IL-bx&wNh`lfO z@usdRZhfTdA!IQQ<$49bn^tx4Hd>`d+8i|P8|>myZ=nSljUxVGQ)Y4BU$C&GB>m)>qhRAv!r_HzCe^s_fwBdIW;XTPRcR?p)yBcq zTz4xN6XX;WTBD+{mG{+DR73-Rwh(bUQ%aJW?VVG>bn@%g`C zG5(&L`$^aPsTG~inKCRuE463yA6qFuDgw=PHuhG8eypW*3Zzf zJ$Q+ShlbXiX#Yb1mjm9Q#{uD$qBv_3$WROC8Am+GSyNB7ZM_fvlQKGc8%IA0{IShG z2Um|2CZ-(-d`2nP823(u;7aC2y1AP-E}Uv5szGGj%PN5v1SW@$wNoe)f;cP;;~I zq(;g%WJaXmXOY?FG5VS@yCQABUq-%{BGoq)@r>r|AL7Lh z8!}f-#?gqFBEDA!_)BbB6;{F|a($9-jij=|;3(KQ!jDwbj?Pnb{tvt))@dhd$<~Z1 z);3z_kE*L{my;{P#8~kSM5KkL-+?AolT#+6$n{aF?d~?w9gWiJI)OzL0h(Ra`qS|| z#tM+JQVPvJIV=8Ae9HLB);+cii|7OJf2~0myUl4xb+AAqnMGKB$t)+O1Kx7lo%;J^ zpZQU}jg(YWn4t-SC+86wDtLz#MS8Tufcvp1t_?Upubt+;GGx&Dj_lcB#R_9Qow{J2 zHER1~lPHC$`cKmhx_KrxiAC#@)2FQdM(B}|>~E4Sb!EVH7_(9F;zL=9Z)=A=HJiuK z$yG2Vl_gIE?xIz!|6>#Rd>d6#UOBP6eZ>b!jNb^ycqLvU-+H*Bin& zg?KO`dfv~y%Y+>ym+#^mr9*vHZ8GrkWbL=#^71lkQXk^#`uCp(SC(7t0(*YrI+Wygk@k)@i}ucz(@x1Y|b zYTe`g_i5{I)9cGszL=?c24NE1>kHjrqVDCcurOqmTQ4|1GfY@8h>!GZ7w?~%s;!s|1iZcKb1_NTrZu?ql8HIw#Zc5Oy41PTz&v+p^D%?JIQfOq zPaPxoVXsKWIl}aT7kso4B3))Bfi>VfWTE(Sqk{S|IQW6Fp<(Xn1td=QS5&Zqo)eVj zxw$&lNU|Ap1g`~b#dC_*Y_irvAN%jY!ZNDhzFw6aFiRRK?ETgkzKKgHx z_smu$G)lPX4S9F>-DPLbZKp^AFQtDeWv@ilRd{c_#8w%amBX-|l^-^Mh~hG??Tj?o z^MJeTebfbkm!|k6+V$=iRx?;du5ev8@JK!G9N;@rXFs~QLhj6P?Ra70;Hc)!E%k-T z2b8OdsqN_+1nI@m`cLqLNGiU(DEVm0+KsGtAWtACg8Qsz>Ja-2?ai0%XkILFjJh%? zy}iAmv=i6c&XP4H{+6dMo?5dEIJ#J94BC}4+6AnM^vRqiUpH{mPnz9`yr;_}SXSvq z3YTS4I0FGBUeH+7)Mo_G(W`gCVX@0`{lm7W-1~gvxPm+OxKCQs*><~wc%-a^p*j%0 zPmc_H9_HoLwX~oQ=tSBe$jsQ25~8A7A;jwq);+uP#A9s+-b5Z(=3AHf%!JM2DEu@J z`@zOnB4La1oQZ#CR^opuT~^uqHvsz{zkyx9X_sl}u+K*xF}9X+#eFm=#0iOrHTXDv zeSPNH$4F;ELHIsQ9}XIRsg9}zH~a!vI^IHP#_<9Vm{;!c_hCH|iG+)r-~7CxeO zr&yy>Vnz&hM!dCOk*eG-AlllQ%#7vd+xq^~YxL|L?0GY9M%o|J+%~Aaz(5F$w5%)A z518gT8l?Sq;Oi*wP2FbkMHjp8t35s-o^S2J$0VDe`0{IDz(jGFk=A6TKc%4K56lMf zg2Nf<{)4INjO)+0<-`ZXUb-$(mP8`w93Y;589wRjaaQ~hbbvIr zV1>&hG7r4Cyal}4+B!d4ePiDdx)9Q!SBv`jK?^ZXTim0AdqR=95>;Fv1x=~{L-&3C zSRITJ&+i2+Y@PdkWhx+egv(9mFh)T4KO=jwzGQ&&g@HMfn%bOUI1cA$!^T-b@wgXZ z9k>V~DLo@A#hhge!FejEA&BF(r?&(&>!ILDi&`XK_|B|YfNkURWe8+}pa_$SKo{g@cCQTNP&^ z#cA!`Q#pj-**u~MAeG1OW$*jahURlOB*iUSTBwX8!4g4E_I`QNf@oL8CFynr_d8a# ziq$dVd!_j@1GY7uoG1zI1)HN65@DIk8o>@V&G<(vHE&2_r7pvL3FKpHO0wS+W7ZGj zPznM;3b1eSzdLviR}va~d49pU_Y{lPuoA23tG0(o5UbR!+bFm5RbeE8=^cd|o1=jn_fWi@9B z!gEFf69qL#)``?m%oo%ie8CFR4g8vF-dx}ERY!Y0Cz_%bke>R5RRs7gDm{Opwi5&2 zcdp;fTl6z!OHJs*acRtz)$voKiw6?*JKvP{-)buIEbEa1(2{p*k5|R1o)gU5SjL#L z&QQ82uKG$RujYtrR5ErxV92~*s2FZ+?8cntaUfo&)xGf9@?XGLBFqCgu$EyxOG0$? zPe}K3G&8FLM{OB^Mkxu21<+roP70lCsd%p6PJHdC)Y6okPcUI^dnYbYZld^ai^Xc{ zojX<2CG)p$l2cYO)bQ89-Z%Vbvj_@nJCze5#J zJr#R!Yx`Nuw_1DZ-8tF)VOh2eR0(V#2h+8bzpV>iZn5|;?V=zqUD}J%wF!q7+HrU4 zBg#ZH?@}{cJk#}BdU8TT^;ceu2qY@X!Q;KFVD{^OX_AwR%IXAw}(<~ zo>C1Y*~+aPaR@$Q@0qS(>d>B}ZEjg{3u2sE%V}cvvVIxzSu$Jr?Y8#toG6N3rG~na zQX8-;A;#qikm9ho_B+#z>G|3D<@07u+C6_+lFL$wEVAfGD%VxX4^=WkISii=iwu!( zd_4Awn7B9kCZzIw^6Fv&3nRW68pn6)MF*{o=But zx%AT3GT)V@hqrNzZ!Q^sipxS!12BEAFz;mlH(lths}8Hor_VojBM597zm4#>U9 z!9jEbjRGMg#%nVm+Zy%J`I%QfbKx+^#lpc64`E8ss=tKhyhg9Z_jU%|zKsu6FpP9j zgoK2g_i@a92P3>U=W0mv1E-zou8a&4{rkAz!7_%GQaz-W`dX=MmRC*Q-YK%R!7)+FuVl&r2ZeudR+X>kzSJPH zkR)OQg9o71xPqzgFe_SS!Gs=XVPnG_c|D-x-Heg4uKZb4P_Vc+|C8;-MidzW>>eH~ z2`0!|=mmbm#L2cIwQ6f3C=TYl$0Lf5?)o7o1Jx81G8lLr0>*G2e*}+3VK#;h-Lpk7 zzaTvw_E2~mpoXm3)w&FcQ(k3!%!p;Zto&2&%!pK+bhR$=(0t%GQ||K0{GA(h0`Kq2 z*2JVpUSAT&AxMY|c{!=wQ#Q>hP^}x5<21eAT%Q!*kByB*-eDM>R&F}zriCD>gMsTo zbUy*8%)qYqEdcGyN~Q~pOMX5SNd|a2ykai|MMUoG)~>z`yAauIR+%b>f%V*-r@%n6 zLMhLGQ4OIuNTB!^>FCs<91HVi1+xuYj8S|lGMLsF6y7ereGjST!=|Xs%}v)cD2tb8 z`()zvJ(XdBdNd*DZv)Cq2U*7XKE3GvxhE=8b`5$-`_95_ ze!$x(ooteirfIRWdf&g-wWR0S%hC)~pR}U-O^AX;V@B^2siwGpTd^6F`FqonfdPni zC$|BrF1ia}HSWjaVR`oN+@OhVE2woDvzChDrz`xQPo6PPF7N-IB1{Jwb393X+%TBi zSUyIrghSD$s{Kjx{3pS+spOZ(Up5k@5<}euuqO1FSnzQ)ez|NvS03G5ET#nK#5&u` z2|KN)!yg^6fP_Gpw{hca;S(otY#bXC6k$}mNu&RK!v37D!ol|bVAA#@e^N_kb?OR& zXfQT*8Gg$VDI9u`+1?K_Ub0K;&OsX|7MkD(d^#sE;fHj7wapr~yYn=~a2t$`1slgl z=anHxcgwZy87nfXehXKtZt}kpX9jBsLXc3`{?$%mT;B8mQ4i^Vr1Hx;o3Fg~(FMDY z8O`sYxfc+qDQouu$E~7T{e;b+Gkkf%B#j6U=|cScL-z$3euV-a&4U~9eUwepIR)ms zqd|sniHFf$ZAsR3L7;pt#;WRc+04C51NZL&421N9+!+qxZj4u_R2t4^GY<{tz#h16e2%13@bV6Pq5l@}TsWAT&3fqQ3*iFPPt)G}YvV=zojj%hg9 z6`pg)+0|EK5ia5zQ-QbNC8VXpw$dSX* z$h;DyWDdh!FM@=j@P$S+5X`qaqH5BPi8G(Kxj#`Ep*G+)5(u3-p*A=l&3Il!l#J7) z{bF$ur*&F;n4~VcYn(S?a3}73r-Dk_>6>SYz(Pg)n+GK4TVhWFNU}z$4|i05K61*4 zjWRmezyE|Ced@%dQiBuxh_i_CpFAsVJuXX;VvsHo>wSJ`pH2dt`_bBOv>)C#7jw<| zZTmjqwr_hMGB$ix)pmS}$DsAAp`k%n8J*iF4Pc!i_$_Nc{JcBsc{&3v<`Ve)z0X6u zZ&^-O1i_3muyW(J4EHa2zfV}2ms#uRq>6rOvOMN0)~F65TX(3f-MElW-0e@TeotSZ zQQE9wcGIZxipl%;0~IBe!+P?9=VrCzcjTA0Ttn6{(af0CYdreyGH6sKf|dUw-@Mjw z)jBHJU@Mr7pvGt_V!Fz{TC^8CtmEvw!bVBp$HG$!)3S=b9CHf`B>ZK9cwN}vyx?g- zy!so?dj6PbHFz~{?e#D&t}YUNB^AH)&Zw)yhmukQ`rL@syzO(?TDLI=U%*1U6$qJL zHnS10?t$k=7-&BbE4&qnf1Bb$>LL?3T}!MtXX-hSyP`)%&;+nXwviPWX^lCyiGpOk z+`_^(c<|-4wZp(gvIHQBd#&5u25r zodiEDe+pS>J)nr9e1terJwRaaSotRSkqb;p_MBSa9sM zz|j^2e8#nG&{cfDtk`E^0|QbTT3UH|d7sHijYmXWxZqwTM)`aGe=MB^RFr$y#fK0O zX%Oiym68tW21yk~8Ug8$Qo6f41pz@?8bPF6q(r(yQY0j#zBBjz*3xykpfJwN^MB4c z`?vS`mE8mH3s6l_5M5ne(6K6bc{Q&$dcqJDNo`+$#XP+AvAnz+T3VcpK@0>$EbKW# zjb;MzZm(1nKuc- zIyDSlpu2sW~1Ez0`_2LUclm2(-I>N*i<4&vzMFLO(S< z?A=Vhf!ez}uR2ygL@UcjDkEip_lOkp-|@Ymc^jFxO8AOZ=!G*O@0jHfi3_z}{HkOk zAGu%;ol#}eFvopj8be<9BvM9UowG9Lu*+IivdMbZ(Xr7VUY{R|H#84!=o*CA@a@(@ zWB5J*&x|=}Yr#(gy;(Wp$BNxb@78qDz3k zi<+etV(~~apv^x3^H_5xkU_ISY>+P0z^uzc49F35jB4tDf60u@%sSN7>yiYCVd87d zR!v51k<0C#=pBToX!Im^)}`$0q~gQG&AQ;()I4Dw;K4B$<>BRxO^S=&v9ldAl_D{A zc{ALs%fh52FJ<7yM^>shisI#wBwaWrmB>WKftPsuPWzTap1lgyZj7*)*llLH8(RYP z7370{1pkiNNAzAUDU!fL4>j<)rK6O$L7OC=iBHpM1n z-H;5K&9np8-}4>L4QG*~-#U@`$(O=&2Bzlu9h=AR^A%8dMw7N>NSIT1)kxNVwg1A; zeFbw-0&Ioh`e)=?kEgxI07TtU!~b`}`8bA|BC2iaSUT49=)S!u)HEX&RrIW)x0vP5 zwp^Z!S;!78GYn2f(dqWWqaR_rPg2BKG6_!MGvnV1TZi}3oTXfa(srmOn83KZb%UB>Gfdj0!k}NVq z5Y5btAU@ux_TEHL&#pWj*-mi>1_Ht^bJ!WtHs-0Db<4BC51W4r^))WWY9yJTp8{l%aw~IMIafstsWU#v|@MPodU~$6-7mD@VY}=zr=el@AtJ1WXltV zBe-UhqBm90sd(sa_TZ>?hYvEilWcI?D9&Ws&WPU}OTlV5`1XX#Ew##P9G^isBL0&} zbtomLqMThFo_R=Qk}i|p$=(ysdhJfafP~>f_M7-$$lGjF@|R*!reW<~Ug!}Yb-%|3 zcSFZp!)Dj}AWg7;OMh?!5`6B$s51gqe%&T!Rl5jKILN*c(%}byA~AR#7y;9yK@PHg zGWhi}27|#?vIHf27aIOUWZc*yOuKisK3xH|J8d18o{n8O76uJ%sctZJTZT1ag0fTr{K_C#41%d247{@4}!6=q^cL= zcOf$-$XBBaW%#=A2QB*HGC^;AB(829r2OA}_0F9qlT+}B<(3+mH%V?K87Zf3)X@l`oq{4XA6z8=NeDgP*2U-}4VXW@=O2yv7EP-3SpE z8w>TL~HEjiA0n((l(tcj`dtCxSu?*<~RXzwr)rK7FPi1WneadK9DI_Xqa# z4`;zi52i4QU_9#ZqQ`~H4J+!t6L%yls%U^%!II0x3S!)tkp5j4QV&ZW?+F9tPp4cJ z{3pqj+&ux8X{6#=QR4=OG7-D$9qhx`i^pdpB~O|pDZ;wpLrtwj{Ddw)xV2jFObwUR zV8+KgbEodIac%pp<}>y3Xch{-S05tEFZ@uv>f=Ng3isap+Sr-0Ri73Ev)=wk&ra6% z&z6q=Y3e10{GO}ga}=5t4_dVqW=W|hzmmkd@pdjQq3kxh(408j1mEjwv&@Q1XvN1b z9t-5rNwAi4?wvyJmu&Yr1L?vY8k^ZRH4zD2Cl!}jimt1Us*}pF|oJWW6n)0j}kBXBEEQ?2X zxt{jWYRGcbM#)n>{KOR>KOM-4mx9@t&qT7m_HoyY2-p4A=$O_>Noj~kw<^p1Mz`(Y z_6w?7L!IOiiy8(F{u|Bx(=1x^>xeIdC`a2*i^&;7W+Kde2+;l={g|zHO}VqA_{1p& z+dtGj;D+4U=PhwFB2|v_`}S`3;ZQpS++Y?v;Gd{rANXj;o!h{Tf8(w2Rvna;Vk!}_ z;|y2;y_4~nVtqZe!uT$7`6R!f02%{MNqEEWLjg|@-VVjZy+A-QO<+Lwyv19rs;Y|H z{m6m}{913$ckAQg;|~U?FZz&Q(k^}O2X>=&lggaRN}$zNp7}t^5ec*vLJ%$a-icFY zNQN_zQ{M#^0u=q%NU*Djt)VFk!Kr?{SCOB=bF&^`kXAbA z8IN;694P_^2wpLP9RLaQ_RI~r|cX9HkJgwjf^^22mzWfiU1`_+@CMfm#nZdT_Ojr!u2fe0W zuH;B=dqrLM#ta}B4U8a{w&-|1 z0;KHEeP3mJkplh)Y24j#w`mf|U%;(mU+HCx=>&lD&!`HQF-%jRe=6@79no+Mu_@@w17g{FRs>Hs#cM3t=9fQOBQ~c|TlUO6pYW*IV9yXh6&P@C4_1 zsp}+_8r9oV`cI;IAIq&87+_M5b1qb+;7$Z8anWp&V+)Wf&1$AGkiqWvj}~^@mTGe3IF1X+dRDGV$^f z1G~Pwx@bh34+9hdC%ZE}Ac?zt9?xIcdsSi3{DVSs$ssi;D9CCv0=FL$zp44pz2Zc zRJTS1u7?7azaj{pI?0xwCdrA>w^VKaPDYW2bgL~`IDV}CWveC#K+&K;VYj$Gm;k{D zZuK5CUr=s`mwx`VV!f?$)GyL(Ww>{U2_C@Z(&;4ci{cONd=z=(O9k_$+ckz-chxm2 z1WQPd{n@ID=X36=7Nk^DG#pNlAJ#KKX-zN*Me_9ZGM4rzbY zBc=FB()FX`tf3iSZ~8{N_i(e^c4ux``^mop6@G2^?< zfY(Qnif*UZw2a3yRW{BKTc!oOH|sop2gEXTh1c?JoitY z{{QG2uy?TEBIiqkFt!7v1B|K%(7BusOxZ=~R(vo`Q_mjifmxOxv;j*iCx#uj)Fnb$ zvz&=S@TmJoN43CWz$40=rhvQoy`;Pm;!ggczWSEhW~Oq->Cz+Mv(Jz`u;uQ}{fF9i z^D4%>O8Jw&vt&u{+O@=K<<)(E_>v}kdnkWrI%kS~=9$K%1L@she%qj=kjHo#wYF$3 zBwr=cNTzLUm>v(lg6OU)?QmYM7? zRK`FYO#OWy#dsS#^POPIUc8^(5ZXzitDI-~@6B)FexJ(pHb}(?V|`}?LqLeY03bKp z*+uc@ML;De&kaO~vQuN7NZN-~`8^{7En&fC-_M1XH_FiGA}$s z#Z!2$OUuiz22v2Iknq*yvfYjJ4Cia%EmhtOU-MDeDJ41k8%~yeF4X>4A{>NPa^B{r z%d2mvwSqyoy-c7l|M$L(w&d};jJ<-8-Rqg>axYgscypcKnrRCvT^MO6<|=-b;O#Gz zi2U^OMsIU15$m_mgQTJ5iOIYN2XmI1A`kn|-mXOrrN)g!@!s(eGU?p+Tn=OPo}pvT zk%F_Sx~dAx7hX;EXCa3n%a&6v7Jg4J(RsP>)tU>e z8P1Kf7drzw7bgD%N#x?ZeS8O=)hCr3fMMeom%)hTtc1-%*HEF?;B$8Q`iUOlTgh6< zW{tBn;roL-3IIctcwV;iiO+v^f3-$aduh0;wepA%nIa zZssTaXhCX}^AIwre@6qA;tRAtRN|gOa+;=)?{u;C4b> zsh#70pREncf}hwjpGA>270`$(O;gzkU`kVH)n1-Z;FyQ!$`9@RQQnqZ{g!Gf~pn@t$(Sv1CoDdW1Pm+5+)r5MDx2 zhKD=CuEYc2jB|a-0%s@#E~&?W<9*wg52P4-p`#*;mVL_8$ZPWR_rdVW;bZ%72`=pr z=?zK_S2%Y6rK6P=PsD$Ns-AI#42gM0x!YG~1IK!|hVh~7-FMvfV%_N&i~-cz=of7S z)|9moomak9WqE8J7B1WbI3*L#*4Y@0%>7<3vYR&}Q?jg*Z%BXT~DG zUU-<^JS>SCof~4z%KK!&6IiG~nLTgw(l$?%IHF@0l-Vzm{U&Yz+*FV^Sv)e5dnz;A zyoMj#&{FrHJVui3n^@A{X^%ux)x0=pQR*gig!Bx4@{_Op`#6!|qm`+lIhRAo00Ymz z?<`_i@{+Wh5`YVx9j`)gDG|XfzWgfdzZ@?&e>yCrpXcpCA8ScV0wjAQk2jIq&uKo! zK3cF6?0tY8G0^)wWZ_?E)bH^W2M`!9U?~0uj6O&s^)Gs~|MDZiZNmc*^w12Tx8`AP zomObF?njIMUd*un+80L5`nu!GozE}--GBqaTvZhp(i%8OZ_WK3n4GPN^lr)alN@k* z{zvn0zRQ~pl5WdU5=YGC<+`v3R89fI<7!FMo$Sw&{%#xW{rVRAz% zFMx7OgbGw`ND3DGIn*-e^SL%-KWm@Sql+wD2?m}ng*eWvZdrvBvv%92z}@9_#A%aQ zU|}|ksKK5%?FWdOn+~t|I&Dcc+UT-xD{&TB^*&%@WcvfRtBvP3-8zkQi53|n)F6-F zVT{-0c#l0-qSRJ$#IE!=Bq6BC6F># zoM^(E(o80(_cm{CqOPl%prYrmdAf{ya!(PlNl71y&d;B=i}Tb z<0_7}+c^K3*}@jOEnNC9%s=qFvcc=mJS{t(WDfiCN-uRlpmy-S4BGA;{o`i3>P4c(i3tWcd5 zL3-zon+ux{7F39|wuB!bKsX#3y##o^0FB2^rH}A~6Te_&3K^u$F}Uqaww0E$3-E6W zJAFs}`Gf!UIcLzK;TIE*=C5^b&IwcfK?B;4lg+L^y}ScXgwJ+k+Antdr<<6R^gg>X z;Wn3NBh~`>*6Aiss6Gd z1R1(zo{_br7zlYJD)BfNGJ16{MAaSNez{nU*i5%Y#+UpT6IJtKCE@oKJLq6U2q;>X z3~e$95J*s4A_82TAq(Cm7dem?1Or^{%dRTst8(Z}!5n}zY+UTKB0-Z%xbTb2jMePx zT+&h0?wLA0ZW{7$<96Gv7$b9Kew78DI#quStI*a)ASI2IXZrB1{;AF1Q9M{-GsbhyX$xPDk-hf@}cje6gTg8!Fpvf1wnYZE@wY_DM>?jFJ&f4bM| zPqSF+G=$oVMZ}`C%ZA%+oOlSZtNScIuqh`(WE2f{JqH_G+NCQthaKAPl7q)4exQZh z+b4yAfbFKsznneotCG&Izmvt*WBeW>N8Y1Z+Br2&{g4V>+&|M9BU5h_ofFsoDNr=Q zP!uZax?2$?s=lO{*mOM6Jr@zr2Alj@JhwPd90tH z#Ha?wvYN?deN-DY{6T>rk~aVzy#X++q~{OJZx29Nj5$~Pui9NeX)}s#hU8eY6#as$ z-+y5v5pr~_5{B@$hwK#CSL6WU;7Hx{Enrnhi~cLHgWkf%P6bR5!W7LRgBr5ZAoc+1 z0Rc7u=;$GSv&XV=q(QtpDl3^0oe1I^wBr$b4|N zJ@;j|J0vY7rBM7%fse+3+up5DMOC)nm!0}EkDO!cSi&IFmhU_bB5du&t0EASWx9gc z(ifS3CA;BlelLEKo}Nw?-T!5!i<|P%{%Q*sozv_u!k=5r-}NX~4zt;qzS707uqaG< z_sJ*sIVLb>))&}5WJ>dNc^&xpjzthEtyfv>HPm~DNjxcYy7%`xrZm-b2)OR0ax?a) zsk|%|?CueIX&HIjh3MpmGS#6)s6fg$!2&kNdDj_RKtN333G+DKQA)Y*O_Scr)v@bZ z4KWMot#CK`T+(s&IPTrM@1QDo3EyY=Gav*UV`_SOKdceDe!bFyf(#>!;A{*Qu21NTxPh7LYPX1e{Iwn*Gt4nVH$*`8l%X z7PS1m5Ma(rg~ux8+E41^KERh;RF&@rMia;>$z18`^7Nf(^E;20yuNTzHP`-G8+Yhl%@NIA zVaw<0&JHkg?Sj3WOY=Lhf!-Da%rf1Td=8EkgKWb-SvL>M%AQwdH~fvKGwr~~M4@FU zEFVW25)o$+soj+E-XRdkr1S_7MRj1exlG9GbYjB{^6yHTxLEH<4iz9{N73mw_Up=vR50(K!1l z-+6$uJIQ-xYxa@URtQAMYGm9P-2XrWDtAYmMdr)EiC=|%Tf>pr5@g{{_;h`eY@J)~ zXlQK$%LehLzl%zrb%~S_Foyx{wcL*r+s{7=1x*`-Kd9ONjI^wl=!N1g()l#2PGoL& zCTvEMyEK%2TJ*`J{=ASBMo}nh@k|&y1tnz~ypJzAq;Kma!$;-GkxKuCCp5Kf|K)-8 z3CPf9W+liyF@lJbQvfubT`Vx!Wi|8J!U3yC{*q;Yq^zwLNxDA?*VFY8@_Fz{61ja^ zjt0fW><~~(Q(@=_y)A19KJ7a?K_Vf#VNC?Lhi57!Y3Hj+6{54X5_+|_!uj9FnUn}o z9v%%dqPK2WM7h;q2@H6ylp_Q=@D39nt7*d3cS-jrDFcmy)mKq*{no^pW$~TQrdCb- zxM_NyZB`7m5P!g|s8u)yQQzXrcMCJy9Rj~xM{z&1+kB||cdjBAN1-=lKKjph*Oy+` zl*aBEwL{ z02_E$+U7>ywc}7Yh`@vh2tAn}e}D1elh?T`j5qlA_p*`Je3;7~lvaN?(bgvANIs3P zyLs`DAr5P7Y)t3rQ#tS!LQsXbfQv}p2-_>qCP}5&);ymIz79_DQAImI)>)!iTFFn_ z{K2K%!QZRVbGNNB%0ykO>%PVvl5$O41dj_+H0<`w9i3fX95*o|e?us8JiF5}P`>rC ze;_z?bv9@~B1El_+!FT zb9|MPnKY*P9~dO+n^=M&x)P6k7GFt8Tfvoat9_ja32To0l zykfX48N{E3ryx4BcL;c=-YT4?u?T|?VvvM3+*FGBg6kxej)5RLN7r1$-}nQqW+D`2 zjAHz^n>|Q-x-!$_{c|t$TrxV=^!bgi+#a7`BJAvW zrI)b!Vb}2kT!lRD=Q|sZ4g8N>TgKouRZF=O@Gpr`xM+7AMyze?f9RN*$U`PjDjXIkd^W*{)4NAmF_V`ufM>V(I@CWQ4%{If!JzC)#1U zO{YWz29yX`%(4hk9GhEz@D!O?SwR6O2R}I=OLFe+B0M}iKYXCyJ@50`odJNIo}d3V zbZJ>%7g%Cj7YN~a6hfvq0C0q~@ji!jF+iRSyt(Nt2AM8pXXjNOJOsFehEcby>sXml zCLYFz1w<>gWw>E1oe(G~waU4fOTNa(KzQ=o-V@-6w)&bft{-wfzlmXu<}m%fUvC8> zo+S6)6YBldz*zr6D|Jb86A}Bas3R|-s?zwYZ+jIJd)oA9irVI{3XJNh$jXWA1r3+w%kYS7Fj zL);PblxI}SnZRtZW<`rHd-r3akt0D#efXQx0zRSUK&H(~@S zx2}#97{7~a>V6**1Mf34B8gAdoa>MX{ z&rNY8kF&LK$G3{*i}|H_rc?Lb=VzDlRBtEgd;5L|&z;|MTs4+KL7W#OWtOmiENI__ z)=d8jep`2Fx2ZLjY!RRQWJw7OrQ`3kLWa9aA#wxSE9nEAvhwaC>+H{iCAlM=hJ^U- z${WCu(TR={XFBY`ckGa1x@FO1d5L|xz|M(Z~BnnO!#`dvlIgl%`M)TL%@_8qq! zrU$FT#trM~)($^36^voQjt~r-uXEzAESc&{hupS*?%sKUdRw_6Em2`k@ILlz+=nh{Sh>LrN0eZA)6_OPg>?pa7i*ZZY zgM?1_?#%~MvH|mviL0hZ_pBLZU-~0LdT!~*Xs^ICG$4<=wt?PimvZ$y4wf4VhY()o~<@Pl%`M<2hoXMAwuC))MtSw|g>)B};!5 zdxwb=J);=44*6#9cPnBu#rR111Y7_Sa|=cS-*p9Xzt~~>HlTUqmRddjT$35-`nTPy zam6AcepDtik=I7C01aUAuXa*jQB_IvrJWodi98$CNGOeBuA05nwYV5RDr(+13cLm- zH6hA@A?<{L?V5n=z*%wwzFnJ*69b&h0A#>(I_M3S z>+is=!rCB}S*QDJItvxwr}}#W2t?CNL`iuwT%Mdy>R~TE5qJ{*|2)18=kb5+3@&9b z#CBktSCb$98tMVXF_JOnAyJ#rrxKY*P|sd1(}jVwN(S6e03B8Xq(wY%`>hAt1uqX$ zh!q#rec8etXVjV>;)Z0UhP(a7ZMRG0&!%=WzsK1kfoAxHEVxKbTKS&C2(AnQ5lzcwO;3i?{_C*K!b;YCq4soV1`~m(e2#+`g64&<);%6$YBKUm^T)-x(sPgeRaUD1Hymp5bm|Vh%SUG94il@r!`%@pP zOAeV(Upc*{kTy12;k>DUqa^KG;M&Cu_o|972iPp??;?TP#sPS2^?FF-Do?}EFb0;D zpg?%iB&byq(HEG$`T0yp#p4b5q>SokU~t!hlgz0fj`T^DbexW9>gh!S<&6!jo*d{X zz^sET-hzYbhd_oBdIC&>72JM$R6X=(gAw2Oa(!3Ge9{}3rscl)$QT!uY5~9&q0_BR)oour@0vqf^a}ALt*x zq5c2&cQa_s`sd2%;m}XdUp>v|?^Q;ZQjJ5ps6F^7I#w+WSK@Dd?*b!%ArcLMTW#xA z|Gh8QSLZ&%p>EMX7tVHeb`0)+QEOn$*%33k&XfM3us@z4?9G6eApGhZv4j4UDA zAYaM>K(;m;0Y-`_6Nr>Tyr?>lh~S&*Mg~0qzog*);vSQdbU+*GhBLB2D##lDEg#RR zMDIUi3)xI;TzCBkW+IneO__2`MRa#L6OY*>14hD^X>K^Y`NGDviISy;nK2dxDD@Uv zF$5sxxrk8jsBtNW9~f5&57&RmFH_g_2g+&T9rM*A3{vGYG>w zhfA0{=1rY^nXq3sS<`HwgPxuDMV5a0hGyX0rW6xJhp)KUitG(UqRrn`Q&YDA=@(Ym zCjN+sjv#R{l_ym{)}8YJ5JTe6M>e9#P^BCAAiO+ksUyahq?9yKAV08QC9#rwDsw|K zH3aHcxo^fb3}RxqL2rfs@Sl%VHz>-qD#zmh>O~{>Ikd|J!1sa+#-ix#xh45M9_Gj5 zVq&Vy8~XF+Ow6qG_j%D=+>v;@HO~_G`eXF3+B;>`*qL zFBnQv(}H@A=Np3pp}B!oG&hPhm;q2b{uh@$L*3>;2E{p4`fgpnE@n&)F=c?CTdy@S0Tc=5CB$8D;(Exj7&|t^&Z=E74)CPo zXya_?7dYOGRpQ$i+?Z%0DY|QW5(_o~A*!youd@A&xI@8!h>9o&;|21a_`h(FSHKZ3 zLOxX|%-{Tg>0N@w3LP--3WML|0DiVZnoyHSClU+e3J0@PB-sO0K%o&zNO2p?mH4r7 zgc2FH;&cl&tjVpXsF{u@|n|=&t{+c zmzFw%jf0Vk3m+)fp3lFsqT~li5t5ek+^eQ{Z{Fx?j(6xtNP7;M!5)IOVT5X5*>Jsr z^IJ#RUY9o1fJ;{b~`LVNv?zf z!Fwj1OG-i?4T;vvNx9u+vE5VZ1~(}Nc?!4)_j^_QGc^eO91~tDs-TrWKB0pgQ_llc95Ig#4XuwIi(jW}>wgn1f54?86VIgQ zv0EE!c9znTh=8dnG$!sRn-RKpP1I*3UGw8fsZI;|udMf`gM#OmAB$~$2UgnNW8%vjsAxT5qv}2w-*%j&$230w z-mIvk)ICg(7MC(cty{O(o`Z4;O=v*Z{h!j9)biKXOb3TGc-`kE+l$tA@&WUqWAa&{ zlH6F$BawM{66o84_8p;lF|zfpc%`Zs#OX&Y7u%gHB4Woy)$5DPhu@KabL`LTqan!AQb;MkX(oaSsDk*bPfchs3F4+TOo)d_S5V`$xk*@ zJbEC&Kd-SMl#iwg1MA|XKGLSa$R&8yv#@=b=iyqQthF`hijcjY|3D%A#J)<3Wu#g* zLub!cJ5(nF#*?8KBt+9TF1hM&Z^D>9{MNyugS*CyC%&tzOhCJ?n!3rn7_b?(aicHv z5HvM2Z#8kt%~ufr=6a`JWm@s6q$4F*V2X)On2hV1VrRrCH!X`C1vbHTVxz`|QHMow zWKOB|SSvPR4tK7qX6mPT37Q$R94v%wqO45yeb*(|s~Gd_CoJ{=B9jh)>d$jc;AiEgHgE0|G3PB>;-M++5xF%l9~1VW)A){ylmB~qq+_Io#0&s4W+2n<^?_RvL_;*NX=kFwC&Z@Y z>Fvccb(p#UCrkvj=&>7eT!I0*g~c}%0=CnE z4$rNx)BY!rcS00xB&4_C!?}lF{&}4ORw!eDTdy5fcm_bu3DdrADzZk%hOq($(t@U2 zZx=MS*$FIQ2$)A`AkwDgVhx>0NYY{W!QFwq72z#|=UReu%mr1a_4>c7b z7W$9f_ZnD!IqG8&mP~_8zJ%N^A_!_G`fAvOGS;KykWLa`;PDmjnOI_&?S1qeg9R2; zp##IaLWk4N8zrWmf_G6NwrtQZ(m^Ov^mhXqn9QZ+5nJuQ*610Lrge;ELEJ`2#sk8Tl#%}EH(}Uy`$5E z_ho2UC)z}Q9>03~CbPCpoWshEB8Riksr#~euV36F_2kMeRUk^2>HN%U=@aqOE@jb@ z#qP7v?YXAfK{I{v<;{~kGB15AR89^&7>TSauw0_6 zX-~b4{EnC(a@B6jFM?~1rPrU4%jS@46<}qQ&9jiP53q1Y3j9I);Lm8a;B$2m!hFc0 znjQ#SaC1sJHnj9$S2-E4IZmC;RJ=pt z9NXhy+Oih3X}j4G(|$r(XY!}@E8Fe}+U38a_Bz^=90g->

X>fn)vN&3Rlc6zLNsYo&}WU^5>~*h3S>G2`6{C$Yxl_i+KLFq6Sx4j8G%WxY03 z(Jt-k>%mv>HsA!Hv)F3OaW<6D;f1QuL9a1#E#N-fwyzw~7EsFig1}eSGJHVGgn;_$ z`BzV20h`I;@%MoFpu|Yh`NEd1C0I3qhS-^Fx|7Rh0=IE}Djie=n8c7Iai-7J*&ER8 z_u(a=hYAWGonK(73@fmmfJv!r-0ssPVbo3N(trH8DX5bbYs!4TNlcRhCx|e*6NFVD z;GZW@&ss$oYHMNn&;{1~Ac4r34b9k)A3sPMyu2j~9|vKI`yi^A-m^*EVaY&)=6TUs zFfX6Y@NpH=Kj7W0%%8l*sa~y&b;nubuUzf}vNBhkt%=8ddV*76-;06{H;K+JXy%uK znyVl`;~bpbWYIYdY_wClKdlrk!Iaw zGYBYGnNB1~>#vbzA ztbR(UPF98ocEW0GJ~%Y#>nr%!B@q3bIIb7<4r|13?-x$GKEx%fcZ4Qn50e*vuruGYFQd-m|*)R z1BE)6M_{I5EXvbT+pIP8wT#{GKhp&awJ_qTm1%_HG7|%H;VdjD@MQvZ3hrGRso5o) z6I%{41adI@p%k@uLJVFJxS!bEzd6z(QQm@w`0@Ni3dHarC>_ z#9hOKtYUv}SfTaQ6Ko)nq?n^xk4Ksw-7l>aoP&y2^iaUP419KA| zvnXCaHYM&CyA(^EutP5fHet~*azG_KHw*|JG+;>G3M40AWZ4A5tsfnh&_+%G0d|m4 z^%T;V45|Y(}2*7z-;FFPPXeti5rb8M?R13&+} zUR${#6s3mW`;X8cb_(vvEuOHYo!Wod$Ivd<3;UpwvOLbZ&cLghHsQSe6L&$4$N0pZ z+1h9A8~0bkMdY^6PY^)#5LViddCCgT9@&t>MjyY=2Aa2>2S%0}dw43n%y2{}YJ~9; zk5$`KE6Ejyw5B(m8_~qmKIG=2LK2A3d8X5!t8Zk*Ze9^!A->&YZTb{Jr`(z^jD05Y z(T%8pgVe~0@a1>4u5b>9gQjhVA5t&gcWfJWIvfUBvT=;I{bmu*x=+V{+Lq~2Wz4zF|(<`@)0;2zL&T>oc~giFY24Z?%pzKxQyo`S+9#K8&mmm zVM$UkpAl`qY&!#iZXK$3x>lPze^!ew^QwXvMVDmet9JJcvSEnx#tqqPV zjY8>Yw1BbqC*CFV`uUmdet!U)a>H&i%S zQJBexI#*To$@@?s^Udr?Z~r%6_JPzU2Y}jn9D1b!GzS& z9Ci??32(mHDf%cKlZ$%9K3+cLE$oEHd3FNgROGK4R$CPB$2J{?7g6ZKq9k^FALZuP zux@Q79uOka#$e-olf>YS92Z<+F0Q?lnX9%+DsI@CbjJba(8~~2(+)o;xPHN#66V{o zUS3}9%PU;IH*S#{?Mzk@pf9CRvi`Y_0;UNSMhDoi&^I&>0v6~hPaP1(OYK2Iu)zY1 z3urpYjsUE|9#S|K`r&l;@IpN{oCjWh~I>rplsi)2l_D(Qi+Ry zPN4(@g=Y4R|L|BOh))FQH?NGan2`Qz@CFdz$Pq+!Ya~|0)*wfCA(+km;l?#=wf%GA zL7sNZuat~@Y>E3-!ubMgdVW1yu`y&pXkaNdPepsLeNa?V^XvL8?`_}~*ltX+W?l9( z1C3rId89eW)3wp-&lwt9|438gF@QPR=q6{|`e zA3MNfJG}rUsrah~mVYRyO?X=iD#R+*X}(p?kOHi_Wv_Iai$*PRdhd5xzsD+P})$s4E%cwW(AAI_k&KwS^sF=uCUv5uL<$CJ~>7Cj+OUHGAY{VFYKlW_Kd)185-hJRJJhMS2+y)x#) z_$liazm1Vpe)~5#!~I55qCw0c)8jY|l19)G(7+kP5FY9IyN<2g3)Q}2rAJtVz%;7l zy5i$Uz+N8uj^ru4gium8F0P&*MURQ`gVD%<*rcC_2v}HhURWY?rmzT6m2IkmbLa@} zv`4A15>Q&~Edptb7XmoC(r)hhc_q7YrO@rmN};w(KcdftayUixeql#qgnyp zx@;QnFO+Y*xF>|GrH4izNI*}J%S2FrsOy%JmUVXdz|BeUvr%onnPE(in4O#JDsOgL zR@T`UgT0AuFFf&6qsA;6mPk8ld7_Bik1vJ-w4T1}Tl@6<8%3e-BLM|EsxBJ=t-5Ey zUnXduC_0nllHbHmv*07+roe6RLZM1p3w;|`HEMG3HcXNI{dD55*Qj$|G~H1)1w$)^ z<A%@r4EVRkLIwKY zsdXb;(W-RYLr>hQc<0pL*orC%?i&&$QkaRjgu&?Tn#$p}&)LmI8ucUXY=;}TCQFso zau{6ivJ$BnJxHreRV`L&Z50r0(vP&V)nKGB)hU?R?b6hVmY^Dc^6ebexYV3jKEUsQ z$BZ_w5`AAbypl14-wOusiXTQV$UdEOqFl5)K0Oy{>oy5PZSWejnwtA}SphMFihNd8E~Xs~r6mp6hp9&m{# zPoDgN1TW<8oxOtFuqKOv1Y7g*+Xv1(-|(Uop~XPo{EW%T8aDZ28F*aCtlka z+_efl<>ewHX|$R862bT@o$cPp{*pgcNVbJZrzzth?MOEk!gKeRoXwSn>E^8fR6iB_ z%B9`7=f~1^WK&%0rk$DJ{KEA(d*I@2tu?$BPv{~0=(GX5>-hNjU)F9A$ezw@3$LpQ z^5kl&KV+_Ub$Z9hO1ZJ$JIO9${%Ab@`f#2=$TQxrFYjn)C;pAs=iX8oTrynla+g1! zuk0&sWqSA%dVG$4d+4P1CikXPpeRl6sCLClqz+5*{o2k;>j63QNBj<}t8KRCufrD? z3sk;6Nyh!&Vv*DvOwG_kU0`d-!0a@Ur{nFMl*Y09*1-Ll(E%IUjOZcRy;sD)-$se< zU64z(9J+ohKs%Y-RX61!2iUg*P~f_jjMuG3-I3(741}DpRvp3w3NGg|eeQxlL?Tx; z#lGqM;xb{4>*O-DXQe@pv%kOpV_Dhd#f`714Zj<=3DErj&2se-T=NYYqFuCCzsOQ< z3V62gTv6Ta&K*4m6oja2J#SSRxfDs-^)XQawLsE`y^7^rC$$a>zAC#8mUPPP@#6E{ zrsG)U4{6+XB!jrhYhXQO_wu=Y-2Xba61`_Iv%Bi#nbU}HVRd0%#FxZZTlxD(pC9gQ z?y!)Az1}=~WjWnoIim&?VZpRi#aLHj*ownj&nLq7!n&7N(f#^}yNeHD?{MyiJ>J8Q z9e3yNo0HK|L>%r6*#G%^pGzxzDHOwjg;&FYeEAWoI zy9^%H-)cesCXubN8HaeosC`+Wy#un}wCZqW25))M&<5`ly}J$UUUdcUwH%e_gItRd z8i?V(Y}P@=lCidCM*Ps!B(C}Oj64Mry2X!nG8d8P4L@LI+R!Pm5XgNvj3(0@XOHxm z^E3J}Y^4$?#6r|ZWXbrJ{@`U&cbP{roXw6pjK->5kFXnr+K}hcx27!7O;C3J{xzRS z#eapOs2jkPC>5q!)s6nnfAszxM=9Qk#4`Q?4>VS;@#hr{yy!N2UK$#@&wOkZf*9MQ zALG5eDkUEApUn*BA_X=Wyf8mEdMxtuFZ!G0*UzZfy5cwJSf;AT8ZFZH2vQJT zGX$9=?1#}!>FN+*$O+8;4WO1FTC*&tl^ouP=>Ansn)|6KT$(M0VL&?>H({fVlP1ZM z5n!{l#+3GY$TQxI7_@HxY`P4h3LX2rjV%%gapF1lrMi(!UGF77_7&rebeD9O!p=|a zBZ;<&)qv;v*Y!#8yY0^5nTN2gV`Y!U#3Ju2l!qQ7ZCji?!MlE6(zvBL-VdzrqtaWr z-&qZPzaEf{)S#2pO~|Sk8TD=36%F` zBa4FhO-l=Ru`*?1jp4+Rwa~0aPUqmrLjsj!8NHmaf9Y)gukjpXLayD%b@yO?iT>-6D;g>d3WFvLHY zUOq(SA;Ttb3_&Rg{_Uva$XoU;__8~|8wx2 z`z*^9Xo&-b;^z<#JOB)GDBq}12wb?c5PwV4b;1P?6U7M*V4`=l7+1^a63bjc7npS% z1vA^ikU{DLSMab1M@*LGulGc#{LkWgM1xGzjVTT@PaB?&?=wn#sCV_)&}lv<2%2c)WKA#lT`0@pS9Nv8$Z>DZS@R&mTwdMSM_>QL;oam-Mmg} z%;`T|XtU+Kd95kw(mV60@|8dz=9y*FAM}uSXjTP(K{%8R!RIi06u|7?0`q;hhIFKWC)&BEp`5oob*M(;#TKUbY^Um;H zvvp6##+*q-N$ap0ZyCE9tkZ1EC|YAt|Kr`nGV=M7mOuyf6_+_gU`MMQ_LF+?EF(Ko zlS;56_L1M=!u9Ixf=icD#dy}qV}${oz+Dg!^B-Ir z2Ns&$Z_Vd_&iL>i-*#+2=Lj9!n7o4F`|CjH;FXb>GeN}NaU1O;=Ez^-7v+dlMF3g; zE1x;_>+V9S3EVGl(~NI&q@YL!LaO+nER5CaXum0;hBsk=zhWk6_(%Lm6LWNl{h45O z+o{3eP|}+^lah>8Yt@aI#uVP|QhNQCmxLQRKKGI4y>^Q8TceYi-sQM7w^c4RMG(ei zC-|Jb;!_zr&SX*)|e&|h66r;~5FG>aATk>O;Cq`eaD>{T)3$r#D>9HdmCkXUSu zq05MwOK1D;yfD0DZjGnLaAC3SWku=AV(7hN)f*pXmpuE~{V@}CCKSkG&08xZytSU%QKuTLAW{zTgDD zi|>ZcR4#Pyf|${q-h&Qq6d>R87OzLUySr20zq>8^Y!G2=1wH%=D_BIRI0x~X)y-JY z4d2&`5$99)iuoIuidls}%Aw$*xQ$pM`hH|lTe66n&Y1jt(s zOd4?uIy=Gx^E!~;TWbp}nJR+%77;(P>ABL;uNCzv zuJq~mWXvASR`w@U>ORD!%~oE;mDTRN!%`T)kStj| zNc`w;(kIUh(;lrKy+V1Jw3Eyb;pq2;=g&#BErpYJ(wNfD){ahuTSZTFAI*O@-?V=2 zJ)jNu$;?J_RB1T=62Ol=fOB~MA^K$NVf7uvfjL3o_C`LbEmK>hp!YsjZri+qYdm(w zh@wbhk(K=4FB5Wb@#wXffMZ%aImJ$VX?nJ=KiQW629$QXHj|Qy4x@}NJEBGOcw$b? zh285iq&tK`!Q-^4v`){ayrm_}WM{32JP+--MZp4fLN_U5AS;gm7%(ph=bj1DZcPG*~A}kEVn6);s+Bmri z=E()Nsa$lvZJtccXqWx(&@V8|;Q#$~Du{BcVz-NqdEth(j>Fyzm9F+KQ)0X_r(8gT zn^uZ^(Y?11Sy`3>WX1(}$F>{f~Qg18|IlYbhUWO@g2 z-ELQ;zZ20-P^hUd+h}WyD&lKoCoKu(b-r!cKX;A$Ma1&dZGL>g{C}5kMBEM>CkwAu zV<^3TyhKEgT>T>X`r^O0_ok+%KKfs{ZEbCd2g#s5Q11v5gG=D!pB{vWOG%mb{b0z; zJetx{)6mm9OCn!lT@Iok2^{PF8$)*%#5JH2MEanG#xp(Qbi`-TPk5;QtBU((oP%

V_FG?*}!{_#ED5dVjz$)VG7j^TC9J(2kZ8qO;ahpgY z_6lS;{IknieOYukEV50>m!TUe>Te%e9B93aV(bC=(;~2ED{MdWZ6l45O=#96t{OaA zwdeRiEG3Bk_~t8rqAW#e!?w$ZZMb`T?TC;jUGzwiQMXL?{+C2&X0rZ97(&2=`{Js} z{Z)-F6tc%heCNoITd(u_n{>s@4NqE|pIOX3K*3xjX#^eAc(U(@EBEacl2m~cTRb&&y?hR{esD<{c+b=b>se)X8gK=*prqc{|y4Z z`29`~uM^iT4)h>L+m#etne)!+&o5Q)-eOH`^23&)bJOG3!K;1WHaP!BwqJ0|i|5EC zFaYdYcYcwsTGHjzs0FKsu9e)v!nZ&HDk>_nu(IZXpY7M%eJ%O{y!^>-8W+n_4#PIZP z&GIFaTNm8Nx9vCZ?<2(ND=y0}LQ_Y@pFVv^PbWWVW#!~7uB=qPW(T&+f;VqwdwlyO z-7gfG;4}&m?G#Lx%<=-Sals7}@vgC-skVg4l=HPi52L3?a8$@g@Nrv_bnMk%@2^_M z5>LMPy8d9-KmU8bSe1G#JC;*5TMQHW%aueDE&prcL4q`DlfYHSQIQjA@-6hB6p4SY zFxw^HhcLxj@BG1MAW+b)neWfjVqQH#71~)IK&muqL5}Jfeu%Cz@yU6$*zrS3(zm&V zez^R*PweQ!a4|97$4SZbMjKi(72sgYttg4hYI!1b7M*z3JT}&L$h6uQZ6U41m%wna z-@4%<#3tl(%8^`o{$99`DrRVN3IhoVJ{o^EH(x?};jMIBkoe(+C~?#~u%e7*ov}Ye zBK`QxrR68Idu4z-2U!uW(LTZ{#$F)0^dv?s&Y`RidwZs4W5IoF4Hi#JaaG^*TU#?L zYTDBw5#s=wWf9g(cZiJJ+q1f!f3I4i579ngk6#B(kL5_Z01}5$)BAUrBzEfo08sdw z%~7x1&o7s5M?w69^t{=PF-DZpD~XOhJN$NEE}shX=ENj_%r&Q!o$!l2{B*cqfjekbHnn|6rB_ zu#!sX6PCN5Qd4XkD|P#iJ&5%aN^4wgIHr*o7IS-<+UjRQIJ(X?9_@&Y7XRqHB%PiB z9;B$YI`V~3C~c&j&S%>2Cja}d7spr4q}|)TWWNxS6-lv_p3JpDKo8@jEEl5)Y5Ai1 z?thNw)8=y^0Hx|-7b*;7sTf(i2Q)C$WgS#-Kd{AOGBHr*z^YloSXbkRg6jI4ju z`(Vb!#mJ@LanejSTS>V=osv@grSk^w$VbG_=tDBGz5J$loGHH84x_ZZb=_Th9$9RB zdoat6#!|~R3cM6do3f=2!i(yW_Po8oQ1p}MFpYA%- zDjPRY1>D{de%eifk*HimCYFCMOI1!4@6R94Nlnv&ze`B)-3gF9YZ{|ELv#CWHNSdL zb=>6pKa|rO0jak%NGs3Z$a4&&ewRIIDxA>Q+{9|ml(Y}Zv8WUFf6Yx?pGMxdEJ85A z52!K0jNOuPjx^4xBqjdQiS#OKWF@f-ZLOXuR<<1HpkkE3Ht<|cZBLn^ZKsE~@4aQ= ztSeK+W@Kapydv`DOPYw~{i6*Up)++D8Is7Cgb)KQdNj;;e0=Nz6+k@p-#KKXNjo9O z0ilC<^ytw7*kX-*H;XP?$`C{5D>4-o)k5IqE~0x6mw$6eo^d` zuNj*;dmY3-92Q&Cpx>opZ>Gj{tXl#7V2~i*In%oM6KS{T)n+s0Qu~kL7W>0!$8q-` z(vL1^Vh65%-*DhwPxZFgOAkhRDogbD)7Shy|64@d@TPr|bp=x)Gaxf!I^*zxz9*&d zY{z%6i07JCtD~ZgLB^LH8HCH#a&-$ADwtCCe18K(`Z70VRVXDz8Qu4^BJDfV`pC%A z?aqgvz5eb?HP*_wA4S2@`ssOL!_%>BlZcuu$KKqq7x@VZ6N-^BsC>+Px0W4hYu#S|CK>yVR?BZ#DA7oRHO+yrYjE5?RpJc z@qzd~swdyakv9$4XVRG7Yp+1;aP+ykxznLjTx~=#7Pyn$!Su1eN^^ToLx2RPC1HeG znXP$dV&WNql&Y#Kxb^Zv*AtwWz!?=47T!%9csuK$drJ``J0i@xe=B)D5(HI|$U>(- z%+K7Fa5xa#g#STJK&osQNT;AQmebQCf$}wk@F+uttYoRlhaDJ|-cW=5S|Pa;z1cp( zdtQy+yxV_l|9B|Y{*0>m+Ox?#qWcpG$!mAh;^;JMINgmE@D6qMm36`{KyqNcYWy8#`w=Wa^Eb&C|z*)hBshVs#)hfOxl!Sb-X*1ygYU5h!c&w=-4 z<9ygXq{EA=l-RX%xn&QLuPz7w^qwKPfJOfZpc|69fx#dg+#r$34E(qJl#`Wp1&lK5 zkVs_-8T06w)2&dK=^S)P0c+eRaGH}_mO|kKppORonf#e;vd2!8eN^6*#clGDx5RMN zgI-g-!53P9-Tu4B%xyhNiTEJF8o2nv2=cI;7Wz$Z|LJ+{&b^!8?X{|Rg>(|4@SU+v zDE|03zgvzy51X$ph>C3sqRSzdG|i92^mw!H@pul0u3;wmig95#o(B3JEs0 zrHl_EY#n;qkl(R5R#tq!deL8}%uR_xHKj8vAowfy`QJvt>0ihHT(v*oW5|_NxW4$d z?dp3)UhyhU;_ZlCuLCcEQeo+5y>Gk6-{yp#uOy7zYijL$$%zJp&}pP*o&IaKmhbtZ zX`{=yvZQ?ZX-2BFlriSer~FwzYL6d2>r^;DP= z>F2oak^;{qTf?WmR@y;=g3RXLymJ-zB#PLg_4P+H`q{bMgbSntX+?Ler>(_2Rizb(W22pG#i#Sit>8%WJRz(71YmQ1CtgrN23~-W0v8W>XP5L3muj* zQJo#_B0mEy{jQf9fl*{3U`!z-8=nitS!0@@z<@iSz=oP2GhSpbxTS>HJ}nWM(keS9 zQN>9}+nbBQE<~O~!BA`WmWD;`B`I7%i340l7T0~NEiSF_tkNUgym?aFuLeNeM#h;cd@ZJ zoRFGz($=E&tbgEBGzFzZDNWu6dJyZ%K~9m-Ap>C;b&V*Y$kuV@(IfV#?7_P8G!Cse z9$wGnh-UYV1m#}gyjDqD`7lvOg**Fcat2&C2bgd&EJ$@LU6+>J7Y?xblx8A@*gPo3$2&r})|!^32~p;b z)wS1mab>C{mWHdT@2RP;SFT1M`!IfY7$Ag!w0yX7Iuqp2G^C*wBQ%KK_zy8ZM5y{G zc^M8>-}adhHEnA0orx!{raZ>Ee{a!*wXR+%zp%fz*h*)TUG264aoVXNj0OL1+G&X;c-YDxGF`$2?PG_{xkSt1K( zacL<=#dPGfT@gTs{QMQ4v0F!ooDrgoMH%com+XMyjtF~2Fx~(|Zoi2@MIU~`00Uee zJ|tnssa!eB(8YIb93fw+?aq9?BbX5bq3{TcCZZ_m0d-$U<)%oH>YT5 zNHIO#W&HRyV|;pA#?w;(LDr0Kdcc{Vsw`_)v6G60R;)~p%koZ|rh2wxfiPBM&)d|0 z_t84#<3#+bkku%snz)`JiRI%&)!YVs*84_ZUHOtu@3)#r$ej`H*R-ABsGOWUe`M7g zZKsv8^w}H;r-*TS1E1}zwPBHe@*GxtG?gY}06%{JJkxbMHGu+S%9U%KZ3MzN_2Fvw z=m`VI`KX8|WQ%80waoL79`jF4Cn#E$)>K(-WUE3$5{M8ObH);f)~0@hzxKNcdVK#* za2q#&qdA1zB9J-^Zpi)~y2qR;DW21}>vaT}0pbjtkVay5kaTNIM#ch9`E;uc-kH>F z0Fkr-+X%wdzBnKY1tb?93kpswEL>e+=KeOv9_b(4IP$-oUs?x`+tl#P)KoW&n)#QU z$cR`?=)^Fet+NEk{uG_6+p+N{R8#ED|NTYt8g?Ch5;r`UB%BOMiaam>yX(|`VTFjr z_RLmifozX=im3}hx7Q9oKE6i8mO!L`+2E^S$g&3c6R>5OzFTFjSBFE-tG>5AR$_VZ z3xrIofQ77&)_bMeQ$4x3JnOFTQ zfL)Gp05x3PPZdI8mrW(c$rMBOnnCH+otKJHj<98ZX#KRdrlzV~TVeWzkRl7YxrhDg zMEM9)iDE3`BSrOXl zfa20W1D1(Bb?W^jVCS$ac0((hsFBFHv~gOoq3iRtBKVmd`?C+(H-9i1C8U$uD4l7g z-GgNQNm@t&TL$O32UCSC6wPUlFMiin;)w6I<1w!{aPhqZdmy$!MeUT}*N(m@FXz(H z$8LeS<4?{l{OTLfVN92k;^KqyVBP^fMmnFqO>Y;sej>fZb2G+AWqJH1Hqy;A!zl3H;0>yUBQ@ zKhdur?d<$ni=B6}oclz7(fhdlDeRpQFh&;2TtF}{j(E&}c3g!qWxm+zgFyG_)UOb= zZPCECry+ul0IXbM&i*4pd>aiU=77wO8{V^vuU4^v+d-P1o<@+(M$pi$oW^S@(O4IJ z=SBe5$LR~RbGl+9hH+aE)rTEzP?o|%JOR@U>4g0o=g_2pgc&p`ng7N zu|Dn2L)Q5sn1mQ}ShTyNLP=&+c(MODBoz0b#Fbm@WE@Xjy2iFzkfW z@wdZ#&Y1p}bbL1-0UAlS`(&qi=&ny&(w(|dk&FmvI!t;^71XRFHQD@Ku0Ks^pj$~D z#wM}r`(R?oO~Yz7QoSa;`!pa+ zo&nbLuv9jyP5tNl$0HcHB-o1W3mZ~N1AY?EiG%lN#^_&@-yR}e z6wX{L(TP^(^L^f4tHo79KTTA#12|*^DQ;eb$31o~m|MwhuFb}sam=l2jb2ByjMgui zTyYTsE_7=zh9&YgelNLil@$Xi)Qnbw8jZ^dLMkB%7IY(pX^l`uhPWy21)jB}Zz3i> z;%3M{iTDK~bWC*d{njRbAm5NU%JN6>B~&a5LGj~9gDU9)jTm+07k#l*2=;p!OgFl_ z{+eYE1T$3HxJ5xu$adk(n`YzW)Vt<3J~PvY5TSrCtOEeA9Hm5YUec7{6d4~L6CWnQlc-Z^91ODc9oa499Ux)3`THD#;DncIcaqAMwsazlp)^4f+ zk<$w~-bL{LAXF2L2VxcyDPfhzi`Y7WzTk1=PcI52Fg&xtH36JkfrX zwYR?^aZA?n^QJ7|I)vsb7)oKD1q8vH(+ym}JYGA`J?ByVMc`hgl+3ZR7j*js385Z_ z9zRQnPQPrn1l<8!(tmahbv{Q-H&7Z{HtGIGo*}!##3Yo9G_Sla(?<05`;kfQC-V82 zZ^F6o@>?q{o`<$h5_VWMyr?(OHrzvgYkqHVC~{!kEp>$H(C1!3r@TUZj0=D4xQ%*> zLg9q8**(}x?W5Df9%OvuV6urMZE1Y9u(sWR2wOhk*$u$NiS6Twg;GW(86GvixF}b8 zLl~0BBG*D}g9^nHQ@Zm4+o$JJ+B(T)3?8EK|845*tUXFzvGWRIc>^;%UQui5Yc*oc zFH}s_W)jCR094hMeR+5{0c?@Vb@$Co9&XC<&%FFgqJl+<0_>-f)T0EBl~Xj^avBEP z`dLbVKL-T{z{=T_R!v|KGAQKF?3PrO#Zsrl{UdJ)4Ub{gv=j8_&!$Zwgcrs?!+yPU zh-O>r`GTKRzHovx+E+0OUFYwjWNIcYZ=KJ+6<69Q9!zxm>dDVC$+ml21yyA@L+_W$ zUogmuBxaKz@y1+6D9QrANbd^4%jx01F33?fjvecL!S7fb^*tnJ#>arIM`i#~mODvP_%Og>c#Ic*_OE5y4S-)bT!RTbhdKK8d-9Q1x<>s#^4)_+_tQu&Kjh={TLN$ktn z1Oo0y&Uf`EKTSbpH2bfsf87^uFfcS-gcfrAm~o?iBi2;+VT%9>g%&`xFZga(fk=Uw z-af|{dm8ULXB#DfU0h~Ndz%f3%Zsjsm+5QG( z2y2v+BSbSgf@p{EhI1uC6abD(Xl>M-&9j5QtldO!>iIVoZSwMc_k9kdYYZ=J$~rKE z`d$7t5JwJrkYaIrCdVJzQI%vJ*drnCyC8&06-BWiQynzwS<1c@28r?{WERkK1U;o) z2A_Z{lSt#b{;@}LA}5W%NgDspaE>nbuS)I%j|7u;>U5=X zstti*7K%4wi0I_h1U|aTWfzl+_0jP7vjpAv4Z3YsM$ZSrpNWVtJp4*aJu3bh`%gnWJ?~j~2`JC(;hJw%*xA_A-CJ zjF)fDJk|@3w&znfsa)Noxku^w-p#@5OkVUXwqGU1z@~isV3aN1pZfEeqJL2Me8&>!XD}zbLiNRY8;=01J$CwPP)@ znr>f>&{JK7?d4hOQ-2?;<49Cy0-R^p-JN1wLW>ceUwZMi#$6?W!L2#dHxTEiF7TW2 z^57Zit2S;C_iicERtzx4L&6ICSsjfY1Vv(|21qVoD07Dg%Kg5>o%%6?s2+KyLm`G&$SZ(IpPpRLg97q-(Ru2xjPPJmTck9aDg=-H24uSB~|wENpC= zGO98%G9QwYCw}~p2hj8H%+eSKgcXRC#+JD!_vG>mCv1UNP2)Ggr(@Kz6*sz!54{Qk z($OHDoK(WF-eY)Vm|JJUE%IOA}i4kwOU2p@hcdw@%Zl46;*VQScMz*n;RQ4edrFh!K35J1K$O^bVL#pn)pF_N;Lwop&d07)Ay+zyivhJzR@0mFK8f<* zw+DSRFbx;F5H;u^(TDp9;-h*r`a~^jY0d2bs*3hO4PEcKt*fJ>1<{~jK8NoQL>N}P zpwvR5{wH9+RUjT^@a~%UAILx>3EY?wBiP7uTIaW$Fya%3gvh~!RHoYJ8~{o~aK-zf zN3+lL=8y!Ts6cpr5CTRd9O2)1f7{MXK2!sub8s7_3}huXqu|HN`Yg z7%uhZ3r(&LOYK?Un|%v~59s>I7o*k$_3ev0T--_zMi6Q5Bup6AzWL>$cE|28R-EpZ7d`$ zEg1nOaKPxj9v0_(frl8#pqW<>+My0O)&M8(kfj6Be$G?gGzL zPmQ4Hj_OfyrZ|eLr`|p&Q`$NS%0&WZ=)q#f0ns+qdZ@^G@|01t(n91~?kj#-ii|f% zWt@>dBECPPX?`l!3@J%DCWTT0*_!fL9%xTzNqj!6Du=OkH|byZ(zosVt=xgc9cChY z6yy2l1Z|<6b~%yT*`{3npIMaxfB||p6ipe?k z3fC!8mvC;z;7oV+2%w8)Pvp5e@I@R&baG}On#D- z-7b7g!4*-0GQw~38w^|usE1$wrvy}b0kS}-M@s?(k_59G?#PHFVML?f((TO|{@vFQ zZtk|*FoRg<7#JGbLYL}~c4AP6V8Ga3Tr~0Z^GlNm5+9SeZS8IT`1x}(NDf>fgVGHg zFE?Yeh%Y7Sj7Uk>Z{zMu^tKi{+9sY0wV0yudJ_@JMx}NuGUBuJ`In7)w!|D0Y50Z+ zGKP0F9a$nP=OoXX4ZZ%$Z~D7k?=)0fKP8{+-LEueaOpN8jP(_(5ldZxSxt5>oaz4o zKMN=O+BnqzcD4WYm0_AP)A)9b=s|JERIM)#mGiPeTn2=;dwD7hi~f)d(=wAw>2@@& z$1)7mYwSj+kQo&nm1bh+vrd0)68H8At^415A!E5*?d&dp9SB9lu1-7nOTm`?e7FN$*?;Z)CtDT-bIfYKf0oO8iW7G zOF?wsEO!2@Q(m2N-$VmlXKD14+!*rI$EM4_(_}?18NL`1q-1IIT$KcTI@^5DbFj)@ zA5DD{MT@VjC?6Z;82TsnDqQ;Z2vG60u zzyA(KS~M7(;`*cnGp@sWUcq6rh8uL=_4RZ7E^_Je)U)~S*?Ub)uZ8yz-xQ#0SSjEf#U@L!9Oq2#s9 zCl}y;(5e2`lj1Q?-~vl;99aZQB!@{mLB`0S0yAY7LK8B#unO7}zf7OH@5cVnOtlZC zyIMLrCo~Z=`>#YzpF5l;;UT^4dU?_SG{Q+ukR?<-KCTljUzxVqNbH6p(sus~WHx(2P3D@@Rxqdsx+m^zRwIW?~lj|y*4es=nK zUs6S7=-x1fH%()407m4RSn1Y-=jX>U|Z z=m{b|+D;Y++44&x-;%=?$hHBRJ5A-=K-0mdx2!gIIz5i}WpBhLQ z9XLUYG3|-6v%@TnF263%0emoq@I60EGPID_qb83>PC^8?E?TAi;t~%E3=dv~OzAw~ z|9vYN^n+b-;NG*Kq>JScMRBebOj%HPn%(#EI94|>k_KcdTYwCUG#Py!-3t9$SMN(w zXzVGB!;fJst???wxtE`6E$4@8E-P5ZiH5R&E=5~FFr6EY`cE{lH_N#iRXq8C!AmXB|(zqnXLkTghixT1}JJ}gnJi}s$vu$JoUc4;G zJzSYu1C?&Vkl#T<`Q5b9wvz;_pjhY0(PO*5yyQRHl?nm-Phm5 zM>dYqy@bS&BcmKakD{Z#iYxO`pMw<3SI2{QKY*JI`vdVj|NO8L!J=fio!>ixJPp%H zF7+}BFC#{U{Q1qI>cQ$Sq)4y)A7Y`2dHOZmc?gT~Pl-&qQ$2VYJz&#q!gKzUjf>0r z==*N^#s54<_s`wnvxHbaARfVPw~Zc~nv1^Mk4$_w84+*sH=VT2MzM*iA`u7$JiK8w zKvJim5MgI;D2a^fFO2afw^q*4*?D8PacLbM#r5@CSz|A_A)$^8unWS>v2=6VD17`) z>dbrygz+7n$6CWv4dlr*$VDY{bc35YL6*Lb8NUcB*L`4 z95Lc-YfeqLPo)({?T^Fh-tnK0zUr6%-ZVZQ+s3sNUSs1cF>zcOXqhl_t0%Ir<&71x z{en-me@&63IGgvE5}h3Qz|G3xGC7JePZrlV7yj8Kivs&?+gJ|vDbGJC{vLJ4VZJq2 zNc-Ae`r|4rpVyk&o`E#UXP@!4>7tO(ZN-puaqPI3{memsg>axp124`;A8+acd;->{ zLyz5~`Qn=~q}c58XuWPN-rK!ej-adfWci=7jQEj*MlSjBVr>LyAvpC4ayfeIdU00G zd&KNLf>i8+;l~#i&&F2(BFK@3Gg*B0Rxk<(jmUZuwU0I-dag-bKPzc6e5Lg7()1pP zLk2AJ3k!cAS>KS*=Zt~T00vFhnj2AX+u46mEP_}TB1(~SU3gI+Qy0-FDk%ZVMAfSe zKwMustVwbM@nHlV{wavNVNbs+2?renioF1O4KiJm^h^Vir0Mbw69+5^p5mp7`tWL% zs9HjJfO0y2L}(~lWK@)n9H=>hRZA|zs`XV%2qFBT1nrBCUE=^NU^zkck( zFTZ0&qoE9FM?PLOQRE=rm>c3!pNUtses!AkxM}N`yTI?0Q)53onfT^D_Fu1UI7Xi+ zN#qtbg=3D2>oDH>WjTHMa+&JBFfR0y8m!opaYAof84lL-8KcR^6*V;QAE|`QNa=?% z?p1Z0=#Nef5QgsRab&-^nctiEzR>?riwSujI9oVi2kNa6b-TsNlzj+xbvSVYi|_#s z5mGDE`VOb^tSX)DO1yc4^nwdZ$5O690|Y+Jo5dMT=f9#5vJtqq!D8;t@_)DF5kKD9Z|K@EaPRxF5OfwKSc#gjcw{2z7G?e~#eGv9MJ@kU|_(iMOdYxFF>h6okCsE;YdSxq1q=%JMqO zf^kW1$^!TWO0yM}o#TX>7fz7%M%zqiud$_VQQv)Pp}CDQ@)^!q3aoK$)2bhVje>RN zYPhR%jqgY9A<)aefgpX|@(|`Y$AbB_qO$hTS9I^jZE`pvVQlAefZy7b%xk! zGf;@YHnEFi@Mk+MaeaiHpRh?#IP4=4AjzR@#_GZXM2H$H+1J&vCQu%$^UU+d%|#cO zQ1?;0_`mDIV(?`ROP8GvwQSZRlFsRU3w@?!`LavQZ?oIQW$gxxB7|l9Wy;BP6*8`b z9}PCQj=u4Bk4Pq4&b4HLeG316AMBpVWrsuQcq)kr;m7$(v<%;pMesb(fZRUbvIrZY zb(vxLz7ISnOzlVq-6R%j#T#Z#+h_5l)3J&=!4SYQf95ecryVwuPo<3^v&x`lo3s;p zBNXgJf|wm|#50CK3WWbwls+SAlUI?^M=B=7gn z$}nfO92xW=Ey(eRyZhq(6_MwK}?l-(u&CFMM#q%!w-k7PI>Rv)tK#OS(s=K(rZT>y__FcP1RzH~TzKidb z5ZjP8b0jhl)d^)-1@mb0E2pTndLL|0nUm!o4)DHV`DR12Am>aQJ#N$Cz^hT0QXnH2 z7xlT=SG4^_(83!MIT_&|K}QM;ef(d~hnXmo=Jw1)lO{6xa;1YwJ$wfh(FPbyx!-3nPeZbTx|0esvJfy_?>7^Y+XuzYeeRXjih?&Ei1 zHD~_scoTrz3vN;vV}DM3Pfj{6k@HSF>gTN6Zt0G&T5mtyN|}~uvQE2sm!ACnc9zeQ zFt%}NZpCnN^6e}~!1zHA7ylB;Y%hs^ifYN^@yX&Xx=GO7aht@gb#c(#=hFtqkj`9# zaGd9nI2N?H$`t49ZP<^53n`r1Yp1tF*QDA9gWD1$t)RSeG_=xw-`;GK^rRR+TYP{IYG>(P&+I~L?k6dQx^_gO) z66J@C0?Q*DZ~~8{ll`2?8AK$m7}R9SVh0MaMNiW^bgWrsswSpcv0q# zvDY>jn;Eur-x?Q0$NawAbj_vz_oCWsQJGlRanSuFjfsIQxti#r*T+gElfYXdd96mG z_N8Fw+#{baKM7V?{P%b*vv~J=-Qvl#uv241zB>^&uX7Xyw1;CSJ{-EfwO-xyI|k%& z=yM)D`Ir422r&6Px3od(A)}|K_Y~8ljE3IBope_BEqnX?%RV zvpa(D0x=vPmPFoqDyQXZm*d7L3gLYN<&3>OJCp{FmW+b%R;afs*=m^6AWE*Kb zn6_q7NccGGa5tori%JK9W7clvXn@Rn%1zp9#zU{bTf_MrF_Sbb>ItyqUIlcTl~?R! zdan;`dHsrDP!qqrPm<#uYBg_?lY=|s^xOXW8L~pc^V1tPatgaNG5hz*xbNt97A8A0 zS7UoOaO+^Qty9-LaQ|BCat~dFJZ^?$Ia~^7;*;Mm+}84Fwo}}RJ2e8UmR^54FPGWbh0gR zKdw(J9EmhCudr=sO;ce6>{(bI?lefHpz0De zOSR%5}?Wi}PUj{uy#1;PPdpn(uzQ3qYQ zRyX~+5C6@t0d_$FI*3pe_pP#!$a0vTv0!Pok!UOr9(yJwg_0+q`JArj2mXj+LjwnW zI33V}N~^T@I5Fs8L_p?1s7`uI05a@K&1JDj@K1dF`KNRzXSc0@|MK!a<_*;5^UYEn zaHpZ;22J}oXk3OU3TulaVZoJ?!jE(Jz#?6pq57dNeeGRZbN(pXK4`ejUM?F64c;v!Ty9cEagb#2Qo%4?-O1xX+9VYk76Ppwar>| zFYY~9#ta1y?hBp_*!J;XZet^iwmNQlc+zy^cP)#0EZl`5jG^ft`&^jGkci(;ATnvg z2L&1|xZH%u)$YK;)8z|M#%120Ki-7jkU359$kHO~2r7yBUaP-NpsO%o{d;^q9qFr% zCiZ59rHIkM{_K>RL?AZurPp4qbmR+`!9Vy0ysxbNaH~{2?Q~jb!;imQKOmX4sjSjP zWr9~K5PN@OmNK{N#ka(PH5RT%4=H0RRPTfKbnuWsnTbA6K6)^cH|YwzY2BB`Ou&Pl z2~D-~Goks2F*G zD)mW#weLyVAYK&nz;Cx`YP#nbZG@~?OeZH^FRL1GJSz|{qaKb!fB|k z5^VdzufbTsy4rAtQs#kXN?Uoc7+T<(f6&dH*KvC4RNZ>0SlzTU@$${U{|4fnn-j!> z!F$n%86G7DJVKIAUyL{r!v&7I9>kgftjRi>zAOqN$6{Cn$EK$zB?0XZ}Kv4h~% z-op2`C0rtGtX3K}rgQ$+og4RP+N1klmNM>0M5)nlNI1{6zeO=UzFwR>^P*Wu;g(4# ze7W%ws=PrDz>Rnys<;486qS`Jh<$%&*L>%`I&dCLv7(3aBcD!`?&9Q#u|%_S&bY2Q z%3Q7gV;H*?AF(y9*PQ1)$GSZ+C8`CbDS`%V^*Ji7(#fv>8?f&;EF6em!IjoyXrCnX z9eOB!?bW^HQOXQTOd|QKte$1^7$y1D)sgXruxGm zw-N7~s_TQX7acmpv>724f8`&B``ny{#!_(deRRm`U)bzIm6P~q9y+H;CE{V^6?u38 z`kbcA`HfvyNQ=tezxNLAovgH26HkiXziT(zb&veNf(%7E~AT#13BIo{V{PW3OjJ7PH_ z050aS_-{;fT5k~1MDWjS+B$wa`b{BR{Y37)c=cj{+{ zHRPy@i1+W1^Xz=PT;@``L1AMikLpVc{QVLUGhvk&L1O9`CEQ1?qHoLk#LF8GBUI?u zw~*4B3JuKqJDuEIt)GLn)I@OPf2x?u4#WB4B* z5U*2u^Vhz5!^kfF8`6i7UX1*UOLP65=djcewi-va;2liW^5n<)fo^-?Nl*G4XZ7BG zQy+Ky+UJ=R)ooZ=s$oOBYY5rsh~)--qz}T`Ha>~VjE7;?faAU5<2T?JO)Z>|{nrGp z^#l<%bR)M};NviT9(@HQC05m|JuoYO9nQ_%2Sfiogh;v>5;+kfrG5)uh~)-HOxdXS z?|8X7Xo`X{7a@HoV+z(-~C8K;G?mS_-IG$A7 zfE6DzC5}7r`G;U9$pJq?26zc($UV1UtzLX1Peeo%uUG^!vKGbApzksRyIvJr+3eL# zZ{TqUYEwCgyUc3ANT~g~vwa_dIw3y*1*ntF-Cb}Iy29&%1%D2r#R~Lt01eR|xQ0yI zfxn0#LYg~c1RCXHP}d+MW@-jX8T1AM5VngnKA_NWZJeEFC>xWLi6yt!zs=7_jcvd) zmE!&a)12%B8H51>?mhS3J#(15MGOhHz_y;h!3Xb4X#6egh=5uC6*0vfb)qd&F*kcA)b>fhp1tDdBJIhv z*DzCoVu!Bo2TgqwMTX>V8~*Y)cZ>eD8u6XaZXB5Nla{$+eR&8*ZxD0ZojEBnMNO{i z@$QBxFV&WWt^XsR<`vILfi}=G4tDr0m;U}bL(^+wqG*!PZ&Cjt=>cJEFBqsnLNcjc zk|_ZO_g{1rJ%a2J7GIWxpXxWJjuB$@*;H20;c0f$q$v~gXb)5;BPlodV7g3Jiq+uoa3ww_{*JQcy7b-I%8verzYxBKOxmKkLvGzb ze>{CNh@_F%OkR{qbu0z}nj2b^FVbWaZmapK%v3+NDOI5a)?kqP-L{qNiUsVA0RcE* z;g&OG$(x2!oj+++Z_D;4>}F|ZJ8us$z>0CNXxI`yKIbS8Hrt!IOcy*BzMd@K{n~Gx zBe0iJwtmYW79OBNK{~wP_rkrD)cgg-rJFpc9r(#>4XVoV3IVU4`|lWkLQyv=UsYRD zbNy*7=y0ujCCj&PD2Wde)|Zp$bWE6(-r3K@H$Aw|G%*89x`)mNGyO&{S`W9BKy3W0 z$QjugIkMmnq{NXu9vu#IUdOBF-B_|PKj72vr8fKh$)3Z*VYGhv0Bh-72Y>8FuuFf& zu(Xugzj_n9iZ-b!IX?kKt(%Y5e?QEU+LyI&)qAe=I;wlEgs9#ltDmf1d8bOu_dJQx z*CSO~q*_?(;wr5X;i%|-pGUC%mD%N|uvNCd)^aY7uPi?s&Elm;ax-7Fa)v{P22Ycl zv-*14JK<{K`C)fV)Y!~IBf_9Fi{#ZfZkO)DPJ0X0;G7%Ap_+EbrzKjyFhsHlHaQ%A zWN$nnVgK)rx1`zichNkApf#0@yU?hGQB4H1kGgncY;3LR<_@1o*01Lu{oDH0sw;ly z)_7? zCXWUYP-s?rHP$4G-4uV#{@1zDaUNjyBhP8CyBZe1n#Nl%Mm1wIgsJ>2Q85?wd%Il= z$Rw;7TIO0(&GdS+TYUdLZ(gU574s#hH$dydzch0C{CA;i$2_k1(~{%y&o%bi*~$;Z zL>hX0&xad-x8bJwUGaopi0;)G^r2d3MdJL~b=NX78vYme*!^%vKmGr>It!>M_ciLH zQi_ChcS$NBEeMiQ(jfxU-6>K6(w#$xNQnptNGl-?BGMpTQqm3IGxvVYT65OD=fJ=) zy#M!!y?^`2{a9GH#fd=M(EZN?G1R$CGa6b2e8ZMKW8hMH$uCp~N7QXz@H!pr3|7d} zbR)m)*X^ozoYzP3$CgE<-c#cK>=k%G=rSZCAnP)uiv$o>T$(7)_4O5L@j#cP)6dXt$^(5} z0{9ECZHB+mJ{Qh}MR?XcQlD90e}<*oAP)VY^(>;_90Z?KnjDTu4`dfZ%s;q6+MuT) z1888=HVbz;M?uPS{T+X$Ik0NvOs3+_!E-Fb%yJGHLk?OO%|cwf+u-`2hv%mwk8%bm z14v20;Sn-f2)9=eWZE>&T?gW-9;JIh{U3qaeZ1Vi^J-36cyEt&MOW(ol z2u2&JeDd8wb)dSb0}YO*w9yR=VnDM{9?28@uwe_#q!x$fwEl0;_h}d14^^E?(z;=K zL}LD)^wNyUImOMd$D{mvDZUw~%Fq(RkTOk9G;GB9dyW8W(p8k(Uk@(4Xz=HOmLk4? z`SZlbgSt0-Big07#p|}oP~BHNWeep+Q0_{_lglzGJNU2Ay}kK72(0>{ab;1L?VdPr z9xLO0p7>HCQYi2Q+OxAO?057|l5v6wol)Z!Ny;Ek8Lwe}tuF>Xy-=ojQy$&4vnjkN zrtSzYKy`$_^i$8Q-MQ%kKZs%{8P*tDu;2!wM)<)EYId7M7xuAosID4mdtGrS>Hf0Zb!>(0~Ld|5{ zl?nFM2%6FkaILhOS3?LTEGc_>!iy_YO84)_VfjYNw0+(u%YhMdZq2i(_AXNwz+1I9 z$|*d>hK-^G-Yyyg>A`EgkT-+B0PCiRF3~sqT)RL>V4<6kIUyi}_PK^BLjsZ9hhLGF z)XLwWYhcT!ZSL-zXq&Xg$^6wj>^qlELs+CDc+|ZN z9_YI2?=ev9h$TpukL|AH>ngW=x<(jaj3~VxxcHE?)Z{ncmR8L;{A$x(1pbHtzMaO- zeQ{5L+UCQS1rM*XK-HPu4OKj$b!RM!S%MSG;`aF}rEwd##(yDqSq#ly4GrtA81M_) z$HvxD#x9Ao-h_x$-FoGv>rXZRrdWQ3dLdkif7Fm$7LC?;pPq>_kl(ePWOBSbC61Iy zz0Q(Tk21*Aj-StJSw5L3GB+^f;`yXB_WSsSEIi zclDLi3_5GKBz$WWKqa?1>_hOMT%-)N4pR60+o#~TD=I10@cf&le@x9PLS#nHB@}p%fVHLTt+{(zXEdM_EBi^ygG*>cxr7^=qY}3k$|j70Be@Yne3+jULCa<)?tV z%*h2=i!pLbfK=W!Ru*4JHA8GQ?4DhI z;TPa>nznR%x%D3v27Mf9ezfMD$YQ5>wEws*`g8bm`B5?IpL2R8ljX=WrwWl~T2ZJTk z6(1ro+iH$?o-ikzwhkT|-d@*-85P_RA}Iu-fr;Vw_YG3E_;uq`gT|w#8GG%py{Z9W zh3b=8;Nu5HQ3&j{(OkvSfB_u^xLk1K{kl~%wYI&z0+Ya7IeB@KF9&S-_o3t1gGUKT zGLnP?gY8O?Bm3?~7&^kZ)kYb{mm)aN34Lz`m30`OYLw1^=tvn>YbeN>8rbui22YU< z4!<3^+F-#v@UWAz4K{53z(zo_y8+1~AP5@E->03CWNnai42=ua=k|@eGALL7Ld4%# zyZP{^!wrv9ACkcTqm?Z#%p#W&1=(Z=tOtrT2W|;2#Xc=6FK0>Vb(%0s*)4PMi7GffrFrE12%U~w9-M`h zLIG#iAuEjoE2w!DjNm2!QR3^&=;wSjgZwZ@1w~5QM(HEt&}e&-YMch#YR%#87)J~08>dkeW zV|32kMHgJZgC&y?t=14NK;4apg+pxlDDtPvM8D>=?;&%IFJTVM!~v$3<$GqelI4Y0 z#m%$dUjA!yGS=d(o7wmkqRd7LN94PdRdDpNLAR=E{%RhL&ctb)oKJcH)8Mk4Op+aC zmx({jG=QlPokh5!GPwHnf{#Gv7Kc8<;)lfK!{M?me6E)EDhI5K6LMZ@trJu2tTzVc zqq#ke#?)5_w$TnREsZ1X{_Z4C9DMhcj*goGN44|0Db9U@jnYv$ zoKL<3kPb2y2|zm7_`|9e1r1z;B`8MU>?{qXa}6I6l8_|GF$$h zKGqsX+GO*NsQ{Tm4oCRyCwsoDGyfV{d_T5sKH$WApOf>|$XQgMn3hxG)!JhJ8n!Tu zIfik?b*S>En@g$6#Y1rfH+~DP9SToV(-;`yfvEAI4HunIZ2k}}LTK?l0YIZU432IL zXg3P>HKF@f_ zbQ#4Y28-kQg`iv>^kyG>U@e6ONn*zEgFV_9Sa!a+>=-L{9xrBPUrZG0j8|O641Rk7 ztBvLBtDWmB9}FYzH)I$nOXZlT60v9!C$0N9Mq{5|R(2T2-I%cC>zTb+5m<)3N;l}+ zGu~{J)%HM19`7ccdAAGq@j)y{ZzW}vj9K;|3qZ0zS+u>w=Al)|7Q4*;Ly|pM*z9{e zm*N#nRSB2h zh+Wrh#WKoovDZ#<3OssLJoWWoE7o$$)nWM__*Yo3Tdt3)$J*5 zv^>@xkc7r+f6~9-TglOKx{|&J68&X;uG;JuTHgv^!R>~JcIOWy*O)Dn+RqMiHuy;N zzcKKW`!g~jV(TRzzXZhVN&!6bBVC@PbVPbu^7d`<_Eob|#>W7G) zt_BhBWes5lkr`j1?7`klAN&3@Db+*|_k*smrMY7m(9~6pajx!rA-@t4d32sNsIZs( zGP9C)WjTYkYeqRHW5Cp`uv7eFQxjKqhGN0U@w%ecpD+1OdF@!i`Ar$YvaCaY|2*)X zE?#0Y0qU||<}(XZSYsR*(sHnz1Ft8r)&iQf7KLSrpW@S}=6_J#kdOStc>oh^IW zAIB&UKSJ+B2JI@U#QLJ5*7p9%vJoS*tjTR95$CtG$X8bN^pox_A7TZ@Pp8r3x7@SJX6LyKYtM~ z@><`SC-e08Vj8?@U@aFSrT#^8c|N{+gq_|3zZakHp&zKc>(74gbfQScr8b&b6s}Ct z_Ew6%?jt=-e&Ne!+w@-s2-1IzBt!X$qf9J2cW3i|d+$BUGbU6J+<4^ z6#nq4Bb@hIS$Oso{GRptcz+q}tjJ*;JU4r`J?EccEi1`O)puK>?O!qH;4c zI+k9rQFl`$vkw|>rAvDa4U_3^`f*HD7{nU-v2qRyt~ry%c1BSg@2rRV-Gj7R4nvn= z(YmpO6lvAOi16?wPLVH*?8@1Uv*#LGzP_r;2D@>;#4`sPFBXiqc-JnDQR{~9ciYs4 z?)|C`9JI>vXB7WutaY_TK<4EjdZedygwEtOY_IBTR$SSPD}HZz=Zc?`<(S;SYkMnW zdYPP(7yh_U&kPHeWexj#xVcqpNEDv!Ib?VBwD{1ov#13QPI$lG->Y(1U6OhhfBogL+qY$Q3m|)d)=5yTYqOl`chRn66S69a^*Jkwxe$){p+{qA9>A4IHoKv_ofF(&QAs&w9yby2=H=_AY{Vddgd>b(S%mg(Q40%Q~Fv zuVoi93wC;`%X#?OG2dQ*JZ|r;w8FBZsbi#ivY8NJbQkgq4C;+X#ExHWy;IsF7;K`) zc=}2=Iq5v2mY6$5z;1>P&a$2L&13;3l;9-zl?wum8Dwtli@Pe-MP!Z)U+3RZi~}nNmMA}Vju#yTi%Lru zfdn$PMCm)S3u7u!K&C!4L62v2f;V6U|Wld^tHbPAp@^8w!n=tewyHmx)?h%O8$Yq1APc!>(17m`3 zWCXI-t%}N8*KhyS5b?B&H$y}l2s}Ev^C-n2-t2XTfgXj;ZU}uM;ne55#lp#qh-hk- zd_D39Yi_P0w?JRUbwlY;Pr(hg8M;<+e?4Ofq=0qKH==2ykB1J#p*XV#h)B;1I(U|e zng*m*7|Mqw=bQ7EZ=n>A^SoIF+2Sq$^%Hsgh3|LC#8$3l25jh|634FAT-wKnREK&x z&CLp_I}Wt%Yc(>WSI9kwk{Oba0l?a^F+7zI6?%Qv1;S52oV|yp!G8RBM%i`EvbbC# z+3X@9Gz~3#Pyx(Hlkdix&!667)Xi(Ic{XD&oX|mXclei1X@z%+*_xZzjUsAen`}9@ z^9S}zIo3R+(Z&zcF@y4~e>BpDdR|CC(8ciLo|?8@=N8t`^9@?UC}^nKOnKssGcH?4|K+0#hA{|QNdUp)f6Vq~{zyE)ax zrzNJ_G2C+MK5l-A(vRS_Nj5zCUg*JSa4QYHj)7J;;CUzd8QbK1)126MTtUY=au;41J<6 zR({^EIz1Z{pkKjY|JU2o^I4aTm7QJoO6m3@PNg~k9&~ldz=uLelCqa+AGzBZOe2>c;HHUurNIeLA>`R+w`11q+o zm*(3$Dlc2&E>knn8wlQL&8%THGDJ!WCh)`&qIw&NC$(c}%ECQC9d~VN<=$-B5@afe zooDy&j%DkRr7_pd-HHp#zxxFz3#h(r~T#%5gWI09(H`NZJ@j%pq?$MPsAdyUZ0k`+q_e0OV z-DG{B23!BV5M{HW_+z0_4<3yS3NSMw2#72Rj$%jj(2ImK<_a#_XHmJB-{!xknJbdNHYk-#mcw+pc7MM!` zkpKyeOxKe**LT6C%uRXYU(!943tr^pWW34(7u={w&`Y~9#11?L0Ty8K5@2<_iS?4B zP$e5&o{eqNBrn_HK457@X8&||M-rqUG=3Kkkw#GP5W&(Or6>3r{^V9Fd%5!K>P}ro z_a=%yeKLo&&YOSVF)sdUx6tEZ#tyDR_p}V_@a1s!Pe!2HzW&V*vi6qe$?w434sIjx za$RLl{<#(==Fs^TCrf>sRh1Y?X9DZ#^k5m=TveyJNIAMR;j^t+i(3HF6b9#EQDdIdis2fDnLefLB8=H0kI}n zQ5-dri<6g8044xpUyXy+VzA5fE+SC~AkZ~%$oXjcJdz<_r)Y}3=d)fP(sVJZ33G#D zAiC1uj9WW=-@*{d;d+%t7u)$USg>SU7sp$pNwe-UnZo=H+ zPidY!fZZF$%LKzo+TRk*hyxw^+m*((hA+%8bSt`O*wkPi4S_;>?JG-{l)xew~!{kT8pE#ss2Jx zE*CHn9@2MEWp68J2ThOU``9tD~z0 z61kGuQz|}oMAQzkXXGLko_Ao65jPSJ#Brg=mrE@;wcK)1`I1aKCSiok`2pY^h1=U- zn=^)cY`tMj@ogN($F3q)CFNRSPV(`NuW!LGYiME-HTqe8`6I3Iz5sEhfELM_?ts%LgOlne#@&JXx=GwVU+bb-CTQ|U}N_fMX zk|<5k|3+?9n}KduD%XDy4gd2W*Aifo`{-@<_uVBA_jLM7doO3C$nQ|)LaF>8znXrJ z3dwaap{lmq#&s#shiSxGPKU5AO$;|UIQxE|{5}R&Z`kyurQJ=nBi7>Vs#rSZkR5n* zZ@0Jc`gJs{2p*=nab%<{Czf(1YOGmO+xQ(<1B4nrd_&J|QBkk~b?u@Q zcc~YCoX}laSd~sDOZ{VEnH}Lit}$&Oj-f;uQ+91IpTH@Cw@;6Bpp=QYyDa6Z$C@Fe z6|{`_(*^&AKE1z1vv|Fhz!^!yd&k>Ms7YC%dPFmlJhz?BuhB~+z$g^JcUIP^L@8e1D zVv+nKhA5t!;O2P*k3;ds3(yiEJy%Gtqv84Fm`l|?D*ozYtFoEzTJmMo{Be3-<$TJQD0OwgahC{r^m*bz4 zImtSL`rVQ=@ERzV(B;>D>!Om;~)4;jVEIdlndtYfe9Sk~%gv4i1(F?n^m2oLO>l1pB~t?0J@ClHAYM`T9Rc zX`Ebvd_iSb7YM0lh)8i7Za|lj&wWulp(+u{%=Dv-fhoj?0*x=FoB@o)JX1Ye@V8 zR>r&P!vA;Dz4MMWysf>ty8m^vi{V4R7MA|BS~RYUPectf%3zd2iAPnL$^2pY7OUB& z!*+`#v(PN-$}Rj}NW0S!DyffWyeIuLh>J<@uk@)7=!w^AKJR51+DJ0{J9R+m(TotF z>91T(5Us(dvLm?LxPQ#y!d3m?l8|7ZrpNIJJJ5tMhBNc~PS}No@EXlG*hc}Y^UJY( zhCpZg^>D6O!eaox@y760Sk<muTKDedFOqiz)p+r z=kx?Jq~nbugNGz{ov_BU3Ul=a;j6H;!zo>gZUh>1HmO*odVP*-W5KyIJ7(8UpwPN$ z^)y^;8XrC-l>XwZK}%7_(>?A9GrP-QMS^~}9v4T^`}+&V0TkSPybjIZ!iJ;xM%TJl z=LJ-j1YO?-#M4u4`}Ym0V|VdfT$WL$WLhswT~0aI>5~gg-z5A!HZ}|f2v7`qLG^_k zZR<3i{VhdugU%bsFi_v!K|$t=04&OH||cuAtUuQiqpm0mPNK z%1Xz~69RBpLP4ci;iH1fa8%NI6w<1Zm}JCH)@@n|qXIa+)YNSA^QYtiT~hzp@PXpX zsFzff;5dWw2@CuVTugSO&6 zXXUFZo!>srN5>9)S?$2Z4r3|;RUx-LG+LBNJxVUwB02G0-5TgkI7y$WL$l#zBnWSZ zokqvH<^s)|@BcuDx`hoy(~n^aK;Q&U8XQq0f8CxBTzc{Zm-&f5i--U=7gymz(1XMa znCqZERqZ1tVCs-wJ1;%T%tARF!NLBX)q5K?Bvk7i9B30G6S5|DNo*JY&=Yqck3lt< zZ9*T2pAl1qQY?2V+n)19OE`e)5r^Tr4VhhY52QD&_zaK=si*(`2l`!UR(zJsfnTG1 zMf$|3arP-Os&-LnHWCK$;}d_{z%Nol0p}}ILa(Yx`ENnTFCKM3Na4F1)&Mi~gRgJc zPHAE~L;nB6arTGVwMBlt!61Q0!9=~DM&ZVx<&9e-wwhj**9NrtP6 zGZ-wQJ8QkzSh56DfC-%ff73yR7rTG28yEsPaP2z{N>?w^KSDC(1E;jOV}k*w z)?nhSIu<3bB&a#ly>?>)D{kE)p2J`R=)Gf|zRsG74^` zL4+9g><*W|(6I3#lV)_B3bGV!vR=wxpIes%H8AtTz?8*D#`@9>^83<&L zVq!2IksxT#TtrI?(*u)|09B>ergOVvE}QG!E*)0PTK0V|XAgVMW?wQmcX{3-Dgb^7 zDB+ujrlPU9zDEBC^v5-V_T4Ht{Xmf z7b=T@GelzrRbIgEV&~$nXHiiR>{YS+Az|0~qHn#i))^ZtUBW^VMkABhiXw`1ZQ5m1 zp=}1;?Y^zPm+zD^cvc1BtTV`_f!pbe2%dLSGaHNv{XFi!G4qB5d$m8QcKgZ4PgHp0 z7BDeU@?K&pH5hoh-KdJm3;=s~kmz&#b3w`PEZCFJ}}8f!11tP_1;}sOmqh15Wyq4d2nbLr_GlC9%ir? zNck+Sf9>&J-^P3kAQZ|jK`_021;6yM2k)JeaAaKZVya1EV{&t|n3Sr#|o!K z85Hu6rTSa%d4+&U|Cg=omlE;{8R`Y*eB0h9SaU;>8qcV%TB2daqHc9pn5vRP{b7)X z>+6d|LfxSaw{Z?)MtS-a0=^{+P=gI_jp>wwBn{j=kcdSVX8P6H-)PIjW&)jMP zxNWP^gP*Ini!>)7z7{nThcYSfe07&S`4U`I(DY?CA>(&d!!$rdy^IYdJ_KgP_L#DA zkAy#j+$vBhkjtXMoHwDf#_Eg#IQu;0NcKbj?aggxHBH@jO?i3{vcOuW)XI7J|AqWzxD~MSpT)NCEO@3NCB*;~6WU`~(0M zkB|gWFuWir009?iS{eTK>-%vi5H&)#ecz6+V*@$=2J0>EAt^~BbQsoB`^DCL(2FnT zi)XwS#ZrXSiXUP37RI#Wn>y=iPNU3W$v~7ZGiew$L*L^oJR*>{t6NbfnPKL20e=ww z2YJd}c)=8yi=R`#Q}OKlB|iM55ba_m#Wj)fq`WhRQNG0N`Z;Lj0jO9cnIXp)&~FT` zX;6=RGvzDE!;ruu6R6a^wEwXnZz@lZ7_e-X?b+>#>t6wH)(;9y_%+2Y`_$4U%Ym;M z`4mY!D4@%esrhZQ`I%AP7}Cck(>6v(V;uREc`|bw-|Lv{wm|x$@Z&VWcYsBP?8x1G zbSg9{RZ|sjjEU)h%xXZ@|6e<>3md1NN5slszx5^KN^f5q#9!p#_}{DI|93fC?L3e{ zQ(-bQhoL=fZPGt=a}ifyug8!FnO%Zp6YFsNpzB0@onvPAyL$DkDl<=w3aP4Axnk=G zo~_+mrC}D@PuDWIv`JP}TS=`_+MLX>U3@}<0>zTcjN905KFuZO^OEZ^yL-i~!GS+W zJ)J7lX0Fei1BT5Jr18%>T^z6|`D6wY9PUhCx!D0pR*4?Yx?*08?(^r0D{mD}QC>20P#A}z%)JKrAf>%ZR1p{31AJiXfCiL@SdneErI)uP&{Svj@z&O%^N z2s&3vWbSCkn4Ye?5gX{zJFlVMT0HB(hVFdscF)B;F}sIo?0i~P;-SMvY}eA<=iBIj zd=$>ak&g#QOk|1^XB^6 z%D=$E*k=&x9#N27MrNd zWWY`D`JptGD(XauS&*+8`5?yv`NNCpKS2&4mg zdi?vPt`uUDI3iTar=dQE4?ZfEa%Nc#((EH27WHnDE#uPtYHk!*{5X_abvpbI4v4*#P`<#8W zqKbiG_(0O)1#fVk6i;S{DZlDV&El!a7$rrJ3#hRXb~`jOe->8J#^ zU4InJA|}&jSW9|FC7|Qiu~$Vc98a=^%0> zr;pfg(+OPBb_J#p`pr*N<70_}@1!L3++m4lT!dk>FeP8z#(@QE3U_SxayLo1*|B4g z9EXI0$>Xdiqz#Y?nVw&0^2V?dz7`N1xiug9m|hK!(h2KP6uTxJZ){TR4R3xLODX)N z3yNc$hBA&D9~`)CGPLYm zc~Ef@aMR|`@1T>0?K5`7n;@gtL=Iw_rn|S&N^?3&>M#z<1|3iP&p!_wbGz*o(pYqVkO_WcQIL%4c&sTjP*?w!371&#pOgq9Xl~1bi8;A_jl1eZ zycLZ^|C<7L6HsnijVtglS0@~_*>j7GX0(V((X2}pU8^ITJ7UyP5I{%-c_j56R+M`tz4Lp$EeOKm*J+E zNgY_?OYJdN+6nPOKYsqi1JuqOz%1uLjygF?=VJvYG313KO%ibCDi`i&?;IGmLPgmR za0_YX1SApyiNv0P&%N)tUXb$d^vxzLOJ1{U1gb)$h4?50Zy{Gz&}}}WN6=HNIIdk> zR;2cGysXJMJ>QA~7H7492=jNPtjS)beud=Hfrzr}?_)-3DnO}^M{ z8O*|@^Stc$5V~4xdyo=Nu=8f_9*Cm%R|oE=@B6Ga;Z9hcuTSsKnNSM*T~`YI)(~!M zQxWYHP!UotkN5LC?EHB4k0Dy@!ia+yi=y^m-f`SXn9B1~LU^iL=+D^fYmyj&Z%(pQ zk~~QZGsj{}-!6}Sko;U3ldajHpfo-_XRnyr>dO#H5T7|kaX+`LoL-gZO1bp_ALZ`d zyH^i^QA`W*79QLBjPeUB=?*J!{LYTPB71`lxr>NHM4sj7%#_kL84D?n=S{wbjZWR4 zoe5UXr#F=gGe5l3)6o1WvX)!jtwdj-c|Qg(j46ys&9$}~$)0u}y9eeu8*(Dn2GVJ2&Ot^SZiOzV)Srd0E8`xKd|hV@5xqH-(EK z)0#R`d2;h@DJj6pAq8`F0sL@lqE(Zv#gI!n_K{1t5kH3@N~7@8hi5G@r;-&*^f_U>l9*@cO>&irvKyA( z{t!sToX)<%DPGysN%Fq_m-Ieqp}!Gh^zEMMTgcNKP^ULvHPwretIWGvZDF@opgww( zWLo6~e8CBxx!h7E1=~<&H9=c`>k_u-gh@A4VzVB)A3SZQ5eph@mYJTt>u`=UqYTBb0B_0;_wOwi|gO^id@UyPZO1RfP_#-jU4{F>d@aqD z(l%qhazZ0mcb!K0Jy#WNe4Ec{pq$NfC0aBjW8J7f9p6ofnK)ycc7Z^LldEP&^hzNc z^#S;_n3XfSzzOC7mtK`e*DHRYHsTYA^hqT=a5x?k3hEDJqp{Z@{a5^ERfmFMhQJK% zsiNoylVIVfy?`3?a_!*bfW}qHz4phSH?&0;wDF@3Lp& zSM!E9Bg+WRrm*mvKlhxnj9Y!=&1H4msgiy@*+ZC%zIzxQ+qy?{bs$cAUvu$s=H>`89 zz0KY`xe-SZhCa7ahP!hy`a2#mhpk?Y*jN>no2Wa87gBl<@=q20qmJtETA1jd^ltyI zp>2icz^+xcl6fpPxAt^ZecpKiHwgS5is_k{wAya&O4Qrr?L=xe#!vYu$9vvjgN9)sA*3jd%@4>t7@Vcdk=8WodkX_ zI9nV3l$K}4SNOI0X}DeLC+CPPT;_d21N=dFQj7P=|62cxyocyCkTvm%;!0k{lCC-M zM)2gSlc;p!M+2#`?1ynRUSxZ*qY!2%{=t`8BYp&?EP+P7z95zR#XPIhHiR4Ki6615 zGk@7IfkVIvghx?KRt|QI$#Ta0>AfXI7)=ryngww95nR9car}V-0talI0p#%zp|}`4 z{v`E?xWC8eC&9O%u#sp2i98N~MKm z+LdkV2P*W9Ic5va1uAp-eS1}K)M!42Q*LabJ*AwFljXH*dn$*jW38u38i~A$;qhkx9PoePV_8-KP7BtS?(Q*iYV0uCn2){7pKz zGDl`k?MI-BdWrw*9xjWtZ&k**YD2DDSo`1eO95f^da5;W=>si-!6%; z*WRGlJO2A;UVX$mo&gC^P^S8XW*)K=-qDq?i6J8}cegxecHZrMjAmc0cM1~nR&bHZ ztEk|XYL(F^6W%>RZBP`b`uEMbbHKtN<#q>ZAe^(Z8{l ztE80`^P_1O738Ycej~P-5E+?s7aRMAeF8yxyvB;Z`Y1?TK} z4USDZeUK4QX^Tp%8Bs21GJMb0k$JRVzI?&MIt84B=bIap&@X0gH6E>q*oynyQQB6O zohC_X9Qc^&v2^*!W*4J%vv!PHV+$Vj7-KrloHvE0+K5n1JH3@oBG+nK6 zkMbEemK*+2>9u)x0w!$pDsr>Z{yyCQ-i$KH<(u?$bw}l?50xb5A72lm7iZ{#XP*Nd z1%NH~L8sOU%X?rFkz(kb@N{Jd_gh1$)*^0@;9l%4)!?4WT5ZR!fOAz=Vz1q#F(D`^ z52f;wcFJFErQxPNcy$xr<2;LnrOKd?!E_BaG}7BBaKFCBK*qYTWH{bLK|E`}BO1|F2RO0>MJ1#b5{8Ko8u=iJtI&RyMv2i$a5KoHntAwcLANjb$`E`fBFS z3`yj#UpgwZgq)?KG!Rl!@{E!xz#!g~|JZxY-htYqKZl7!r_Wqp86z;u-#ld-YiMRP za>SaBT*bI1wYbC;QbHUjw=|N14kcBM$4T-Unsin8AzMX5>uV?r-_*4&R}3S&*%mmK z;GLs*q?rV(k;!Lo_jTUAkw~N1h%-eO|YF z@YI=IqiZDOK0HKx35rPm0Vh_(ifC-lujkreAIq(1PS&AU@~&F3{yF_Tq`$;2g-htB zKQo`equ+n+v2!gYUHBlt2%UlNw`Rh?66|wwSc!_sOkQxa-wAB?KWfyG-J*bsxJk2B zC;beDze;XQFol)zpyDr0dIBNW=@x3*=~-WI>e)oV31{rYsoP;*4FcHi6|BY0%ccO# zl03LK(FlB)coSg6_)OcImp`3ITJ_5)dYRJQ(`Q?FiXVRM>q`hH3@300;oZG^UR*lL z^SQIe^`~*<@X)VN#$Iq)gD@)7PH9of<7d30hIBzywS3EfoQ4b0+B&NT2N3mx@^3S{ zTFc{<;NA5KMEUZl;ZQ1GhhCal`&*E_1JT3y#Gii_xLiocteSG`7~a&C+CW$Hzdo)W z%e=1gi+?)k`uQ?TYx_((`LhQf);H_nQ8V?oPjQ4&KYg-?zzk%j=QclY;pKR6+YOS- z!5EbOth0CGFMoWzq!@P?B|`y>r{B+*ReN5So(?Vp-=Zh8o%!-8I4NNa1MZcZVe}${ z)s+{g^c^?lp1o^n@kS+{TB|YCf`n-SWzk(JJw3gz;K}qnVG7GXs;%tA$=3{){N{E{ za-R^5B*V0@#`?gZu}LPk?vIp7=55X30phL9>-8~H_f*r|{0XUM6c15@rLU*_)P`(y zrH4>t^ zbW9D1c$%+ZJ^~5Q)%RB)R5IIt2_#nyf(a{3G^(g@tKk0#|a9v${o zySwlD500=M59To-P}vjy`^q*iMS{GI3Q{EWx5fKTFg@+poj=~lTra3mOrk{(ut+ae zBT-Rq-3q);08k>qU3A1_Z~jUS7kI6E;@A7)Q^Fj9gHJ@MSu&M}AuF``xs)m)`H#?{ z98Jr@DqZ2Jp3&0IVHxwj~*ii;(LkYEh-gB89a%Z!srZRqmyOS);XdMt52S) zPQFM#C^7P1t{g`w>)2^>R={l2$$xi!wr-V%(43v}UGkU?UPwjN2 z{Q#%4?)qHQ83%@9{g-WDH%dC|$o;jxXK@o2mlner_`abD!FfsjTKGl9lwEs*6{9Yg zRMQl>RQR=jc4*0Kk^Vceva|k))^E$D(o7pSuw-ff@u{jN&%|5b$C8%DZ`{n#m=nx1 zw?CCrvGl#mbAm*yhv`aEi^Y|U7<8!AeyP!;GO+BvdY%a9n1u~*cnQAJ`<@?zi@1<4 zS&lUiEoW87$KzxSPA3M!FqpIzE)dA$U$*(~NrxRJ|4^kk&TAi~qe<`wI6@V ze8<6lfO~3bAj6 zv#Tvk${%#u8}C2maiTW5Hre{3@}+7ulJ|z^=8&|hIvX?Z=4;Pbx>;eAXpv1xln>k1 z^X*kfs9&@~1JlFLDoQ96GmcD1$~oB{eD*yMi|19>TEMq+VC>FC<;V`1W^P<@+SpW42lod^we4%Ea~;>CFFbHP$Lq z1X2~IsS0QL16`ho!=hUf}TuGOW8i<IEdC6C{3lIvkb0muGfV zzxbIDipXpy9-C2)0l~#cJF0<{WF!ar0!m2pn8!y(V2Zr1?#2gyXD@e42BbA4!O{Z( zICtL{AqNF!@RvpD_FMH0Bu05Q8K7feOxP7t=`6u5Hh4 zd1ya0Q^TE7$-Zfc7R=4*6KE2n|7|pYuB4P5!Bc_ZFU8X_#jXW0hU4%g2aD1sx8F7S znr~Gy>@*+W2pPyOE%){qvDRvANoz9SlJu~kkd|La8k_x4nfJEm8HqZJ3hG=nH1I7= z$3`~Sv_wQ)Y3b7k*mrmTVAu%GQ-mzK3L+0f_bDr8%_JfYZ*Mdg^FcP3QJ;+&&5wrX z_qbB9bc3_9vIgDy#G~SfVfyH4v82)_hw|XT3HRcl-xV(yfkj~*_U2?h1Og?afdpVj zmHgcTSvmV3Vf&xxC0=I2v2x?)JgHVh0b}$# zODkJi#J~c0b4eMjSN70?@P4PDzSbmy>(fbyYVoW##x}dh)XK{C8%a16Iu@775^xm4 zH+cU`H#}ZZ!K8`Hx8^VL8QTjCK>v?Ee_0~RO&E^a`Va+rw+HEcQpWO=X|f&bmh@r6 z_6%8ybf}P>6Kx|O+rJFMPBylTyEqs7mHz1BMiqkPmos$jWtDcpp}F|&8>QwfG11?9S*MHR4FWJ)knK64KF=#;V5_H z%e`)>FM(+t*%2B-xPrisRr2Xe>?@<7qhuK#-7WGB>yqc1Nu0OZTvIX_7rO73uoV@L z{&-}_im&kAacpc3R~oQcfTxT35fiHzh-R%D-Kzhe#R>Z zCom-RtIoTEVF?C<|AVRb0H;E6=)w)B!TwWU? z%pXVnPzqLa)e0+?`{D}9Frdu^2P%Xyn%nb5p?n$`_rRA@Sissm18ih;TJ7OE2c)x0*JUdB=|93@6Q_)8x8#hv^aL1FK~)u`JEcv6+(Iq^9;6TXRdN@B+tm)reFOq}cJ4`DMA zrpbo4IgaZJMNm=R173!#$PP+n$#o|*=q2J8Zce74-|OBdH>$Xv`SNyxF0M4Rgq2L5 z$CSDEOembla#NM(3`NG)+oHTBZ;KHL>`j`VbgDZ^mZMo7X4TcI%>BCT*1W4_Z{kz$ zRA%ea`Rwgd2UafEbB|*eQe%8ty6HW6ANH`}NfiE4>1E$Y@s-y)UgE~f1lXs(kHbeb zh*0uobLKIzjwbCKk|Y%7LTQ5h7Q|HYJtC8@`@CK~6-F8gDaaQJNs$LHGGZH>3gnOTAa_h~8a`>fzTElm1a+D58Xo=p8=YFo?2-5KEu6SYp)mek{_^sX z&GPfC(w4tPG1|_PNf6BO7*D@rEAys2Es^@s5teyQE)P)Nhg+r9WIKgdpFEEz0Vh;z z%}XrbK@&r$13{A&)Pgf5W7J;ONf0C!U3Wwz@CVqH8<%y3jfYINTV*#RsOApQC%n!G zKJp9dxNnd*xOUl%TdK_H>lkb;WLqoDS{wn(Fv?=UFJJ3}yg zgX(hY)ANFOAIgPyF0YzefujNn0RZGo$uW=1dM?2ht`^ZAA=3a5%dP=iTzg>Lcg&M; zmx$0&YA9<%5Xye^6xN8}$S%nHPTTnd60rU_{wne1l4lCCUMFqWb6#Bmpz4YDr^#=0 zZ~_y`qKh`5Y%!*F-pN+)5`JmDAh9hY>COZe{Dse*JBUTB1Ah6>{a=cYuV~EW! z?wzv5puG1G)OGT@`Tzdk9z+?^DDrpC?gMOx=n-VL{=wn-%pl2B)XT%u8$T zz6`ooOem_CXC~4!os#2e|Hfrp4PBMT;@%~^YKp=RK zNhW`N(4nn*v*COj`CkPhY2@0soc4QdydrCN8sa@oc2qJRK#$ZT*FByF&$Ga~i^+I` zXMwKg+v$aUh9z~!bK)EjIeQT?7RT;R^!-opPpR1^eNOu}U)Qsvpwxdo*o7=Mjr9|g zLT`+WZiQIq%k0cjct_VCjo-Sl$!70PC1Jcf$TeyXtz@wmzn&Nl`kskqfYT#NL9?jbe*Xx|^!--OL*KweWjr^orR#Jf0*_}_+@V)N38 z{j3?8?g=nGICF0S8);#G^N7`HqthW;o}+y+2?x3|m@-vd6T1u@eMnwL7}d|7W88cs z1f!-?U`e;;>77#%_@xJAPr?6j(yza^K&*|0l46KEj-kjsZu_x4uweP%-(QFExPBfw zLHA2NUX6I#=$y$ht4H5!(-0Jy4%FhDs*e+U*&RiQPqlo`!uH zr5c!G3l1?_SRXH!3`p;EIP9t+%8&4sXq*&ji6GAt|C;aHgQI}~#D76qjK-9CQ#~3~5Xy)un6@CY322q|@6N8( z`mB7MY0s14Ij;eD87+RzB=~U%u*9UlV(&g(#1`{tro~VoatE3|DRk5>A9d8~L6dRm z+HW#=T#}jix-5#z)NHGF-$LKH)7Y929neD6S`9spH^A!?`NjkYeS0krlotrX71xwsqD;JfZ4pU6x%CxTPzh!$dl{$tfMjK0ZgYqhF=PP#) z{{!!r8xq=@n%z*qj@tA`PRiq$n=g42-RQu#Usv&b*jf(XjCUwL zVd!{hLJFe&t#ONtC|1|Ki`}_Yug~{pbPH`eBE<^$<@od(9#?IJQwIDlE}6QsC9r#Z zOvt^%%@^WGZS9qHK@yknm$;Jn7hy^cYJ;=yGAGNTs_XiTz#TG8uZDQArK;ahP1^Vi zsiuaIL8@ije2X7a7#3#Y?zXc7ZH+9X#Su7=_qtf1D1F=Pi@Z$d;Ft0#9>%J-!JR9c z_4RdRxZDW@5%nU|w&AzA19q_rM>Sm%$c9U;tc(Gr&{s&7|MKnIu0Pl9B{c>j9hbRV zQ0;+>L^L78?!eLKZ7h;uEtL}rhvV~(yGMdo3s+HySZz-AfbJlZz!A+Z1HWUv3^R9HEPU-VJHZF}^P!qr9}*GQCy6K%Y0o-_&D<-%ZzZ z1JM7T<-M*`5V)jL4b8Ak(v!X2_)Z+tD(r6YbOr#WZ*+qb(==#q`Mh2tCZwT?(ds3J zjJj|a(M7f{6>V@XqQqGyy`;?vT=J)!5b*)tqR!_N6Wa$B*IA)O6UDKtPn&oBBr-~+ z6HnIW=Rpbwa5*dlCDmk5Dbg9S?a(i@m{v-qW*#+{M!8ZXMEh`BDV;ohtLZz}r(Sc$(!b1-_Z#HDN;7d4 z^xppZYRB;7T4e3vi8Rt?6Ie$z0BlMg?ZMsCHB;b}XxD$Eo{EIKDAp#8zUE3!=pZ?` zy=rL%T^0DD&wde8Q6GQyz6y@$DPl&$sA;bfvs?q8C ze2@`(Z>RDAS82{cQ3;VvgZFA)0!k;i!-b%~BB(8HT01ELl2^lj>|yf!zn$zREM=v* z<#xZ;aIEWCM0r&T%A943Z9BIR{I*u1VKbvM_@K$?u&&*@Z93wK(s+4D;kslqS!>^S z2OB*B&Bp3P00onX1R`uhPr#YKzcaoQ$ov7CAb|wu3EX%~N zh-2A|upqMN>3_vEy50iN+;{VPN<%2XYA>nd4Nkj`dA3(Yv&n{J9c{;@WE=^ZnY7za zXL%9V2?ZYBcS}-P+>aBp=?S@?2er&5yF~9dxMJbdCekEp=9QzI{Cj#^Krobl@g*n( z?}n)@6=U(YsF3K7&Nl6Z)-+}w?D9N;J6KuF1>Y2k&AloQ7v^ zwKc0Chbq3CoqN+LhP;x7NUo^d`-^(**UH3jBmb~bE5n~%geNy|H0*ELARAx*)$e%j zU_ab_m#4DjK5h%B8w;%{Ml)mObPEKw;x>WQrw!K9-0K0BlRb8R{yQ%X(x@#ZOIo6R zBHma%zqa4r4NyQR80S9Gt$y(c#AT6z4BVgRC#5<^hU)sepjI zV+bROZg+#5He{PuE3}vp)fL9z5Vhl4g-E1fR{FkXb z|KCY#EXgcW3qHos1y~?SD65tJ7)|>VUCA0#CiiXl8J~*W)9k$QLyaMnRP_H&!VhvQkWSUU zuxq)&0y-6(^7^d-z@L^<{EhOVFMM)&(>5Ka2Ut5I7hy}i9~mj{XWV4H1w^pqP*GtDcR8QfI}Ul;=kyqj-^uhcTI5E9A|uM{SRm@`arwWECDni9yKZ;{$7O{NJOwNBCiM=m3bgxTqp1tTWFE8;peTVFkMW+3R z@+_0w4{WQz*}&GjJUhHt6&KQnMKQr~R2^8?LbArwUK?%*EpUagI8mf`6QbE>KnT5^ zr&$>){>ar2*|Y`OyWY#@<~JFIw~)TH)L^E0U^YhmZ2_Pb)crd^lWeg3*=%?unvc$M zx-QCry1UNvc(9|k(&jy+=Cp0gW=L-iRC?|w9&?*+H&kiK3Tjk_14`$E)GRxT3QWnx zx9l@Fc7+F!{X(}81gV>CQ$wlx-dTK4Vkr%bb)YXmiNfN&EeI1a99ZAX&Oc3Qa`nHO z_m9#`(?T4mTQ_VPS}!hFzUR=<&*=7E*&oqLJ-#pUugyLfD&2^GUvD2*t+kV?7wg}G z7Z$R2SGUPD%;Z%!I~h{`+yy7h%g0T5>ffTe0DtT;%S@~YNn<-vg%Wd)7iLidC3(p0 zDzj9EcK>~87dIfSEF2j`y#3-y#?5lSNX9nNU-p{NzQvOZ>O*h<@ z9AD9{kZ^NU#)NhlRVSKAF&q7HD%Aj7zRw8uz9?Pd zPvT+A{*HFumhW6ILDLJ83UBV9?m1v8zA~)d@@y5PcB<6ljDtunxS7{RdInsAUeWc! zF|2JF))rAk<^=pDxs?o%o8~$6jx;xc8c3;a!KBVlZ_W(7hFUZ*fE%hmiRv9V2-I0? zy^?>P-O^UX>?eOK(tUFGb_+->?7p70Vm~QE`*1{5-*?$mM4_$#M_E{f&ZoBLl5C0j zGDg-old-RhHOebcpDuipHi5Djhfr507+X+g1chwy_K>H7J@5)Epfz-84s5!Lno1JX zs^1$bzIlZuR~U1_n1ih25&eve;nz*XpI1Bj+n<8g_?dAjrrcv`BmZD z_AI)zFq>Gc2DAFLz8X^QqNxPm3Z9TYjLjR+)BQ4P%oV8o5ikO#%nxs=UN^|URg`m| z2zvpaNZXOH_|><`94wIU*5B0e!*i$%!5S~gW7))}#5?$GcFw}tKBpLr0p&HRS!hpL zL2f8nfU~W*_v&lK@@@VrJa{{)dB21E8YHxBhWZI_D1zGT#zr8l1E_D-cxmZPYtoqU zn}4yzSmR_IDqgG?ALE`EtLGbqlA{c~3Uc5S|4$SX1(bOTt-@DB_=y}YK9tKg5~Ge8 z6IxhPENT6!INyZfpW60*OwPTumLdF&1=l^MbITKt2D@F-|I0&s7RP8diqSrb%XPhT zr{Ob4MLf@jB(aSk<8D{$(tKU+V4InclLEqE{%UrSd+(RcKgiVcMbe94rHvi(8ul|O z^{wq~+jm|$-LmmUf8RRyBj4<{jewHj(7A?wGmg7voOeVbLdrx*_wT~hb``#Bz`8DxhP`p$|N|!u}sEB?>nvS zCN`zM`89*lVQ2> zMbH^4e(jfgok_@WEsklx4i+&HdmY|NDhOCCHoUUN0n1PwmmI6L2-HOZg@(rlDgA-LjDZe-H zwE?EvIeNHZAbCZ6hRyladO1>){MuOnWMSQ-H%4vNjgsq)2ZlzPOP1)lH^1L>0pfIc zY^1n5bgI~8jTESj0vMa`PWcNPc5woJ&nCa5&Fc+zD%&02X9j7^P@JIAeRxX;7hkg; zIt7X;BgXPp?QvBj1u3VE(b=%XwCC*QV*+Gy7p~X8n@kNS`U$^xzx^a`euItDiqtfJ z|4gjw5q@;!1SkCoMA7V?dh{dleC*IOE~w4G1@IM3?mj0 z_3JOjQ93-Y9YWe7;HaT?tc;XgG0n=SR=dw8?k5g-5(y4?4IC{z)fF)O49YmoqW;Xh z{gKC2m6`>B+u*?D4@5UQr9&BqS^W~}A0td`0D;aXA8w`_fj^`{PiTuVBzaPZpG@X& zLGI7BBKF0*irVVJ9N|Ao^B?5?1ouKlZa$_|_Ta=^Q51q5tjuubIur{e_CCP_b`T^k zR7ww|sqv!>?&I?Nuz;Ead<9|%GYh8;Yt*OEJe5OV9(4MKfv3Rayh!L3xKTjy_Lk}+ zRcBdcP3JNeC>1vfuuyz#KWBX8g+T_EkB=9?_&`x;x8=kT(w2r;aH|iif}n#c2Omyd zT613qFnqaF$3NAj)n|)`J-tn@(rh6Gj1+VbVGjImq_mNljT3$lwQ1g1yI^GZ(@v~y zH^xr5Yu7tSPAj|3GFz6A(ea2m5kfjxpi-$6G$kLR$gEQA-6~)784M%2aU!CD>XVvt z=K4}f11M1uMA-rePavw{O-9KxJ1++!;B4jB-ocMdnmWstXZJ+p+sQ2Jw@;}#s5UIV zy~ZQ>WbJJ8@8=2%QwEn_DIfs4sF~CtIl6Udf?P^9wZ*{eV*mNYHEh{HF^grZjk?`< zjI>Xx!8i1_YH^x%Z{m~++$rGoD99bbR|et(9%=z=vTOf+K>!EKN1~Lu$q9lLpPRM* z_XSyCvj-c9AEh0yWFX+OH*{Z0i9Fb7N};6#99U!e?r^At{ue|4886%^nYDpPmH#W< zyCcX4@u5soCpz_RFOXf@C{V)-%3G`Me|PQ(vX2~6%T~7qn&h?^R4yvC$KHtTv2EIQ za*8)uNTR<*oaBA0@oc2f)Y7ue@Z%-%WHFC)6`hR7dDRBbTx==Sj=!L}F1lU%v+Z2! z*^D=$^eVX~I0~QnU5Qahj}$I}{K&vQnM*`s;&JKjvXZX0c!{zKOMORAU_hlEpx$rG zHL5|#Rt$=K=L&E zr#_$FdL~QCLa(C4g7mGpr9f+S4^uyD#`$@B>?*BW8X z-bmNSi!rTM6LHYze1Wysj80(-6+!kOfwmu#;j-GWZ74zVyZivGB6Dmm zpuS-_Tue@pmVg@+!`&51h$uKpf0U+0VgAu55lfL@EU%n>7SqjcGh2UH4Wp7vuUx~- z!s<2R7A~{f$Iv4p9NxDSu*tcCCoY!Nk~_fAyPIXLcd&j^O2yRoB9}s1EIIzlSW%p5igTJ}y{E<8 zxq=s;u6%^8>(|_@tCxtpK({_PGE&~Vd9BBB<8LW7-vgwQ`T7>dldAQc+^T~?sbVz- zepr(k)Qa8*1?{5dHpj$hF^M~tt0Zd9y^9VU{*IY8;gOUSs(|ezrm#wXXh<|8puIVpo=IV2bYvvzgtsf)kb!H zK^p~B$Sce7#{V=goj>#I*M{?PiHtCt>+mjg<;aNsF{Ssp2VH%zKf#71+JNWy0)f_| zFd0vpZOe=CdcKO(ylBdDaqkxR5zZcNal?W4mP*!2CQ3&_=PMQ1uV8p9i87w*hif&w zuk{*EHSH(;o}Ci?itg4+ZO0{UJs?XKY{x-|1_8NXoco%O3zb63hr{Joo-O>Jdm{Mh7CB{D$Kh4@*>@nL5TH0KN7rZ7&;)DA3Br~`I(v!e zi89Ae+4y{iZK3)KwMX#iA)`WjEmfW&LfQzaI>CMoZ7bAKzv12C4Q#>stS|~w<-bBZ z1xHLyZ)EnJpd3Z*noy8o=j05Jso-ODWZCj|t^|YH4s3_uHr&|l3TpY@A!nf^s{Z2i z3(ZRd?Yq?;!a}$&=^{auxamZiEluRC9YXEU5y??)kxdeLC%kisRj*~ONQprbsNQ>_ z&*YU6h4W(_j3mv5z8@#oOSlhQ>V(ZLa!m6IN7}q?zF;Qi*PPgtYhZ}sbCYudJri>1 zc(!6XPLbM>@$D6xw$s-4XDe+H&(}9{fYYTKnygY$`P@f&*Uum;=X=gzdZFBpL8^!~ zyLs;hc`C9H&;Ww%1~3ItfNRS~ewBQ#9af@`egh$=f$fpftLBi$s{o|0>xZ&R&gc+- zl(PU2c+2S$d|HwU7k=wkiL?hpPw^Eew7jMr|F)$JT?7|-~N+xov+|;)g-u_{S6zI&U$Jr;q>tanO>nr#U+lrI?rV$ z({>}Z_-u#dM2!IP__!Sf{r(Yel)-VmKX1%YHTRw+`p8}C(ZVn4#cKHJ8a2HMr|iU6 zf>A3wbdSRZa-60+x^x1j?8ePm^{%?gQL+kHigyYo`IZuR?2$M-%1aOlgJJh^WasLX z*L3duyKgMoX||^)`>B+GS?X0jAR4t)`R9)ydSq&F`q01T+q+{FiI3tjZ+a#sT0Hv*s!tsF zsPU&yDk*3AyI%Lk65W8$_Qn`Zh}-qRbDu`&6cc&|oEy#YIzqs#;oJBh$O*!Io@v)r@^IbGlMI)$?GX z!rlGnIjoVz=S+O4lL-f~WI{lby6s;tZb|#Fo1%|Wi@Eu>SU$jooNA);w(pOVG5pUb zn8*&gD581Ny#_ve2!&FiUa%w8&L6H(K~N9&ka=)d-~J2C(qGWNPqiyNxX3kRdjKXw zK~HT?IgW6;QYKXd_+8&qJA1}0xd6+G7rHjO>+^;{n))H>de^G~d5+gis^8PBh9|vY zMx_ns3X%-uBz9T``_f62MS(BMkOZY!PUTf}r^bgMT6LKZDmNPkEC5{iZpYi0=;LmNcb2Ht3f z_O1*>`I&B1(Nu|@`ju0*JYC5he*XI5?5nZgV6!wIP5ySlEkZ=Qo=o>gdosMFl}S zbYMYbx-c`XCPx)LTxhUZ9YUiB-vGP3|nGupe*eIKg%OjBN%=a89_06FWf=yHj zA@TTrTKzEPTQtVgCofes*^zbX9I_w*y6gC2K_7BSXoGya^~T_pP94BZVzk~S+dewo zU9#M-S2+PX!_juvadBr&_BpkP<-yocsz$?oS77K_SSgp?&8#hHrC9EZ06ZhAf|%5J zoy-Rop*=KH8?!dSWt)yS$<1;Wb9x}S0?C!*62z1uIln31J1`gFDCX`Uv%mBGQb)J? z-7|300ndlb2{7&<5lS(C`_E{ObcNS-t$~txTIq58u%A=zUD?y(;L%6y&X<%OqvcB? z-kzm#JC_9!K&KRW`S+-KHW-xnyV6W{X3kCUVWS0-*!qz^)zakVUzs$JbG{{u zKU>!#HlDe+xaInzWz{Z^vq+PU+qA<%X!gblDzaRq8r%;qASQ_elf{%(B{^tLGW*g7 z2Q2kBidq!THBU`TMfh(q*E1^7wzS;r#wJN}7o9I_wfa14%YTsCR-@xj-{)9l=-=ZU zTK=T--?@8GthC17dm&1-QuU=pQA>{^;U{?aHfyD-{}rtg2Hrd_dbh)Vf0nzc-H(%< zOGs_;0)NYGNywBDjT0wJ>#SE{*Sj;~*~PYVV_HQ;G)Q$5%!V@b{$2Muj%g#)xkVtG z??J(i>nJQJwV&AUBpaT6P+;jYkYMoF~F3*Uh5~U6R%L`q~ZytEJRcT(+0SOtT7&j;#6K z!3)U_bnWcW@QmCsYietScJwZV9^AVMLE1rXCi?i?0IMLVp5d(ml74z139UZAmFN?Vub%EF$ryXB@&X#X)J|=2BY$jf+Sdl!=uo@$Q`Yf=zZp2F$P9MS7*`uuznR-MhPgN*R&5 zDw3I#b8zKK}EWQGbdEQd4x!2ueyqo$-3d^F|HR?kvZ0anDb)r;Dx z{KmKb9qiXd8u@PeKD3yr;672VrYoAV0?V)V1vB6vU*D>ENamy&XI1YcL{Qhv z!np0`;lt9q&HE>sylnBkDW!jH!AHQxICWnl5I_cFcHA#}5fUn)5J5{2#KCE*avb})!+4C6KA84_>(nOS7iFC{MZ6g(#2 z3E>2)0wI-3Qcd#|n5=U1Jyu_52pQ8>l>))a?+WTmSEz}2Nh@ihiSPMABQFOJ`X97p zGrf9Q==3W&tJmQ1Om0$#cOeUwc{hhgK2vLh$Pf)aHRo>`J*r2&w}hp%0I~%lgq9$9 zoI!<7OO{F`NRR&T|PZz^k+e*Eq6(b@K zRMzqc1o4jjDXJ!Vz4Z1P?UV`v@$vErARUQTlLFw+j+_1?yYoHZx60N7(Cz>T!a&4+ z_yVEENTmE^yvXPawKKWni?qHPA`22R1hDf;l_$igAM2jvTJuz&gVb{;Y3U%E(xK|1IhMY?0Ddb&Gn-zkuOABnn`!I<9 zM2J3IxJd3gSF2kd>~qIx!1KC3pGE%KU4h6L*@{~P8kN1=4dT*k272U_tI4O+ynU;K z-0)fC2Yce^K8pOQ9R1QbQ;Xr5HEZv@U$!ro_Uy^-oFeGRsL43-$7XeZR7|#w%-YL; zMU!iwvpbvPxsznSmOws_Y;6RAPE)E_u;7p&GWp*Z zo9XX-F!M6x`9C8H_WZ5j)qwHjq*KnT#m4(w^O4)?lZT|F*Qc_#uL;znCp-&1xleyY zs#RK(`@DP)@F%K9?a28fFW(Q#+i91Y7|~IA=jUhWapP%bC}0f9cUCR#DB#v8Z z14vu#K!b<^ad$K4|DL+l@NuuAY^lM_1(PlHk;9FCA@0wqw9*OvKl$hd->$?>g?BHx zKxTu-{?~;f4F(3?Q$c+98Z#?v!>yW-vdZ%QX=NQP8o$5k>D{K5Sq4{E_Q)c*YIvnh zz<{iVj6|M`f31yQ>@FKg><+H-Fi|kTTvsSylH{6kG7#oP<%fsNuOWixVH~7~7knuC z+tbza0B#J$7`<~XE_Sw^u4rxeLUI()6%8vF=keXZm$Y<#c>4XMc%2R^Cm+H~wj$O( zL&yS{BFY*Vh~$C{dFlP|C}Ws0W@(OgPg>{GLNpH#eQc$(a_DY((_+^fp3AsnDj}-9hX--?<*Np zp7?z+QorMds_(h;{)?fyuHawH&$V)%qB`Ns1l`-+qrz4_sqmFE*v~kFa-x6MRC}w@ zcCA*U^QU#dLEIRS?VtiRmIFdR@+tPv+JEw|Vg^02DY8^&OS}g{8OwE2qLS3lI<NWntBe{%MkI6{6UL$3Jy94d{ zA`xY*cg5x@!zE*qM7K)8f(D?cSzo!$f!jNOUodgNb4gbVllk0NL_s-)Hb|{BMb&bR z;%bjE1{T2nO!b8L>4M$s~qLwBZ&M(6fm9~u+9t$F?=12l`LTZ z_x3%x3@DuLFI-rV@l56=nt5qBJ&yUruY=;DGp;}0eA-=2F&UYBlvm!~*O@vo{~Ly? ziEmkGdyGq=yf6}eoS-3jvspf-GsR8!p^(m|;WQ(b+b$LUhZYl!G<75-N$HQ~sY6Dw ziZTKb$YZ~3xj=jO4hJ&ZePCeVNu5sNNtxNw*dzK+9oD}Cr@SKWm2lK}w7N%Bt>omn zoZ#}L&@DLO+XyM)IbX8Tf0a`|o=o+vx;J;Q=vyo%YCa{EKk!r_Aoz z<{@>2)*-RgjEI8Y$pjk50^7rQtAnY1qvKbD@OTweiG{vKtF@Nsp)F6%?0Lfo)t1ZwzyYDM(N15+VeB6l-BTX#@1~XU2?R zcpq`wp?8hctno;I#m~P*oY(bULqxj+H6h7v0?ZA2HEIp>o{*(;vOsZ7OD*$_3dL?c zJM*em^ne0081k zFcRC@eH7Z>O;>Q4LIQQ^Tba_d`Es!B8T8}R<|?Za_%}QeO0-IYNY7TU!3qAz+HU$d z@qf3{9Lit7gaFy`n+P;&d;#)&!~I*Zf$6*u+Qel|DMidWFGSxk6iX>BSFe17>sMZJ ztN4{~`Cq`KIOon=*&(*Uiu;+Nj6Tgme zPZj*%Dyr66@9*4w%_2PExp#}QjI;W|k^h0@9;m02LRBC?FeeA{{gDnny~?>={{uCF zE0)NbNtPH7)JmLFQ=3Pg4Pbf6klhrCCIoE|(8#ds0WE?=AoOuQF1#sGKC*;w8a`8; zbZF-vE4B7Aq^k@zNNT_<@^5n<;XLt^RMo7;Z~4OftE^KS$R*yX_#61)YWGWjN*~AC z^$`aVJOef-CS(8uvRwH7>YcSQ0jE@K2uSIQJj-d`SI*+tq77fJTsr7;t03oVWwTB?d9(jg-W%Eo{5$=bX zw4DnHw4*kU-gf!nT=KB~mo0NrQ!=LGZU<^OR>yT^kO#NJs&R*2;e$7_4o<%E1lL~# ze!(GLFGk1hdEhHw*cfBWEyk>o(@v1|oVl#DqnUG(M$=_32?7oWyUsHEO5Xi5!zpWxXq&ENm6A`pJL2djFQfIdPPE*cb^1GzYCFdK?o$Nj z#HOef1NWb1#}$(UBkIWP&BTLTSzTQUm=6gad!3fZ*jczzLDwAm!eSIp3e%T0YO-r@ z!}CNeu)AUD`{U;0!rsy$W$al#1$!*5)8ZDK$X;;MO>ksIbYeo**2TdxzYtHi&j>2m zMXFYn&#Tm!F0_-hGc{c-1KX;xtuA(V#6ku7Un9g$Prcowd~zFizNZ+XxNMb=y;}?y zM=f&PP)Y0UT&BA%R@I)yI0(}_N>4AqC;M95{8 zg_}F8mg}JTBno@ zipi;-URw!}a>Hkn%yR*S{I5;p0=A>zd^FOeo`B!ahppYD$Js>+OS;@^kl$?T!X1^Ld)GfQ@Y?43G302)O@-LJ*oQsQ z%*H~mP{3&k;uQb0^*ss;fJB`~E`NQl5B^>5DI9YcN861EuVQQ+$Idp{#=6SqnnO8v%&O?rKB9^`8MKr&nxg+*sTq{^1X zYXyP|oYUB00Xl~x+;fQx_$cdEbMou-7k@GPCni7T(Za8_r3yXBv(28R@I=2E82o`b zft1pmY4jN5i(qgb-ko#cg_x>l>!HoH)LjCRmke?x%o;ioYGFaoe($Tl=M{5@HmFD4nX zAc2(;lVY$7}Qe1i$bY;BpqR|8phFIo4JD^p={3t_q)o|ne=n*{Blq@=m# zCP&C+L`2D!RHNbqXbX8=7bP1zlNVPV#ocDaL~E`x5eS!z>|D;y$@{J^T_OzaHPO;T ze`b2ddCG3syw7Oot^ecqe-7j*ipy2;gM$NGcuP;E#ac=B7DQGpU+e9t!};4DZ`08E z$bIuh6QQbBGc1e2y~a-oux@CybV5_7T8+b6#hsz1zwCN!!pC>8OL2?mHdIY5+4qk( z>=)U6k0gvkFB|YnN7|GsR|5TCO*H%&VoY#M4~}Q*S{1p*P;#pHmzYcEav=vPU1kqZ zA-~2%7a`o1`EK?FkDvi*so4 z{&@F&;-vNO4ef2oj~Ms13^qL!Ez}NzAmecS6!eNa$)+pST!x|XHfiZ8S{K(#$=C^A zRUw#LlkRK4Id_27St*ovoi=q~m2)TaV)i{SAaJf4f!pF0QG?3X)1rECBF>g^{+&zn6HN>a#l&#CP*(pm1!#~wb9=S&j-t6YLx zcibp*3ry-A3)tW3=!P<7+c$6_zHr=V`DmeFDSIV`#Z{k+&zdVS-nu66`ZW3aN%$KU z_ajSGpQae5!2UVg|7CAIl!|8114AO(UwGrm%+3y!h@`e!$hvcd0HbLI>wICWHGQrV zx=fLj$-9MD%MHR5vY8*iqzf?y0DZ}S1+<-sI1OE3JYQup{spU2grRTvfcZ+Vr|5x0 z+3Gn}gjs4J6*;3!Y_G8!I=}R$`BQ?BAvGa7w49vpHsgIi3OGGypV)kXDF;goA{kFY z7@t@C6~dbZ^usiXy{zfAHzP3t3f$_o2DA!V;JUCfy#zs_2bw?I{{v+K4w9kVZopEe&LC-1HFZHnvxrFiP3_s+@}Sx$iw>0AQ~% zI6lVnI3XPc;-VOz=zs3tz!j6KpDQe!SY4FP3IsZ@v4(zGC zR|^`49%5!>qW`n~YJIr3#Z6QicS8;jzyHq+9O+HQ<%&u#CP@XsIrw*%-AX@3=Lu6c zmJd=;{c%WdERy7<)H)!nVu)K248ZG)Lf{`1)7EI%jf)H6g6yaZi?}2&qH_sy7=D1O zWWq9|{%1b+UonaH1>C0+_wn054`AfrS;FckuNu<5^qp1$enOy5+6_nr6(Uj{3WnlU z%b&1I4~%kryK`?iXFGC@p?CYN#K6Sfl}e%(NB6NW)pGz5NmlS!_AH?wV%;LQC@S<1 z*Jj6>!{36!=Tn@(Lzk-GLj}m+`() zAkX{d`7b@7bTe|oAmJ%K7(hi&T70|qRIH)^k!8VQ-{VY9%f{ed_bYm2 zY3b^)*SKLn7b~fjzVEzHa%n?DI>9S0IOY*;(cx38MvcnA{etY57%q$LWQxL56)7V< zyFyv14e#!=^M+mx%FJBTR6#jC&*i}oGWru4iKW)4hu>Sy3X`kt@_05Zl@<&v^3wgI zrpq~v1U%PheCRtb&SoyVYT3@W&6fhx@8?_NXHr-j^cqJ&?97%7^_8u6_z+Iz+}W#8 z&Bv5YXt>PLv4YJ#>oQmY+Xm_N*&YeWo}$J(iHw`&k^ZI}T|l3^T%E2|Lm`q2sJF8V znZ0c|Pdp|-inHs&4_CLKbUbH>(A91jY=z!kXNMbW>+AYQn-h6j)e*qMBtzda@%3nN zLF1L1Vm1LX#><=Y7xO(43}n>d`o_j7#GSoiA-T9NaI2iKM1lqk8fBl(&fMte>5;;x z3E+Pkzc$LiXc4+xljAv_wg)^^72eXGD`iLzXp7$w)CC%g0!eOwXs8G2FJ$(1CFJWH z*3yBfETou`naMTrkMp1KPW;-#)1Na9!t3T1t|m+92e>%){Wt8h3I^M*O&NZF3*#!9 z)v4W>UFn|jiW!nSxee=O*q9BH?biQ!ZUO#5Fgf#~rm)0USQ_=I^F>OUsMUHsb>C`` zn%KF4K)4=suEeog-Dl5`T}Jx3kxvV_1eIiY{Ks^?FR>rh_kH}&z5IP2-i6li6&uh0 z>>x{71otU0J}@KGEC=y<-rFH6K$$2{C95iKA9q>7h1YKQG4NUO5JeDWCmY1Djhuo; zFyIn{ux7IS5`h3g@B!8ughR`T=Q62el=A$Es0Vo;MNt2r0TJH#R_js)1P|RhdW!<& z5FD;BLGxwjlBwNTSZ)%r2>$g{JZZ58Ba&_Aq|bJUhB@6s#C|jBeNr9p3rKl!YF(@G zPeZmf2q!>hDuYUWV)PYTRI_<85a1BN+RFNH-5kjij0>X2=I-GhV1(sd@g5``H{acO8x9ov5O}*OZ{TOSC;YxLu^y0}}JqEi@4E z5d}Xe?c`vRKnev_wIw*?$F?T+{VBm#1ACrZsxZVQv@3xW=X;LtdJfkA6iSjYk@8d@ zf$MRh6E*O>3*i&%U6QZ}f{e&ey;-{?P%fiXEuTmr+J!A-!Z0Ie@Fxfl)4P3Y3vMG2 zkf(SyX)ZA1A*|oOVbcDifUOeUn>KYepct`L{7SNLUM_xQ%L=Q zMp#I<1tFr9%v~_ZQ2KpO&lYxe`CoI8!zg2)+JIpelbfz$h!{`2fk$Ifvn-0iy%ZqEu{;7un! zq*EmPjo(D6$v)^Z*zA|3kwtS>KDvShG%ZSKWM|eNPq|!r1j24Kop6B zA=T7K+2Uq#$D@mj^?+4NBIL2%;2ja7J{1+#x-1-p*dQ>LEq`0_{?Bcj!ss}@?yj93 z#&<2kD(ku3vx%jyCmWGsoCn#oSZ$E*cw9XAM?)}RJ`CgI1>L?=O4s*lFJVdxcmIpH16{1>bLIsZ0bi7WD zy#9Tlu>DD>dNI|(5A$EW=dL*WxR&G9`TA&){^EySo$@<#CVoRj>*L#|#AKOPI zSo(|N$tW{BWN(t}6{3*6SF%IMCVQ{%y5H~nI>$Nh+u`Yc?#F%ouj_aH#zM@<6gy*R zcRcD{d_J9nP{-mrA(^mmpv?>YmZe@Onqan8z`PSk&BX(xM0#k>sbl-zO5@K%#a*>g z5O(J=06EagLj1#8sjur*R)?4gSl_0DrtN6R1KbE?YsfAa!pI1-CRpw@s;>FrGLhVsZtRW9}Z8n|BKHvmr>b>v}Rd;bjDFaY$7&MZQBCV$XmAPI0fjaD65o`PFKfs~7D>p&E0Lu_a zzSuAc!_eH{YFFb7lhbx=#`z63HN@>6K1xI{hqRjcDtSL=XB&%KxY;rb3PQbvq6U_k zD_;mfwhI5I#bSD)b8Ab^weNk{l3r6y&`Bc_$iE&l*3U!@byUTPu@i8bYM2KrQP}?cK)n@XeO36gF zJj}do_(vqs99I6ER&R(xq3{xEzQM>bZcE?5?*9Ea zqYvxS!5}?^yR>pksPbg?>cRjfY$pWsh1mJs6YO_Q7V{7dvJ6VVmh}Utaid@)Gb9}p zb++V0nRI%WaL48$K4qb(vLB%{E=KvD(;&u&SB+8L2O}D>1IxAG2z_}y+Nah77T#J= z$`F|~gex4?F2?I5^6m&vVdmD?ry=q~=F;ym;F?-#)+6gC^e+z4BW*G2 z=;NsLy~=G^kRUTcNyDj?-Pbv^f3?dLK)k#TGlc!lFBQe1pBxQsg-y!?(Dm60oo_!M zEz#?NpXq(NoZ5oLTHl+scf>knw@TUroO1K4r>`qs%09qa+C+c$IKyJKVsWGKD7`Lh z3w*_-xFmF}IR_qroi_IGC^`BbuvA0{L6CAaS~UIRyw+JsEzZI~yIaKh=%ICCirgm; zXgVicJ2pLz?T?%XMbOJ(Vi}y6=*qa-_q_MQ|4lTY+TtA3(E8E1p zZC$}LN6JhpxAF^jl)B8BsIo2pw+JyN|7^y(ZDwf>&@^{rCd{PhVL;Q@L3%skAPwTF zpjVo%6N>A?i)?34c?7eXhLk^j__tJSrc`V|$*_adpqc_`#yt`TAqA7#zY_;r2c!h@ zH5wrTMxJfp-pc)95~J}4qHpX{0CVgvs0{WJnN0p2kmCsE=H_7`Yu%XbAV&e;59lrt zzGB9y^2pjK&A}z*Z{_fVt8e-!S16oG@F3~Y{*`$ z7R7(q32E5)&kt4(QtXQV_EQEJKHs7Qb-0Tps<;}|$}iltJMp@xkaL(Ww@^~r1|0ie zN|KDpnT7R9q6k{qtf%2m+XDcB9?D}GythiKik|Ojli(df+EbGh88~wRv0PP+jg2g` z()-KJKoGTiTm!;RcYD*laIhZFrj)K56y%7XlS&2L7sP>}Qeq61$^5X-rmo8xKM%FD z?Or&TLBOHXY+U`POy^+lR#w`8Rk@OZ#zns18YDjaM^rM0S_^BrZ;m#7>Y*zzTy0X%oi^9cd^&-%~M-v8K8VUcaBe|$0N7~ES9&1}HnX#m< zC5(UtxwV(9XI`=uqKqURNK(OQI7^qdPFN7%_e-|ozB@o$gKiuS2@LTb7&g0NMV7F92P}SVti3hYOys(AvfX&F~ z{QRtniWoh&i9jHlAP%W9U8X>&ZmGte_|r(1&=W{2ol$2uS)z{3Kiu@Zv1e}Lh25ra z=Y4Fy7N$#|xK|A0ua-)0LdHQ$ztiB;Hql?DyUq2199gqg!79ER+1Zn=-i`hEWF%GB z9MqF}7x5Tmuk=5<_pQ`M3QUPS4-09WeLswk*=xQxc6sp^xY)~Z5X{4Rm>aPf1Y73z zRfh6cE$F(wn|Et3-Gxn-dXtaKzfX z$q@0Naydwvg;D1{oOOU+oU=c4Tp zw8{yE3q1d5A_aD(kP-_$Zx7^MBio*xvx_O(Pa}mIoZujfw48@CukGU>>vaQdzYnan z&L2Fz1XQ_+y@RZUb3mYlf9w0+gQrUgZ_Rq*9bxLO^w=@}f9IdQ2z##u=cfr1Hs;WH zeh1*<9ORz>Pxc#q+pIQO{h5pCcD~fqIDg*Gl7 z!CT^(HJp%222>{GqU{<^%n@gtfk$W{w1m1<+IY!_Ah3uYz4p99mKS`2kP`3HnDHdO z6cz=@nh7x!t9P5dIjH?q;OyrMY<|m<5;M~ZEgrjxZvEFA-Lj=D3*$6Il+2h>0w-ix z*uU>;#he)S@huIc`DcIre%#H0ZNR3OQFCK_Qn>Sh-3h1ltZ%|A=OYuKsgn5|&j|o5 zgFqDka`@`p9&QX6x&|6YaM%pH#|fAzN(s0-|CiKIvZI!VX5hnPJK6eCI<`ChA+%zb zM%5X7Sw$SYP(JR8Ck~DTjaUsaFEXd~SEO$w*-$_-ysc}T9XAEoyF4X_7OJB5`(v;5 zWzucXgIU^IQ9_1X|I*9Pg-t--%>P!;g7#MW-fhkafw%9?ZG~J#J9y6yOtmXlN{dTq z)8)X<3z$B@N&Uh#3c>E60x_=CR#pW@{Ih$~DoHFLRgqR-p`b~;dm^g&X!M2fYtkmE zqTcFRe6V;Je@2`B!>)#gx%(@{QfR^aPB7STL$J(3N#Wy4TMYt2h}UyAxenqVRqZga z)K7Au&kH-v44DJ1_~DCdqAW@60U=kVh~S=CWs0ay9ew~QhGi7jbKf#XVU&0NRHpZmvTDy?8S5Fd^H#;A7cJvKVrv; zeDkJ2W(>?CQQEQtSc!*}cs(Nr3$rU3*2dm0wj1rRTD|;Nfy}-tD;N5^vbv=X@VCJ_t zqR*^{Ewh8NKi~E|N5TegJfOOvb1Dcjo`%C%>;H1DuJoj0T9I%*_<9BrDsv8rI(D^z z(C3Yn;c*{dnNQcbZh(6i0^qF<=fwg90Bd5{qM~=2SIVXGMk5i^z=I1yylr3u4nC?j zhTFbBf7&ihHM;El%D}SGrK6EJ@LGEOWubtleAg@_)@S8!%b4lu*uxFSvWV5nqQ4w| z;=U~J2O|HOax^I~^Co_eO=7U`89v(ioM`Hv%)6{XDBvvl_JCPpz;z`Fdy+xRW1gp& z4l+kBODzLyp++DDB%Vd9nJSp98J;?nS%EzQU_rHHT%B7W1x)|-W*aIx*^kIar4>CN zh+~RkdNF`$$?>%5R4A^tvOQ`nQ*m)AI23YW+J{SJ$f_FS?&HgBqKApYO+peB5OEQA_|_%oaIwMJl|zArg|SvTn#PQF$V<&@y-R{e+BaT3BsO) z(Hjv1HokWrNa74emVAg$hU@z1j{k%6H}UaAh;P zZXQm(R@7BsU>4SFf&LUnbo($=ZKHm<1OP$E1O}DU`RU}WV=Jzt!<2lZiHFBK2m?&g~mt?nB0R{XXUDY zY1bH%kjm}$@`Qc!<;&@fTzD8nhrYg}dtp4TAtXdoG|NG0s`W6; z2xh{zNqKJ|9#Zv$!rHtwXhsBA@nh?#$ktcevmh^H+5jM3o@c!~_E5N(D(l4a*aQn45?~;v=tHRATmx7QZh+ z=m@B9dA_|AFgGuNC;p&PiS{u^ZK{%N`N5sj?tf_onB@)b&pJb?smlW()`mK+D-;eJ z;2mkYH9X^G0@^4T#V*VPn?YQy4c;VJ2m>DuUNpMfRJ@!q1c%WSDZR-w8PeUb~~>f24#dp3wy#-PF+SBgt-En=Aeu2*MejegD-{Gy^({SATI4@DskF zf+Y;b_)vqZ)3N{fnpnemiudfl?*xH8KW8hrVNoN*Xk`@@nslGjmyW&c9V=7hK(?7r zwl2Z_2KXFL_~?&k?|=YmWo6ZB5+z8FjHB?vNgtUAUY%DT>BbSV&LrFm-3>9h%{>2h z)-+I;VBUX>ZfBJ9W)@TK;+T2cMPpZ}r~se71_jl zYm5|Mm+u>KXo=a{Ze9)0`7Xjoh(rbu@ubK@OCu$f;ZdqeApRRxFWHrqD&w1O<??t?wXBqK=lr z(7OC^_qFOBG~&~fJHq9TIg;$q{UAEmQvYqh?^Ug)aa*^4BY!t*mqE3}S#wdu=y zpEOxQp{Poxf_K|IOa??i)uN8=I^QD~wrHL!LyD@nq-5~t&!2JVW2I4y!G3ohiVVI_ zqk&kCacfXM4bv2O3zu6C zQ5#iGAfM{4xL!e1W9Q)D`_HAN_tMhRj90CDT5W5t#N{%C^xy`vLrb^=Mk%7_|4wzp zXz_3-Y}7OGeITecfE5Ri_4_&4NvlE8OV>Fs&^%uwrB3Myxu@?VPk?z7F0Abvvqp%9 zz8*dNf;@KI?P3TE4aHPeQ=>Z+Hca=}X}cHLv4AW2!21iB5D>^FJf3c+*)1I&9i;~o z&A@>##mLfWn;V+;N7*GM3iub_^+62ZNFa=I`%)16`KXZAUl8>f29?kWUlC&gQe8Ti zc26r{C2u^;Af`+ONaAit1J!JzH{s9|E+%bop6RKc1)?*sFV|Qdyy?o_#_ti-zNl!z zFcF^>EobC8#10)N+60Ka3(leiy4?rynUl}GQt~*HEP!9f_JUmZ6k{9+$(o5 zd>NTuBq=KkB=x^@;G(*gW%&gVYnp(%mg?Idu$qA_&%S{b>LT4rE}~ZD z+!p&1H-h0{@vVc4Zng+-G7gmPW~h)@Q5wfH_99ez?+nuzlRorv8DJU9+ySLRScvZMWCd=@stdh=Wno zsPS=r-hbmDo)J|hfQ4J2FahC-s4{eH?P>P3(oOslpD$@4A1F*r`|KOB>_;)Kbb=U` zusb?v`?fr`r9jaB z`LB>mi0w9-zZ5NA^YUJZCJX((XZjR+=;`TM+m(@!7>}T$ z4E&zLIhs+@F*{?}Dhjh6U>vxG!+4Ltadn3-i&0ce(#&z-VC^!Zj@s@-S^WGmw7~M$ z^5M}g{+*pES?2ljsVM_h0xF-w;+W$z8qTArqiOeym9nD{l|$bhU*3hPgeS9JDhOdb z&23UzmC1RFl~5eF&s@c8Ayu+%s3VoK$gOd3{h--~C#B_OEMWl5qTy;RfoSd zc5zYVN|AR2+2XB613en3o@CIHBXVx-Mfg>CUZqV4RW+53`1Uo+i*LK#%)&8$qCv$Y zg91@MUMn4^Y)^MPY1)W%Y^Y|u_T-^#i7_3_4Ez|q@n5-Xg;n)kq(%FpXI-z)3b&n& zBm%kZz&TK7(f_Uq;nqeR`5ByO{vs<_8AFdpN`2^hiNeNVUJ&Oz`|g*gMorLyzWg@@jAX{c~(w_=}lT?gbaVw&lA5QmL;FU(FXUGPlzB zdTSSHmW$c|z#v*W7FO!LX>DzrncQTE153!UGb`)!KX)*=xVVI{1wb-y1Lp=@JcakZ z@~DBYci*M&Bl?Vi1*h7{>eZPM^OJ2}|9a3|kA!mJDN+rVJCsdZ*?Le1VQagcRAk5F zX$mwNczX-T_#TDFU5asqbXl5~J$a*FY5_mE=lM;_0t*}*;m7S7Tdteo--#YoyN+To|C-~70e3+{YFap5BmTRKo;(ohcgvF>&aN7huk`U)in zpB+@&p7jl8oBH-#UH7^%s8-goIdElv67C!GeZ8J)pD#_Ds425!(@80Rlz3y)%9@`-o^24n*_7)==@Y4C z%oSHYwAbd7^@3bMQ;V%em0z?fnM0d*vu-UNxJP#pt|aqa>1K&FD4EZFpHZKBQCrYo zNNvKC5mNQWQjqJnOf4~NFcp&K-+(Vi^Yf~&0Z^bfqwxnA;6T22ffBgQY_-dGlumi-i2SxNuubtU-LVR7is3LY2Aqs zU&TII`oR2-{-+1OE_@W8->%8E#@yZyfIJ+V1~J9XsBDaio38rw*$9M+8P;AlyFJBH-`S=*PZ)IUlbQR z{o{sTUM{XtA_c(%RV3oiZPwczJP!~jTKoC~BDZhe2mIESML~#O5Al5f(67w23lAx! zU_Az%z+wnP`X4BoK82gm5%t5U3=qFnRl)R>brE~G#6oIv|I|P0`K%Pu-@Ybk^-(Wb zBPv>!`#eNFea@>m7%n>%f(^W5jNDgwU6cHE9_Kr4KPmP)#4^8hv5mN{@WpB6CC|Uw ztoDqO{PKmj=Asu+GgCG!n~leIeqYGj)qd>8omHE~;v*;XjZPySD9!FH8~L~9ZQ8)m z->3MmJ?^7W@5>X=DR$w-vl{f@w@4LXB$)r46FFZ{SYlOQGQ8@s^N3uM{P4a|)t^cr z2aR<0VM3UdM2<+lD%8XeQ(HP+^y?8Ot~N~+XFf^w!rw5$+C+0-%)!skhjaF40v8iG zP;BZCMiVVYv`011604U%WhwyjJ{G;Y5B>cL$P#^i2MpjNA;25injkgkaN6A9Z@)G3c*c= zh5(fU%f4|Aehh(jLEr5Rl>glU-%ETY*=NMJ(R_AeGmwuN2!r&ox9atyrZ!3jJ`3yEtw(?a&7ny^!z8K;F(-j1cJ?1I467^K z3R5N=DKB}FLP5~9FVF|^dx&O?M2>XuREV#- z{Nc;?9BEHcAp&tWiV%}3?aw2m`vKP(Lq&NEXvwy-hgYROR%kvl|NR(U6l#6;jf*r@ z>pOl83eSo=gzn2`4X+Y!+==a8Do>REO15@=(laL?b3%j_f9M9Vr=|!~RcZudnc*5& z9G>2fqF=d#KG|UUsqz)nQoB5VLYmkDheDOo=Ndo8cxLl$D87yOO`D6ym{IQ#pw+*{ zuE}vNmS9-mwNZERByUh~pK>K9iQ*W?H|9vsE@ghFke8!cf+zbaX>2yas9>~?R=Rcg zDPJmfK@}gW0mF_eQC>?Bw%VZUq%BvT*gs7U|7r8K%`dT!2t?lGwQ#%o+zgkvPW~7j zWdg>10s->J?K!t-r9YMOOI4}oO?=qytE*1(itf^kY1=!~5dr2X8wYNSIrqYz%NMNE zuG{K`H@}UD(ngzSGKldQeALaKbDMDFC+FNtXEx*_cHrL(murQ4!-*?PO(4Sf$M#?f zaf)JzDH@z5*Im3LDiD{eZ$CIj{!!ijJ#+jjrbkjZ-K~>);s`W)6KSt>Y!dlfTJbu2 z_#t`QE?=cSIzt_uJdHC8+>uJXoMCHw5x8UNck^PpI5TyEmh;tZPvGz4@4wQvZZhYt z@yXDNy$ScY){kkwLq!YT8@yHgAzJ_X0uNyd=|S>ceCdxX*e64%Tj6qsJjuix>&Azw zf3eK0BG7W3;!faY$CyS)a$;}QJ#QE#Nu+o#kY?s%;BH;QVDivZoQf>*Rt4&|hrll@ zR~V@1h*9)I5!1t<`V znxSiS2F|53u+fnUfd;|%gM&C(gS|(6;AD{uYZg*OAU~gI5~<}+Gj{wjXLuFS{B=SX zi9}&(PhEX23L42WnKfCSxhgtO==E+a+B=D5*=M!88|iag<4R;>Ogosfyfa0*abSkU zCwP7ki5AOjTK#*v6-}p#fuP7kYpM2(v=h(kIfqTTlJ(BGfRB!n4ZaIp6=s4*q1p&=gSGU>+PiWz0l%zgHSQD zXE9s5nB(bFX^te?XsK=Bo9%!C8>A3m?!h$tGj_=GFX$eg=M5_tj(}zeX}J*vlg*bQ ziA1?iBjJDjmrH#wm|=y*2{+!9qgMS^2kZCkYZ(_C7PUZdJ3%3tQM=+%D?G8h7;j&Lg9h$#V_t^1~HSKqK6)tt;Ag+g+lSW=LDpYQYrWIYD{ODM~&M6QN z&7UH6NQ&OT#%uwLia3mRlB2UD?}l69gcvXHlT(+vdWBLG>iAY0t{3)P&^eWB^+JR6 zsEJ>Lk*bTsM^UoJEVGnZiZ3OPdF6SYiukYcE?&8it2mhmiOLXn86{BcgjmL;mt}-# z;-$2S2KTT_CO(QQ{J^9JB^FBG?Zcp{1_2^j!h!=nk&IwVYX2d_OwGq8Y{VF+g5xZl z(f`_<0zGpkFnx%l=-Em3H6Ja-C#H7)g_-69vzl62nro31bwpl^I4uxk3=%wJRqfd* zq}j$v?aY>$Mek&;V4lJa*bln?C;>HZT7o*3VP%stx@#DJH~P^BC4!rcdHznh5?Q*R znEzNu)IRL;$ZOZgA7%W&wB{w(in@Y%0#45_az`;6l#YryC_*?(OABrF8H}ZGdkQ|z ztAA7|sKJ=U=!ae7zau3|xM1xH)<+?&m$(V2JO<+jlqSB8CtDJ#EO% z>zj`$7iA(|ZW=2XpFeb*dOTe#jULu3u z*Smh20PCiNgak2h;`;4Om?g8a%6Yc)6}?086?oqwbq<~?a<5Hm0~1qI(_&BFqMaRc zuuC(8QFH*vO{ryV|Y{dy&Nz!0Pe&*}?Z!fR&eY0TW=km4^`s(p2y^YtW>1gzok#~e_|CUd!Zybta z@#IB5Nl)zm?i!W1{(&_U1Gp0Nb7p}&yI zYHn&8U|T&ARO)k3h_tMr_c~9Zu`06ue!5nOK%>asV;&koPVmsUy!7!B0%=X<0l9Px zEp%#&PDx!|(l8zX{)iL{qz6TMT=^%{E)nl8EAvOJY|5I@Zr{C&)K_Pi-%Eyez8kC^ z@WDbxC@<9s6J7zKNFhv6Ub4uMt#fus3=y6bI*KU#ur^r#{dIZ@opP1-9pGP1ZPF89 z4nc;cfx&D?|G~3`#ys`HozwMFgxl^2C3APJu=MBZ4u`!1(rT6)HeG{$HD@9A_I|#v zFPwDmIu^iVRi)%Zmsw>%ixIN!ar^BkSv~Ehvow&=*ncA!nd^~j+2W_vWJb-*D#zXh z!x8ELL+HZ)0;QsPVaIa*N;@3KPa>zhJ2+w?I>5?Ai0-ndqFIwxVQ_4M{!+u*HD;Bp zUau+sI0O1I?Ov#96(@KK!Sz*&&}&S5U`vsMn$|r+GNPTP^rPq4G_~QMVc`}Gft_0~ zL*AK0^LnOHP~FH*(3+a~n-C+Jj{81IjATd)9_0QwO>W5)?6g=em-|a zF{ncb^iVTMI@y@N@h5j?4zI3kVh-AHb^XQMmp8M(GVewg!@r;EMoLuqGV<8Ko{RAV zbHD{fJ9P+8inCG=`iDa9W>nS*FWj7=XB=s{CNd~#kC1oqb3&Av^EGwy54$Q#bF<0i zFf$iV!CKLfx9Lh#_o7YdgQs4Ct5UUotZ7no2%~?!9_M^xHXD&N+aRgA4Yh>5`}-HQ zp-MLtYW&6cWTCTUdJcsq+AQf!jmo10C6+3d-Sr6iwimgZN4i;Zynl>X@H&jShv!@} z$JV9I_V#-#MoUN6!X=>*V|W z-ET@qNTFQU`3X_)aCuLQKoiMgUb(i-CVQR{o?jigh5b9x`!{slcZlv}4~x*Ie|ka& zEK{QzEnw9X$e5XmHyPD%aNy=N@&hiWkMA#{NU!lRkC`oR%aS@`wh%49h&YY*&!%qf zu|dfwV;nO|@sTohTF%3s+YN=Y+CXH|L7`mPd(RA?Z#d8=sLKiJ=TI_DOa&y|#}!%P z+Lj$nI7mY1f5NA>bTFXT5$JTYULkmPcxXwJo#%dD%c3Rzm30JKxmb`wO{M(#*!qEr z0x{>ZX8wru$BGr#j16>a@E0T@u@*+;fJ9+|rCy87w6W5N1RBY~vYu~jNAHF@g!p2@ z!?0u_e0Cr3_v?Q^2WbRE7(XN+mLI;YiJqEJU5Xyu3qEa*HQO5iX}Ok zRhh&8No*muO@h?675M^!5!R3KapKbYLlT%Cy@V7WCtL?j?a=hJyrd*LSl}VN z?#Rg6{!D9!f zzNuioswgneoIOuwt!N)v*So0j`|1oN~n}% z*fd6e087jQqOGF$n##J1QYT!adf|aLy3$Kj6cUnkN@ScL2e&tr;~s;{W_2CgDs%LY zJ6Slg<`9MYh}1`kb4!rCga_50ukZ=kesyb|u~zY`e@8BI7Y5D4L$^LFpvKUeaNLJ8 zmWwXLl&{0kK5?#`@j@_bG{TuFw!^N*s47uM0YxD=!i8tR?~=k?7Q}(@Xutz6;r|wX zu_tUksumbm6+FI1E7EsH5Dn zH{w2>CgWbkekH?(!(XJ)!s@cnYWkH{I?biUr=ov_RcNU8=4Uq(c@%{jbbIwC{B59oc*ONE4#Q1`<8CP8Ds<|g^WIj5J zd$wHW7rjR3tB;go2~;Sx{^r*ANsK8a4{AgTD@15B67-DQj#gLPgbbwn8nBD1Upa7Y zyZZDo(s|C1l`W0^L8elHA=?TJo~i||yMY|_dZ+*Z!mK#7?n=-@7;sHq{Dr^X%t0A< zbhmMh7CZ6nBI2aXmPJakL><|JAyt2s8(&5>^L2qK&hDE@dQs#`Da`>ba7|BSm!Vh; z?cm#*VR??&oXVDb%!@99tT(~XbxpnOuwsvuT86G^-xYnTok^M_(fSg(@xT6R|8}o9TDY-WXc4B8)y4a46x6tH878mG-qy6?)AuOry53k)*%3{Ynr4Ob<6nW}s47YFO83RWlcM}IS1!3K>e{@v3YILr+0jQaai_80gWgt>Je}%F$y8nI# zpXvq#39vtZ{v6mLf*UXg>3Z$7X|IY$xW+JKc)vA?q~NNlpm7M=wnecl^7{|I5H=-RGF;M{(PzG;(}tGsa@P);fv7q4?Hit z$)_Oc2GmY{6BL+%7-$w89@BR2)ARGB zYRQ@H9`lWmZ10$&6}E|%>$aP6{3PFJzb{+8F<_>Z#H5iK$sEgS%yIwWhXNOLi=y4w zB!M`qw9H2Qu&!1ymJxbNfI@PsVJm zkDe#&W{yZH=Jb#5Aq=WU^aK<7FrlQbg-sV+Me`NxGv$CpOk*YStCIIoQUc<>?l|JO z;c^mMgR@~cp~)vg+!XNtl6Wj+D`@7|xKHc+K32TCjYhuW;_@{$JX~^h3)?tR`+5uu z=Y3gtkacpgQ7Qtd`#uFg)wt;(H*l~m_95_ky^8Cmye5k|efF)}Xx=^8jjnq04^QjB z0mF<|3r3|(43R2rdp!%P8}MZ2h$se6O%x~!lR~C@ic>l?5OVE!`>r=Fx#*JRM~t!` zN`G{Ig<$^eD*NuigX^x)Lv+WHz`};EY~Yh>zC!)ojoVR@Wm=E+-3KmyKI|!Y4)I*H zf7fl|T!_P3wFDX)x$}k>B4~f!@J@qvkBo<=Ry67T>~ngJR#UF|GOez^p}dir@Zbd1LCmj!WYFazWN=Y0qwrNm%AXc3*P2;v@Pny(xI)=ncG9HIf7OJob43D~qDIe{HOM{W|Jxk_xt(uoFV2?+_wF6g z|5{sbz{|tk1O=dHJw;02sAriSN7Cfum}#L{?%F0a(sP}u_z0d!jFi5^^wNP(fP2B{ zG%(R%+!k7$>91Wp8XEK{q34aZqC~t|!3dKm27;2o!Vq>kW=OIB_ATWm77+;H$|r0z zi`8v-Qixc#*)yTLK@>ZXGX>iNDWMF5;pab(AK-}MVBGKk=MY|4gZ@99J~;^o_Tuw} zqDF^Ix8nXR->Md3Ilf$tEmz!|xdjdcuios&e2}KkDTHCRg0?C=oD=o0$ zWQ-e;JC#cp-}4+=cPS_%l?AievSfLeUOx4lhtb_j4*cW;X>$2OK8>7Oy>!jU(H8MF5Wsmo31@HBg^x~^#{C%&V1};vmiO;R{e1s%^mVcAZDjG;X2reW#`j6=jJ_wBe%ztHshaZrs`dPl+j}W< z==QK9poe*))t3eZZA9 zhvuhuecjINpCURIMrI@bJN;4yp+zSgMYBZVs`y{7ncJh?>^z!W<7>kUI}n^pOhIAp z;9&j@@PMG8_Kq}P=w=Yy29N<}cQagRRcne}pprx3Vx%!oovXUg|zfn<8` zVB`1|r|0!fyz|QnCZ2@IizxfM*HA1+=k8e~y!!pbjXhyzN6a@p@jI}QMVWfmgOX2bN z*5Qn89=G6!DDr!{4ycIi8fGaZW(#C3W>sfi#^fjj;#m_m)Y5nW>~giVw8+WH{c-y? z0sFae&^Mn<&0e;{%Z}jfWq@$*>4n@C`*E*@&~r9o)1CP?o9YR3VA>8254ZT;#L~9C z!e6JY*T0JX-3f7`XCbIBxkv=S5*6Y%5#a|6hAl8HA#G8FL{71-udgs*>*(J$DzLyj zTyIK#ecwwyBcsW1kR?~hu)VzOJkMVB;@%`4r~%afhL2 zQ)loP{0bTqH(2Os!WhYGsOtVAuzagK~i7P1N44b$WPS%e2b1MlLlX<*VUa za%GA~f6SG=DCE|T-x9@mF>&p29@D9W@k1{PciD3=C4?t;Y7@u?lwG?_l@&BQTvUw- zp7n28m_6z@xx45W9>R=9pA)hpMdmL8>wUn$Wr@B_rHbRfyf&jCCqV<~kxZ__CneLa zbDw=aa?u~ATp?47&oGAQFR`fJF>qR?orxu1=06LRD|wcyZZh{wvCy_$H|t>+uMH7X z6(4p@Y>j$k`OLY{f!8(t&_ztp5BHTs(P^<&U5Rl?^ZtQslj?q7H$%v!qW`lF+cBlW zvHT|_@T}3NcPnVE4-}OSbrE}}MZVRVzj9#%etGY@b*Z2*u6%_dgy(w(ehEHKa~9^)YzTBT28x(fB|d7q)Gl z&6f}JB1vWxoS<{Uc|7DDwjJzUjaB3j;$8B(lkC@ecu~40%{`7o`XTOr-ULwo(K@S-V ze{t`y`;+l~PI@A{Y@V`lS2E|4`l{J9jWKAJu-;M#LYe8tcjx(f7>>zcU}S&r(tY*# zTw2}k>}ABalH&3SA?$rm!3q|{UZKI_J^ z0lPnC%WIh*NT?-W=P}ibG_yjkNY-gXRFfOhx7SF0nsG!Oz#8AVk9*7kgslx*neB?Y zCB6;uHPmB;dgN&12i_z_OTtXP8T0V-_3GkFuHf6u9Uisx!E|;a6Uf^DW zC;z#6i#{j1Tq^agz8Vhf^cSC-0C}EhV5`M&um(;r++S-E)fLy^*QwzlyI@?GNqHi& z-*LOSJf5Rbz>P`Aol*}V#y#1;m}WNW_;YXUyORyZDrhpo^AcyrfTfo=V;RHxVcVrH z&uXmj{tmM9rv~>bKcTy@o)2yrEwB)>JrXJ{4f~HnFcBXj1Y=+8X2o?sL}h%WZIlOIPJCZFo|6NW-iCjk|sMuAjCuc7_DBwgIXw(wW+n?F0nZi_1ur!Kz9 zWb|_n>cIVxt+kH*i)Al!)*r_YcnBegLM=1s#8yCH1jx~Lb+ox$Ipq~QJ20B zJ3#lNUhZxp8L5i3?LS9f!uBdlvuK$CXkm;mgn*%r&A)wwIV@VdZxr$n?3H&VcgQA& zIesNgo7w(u1ZQL8YlU_JIjmWeqQ=Y&K)I@wOot2yQvcjj8vD}oQc_t=HqqC@!!Pi< zAn`1rG6ih?zoyX+KV3R--8tOmU1wd9%u`8YMDuJ0vT4N$H7o;Rm57D@8dha44R1z_ z+pkI}<4yS_2gkC|1bQ*GAI3970}I=zDc%C<$b3+EMC0P{_l$%8T>o1u%%e_3Z zi*0^UsPv9Ks|M0Wdi- zAw<~*M?P{0UjtYLDS?QOTTPAo0l$@HQQ!?PeNogaq%eF|FH~hBL>4ocfqq0N+jdCC z1VWdB1O9maUSsF=y8=I&1F=9xx+QB_I=0T0vX$1Q&O)RyC0VQMV=!@;sXw^Nn*V?v zJ9xo6t#R7UcRgdTR-`-4}BB&^>F zFw4PS)oCfoq5`^_3wws+TYtf$-U7q~*Z=b@M(`E{WeJw1jix8Fxwj1@muQay`9$XU zLU4dABI3+ZsTyOU zW1I+z#jeJS*JpojE^M))BHM@AV`zQr!d*po`(bj-P<-JrOoN4zeq5IH4fR}DVhU#dhC^c zf5(?2sAC~pvuKdN$IbwwNr$;ZldCjgsEZdsP8eSR+X=DsquZO}EW}@Bp=u1466kxT z_zql=+&dK;l-gg`P~rR{l^69H!Je_?GOkK|7+**n5v2ZJtVF5Vh6B)5ZR<}n?;YGG zL-Hy*L*?vwpVewVk1_k6#+k7Pj8pCUrt^l9Q31x#KWWj90oX+Q?lmE;`Em_}&$bT- ziGk0m%%t&>D;Y*JxZ0*VnE%GsVefW3LLAB!EUG~V`JKE(ed$%Mc8+#&7NfDkpLQ;2 zp8yhXeyb0QIGE(#&$ejN_IvJ~4X(Su_83MaXjEd$ReN3mVhU9w%#wN-voo>Jl%>ma z&l5ni)))7e4&17i+;~=XQ4^*m%3tKZW_o96$s!7^laame=al{>$8p=gm~?giR)a0` zC*D8=j=x(-$4;w7ozyi9h#3$7N=vcCmEX0hICqP^A^B^}wiePl%#u7d<P! zo*wc0fjla(BS##$E5L!mr-x z`Sv@!JcH#QOTVg4`Hxe0eB2j%FNToxUBo{kDy_H%8O^T4jtI!4ow15SAz-5k+?ypJ zYHAO+%wJt@T_HA~ub_L-^u0JZ0eyZCP-g4 zB0r+C2?+EeBu?wndbA(#9~D88A9swhMlJ2X1c)mIj>D5zuP4&0(ws^Qds@2Fx@&;e z!_=!Ss?QbOZqB~2>HxG|h)9uZc-FZ^7cVvZgzg86$Z>$0g|Dd*)7MyFX^lCzFaH& zj0;MXSH7HLry@&I)8d!dv~Is6H@7b#nGxodYO!z;*&#s3g9zL3mrN!?fYEMkQ0g!$ z+|=(4Hxo!UYfk)%`Eo-65H{#4#g`#0rX9)%QM1{&{SpV4QWrCXeZakH>Z zj3p_TVIxmB|K|r<=y|Fn{6))u<8{j}G68Fuqb)@~->gfRB7}z7FTIT93q1c*Vv8=- z&$3iM*^i!H<942gBuc{lDzcl;2={OLDb%P`ec9R)i6_|2X!Ys;?a+<(J;|drgVM|Rgkgt$!>_#S2A@l@ ziJVHwF93u0J@uwsUvkj_0)-tM_ok+Dq~Rezw}b@ZpPsC!$h!X`C618Tmw29Xy>;7z zAfqFq$H<;P$|+Z-LDdcmW$3?QM3WRT3u;*wzIt`-EeZ_{jV?St2-#!uI%CA$2;ky^ zkXW8_kiy0jp#cLc@M_At&zuwNexD)|-e}rYVW*T|gU6IBTlTEM?pG16c=(vF$!)2A zl}g&b86Y#2aM$pkHBdIGVNG7uv-}g5r}JvJgx`@ zzUjR?za3p8{2PRvSsPce>4OXsUMTzbSs!HUV&>tI2&pyxjaL_2us43B3(jwd{uzOQku5R81h@A5 zYJJnh#KZ5*%XeY7L|_vj0U8!TP< z$Tf@O^Y8oZIh(DvA1`)~(fN7~F5NN|`%N1*8#zNl1uGymn64_Zc?P&q?i6SK+*Z zvE&)>ePX4<3r^KoAAr9alDC3|PD~N?bC}Q)YKNr^big3701SdCHbj%!f7-5Twalod z@7$446el82XBn{8@~I+NsH&^GHfTbEULnses=-uwOcI%oB17Pd`}TQQ(9yFn16g_n z$kPAA(|N~Zz5oAT3(3mNEL(`ko{?ErHrd%Ln{3gs3E7(@aS^hzv&qU%R-`*$$NiCP$|Q!o^1a*7K7!a5gd6nQO%_}#ESlS%f{3&u$P2Ee z(0LZ9uRn6Th^qeatsD5D%u-ppkrE@xmLJ;HO!yqAg_K*gff9n6FkJ}C#j1W|6(lC) zDU#Hig_&zQA*KNOVU+^HE!UW?zp%u3NQNUB3(JO&LVvAq@N*{<&6=vOq9jK!+U2D| z|CL!u7|`^%NLeD@1WNpXI#>+DWu{-;Zo;(#vqvxLLrT~WYgoT~@SZtdZQDSsT-14K z;k|ESI+GOh*t!N>Md^y9X0g%Gj`T0K+Hi5g_KSGywvs96Tn3g6ZMd#9K7LhYTL|NF ziTvOYx}B}F^n zl#l72UwFS-FuIBusgNY)g7vvTLd8rqmS|ae(Ly7r-s6{dh&}JVY+CSh(~oyQa&t+> z>0dR0wZB7>11oK$epr z0V+7c)&dX`z~Ot`jR0T}A@Fi;LQzQxjAR|5LU4joeg{$Rwu@av!$kHO;Uy2ZsJ~i) zx?SzIG%}t*#BwkNs64FO6eL4?2xbE0xeW@APP>}*Hl$Gi3tI^k7A54h`W(nQ>k_20 zNPYEhLkxdtHEMYADy`YD1|I(R13#Oa_EbH+bZvSylU>mthSPjb)ALAtqbH0UbA4nEYXu&$fc;?ilK``wCQT2~XXp z)ngtL3#d8=oqwM6;qNb%#Ngq-2S%ugT%GCB6`bt*!?aVNGPs!zn_G})HZXx^>Lr-3 zEGx4*xSH}~&|!r3v|O=XOQY=6{N@RW%VGy$LbxhL(m{k99dbuR?+TIBJ6ZU(B7ikvi?)nYQl`TC8F)?KSx{X*it(p& z&{?~i3SBDn7oNgKkQ#?Azj%BLtr)6aj!-8#gWsxnp7 zAFT*YD#F-?pR$4%Bfv6QAw-ff2p{WQy^I2ivwTPn1vrv|2Jg_fh8jZ zU491~Mo{zY3>>0c+wQ3^BL(1-3S-IK0So(~=Q1>biLp{?vh?=+l$)$Mi=b#~vfzeH zV6nn1D~nntq%e5=g7@4UulydLQNd)-@565LSXR^iCR~ifaT6K3sPn9*g1*m(7%4t& zEmxRy$h@B|z&b;=jHO6W$a&4>Yur3Xr_F^notv&6T|z2|bk+vT_NUCa9Nt!5&~To7 zWI0N)6^v}?=vDQ#+UioZ3eqU>i24R=VsO3i10M<;T~OO-VH}NA*0%!DsTU$Ad%&3h z!vt#+N-(K&Bbx){Yx;ERunNTL%S%gI=aF!EtWwU+%|X#53ZUoR4!Wo_8HUD@YBWDk zdqjgmKWe@^!Kv{I29lUOyX~&gkki(V4qXYtfa&PKWKRe~h+jw`3&LkXRwa-?ZLEN| z{j&R&&n^Y7@YZ9r^w$ey0B?n_`Y&RNwRX)H7hOO$IcaWlk^tl<2~muGD>uF&(z()u`M0bTy^aVnPD z9kQs|*9ndKZ0ex099qGl-eN5!MDaibRRd~pq~6XKC!q0t)`8+n4wsn&Gn7}_BXQzmW918SB*|wMml0!UVT#ko(^BpYSUi0|3L9XhT#Z*ZRAH26 zA{+wKR@Nn7C7~(M!*yG>8&#G|?nPg#HlgSHz~Sjn%w^+3c0TNV(~Nvl2}Gz6%W zz0l%N`73S)$5t*VQvR(%d?HWL*IMQKgJhpkGqbp6^8OQzA;15o*u|>7 zZc-UOBnMX#lsr)0P*;Y9B(4O4^L1zjtsu#QHV5kfOrdgEF6k;q!c?G@_b|V~94-^f zECDVqo1RPOckc%6nkQx@@|h9b?Nsd$#Ccm9@yW= z{MG+&AP08jDTMZ>#TKu@yASe;R`8`*>lRhxia`mCz?E4$O=2!CF5>wGKpW*gJT9&714WBto(9Dl-k-sDvTkVD#+CH|A(qbXqU})Iy&3(aZf(L@XBzUv7 z4ueu8|1-OKJyLL@?F9hOcG!pY8s{AmVK?~TUW0HP?k6|V!v7s|a62Je5kzH=sLju2 zFvN~J+U=pKza7<%JduxnCe*j^H5@A-XWIAg9I}9M-+~vC+|(ovE{&mn zQ+8IYm#T_aGSOM9^l$@d;qg(G*9PcISTUsAqk#g4HjQp+H#4J4zlbkct-z(w>`{bS ztV;D8Ii@n0b`Y5^&^!i3s8Qul3tmogY<`<{yb0qF?kgEq1x2C#B!wh|tue`UVkwd= zVbg>DFgb*!MOxQLx*by|oPdvUmSQ!cUZ|-|MIZ}YS;%tN7O=a2ipULQ>YRia)p>AVU4g{6!x`7 z3wmj|=V-218^8YiL6QVoi*b|hqShNP<%W%#GDmMKd@`l8Di>^ZOxk$2Ld!{IZ`8lK z?ecB$dYXJ}??SZ6TfgGiPWPCi`&ojT%%5=yLw~ZE!cgMH_&S6UDVcrI>485s)%C@T z28h&E_<)H~3%wcKUnJNEKLjqE@k8Kc+wg*T+=4qLTO zZiBup6d!bR@PJR!jh_0F;mjV@z7BVa@xwcCU(0?dgOz7x^RP#;;v;GWd|da5E>Zcu zvg{7$w_dip3BANnZOE2FfZ9u#D7tmXuq9dc+=clDh$1U*5Xol*t#oBJZzfTM)DN75 zX}(7Nb8Q240Q^_%b$coZ+%^zD)%W5UF?B#N4z9I_1^Fs-u1W~#zi~kV4K=aI;yxr% z;Nu2OyZL4*2ZI~-D1b5>mIe@7fpR=$6iV!1yxt08a*o93MD9Di=%LszHq4Au>Uuzz zDb0%6XrgiBv`GBgXoe~SFP9VEEAFN%24y>Awk`+9p>G>R`b}C{ABm%ZrKP#v^hsWT zj2((Ft$Y(U3o7h@9?xp~pTLPtyY#vrB9yq>_8y^gJV|xzx-Sd^N~8ftrW0x`aZzhy zQNlZzU@IYn&@2w*XKPp4)0n5ZjL5ip!8S6A6inKPAKJR8`ahuY1Z`~C!JbdEVUB23 z9_?|tjwy`(+LoQU9zJXoyFBjW5uuO33%(222cm2>0#vUlwDrG%?t#2v+$lX^MY&0r z(u4f`CwL+s#xIL=V8Cqhdf}sHjL8&PlHvK=YY(2+4yai-rt~m=F85&d)Qcx}d`nwO zKlq5@0*Yj=YFgYBVw(3F2d_c>!|{pMMm6RRc@+4k62JnCd84+9a-Hw2RfQ~Pz;Dtl zsq3+8Y%fW_(#?Ql;~NQ+BLsJ4+HyeS3?D?;Goc4uDeIJlPr=RQP<(8qmMzz+WqX04 z@Q@D;{^i(ej%c0=u%|#}$^_jC{!8@5go zEU|e^f0+n0%lgG#v_J7B+iI-BKv0%ep>a@-KAm=E*|2r#x(JPL0f|?Mu6Ph`&}17Z z37|@`;ri_K$NB5R{q!IGgz25hO6I!|jF227S$6;VE7|Mb)a4bmeS=*-q@eqJ_Qvl^ zF(HlQsxZ{m8et*zMXuuR!ZFHv=-3`S_H=)lG?^9BRIj~S{PCOVr@(JTA&s!4&ijXN zp=zntd8u!}_c0%>-d!+EKd;%Cvoek>Q2{wLYy;C&^l_bY4mF&2aG+I&Dt^U9?!z+H zvE@^)oLJ7`*H_hqd~1m1R2B+xl)Jg$|9{{TD6OWwh3>IX^$ZPA5o3iUf!XM);AMja z2uLc6Q9~`xkDq|eWj6m%y;_A~W?l038!eV6zHZ7c%>An+T0=i6M59t*7g#sdGXG`H z@z+Gu-Dd8hk zY-=Kt>0_@p+OByWV<*>6;S2?`fQQdKD^#6~;N=j3_LT~uGeFo04xp)icnacJTAURN z1Aqn4i{Oy>+J)u^V5ZS_CF^M3basNdb7c7kD^)LheIX!x|XkJ@-) zc6vCPd@J8Jg6(%pV@`u+02=RQVTefNm#sY!9DNV3g`_F=pd^8r>$e&N8@hH#&2?M$ z(7{5&OI=&tdvKbPq(xI%XZP^CIMa)5m(}Xc1Rmp@M~5^pBQ&!2rd#-BhWNo?_YY1b zh`#Z8%$sul<)ieICoie)5n(y}lw;a1FgOo=f-4XRVxoOI0lN~dzfKvalt`TtC30{F z5g&mIcvr^J26-2lkRNL@d5D+%S0CehH=Ct^cTLq*?KV(n1B^UE`ZOi2y5@&XF z?$axJYJ)5o&wLtOb57YIWDq=PaAXjj7dXKt{@n~$8m6Z;7Pr^49C&qv8x%f7LRa}M z?R{4qWY?tK&@UQUN+Yl})i&aaw=%^(6-h&Z>xc-}VJ6x}CcDj5j?IkDF5#LSeD8A^v?3J%9iU2%dW~=j zhjAHXprD{ywEEA51>rgbNI`*|`D^=`D$fi>4GQcoWe6o!Ur!$)%COt zsX^9N9-QWtl~Le?gjkfW*WSRF`;atBVRqbcefQ*>)<0_~-`v1;J#mhRkCG*7;AV9xe%t+vk<@fnB@he@s->B37cn86|5(3R7N4E&VcP@S#pVvyzSf#DjDx> zd_h4${`IeeLr#=o@Bvsb34mk#KRZfXPV8RbRS+l+cV?<(nQNSSWd*|4%d(R5_Om~Z z{CCoWjd@cNM@NSpS!7LjxGCVidL2i$kU3Ht1zWSF%`CMdSdfDIw6@y#{lx}Z&f?Kux`USaO&T~lR^UM~U@FbtY74#1 ztIy>d84Td)6-6(d@JRYXP% z5*q++)OY!vL;hO~2yaf|bOTfgmR`&xVR!#N!?As8*Pt``%8}~<3@cEdgBZ_I$kQ>+_L)#iU5)-{S4 z27-fa&wOo5?Fp~VznRI1DrdUB?iBu3p@Pq0f_Rvh$uFYfQR$>Ub-Jf>xYDa~p2^8c zxPHJnenj&l6ejfFT96G{y8EA-$SXl!mR0$$Adgy*`ani90)5p2)i;o3SfC`Qzzuqd zpk^!z%Ths2h&-lgQhHQa;>N(Cdzo`S8p8Ayxe<^UJ20Lj#*?bajp5A?B_rE|85c+) zfF97)2wVOp5c`b8-#nLFL_)dX-n%gaZA}6R+85kMX(1xqB=9-#q}qWKA(*O&)#~_6 zO^v8&g0(%LFB#NsRySk7Z zZFG)syhRvIPhEMdp!JP-F>J5?WDrc{Ng0=Qeu}3{bYxBpXVB&zwo_XD3)!^d)gfJwF;~CH>M`*T`OHC$Cf0%WcY<|h+tHPX%3!cu+b*zv9A21 zEg&fOO4CnPIHIyS0{(^r?}2U-jx$POV=Z(WI89V0!pSD*H?D!(WA6|!nofad)Xst; zurb(p7iERa7U49)kj;`eSK~NPr5a0O#`%*!Daa&QG{20q5j`_aj`{J%-wxXEa*w+C zi-#hYrEIW>+Xf&ya1)Tn6GvSIrJ>@2@4wx@Ar~1LdDzcss%(&4Ckpm{*N#8lmk#jA zk;|KqY)keRIWy?SB)-EHyg;;~yZ&p3st8HCHHz&(ED6%!a~&^>EFY6Qd$ z=f^X8{#b8~AvsDmOHHHus@5>urA}Rz&SiIYkDf$~>Cf@2q4=O#gq{AL8%AE2#*t3% zZBR>ex_>5S<4jTGRjSruT05+GO3@Meo%L0bNx<9l&HrD@vm@|wwigxQ_WVD<@hB-w zZ9vDP&m*d>>(EM!5OjxDT_my&?sdWG2_hPKB(v^(##ezt-^9N)lffG z#5C-TX{~yq-rvMmzm0QpdRh%UA;bb88lI2L=VX?amuC?)J$x7zfJcOUT_{JnLC71V zKOu1FM#Pf|&3WI?+-G7#FRd&rylRgD90cY2JapVI0Z+2{<`(-=RKS~&5H!ECUcoSA z`<|6W0}u5z9udk2yb$11Bm!qX3fTxC__Yq09+QCE@rP(n`KoGl38Oo*8t_FRSt3sr&lmNgN0K_ zZs}g&xyi1SpinANCg2l#F`mOtWr2h7iBLK@AT5=S0ap=mB-J6@>?|6R4ZY99*Z3S6g3q zqH&_SYX>dYkyu&kDE-T?J}?t(d}-*3xOy*^)v!DfoRJ`Dc89Qoi}5DRv^#qz8ky}& zCaJHzC*k`CYkOD_Y_3{b?F#~f4jMPu@)>DfBO+HV!juTEm`Pt=EdcY0vyhzmE$1pq zsAW=dUpzZ2=1Yc22FS$i>NxT`NxR>7e?Eog0SvTkcgf9xNB4ja><9m=>5(uq=w<05 z`QLK4+Ea zXzXvjp31|fJg}v}`>nQEZq6WhCU_t+^8HD}^Ri=GMXDMqqgjv+-zUAEo|!)AyYY9q zhNAZDuROkA6WYYaOGGqyyxcj@`{MF}+an{lmEfrRY-3}MX_}Shf7!>scg#+g>;##r z(>}5K-Vd&yj@4JS@cqHEkN zHEMb3-s$jO6)E&%s0aJ(MvRVw!9w5y(nieZ@!_~(qhCv7H-9k!SgeiAj0AnJXa(M> zM5lT;5+S>W2f1%fzCe(G5oZd>Pu&G|??gPpTL<^FQ3I*A#*pBe9TTj$gm1t6!tvV? zFVSw*^%s07!QsOaCm5>yjsclS3>v-UiU;x-O*dA^S91DH%q9R_xOD^p>+cS%;Q4>&NGL_CP~@gDMx);~uU)(r1BJ z^+7~{n}P##2Ksaf%q>qron0DMEKuuvG+$YXFzbFZWmNe*t#l@VJ_NFl(_c0?V#gAK z^UB+)<=7IoX|MsrMpx(F#5$HW-F5yp?1)TX9dkp0tpnptV_q=456=WUMyP#!tfXle z**Iyz+ez~DdTmHYYjHDW|LPNRw% z1ut68=xPV?voRD2IJ2OTfVHF+mx8a7tanB(7ueV~lsz*g;Dx3=m^u6;PBn$=w(226 zXbG$i?@1anrRgyf%9@_gNh^ShLBb<+`qHZ3dDe+&y1}SPwsx};WvWai2J0WQvYVS~ zk6ZRm$d_*R-1PXT(E-Q64p;?ZK|+h2($j`K8Ifowkf>B6d+12`t5pM9+M#BWIkdD3 z4Xt?el-jd_57tqjpFas8|A@@+3?Blp@5Jl%W(t|NQ(Kagq4zZd;pVooLq7L)ft$XS z=}NZNfAedGZM@BcBGd4Acb9B&KK!?Wy@e{EI$OVBLC5I#j~P>H}PaXW=r9rB@(x=)D+O)ndNoo2M_{n6ad2z7@%{ z^2N0>E7Q-*Q7M=^bpG1%TfSa4sN28C3c7vznN>+lKs=GbH|zFQFzw7{CctwsU$@n+ zJhKBUUp_MZixGt>(pU>s?nUM~f}KApm~qT_R{>m6vN1X@wthP`^ey6@NW3@|n{>fA zT`RfOP_?0Rb;`Bi7?ilPnI6=>zT3GPJ;h)QjelV$Czz5m{{Ec_*~I%`A%%)ev!s-8 z>C)q?Q(y>!`O>UkWt=UJ)l5LEjYJGFd}uw=(jCi@kE?F}z_Kd{Q2rbuAeLbIpYQ;M zHQAU+s(<=J!8BaBKQaTBckm?sxA*UW&$$#PNc6!UGLSF&@qRoDNMm$yWmxHaUD5oq znSbx3Oxy4F@Hv5weG34+aWGAN@6_PN$mF}5Qm88s-fd}a!xaf~#r3Z*E46h_BV`C+ z*`8PQ99{c4BD7&0=4}tP?0UOK>EM%W!4&`-y7{jKi=6WkS=Zob!hHUjDGXC#pEk^p z-7NLbXldC+P^afFP`;89BQKnkE3~wj7JU;`72cCg0@m^f*N{8rLq9P{v+uxS6U4~z ztKYeidjy0PX1Qubw91$#!inMNEaig3I@c<@n@rfG_?H-9fOzPdJ(w>p5DN4_LaUUp zkMuAoy@}x^NsynN3%Uxl0KD%r4NsGaKcSDw zV)-I{dR5u^MU&02!&qhE(RU9YCxmw+rXh z%g~#N;3l#;N5D2ua|sagEV!cZ*hh6Sp2ECO2^V>$C+NA7A!Z?P`|(X5_qvUN9QUH2 zeq;sBPbD8RQ4j>nEU1M~ifQk0k3fM)hl~{xbRBQrB4sxL#+|i??A=>t7-)!U=<@mG z}~AL*le5aAz5dmO?Z@&I-8)S+{*vLhBF4Bxh!v{W(0~*+bbiHz;9OBvxiOmF@BGmN zk>c!26J`a+YXPzYo_)~E!YUu+E)cz>^CHTDoc)j~1C zvr{|+qoDP-+3M&!Q@hi16n%_by~o+8jv^V*$eemF*x}t)E;u(4ma<XH1%WAf`9y(nNAa+FCEr2SYgHwT3P;w#D$r#3@1drtkYyvMFe;BaPz?A zb=D826XN9>?TgEa9!f3Cn^@2wyze50|4Xk8c16Q7ElhWzi7pEHy9PLnTaQRW=;XL| zPiqznIkh!{TU7YhG~U$XvXMMm){XveDD~w42ZGTiH^_r^=q ztpFIhm`-8`fi<`wfb!%~mkjIlgwrk&nRs{su?g9sy|Iw^PvuVIUR;qod3A^VfCy zfCs1vFBL$S-_K()(<#RD z&eYy{X>1PjmYbgHlk}fcQD;RFM^)c*Du)*-t-B}X*Svz?0#ux4+OB40nES) zEoTH~3NP*xVIgZmfFK*p0Q}Mj#QBTRSzP!|PU}#ieX$wmg3{yel+URxqVYpU1X002 zs_}zmXClxHwm@W1ygcj0hBPSS7C8~X@eGfM*ajk?6{6OoqfJ+PFuAzjH?#r}^bch> zelQX?<;&~AIQb4#Zeim?pNaMJM>-as9p$yCHMB>x+88_2g5h-li#m}pS$1yKK4)+h zDpe43d~kwaO-Iv^pe9ZH`GFbX&il+?#LAB@=}PDUZ~}(;USgf{Nn$A@XD-FYiI5(v zES&-XOzO~9XMfUO2Z*7ZMh7V?U_oB|+2@8Cv=xwfsWiesJw$wHju9lcF9@Bpac)zt zSbcb2hIcxZ>sZM?+D1{$hA{{wUOL-cP0%YZgGB^vvVM?I6xW_{A@kE0w)>Xoi#>F= z!=iggCraxJ$_0~j{WJTTQ@VrRSur5GXrJAQ>Fw{kzRjO9UNZyB7zWr=xO`yhE0!S+ z4e-I+R*P?%dKjlem7^ckZ3(2t8HXC!Uw9uU~;Dq@(G zRmX8|?pARsjfV_VD74W!q45Or{@_Y~W96u`8eXq74tx`!F{m zRQRy%@tY|0^R2)hD>D#A8ur8V@rU-}faHH0?+hMz;wRqZ4`z6Tc?CD6eE##56%RJe zAx_sD-k?&vI)!U|pC=f7h@ua?&u~*b82xX2k@L2kF=rA|r_KRH4wTmM6lkATkJqy} zwaOg={wL@oe-dx{Z%1%YUOX^5+8*VzM`xmaQM|%zWpT!VP@A+&8!teU&H;+|u*<3j z0JvL>cavKOzTRDcbchh?oHe)98-bD22KqTF%A0NxX8b3V6dWQ`e~SvNozb1y#yUFA z8$PSD-m+1o!bmloA0<_@P_(=VN;PgZ>~}cqvv!Ictx@g^5p&%-Y~X&qcZieh;y;`< z*Z)d8NjD#VX~1cehsSC@>C`qsrk!)T(x`Z|5oB}nx!Lh%PJ(H_JyucwWc7esG=ZWK zqTUgGR%j(uM8grD}$?3tA&Wf`e%zCwT+{5BS$S+y=$2S(%)S zjLd5HDN~xzMpXb(vjX+=MnbogrsilebxfPg;}W$jt)ef6nR&UD3Sk?tsh%pj?|^qF zDfl+lC$h1@q!lYyAODMud)sR0NUU$?G5^#0g_xH#i}?G_*b*Hi9+$6dsyVQVDwx;RWttneo zofU~M^A>0*m96t|BibSRB71Zhq>IYzq-0c7Ph=wm@LXb4ac_ZV(1fwC_JUpCV_Qm5 zaG4${%aN~oaP~W%7u$W%xUZT&0n z_^IjPHqkW^qn*|ma{PPwE{}wC6~T5Ok50#)=N)+p!MRO?vmdmI`j=e|lfO+>8{dS_ z#(+iRqC@2`zGMYPW4S{s>1D%5tFJ8o{^wLM#Ijyq zp+LgFLP?QZ``u+4rQCbVS@39$#*D<)O!8Hr*qK4ZO5r`UFg=;61B}N#pHs{rlx7~9 z_&E^ubJ|s=023f7CsHsJIi@8gC3vq7kU;|@B`yYX@c+edgYXxT3STgh`XK?_x7e%v zrq4IrklVxrQXLRU(A}9}U{F;B-@Vuh0>D6|76Og*i#FBfPgCR+ z6=nC?3b-3gs3iP+hyrK(M5x*67yDgmQgF9d*lw#y53RTWNbPII{m`FQ2|xZyW~l+1 zCAycSrE8#5;2dPO&@l#PV{RYxgELq=JiHyEQV2qvH9(ri$-7zw)$()H@bJZ;_H*X= zVQ(GlrLytuhXz?`0eJ(A!Vy-QQ4Eas_wIo4f65aK8G2Heb{SnxgC-xSvQcFR5RcGw$sM0T(8PLI=D^MxV0k?gg_`) z;NU6;;zL3EtCu0}X~?Ve6V%AX=KiaTvtv~!A_(2v1|tlOT4Ns|m0;i6y-;RND+x@R|Jvz5cNc1f zVwYxm34deBiQ0rNb>tQ0s@w)YM@#DKF)+eP)#;ONt7(91STbLiI70f#(*B7CgIJCra_qG34E=7CQjH0um;3Y0K$pXY_r*j>oqn-9Jh$HB zOF)?u-AO8yrAAU*&qM#S`ppr9a2~_)b6f#! z{Er{CBq3pKL$$4{Wbk$Y4{4PwgrU`h?Y_!*^{Q}MXP87jTj_Cj$6qYeI!KWvW0OWa zlB?@OwFCt3mmbFeO65=5tOImE#_qa$=7X@vr6pGiQ-|+=sC@lc#S~X{w!7!N){7c5 zRRWIs`r7^U&9yZm7?)|AX8fP#fTzby6$xNE13L?;1ZK&!&uIvU&vux#yBc|h+MA|K zm_+masu;H70sXzkah(d>B|ng&;z}@R@=@jMAnyN@+rZm`O3)1CcA|J-cV;AYwp;SC z>U5EZ=9$Fr^WD$j2u?z;)YrdVHhp!QQ(M9&V$3AYkVUu zN=ocuj^cGng~L{;(<+E*?p-$b^3sJ_Ab_AbLR+(|X5QQi9Je8X$$x_^J%R1v8N7@P zUE`8O2pWZ4zhDL4aY@9Qcd0J6@1b`+Kzti8MH<481Nz;#qf#{MY4h#6- z|LBm=>cDV;{O*<9rx6Ud(0^4#$hUxBZ_lO2#E>-9w0}7VCR$L>uhJ%rL@{5{Ly;}n)k9*|so7uFMOl@s zV_k)SCjq#Aa7zrZAZ^pSfz$MapVDkRPV=m$8_}2`5!nMK$@Y!ylH8r&{i%&168>G3 zg>VmE?>RLD8O|Xf@z87-X7-g#6xDpOzd&oKxO zPsm#492~1DA#}mXI-?ht&oyO*;P%#QxTit(cLxU@mWP$UEGhDnkGVIpNo9t86L#PtyF zxMg2MbhL{67X@ks!xcV{lCUZ4s%Q|0_`c4yg7Fq$wgKwsh21xY^71%E;cZFKqyn=J zude>QK#e|~acTK~eA$mh2`e8#wY|UMYc%4Ts@3|j@(p@DSbw{B;TZCQ zivnELSz30eN|ov^Nw&^6P3cM2^qsXAy_mbF-vF0@T8FDsF8*>|__y{^Js8eoIy=)X4SG>UZPpe#<^jDu-}8}Ba*I+#2NY*fwZ zOvL+)bcAY};PN+hJxEad6hCBhmQ$Y>e{IYJgRW9v!?dAT=jJ{|)y`k8h~$}>Sli#O z$KiB(VEWDPrab!_NKh;)RQaPdz9xx($%F!Q2BybhkBT>sO>Y?p^tILTeR8>B*KqDC zk%bSrRB?;_6BEw_)7n<|2{5mN*Z|~0rJal^{lmkpGc)&mkDBj_zB@*OfIM%$l|X{j zK{}a+%%vN8f9QeEcpPA3h`2}q^wN;ROa|_~3k+c$m^qgF!!8ats=YbMJiQb2Od!KS z@}6MF;$_(Wu&BPXos{M}C|xEe>rUFpSBbz!j3ahB-p3EW)5xCk2!E11dckmLli@w` zh*Fqd{98U;cNh88lnAXhMVhqI_$hjZtF*cf&E3;`nC-R?O zWsL46*mABoDND0=Z^Cu2qH8=mh7RqPrI*rCj;H1NXWJw;j;l51+NjW?^0Ce?mh$fMwNh) zW?6YVg2CPV_{CnJ`5W8b^NYj9p(*UQ61r!x8Ks-lH8b1p4;5}u08|wUZxBBhw@Xzh z1KEN$sdwkxsrL^b-}KimYyEt+qhH$w^TBi(f8j?EyL9Y>uj5|~9Ok*|eR~h52D>i( zuHnwW_XG0i=~%JS(x~p_)vJM)JaBYcCo+3U>)}|KezATS4(}MUP^Cnx9>15yI^#U% zFjg2Y1>#KJHD2oik;mNWW{x(&@M>bcL>szF&}z>|Ui?0oND%;oCmNhK6c5_Ulb`O0 zv=0)%kSvHfsE4!ki_7e1X`vy}#+D`=)&YLVvl87aR{#nHt@QbeRcWc`{4;*n05)he zuZ0K`oVnV7^F1qy6f_z7 zfq94=XXh>YHPtITFl$K}4j(t6O_m)X#6=S10Hy#pd*EYctR}x#z5Fn)1{V^jkNeb) z!#~n$>?qG@5sVT2=1c&A3t*E6$ezX>%dFXQhJn=+%#&GGE z6Mb&49+5{R_AF?e4zKU(`xbML*dYl}2uKOmrsBYc**ZN(!sw8kBL1{qy!k|A5W?-# z;JY8eJ#rrj4}#qQ2+4rtsyzoe#WuVYFg%gSdJLERirY`EJV0)OqHx53zn{Q}L5`7J zi4Gt8z=iVx4wxFD7%466w|RQqE3KpSyAa(Rn|^=P0RQg8r}4xvsRybB{PSEy97~nf z7_K|kzjX0xH{>7$24oZ9xZClB;4{z>etUy#Ou_X~px%S4LDaNl5)KdJtkGq-o2v98 zgX0m1Z&R@>O|cBWW((XF*%{b?dyXKjwsOgUGdOyIS9lKc>1(y(S8i0u@(RU-2bzhR}>t73CwI zX*k3RWF)w*l_AGvmu(lT!=q!zWk%p}0EGywgS4O93z$*+fPjE=*#z0B^KQc>i1(-P zks`(r_6aQ8Zu$YD?sRnLca0Q(X%7g@Gv81O0~Z#_H8W>f`*`D ze45Kb#ZCeHFdM?l2C%;)G0%`q>JEow9H?DahB;@q^U{rK*4;AsDF2qeu(O&(yCQrt zHpixYsoD`Lq1hj#Up%LDDAK6JUj42Cgvk&`0fPk4;)8Fs)1uLsH(Gx2ok`Mbz&GUU zN@4nU&jNihQ|Pa`GcA%L3g{(Rx_+b|AjEp;iDe>XL>rUYz|`W-EeA+i#k?o+#L{gq z!3lkJwm1GB-n*{9P^JKaQE-xdWgnd}_NpKiZjDD>*;@igVQZSI_T>QeSmw89Fd~O? z59C@Jky40JWB$K^_w*ngxn4^Ju3D!PPI8>Uv*e@Vf9XGF)o|9wz+pHJ3xaA;A^>6? zHl~~aZYIxA{&)>-Ds5Sen_E&zsn2r_;6mFAU^u*jn32p=9nix_9vFkM$m}8y`|RB4 zZRO?tl2s$KjU(*{k*Ly{rT&s`BeSX3+GfXL1irc&b6bP9HswJfNZ!vnDMP4slLDy* z|6d{ssw-3;B5OW@Z@1k{io`0vD`F&#+pi|Ik57F`f8;y+_g5Lq7^NVv8$~s%0Of?} z=Z|1MBM&g>0gIXaUKi1n9^f}Lf+rcVNFm>5fFeLvaT+p)VAHL7hm1&&NhXM-ZY6Xh z^$`3Q9WufJxIt@>U>|uw-ndQgZRzO;!9`D*_0&(n&`vLh${4)#EnY7;J>G2F+pJK! zY#6ZkAuzqXd#rS+AhxR&cki^}>{RLGB=eVI1EcTtjR2Cc8!(H{8JKyLaf}Vu2J#0Z zLM8@srOUx(XH>%>9BMf3q8UuE4}w|MSOUcjNp`tHpxu!e`2AN^VGzj^coi#!r zvh$8;2-^j@8W58wDzFJJtXve$5cbW-k00FT9>M*Hx?Jo@sr*l-bOCvsjGHFt!LQ;b z$N4^6J)TN#%?Bchu-2c9CQ5^VLywe3Ad3o*qZl47?Fo8F_{Z~W6%9Sc1i1t}ePMAS zbsNs}Wk%gDZtMrP!-GkI#ovT{90PtaaUbp|^Kadxd_J=@kb}v>d=LFGzpNq)x*AcO zQw!QBcgd#fv`;m(40k{D3goSPKDqdml)Rj#r~LDa&EzQ0sB=SHx<`$b$2Z@;UC%qA z|Ea&&I`$?a3%!2-J$4~yXffth z&1ZEzy4B}xcXzJ&xuo2O@r}>jj@7?iTYZb$Lz)Z|U1YW#NAmjA3DN3LhKrGZWgxf- zBdh%0dTj$xK6?FqX+HDy#qoy1G)Y-OcY=(DqKf>%Go-yk>?D3)(Hqg`VE}Z$4o3 zc>~AKl^>UbFW5A#{BNjHb0QngsWuIdKbP-GMJqRk4Okco#?5a% z!JAFYnd)8WNs-G{0bg>zMfvR$Z((+(B(K}gVb*OMGO&76Z58KF^u10Sak7J%gZtMJGq9 zah1{F^QTeox`Ax9qEZ`r-gh=={Tt4fZr_5lWji)oEgxZc zxrdQ4*0Kf{?Ux$`xe#yzqCvwM1^qR|Z3z4q9eo=~b37Pd#E~5y{(X5mzgBT-H6r%Q zJfk_TkG(+O+Rz3S*KnqCPw2sIs=>Lvm9a&8!eZ~(koF zg^UQD)aBS_Sl9V39DZ--fF-In>JQHy>=j5Gw_JAmfH1w^|$@svI zZF?ngE?hU#IS!f09xtBTdv>Q=Pc9O0WJD=f=ZyXt*Uzaoby}>V^MNcP1eLW7+chLy zvILyAV&>NQJt=%;Fdx|gWLT1PPSgL@`+Y0J1qSa|Rp)X|XQ#hcx5QF05S+8h$y|2% zTnw8|TucUv%Y^$bELIa_UKaJj0V{pp^{H7{H-Rd)8~3LUJ$QCw`7;+ok>6m zO(enA>rW#op>;KiGFduLf+C^!1JD+bEWc>Ou`7McMOXG_QC_g!o;$FAQ`Nd6qz4No zW@VPNO`!w)V1;+hdsfW!sicF8o(lqa34Nl%jB};ICi}IAXL9+#qCyZrPW_8y{BGLu zNpkadLV^ZX_kVoYZ86@JR5*^D8U^^b$VCqA9E>WLY#+3bEW#1KYZ zhgq0&jh+sQw=UgJ>@QPNrxOgMc6(lVrNJ1N)`};0Mz?F4a!4PYaxm<02 zU*IBfg(Io?qy)U>tdJ(wE)%Lvz=oE>b05K4(Dr@VrIrqDjg3L{*W0_PWO#mCiaV zuRFVM=<(%ZWaqK89_+pptpu>(n^v3b;T7z!cRPr?TWG~WEP+6thqhl{tn;&<|6~Aw zCPam>Br)8zC*}%d^0mRBkW+w%`GpY{eQxq705k zW3E{j)_8gAkwT-MWS%lR(g)wK+zpY(1%!w&0Vo3Qb?+2pz^~nLPYNe2yj}o;iZB0KQDHsaY$~(%fCP zi5}3VxYWpdR-tU>MgNU|JbRi+6ZC`R4g$ni8kqKyo|<4D*YU4CT|hC1$}>EIH0JEt zCag=>%62s_p&9yYq@2wvQA-7tF!K0-=fGALo_8>^6a%-AXMXd~vA6f*o{p^_NL>cm zPSqwI)-H-XBZ~X4?G}*A93WEmc8DbZ{*dajYYp(RKIS#F1N|z3tveh$ijzxFlAn^1 zAOum=fS8Vq6BqBtW`P!)O2KeqdiqCuZGT`1LHHaXb|tbN+W!V{&LeJaSjiY6Cs#@7 zA4LZS1Uu1DUe})!AzlDD{$6n9TqN{%nw23J4)O)96awgZ4Kv5_FGkl8!5)O6=(1z@ zjjqD?9qfM)&?JOjh|1{GCC5hJH&;xYJTRX8{i>f6?UT`SMmmkNTrmHNB}@ zIJ?usuU^4@Io3zq9ck>zzI`@cF`?0);VI0tVl5R!KH&&|k`uK5jxgYNfa&ILMP*$C zA`%e#y2+l2ab8{x`=rED|QUz@H`Y2M-O=m$E zw^mN5$Kry1=`q3g? zdkjfX*(3jDSv*zE_ZOxqzWGlEw5|eg49%|6)AHKu8(RP;vIiLBE-+1y5IooX<^lJR zn?cQm1`X88Ej#v&#TxThiMn~9|YfPzn5gjFf5Cp$)!#7mx90&s4C%lfaKb_ei(BAsqn#8rblzxtPpgg+*Lt3}N8TNs+>YuqiY% zzTgTv&VZU^b3s~`BrelFDi~NH4V2!Mz1;@0*H5L*6m+4LQw19H&=K#0A=^ zJLN2JxUZ#E;w!P19@cjFcijN90@CUJ0Mw^2p#tW7I4ebF`2J z(a#St%5%4iyzr5=8_@%M*S<%Dd=NjDq{;p-_6yN6bah=#7P6Cyx7zl~x@2k=$G}FK z;9`{38s#UR`5n zPikI%6X6q2Re_M>)6Fb`d9AkH-Oqn>Bu*55?Rh_-=^*gh?U%EXv@g?#|A_u>Ut#s4 zkv=qo&7eTrYasjK1Dfd82LVh9&b&8bQ=vZ#`AZi2E=0+740&txkagm$x%oV=aWa=J z6ncv@q0B7cT*3M+CipXFcZ7~DzE`KFbANHi&=*6_kp=ZC7IH;WJrWr&_N2Jvg7SA< zu~zBR2{|!i=l}F{Jnu_j){EB|d;Vi%&r%_JflEU{kMMZa|4LTwyAHD04NJW#!K62T z{M3w$9X>A<4BWb_!`4~guEQOh8){B0qcDX^iRKszG{g*7Q2Ago!dSn1RjEJuM#W0B zJ+G1ZfUX<81tC}DloTqjhRO6># zb44Ak>`^T$7Nk?xt0%Bpe_!=J@i#ezY-o-m!z#t$6<3@;uENyYZYC#Ryp7?HDVf-y z)de~^>0J-$;EB}^b5%dObAEn)6{QjU`R}pcRGN5dQc~1TONe3KouQFskWx~PZhlSe z_5F3lIbBmvFd=Eo0-BIbyoSnFkC)yb=y|ck#Z*ro9RxwEXj$o2L?_fA8rMG#{ZiL9 zAGNmkg;OC~G^V4&AUnE*Ng*`iP7sqqNu#MQ30x9!F$#;`-J~4g4|&5y*4f~tqSSpA zjfB#DgQ@icx=f#Zqu08$9j=RgNzO8TJJXci^}TU)*Yqu1W*&^FXaa(lqP!%&}41O|sJf(Q5Si^G(#u+uB?)KrNOcMSHha9NAs zjPHG;M=+LdLl(E+QKCge)HT#H{ZZNrf@letF22DV#bC7$@Ok4bgpk-qb~i83HkH8f zUE6D!)|i4Mv~4)_3;y*ysa`*m{{6dqbuDf`5TEP0>$@wPNVIp)y$szZnqyPT$-^eY#<7lz#ADasesNN&r$)VT#0V@$hr>6m64z_w=f zI)2>k{KNx~A;a3f&DY%vrNgj%z#I3C7eyiyWKR0NgG^VPl1@0n+>a63L$5cB!=7zf zF5kCw(nn<>j+xJF%OaWz!8M-ae6{UsF1}}#e$JSE9%++C@YVVI zYF95Ta(x^j2Gf3`(uEp(2;F@Z;P6{N-YQA6P*RXOjdAyP<;8-s)o!xM_W`~0A2c!Z zuDVHz)~|7En=PD>?bdBly{ao7!_-tvcqA2U$lc3T@%Ng8xaU(uLJc^g#K$#we-~D+ z^Owav=j=Shxw*GhAaQIW%6)P2y=0-LwdXVDI!KYH*+d2Ay!U@gceX18m|EXTtc$%+ z^1HpQ$gQegzX*6u#D_~mH8{?a;N828ckB-@r&TgSh9_!+Welxt)Mm~%u;xCv^wjUS zS&K~LV>v=x$W)#1Y!jJE=bna1iGqqsBX#=s;+j0`qYDuR`GZh#0>cNW2wMU9Rn1b;LVes8_C3Ta=NOB5G3aM-X`{#! z!(__};EzJ5DBrx~ryX_;UsFXfx>=I}6zdE%4RMmuOqa8a{)p)#+CYoPm0&NVprDuq zC}0ux{=3k}1B`$N>=1o3bZ3Vbk)8WutB~C|Yz3{ej+|oy(M&mLY(*pBOirNN-=aEKD}@q44rNxcAgX&Q9n~!E7sr2aZ%GYRky7yu$?w zIty!WZ_|2+KV&sv&AD`8>HJ8N>-*aMA4Y!0MYql^Wh+=Ij}#hpLQeAoNCbcdvqBGZ zTOY1Rk)w&G$gS$>GyfU!T9|X+ct2DW@uaH;g7YpUDe6`Hy7lX1esc^q3!91wEZlSV zz%6%?&bji-Dc7FS3H1{1sIrxlKWP6Jq=%&Y?oJy!74>t5#w6;aY}&y zO%FL{nChy`eH?`=0GNX%a3Gll?+9sPU>~}W9zZ^jZ}juHKqKFXFPr5DCsQ29+>Y?r zC$~T=Mn0_fW1>*$L{)h$hloO2!MuCBUuNU7^>;7HwvgiVXIoNLfxT-|$icfmzxb2G zl(22;} z;*kVA#6as@mCyYY?DM@Ibvrb^;&ey@E7CPSXikBo3?zi6ps)clBfz@xG4RJ2(mDq3 z<9Bcw{)N}P?^pzf0uc!6Bu@O#tVDiZnQdoH#f4WQmlM;2PQ!DOP<^=Q{8;a0HQPWt z6ux*!4(8v{hd0#aMam}$kss&RDwImm!b0>7p&8$PVn~|$3FEw9IQGdw#N+&Djoe|~ zDs-Y@ao4CL2`tYT{Wj~ena>k89$%oA0l<|weVm}l?8}iEY*Af5_4w8VRUJNqmJg@x zy|NWHjm3SwY$A$b&ev`ba39fVnl@#_rAPqYd|%90c}}IA3C5)Et<#v(6vk^JiSmM9 zr+f;TsR>GfWXjd*e0tJ(A_}h>`RLC^ZE<&C{ubRE6!?3paxlBo6}mTop#~=z3@L7C zSQ_&ucpKj?W+IXO=&_Plh-0X~UyU0m15mD@0Ox_zrB~&B2mA#{wAhdzVmd&N@@I)Z zU5j#HBurF6AyVPi#*4pt;b9c{?eEb@UN4_+xLERlTiu88`CQn}3%b9iGe6+f^>N4+ z1j0d~K#>`oS&c$o%5FmuF&SE$ajj8%N?* zSx%OD$sgbZK#Uc;-sixwm@ZGK7vKJ~(|04V$TV@o>TKB}a(i6NgGx9DbIS@|@OS5C z>63G!N734{6)~pYZOR|_uxNKV4c#6o`IB2L6LCICDmZQWK?Q3cx(v!g990z*F?aMK z%Cg(qj3vfGm@-2(mx$mP@&-!hs+1hdOP&(w#ovm{ZDz#&KE^O#YF-1Y`P;;MIRkt` zBrTt3rhtZ#FHy=GH?4@X7)<6m_QOkUd(~}=ZQA+S*~G!%=o<7WLH&m16)SgxtV$_# zlA;;!jll(+0BO`U6b*+g^d(xBwfyV9r(FRWw5-1!9jXn<(A~Gb}PJw$gSL|c& z-6d*d2tw!?WcY*)?_TvJAYNC)9L7ICo;kDrr+L8-*Aa)!^8t6&OC;(2e|d6<7XS+} zrcWIYSNubi0Lt2W#pC;Iq~_=C2h_!oXi@l0`J{XOU1E%4R=jXDc_h3vl?o!kF*6`3)NLgEU z&8-!etcFl_hrX44T|Oa)syZ)r1ySk-vs4ul`0it*Np%yMxLDDoo+YaRQmceGFNufZqE6rWA>}H<>5ZTO$s|DVY9t68@a! zYmd>nte;sFO=_3A4L3F}?Q8`}Om)u6)wt2Wx372r3PFe=UnEXF#Ldjigy)3d$5WHS zx^4dNM;#5f9Z&W~z}u&=6cX0V@H)DAc!i{iZ}G-OT324I7-C|iX#wDVsa?| zr%uT9Z8q8_^^$D5`ouoL2WU_{GG@h+04jYT@pk1CwxU(t#7e`4A)3`7ckokU_aK#> zY!Czd2H6^SQMI)5O**2NOt!UFdTqK^)0EL(ZyILg%c~Pw{0(WlF{h= zciUxIHU;7%qG?0N#`B0jZ3gV&GfzzZjH^SD6SgEqbG)$o40z2lT&({jXDM$G@(%zX zfVGUNoFJ!h~HVJ%3OKz+^y+p!SjV1RAI6mx$F=0+L4e6sbI=pU6)2o&>xrZw~{ zVj~K~r}6~k_u{eNS?Q(c4Is^H1(;;(>8;XrP^mE@_VVhDaw%F1uVag;pyR(!DG0P&-L3Q*y)) z;j2-)x&;m03n3vlD#JHCvsD==_hYzXf!l#Y1S?s|Rro&Fv3>vL8vF|9l3$d0R;#0+ zpusC6b2iEh*4K;OyVRNR-bI*+l8hgZ5Qioc2>gQ?j3=+J$s*G8}eW`tsZ+2^kX!pJuXMPcR} zt6C7tX~FyU<4`iLFKzUgsKx!Alf}*4W2{X|Q^rO!6KrJ91G;o0;Ri#*;m6jlyE0!-gOw;zE zKgfO^zv8iShD<{w2}UKg3A8?P{063D@>gq><=~!L>HNDqOjr82X#kd*>6xdeOHa#1 zER57$Ly!aeT=?)og-WEj1e=yKrqH}<&F$%GBh2^UrkZ=Iw`99#X-Hd zUjYLow4);`zkfM{Y3i8Yn{9NRIXy>LH+@I?!|-J;c#r7hX(3*!-hQeAnf7e9uIvGx^KkVgWAAMBBmss0&#mbkK9NuQmR^=x$ zMM2YfOzB`xx#~@mZujnj=~Px4&o18K53sX4zv8boj$CBq<1FQ(<*y=oPCs+-WvKWr z{}E9%)2#Do@kHzbp;~OBlI3roSeH+_oS_i~|7fQzb@)U)VI=86vfmh@E^Ycu{}Eox zFV!tqdLTdGY4s@+Hvmp86epD6UPsKF2>L?^dRzu<~5SKUdfo%h|=h8;w z@w?OHG#XM|9o33sVBBr(Y*9|d;uREZhT|KL3L_^i9jp1cEs)wveEHTn`TFUl2PVfv zP+G0xx2ln^S1N0?*(K1s0I84W8{=hX>gO7u6#}t=VU<4#K2e1AIelae*?C4wtU}rM z*F*9RDk6}f8v&IO+zBCBC?Ih@#LqJ40gI0QV5(>w7o4s8{b}qU0W2M#d_O+#6Jc2V zyva6Q7lWDeMVU0)7sFG)n-m7su7Mdhr4`h1yVEbhd5~sN5nPvxW`^drreZIrr~FZ1 z$uO_bhuPJ&PWTOV-v-_X{`2vXEttVoT_qMRX%l&r4&2$-@s7x}%vD(Uo2C4r?gxX$ zR*Qxj_ilU+}O_i+7l{w z$UQ`#M1Nl$dLn%Ge8+1Vl6fOphcWYTg`(@44^_j4I%U0anfHQrW=@cPyI*on7@TR4nLWgdxucayX!Kj4_cl9c2(VOo^qM|iL z#4DTfaoEQIF;Kkkp0G|l)Vz>(M{VMQY10mFMJ| zu^baox+Z(^;{g_!2I{%I>kZ1>Ro;7Bi_s>yq-81QO{Px}U-C3k%5h!}=%2lWiaqF3 zG7jd_VI$u48hrWIxJU(~E*Eb#*FPh94v;kXd11{`nz?R2xP#cUmqJLPPkJ&!l9TAh zYrbS2;@?%6p=5YNe!WMRWou+-r^=?af;Z=GK=T=8q@Ti#0=YkUfo^Hp!hb*aFY>0d z-ew-3%N5) zA2#SRq~uJK!;{C?efki#Ps?p)z<6kMD^v@yLqZ$mIm6(SVd2L;p@oMM(77bnh;(^X z(zCXpxTpFtNzccpMWWh4ZZULXeZ>ljU&SLC!j?j;7$7udR{YWRIv6|Yckjs>dEA68 z8AStoTvvAE>bzmP5-6q2`5^8A?ta&RiN`quD_d`C59V(BY@Y8LtG@FZ_k=`IYh8OS z;E<=2ACu2;3?$4~|04D`2e{AFwVXuRV)FpcgD#aL_45kRL$*)$e(cDc6Tc%v^W(7F z;de_YkD0gg6%~m+k}})#07siV7jrjgT0Z1AFBZO6pWvR;DMj&R3O(;8!q=;9R55N; zhX1_V77hd0BVaMa^|LRl_ko9aVB_ZQ9~?Z2Xr^MPwW}PhfrD#ODV^%Tf98d(GH&rc z@5`B70TW=Xu=T-H*Z`epWzNDUZDplY%BC97{I@o4hI!{}FN#@!49vQikbppcvX{96 zpnm_?8O=X2O#NvIT*NWbQujoSdG=%8pdfO-s;^F@&DltEA~W5aAysDJ)^Xmb<_F zNTBl3cVuA&_!sEvbVEj30)}>eM7=T9@EiTSMJi;G5(0J7dCs9kOMg+fIOEqE%asn9 zF$uBFJ#v--U^~thz=#me`^D-OW9zkg1$Q2Q9$&5fLU?akgH6i`l`o(1VLfLUv>dbU z<6B?ics{ta@B_ZyWY0W9y%8dN4<~j5sMsp4z z0Q!2U3FJ%eMuY~)hTgKa$L(fANy!ejq5A3iNvDm{n*S0WdKBmhXF&NYiu8~nLP5PU zTT%dI+wQ`xh9Km{gR^kEuFtIx;$cDbIf!APgSm1r9i5Q+cY*cq$kP)fToNK<>rW=? z`#0Q^-oO9Z!BMmzake9Iw8--m@1LxvM|4LC3(8SB?dy)Lg&d}o=n2y%HqJRhi6X#2 zT3d9n{!WNx+N;-R3a0^DSDKKR<>aZ|>?KkcAU`-5OfdVkE?iigkPMXni32!FjCS6k zuQK3Xs zzNQv^BZSTUG*}5yG+@+OY4HF}h1jj-UGM0m8hz4Kj@Yfrby$zSS2092_`3NQ^k8*UFwp4=93P zX*mI(lO3&>G24()#Towu8sXu^0~r6T?j^nGc5)8*l6JjQzMLsn$kpkZ!9)nZArAt1 z0=?q>JYafFEE#_;VN+&w}Y+NvSINP z-~~1wC}?P)c3;alasMLL5qCky(2Kb(#gXWa;nUk7EaQ14Wo9Y{nr|jbtBtp}5iXYc zbhv^zSeiWeEQ2k&gOq|QztlhLa}gQr{wTO58bOiJ2D+SXqyZ2)C|qv3l$1C~C&@ab ztm43##-7=Ej2H8tV_JkUgN<#6qO9be`$mjqt^KH6U5(0J3xnmnX6|~Q0n@iTNi#~bTbsx!SKxMvO%D2rf%V^)7?bH zh1aie;2tt0Kq@wr3O`(W>n&E`{;2%;hqyzK1P43Z8|q`v*~3OCANH*{qR^|w0!RF` z6*xJQvap9x>KkGM)XR5cSD40~+I^y-#5(Uj2!@uN`hU4%?ijd!fzRT4N8I?oOAZjl zt1>SJlS4j^+j>yw&uBwbYVfObpR2{ROy5tqqmrz;x35bpWT@K_UOrsj_qGgOl$m>4 zHr1RqGO&RM4>u)ORt74mJx^#I?!=0LXZ@!1I-CNOk@Sj029Lgng*RUcYD%IZsze%EW(uP(( z`3e@Y(4!-P@MM4>$ofdm7n9b^I&67!_4KL%3|Fa9y-V#&JAWZEa~Afb>mDcj9$@99^;)O+j}1E@d~FUvd5N6nulfMFW)HGVlByT^_}p)PLTN6ZZ4`8{s^wT&$h^ z1~~&^*JN(#P{BG%Ss(<2NQ!b1$`j_#ahk>PP~3}=5*1I?NGQVPcyH{kmH}XacfK~% zI8v2YQabN+1PM9y4A!gn7elf+>cI&B@HfH~F2G~{6M=q;U*l5=aoP@4!iWud90@G+ z|3hENyTH}b4Qu&*@uPN??Wu#h1cdM4D5{Bu>OH9we-%KMEHIv0Y=JB>!OA(MgJ$b* zpbe}$-{8*l^bs7Dr$J(57 z$&!*er1sJgQgt%NpnMT7r6<~9u-<`HaKJ26wGb-rKk0MYzN(sD%htqLlVA9JTTP57?od|+s%F9WVFEU9B?7q#nNH%D- zWIjF-@085|{Clpyui-eEg(%-xD2%Cnkdg~dv}0f16f4X%sUzW#+&gA_t@-V@1wC;} z*gjaY2bvkMWu#LSO1;N^H#jjVWKXU@clgSI43h%1l)`|cSBu6~MGv^4Xxx)uX%;q= zMBTs=z)(Mlk7SD*l%mUdaQ(*}hp8}5j-f?p#?uvUP-2$@lyP!JEwzAOCxWN=F~c(m z5@w^M_NeP%T&2$3bAxaNhKbWyCcgqYOyomt@9ka}sLO zEX@B5nYR=LfQY=gye057c0H#>oLP+D%72;2J?cdbM9faj1l;~!x zWrm=AMNE|L4Txvv>|_$~f>*P9HR;-O?W@sDxA0`6?#cZRB_#)*^Isb+qJ##T)B%Ll zh~yUH7?0G0vf+3kqJ97XuZ4+$o+@9z%r=+-obmq-=1t(lG-}q2Hs^qeo<+j7HlLn} z{Ij#e%7=%0&KGbV<5fZk%W-!yCJjjk71c+>r&&ayw=#MS_eTInb^2AaqwMBYJ$b>@ zMjkWjx11h?wfJI=I#0V`4yH|9l8d+jvYF}JXRtkokE|_%%)rl>FJ6&(Z;L}q6@+a! zt_I3z#Z^6K7V+va~A?PW{fW`KIp|ac%)lxj-J_5SqvF1h?ikF7D&z^ zJob~Fzpli>xS^b2n|Uv1LaSvJhz+)xEOk56K2UUQTsjdSz;FV>06(LSEtl)kPu%56 z?eM&|q<8ICb^nYcb6sv>x45tNk7A^1{M0=S3Jp1w0T$C&?9GOfx{sq_k=@#94sVs{ z8Hw@FeO`&U+E?1q{`A9|<&sRxaOU2R z3$};wWo3Auz;hZId9|6jonNC}(7!0u_Wt#kR(kL4t`lmEaOhCM$i?!AznSsoo^a+B z&Vhj=qEo%4c*Fo>$r1#++RCS`uIyX4Z$btce+~VZ87hepYQ$ZtdASagbuqWWL>r18 zV^+x4)EXBR)VG7fBwqBu%>j_RLEQ7IhD1NvckVo7DNgLpzd^;2bnBztO+cX`utaNu zNe=x0qn^``Cce^JZ$E%}=PjPWR_w)qp#{Xsij4@0Ev(h=f*6az$=mE*v1&lLK@Jqw z=2II`Kgb-*9#~Ol(;3peW1nI+bU5cmydnyFFej2hYz$?pM7?V`1hqk+g$S&nHZ z#ijunO=&-%L!xDmpNnZnTJUoE_bNZS64u-V#BF-{1uzjTOV*y^rmOH9rRhK6ubU}L zwO~mBiT+&4#GnP}R6WJGk25}^)Vj91ey%)|hI=3W2~P+C$?gGCon;92NCZmy3zx#; z6_dlS|86ED5TSg9!BB`7XsE~%@4Z##5cz*}#{KythD+xV^ChxtAt@5zv_^z3{~#nV zzN-!as4plOAEJ}}^$(AEnf}n~kq$fXuYZ@Gp5`3?wR7;HLT<#uu@DtieH^T?@LMNe zD(ojOH#A+7$S*{qt1)p$Tb|UREmEiy?S$U3&aSd+hdQ8!TbnI0Obdn0CtE?C7LPl@x3y~V#E9Ki!vwG70Nlh$q&O?8DW6} zwx&=Q66%Rl3Sy@~zA&r%+B_?e_X#$S0uj*T9B^I~!~33JUWXTCYal}WXbI9$z1@5S z^|b+a%MREI086kD2%PKfK| z6Yl!-uerilWmfvuX*8|_txANy8qNYIBAi$ttof%!H<=<#kRZtpc8!?SW+NS$|OojfYQj<~^WkqM0yumsp4?GNQ0#U%W@xz3g0(EE?e`xXeNf zDBH`Xj;fJ0w_kOCDj*F8_J5yz<3Qsg3sf9qog|iu)s@Og0{xWtwhyi;=DmbSgi-h0 zapELuxQgJFp-okt{j>EJ<|3>huBxNi3`#+i_ zMtvxeX$>;(efv_=q5RuIDCob;*IeJu+Pn!)5{ybyNeNo93>@lw^y(;>Le~7ID+;80 zkn8wt&Gu%y5^ZW*d$ST{=~(kGjyAG_0b7sQ`0HZNZgy@p@#QlF(mdk~>b{ui=>bC%eDfRhj3xy6@ZLK-U60JUrZRD{-{W z?fzkar!1fspDk`BbL&zxlR(lpUJ^BFJ=NTR6XOc@tJy9x+Tqvoq`XW>z#672L=77* zOxJvNIsC#&?LeB}2<9R4BwnE|cEkd04S!6VO&?+*8Z7{9eQy*CAPgA0hS26s@sI!7 zlV7OCuF$}!YHy6($1eNs*Hh@vEnYF8T&R1Kt0K-$sox6+px%zV~d70YUS6G*6z z^)p|w-6}i}R^lOVag*GcPX>sBu0zXGp%o(F6{uNaqLeGz{xRDP=qaZbkLm&DIId5& zzKPv23avqmFrIzIF7g)3-eC^3Far;yMmr|}0Gz?D@BK|E<&|{hi`9_^9Ud(RdI4^Z zoVm!&=UkR8jRN3JxKUQR^H95*3IEj@d>ib-7s9wHNDB6gU*H;iXha=>n7xpqa!1kEz`k zk6rGfZCg$(8Jgq;KTq2JQO^o&w@DjY{W#KR$q>#bxt#qVMo>RI`Q!QH3dL)i17@3DkrIIur`u@v{_QX`39zz8sr|dWG z2D0`V$h9zD{Uj(@{(w6{qyCz{ym+ck3Ioh>$Up7`b&vtE1=CxU$?0~2FOt;_o;_j& zn}jDBVY(4)QYL4>34BN{kO7xl*oy!lJa>vha07JMi~Kg8U?0wr*MCD;Hc(9=UqTh= z0x#kIJpTodrMLX>52VM0(B7LwkC6f$zx{QfG_cKDUNE}+p=H3jz{06Dc4 z;UQX-xP%*T1h^qSmO+;&Fz`%;;L<+pc&lw*yYzIpR7SAqAUfTy< zWo3Q|M-nJl%+{4r^GG5AX zTt7%s1vT1+>($p$@T7ytdiU7xOD(N^<3$g`Ya0xzFC6yGVnQtZaYeta2Y_NQ zVSc|ami)-q)7bN*FG%9K)9~yfxFWuO904@#(GuSW7Hx3!ya8|?|1%HPs9+p5;r2ZR z-BJD&b0)IZw2>Hu_QHj^40vSdrO0UtY{{ptp0WW#bn8nBpno8K^sc2K#>$bp;X-&Hox28Tbq~-gCuneR&7t8ld-psYB)*yFuE( z8XwU$yOF87eso@uNd*TXs6v8gLD|=x(3jq{=>{5@#6$%FacZO3F!d*b*O~Du1{z)Y zL&0>|+yru7X$6KDp6ijg?LC)wz*m-T^9IK8(y?^bOU;bwGQpPt5e8tjZu&#_7ALO@tz+$?~0D+)U=MjTt}1kX??(U2yQCg02Ae^cq4N+oAFi=t@8`mhmrS z+>_7_rBI8Nw+c-RzaZtmnNbBhC;a*F>2Em_D#XK?Mn06NTOSqfx+pQbz*o2WmnvKq z@DmK$><{_>w3t`YRM{LMjm`yr2=MVWf>*p1&d>lND6t*HfD%5&J`ZvdO8H_?FF)~XcYjrtEA%Jh#GSS6}##EM29U^LZz#wRx1h0vn`Y#{;8{yIi(L++7qa(w!1 z1V8DM_OFBq^vsBDdiCy*{f0tSS0KocIOUyjs>;O~5O$Q)xaU<0Ag(B(OFdKkQ~pjiVvWU`_DCe1(jkh_agC->8p zEOv-W&udX)he0X;05T|2w=A%P>W?3$OOMsFvD^fhvXq z_n;_GOM3eN&GOg18(+=&G}v@*#XW)A;6R3Z*59k1h5UvmA#;p@^0F!z04C5R9Jk$Q zK)Sd$$+5MWi88Jy2>Qx0X zL(`5E=K4FccTu-LMOSv?@)* zhyuLj)-eEnNEJ-sx`O4QqwGSrc2Ix%>$e|^-@xB?+{XNN^2cE+>F54?lyo%_+ltZe zngXG@2BzB^zpo0T0_N2FY(4Ox&YamL?ZwvY5sw9Yvtfx-FAHq41_=W{Wn|xGD1b?D zTNbODs%1!J4L3kDi{K45ttYu6a#h2UIT}QdS_3mVl_-{7eO%9D$MnOilp%%3jGuCd z=^1=pW0usF)8%4N48fa>WopF|*Rt}-tc-1g*HVuKUTgf6j(68rYkB1g`lXgbU472M zIu4UgOb0&^eoP0M_2$~qpW8{dz$8jm#>3WKuJ}J(qWB2ruBIqC+ICQpeB}d)fX~#< zd+Z!h&}34IC}Kh3ss7fh|9CRx=k%A}dCmnRW@rSmr*{5!^IvZw%!{iiV1j2LwECyW z^brA~^g>dM5k3mCYL48`$p4`rx4g8NupmK$M;qUUCJ~=4 zdETU>FVN;de$CF*>EZ%NkCqT0-zsl>m?3vi(`tHauEG4W{?WUd{>D*bke$I@pDX1+>Ho)PzGuWk0jg4&YP8O`$j-m z%cMOZ1^qVVKn)2~3xNx1#aHd2C>G%{!z<^wS)11YN3kgiQWiXO1Bnk%!|)7^{o;jA zd%(@X_6va^AScsyxE{$?ez>Tg5@&HA$7yKwA&fc~UeE%*iBWG_a{9i3mn&>ga0h@x zK&0T8f#w}P>Zo?W-_w%CJW9ttg!E;w@x^sPTkuiO!#l<98~UdkiJ2>V%$#H~?0vv1 zz1{eQ&hk>N-Lx`wW_if8(Do&z>hs$;KaeY!-;@Bwtu#tScqU7?5K{!GC$OwzM+gH8 zS>lJa+|Dih7WX@RIxYEMG?_7ePCNTk{;i}7)~D2wSAFDs$q&)3^c>P~`jM7-FW)7{0o)gW(zO$$9s?P}c!1U^RvJ ze2(Au}jpJt0$N)>rz}o_q$d`;Ao+NK(MDA-HUyDKm|BBgAdd zGe^Z)q2(VtsX2TGiOaX>uEXCmf6C~0Z*`a7UhXc>?_1klt<_`Gxy}qPyiZR#10XYk zO=qhrMNppxXoLU2PU9R6gR3wn?4rcTO0SOy*gg&Ue zI35A}-PH>x0aS7(B)7;OG&eQMw6pzp z+id$1%zTyNR!(neFsoPlD1>$gDOo{Tso z;f4k6`(eXH@icpZwtZTMKQw3it*K9{6Xw7wOq-&7=eOs?G|$lJ*QE(6pK`w&<+p?f zzzgIFXA@ZN5D4A{y6#zb`RF8X zj?u(y^2hIA8WUUlVQs{i zuw}V&4%CUk(5bjcnP7CxY`e#@VIL{4dUx92MDzH^+Ahw<70JhVrltDiW$Yh;e)BmO zl@Suqc58iNUs^&`B%``t|%$s2HuObNwY-W%hBN&^Js=MF<{mN zE64_(u%R}9fZE^_Aq5f0lnsn%5Pbq@O6C@?SbFd9iXh__q5Ik#E($VC{_RPyO9Db$wXbveGQ3gNQ5&+80n zP9oJgd287HDvb!38_lrc+XZfqL&sP!skVapc+VS7gCf+SOlIukM>9orkm4PSl5}bf znLlkw{7wMHqN+~>^~8xxT+aD6Ab zUNJ?mf<(hbom5<+I98EH-uthV$hJo0kvI- zOlhw=!c(*dVs-SYoS7j;NR>^R?Q`YOICTjp%#!y|DL)bC9TG$Ws6<#~WPRA9&rfcK zWU%u(Y(6r;K5ikb^!SWct|Y~MW?U3Re+j~@aPN)I_FgfnbEHutxh@msUtc6X)vsGW z4T__yU5n*&i=8JpAE(98RBaLQcAXK;3s3it$}Mj6dOjymKFaoNrW37Sz-T!Okv1_CJK*99Cq$45kUM?OW_!I{Gmx zv4ivR;AefwWW@fSRC($jkzy|GJkl49pPJ}EY1(gNC}=N8IYbxJ{<7Utoz7wNpfp@; zWXJAluUOy_Qw;fA%PpyLy~9#Rr9Vph9Sd4fSiFz6^3#hs)QUzRQUf94UN+6%aKr-A zU+`@KcB29CT12p3=dnYMQZw*9y4}{Tav6YrWIabMWE$qpe;Naoe8?ry81uzL-ZF0& z$&Vo6Si7TAJhAFD^w`u}dnT39Bv(D}j$W;=^mTri#`ZUW-%pY!#y@sKTQp6r?1QER z7`Zo0GH>=P=7i=&c;@Z@r7zs9Wzp*y4*?ZXpv66qLKV0tSm$N-Byl!r@`r-X5-lvnP*8atq$})YESdKeYanpQiaoe< z1v(?l`FeZdwUjaRi!+zW!Q)FTyBk2agOmkh(gau9m7b|KJSfz)_OutbOjNekyDNUR zA-)|K(tFZ%U!&_jynj%r3GwK~)-7zkvfkyJ)gy}^bk)$C;0M?G3o#>>LPcFwoX9>I z9&}wMc+VgkhL|BT>?=tw>NluDeh+3VAPt%upW{Kc21qUt0r&d}vuxqjYl;>wv3UyI ze?E1nM&Y1Mc{PM@+@3`+tqOf!ig1V|`eO9VQ}FdY2l_ZJHm%K)nq?GxS|E5?%ya@B zOlwe_k8(5<-2W_@YLBm&QvYr9WRGJc+ty&awUvF`?-c~^_$DvBxH}8|qt^*MixVF%JRk*$9Yk>W7JrrZopOWP zVrUy5dSY!)r(*ZkvU@v9rhxE2?8Nvg0%TZQTOWRt6aR`NhQKlQcq}Iz4j}K!8yd3UPrjl|zs z!~je;g8)s|rAgQPmhUe(--T!llXc#9NyrxCR`ab)LA(zBI)%BUu!BtBlluygs&N|$C<0~O_2MLAeB*{ z77rl;`A{d9N1EIn2sH;ZLmsi3d*93eCq&@$T9F92~>cOZBj4?8Af zKQBwBdcS?C{_ii;iPkb&Y9uRr-Y;K>!(cFY**ySV2Mg*;;~a}ML?D8jd9N4vQnKJP zEQ9RWjzd83d1qj|=wEdQai8g$YqV9S6Btx8N);|G+R{XJV*E#WxIR{11R4WoHtnz$ zy?U#+TAhJWmR=3^CjU?YU`E<(g|2lyE*)Zt;V8BptGOg{r?<-dRdM<^w8|kt|C>~Z z_?lh=4?H2>F<;lI00`D7B7*u?fc*~dw^1GD(S9uxd$;J0{)bYTedFrCdJ-Q&$OSv@ z6P>W=E;58$QMA^&gG-@XVaDN8sFz(6(axD!`gI%JQ}^l(+zLkH(TKQqu^pcnzRLK> zX0oaWFn+l%5Bg-n;YIWe-PF&VYm07#YB;#TW(`cokWBgjDSR;L0SOq(3ooW5nbx62 zIOaKQx#9#~G2e4*$3Aw$Ze z7W9?NdhuS~J4`+WdUcatNlG8|Zi{!pHyWtwSa7RywhKr4$@rQLY&)D z>7Q&;^b2H5$Czj%OWoGA8D{A(PSp39vHlH!;2LYw$NhNL_jR0oRUCnSJm_n0SnJUSkXGIwC@hOuf~Dpzc=9w(eQ2rb%B zCZ|}W>+#NfDEOfg_k#;FJguNYgO%}F$0E<#Wi6^rH*hY(O_kD%XN!DZXVE`-(uCvB*@e(B$O>vWOynJLg z&Pq%HmxT2|%}4eSqF|+GATI^Acd?^nfxLF^TRXt+)yEjN4)dH)6J99A+b zF@QvmUir~s46navZ0P9sqH=GE7FSBw1$msg5Y=N3?@K3j<5Xf%J=ZiQlh^xw(4`4hOW*mD9227qCO=7J}fW_%Z*p0sE%KPmtii zrb9)~5qyDh;`(=BTAGTXP!Okd>?6wu^jmqbA@#2`%+vZ#=MGasz7%X*vuTm-(t&_b z03rrj8rykXpM2WsUq2pgGs6r9ADP)ui5X=9K?V~D4mBUW(XM`wGm$w3s$ik(788Cj ziEvsAfU*%#l19^pd1v0ZF77X_AkYBL^TpiR&-M@72NuaAmgX!4EKmR6uvGW^70)Lq z#MwF?secb?rEfrV_IxN$hv&cZCm3k%-dXU)T4I z%K~wx-MC?Mr$?0ChH3r>bw4JLIsf?-Q(TFV8r;Kn75}f`u)GUWRg+ksha@r}mX}4k zv`fY~Vc>@1sVHP(K%zNd^;b3;S=uQ+JN@G_^X#HC{NVm_hRAwoPv<30aU$EA;B8@r z9MA~yV;gZ0e4lAdoWk-x4-(Zm&=qiCA-z|g|8s#Dcf5gL9RJ#qza7D?YrF&iy1qkh z~CwuT%-!(v;xg;BjcSLQiFoK1PaKZaxy?nwRyqVEALl~+I?dkuh&JT6} z{tfKt1>JXi`cFQ8yiHQQc{lx&`V&ACoP^hNXAIazAQuJnsagv~h!mR)&V@$%RmQE? zUm}5oZcJk(QqLXtkxF&1&fMp5C=U^AIXgA|L4m=KG7Vqx}o!Q%@#oh7?B7sdH$76-@e^xZ5e<;Bd*f=Y#@}v-oLX zWsgnEcD2erNx2v;oZIslGIjNz04j*UxA*;l5(cuesXy(DF^dVq@{THIXo&XL>fP-B zqv<^0x!nKvZ)9a=&x|D5Br>u>$PQWAD_Pme%nC^e*(=H3Bq1StWoKrKkG<#rb)WC= z-{aBaoYN7X5BL4PU*o!-mosL`K0OeB9+Z*D7_%nx!dfEp)?VJ-B$zI{ieAf|`ur;s zUR7pysA_JrLs|dEo>SdA$ifk#$t(X%Yt$;t62Ne@Ugnk35oGo%U11qg4dJ;fj z&PPiC(<9)3!j0ittWYv)eY|oIJm-}i%+&?ecsE<@x2RxfGCGH`pGup+4dyDE!{USL z9FTL-V%MhNzs^vri3L6Ny$sf$pbW6fY*FCq1_9&Iq$)_EKqCm8l)6E9!8r6cmlQrh zEGiG2Z4M>5)!F}<3ZL#>k*;?fdP#N#F*do%Hlu4MVgNq^TxE(eAh|FBQvm9ZPOsix zZn+#B?AK0MimFMC#YtsL24O4#A>c_Uo`PSY)nZiV4Up6Xqw7JNYWOJT#!!gvQ@6PA z`Ve3(q*89?Y9qChUguZscK4&=;+V&!bT0A#^gqiFIO66Y!^s?;u80p1cndVvDP=>Z z&uq>Hl~q*AoEJL-%5(~XXEtY=C;@wO3z8vkQW-j-LjemJQg09Imz~T88Gn6cL5s)2 z8){m+Hkuk(yo(v1>couriO&Se!j7WU&{#r7XT&AX4x`L;Uoks322>vF6J!bpwqkmg zUO&HbYx+`WdS@v8-co(~ECgB40SoRDK|~)B z+m_GP0Dut(k}-a<)_4&C=#h82T=L*i0q$PUBl>#8f!PC$d1Qf$h78S7?#ZsADv-7f zx9>b`u@;a}XC(Uq$r2oTthR_i;Mrn(uJ+4fhvf zC?GxX)8iMM!hBZHAQh{A4iVZ8j9d5vU$@M#Ib#%ZC>8s3h8%&Y`gsCnU-MHUk(-3* z1YU&2?oA+O9LM~26p!w`-DTDK3!V-IQ>~kBF!|a3nyL^}sN(?jw=zfDn9~ROibx|z z_0|oYx(9WGH&zY-uMIOG7_e_M%sOpHviypNn-5m-Fk(9h5K&{S2LSf*$p8j zgyYFt24-@rvOvL@`mg5j_NmPXD2YR|P2mT@oK6?md^E!^CAPu&{2h{!B3L_{t&Ix03(0@E6)Z;($z=zp?3OF#ihzC_w@QKp8y` z{9HEpzQdw)0u?cdZ%{-jk?x2v=sdTPZEHtWg5SH%DNhQZ3nOT&w}q2b&1()>m$R1- z92^pj^&Fc@m$lndugpD8N2l~|w01J_L8>b|4(Evk@1B_4VBs@*~*1Ta!%M3;<;j%u7QzyutU;+MT zLd0WLkzfPI`Z$0xrD*20AR_r?i9->hd$VW&R1IsJA87G|JsEnG#0fjBtVEDO1oC^- z*zZ0f?Y+pklvZIQLmt)$a~D)lk(XzEmkUti5B-7aXB`7(_z3Y(p&A!Yv_OUPYtNFM_`r1oiOwr_tuI7#-|Vh>D^@+-#Pd;#6NTn+RqI`} zr!K_=#f#FOBL(GawU&vXql0-fY`@c()C<|Gz5iyUTd*}g4QSNL)5ss9VKg$F0&dCH zx73d+@=C*^CrtxJe~5F>PG~p;FwJ28 z`ufxzrtqMFK-SUW1U+ohu(|mal9lfXkKRwtSP+pzlpH5t5DCpN(8P&g5m;qdG`M#7 zOlSfVg@#tGd=sD?y4`^+p4IAbjha+^R0P$E47!EI?{U%x15BV_%GC1+K({^iVRJJP5tHq~MvDEiuw4N!t{874Gyxp6$>JtnRXGjV?t(C6tMBRv=wkAN zFJ!>s67nIfL2&)agyNS| z`)YK^D~4%iRc9mAPNQp3>a=7xq2pkI^1|w>>}&9nRI{|ukuzWa@bB1vwEj!}^$+oM z^+t@TM#we}JZjzKXbb`42SiZ%7_gi3tpF_}|GnfxkUau&tm-3pZ$RM<4Y@8WXz{i6 zA=iCsa2)mwuQCYbIG(iFeDqNJXzt7w$*R=nHTA!u{;UF-x``hz#QqmA4bfC$z}sYn zC+8TV_uW>DRbU0Qyr1fCdNe^J6$FsAfOwn_&t@*2&%)E^2W|~niq6pSt-riGmz~R$ zz?e9r;Mla4RPsP9^62OplN%M+olN6%n!U35mCIw272zKX=-K$SduSiMsSmzm)Sle? z-Ob(kf{U5p^b+IMnzXM|b);L=WiZ01_m;Ti{V!*+8AuoH0XWy&&ktw%5Gstr*T2{Z zG65KT|8Y=&TciNHv`V9CTG|p#U{c*m^JJki_U4B>8Hm~M5J@u3>JN;kM*+hV0@A5e z1)kakQc#HefMkhXvrr&8Y+r5r>$HUsu#srI{k2gSi1$)27)II(8CKH2;O~a(p#DqS z(8c3{vD^Qx6dE5z%(x&221;aP$g`>xL5qNE2cToe-f0c&<(8rdS06LWLf|oAYQD%= zWRL%{y&KCL5B@G_K3PHG!%2#N3cgFwIcW2tTwht>--P+JE%7esi=4EsU7luX-Yyzl z#XXu_2wy}UI5<{|fwn0}_(&-0S+VXf(;5yL&kB`V%{G+tto!(C1dDw3wb8;c_B}bE zyMb_be_UzbwkT8URiD)eG}8d)#18!3X}VmXHc3O+1QC+Gm!R5( zCnn(E;g+_n2%|@qSCrVY#;K}pPF%7qvA3=hWenbSz>tV~KaLQ2hX4BvEvRJDCK6yDH3ZupCXtKx8&mi_&5608DU2mRK7of~L8D*rHqSf^6O zjQ?D(YVbJ>u+0vGlPeB>YMi~aJ?^2lMT$iOAS=7B;G~`+cojThaE|kYM#2m8uC@dj zSc@(H{5adVse{B?51Umip5h0mF?-hEA~!{TK5JVo%_}K`fdtB#Kx;iP=6g&>0`yX_ z$-GO07+4FYq)P^gzi&WMP_HVf-UFkaXxV2OIRRYFpc5N3HU8iP39blg!263%0Rog? zf|(QtNSXwZSrj5=yxSyao@)l>ywg2#J|PhicpB<0T+PTVt6&~Y$1ZDyhV%!5R67uz zz4Er?k_Tcvq*oaw=*G%V&rirsXaEX>FRO)xW(m_4q z{qBPCK;FSlw(*SM(-5kk^L1~K?=cNib+kaErV!st{_lS&G0N_GpA+EE9Q_gotIK^) z=(I2Lc8`Hy)e6gaFJyUubl3>_3wgY&f*$JtK9K630qpI?@N?6BMM0d!4i9&6gmLk5 zv~Ic;v|Oi?bPhNL!<}$`wIQw>S&bH}ify=8V}O5zCQV|r(2jKTl(2tjM!-E6FtynE zGujaEP}6lY!SE*{mj0}gY*aaKlb&+dh*UJSx65pxk<;%uTLf+03kWJeTEgm{m}}K*Wki8gx5GjUYY=2hiR~b{(*9ej?>Jkk z2WwQ2VkmP!GJC5n|Dc@*z<%<%FX&$boc%Umb*j>Ot*v!bmtyY3#Q0-%>-aj zt5xQs+Wj^dv+~bVrA5&q_vCG?IeQSBWC+6sTfsWwEpgwWb-E&_Pq(GswU?ph%+N)y zTEtUiZ=*kNX!xS#^$T$XB`=Y(j6 z?~Y%&fFsmQ02&6c6capr0C-Orid|1&sXhpji1Q+Y$;r(8brOw=U;)ZlMQ9m8zy*A} z?Qc)vjsr>n_?f*kmHU?=9f#meW!QMb#tKs^ch;aZ*Mc=$+~Z~NowPYr5Pjvei-JVg z9y7q^!`%u?gJDAS4=qUTElw(xqC$%p^FFY`^B-DqDiNA5yK$XhmAU<);4A$BsyZIZ z06y9{%Bwt~sj#^pF!|9KiSh>h0=xE`;B(drx%V0dCjQP(JttL9e4?GJ0>-F@pWDDY zv*4S3Ay%q~6YPqxd#v7sVnRW}BFzg%c!1;^7lKR^#Qc=?Y7|{}C&8^AGp7qrT3w;6 z%1u~x0xYpT&TK;*4pXA!NcY?aKbG0zZj)@idLIp4jF!%eOSoSnV&s5q317=jfCHSx z(cK(#exPW|7sBnXCO~hE%o=wfBka3n3^F~4Z)+|)9wB#8B&3YctAFx;D>QZ_xop7Z z1H$)0ej{??AeqeNY4r#56fFnzXiNFwcLm*jEqKny9Il)@OhcqQ$Abg@x2IDfM|;cB zkNmVphh7>5zPJ>-mVUPP-9$3NG<`F0`t8xm?S4)IuEF4$5Yx-_!NyCcg`H+<9p9_l ziTKhw4K71be$-xa3oTJY{LR-Bc0(ShE(8hFAiHmXY549ITK|7ON}I$1KT>&bw~(nh zfyvsXUmB;r9}j$UV;u-E7YVFDX5i@{RD-%5%JF@yXCw&V0OA?XfF*A=AS1pBRH+(A>u2em z_ktu~v)u7k(`RO5uRVc7_I0?=H+J~!a&NKk^Mv~Ru&tlA`E6T;(@U5!v6jcp03sQX z{Q^=TQ%hT))%Ba_Q`Ng7HvsBFc-wvJPF_mP*zE3m?s)AcF$4A+T|4dtD5{uFN>F*h zz6XTjYcKdyL9|F*u$$Q3-QXUhEMQ>aEi3XXgnJO=0q!=}(%Tncwg%;l9p#;E1(N|$ znD>YFgfwNeZMuPYZS^2X?!}`R za?h3mVW$VPgFa)D6Gne9--2TbMrSaME%xE}Pf6-dQJY;9*LHGB%&zo}V|K?`REiG4& z{tdkH25TbNRQ=~TxPV=i73EDdq=!Y)6%c&wuZU0!~t4}d@fnu3g^5#2RV zHLYr<++U$rfPkm0Vh7TU;tGntg@acw*!c2fbkb|B=nS5hzD=i9<96P_Gn)s_Zq|hM zg*=4Aq|XC&ZqN`7&Mp`yA<9zZodIeZNp(d|CTGa$Xq;tPdm@J+m8ot5tp>o4o+TEr z;@Ai!gDVI)@%vt>UiBBe#ukv711oaG!Ss7o5bZS+Qg9EK__hKS&H?uNfR99+8%yDw z#sA?Ek;4M!e>W~SFV7pT?|*Lot8Rd{-Z@vp9u6slJ;1td-aL%$u>zY2X52R<&hG;} zSDqB)->GGhh{Dh0CE0m}E;sZb9_}C*!~Dd*QU?<@@9Vb$^tSc$-lUf>r7K%dM?6%t zjDSr`P{SK*F_!rg?!lJk0J-z7U|&BSG-6<&C9_$>9P9RzHR%4pi;UQk)^6Q-`BRz^ z9xGbKZ+dLmzGevM$5?VyVMF_Fm-)JHO|xJLfc3j#j7%Lz^(5U2LjK!Z+9`y8NhbI|^_5`^5 zwPuO>^&OpM%$g!O%VGkbT(GoV69OO!!RjexH|CsQ&8Rs?oVhov1W5_VMvlOt#NCM7YqNEM?avhsQwz6+>LN9Le*9-!ovsEGi=#DRW=p?%MLASMgZ8n|*< zPLo0~4K!8o%>)SNB#+p*x1zl^oHC_)A6SEaxSORZSzVKRA<@K~`egHx8~EltjM}kg ztVqC?<>KqpvRk;_pLVzy8P5C)bn^x#RJ+2#YX$oDiTnST{=7T~;Z1@->7(~;Oh*RN zFctD^`kGoT4IQW}bf92t#WY9g+~crqN?>OpjHHS}3e)U?c8T1ZBtS;h+{yi7*4XR+ zcXo!^{yY7%Ir%D4#vh(PO5PXvJ@lUI4mhVV+P%9MT{Cz&G0+|FWQ5-Sj=Fmgy+C9L->aMHzkIN-F~&%-Q)LNQBuNn>=(d=f~1EpqiYM!waddZ z@Wh=T2Fu?(AROlbD26@@vKmID@Ywb|6RAF1zBl*g2K*?HI2yFGHor3*fJnd+;84aF{%Ue3C5*p-P>uT)gl`(4u9vYCB>z#ba=8P@ zB{R{Yt0{EV-Fy<{U~v^7jdX=n=oc6M(C@L$rQ@9vLs8O#dyf98DyreXMa zqGXsdfkXn~rIafd>s}|xe7;ZF>Ku8OQywbbqx%kkHtHxeJCfa+Ceu*p zgY_7p<=bDfvQ@ksuD%Z@Vj@qts{v&MYb%|)g76=3Uev3CPo6oJ*EGeAbL9~!P)xys z(l}$y5^c7Kan0`)utC920jA4O)~TeBgv9x4Ns^@JKZ7jv%ODyVfzUpwfKyyK;W@0~q9V=eqZ#)JWPF9!*g6!nXzh1i+$E0iK%1>Db{Z z3kZJvM|(yR1${JFD0cw&53>F}tZ13qZ%=Op0hn=YQ+nm_nMa*1X*byZ1|6P|%2eO@ zLDh8^_GK^&wGfhgZ@9C!loxO%JQz+zSfs=5ou8fZi2_Gq6P(S64 zcPx_#Dx&ah<~>n{4mcN)Cp`efY*(K^o6&zB|DK775n_)`}yz>8NMUtQS@pnIsO1R&6Rxoe| z8a@2ppZ@#myO_(j?#@3K*v4&3fo3AUApL1|^TuczG}H5YevPAN)$^CZy?$tOA(uLB zNP1vehZo7leG?N&kh&n&dIb9jrX~W!7WJQ`{6ES$A}KrH^pk{plL-FrdPMc5NZBPU zq_m&<@P3`e0Ho_bDik%+TqB_s0Kw0`=o1-vNB@y`4x?Z55RJP@3Rl_f z>Mf%vh*U&G(%prqvuY@UQ{2Ta>`#VZt>vne#|F7D`v$on)$B;H_#2hO{L06nY#JOk-o1fRs4q%*zcreo% z!au!m(ZXyE?6c{kDh;|#c0y%lT*Vg*q;g@-;6$;?nL>d-0kECmRn=nlV@sv`xCSF4 z(I+OuM|?dxI1#Nu%?{eKlQs%d9sm>x? z^Fk-+$-87;9b81{?WX;812D=6H6R&g`KCh8I&Juv!bpv3Vqp(WItE?iFikcv}0Fb)^aiAdX&pLbycQNb2Ly{9gTJE=OH!}4OaRD&aNf8!u zj-;)Gk(TMzyn%surXi?L2w9(X!wvz|USK-};C*z40K&7W8I-P=Vk=Mm6p$y24Qb5% zRI7MCld3oeq*cV=icgf@PKo8 zf@1FZK3j)jbQNbCpsGL{k>XAVn+7}Nlwr7c05u!}{u-5mqG;aY^?j?Ut4qi}O>|0* z8y^}enMO5Nq0IjaqA*^BhIA1DdIF(r4ndj2r0PU76!c`@6#@Sg6;;rOB@MALD3|ZQ zRQo`{`S|BwK18xxetyO+yLlT*U(8><~Z6k?=5FfnBX$x)Fvhhv0i zo$(cxQ>(vbrFfx7A*UCot$VFi_Fatj@An~!si#XR=Gv(Qfe2c$iBNd5^wBM<)s>YWAn{cOnE(vk_yLY{3Y#y4V9q;P;$5ZG{s zmz5>LExakpY`fv(W$Q{%F-Ytt_HDiM_#Oz-`@mwJyXM=!EP{?CjpwZG0=p zMk3-aW4acPpy8U)m_Gyp0Tu$NhAHC#@;=b$Vc7Ud^ERA#w!lY->Z>t-@x!*t-EMHT z1?4_yBj}3b|CAtIje;tH*O=%FM)}M{a9jNr+U>ZGZ1&gD0Y<3%LLzkLu3w@OZDInZ z(d4tgJ*9A~Sg&@L_E-KqoU{Z_TrIY!9$V>n3Q&h|m>lAWP1n`Rgn88~8h?3wmf}H- zHYO?(F*?td+A~PErQs+Py~H)J-1azcWOdbmt$TpZq%S2$(npS~cY= zN=t;3$RAO>_T?Sc4~!rBZTGu~5+B|1m5zI@K4!*%4G1Q0$8olY%NX~Ec2ug@-H%b2 zU)#u1H*&g<+ooBVMwWKV+T7j+LJB^Fj~$m{|7c*xC*S?gE?8dQ zCRsIYYcaTIr8KD1!vsW!qQ-}e9{92)wTuo1OL0f@36RL(HnpTIHgI$ooZ43Fv)&Oo zd4SGQCE61&TR-q=V*ayX5!sg8V9Y|TRN!AKpOt|aR;z5q5FD?tTX$fDLYhZ^k;Y^H zHb;pfP{WOu{R0P_uHVFYl;SvQ%@jFbmfqi&k9DnHOO2Pg6(YA%ORMs=%_{ z$L{`44?5fIiYo~G<`!sYAigmTxz-__0Z`rp<9XxL=y1MR=mF8DUV)b( zbyO0`evE=e*F^mtHk{AGmjqtZZ}&$1q_wD&Uelb;1U^e=WWMi7EqKSpfROqu)WmDI zFZ#NVq?yLA#?0lOkHo%cH(!YTo_U8^`)%bOkMik-A@_~aozg&JXZkbTe(e`^_VEpCk#>NsX@=SJ@dJx|687OTy zi*(i)r=u=7m)+iEvGY!?SUN#l)jv4c3}r-l^C$pFB#)Yik&LCUJDr?&@xLMKenfr< zz}V0@&LT>ao)mkN|gYJbouKA&5ZY@VZAd@5o) zJ+FZ^`1B=_yPUkdmVpBxf`yD_uf6U!w7T>}j|40ns|E(IDh6=5hS%hRX;$#o} zXi>fbBQG?Nl)+fQFtd{VjsZ(5cR%9}j3ppHqLMW+!;$$82=4~z_Dst@hUJ#)$G@l; zhozuE|61;NjKcEH6(3}V#;Uhb>g-s}T7FDw6#=nWQbp6?Or1r!*I1^h_5NBV^c;TC zgSwH}LAG~jB9!>?qekoH@$qv-f!GRuH%V}y$c9{GM0pB4h-xds5`+Z?xn;544-?~L zCovJaY~k5fxb;GmEO%XB8Q!zgF$)uH#$*oqh`&L-ZpoIUUQ!3FsjhBWttYR+cFC3- zzFd!ZGXtjmspK?1UTveH!FAI`@qWIks2n_hq&%5p4eZQRS>h%#Gb9=}VI#oW0e3Z+maGOFor?57mA@os))g8HZ9+3(v!Y@6 z2WHquV71MGbI)B=p*yaMdY#aOkNUl3umV@BliwTfGUfVl)>%{&%9)AtYMBLCP=BZ! zXUsGAe=i7W%#Co8nZl9E(SI0C(K$?eUVrhW&C3S!d&6pJMz; z(p4@q%p8yTGV+h$Fw~}6zmhdfsY`D`c59gkJpSu2Sk_YvtTb7iIZoTz8I}E|{)j6KftUo7qXOGvy z3B8yGKJi8UC*%y+je4A?g-pgfx~_Nc-FW1)nS#t{S>N>s)Zb>_K09|~&#Ti&6Qi3f z;P;%q68prKC>L!8B7Kl$z*lhY?W5gcNPP-V>1IU8ilkJ)d=8kdY!KMxHgR~s*#p#i zd7)$^)(mqNH1g)Iu3EfU8QIx`((L4O(CITEf}2o!XIz*?2I!EV$>5E>68;lil2@8p(Zjwa+xvazuledD)5!HDrkf*r)(ObrkZ(M=ws zg|rHJWa!(~|6cEm{^n3)`?7eZbX}i-hzKL!6UfMaYLdWs zoV=DOq}1k%vq*%qeA-`)jL9Gd+VIc0T@UgMB`x$dD~;X5FF?x-1K8)t41>XG_1TE zbbpx%%$H7lBx<8sCp1eTtD9ftI%iHlx%~9+o_gL$2URD--2qEI<|GBOJ-5dNcR}_G zN^zidu=!%N#L{>L-e(;Y!Cq>af~Qk!);s3t-rrPvw&Ipl+u(ri@lBP#-9BkQ1wG5B)cdZ0y!*1RoIKT>H z69_sp>b4@2!JX^i>I(7Fun6`Ph$v)#ORz_(X1Ubc`a@Yzv3(|T(A2MK|1&lzYpGrl z?B4B!IKlQ;rC%9Cz^%vvvf&}BPd7V0mD~Eu0+fPO?T+YV&4U^$Ib1U&1es$;Y<=Y2 zTW39;;Fqxj3XVS_#w{#3=yQ2jQ0&XV;{AUNnUpq1d{R_x-eY zQn-ro+HE0CMs_+8|H$7bHu9)+#wAgDV&>DqZWk(|oc;8xZDJVYQ2yj8r)8oh0iSlY z_O*Uz_Qd>8hnnZ3x(|_G%ZtvS8gT+qxBwsD4`i&3M0)V-zjt-LxD9!r(01Ftawr2{ z7U;>|S=P;@h>D8c06x*5OSyySU@8Lnx6O>LUr&YKr%U2kF40QW3-f)|F`K^hTCh*Z zmUT2-`CU)-d4H5yfcS>wF`le_?|qYL?H^Sb$J>udI^=kA*lvCH@xzwq+#zGcVm7Bg zC;wGt{D5^+Y9alH>7D#ux)@m?HScusE{95kNJw>v!rshmAo4HYn7oWxZ1)QygBx;t_8^jt#!;02dwVo4VpJFCWz+*34zhH1` zk%eAGAOo*1ze~!mzvSG{qBKG05evOqxm{_E$`>v2UeQ`^UfrD z`!*=3vxSo5Af(f{J#^eI%PXoWq8f? zC1tUrc-+m|wO=-Tq$;1wbd?E$FoNc9W6R_lNZ*V9iE`~ny=a`(e@q@ct^ik%@{b9* z9Gy1Xvi=wgZx{5d<%RDRENM1~Guf}+i9SlX6JTOhOz`vI=U4ZJFaqi;`|@OUQ=ct| z`)&j=#VD8y(5$W=nAGS&u>`_^cQ0yK!<37ENR_0I>ypNHw5!j&zVm>ks5gw|N=4aF zic@j;RG9^~9J}Xa@4JlG$7`Y_SG%zYzm|><#ZdqCyqbyoy5TxkygN=XJn3`(WT+RD z?i}(_sPYXlwwMXUDlhvVlvDnBS;<=T$-lz}!bbS$RQ{G0NZ)Ztz+}DxpE-02SW7L7 zcElLT>8H~fRM3eSC44zCg3Hd6A|t8qG50`bMKcT z$4RyQ#ILO_r-^FBTMryS*D(1{*}I+PM>J{dj?%vC&HW0t6K~Jk%>S&tVjpajns&W- z9ofhobnH(@5+cbRHO_Rz_f^9WtLeN1 zKW&;SXLy?=-kcph%!tN#MEr{)z4hxvE%mn%=0}8|&hCKK?lzJ*1FvpXJC%w7X;u&^ zz+b12ND@5qqNk^gK-aT3x-gaw`M6>7*^|y)+-U2Kmlw!F)X-}+r=(-s&DGVNM^y|Z z2=a6!R8j{6C z^~%h!B+pmXL;%3{qHe0YySoy#i>wQgImO+sFE0_sJt8=g$?QJ}9X!Efz6Wrrk0~h@ zuCCUM1ox*Lv@0Dd zPE<{6^i~OvDlh)(hEv?Y%*U6?Y>{&GS1yK2^wYGBZS~?Vd-zXR)?RhL(w^(|TGD{M z>K|NWi|_8Ysl+|03(IvbG!&9zNqt-q&%GoPEiC894p3#dW!H5$ANVjAX-QX%|1KQ7 zbIj6kJ<3Xt@zH0z6ECPt@=Isvq;$xIBGoD!_MPn+cb92YELQMO5@F83ux`#l+V-rb z1~041D^RBFEiesQpH?KsvubcXp)G>cv`mcmG_BPueQVl0MUkA_8+lB`Mmm84ae1Ag#@ae$Y~} zr9f(S7o0_9B#NJ$MAu;N%FKa;4J`<*0bryRevJP=! zP|#gUIU0!F#bs`jf|FJ}a@h8B+x$r9D7+1Eo?AKV&Pt|mV1XC zkr>8C1Mi&u35bM1EJ-exO6OREQ_(Tx&LB;=qhRH{=UNMc2IYsO1bZhh#_Sg&Ec2^G zv&GoMGUA!r`xjrpiH&>oZ8#&P!JqOTrmYhMRzoO^%!O2&uT>V_*|YBL>o}6ASpk$(c1@C>?B>E|Q+<1C#?219R9yJEl~)r~JLkw(tHTvp zYNmll0zh3TlqTkzp82``TY?GS%F6c_Fseqv!jY^`SR%ZI?)x2BVjS{9bFz2U;3+;5 zZF$!JfcOP82Rs9BTwRUy(mdzzknKN)3B3qK>z|0d!+Czxt~tTeSLh~NjN#8_(lf*_ z-4+OQP5+!eG+9Lt1i_r7^z;$M0wH5tGj2$NL^rZdBm98?!5!+Kqr%i3sF9g6Il6wxb2)`CVYW zLzX^O))!FTh-OGw*Tb8qW?*QY-`)cMe)VcS?=Mvp+7^7yIj4KL*nzxV=>O&L!6|Vg zwqpw_DXlKUMo5N2yn&VMB@m?t5Pp@S8APhM-Np#vzR02m5m^8RQsQDes=~Ax8(2Qu zul2?d{&Ll2<9?ANB=_z@#=(DE9+;TULEiUDR>T|mY{Y8;Vh7(XNY6=MUH~iN^M~(W zBRd&bGgw1hGDVt%7UO`PkC>Ww+n8jZVow%gZJYM;|XocDsEMs_8L8Pqi28eeCVHB3Xxctt18*$ZmR%H zgmMQ}@1=pMn)wRu6(|johh_2%bM&9PevlhW^W=C`aht!Njwwl@!mczSY@9kUqI}Q1 zgG<_27~3Y84A0|af_c0Po_9` zS_O_}!lF?ve#}?yl(E7Hv&@1vc^DG-24#Z@*WAva9p&{Jh=C{f$DMM>x1j6%b=N(r|lUbD*3C-Dh+?bA?kVWZKk3)yu_=w zsEx==Z**OK{MR$|Xmf4V7C!`LHAQ51)@$8gCLb9W2)9#)f9&~OeMs`9@dh;F+)Rr9 zYn@uZF2)-m*SG6$MQ=oA1R{mZy3=4x3O84GN_f1B(W71WJu;m_ms%!_DBGk~UgLQ% z?IhaS5=Ciwuo(g;OkmUN?|)rwxbFoejric3Z@y3_pb)|%}F9Y`D~{3&wOpq-ze z(?eptq3`ix$?82IEx z#LfX&(#J3_MH@2lhp8u)Kk{dY*oxj~ZvbL-eJ@;wo%1#GY}`*LT!4AL1qLN-e_56$ zIKxH_T?nxQ11=9QFBr$|;$3=09*PyOVim?1^FZP`u_PpjAX4l~rV!!!RTc1OE9H;| z%LG94mYqHG-F`Ib?(h1WUFJ&l{4cM+PRWZWHjPd}hPI`*=+B+M0q^*&cjeXrnZ!bTARxgY7h|DPnRbi* zkw8D)rvbZXEaxL(de|ozaC^v#7U1P)jHD9xSvEZ3xU26Xv#l&Rz*hK9<>8jQ2kdsL zc5&{my)t9#+$e~5yTaA|OS6!cAxp=R=?Zj;3Nf<9>RH0GH1zv3$h^n4DgnRvhz z-Sn*Hv7;cU8HVWamq`^rRZ8$e0dXstS3rQ8C8@m>4vex@+`1xakXvXbrs(Lfq;=aa zl3B6feV0EqYXOL!fr{HNI}6@44jSo80VOFBaO!>RMi%0`1Pl9isGvm6i3k>t81Gn*5kx zNb;IpB5*IkM>>%c&uaB&<;PpzL5o+X7W-x%cso-sCJ}BS+ZSzyVBb=OLQp zEVS2}|M_U~A`&%;%xRl4lBx`fbd1Yq&R&Z^R*G@u6jVr$D#jO(P)a0*}DuzD$!vqAbS6wf00|QfK`inz0&`O_^MF2k9Ajn+N z-MEUWw@ROMQ7O)GuGP$L6!z!he5>|QS0t;!12ms-3_yM}%q(pA zZ*XRDe*xT1r&NhPtrP{{a9<^CGqRCIrlwqb?K!kR-|8mst)pH$ml?|8ei^3qth?Xx zSkFRkN#*&UJ4wykLn#K*(q!J&S$J^nmSq}%b7O*1m6D$mGt+|@02Fq^+foSCr47raNx~Oxr%=+bKN_19mnEg)jy2RE&k7UF zlp+QNWayMkj?S&wwcmwMi-^})mOp5pN&|ZZkFm+I695q3HBFxjRg9^k)Vukj^wzmN;_@={7Hr_JC3-fCViU&Q%87r_!s%YSo@NCX_ z%*QY`QKcLH9KQ~+&OZP>_>x<(z#L*Oa`ONAUQ@9EH8w@_zdsOLzMV+DXloxT;{_XE z5Rfmr3&8Eks9B;X`UIvS?|TydU^px#hOw@!9*{h;a_SvF->lY5t%)?arB@N?g7zjy zV6O*@pI{#guTAsl+kNF=^CY)M)-Etqa}56>ic0NV(jc(mVM_GAme34k+Kngd^Uh-v zN^W+kDfVI>Ra`-3*<3AST0xl&y#kD7S|J-e!ZPgk=~#J%|2CLvjC{E{;8uaXL^47j zsA)V$LGK)oOb-y%vcW&*)s^pN78QCYr&O;UALWozMl7bQp7jsBtT$izgFoNdXNJoV zytmMnc~p3i^lau5yMWkou$H}}j_PFWw2ZzMYqg9xv*n_+-*>7;)&zexjR~F*+q$v zy|%*Cyk8^BUIYpMNGJ{TPuCPQAVzJQLnt2?94Iwa806s)M6b;fhB{RnD%qqr@0Lf6(+yJ)2I-j=q z{!4aFokCB!gNi$MjPs>E(8j9Xw3?v>HJg5JexVadR9$Ngw^R@dO!{pnEaglGj|iUY z5SS)*6+a=DW=YMKTruaMx2^$VGXeJJpD$STAdJCH{A1_RBmW z48<75&yZARr2R%HdBwR78gQn5fq~^b;bi@#$|(mh&cjMZk6*k${zbZi^@}k(P>x|Q zV-P~Z9l9g2P(j>yt^U@o3rB|l>?ZZ-B>-uVvL7Ek&F`Bh7ytQOncoL}d`9`6>&=&^3G$5Qn zwO=tVqeD*H6R$HO6Hk%VLEwWfIqn^;ne%f?idQ}xT@I0kWU*f@h$nRGMu_97oG@B0 zbra|=Z!#jlzPo~s4jPq#%-;4bEg?0vnoyYgKv`3){(Cw(xy@QvLOi4XkP=>*lUOSJ z+~LhZ`zP+OY}oNO_|Tm-?XynF9g-c7w<5+|8tm~jNd~vTe|^Q(BX2b4cGaIHFK9pd zH(OE8Hs)Wn=om&F*kUp!=Cit{HY)EhD3a{#t8;1Na(OjUbZNsrDEomAdw7<8-$I0i zGcPZPHqupCkIo%0r!W62D1ivUV?xCI0Op0$(5Z76S@c-|!Vk?Kf)#MkR=%4?Ul&)d zSjFM<95JQ8!2OkWey#SFdUdyDZPNP3gA-nYwEHA~pKGxBm6wvaP%Q;+NCo)??o1{u z9R|HPrZI6gDfPOxkGr$9yKE$2X6P>P*~d4$UHm_ZhTu|KKob4O(Mw+WO)n`G)<1Di zWud2WgP!I;qRH?R;~qJxr!7ZEN0M&aoK8?FFu)pq)7|>8tGm&2{^V+usA$qOhyTk9x_t6j~ z=XW3}&Li|Pn2&sd3~#VvMJweT0m}jJhaX73w~qV#j9zUqM74>)mIR5_g)ufFw}9y~ zxts>@4PE}TU&0_GH}WqGRAgdU{ryBAT%SeY5wG!iX5kw>{dY|U7h$^g-aj=*^`3gE(9ou-J z{>(<6wUEFVqg};b+Q+bgn4xc|-MhmSa;A11KNb9NC$e^~!y_T!eY&nd!H9`4O$UWm zK;!64;NbLyMDZ}4g>!K2kJrH{8C;ey!LHDg4l1RwLY@CQevoj4O{_8{_Fer3IFS+0=p>2)QWncLIrQ0ZzA46pvHk`T5Tu?A(WyWm4F`?w zzAF0TYp=ER2^G}Iu_8JY*o;5HD`5LY`G^AYe&h60|6Y(_V}2;O8$<$g1t{CS0*$y$ zq~f(FE-VAXAy#yRz5TgiVUafn-V0q3;d_V zPxr`_KRIuN4wRwqly~7XbHwJ}Rs=OeGct8Ci-(ua@bhJ!;jr&_XlyFRExxD`_Ch$5 zlXz>p6mOiqiowjTmcW_kblQZ^=6R8hBVYfTqe~>3 z_K{Gs@`p}s5KRE^DAS*+V!ZHa4MUtP`x9HiB$b#|`UmH5{0YGB47N~y?NJ>zvKm87 zxUn5l@E^l}Hi{p!$A#S#D5TX6q=BvRus%a$O#}=vGPN>CH;4bU#n1rfVVdXA@bYS+ zp$2CppBikXEQH)XhjK1y(o+Wy&V63H~S3 z$j01i<@JpQeRCpI@liSjIaA;}_^vS~fd;u2CS|&!zG@)ZCWIbj8i19V)r?GJUG3H9ZR-Y%gg=s3N^PN#x1njU2bugeo$8J z--8>r_H3uCW6>D_pH8|Cs_2_7YdY@@JRG#G`UO z$1uvd4&#J`;S0hSqub7?+2xA9sC#p`u+3qGjDbTC+aLnVJ_sJb_~Q0Y4Hh)ya(@vc zR#pVh&&|WfJPF5e`1ApsK@Bb!vMbc|{)y7s5^j~|(mdw$yq0XaHHC*7ee7HlAw8y=0xQQiwT{Pkmf8-jkC73w^1 ztM;R;x0;qg^S`cJ_B^KVvy%uV{ax@v$8dRAeCU&ng)Uvcnpa9Nx@lq&svpuVH}k5G z02hP6*>Cq9zKNqWwt?*j-RE$Z*qzVNb=0Ht0y$j{SKt`q{!<6oT1`6;kIUcz&w0v9 zE=G&FeihlDjX3YspLq$wXm6K zm>11SPP4Ju-I*52w(b7Prcxi;blF&%v-?l&cXb>E;jR=+=7zZ$;JM*%@cu^vQ@UZj&pk>W_&)P%?tXLUJWnx7b85aV0IePQPBdL3hOVTQcSSg@|8+Fv9f; zL=iJ}#?gkethkGXvE4}`lNL2^zqC4Qn0{;J~g* zJ6r7Hdb~yADzXKBTgVP%nvjzR_G@3pW3-m9G7B1!$A>;0AYlv=>;?pWA?PY_S|Z)OzP zE{LWgIg6kt-wo#V4nP3v$X0-tTA>*F2Dqkg$0^yn1*gLd-~a}ORBnsE2gVZ>ay0*3 z5~fYTcIEK`VY=)>y6)J9Bz55XYpvFNMHg3hmK^g-=-yI;CiKOgpzMkCf^{E8ZI-)$ z2tVpLofLN+0UkUisfUiatw`=dYZJL~<<>pai%oa=gAtHv%k zW_o^<=*M^7+Am0_swd>GCRma*y@jRF9|26-lgSY;S&X`?~Jx z+{cg%ct~&YnIo9~B^G6QDY)4E7#SH!?b$r!=HcmrJ5LPW62r{PvypA7ftiiASrmWd z^}>`1=@U>y4svmwKr;SqeB7w~kDc#ghz2(;5Uu?j6EN6-{MUU?&fuyo5P;+ zZaeAb;0Fz?E~LrcR>bs%CCo2)7ZzchbHKuT^t4o~#q@ky_vUrYCaXDAlZnsV_oaq0 zw)~kTWC$O?*r5Jqda)9uhfF3J8nZsFm?;Bn-zNYB@|H4xlueklBZ0AA-SQ)S0*ys} zTWy?b;c(X^huBDXcmYp6WrKfksLJ#dZciYbdj3fN0Nn>he>`!PQzq2c!Wh4k`3JP# zs1F2nNb1l-fiYP)wu}0sAp;Wg(YGAYU$XdEaUKyPxgu$7;X2%%dyAL6ksh?yM;EXA z&{7xMWr5^1RW6>rzLbqn7e7#kI|#r-J5SLNKXjZ{$f4^;v6snBY_9}iB3Lo=_t2yF z*BA+jI0CmT=g*nDM_rJs3WE)Q!RL}(xX)V+-MPw=cD8r=VXInC8 zJa?2OP|MxzUwxVdx0$8c12B8=riXY}!blXpC2i6Gz%yPli48*nk_+H^Jno?2PKZ9q+IQ!88PlKtrmWVSW4ckrw;i z$;DkZl4?EE4r_D9w0Hf>+hoVyIqj-`VN=%E;4L-elUB=QLihTIiMp(TfA;F1>);JV zctn}U?@-kfCRacFESU9`;NTEOJHW=9T)Q>_3uVB#z#ezTT^wY1VO<&xneI`HgpCq6 zo8}aZzsWLczZ(KPZVf_nIU!+fOOt19sRlr__!1F!S!bIzMDm}Ay~Jfx z(QvdXj}yi*qzAP(Qo&c9Hkh=^o`;A^^Q%)_1qP90)9-T4@4sO0dolch|6e;|q=rb* z$jH8JFQ6bCB|s4WWS~fl87htJXt}fR+a~3;M5NS|9oZq8Z1>2Uz;vmHX}!&4dJ}`m zY(bJJtE1tMCQd*nm!3ELxAgSi2LrGEk?Nq?__yzks@c)LYx~gCd{xeM9k=J_{H^7Z zE3v7Qr%^1Tp%=4LWZHf=C5L@YYD^5yZ@pbNrm-xSWOrPhx_Q{R`6c>pva=10bd$o` zz)w5td%lH4L|mPfUoF9sWpZ=KZ3AvDzaWnG8f-4jfh`LlHIdzLl^39paoH@ZXFvQ4 zpKfFo6%(`9j)tqsMD%-o-E*W4eb@Q(b)%vaZxIlgIyV+jDPTZVX})f1&qtg=%r<7)LZO-j6nX8-vE%2FQ=bR>DZJpHfF zQ_`XVJ4l{V8YeRR%Z{RHFmMKF!tOSK+a|`U)nmu6Na0eRn!#w3RQd?Msv#BWE*$%p zZ``Ry8V4NF&k@3=jhnkr^K!1$2a#eMr>9DRT2 zQ`-x8P9gJQ%bg&+;P3gT;)%PMs3g_lFt!6Rr(R2Lw5=@6H$auzffD2+{-r1%j&t~> z4_*x<4_38b_L*k~UIgJoF8B_dIY}A6aZjBdZwb!&(Z?04Ies5sQMa9r#i?o3z@!BU z6(`yF2UkE?K$)AjbB}`ji@b0gG#}JL%VpP>jSzK_rVo0x+#*Hdb+E*M0)y>ufk9dE z=D5we4jM|h>utn(HZFnE>M)!!DtWDF4&uUcoBHQ|$fja*-#l#s$2yQG@B*P?ejOL+ zb9D&)h`Dhlx9rMakrF*=&kLns4uVAUQ5OtQAUL$p$nm>1FFBx9M9B-N7cKJz| z9rUMKt>U+8!j~t92Jc90Lsa&M2SsbAn{^r)sx46>X!vH6onZxqOmKV1v*Sv(CUXr;2)WFVXHzOno> zwB6)-K}_?yP!!sZZ1_J^v)_hf*FvBBNUc75Te-7a^s0WhwOknc{SXDmG5LE?wGFB8 zQ&@sn6qMX(6t3UI6@@cMR}g+G3_hPNe~E_NF`L>uAPW;ja{FwPSWBRsxx9eshcgUP z@otA2b3gViQt20fl)<+i)jC!xM4LrwQtYfo^!ZlZ`)6O6&;Q5aSS zps>ZvwfuL4?rp#1SIvO0{7$!0?t;|CZ4I6{-qNQmH{U3&y^$_D5#<;s3GyxHHV`LHJ z-U*MTD@Uv-q@QGSBt6jHz&Q*BCls7RknRJ%C7x0k+{Oi`;m_MCzzZl=2dhV=_xm>G z3sMPA5kd>s_sQl0vh_XD;55~8d^1FhdHM?Owl?sgh@4%BigZLO4WdpMwc8&jkI&Do z6snhMFliZS!siceV4&SYYZ(Y8{6MyTKbga3sm-e_64Hj?K?KEOQtbQX?38ePtb^v9 z6KWNU2Z^NzuCL@QoNgLgAg3-6(LXGFs*1T=!v;80orFFHk5@&!4w; z4ZEt<3(}#JY@aB5Oi!;)ekSP)*QB_HPmDD&a9V2-vi|^r3^3*FMQTP)ZHXw<8chE2e zxA~UmC?8&NTh8l^pyz|yIdM|q^I?@&Eo9n0H`jOWOFX<)XQbqKNj`avNhLe{y==`K zd8kP6`p>qU0fW$siwUvfppls8bM^RBZKf~EW!`4VO8H?()SmI3;C`-|IG(!4IoO-= zSg(4@yR1+eCZXZadDFcQ@~KxMjW3oqjV>_wVWW35v@*fd&^`x!6@gHTkMz6r)~!9{WF=%-Vye(b~@Wxe>|anoGQ;*rrU zMq_3TTb^3~An58;{U9$uia&|*xAx`;?B=7;(dG=dd7J*efFafz?C064$#XP6(p>j2 zV9yID`sebfuUbU8yC9i5}p+t*rIZ;qwD_W9~V{(a5S|F>3HEwDc7y?LGyZw=}1DKYfmS$o2THt>BJ?eC8F{8{~urBjSAFWx{&1SvZT7l?|TX z-d|X7e}*&m;Ko17!_75CF^~#6HBrDI5jN(fFKIG{zr-J)dj~H+2Q6jS=;J^a30K%Y zI8-Apsdp}LgvhMfZLbCCXgalOTsYR!gRl{T)xhAy@q6NEy=n$dD0_x`tMZn69<%J0 zbWUP023^NJtg0-B@c6Yx|3Hi{9J{&RECw~Jzu39ZC^O0A!A|6GmAJv&jUh-r7-ytV~iKygU zkm%*kTYNtcy%3ddvX>6>yv<;sK*|qX6Ctrj+H3vyA8)z%;L0oGxNO0@3{EqjvC$u{ z23-Ym``-9EP>%96h3}&Xw#~D*li3}AI=tI=-cSkQsHycgryX75PhzNL2@uBl**trx zCAx~{7_i6tPFCf5LGHHW${MF@xhxSMnhYFj&&5&r(LL8ZNiGaImb75KzyJCdY|0q# zL_34mk?l~`hy=h2EalDd2;M}m;yX;rzZN7Led@Y5Is&W5O3T4JVr~OI58S6wP~_vS zYxNKzu%a+gCc{sdtLbjgHnZzM4jOc$#i6;?*FlN2-FVDFIrV*}@G}nX>lz#r)rW49 zzK{AR1(Pji!RH&%MZSUUI_8)i&9K{Gpf}?QZSJ15&=Xjvgv{q8YYK$+S&wy{Ldbo& zY^^E8nRvX?tZdV>g>_eD%AkhIsM}BTv@}{-#Aao%7 z=pyjBIyokL8i9g~^eZMU$od{UEdjrt7KdZ~4Ae;&g>X00{c|yWS!vK=&9me>8Q!Jw zVRGmejOz!|9;@-i%T;?fWW$VjV?^A3J3jRy(;mK6thw2hpbL6pe3jyVb;h!A z84ntP;Fkl5{1J2Bw6RqWE|^s1lmjte_S-_|FT1xEK^qq&3L9>l&Vr!SfBSX3&SgPJ zM6#OU`R!YgYO7TjlBgWvN>6MxK4?Op;9WfM^EL$CTUrGrW)c zf4>m5OeoGb7E0`1tQfJAKR|B|Hhb_IBTz}&6z7;DiX-l;jRhmgJr=A!C8~tJchR+a zj;f9|zxNt?Nyo+fy8|AXjpA$JaC}%{TYd=Bczf(G*7Dq-92~HG*dZ;>f&!1;NUkqI zF>60b**Zj8d9deLWWo4eKs)C_?VC*-)#+x%=>M4ugpm9xRQ`i+l8s>w{q zY%H?Wgb4aMlQa$IjGh!`KzDdI2zE_xL9EJt$azVNgmj-xhyML!0;exvxC^LW57`d<{7EA$ENnM3 zdnxG-<_cH`1e>dFI`vm66=>b7U`qF7j5mSnT3l+T+!~a=6R&CNC-PMTuJi%rMUe)@ zN`Ad6pwY5OB8{ma_ccFG#y2xeAtrfdK^&vSnKAGUy|Pw94je{Ogm{O-WGmHxXftghfn3c?T= zQ>OiHd2R=tZIC@G?o0wK(9-_};1xI%!>x)n)gtz3{`4t8tYNDq`s-fY(4uPz`6_s$ zElZ^r3^?15U_ApXEiDdea3ejdpAz8yja`5hPiMwW2BT|`eS+4tG#S_JdL9gp;YtJE z{dosIcyq?{ZAiSp!#uCbb`G{_h|MMLN9UP<;W`0~@AejbYYn>T%%z`sGk@+i;m}wl zlXMC6D$pv@DM1$Wyp!t@Tsxd-s*2U3&ra}Fb0*3g75vj0sg>V7m&fkFf!mnfM=&NT!cR0}GgA8= zr&p67fC3)NWpEuk88N)xvV{H#JzQBO!i@}H`eDxD;KEY#oc6XUE?H!Ij*qO zbHcA(xZ@Mr-c-vG!+B9YxE2>}B$PeD&)aM{k0>J#z6s)WTa9nYuoTwMA&Y2m@H8*- zD7hSu8X5}THknWpO-E)wmjL}vN0NMyJumVH!SNs?Q)Ch?1SDQN7_$_ImgMi_KI6H= z#o@)+@DLmrji7I5+V~hDAb{j)5VL5#SWUH}lP3V~HPRV3?`Q6qkC!|>I6TD1L5Dih zV8Cv`PTFDerlUg~8NbdfT)HEH81t%ARi@Vs{C#9IUSkg>GqiJFtGs#MI(~>4 zoT_@tuKvYx8QUL+aIQ?{%dA>kFxlm~Vc&9@lkNI{mN6_ChfJA&S$3wA=LXw8>@{7; z577Zq0UHGVq%OcyG z70>NngUe9!)uATLdDA6|0|za;EuTItgUKl@b(82pINok;P(ebG9rJC)x#otEKXc$+ z9CX0p6)TBX7iHh>0rmDGCeJU8I7Ff9_!tWZWGRd z8<^LfCd-=fx}6gZ#Ba^35}~4_fqo;mHOGB^fB*61T^emxv;U+qFrB3TPVTPXXczj~ zp_GB^(OoOociB_c1fY2FD{m`PuGP|>c-gowr|=$}MO(dT*|;U(o!>a*cuKZ` zYszy67Q#v&d%>9~Q{8Q+#OBFKeFE>88c|4Gfa=U0lt=r1p!MnQMOhxjPr^Nf9MSiD zPsI(jh+y_Jc}WCsK#1v$AXM1Bf!6>wZ(wyAkRpT<>-9;1`Zv&vVW$j-b_ItRcS7g$ z`naJLgcUNlqRIcd+wRMjb6w;?l>~p%y9e@7w+{ifxT+v!PEP*2U>rz?5jbq__)zj^ zKIr8rfjn@r;$;*^Qm;u}3i-|2CV!<6!h=w6DEH$wnXtKYo8{)}|E=1_TMlb-XjDyx z3ZA{^crQJnw}5j2d<+RO`mE2^Jhf95ED>Po z7__Ka1G~OiWy;fCKi;p<34+oI?C_w$s@1w5H&o^PuKJ@7e-=nIzT4G;*Zm#!w(^>2zYV^;=VGbqBX>6F`nO3_Rs0h3`I-1oKLBMY#=#)66&%=nI30vsK2 zt6f*!bndXrw&QivY1FoLgB2{Sr~AON;*hXTA>v314wG8?!5HDBfhxQAu{JG`1eY@& z|6*wT!||aj$UlG%0D?^q1CEP(4KP!Zv%2Zk1U(Q?ae`HCNAyV&BU56!$p2ugmpuY1 zFm%i|WkX6lQ>z3ODO!rA+#inm*A&TcIC=1F`b`~q_1{(9z-ez6!wf?GVwHDrqnUCW z6CNhLmHCUE?BlOu&aw5@hOSk#y@QZioc$T43vTj2Udy zU_B+xRI)5l5HV#37CBg(*y!xLmWtoMG7bn=8 zIYEnN;IhI8$Hu8?>YIgrJJ#YGw+EAd@16@PG9d=L-+$7CpkN>?Ng*(+Qh+2J5legJ zqzv87+CoDC6!!b%im|CjoUfo#w8imxW1WU4aBaPML^dww*7+@zYf3^cwi$ar;AM$8 z3d@jG>ItgC$pl%bNL22QPLviks7AVK){Z${ccRzX+N6Ci%)+!#M?WdHTVvpmdBuJ) zUl%_+znQY(R9k~$LUBfeKD^B|>=&J#fl4~}ZvNGv)xeellDUh>Y9SAUvU+L&vA+WG zvlGlErP}3}p|kjqtJl`@oG^8athiUh`?n;ugc=dDH%0JWFCe71#p`}vH9fM&G4NOn zk)p##!gE|8uM4>+?*cJ36+sSgyb=W*utcs{fqKW5`|_a|MAt&;v?`KPBLZcLT9((w z15nDrsEVH8=H6+u)(?FkuVr_ecI+HO|AcS{0Eh_GFeC;9l0r|GsjqqwN&$S*6hs+4 z-guvXQtx-pH>dpP)>j}0wF8#??sn6aFR$>GkObxvpN+B!MgEvg&P#H zQu%80FhMt~o|=TOq1)ywe0Ah_^#vFw$RG4}SZEKm9MAWIp>G)6a|j3pxVRjLjnurhP7rL*`0623KGhHAKm)A;tT`4N z_t)OKt6b&fD%bQnGR`~|xLva_^u+z+3Ox_ z4b(>$Re>`RzQ(m1kyBCq&t#J4t_zkwLkH3sP{Y7h2N>%I z=q-BR#GqpM2jsLBv5}c(0o@B|0Cv1;WsfTK*iH-$$L`#6?j~T7MHdwDgQF!)?w)^K z0N$Wwq@{)gJ{Iig*n(OOifQ|?*yMlwAjxS-7GlS+tiIW#>|;Wr{O;z77bn@6+oy(K zQs8Y+{b0SXQKH#LEMWK$q;+ICI%ONpLx@>tAaM(p{~#cq_AIEr5#?fWcXdrbpNSi9 zk5JKaM#Sn8^jm10+o5WSdoN5=qPaMy^q(*G8uv|t^hCe&0FFEz=>=xk}zfThB z1PQ>1Q}^uryw(roZBtXajQ8(@l-}IkJxRS7a{Rzi{d3pj1wRFVo4`&7{ITtA2P8NQ zE?OP%a}UA~F$5Up94pVAU;q;$f;UJ*;00)W8(4Qe&u118>mURQj(+uM18BffN$JNW z(_<(bT*(Ve{M6UxOii_&F&*eRl?+b<@K_$b8ZCHzglW#j1k zqLgL9p8HdNL{&j;U`yp?!coTg>z)eL-rq@&yvFZZuJN(smJA-!IE2`XH~(G z6P_%Ai7*S#Bw>1ar!2`bHXBw`1K~fxY7^ICL?r#pRN%9iafR~x>);m9QL8Kb-HCF=JGQ5KevqVay6!eq6@oJjZpxQmJ$7MjG8{R z@40Cy-Ss#ftk|BLfxNbd2DwtOepsmz{}9<_uA6kTXB4g+^oRpMQ}7)0KDj!yKX(`j zRJeq4hHP^+3&-O@fFiZ@0R21=_JSF=(^v68%q#!S`NN$p*2s#JvnNAeV1xx%F$(}O zEW6U-*|Xz8&jW;#Zd1lowVwO(1aKn-rCftyW|Bli9w!%J=d$~f`*dmqy^gTJ`-*AHn*K3jI&-#NY(f12W zt}06P+cjkrfH$o;C(h@@x_vdruFFfZFT>~k~j70&);+sIjQW_quR zwKjCq+va|bhgOUh}9b#AB^=g4` zP@3iGC#X4yg4hMLdKZ#U&;%5FpaTCFdjXHS3Wx*8ZHUtczf9NKPC;|~fCvjd?}mK5 z2c$crVunZntom8~j3->-K(EVyM6@DtsbTkGOkj;axwSQ{AB+bIJTNXm;YH*CcC!s} zPz>;Ihax7=lXeV|cDNYU6{MhN`uRip`(Kz1XMm9*8ba2q?54Q2%8dp=>@W;+oB`;| z;I6lnVivOq-yuh$ZeG*i!4uHYzeiLtjRCszK1LzW~KsRC-soXxt)1X;=S9# znAKBVNRJK1aI@{^{U-?TE|aDi02hlwqD{{v zNLu8q_DpBYb>!k%zfT2D@)B=bYdi9EYx1)btbeF0gRiW~%TZD*Gjnw61bfI3JPD=< zx-W&DApJG9V`hCPy-%)UAuXV9Ef{+0B;01NpjlEIQwbj_oj*kc3)B(2;)&c5>j?XV zXNNi(8;t`yR=jC=gIbCpq6lW~H{I-tf)N#6*4KedeMxxz+x$Zi`#?fCiqFN%$EUHF z!j!~J7iS~7W7`G-YLaDW((}%@njVQcGx@-h**zOK#xAS>z zyoEH(m%D!1Db=hFH5!y5Z-<21+pX!Zri}4;(J~s~N0q7`C#bG`=fx_mW^7xsa2W2Y z1^OUtGqtg0z(fjC*fy#YkC^KL04DD1YVFpDlpgqF<5=FhPjbW!McB16Ey*8xq#B6B z|Ak(B_fOKzErGhh$UIOnssq1E?c-Q*9uvH}4*+O2K=LphTSahIB)Cev1EZB2*7QgQB<*J&GWE>7y!v)a_2Al-@29rBoG9* zLnVP;g_YdT;0qxb90&l$q7KL0v5s ziT*3rX|c@Y-GQKKSh@NOX_W(Bri*h}-+LHKoyL zKmx1lovsLCmM&;sX+JjaeT`Yu;^=~LdpE@A(cKpO!--!(ijwTDD7USg6{(?!7b$}5@2Nqi{uKX-3`YaNt zQ^96iqQNZVX<_v0;0oQnn0yPL45Cv%3M)U5rG00E5DZZ<=7w3WxSgT_n zfPJ{>=wuBEYo!PE!;xU<$ZK18{gk$@sw2;P6uTzF;K?rNVp8pr` z1*nE}34=Gq5Imh=2K9g2Ji}0}QU+wsv{8%xx*-BwwR^vbwNa{V#M&PFB>bjW}&}sTZi`T zLD2Mq!nPBw@G-Rn6ZSFtcgYa1d57NwL*6hWw$_D$ zYbZq0_`4nk-baxOL?ope*RP@O0^mVx7O?tycjZZzG)Tv#ste|i4PR;)T%3^l;RDR; z1CUVZ$Z#S$Zwo3J`29PCQWSyhr$_4N=iYP84hf$wFNrkBW&YT(gvHft7yUvYlWww2 z2Ep<`e;)R-Wz&!j4>Jt*^qPK2=K{Z=#3DszLaG4*j&kw`pK2R2K;oJExAX3BDY_*V z*Sy}0Kasp>Th=e#2NxTH|YGU{d z5OPPr2vX_YsG#p_Na5~=RUiW_189*yU~Lz@7&U7E2v|4N1-6VQMMtNILo5QAgYw9y zp6|3>WsAHXDwV9%AwrVPYB$lt2%qqaH>td}(49S5o9RB}n?DFVS0$3R^H}xV6>Yn0 zIXW(PY?ep`0ySpNhDM9OV;*3 z3Q(|6!*KmhIuZHLL<%t0o*Wz>W4A4o+t0+|Q7M8@t;Z7-*KuE6BeEMM5G^?ur4im? z(dW#m@dOtM05E;@DORXrt@|Se1R~osjrP&>FSXQX@vN{-ml$L;rWcY)c#f~=8sj-1 z0{DxP@XlfT)ITpck(WH#@YF8dAdDzuSO)0gm7Yxlp3<1R7kR8}(1tOOrvYPlCND~H zr3A)Hg06ST(sAl7o^WI+%GA)ygT>2!Ipd`gX3w;vcj3L=x(NVd9FRz)3#MCn z>E%h_8x!nm)|q9I4EGmSuJaZ}?iPnJ4N!K1BDzjGc+S?ZSC$2NEt6|G&y4t;h0qlDgKutmg@V!YclV*Flg(3!!n2+fJ*5{A}yDHtq*7j&rn~@cN0D z`?iOqHkDvdP$uNPbk|Yu=+!-_B~R^UK3OE+L;)k&5}TuEBwYMV%m2mTrz$-e+mI02 zZKikde9w?I`A%fv#BZS_La&7Z5Hw!+nAd}$%?J<^yEsp7yWi@fe7>K!FgrrTH`{2o zW+;un^vh9G`r~_}d}YF_hTW{flbr>=F1U+vP&l+DQR0L_)V2m}=5`S}pJr;3qb8B` zNHBj;2MvT91#2IAfbwVwpcaJCA~kQ>I}*Zwb9}mutU?~Y@!inKCO{7opY`4!Je$_N z26YdH{!3`oC;kvZ2sYG6<9{349P0ybo)mTW_J#qMZwYQ~Ufn(8Hnm+4O2G6%tMusc zK8Rl>pc&ak7*)nB<@fZ;l1`RL+n8|BgI+WmEP-N|6!|VMI&ax;W%)WJoOH6iZ2sL! z<#`~#+oQijR8Vy`uvhQb{B-s-IkaX%aV6s9w4CIqlqsTNe;%Wo+R^9*p$a{LY1_Xe zLdFF74CpMDfCxrnS*?pKddj=jTQ~l|fX_a_2Y5bEC4%uFKq(7FfVIMHJ|zQq97`~o zC4!d@ZRT%sdKb9LYyCl}JHj|n9{>c1RX)Mu_3BkOjPim%Fi|cUwW@CSx{^u|Z<-3? zal!EogDO)2f(c)G>7@tY>vu1wBTrh|%E_ebJIOy; z!>_YQ5B9s`$kRZ2%QMk1=Hqop z*E4T3Chk*uf-t6()@7<6fEI|0GUnyM0=G#1V{}$VeWnQ0V&+29!2~>sfiLlS41IKo zef9R)DKRDD_;JB-y8=0rTJewWFJ&5OoaYiln31Vz(a#-#VXh3O&fVD`^2*J`5-mm= zF1OL(;O5ni^U#CKp7CEhN z`Pt2-@kgX5_%aRgnl}dvH&iPoMLwt#0-B-*C17ichnFplZCvfyr8itvK`+1k*M~Ov zn*tuvBZ2C>wz<$MK_TGF$kEgEojwU)@q;IeX?iQ00pB?mhGk}FR_-Q z*1(q<|2yO1;%v#l`j=+bu-<$KZ~kL4yY#fJ6l`6!uGMgIc&*b4c$m3Hn}Ad^7nmTS z6uhe(_A8GIW_Qy+dY9bN`-p4YD&UaD6a%sd?$7^CpD5$|-hq(`A4UoJWE2!*na^C^ zAHxYaa+Gs!C3>Ff0i8!<@I9DBt*ir4f9t1FaVqg(dQ0BnZ;aqKUnP1U(F%K}seYXa zKKQG)WxrBrE$kfirjGS&=zWj$=FlVTmFsT{-(;OMsP5e*ndVW?JGH7;CCIdpNF2CF zw>xubNQd$hZa?o}vc1&2&w+Q}hk0jtE8arf`v9?nC9&Mc46C{X`0S{z)&4=@)dVuDP#ac)W=CVm=OE8N%TTla`uIl zh8;z6*C1|sbCjO!Cp ztWSw*2M~5Ai%M z%3!?I%OEq^(#}o^0T=@g8>?-B10kRy#SOD`oEe5ULv_QM-mNsg2i3OSV@gHroWsJ$ zqO6yD-xj(whDA7n@s{fSJ}C=TJGspGUb=1?E}h+VB#6kvg!WqHYHec4s_j#5nYkle zwkIymUeZzAj;wFlvd=3mRYcSm#4zq5isZPQ{dd^DFlYjATvM`E{ftW**1oz+j_8QE$%`RC3E)_ags@60?fy`!Sm=UjlEd-sho329#JH(t&dbGS zROWGkvO=l+ad6K6SbYF6^etx%W^FKsOdlZ`K0c^mpA+}Ph4iBQ5jjR}lfgY&C7FA1 z9$3H=t)Cg?65W>~B)>CABKZ$Bt&9XxpPUPZ99U#I+3&pMiF;;E14ER8iJeSAEx_q~ z=aStg>VH~wO4pBm9>!1&vG7$L^L4x->LGe9PueF!dRA12?HRv<*pV@zGQX_D8*K@#KBgFm9}Bc#?HQj+ zVVi_&>!Qo74NMgOfHz3~1*JUaBq@Cyuu@)u%isUZ$dGqTMKK;is7_i8^*~GKkBa8& z$cF-6U0T02ME#$7Y;ELZUPk66R& zTy&nSb<_Vi*83|GgjyQq6Uas%1oS}1^MuqOpj)_|eARCK4AS^Mkm|tg#e0Rt7*z7W z_OisJWlHY99wZJUV*457!6b-LkX;*>hLJkeInCZ^w)pt;T7^d?1@RY?UD|(#!eO7Q zQ;wP`EdF3AZI(;U|CN99)kZ)6#y)CW=pHZF>DB9}ytt^3$o7`0Lu9aUNR-Ym$QX`f zrB-QcqaXRN{N@M3k6NMccV2c_Z%*EB2#6Nt-j}iP@=62b*F!>L{svWI^N2w@>T)L>AWDoB)@xi8PT31u31P}SXgw)(Y_PZ;Xkvr zdXM@P6O!kV2|A+nfUm*cda}gpvxcl5;j1!$W)n(W`pf2JN+AFM!9)j1W?OzrU3&ZC zx2&kCtLs~swP(OyVMz4KyV;1NuyIXRa0xOdtf0e1awGltD^Vl5L3$h)fDxAY+Cp&h!Xp_3DSV3Wc|`gJI^ zZ%WW@krcrlAGyu_zYEzW!@@tR>>4ykQ5ybbVN^`n0aNG5cXaz={YTTJ?VR4xtXbL7 zsoSv+?lazW&ta||@c+D6Yr$P|fBg8;cs^t_*9MbKerf)kdR6j1Md-xlGJ-aNJ1t%M zV7x*BKR64G|J^*hA4k7#VH+Umh61m{0)|exdS!cvh_yI0f1la;hpv?qWL+(~Hz9#{ z?WBMSD4_bj`5GcRe^YynZZ+ySQB#@`oR0cc?Sn_10b`=km~*%Rq~Go%Uboa@&Cz9#fX_13c$r}ZvJyL9&~-TpQUY)s_^IG1nwhrP zlLmEVKl-a$>ik_z5=du)zzC*qm-pO&k1AG94ckLj7UZaq`mfz!B>)UlZVO7(Br+0Tuf5KnK>iTu~yL)V{?n0NDk~ z8WQ6Lk?A%n;|Q*L(exll8CFEpd?Q`dmu zTna#Cbp`^JN>g(&>+dlxfj8IBJ zzDv)7V3MpgXZXMlTEGpkRnooM!&`$lyb%YNfaWvdOtzm+kGby9WDUSy+3rk9RD$Yd zBG4X|%Dm~)>HD5=#sT+o(IYKRdS?&=U9rJ@pM73_-$&fO3v=GlPbKSCbt-dJBuhQ0I;jho30$Jwn=KEt$NI}=IQp2{*8(FwW{QaTGJ6rltxhPvXWpUbNq&-2$mD5R$qX0Abxq$Bg)M$^!c8j zU;I5>QB40^!y+n`28(6qJ156`TgO8F^+nit6cr2eNDalj@$Z9fqk*jpR8WYgnR1D( z<^SW(Yfo9kAWz}@cjoVrtL+w*_b;y7!Y5^NY~knBo<#m|(@y{ok8_7~@*ZRdCL%LK zkpDJ&5gw{k>0o$0aNj^d@ka)7{FzT+&KOZZ6){xWbzfxnCbaM;V?`p}Q1VliWMU8W zyMRusSUP?dI)5VIh%n+LgL&07LjB{NzKVfn;7E&b&h`W7-!D#DD@AfwODaqN-p(fj&-n4%-1E$5OSZ z3i5al>Ibc=8RGhim+Z?%+WGFACNn?9A*+~TC1N4=LMuU&{HXJJ%Ng#J5Vr$JgI|#h z;P%pryb(GmQ#MbN$p)}D?~YKCZuJnmRuJ3ch0ZB}pxuslaqFli^Vp6x9Gpymi3CYJ z(BI1VAc;)3+Ej;did3VV4+1JC%?_EM?UoYgM>^8OaXWd?m<*6aCM5z<8`{0;J87_P z0H-FKJS1Kqtv*ZAI`)}$9X<4lKB&~_K>}B(JApx;n@=3M?@WIjUD`B#gouZsqVr4y zz=Z+jY1e|dKiHXM@V^>V$OVAlu+pG=z2YHctD4iiyHPQ4FBqm1vX~(qtuzsM5HmUe zC@prC^dKqIAx|?JqHlqiU@UYfqLhz^AIP=w>9ZD{k5HW5n-Y67JVgEG`p54~NvOFP zfotJyshd}QdZZL{B%mBrOAz&A$}3&Q1Qx(~L4Z)!5H}~O98DiL;C1>}MefC?3c07> zf6VxmG5o@>X#`7t(^$3o!6gkOLq38%^QKMASJn{WF{ z+d<{O=b}3GT5i6Wva@Dbg;VDfX&}Zblh!SmEA!e~`}_!Eq3w#LSY02I^1zZ~`O1YTt)=!^A#e9+y$6YbiAKw&@u7>9^efExoI1lj5- zgYR5hSQoIfHG|u^IV6o`K;EHTeAlRo=pcYMMUa*ClgBGW?5zhgJm&X?9jTTxLn&bn z`c9Q~cfck6@oI19bk?I`m!1|}Je(6ei3;EBmWK7CR~336n(eRnZ0)H|&w;W2`LBs+ zXVAZQ4^;|tiupFt^z*CRRcCC~aaI$^+#1$8WcklBLv70ET(0hAcz!851r6OG z>^r&<^&r@X&o-V+U&R?RqnLnK0|`z?jtkIc`yeeS7813(3qr>|=<~5atqoeiuMp4< zF6nY)EMm=@&JVT!Ky4>55fK^jTf&6}D5W;FVXl#60r?9-N*f1QOrNeAT7Di{CbTZ0tEU;|6}Aa+1;fI!3aueqpmxZ&>ZxTUU>( zmmM|-=_K5-yEDhWT;Ic8YCC*$gN?XwtybFpsy8UJOv}nFS_J3_YF%=WKZrU)cR> zodHWFMva+f6_S%1eDB$CZR#BTmuZS<>ZX;^H7xo9xe4ruHs4tP6a~XQI z*G+RRmII~H0KGu{rXMGLYmq4z`eIm4gh^c^ed_rxKU(BqNJ-VwcB_-lABQpRWO+O6 z*Y{{pgj9M5uWml3tIsCY# zdpn;0YR1hHK2~9M1q8tu{vv+g6yxZmBZz!@py3E=ZWj70gd5o%2s-?#a%F|MCBD!w zdMp&2&YaRxmX&vMPEx!1n<{%ez2^(4x?Z(E+ZTM$dMnjuSy>)a?gikYM3H*;=f6o- z#;9Mm>1DPQOES9Ax?+q%icS;MytwlwQ5mCYRqBmjJ*W5bpqM4)Ns=ro`!&zGM>!@e zy_Iy&KKuJgX!)W}DzCX>;9~CdXYEnQVj-!`mcv=mg4+#*u|oCeJFO@8g|3Ddt!-^P zn?3g{e9qmGWy7R*4uw5Bm4=F@scH*FlfEYtk-s|XhkpUx2fN>`e%&b5`DM}nVm8P8 zeiWaW$6`L2Lux^Q`u}6;D}bVUzrUA8x+IqdN$KuxDUp(v6zT5nmX{c6`9AymzcbF@I4kU4?mf>rpE?N6Vq8JbklNCn=d)Sa4PhI#)hg`sJsX_6)P`qw zfb{qc52b={?NH6+h){bWtE!6mLM;mdy2x+CZ!mrzT*4J1GF+rRD{X%8^u3EZlI*JXDwfzPEM%;NTQ+&YM8|3lKs!Af`J7e=RC3garK?GS#-$So?TNVeoGXz8e7SL;+g#-yuk7 z@V#i4b;^o-K)Kj$`otO(8oLfhXzs%WhsqE0EP zyL25Fq3S%S{4P0<@eMsK9lJq0>ULQ#dQ`WN=N}q&lXh}DJMvs^;q}JjKa*kR23wXk z<~LQH=!$gu^}gaNadBMUQmJ!J%YBu$72g!3O*jM`=Mz(5iAVgA2zC7F{bNJD;m-)U zY(}Ace0*!?W0LUYP2m1IV;fxw8VgECS5t%l42PV8!tQr40ue^U?Lsb5R1A>iqz(=a zRx~wbM@NH3eS3EoSXpy!=W13-fNC6)+-tFC|62P3;pNMh$VL5B|NPra57}W_ZaIni z8L72yP{l)E-aRz|Wk37!l;(KdjTX~$F%93;1etYa5AyF89s0RN88}ENvgQMi>q=?4 zj?^fjv^B!Z7Jf8DFMf^4bNno5eJq??`5~CPc%V_UNR5>`u*2|cL|INQ#Wwz`)|~)$ z`Y{mvK>P9#wdDuj1swN31Jqb4Q-|pn*SBHQKX_w)8dFgV8=36$=rgzxVafN~dL)C# z?=N*+@0X^^IRB8s(l(oX#7-_UcF*cObA$J@+0o|u`+WM*n?f%6%~-@3kzHHP&jj{I z1kTW8HiUmc74HmNn91XqMl^82TmJOcV}q2qB`%&AZQhtlxkFP(Mm;GhbRB&@mujN^ zKlPHv;t9VasrvF%7XXX=%M@#oSz zzUpQAl`!I*xX>c>y;lMgx_;M2l<~bs#W;D{q>+qtbl{>1l#A{KavMS|tL9kn)S#;e zxXB}XpM|Uyr6J_DvG2gY+%u6w>%t*oSH=Qg14@bALpG*3%)^-b9j8) zku2T%<38EcCT8?8cK+^{$|1Q&M+&6QD(|F=xr2kt{F&_UTW3nEzt9X9pF#3G{xgWm z{Yk{44QkdFFywxzs!9e^km%PRKR#~{WnR}QvuIyD6Uj?FJ3H%{y}LdxkByC$sAZOq z?M@ooC7h)#p7=w1=WKi)ec&I_ctsLp9~2UOFvEHD!gGkR=cp~O@d1~hN)kEm{}gMU@RTxBLr%Opalx2(bv$MXuq`_GXuM=b$n#vLBrIa#J^chf(iqp2egTo&{`V&He%&>lP0a6o!{Bf2g; zd9<@vEm&gda^}7|q_rQI`+UANr!2}u^y2aY&8PXgtU`!*2wz-;Q$?eLggz>&Ji?xI zp*gdRL^4t9WISp0qBmCPP>4qS&j{`hdf;7xjDz*D96z#4g<9c8y0q+UJwiK1_s8T5y90`uqrm0oSeMhlHk9~`#?$XNItEP9$XIsl`;!2?>SQd*4qZh zGU+hXi9+cxTcA4bNb%@^ew$H=h={lV)d_#7MzT#d_9+%@PSMuV!gm19xM-z|ZG8O0 z#fBsmRnALdfKwrF)R;>E`JP=7clJA$?l9c_Cz+BRLL@g0ULOKVu~ z+a|P9T=uO3ER(^&#I)NT&2zt;(MttM)S#_@I#U)ICVKvS0+h-?-Brm}L~2 zbI3eA^y+JaV|7b%PD#WpwVaim*0mV^!lm`JZib9aO;JrPI@l=6KGb=1R#>5#oXt5r z7Sekp7pN|9--$OEiy3~-wWy_Q1R0SYL>KSbz!JeG2IomJmn0fT5;d|^QE~~hq~2eW zgx>Wl)a1NugbvB@jZh=dkK_$R*7EUVoV$JO+YzFoM8$m1HdLt7KVer@Oe09q)K+sE zgN_W7~uU$sP2=x9;r-0bWXs2*iaWKOVeT8oc;IF3Gp91y^|J@_30o$(19JAs5hX zg~6z$>|7-Na&QVI%4$hFP6oD1p|2|uG4{$RbPOFtsgxlOoiZPKbRdVCGpD;sR$#yBcp(28H;F66= zWe8$u+IBvi*g$xgBPEc2{FBh^VUB+tAWS5i4{6x-Ox}AJEY#0KQAqkI(pb+VeUy+a z_(0v?K)GeeMrE2Iia4lMP(1F8bgKt#PFtbIZ&^-6qw0x*bj4j1yd{YqrWQ7iu-^%sF ze0S$Q!^CLl+2+ZJAZ8Kh%>d@O0f^uGy3a=y+du{fc6wiO4(A9Sd!-jWxK(`r-UF)j zDOfzQ4)BUzKn-iaQNt{WfaZ#Nct|^3s`m%@Snb`O#=(|Kc#rB05V@`E0I z;wc1ifc-(j^JpoC`(59CU&iBgT3Xr~z%e_4pgIy<4w^Dj7g8WYhIZ<2=ykBUzFMA`^k18L4EtH6TmChvw`l0$ zpIFVsGd6{WpYAEtNCnBEF!OaUHkYMpS@T*U=e}AuXJF-711Cry=);`>@Vp7CBr^Vk z;ncrh!uv!J-z-5VDiEh{*y8D6n)7jkga8Y54``dq!JL7CN=OEjpy|L`*Q-=aq^%H7 zDy?T4;9;7RQk-<++g7T{3@))6!Apm#Io#~!O2NenS(}Vn14_q*sY%-T)dNg)vdK1{ zotWsWhJ*fnXVnG$jIy?uc@|f$xKP;Z%n6lF=kV`|a-OxzdnuNK*T1=BbZ;L$DP5qP zmP`j~AF;Nm)O6A6JhDG7Shz$QiX?pW%-FVdve(<-hiT=EDk;Q;(M@EPu7==<^EVtY z+<^x2P=uFkctBtg>WC3ZH$B6Kna@Ll zFchPsL59Zb^3L384VxO}4?8~K9JD9=wc+>9qj}))Jnnq=6s3O?6K_>qU5sNG7aWi8 zOmpPKgRM{)!^39Kd=uxbs*-0(XcReTkPULn~ zLX(E*vl+9@o$oSR+q3Yl=}B`u2XoSl?>KrN4W${#MZqfmTHj|Q@p0q1eVs}ZZKTzAkW%x$v@#iH`67TdELDgw?aLsOnf1itaTs0qYBx(5L zx-5T16L*F(HI-93I!)Ha^pY*iPAX$#W2a?iW)A+R1fM=j(oiso!E-_0cG^dwqpkfh zEhLMsccKSda7n6RHL&I87K{z$0KBijeFVJcYkMC>UXDSm(W) zu4s?)t?$w;?-LU}4bQVa61k3pM>5d+MPU*kLiwE6q`)9j9QF7>EPnDnUf^$<1W1S! zd-xZEj?yhAeiEC!J$5d~8AMQ`PN>l!Nuo?HNY2!JTZGQ7Hu-{%&iC#>X!%k`hPzN6 zHW63QV>`u>Vc}A&CmA>Kd6~U(;gc9-j8Oc+B+c*=`fzB9f9okOtI@$Ew(7Xht39^4 z%{wMW&&m(tfz~rMv@xEYdgr3zQmKLVBgizzRsRNTg8;o(Dr6jZQ#tERK@PtAggi;i z!E2UBKllVmF5>5d=)=OlqRBv z^Tl%@(TqGF)RUMQS1G_G;b08XRVDF?cLegbhS&%aQYQ8b)x#ougUZN;JtwKVo~``g z#UFA3H+XXGLA)+_%q?qPe0UZ~?_SvLe1Zq&qC??vWzn3)p0Qw@w(VqTEUJP6PyPnN z#e-j6M`y{i)P-#?2Syj+b;vY-*KRBX$F;GQ2uGDV(BlQ02PwPz$+dUX3cE|>)f#3v zc#1XOW@=f)4eZk-?ka<)9lBwxz!pKz{eBixeb!b?-LpLN)7zGwK=lUBJ#tF5s1-Sf zkTX|J3QL*;Vfr-8Ve`@)&-RE%-ZrY#x-Op7y;#WGO1`Q^TC`^wQ@N zeX-NvRq)kYtjQ>3mYU{Uw|w3*%+f$TZE!Z{15qM6L73Rs)Q_F3QfjS3k=NS4m2oTc z6kPr83HAaqHY>`ylNn&EvbpAVfDhZ(SPrFP`pzwhxZJ2zH!$#e)#vebNlo)Gn6)JQMEk`diUa681zTa zFOouL2lthnV21X=u6p%W+KG`>Vx9t7O&Ux^1WjjbZRLZ zN>u}++Bj2|GE^JE%23KJ3jg_OjIBqDu*yH)qy}Y@*w;$ErQzNJM z$g5U#w4LZQ`|Rxg=@UtrZ0`rtJKJ!q|76kJf~x7%g93zP6yAE(geH`~sta z{;g_g#>(cZET*ZuERFEFF%FCLjpV}EuSH>=bfHTkU~@MML{F_#R;6a>^|>1UM8u4Hpw zy>0P)xxKX&eccAzED8-p^aah<ba`3Z*%D0UQEcE_BZTRobda_j6*3l7lsOEn& z-3AC%kal$BaBSKp77~}JVvnkEG`OdNa!_|oR4|9&qSxG5w_qmJJx?AWsPV>Wz=aMK zeQnieM_ao>!AtDZG%#c)oBiNcR9uUGDDq2Za>-Iw-SdTi2+g1>@>}nG#qyQ-z6Fc{ zQZf8yHC@|j6|5sYv?e2*xNht^m23tr>Nwu<9Lhhdn%MH!Hu5y2sHee0o3LQiFFiXU zI7simsr0tL$e|JBOdUwh&wp{nYd}d8s)<;XDN{JJj*4IO)reJ+I*Auuv7|~=yn>Yz zAJ!l&-ajxHKD8}xr$v>v-0TY~(V1I~nMb3ho&jp4mAP9Zm^3s^jd9AfL(G^Fe@X^7 zhixtUmGI6>Ay4b2hinSf9t=nKSyX&UR&me%uCAj;RATI;&~eM!-#o@R%2n}&S$Jy> z6qdA1BAJ|Q98;QTDX#F(VL#Hr*tB?{=;6aX%_{wQ#jLudlm?eIz1B#Yi%7ca?fZ_; zHQ!rmL$Dbme5!<(`$s0|#9t+oewfPUDy%kirQVMhVBT^~Bidn{e_tb#T#yst%oq7l zM%uSv)4SH4v~Z$=!Sx;t#c>>ADDTq=ta?$Lef`I+KLa?^(#pEN; z9*tnmlhTsr@XoL5cojw3U0nZFP5bD#g_}&Jc8RHR`kZyG5ctiZf1679JB(HRU6y?` z*vf#59Wz2E@q##BwbE9UHkz1D9H}8gq ztyq!_@d_kL&*6QQY1Pm75m7=HFNXqdF1EmJI5;`14pft+WDq7RJdU z!IH17=y4>R3U+oZ;HJwc{ipsv!qp2@-7Dw1-;jC1_Fw3rS_8^tv2}K%5tUKsJt! zkiHe=HWIe32Luagg0GLp5V$MnExRXthm|*20>Kq{?>*dn#=7)~n4C@9e|&A`L11#r zkZ9wm4x=1XJRN>2qov25o9#s6F|+55>M$;jyrw<>O9(;6&-a z`KLe3=_+vJfquKcFqQ8gS}dp#UWyVj0K~XJ`pA!Vfn(eOuoocQygPEeG*w+?;c6`C zkq|_4(6&F`+P8vOSg=GRigZ1x8_bFcMY2&AJHy!l4ujPB@CpM@Jpv>nlU;#`2P^0` zojZpOlBFXD%5%e8jm#Y+2@=*va3y;GX78#>WFmIrW&ALV`Wnjv>WvnCr1|FcVkcn~ zcyTnqX*eyqGY-eCtv7OXq6eL})NiXd} zR~7r=dq<`nyoq9kxI6$dahEKSplgGH%AE>pXtMo3j+haEpo12M?4@TcHG|~p58jl9 zO!wCAZN?-{slKqS%Sum@I#2S{ty%GQ2!PNT9P?j3+DVMY}koCdvCTq7##><^=*lO)zTso!x(<_|sL zJw6Dxfx521n>D)Llo61bsPt`S5-37IFiG;vr|%hQU4FZK0_@Pwg-1}ADO8=I+j=ldF8)w(8~ze^DYTJ{o*+Dyi`k7Ku6(Bi zvXzdSv|N2l-zi$id5MvfDXv6MZaOJ`c{MiiUjTdM99{hLL_7fLv(DT6BhIKld}jw* zv9y#FX@IN;-F(Cji0NK0{QLLACqBDLYKeDuFp!{`Ej$Z-fQ9(n}S7LJP)z04-PD)XJlv# zv0u{P>jId50uZW!x~AJf@i;eO#hT2{Q?CSRsggE9FWU*uMrW`mb$ug_^N2TxE(QI< zx9#SWl|gMo!;@`L$10qVj7uul{ida-cU`hbK|#TY{}w>muU@_Cxbc<*%QWM`@-#H3 zjt7C_31+yW#vkm2Wai*d-}l=q%e1&YT83xMv$C-Tc`QEmgLeahHtC@_^rE779?s@Pd-A=J&h_ZCvk>s4A&Q(?2hEWRw@Cb^w>vpIGaHB0Cxw@ZMZELBYoM$u zafR9#rubOh#Y2|8zx&=RyXSM_g&VyHZG`GstW^W1!ElDsxcO}}|mMT+= zcc>KsjgOpqy>3>N8R>*KlqlW${4UHKxm#;%*IGI!g`-~aoeSx(=f$_5Y5pcP>}bH$ zT_%%wW^zt~r3l_`nhJ`3hOD~R=lVqwDTPm{k7nrGj1kF0mR$N!&}zw6>co@OS{4hHHeg*rg5br$*4_QI40Rw5pSd(bG)W(&l!{pqAndTt zoeK!$t4bVA=1}iu6dgWZaPD_CNbvG@h;|X4DH2{f-u}12RJxh4B6M#&nIdLZ)rfz{ z;djxy&#?Kely_mLVi(pFlYH-WGM*R7I!PP9VTspeVpYX7uNO{9$U_r{M(Fw_S;QrOxyrzfK(=fkhp2?(kLGh z?U_gRyewkeb{cfB{gc+wDmX>XL_CmrgY1)ABr|=Sb!VO8xpmC&->%TFv2mwhr+G0D z7XSp!Z?d)Qe@GiWMG`Cwm=v(URKspPni~Uvx5Cn`aST}&ZRpD$3vNi4uNp23N-&Wk z&)(i%)Jn1e?l#L}r{gCapLvdcibsxG``98dr`S``-W@?zgDw5CUt7IzyDVJd*=a)&rPC?`~nrvG$ zzaWsB@6zzj(|c2KZLt7}TI@=%Aj%XeXC$)v9c%;WfWzSW3wY$N%D1GbnUcP?p#KW* z@Nsv-!S*Til>Z);I@+YRs~`ZSq^C$ zuto{Ij=U5VHVXXg{QXkYU^qYjCfEJ$88K5*4D8zp$VCBhaqq58DB+19aIcrH>eCDC2a1DU09>J= zrlM@Z5@enp9{{-Pv=&L>2!Q}(j>7)G@142AZYLlpi+c3SgMl;j`qy)arJP(xjd;M| zMsJ+U%o{x*!hP2hjsHLTBb(QTj+4_s{l*Lo>=3D+vtheFAg7QPc$fFw=?G$eJ2u2U z_%f7%&@l))CLORXLRCv93Tydj$G+>ZK9#%{efjd$y%$=c8p{*Zmn@O5v*3m=PnTsb#S%>0^l82<_{_yE* zW)*~eaQNJV(=rrh#vHm4DrU<&)p;&g`+g6`nU;iKx!<-)|)yYG_xgRQf ze2&j<@Z(@wCX+882yM)~e5B+mI5gPRVHPexX=YnT#yqGIKjaF%YS`1#dlNYv-{a#Z zhFj7b!k0jvO=h>j4VdZHM;e}~DZE%Y`G5a3g+i6s6P5TPCyNm)(l92t)#WTYJljxE zgblZLptp*313RG2JMbK2TXXiq$B@@FN=u5b4gZiov>B$xp~y$BC~ZQ__nzOPTF9_P zt>O#2w63+n`SAO{#Cz7EE%U)`?bhHIG`XIyL! zSh~>>Box)x5UE5UhG-&#H`!#6{qc^AC=md}QaLG!#i2%c`0=`EDj4UqnFD6^EzPQ` z{XH({0BiB`VcaI&z>33ZBlCD;HD2*%&4=o4!JTwkpZ*zG%86*Cf_dKakyyM&PrU&b zBLeCpYye(?`iqi(`Fzo;mQFP%1aQSSh_pQxE<|3LdS=2nJ4e+(nnPDjYMD%{nnP=X zma#qlL$Ij&Gl6VR6PdA=glFIs5)6Z9`DH~NA*LNwPpn+bI44`PJ*^+2ZrWn1| zoQQ1^YcWU-0Rn~n;TC@ggNyydgHPAnI|^d&OjjabCvo;oMjT7)iV0U!Qy_Y+A@r+` z^}W0nOJFj+B?82Bw;pCYBxsGcKek*dBM4DS%==-HW>GNN3OA&6l`^N&;*5 z*PkP|-CQ5$xR$^MvPM9qAV7fd1lAKncnWxu%~-Y{LKa9$K*WQJO&(6hYrO`}qkVg~ zM`t6iWsm;;et0l54rC5Y1M1?JIz|D@*h1!i_4#IIi>ljEgV)~b*Z5=8q?+$CHl4HW%#wE zLGU+@79}IhmD_H|tu4!Rbu9)6~^PP1k+~k}zrym+x|zIN8!w zfN|1@pz*vUAPC9tHVw$pDGv`18~^R(oC3;v14Q@20KpinRidyCU!EQiAdZ0%zA#;+ zi4KZ%H#n5TMsft7{$$zN*#Rb6&clO0HZG3aaY+bp^ip4Rii4gWl{7RI!M_gq;o)&! z_Lwza?}~^TzH)MO>;o|L33y-K9|Zn4xcr^}l{Ckh-JM`qsQ@HXK-2#T10po2^=-x* zY-{BPGoJ0Q8m@;WGUd^ru-?S)@$NC_05Iy@IYCc+bSxLZmv*YB=M;Rqac^UQqn<>q zQnqIK*X0qm;R4$KiGh{WbVtz6mW?R6KDM+W+}zU=J>T*aeTp#LdgYN$uqz4C|H}dT z*qEsq%jdaIdwX^SyB(6G#5J&Q5zMlnDfMWiO&!qG&8;BoIS*Gac~fz)GW2ikJXA!+ z${?lZp9Zr2p<~4xl~OyM4Q{`hWz2K8HQY!|T?YShxpyu?;(`5^Qa&!UV0ZuuOmyUo zbxn{G7(k_NS^6z#`rq-z4IYjnIADM%8lhnpw*bh2y2QW-gvNwIdfvHgZQB-Ued~0E zHmeB$MKg5rpOcr*@AFpD=y5RSfM$H-9*BXAQQ2zA_KMF7wuMM5ai-HU!Ry9OTmRhS zhId87NNss1&4DTjpxR{NexvJWvDrsmP(njd?21bESFVr0*wNyaQJRb*_s<9>vX9Pg zK&jVy`^H!45*bWD8Jn7~wJ?$-qoEEjKB(h<9gM|K;yt-RE(;0EdjhhcB4=ceH`|)5 zL}sMK>J21&py7E{H~5W7#ndn@d04e zxw@N5)H0@kc9s?Z%0+dxckhwa0J#FJ@a1Jo|5_no}?@GT& z4O^6;h<#^*NVGh*%Y#Bhzlw0(J!95U@ND~J7U30s;b7;<)G>mI1lmwX1HJWIOl+6r z^jwa=<-_3?J zGnK|MUmX>+vH1hTc@OBWF!02@aL^UoDA&-!!k<%^O|Lin-l498$C9fq{N+U_^gCYa%K8@pK8Qd zwI@>2o3{o_o@J;iC?}`ds3ZlSyWFVzT*gcZ4)CcY*Y~bBgq(^AT$`}`7KM#mn=!^( zsbK|UZ)n?E4$el{M{}6(t!WTV{;?-Y=2a5pJz>sP33+TNdtay1!INp=)yD;7WS0~b ztyhhnro@7%SIQ<0@ylua<9oE=C@Np{=g*)2S?!R7r)snU4|jmWBy}W!9*A+Gkl?G@ zA!b10O)42Ewi5IdkQ?;)FW|%SPgPG(0jj_ob9iQSFVIomN-}Lwy*d#~nlxWPb1RG0W8EKNlB8WtHm~quTn69grj|m%9I&-*{Z8sN z(h?PC;9MK$z&{PS8^>NlW+IGuN`A!Pw-5@v#Ad9QCl&bUQtR#lZ-B+I;y@qpz3BhNV<06%N`<4O+Dg?%KA0Af2q7>1IPfj1OB;~P;w!Giiut>{P-7Y}GCPz+Ji2&a}0+Ynz@&4Zp zaPXfFvi*b)h`>|56M(sEpjdCdJI z(w_|XZ#;bc&Ggkb*gcv9l83f6S<8de;jMlI<@s56X+osI>-!X0^9u{a=j>c0$0d7O(1i75C0v8BfvDn$?zJF{a!|0n(ch>=~3`Z)x z;M=$hK5U`N(Zi>f=V4bS=_Tk#?H*i z5q?|fj__-Q1(}G5M5;tL#M}UM#%aVm-GE^%Qn>j6x?|s8ue3#iqYHWGHM26~*efqu zVm5uUKBxn^rL=hAogXVvOxOBVadWHCUOPhgTCAo}Y~&|bpSZmwA)TNBRQ*Om;3IH~BT9;{E?wzd(`kevs~O-i$ zEQ#*EHd8}GqX+|2odH4IpunT(#O$i{p6LCi_9h*^&it)?>x67Ut`7pb?^GU~EswpC z+Aln-_F1dgF~Jk|aF^%pawKliu?Ea?xwkA^4~|GbgDCtLr;-I%&<#%Ql5KF2R_`^O zCGSzrXf1B&B(ar$Zxz3)*#UlsRF}tP9`Sw1;ZV`^PfY)5kX8BGF_2k+l_*(V&Kc6M)kX3u|CEVcqy*$5C*;dG|i z|M$Xr;NS|QYKGZHPYkFn=#g|4Kh-Nz@$kW;3A#oaL8kQ+PrnBE0YyN$SwVy`{f4pc3jV}dR8)Rc(5iughA`&ucPy=l--pkqm3&1D z5NLB&H!%yp2&~fpiN1aY+$3fmDAOBkZs1PJPP(MSRzDAQ#KS*x`y~%!g=BpIC6V5e z_xd)(roKfE&C@CTBO(^x`?^wf|ICTd!AZKgJSrUz(McaVQU@HMAJ;TYLyIuN$=Jrj zis&SaNn8ce{M4Bc1Z@5&|FOs}Er6ke$6t2h_0aNBqyVomj|LQ0G-NAXFMu>}Afu@; z#*d)10qj7a-6Tznp1kAuNu6S=#xo|)b0wxhpcoLf68aJJ)gp*ALb+raK-11K*~~jv zi6h9W5^Kh_$at;rHuG6sI-}+#M!Z{N>za?rN8{CZQ3LzoAn)Bp`bUM(Zx<+#(z2?? z_4ri_61QNQ1VI3Zg&)_8k0qb7V%#iUqWjL&zC6oeHAg>J+OGktqgga(!qP2t6Mg3S zWYcuc2Y_ODEN>%qr7$BFaRKhbtXfoS=9`*yt~oy77pkDFz9`%f89ZtDk7Hd20-7Y= z3W+}tuqt-&Uw|>dtVX0)n31mbV8r(8^JyaRO#q4lK9yWCHz~U()}9_3WIX$O#Xd)v zGz0Q25TJxAF+0IWi@X8WCMO`~r{gXn>H^U+E?}z?d{+q!c;`Ouh{LO;Hgq2DngjR& zGYfvZs&?lhi#7usWdLT6m+dPpUYvdLWK&VZWlE=ubZo-du5Se9uv3em4Op5lLNB); zj=LN zcCK}Q7s(f15>9>fD~tc-dR5R#9ck>j)eX#Q(#A3`0+xE*$Ms4j%jKpoX za-On!&*yap!1R)0q0^-nXZ|pIPJ$10BF{{J;)mixr0arrL#kn{RlXEay2mk9i-$r7 z-+wtTIy4kDHIadtT{@Q9X0<&4&YMk6O@XmP1NvrP?U&719{|kbj9Wa-Pu4mc*CQzc zQ$kNC`T&J@67+N(1hVmkfIt3pTo#v-e>6x+DLeq|0(AykC5lT*=(kI5v%%6SJp7uP zniJ4_0RzsAew8!6I(g0wwwj!Nl`RWtWbgCn10t6i%5W>(dWF6Hl zmh&L&rgc=j971TEVs{MnK^W3XP3vhB@us|apA$jSLb zP#QSB=@AlP4YYvzv;#p)9^1lBC5PCH!{z z0MG;`zDi?-4ZnmkUIPGwYV#43EQ(-T15HHlJD{Hcs@c@U8Chm1T%}JG)7MjsF~wKYCZQ2m%cdO_SG!f>j%K zIB4m2}>Iqv86zDc$j73xC+Pnzx6?<-?4TL{NtBUP9*E9C`&&nCW zRNX;{P_hvR6HrN1R#tXwe1}qviX5=}f{839ni;yick}wV%x>y)%wmDRpaPa^kBgi} zN2d3W9iL(}(}c+x+{n~E_dD-qKgTpG-!sQOV#F6#Wew@ zr9Ew3`nvmP%Bsul(q?G$;^K~bVR>-!uwhj_2&_@q1Cya(0#x{H_tl!#FxV3!8}xLy zs$8V0N#p#d?jwCR5YYv{P|sE3wHYn_`0-=Az^>OAka|ghvApM8%oxnj?f`oO{?UNY zp=~z6(`W zqGW2s3jq174p{Lk7=KIcuOd<4{;;x(J04XKD8FKI_|hToqIXu6;bA532_>S-d_yR) z-JnLMCSljd_cY;}`MY1|mPwNppx3$8`g1PEEJp-0Ba-j%Jk$9ed5xuTU_iubTUo^o2ZWR@*oBp_12j9h&0 z=hi~?J{@Bl&6q80KJsIizpW0drKe5OqOnO2*-o+>c|0&6_{cwx2EA3@1gmw__1f7j zK30_0qJ6v)U!o=hoH zNk#Hv+^PmR8sD;Kba=SU^JiC=5zkY(|uP%W>t8Ep@F zWQvs1Y?OxWu!=P0{3Y@5fti}TnfIUH)PkLyk21oAbqj;&s)bikeKQtD#7XMtd#|GA zR#G41!2DAxTR&9$P$cFfp5-(I;n1SH1W0`3>T%R;>+>I+`_5{T>)Gt!PF#0JFEYlZ zvA7{#|B+OEJ(LaCj-CL$7!5r=Qq9u3Cf^FOBh0jj^zJ3z5pv6sj#M`Gwz7REha_rr z3j*Y9gGEz)4zY%_vvzV8K-56~6?lC%3g$47_453UPQhKW^uZCQGide@Apae&aRsyz zsGvCSE;0XT;yN{Y_)iZ@h|2rX)6Re@5@p4-r;tl8a?#q}nX7=^YTSIwW$3${L2%1% z-^q<6U;kj5xhYzbEmZSc#3T|uDtca*R@bm-cb1XQ6c@0VfbkgbF=Z9v+KM@QfaO>uG@z&-P;Wz?D*O>8G&@N;x~w^iG__LG+9Ky%Z>!ju>M_sC=Mp_1|hEwtjx*WgauzW#S8@|m3N|8U?L5fXM{ zHNOZCt^tQ#5YR@t*$0ivJ@PD&46dVSFk6@ozj*rUk;5IZ&tO?v{_eD9M(A)lm9ZYq z?&yt*QG@Vb>u=bqdX?R#*2eK1ZTSD%2A`wJnhP0G@$pZqT&;Q)VgriFzsdQTy0{NJ zN9&q@6*S3RzPNTg8lQfAW+pP5vR z!n?I}(w?hOm0s7o7k{bOp6nwmsvHXc`&~`v!WG?%41`~>XxvV|cbqCJ`i#i>Muu+1 z)oK2Up$@C&TP@c7xQPuZX_NXM z>7~UCfjqKwEBSFxG{ZkFLu@*kB&vUZ%^CMAX&{}3|2;a`xI|k&MbUhl?au+iWPZ(r z+&VV-Mp{Mu`&pTGfsHH+a3mnle~v^L9VXcD^ZY)*1cUPM>xQt3Z8 zKcuP!V?beJmdS*0l%O;j7|PLc1ELAg;bfBhYckll15pulyyCBN-vR%4%|}FMv7)-6@Y}CE0FlGP_wU53`4!KHZ_jaf7yF43}FiEb)j*xL~TGG7eAS7~(QhO1_zDs&?>s5xP{4?BQ z4Qb7D6}k6hmUf67+41hp+Hy{zm=#;2@oZREo{T9_@RvyRon3{@iR&SxhX$tl+?awj zp2gqKmXbDX`=1aD>(D?*K2@$s*AHEs%XHVJQ-7icqatYyqX)fj2JWAHjDo7kzlOy= zv|sr^#Snmeo`CXRoPD*+;z1q9%cv|wK-L>IV84y@bCOiy(%lV` zQc|KwNT{TAH%JT8NVgJF|2p^k$Jm465WRTMIeYE(t~uwkVyz}z$Uz`)P{wy?(hZu6 z`+Dsy9{NQ~N8vV_%xOwX&#ffwVt?)6{XtQDn|HabvvJFOlJ14c0-2IJVwxTv;^5N+ zwRsg6(bPr9OFu;Hl27&CRvVn~oV5>s<*nqr=BFEvh$Y%K6oqtg3 z=spsu;eNhT4M}c4&yTl=Aqo$pP%^|mgeFfagqvQ1xY2lRDO&a7lHRs}7Pl6E(&SZ) zwT8#v&;jjYJt}^C?8>@H=jxp)w0FzekMzQ0d89R|gi#UYmVK_!jzr&-MrBpTF^3t& zKKNiQyZwC&i|(>~@Ht#%&IABkMbb+gYd z^%~dQYq55c%h$;-;v#Vz{_$K+r&|7=#fcOrED$1EuP<2aaBY z0ZLwln9FTRTMqS{T{Ceo$i6VWjc;F)A_eU|=;cht84*9ssJH3OiCY}WJjAE-f}|jy zWBr=tlaLFPa!wxq-#J(@Ah8y5SwD?L! zf{7|Jr-qLRzpd4xDt{x#A4?9qH@uS?T%{3fBR?$V(jUdln&Fa4X$_~@-@P}QQzKc^ zzJx$7*TJth_DCGTQ5wW3H>s{!-droQ#$iGB;Odn5Y#~ro*iUeU-g`&CcQ@K>S6iAYt4^ zO)^>_V7N>4`A)#&=7xpiIg}3nbA-Og!VJo8bC5QzAf@+OG2{H_-Jrs9$5;4?yGt#j zLnX;GVf)|Xb!wgv#oY+Kvp={+w484DIRH3^O9Wgt*`=Lq_o%^f6BJ0h=urGrb~7 zJ&XiO&I1(H0%$L7SN1Wn%lujv{zZ+s9|wsYR$5by<;Y!2h!@=pd7UzsR!b!PhZsQO zcyI52Ft_`6T@L*Umti>qS;s-Z(FC!+QRLH54GbqcuJ>lD%ZRwod|>x`?zrtiD;}@K z&#%wdC(F#C;rn>8q2(9&sI+r1OA>Q!CT}+TcVd#gF8=K5vXMJdsH1BV{yRkkDjq3f_{Rha482;=(zQ+RX#9B8;2`)< zcw+ICq^-Sbh?1uz0s!FOX)Ot6cMe*gRpm8S(IDHw8=AC1U(Kqf=y+{RLKMj0_cvE-+%nLyoj~ec3hfqu;+Juv!I}$Tw)f4&=hO-S5k4B`Q%1F z@zriOFi!8v>3dYHPhle9*jJf%@xI@+xQ!roO=sp_RZ}0M_0ma7AQs}wKtSMk+6X$r zYD!Qt^`CT*KK<6WjI|;#LV|~_=@z>@tjG5HFx_6$`uQlU4@ypHUpP$pBL<{+#k$Cy zsf&6^jZP{IY3a=oQbYNbCD^?V#sCHbh49e1vRw|{j}fk5xYu>+WT*g4gU>47g3_`$ z3nrHuJ}oXIxSPR^IqZ#sfGd8k1<;qj)(h5#u!+Cu#5k!mJ}H-$OXZp zblYOWpYsv@p!Ehdb(sZqfYFd^W$kw69+cq4yW8s#n5uGgZ6e1t7a0^hu$(|sGmN80DYr+%T!^Z_4MX*G|BwxoY zJ=`!x$!h-}>RqkL@DmI~ITu^^#fbz_Y8TOf{3-`xu};Fo3=*F$4ERx#k`zxDjxn`I ze5Ae%guv1M{qr#}zCsR^d$71;QQ>t+4)(}{-h99eIhZ2^YVoNzp2Jfn;Uu+VS3!!R zV>wPbQq;_k0I?5{XaZ!ed79+I|Mz+R;f|fuPZt2=-x|f+7;fxPi_sQJaK1r@6iQ)p zz}P7k1M>%l^euQ`qPoVQQ zsbj`a$oKi3_~iTKZqy>)LLdry%$>(R3`;=keQC8)dRG>XHb4Mjh6JF5BRxopt zUouR8%!HkDMh{U;l&j{pcycY2rfZaLP9F4W3KVbj%I_mEV0S!yg1Dj@;z!RJZ(i;t z;esu`F<=;>*ioUUelB1?)0La68{3<(yJkSS*r4o`DPj=86ce%bnt1`~Y4LcE%EN0p zM*KWbE+e3;FI&L<@Pp7FJ$LVGV@v^R#}*dyC9eIw>c=&&Q!vYX=({iy^}6q;wrn#> zomZcQ6EHs5bK;L=3ob2ch zoF44KYqt5AQ%AOkYsU!;eMh*1feoi%|} zj^)8p$LSWFZKpb`9+ghjzWOhQ1oA2P^x(e;vfNe2veUt}G>T;tdZ_^(2(S==ADfs! zTKx(?f4g^Rudj8mq9pI(5UnuhUt4Guu=$F2n|9 z^MO4#I>98>VqLWq_UjxvuYfw5Po^_&JIZ3DEz=Kf^r+64nnaa=pX3>--nYP9Vq`yh zTT6_@oXaHeU|A)q029GTjQtp-hqpqOw9IeV=Pni=jOpov&l0m_x*WqQ_9kJnIu0|b zI(8;MPOq6-fJoSljH+EZ&veYEn(q)`M){rM8mwS>Uwl;wQmn`|jP5Y&-G`Af6EXfG zJSEyeV+gqO-~(-n$CcJaH7?>4!XF$0j32bJV%cD;bFo`@F!GF+BJ8He^doD#5IJO&y{cZ@4g0w-;@M;x{Fy>&hL{e^uh`T_4MFsv zCxTT2f491am7(?vjd)yqa~*!U;;Z?GOQmSHD>ijR4S&`%kS=rRsZZ2c>97diDE4(fCj&>#rhRkXL2EQR5`}d{f9D= zl?fVOsVjdLM!C0=^eRzk@T5YDqFap&MzPR8lZ@lS#Pw_1b0p<#&K{n(b%ue>l-Tl+ z)lWu7{sUnW3c;q}e10HHsz)7?VB<_nN>5JNzj{0x`nu+z(@(BIkCB_30BACFu*fOm zeXZ92C)8n`aqIEU48L_ZF<2FPN|09bKs|(9BpK@Arcpe3Cc5k22yJ{{vdY#2|2jk@ zKVFN2IK$yvH25w8{pqiCVY}uEuFh4T&@cxHw3Ei9?i{a;%WsB%ud;IoUZTIeAU^E9 zM~bCODaU_0Z{c=9-YZWOD7lZlLLlnDJ&)on+o~R+3_fEJwkDL8^kD~ zDUdfcrGqSx#flt8<9y#@??&q*bgky2qh%@?xq#8G+bJhT6z*5~V#qV<`MdyVr9a+B z@!W{}I3Km#A^BtY#`5)5HNJ^i>pZVc&~kuVqs@Nm(YWEa57YNukgLsiM7;lhNE>}o?&Rf>xN!n1TXKkJC3f>)OZJflXV2D8SWzcVp|8k z&3zrDKLzPH1hSdf5evDAnL@0AyQixs<UX-2@kMSti0CzEA?U@n0D#Vxr6(xuxQ! zSs5jS)Tybdhi$)g_xCG#de*II<&VJoqj4b$ZxVb5>0VY*2|fKdZu=%VIRql45Oz_M zIX@4C&Yl#EXw8SKef*ZK!7|@gPnzHITOon6I-gjS1(3sDP>^nGz2kCPq94eFc%=3x zJ>-y<&<@dDq+g6&0UufRL?G?dC?O>UGg%#@K=;no-|sSFI9qG2M^d0{Nk~dA9-UbT zq#?ru(7@5B9929j)#gY(g`B3+O+T=afTu88eAjSn5|_}&k1?EJ{$@GSrS<9;h} zz+(Tij?OBUA)xhQZq;wuu<3~;a-jHKMt1JqwYYi|9{+06jfz)Vg1zcdZ&Eo+(sp0I zQ1vlQW^rdEjd;BJQ-htOyQ@7>3rbc{$M5Dk4P&In8ySF#jQfLWC*+cQN1J^G>0{yd z`S+(ypi5R+iXJN-7Xck<;R?vy!RiGHc6N3g2*wVkI~*LX$eSI?oA}k5~LJm#}x*0h)6;Bq9o_#C?IHTia{Q)b;A0;ux zooznZS4)UZxG2XL_-|85vaOtNI+LBJH_>{%YY87zhkxVx5zC!R_zFDA5v_ot?2438^nSo*$qjR3hoJ3fOOlNsa;Xp2KU#2=e_rL02g9ps;s zLIP+Yk#obW#)t|G>I4RdR_?9=G{MXui}rI`kpD9Rf*im2?R&XdOy{0 ze0MK+{J`_k5rWg@HAIC`E3RAX6LwR*fcd#eeIK45U-XBcxH>vHZ0rG+;$)O8zsH`v z&H-yPb;IB-il-K@>JmTWXaxh$Lp?-+)P)p^eUIjedjrZK_gDa7?$r z7!lc7Gl2pEzLq3haa+!UliC)0RZBCGuaIw=#akt}Ws#i?M@0RKsZ8)5IpekN#GZ!r z$7k`cQ)%J-2Nxd}bO;XBvsKN4EDzNnx_1!}t zjH?{LHzpI5%o!XK1mSGt`lME%>17I?{C_jr1twBJ&=}S@G67JWw#2-N{!!`iM>n7v zv9~WD+|88pI^J%9kI;WWBVSWJj`27inLdH7Tq&nJHKnPF6kin0e0g$uL!J947aE(e zrMAMpU%f;y_9pusMnF;hgF426(MfL{iKap?yIs9db2Q)l4&$Lhnm*Rt1@+468XMvJ zF$M9*^ivw6V8>=TJ$}l#=Z3|Y81&MI)?_PDtWhTL?>Tvxz>%BkkazEs=04u0Xuwal zZm`z)1b>?`y7=1~EO>oJw5)w6;7CiO#IKx-?_~gE+x4yj=0o!wg+11A5wy6qwJ!dX z*!HL6=TTJKdrMAT*})Z)*UgJHEzU}L=%E;)MB2?a8Ko|LX;ZcIb1NRj{wkF+F=NVT z>g&Hni}4|g%XDjMX3(maiq+i=o3>_Ii)LSHe^3E)O;c|v;~W0%IGv@@CtMD1Kega7 zbnN@3;%#A8i0vP^;eXD`RT0a?3AH39408gG+9L3R6@_Rnr!`MtoUshyMJ0K8l&_Dt z16vrqWf*`Qvv}A(X8=aSzmo0s}jc=;P=@x`Zzz6-^ylq>SCDqLH z4h&|VZ2$a*r;<;SoZMp<-xLf-7%Jr6c1!B5lV+IT+{V%5OD~=J(nP+j5<{GI?G)Mq ze!u0`FGJ1gG4spFsAt-nfV(V_B9_40ac1( zQ>X;Of9_f6=xV1LTQunjD>Ii#B`U_52cSs|5&m6UZC(t15l#JdhPsW%P&bu?Xxe)9 ziya5dKL-vm_(OVa=%`7|eR(ZG7g<}`Kf-Ga80Rhs&z}EGB5SgOlfR6KQwKRR0EOJy zg~Xmh)|B$fXMrhN`ni-!oRrG8mJ;-UIy#Tod-4dud0gYO?iT+_getcAw+oW@bJ{@( zoF(cPRAYL#Hg3jtTDW-*mp$wh)(UoQz`^hk@7^Jg=yIlgBpE|BR%YNLb0kI70Qz#C zcLWDYTO|w?3d?wc6yBU35`;24dR?xY^S(axypXS%^%WXZCr6K_@Nz4cW4`eY*hwxl z$Rn$S3FKt-ISSUbG5j?fFbb40)QtWpp~Lv$t62*wa!go?oKv71IRX}U1ZHh9>$Ggx z(?6ZlxITSF>2rc3Kda^u=+tpTah+zdC-h;Jx}S*Ah!IS9GNe(g;^m5q>;U6Px8>}> zin9;{iW4^E_M5}ObKJSn z>?GB1z=oUxB;e%UB{NFfP}qq6POC!#Qi&_2Ts-?-MC&(oEv*~+0-nDgYd_HV5vN!t zi+FKz+7P4zzz`T%faU2Tzt*oC+tmKt@G@T>`JZkCYJh!+rJ$|SCmxw1*WdL0m4w=Y& z96z$7g54|N>%VKBk<8q&mjW?IAt4sBaVdH&4tCiN5{3EB=P-Tkp`OJ(RQwAZ|ZL`%=lj+rkQ*+qwESAKUe;q72 z{ByP0pzCgQ%FEFEpVbUd#1gcA3;TxJe}yurMh4z>*~iSzNU@q|A%>fAs4D& zzfql@Zhjv`r*Le8V)EOGIBZ!iq>YdI0@=a`{g^GT9%D*yE?yOn=LKc21;nKaycJUq z09q0R*Kv)gBkaBwCwFmXckrJa2QPcip<>P)JGrmC1SC~Yc|)2aqXN~>_suuhh6)uPh zks6(XMA!1T!s2v)@me1t$3lwI zM$qrdThG9NSyYq~))Xa7myJBjfy9y z5iX*YX7f;g%W0(cX)6OZRt^2PIyOkcc8{no&B$jiBVg&$*O(G-xtFXDTRhI#xF(N}r8e-uI`;f~m}LvL!2 z{FdRkn-(bL$4=Yw?l%RJvp z2Z~ZK*oA0pIamg#k^Y;G*npTH%NU9zRQ)3Og=HyX=^7$z;P#FDp@f6VtN?E2Os-Ov z2fr*T6nS8I__dkee!S5ix$1mF%pNLjmP3Bi|)+;_JxGHwo&@2&ff7qKxnfD z+-S`NOilE&l262Rec+-7kzfAOmet49OqRJqjK+>MDv@@m?eVL8zaSTrl;s>jqmi=D zxV^hv?l4rS6wB4S!$N%wGvElUKR$b-x+%$Fr5i8L1+?5PF_x?X`Ac6Td~+1HikT8Fr@33!bWUWU2Rl! zsY51s&<-oVwA*!;bCtM?9v)Q!T|71E1G2qUvQpc`glQ~L#QAugOm}#Aah+c!!lD>v z7~vX*CY!wc*cE2~0uyE%kZ@VCd7e9PpK?r4!BR-?A z3H0y)xIdBo(zab9%tTGU*iG_8(69HM-eLw8wbgcg+c06UpzjpO);dnNBY3k(k z;w*~qtatLD+W0!=h4F;{%xg4j){jAZZ}B`XzsZ$7wu>Ce4uanVZ*}ud16riz5Ochf zfM#8C2Uhh5%fIxy)tT?igNwX&mS7J{$Nbf?=rf_`hF`cN&&$-LQgCa~VK$GI(wZ#E{#ZRf9No{$Mi^B&6@fGIFN+qs z$xgJgc2I<99HsW-*qBeZLTK<`B(!a?dVdXWF?}VpABJ5meX|YE?XKW~IQ`ouCC4k- zD_49svw{E>0`J;{js;ANiJb)`^ZX5!wlR;YW(xiW4Rvo6x|itoigqTi>g!MYnX}7c zcp(Z1|DQh}ZO=DdKgB{^P(1tgNJvB^ucG2WRQ4EdSANew&)lWxVa7Q&Hg@qxQ&sH1 zofZH`Sy@?4?$N%N|8mWne8WbH=80M?i1StBI>BQ^Xl#0V3j=1SKYr-Kh7BAKAKh2W3^n^zu^Tzxd$EZWm}zT?CWQ7LLw*|04X8P+@4LDjtMM&WHU?xa>Pi42#ULrI@wGG1#AxsSu#>+$>(W+S?=Z=(=EaKPFGA#i<<)hyV zM*Slb6x7ZTa@c&+GauT9eZM{Sj~LlRkf{AGgtA(6nDBl9rT!j!{;(u_+r}}R^Yv2i zC6j@T95Qm+Keo~h=^T*x9I{yEvWMu;#7IMWekaYMwaz*@G6)T|XrqI(JK+}`_@igG z!u}ikGK(>iMWCWv+q};QtshR;@O-^!;oIhkTX_pL85T2utq0toOWu;dWB{+3TlxUv zvG1ZN*BTKZTN^tx5D>}!qtc|6#Joqri%cBw*^U)vUn&axF9qU(j*4E`Tqt#L66{`maEDaT%g|&w(9NnoB(P@zj@vN7n>Ki#-cfLn>eY@b9CWT#fC` zpMgPOx2TAN%eV?3Iv7+4bS(UFQYTx%KC*a4(rDPS5cy@SyKi%KVZM_|@bzv4qz5hI+<=QkiK!LDi zt}_=#pE31@(F~u0O$fK2uQ?u&k=x9Xch(c(%AXprS-NqC74vG}qGIAQ>{j~ zgt z*;`JBCl^g&40e5kgDPU>nZIsF5eRh0rzXY>%(y=OeKrav)N<=H8c@IcrDTm_o?Lz?E&TLZ$yDR9>8tU7 z1K~<}Z-4AWWBJq7XE<#S2O#-Oni=ku*7#wgU^R)Hou&0Y8_>;^I(FZF`J$^IZ?N-Z z#%=MtpjOZaNsq0qjy`_hOd8vDQr>N84{k%w z&c)TW)xj+%w+h$Z&A+QmBaZYgj3SW;e%Lcc{t}{xFqhSNn7Zu_b0zI=YXKo)Vd%n} zZNDucu~X0u`-3K?r?1~)`32bxL)Ax0aA~7M9ht>8_k8&vbdD@=fUL0zMQ5w#%6f8mZsHpNLx?P+v*(b~NdJiu+f%ZFoUtYmzcnj8(Sv3D&8}d?v zcQdaYUgpf8m~w5{Rn&r;Al*~k9biOo#)K2EBj3P|#F$=^Tt12YZRW2o*e#R|W%^KA z;O{ENxJ1L|DG1WBnNGql@ShdJt~>$?N%wbJ>DSWmKQzVI;0tBMDOAxTLgdr0jf2}5 z#7Nc145aFIF|KdrS`-?2t*z-l;O&ZUD%08ANHhNpa zQEUg5`vr;+y%4wsl(Y@kUz?iJ%l;S@f*!7rT}`Rx@IP+T!#msz2gpx(p$>h!t?dYW3c%C70CN)+SQS9; zcj1JLQ+5vxU2kN8xvY1i6jAPd<_zI?GTU$e*p6WnL$`vN2wi2rXL%o{y!i8Mm;C!U ztHFCMSNoLlFZiLi3-VsLauwaTn?^1Gm3IaCni3(eA)@Xae{T z#%yeHuGleok0qH#b>L(Z!TV&UDn;$G4X2t55+wBIkEitPi0$@^HK&1))>pZ0ib%`M zzwz!7?PSBTsmd_LXr}x0`R&H${O;Qf5lYp6(% zPCRGy(a$xfS_;C%}cBu z7p=+Z{Ul>Y6j#OE@*H=;BhdWWv>wUjc}o|3rOW#J8f2{r3pfHNRQ>tmf;44=r%zm5 zyt}uT^!n?gojnn`xvjq+)P?9^_|?(FmW`aI9)G*jM3p@h25#@idvgs_KqWoi{S|e5 z>mF1VO?ej5kglyDmHf_7?wOHX!38+L-nz^YeiYP$nl%y~H6(e@ex}Zf=sv~G z&o!qe6+8L76iZ_V%jJK=Oy4^9s;Rl|%zh#I`%T!e!UnA~?j}K2OEC3XCQk7H(@&rm zx$Shs%U`m(-sCp$Zt`KXF^k96ociV_yY0%xBkuR;M@6IIDV?U|Uu!5>`{?)#;<%Dk z4x#F$Dk`3JR3*oD3IFLY>`x&CjPNJ8DT1_1UNFPbvHvEo$WOeNmt+dBc`VW}0Ut~l zP9SHhBKA}M=exBx0EAiy2j8iRWo^?*e`?B1EPYd22!M{EGenO7tZowd8k_i2V+njU z-Te_mT9v39f5&y#nmTq-+nn>-r4a4Z6cUL9ms}$j<(~emrQ;uf0C$(C!hRVaseDeLANOv ztE0P&Sl+mA05%_13{40q7edGc-MmK17jh=HocB8T3ui(@H=X$f8HfW3N(kL_L;YB2 zf~J2fDZUO_F@b%9&~d}I$N?dIRjXm@QkZ*EMDAXJc?sN--;yKlyM;bkboAn1$YBiD zX2H#93+fCaBa*t>PYqfr?JGf07bUkep&z-vU6Weez*@gQ&_8cNB3eER{mmR1{D^Oq6(&R8;3eA zcmAz_zjony@e#wp=$M#Y%M8zT)kz!Qj{pKH5iybX*;T*eG+ug?XAA~9_bWi@BUJ7* zL8wSUjs+Qp4bHJnL#^!XLtw9RpP5dgEzhHqr2z(0s^+&Bn7r{_k1(`juk;=VefOjQ zGA6tB%TA2tmD9(g3~UHk^}0Bw_R+FQGXm?gzB~_Fpk=EdLwl|+%0o@95}ob66-!s* zL2=~>82{lHCf?E@5R(ojO63DR+VaOUa@#(Pv#V6u2&;>`71 z&tO99LL+LR4>Cy$&B9{F_9hJzS@v^lum1YgVn*bXx-ttXNmj9M6L@^@Uf_kb{cmvi zSS|2f6GLB4N{==2H#m~v>(abmpd}UHOQ8S)Rebwbz3A%uOANqL0Nog^wzhVv!AlgV za6g|`oqEgEnDB^YX~G4+fIk8kf14y3E)!`4xNX8uvLV2)sEC%Ho<2M_hUBB=R8)|_ z85d5s|FQ#LRXhB`1!E0>-d*3*?~G02+4)Nk9?qiy2Ufj+RK1{5>xL^ltw4_%p8bj8 zpOAUICGw(UO6L{=>ez0}$^*^zT_3FBpK}qJY-nH&+%+@YgsWJC?m45a9fG!FT^(2SJjgW`ys!Z<Cx zdX0p4C+=v#x+wh2PJ%`YyEVL0anNqsvB*9KlO=@z|BXLX`<<`xR7GF8f~7k!eE8=R zC6DJ!UmIwL!Omp#GnysLw|*}AXRmIdCk6(zqOeD?dqQ1sm%v8S`ODXY+9;&hjPV-% zxi|s~f_f}nR^o)Wi*LLh*9{ZxI)Ew(`ba7|o7k)w6fCk8HNU%5k5vU4Lr^+t04DT$ zo}f?{^{S6#K#lhRnC2f`!v&8r-SvUH&Z+mXk8VduO~|B2N|~0l~3}LJYa&tEFCo@8(D|wO`P=q<;`TR%%(WvV*EJ6gur8TK-QXY;vry^CnorYH3pn`g^c4uYQY5gohpQ@q12v|r3l7*sI!B4kg+1K)zj(QSB_x`vo z#kVKUNfa!SbTUSrav4mD@;rRbIjAk=+Mg}7gHUT9QHP#9To}N>=aGDXk#=93uEs9b zQTm;SY^WUeTLFUz`PUI^YjKTMj?;SN2ENkxHf=H&TfV{*J{!uGi`OaJLa=EW!kBAK zKiMdv=ni@5;}|MsNdhdX4&t5|RX7~Qf#Cxvw*laX1Y09BH}{V0v6j;_h!++W7xO>q zCG!TNRv0S+%r1=3i`ssp0hJnQ6Y}(wNK$|DEFO@6r8MK-UM zuMYWaS1K%w4Dw2D;D_XBcfU!^2(sV<(=&EMOEi+Muu_q)ZNy`$N)Xa|>UjVU=V^Q= z>B;3ItI-LD;id}u*TSuJR{>R;_}la0+;HChH>Q^0m11e=eK6ogB~d7QEK;50_dps9 zmK>bw;AgLJpQgE=&0(S%yIf)ACa9V57Tfdr-k9atCc)|Gs$jba7~Ek1*|~zl1~C*4 z;V&>Vq!Ks~&Kgonl@YygUg!pq| z1DO;0$OC5N?$^-|tSpWa5)#mr5N2r-CH7zgi3~0vByrFC5!K!g{cHCy$dU?-{tR@o zFi{OrRwfPOWc1Eu*ydVwr)OjtPlyxb(o+vv!7Hz-qR71~q=pN%>~C?_YLk(c;_NGjim{8X;v_LDZwz0rtTeIl}dz8mPhaOux=@SVM^ zj;wOKWkE!v%k;}$2WQzz+m7pU;!tCIm2b?mXWL-~_fk<2JhdyN@5lQQ-{*tuydb{R zq>y4p02}ESuCp7?3FGoM>BX+HrW59vSmYG&8p0A3_82wLb>wu2wRKQhF!O%sl~}YK z2Ies{>qq_}_0+hF-N3@ag!4Y$%+1Zl*N@8~dcC+&wpDj`&BL8uV0CbsiKYJ4ptR^~ z0^JSaq_+p_>Au*QyfeZ)loM9HSQJ$QBHGu_`{`US0b;c2v zyPcy0|IVhZLklwcr9{WV&9H7BKU$xZAJWPg82ijKTAre$gM%(Q>?C2f`iC`SZqd_K zHHG!Mn!a+F-%vHDsh^VbezZ?__uMi?-XO>E@Y8yiJCv`1)ZX?NFAN%KceKVI|hsXi<- zW98$EhXe5;INy-a)9a=#7iZ_Z!oqNYFkoSYT|dwI z8>t5ay%q~bcJ}boQ+KbhS6!b>>qX$$tXJ%Mlu&$99~_LHF7n)E^uDjZRhQO}+f#K( zEvxOEV`au+qia0FHS(smp%j;K%`B6 zxm%Z@;G~B0OR}1CQr>V>=$Ju9WuSS}B)mS_kK9~J8un%N6gcdpZ;Lx7@z5!}Va3tc zpDusGM)YUfK0}GhO+4b)dE9=t;-&o%Pz|3P#Wn$_lx5wJU*%Y2R*7a`A?HU46=yyM>7H z-NqF8QrWNXhX#0a3Tw#x<|{>Af5yDtEjIXMUGInJV7>jL>}jeo4a$P;quZZ(25T-q z7dOAO(mN^+4mvooN}8i;i}UwK%-HpCvNH>EhW!lC?~8Hzt&1ujFz)2Y&KY{QuU{pF zmBY&`C2ILFS&!q~J$Wlu+eFuP2V46K4aV776-7##QEgJXMz^ie6JHe-%!rtO#O+*=P`-?SQ*Eb%4^0rlw|1ef{#% zQeapZR;6Z~-?vS@!^0d{3FYqLvADH$@0;zowqA_g!v|{W>UlLaZ(&3;HMhdb#`Zu@ z&ziIUOhZl%fr5g9j)_T1Wq3ufwRd@Ch8@U=joJ*pX>s2RF_>DkTMHb3P&qBhB41;7 zQGq6>riNo?W_G=bFJDtnPDwfcRpXc;oYk47j>4R|cH)aelFNF!cX;}~u=?0t7mc+b zMWaK5!)F1F=NRwv72W^pXqjCyj?M2JQ2l(y zm0)MOQbd$nJE5#`N~FHcvJxuzCesdeVhsap(AFiZ$nfKj`Nf39L?`$pSPSTAYd3eI zd8SW}>hwHlot*iXGoRP|ok8P1l|%ao3NthJse5w~oeidoh_-G~h3uon9u&-Dl{?m} zpIjVDpN{%Qy*Tq9b>gT*=ib~p^-NA9@xgCr&sWr_#N=BA)-g3i6%gy{`YzQc$Dh?X4WJ^pcj2t_NuU zGcz}XTI9fWqhF+Vc79_|2kyuuSUpqg`~>cfo`HdeCsuux_89|y)f~_Zl}+9tBWs6* z@($e~-CFh9Gca>rX58A7g>5}mpIP9b^aSwI$RPltMoe;Xb17O`N6hITS3!XdjcDP$ zs1Gr{F*>#a6+R~@CYf35RC5zmK9h5bhGfZvmM;+Oceqybv#wBS&!nE5u zRYScShiISQ&W9bnqmu%U#6yhl2RLemWFab|DTH_Ls^1pm%m{2{2FZhZtV~qvTAA%_ zWZq@piC`W=j(AsI+Us~FKThWHOw|i~da0?oMeJ5{XGhokCs8);UP+ff#u3VEcYo20 z_YPR_^4wwN7R*Q=@2e(MzpvKTm+zh7>Ho?}ZLHL&VRrjyOWcEuJazVK=y~$k&7OmK z*5Z2_HazBP1rDVMV}UC$Wtj~1D5kJ{cW5`!(`)aDqEzea@0{!!H3&#ePw!1$A>)b+ zEdG-C!VB)o2(=z~X)G%W4Y?*bc3`NA8x{F7UD;gMHm_#=w=ji<4Iagp2v{^*>2*>s zA_wBd_3PU{0MzXxSG+h#7e+v6MO&nPIL6LWIaZ%DC z!&gzI5w#63D*CfI-d87+T=MbJYdeJ|kE7pJ{DWO=I-Jc-CQos`bhL(?TGNb|9;E7L z(C_~3NcV%9p%B8!#kB$^@;t!EgHQOKRafkPMWdL7jctaMC;dYNDq`{7$JuqR6R(7a z*#8!k4P#{@c}{P#oZPPRaUS?MGE4JVTmaj&nDs7C)4hdWOqblPw|O^PHt;69g{wa3 zCL_9K)axhYqdi|^te1xDB)lYwr(?2{ebhRs{faihPMYQ9!+N?;sIxbj`dH#R&4BzN zAijp)**PogyfkGl&r?!T8U{3m>-JPjohimx@s>vZ&!+2HJo7SvgCBQqNzjo|*eTII z!2Xj;B|CQa=j)j33tOAtUkFR4pR}}**Hw*taru(=5zAOMsOw{4X2e7Hny>%{(X@>j z`7q@W<)*hDYZzN=OTl`{m;4l+($*}T4{t5K4bee+>FW@mQc$rIA2Mts;Tg77VYpCF za9bgLbIQxavX$>v&vP2qf??_QpVW+-(Y@;dn8#i0+cA~L>88WGN+tXB13NA(Dyx`nA8VYW= zj=ZOiQ(~4pEW+KFRaV5|b<7}NHahajH2lSRuGI**vPXGXrVNzI*P2NJZ6}+1M&78; z&8>Ed`|-bvO13EDP($giOx?gOYd;>5QlY95M=uM03Nt=kXKp zz?*s3XDm5$by8sFfbn0`Qs=S}2+0!+L@V7;S>=hw3`>sLz~<^iD3+tMOU<1#BJJ_X zwGRd-2!hR7DYT#CJtWz(ECnd2MRT(XgDuZ;%u4;8x$9|!?3HW`c*ZX{k|jmBxVAqW zp$wX;zxT~*&hWicT;%zHvz+tOhTieP#c1$Fc29qALBh$vR`qpbB8uU>KMRX_{qvM- zSYL7NG8vx$L-b{`ITl-5@g0;Ks$g%&yLAO3yqEv*M=u^Dm>y z%hQvThNjhT`S9xlN*doXFs!jc_}DUmsVU}88_36UpPE6 zXw8`%tuY($qYYz(2ew`k2y3A^IgIew?gAf(y0&(FgCc2geJ||Vem>Fv^WUH9lc9I- z-nmBKCFYojxWNW10Yv>iJAV#o{AIv(E|F&^cC*JX=SAwuD2Y$~3&Z2r3X`Wk?|V)A zvWo1!Bz0sr#iUnJ`|M{tt0WP2MNMf~T%XxN59^(9;&_c{z_a~9^ekeY`5|5O%9AYN zLOio*8c`zFV~iEg(Q?BQ<@0LK+m6b}e!|ky62_#IY>G^DL47h|MKmR|4mNk)q$TK9 zD4+MA7A3J^Q9Y+&d0-pJdQ$_A3ISyg)7D1^{j=lRyJS-YoNCZ2@u{b2V0j|{)9<%N z*2p&X5)Y2!-FHTHKg2mTPm_E$oisd+N~_L(l{1F^s)_T-;AuvxSU>L={1H-2H^JeL zS${qHd+BT*iT4K{OG+PQ5>=k z0!QO6UCxu@sX5NV`nmeu4zTr3@AK&55VjpdlngaP#?;8c&&(Ia*lmZk+*8=| z^^W`KSy(K6*Yq@w=tD}Amo6U$dIxh03*YAa6#lgNRHIu}rE6(tL(NI;PR!;cw`k|M z-Nv@0OZ4XW1$6XJ7SMhfRe4Is;cvnZwYGUOdZyykGX53+bHUT8I^30&-%cz7>5l@& zpBt9NzUYUeNkY6!h)!Er(s`4kFn89}=-T>Z-#@d0oSgXesnL~{RZ9`${bG}dN5Z|n z6ZUfzYn=(rTBPFu5+A= z=Z6fZqbsueaZwWJ^2F$it~0?DDB`m`aAs+4BLdKe3dP9!pF)&st9FvX|Bqgp>}Wm4ccKLcV7pbz3&zp+r>2A)pPxs2eB*U&oaNi zKXf-oOO)YS3{j0a^}I)x+ziL)@0XcqMYzlJ+`qp(q~pKJJ!+apy&=PHDW{5dDPfr1 zWj|$?YetaPpo3c$|{wOSD=c2&N&CP|iQ|)t1)B5)IT(IfB z+>OdjNG$TXO`9b3**kWEJ z&O}Dto1(8Ef=~6olU>iz+Agik&iSEKJg3y>3_of76B2Zpj~P0rd*0Z@lwLpO?PQHN z*AC`VbJ-#@bHG>DnQHWseQk@Ckd@V?c09{h|dpwnatZz4Ni!x4)LAr>A%K^jun94XLgcm64G-74blAQq?Uqc}Z;6`0c=h zJz2|u<<*_X9FICQX{Y1bD;oAHiRefoBHAr%{Map3=$dh56>*Jf=PL_q;T zKmid%q*PiO1VrhO?vn2AERq{k49J<-$A8YtEUY z=BVx}BtT`c;LnW`ADuSzI!@K9?dLAh0Zpk}!%cE+OJAG%+jg@%8n zvk+1;a$xAD&`Hg}y|CqEds27OK(p-a;DTyjL3MNTRB87pg5zQS?iBgR!pYAdag|el za<0ep*+y%2M9zLh`3~`MM`mZuXJ_J)KgNx>{#dC##`X2JqhNMWpxsLMVXt`pj?iYw>70wj< z%}jHT|EfB9%GH*EeT4Euyvbq$pWj29OoQvwL|a#9uRhEP zXGn3P^oz{yi;PXwp~EtsctC0?^~%{*n=L`5!I=Bw_Ak@daaOjr8R=KYTsOv!YBEUA z?w;fn7c-(`V8Ei(u=@8nh>D0Pidg9{&=E_s!rtuPg^pW)d!j-IHT>BnwgeR(Opq)_ z{Ya97khE9(0_nj&=NP_1Ytxr+LoOatx?mcOj9Z0%pg$z#_{5L>X@GC&>%XMmcQG(B zL?fw&a%AvQIdv>hiT%6S@!zL*z=nhcZFkCk2X$inK180lFy*ruXy_#l-rAs+_~jT|OfBnl0M z0IrPi=zJWU+})Ym!Z4!;Kh}F2^P}t5cf=)f`ZOFb)c{LO}F~W#)my9K`+$uT+ zX0cpo16i5)%*#(U0e9XspaX z9jQcG#g*SjXTeI5cgo2}&S-ME4k8b}9Qk40#m_wKZtqsr)DqQBVn%X@7T;;_H*VXVRX2sK0J#I-_+DE20nn2m%ADY) zs1+3zP4jR!mEDC2(r#nuPcP22v&Uxj!q%1qSfQc$&M+G<6|DPwfd@8way z>T74B#x!eeNW77UU%aAS@qU_AZMFUyO9&;J#@TjZNQKM4qIJ5bhNZjmi+j2GIA&7s zMBK-#l_Yh^t~u|g8fMc@LE>V{&?%7oo?Q5+$*bL9y2D@NXz^Jyd$A?AOi#0KMOBE3 zOQpRZjdbp^wd*#!rOzKdgUQ0H!}Wp3)%ACfJVz8x8=h+MTi^BhXqc8Zl9vU`y2m(i z|KIuKfAEij@A{M+B00&~*}s8P*?V_)w}DUe!KVZm2p1O@eKm`+tE&kk|9on=>FKh( z*Id7prIO*IWLNO*+0WbBoc$Qv8(aW~sFdj3$@k=GoX}D%b&m1A>PlHg%;z|GfGOomKK49#YNF-Ldl5{UNg@kT#EueR9m))-aE_Pruw67 zPHEUGK`!Rx5#l6Katz+Gn93M-y%ffm4{RMNLVSIFIh}T&eUzkni2u7?Trw(_#l>%P zeO4w6_2EO~(=!o)B$+Xi%>L?_(57k3HTz)AZ_~=1bnj`iN9x=^?O=yVbmQTM!(&?YR zv}}J*mB0nD_|DF4160T1#YHk45@)aJTE_!O+z5#K>6tDZ_LP{{jnL50@Tv0>4lIH{ zIH8`-kd>7+6Gsvd6db77?Ah+vG2}|Zk{s-334w>Z3>Lh(ezp@12nx!8MIOgki=pzh z4(5}xTmU-WzJ0scU>j9Tk_z>3-G2|;v^6KMbLPG9`h$u1?=-nX`EnRuIkAz1m@Y&k z7|r8dPwriOZdNAEm(fntxf=5|GQbsSuD)KyllkYE(<1l_42Sw)Z59{}@%SvC; zzG|*2#YRF46OhR^xbIe1|AO|Ffm(~l)U>*@(?`8CVeXqxi?qN9i9ZZk! zZ7ehVbeU6-$ZXZm{yEf=*)F3lx4M{gOomvE4b+5cg^3J#&YfR8Y|l7P)0189mPANe8Wv-N424#%*J%dxTSOt zeMcyya_{{*Yn6CJ8pjr5&r(Q^hm-q}QH14QX>x;IWSk5NrrUmGKsMSFV~32%w;z7B zr9-)%`u+BUx6mRN*Q7mgJ-p{l0ym*9hb{ruWkF4eLI5$tl~&My;aO!eQ^#pxVFAZC z7H+3K*iEF%1}Nup+iTyMo>1eG&{@q&Ba#PC9F7wuHW^uy)cLS zISWii@i z>;OzW6UDXuh8G|u(FO=dy!XgOoyWlk*wVWGU?fXDLUZurW}&-$;gY1a^*s&lPhY=j zb!46jyqhflUj6;kd*Lv=naW2-hErdP zJf%~1GkKse+0cjQtD62u{d8IsYi)Apwu)4DJT+n!EjUwjK<4&ry`NPJ3F}u*w&ptr zQ>0R4(N!-SyBDfWt)<#Ty|+f6F*>Csdcf}Z3h5^Jwu*Z_>LZWf#ZvOox3}i==Cdam zE?A+dvb_j8dzUT-0C}OhTwHB+)VeNl&J!c|@&;-#{+JTV@gf70#JZfn1qI3XU1^8% zsOF8LPWxWrrPOVGq2L$z?)R3@^cLTK@H?SOp{no%!?7K%tZ*xZ8+Z)>7Q}&7*g@$8rJ`%VVb2S%roCpV6DGuOgm{}(1L=>w#CF; zn&LgX%E+r_k*-^erDYkHQWfi5Toy_rYJK=`Isq!j?8AhQtU4`sVZRLC#z;ZXAH7mK zGHX5r?g4T4?;|MSo{qJ=wilAuFY(zluBE9M}O$@GG3sA zM7P}Wh2wXin1HNo`-x!<_x7{8_lBXwTwb)_q~57L^rPcG*?k+ z{<7NRr|Fh`^hoM`ohzy?RFXs^zs*XDU2kwMuiRd~q9-eMQJ9Ps4oi7fl!JS3ZG0x}zh@Pi2^gSQOKA~>_}9&~Yo2_r z&QAF4`9g=YeSJuIL2ZOtEJq<9AJupMKQ(s}KC>?O0)0}H3bVEnb@RJ6jd%MweYd=9 zHuTFEXoxvXBv~Fk#7|3fdP-+5_tW}A6lPY}T19sPdl)3Zyat<^rpr1di{$%JGu~=; ze$E>-!-&=E0_*63HV{}@F)kWl;%SU_oYM=MLXh~FZnwK%%Zy18q zxj4dpmA(NvbdAXk(XPK74yyBM79*QlA7nTmWLRyLze`OK*skJm2KTS)b5m|%djOQ|Y1}W49|@9sHUSb+ zW4~E?)9BGWiOQVgX{oNRUTwE7`*SgrBrMb1%9c1-h;3);{qzLu3)Z+VW1i5BP}kTM z)2k#!C5l;C@P=%T(tZ1m1`JYbbW}df^317Hjt>R(p~TvmlLqn^t#+S@mm5voRf%DW z>qZpP%7P!v=h(!d-+>NSua7aHVL=l48Q(UJVSTtg(8^}3Im{9&(AJ+ND5sv3G+wHU z-9yCwjGZAxcc28i0{%y{VrD^OQlp{sTNAF~igfW~fsbqs_Q#)=e2a*{)l>Opo{}$! z4CRz9goyRl9C%tPY$^WV`0w*h-aY1>KDJj2nr%7seVrev9Plp;*02XSvo zF=@%}c=C7#WpTjbevM4{49qTB4+=6OO&~Wn7YURb!NiQW==3S$Uy|gma1<}4XPFDt zm)>RxI+`dv>aWlY{FBLg(6*7Ylqk2yzKn0>=qRk9;NxQw>5}R8wY6jror?r&=WFd) z9Bd`iTZ?vyoqhs-X*>mgTm`+uVxsm}ID{h#bAj9}vf@d(G1D1BBBu`xk0>m?sBT8t zK3rNd-3pK;oLOC2@^mZh9P!1c;xmxErg`-vsJK5jI+;N!mXNtaM7yLdLaXPEO@>oy zs>mWh4M(P(>Y{^A8~mF;DX9RW4QRh08Y3te3%uo_OQ+MnHlfd?EH7ddnxxxa}PT`Sa`Tb9V{ zh%7lwcs_?>6yJ7E$GtY3$ph;hfxM8FRu*8rc?E4T?BoG5lfWe@)K@r6jy#oUFF+J? zZE%{+KwE|>p-Pz1ZVrVoWMo!>$ZDh8Iqy`ldAcVWvQPeIz9-Dgq{1Q&NSAX`yOpn? zznbie>As7MMH-Z4-(^s_)bK}d#YzveN%9F20)I_RN}}1Cxk-j`|8`RroF2%bfR9EO zFxmf3WW{)8c=@B}l7AT?lgsM}iW0{iO<7FrbkkibRqZ!1C_KuH zo>;4WDZ)jgWyXV4SfmYayIh{W2y9lftXFU-nCK(F!y6id;~i6qsS)It^D;kNkGykT zsmuQP82)H5FNn=vYZmNY*w8q+x*{gT$Vj}Bl9I~pIL`k-{@)Xlm+Yby?^~9i_4R+@ z&E8!?GL6s}oNJdf&?H^-5*!$FoNAu4Z~=%aOA?PH#8^Hv*7*SBOhlNZ&fANYwOD)@BxsA_x<;~ZD z5)KXyMmuRO7D?UZnw~fP85s2Ry}-wp5zym|WKqUvmFV^%`(r=$YD%TvXnC1;rZj(e zCT?dzYVu@IM@ISdst@zYYb5IIQp%cz1-!5NabmnPKhzuI=CQE_A}E0W7jG|S8!CL< z@e78-zr0avk%o6MbNoNoKKXc+JpOYPDQWU^F~n%tago5T@a*s9KGX zB{rr+N4h=#G;44jk@yIq!c~qh{kB?6-I1L}X{d~y|Xb9(~^xlQji z4j$h7(L#-Fl89&WISnqqD3nP3)o&!rn>lDdqk3^pR!YsDJutZ>V98$ETw|C zPnk)yc@qD4po-SH;ULEk1}E6_rDdWs^p?DklN+3*{hm_&v za+y+Rj2irVODsPs#XSd7xwXN<%dMM|GO^eTAMz9i_Pwp&LLTG#D+SpkPV1T9kRXTV zE?3gOk?AE@`7(nZXbSYy}y9!QExa6y78_ z%*18ms38}iTxM<#I9PBFl)HBQHW#B{3V4`!J?$vHS;K`l2q3K<`*lamS2`HHs3 z?}*KP3hq3t#U@TO{DzhI#x|LGm0?f1VFqR%8CluR$}d+=)j#)!beFjYG(<(0jfU<` zX=pr#(u?#%wILeq;<)ilXu;#)c-n5$$Q>K!k!{4Bfh1COVpQ;e(HwLRpUO=__Z-Eo z-OsjVlOOXCf1B;;4HFl6&dxwOD>g$M8oUx9O!|(x3zma;gOKHJo<=0}2}8LGzDq;- z9Nb*8Jqfc9-7c3r$8Tla! zS}=Ng`Wq*<&}UOYORws~QT)j`&k+p+tHn$R4{nXYLE0e^IXC_-t9^lonXX*E6nJU9 zt2QP$NK$t6l_3NV^~~@808N3~lXs%y5+1p-1WTxc1Em?#I@B3Zsv}(SDD2%C2aiBLP>cmvR)MSHFS+>ckqI7iign|W(7xvYgMo|xhDvdBOWil(vx4_~ zF~0cc<%4q5N(Rdg%nX>!6K$+<^Ly*9ZpZQDw)oNe0-h*e4Ii2)4#UIV1I=fS`p?%H zLLqs@7w!yDi@+@HBmC6)(`t%ma&U0)#fr)<1d%mCtwoTC-u_WG?swA*j15%o*X9!}la=7=-X9Pq9mqEoMk>t8 z#FJ-!09q>C$G?5uUFR0J1JYk?O}U1P%_JXg4`)tYEdNMUXWJPI3JZh%q=!Ng6b}D* zL}rtdtQl234Fy^g(^VEx;ijgi)&#?|^q=;i=5!rd*&^*%tb@gxB(E-_?1NN(THr}x zx!x82T&hzE1Z_uWFaLXTAZwE~xR7?Y!)O4G@FzkZt|mo!&L@I^eKj@PgBx8pNj4hyCH~j_TpLGPG#78m z*_xsWX;hvgWF$dK{1r3fP#YWc#T1gDEdHY2p1b>BggN)#7>WH4e2LCE!Y^1>Ssf|f#ZLs(Z=P3vevTP{w3X8~* zlw$G^QxXkDy<`P+OEUP0*;3V{EvuaFjx`3)yQA)_$|#jS)o)0rmxg$F!h{2VcCS#G z`*_V7jf$x@vqK#ky9(U)Z>)YapYs?P@TkSmihhs_;LV;}Lh|;&Ju>O|F=L$ODa`Y) zaOA*q+AvMfJ#M(R`we%?mxLfa?omB|KnqqFSob~>p!h$DyrIL7J$AmwoMJz>E9%$& zlyPoDZ=3OfdQmvy5F!x7#6;qW3Yu9e)l*aGUCFt*6S}@J4AweYB9^KH*s4a%?GOxd z^HPHzBCtT2wIVh5Bqk;rjTVxC%c;!V6*Bk{TVa4{F4fhmnvJGWQm!<696^DDi>m{T z0Xl43)*VO}tQ=o0c*h+cAJOfNav}3D`n_;Qt^%`HD_=pnWl`a&;y%9utHT!Om#S~n z!+Gb?;vH(V=rkT@(XXOu2C9S&;W9=GBhD6#^skJ&;r#hC_7Mi_tvqWwp2&RT$1IkA z!(}4_G%m1krF%#-2Ht`;xzgHkn@6C4$h-RI&xuNn`j`)(XZZQ^48*pB>7wNF3d@s} zG^su0HXhOd`OtjcAB-KmABycz}znbbcn+8WfFgAIwT9HLGw z>I3*S4xOsS%%%*}Ow~w!%`8XWOk&Xx^JeaD8hG46g=Gjf4C=NDjBc6pcpTeNK@YXC=7ON!PID zc}kBPm_Cpi(;b!|@#b$=BTT%K0PL-em5>8H4R96NB{Mwu2=9I{F=qszE@iM1Ec7Xl z$xiXBA<(?+9UWx?B2ssCJPB5;^}@3T3v3t?^#0&Nu(!qLt^Q)|_f!(GjX>XQ6Hjj+ zLDDD^qw~IAqUY#y%~MkZNT#6hA*Hgic4s^jrKoNny{P_}D7!bA`iG>rV&SAjSnu{; zk?dnZneh@s&;p>QN9u2&4b!vFG*fo%703qwC9A_o$U$W9W1^4`#aN%);AZ}XDy!)` z3r*yb>^?+O3quu~`J`l3=r)HVS!mn+jfv1OP_2UIzCewo#2koQvg2;fz`!8mfpQ%F z3e!|$qRpzU?=rt@SaOs;ZzZbhd`Qkz{Af?#qLHN`Vmf)|e0GLV=YzgYXI8t?e7vQS zE@4|%ogoGWu{7e0fsWnz-!c#VpciocO#VNBta-6+I~HtLYxcQ=fmpOHU?cm$q9l@9 zYC<(?XXNNQ{D|0RYkG?Kgoj68&>|bQ*4%EKmc2a+6lb1dK_@Kr0UH+Dv#Pg?pF<2ZiM5wb zXhJT_|45`bxWt$$DckEAI8PC2Bl(Zo)A~^)zd95-mEXu(2`a7K)+Wn>%!AVxbR7P1ei=6-`yt5)PLkF;I2YJ*%&A0a1!}jfPL)|go(usPd{B;I ztPiUgWt3}dc;d5gq=!kBgh?#>k`Xc!I5YXx30DLyxJW+Fru4SPN&v3_{A!CN@PBkJ zk7${gz5*}^S_8T456ch_T#7&p`U1kjvO&4;Re_WWMb{w|RE3(x#_OQGEPAj+lpzxu zhI-_9ab&R;?-^ioOh?hc`*HVO{UJ^W(e3G|?-z%WQdXGPn85E2wCjld1Q@cg90CM^PJA3LR=2p@yAe|~PSo2!kgB52)Lf9)sVCk_R zoT<>J8XMiq;->B+;^I#>*73$9zK_QfjoAJG#$+KTs?bn#CG8F;ja_IHV_Nh%u`4Vr26f7qR>=m`!WiTTBpKyZirCXZ(PAieT$5Z{u4svJMwr=v(n?r1unFE*w_TK z%r~+ryzbx+nui|OiQR9F~{(6OCdm@H%h*eT| z^550kAs6b=wx92Sh+elSG$}Cq3_7T<9GP9_%eVcW4w**78YKI{3dsg7Ey8%JvWUQd%I7gtQKs%GkM82=U0BQRSpz#i@z zPe)vRCO0M>!B6_|InShpl47qXjAm^#pN6NVoSKn?k9b7(brjqF}KSaI$l9T>hTC2W)5 zcM$ejr9GM=*|q#-XYR|7FN>4NM~Qt4uM?DyE8o+Im0Q?h36PeeA9czOee&a&QI0%m zZar0V#;^zio_Jw%vkMHdpk_|jLgVo-eZ$3hM)b${6#=foxdWqgMQMw-kzX)vG(e!r zzlJ4(P`NgS!$oVdZ1&Tezwqjdx&RXUEjC=g^AUb5-R?FQ-s02O`^%lO$s{_@C^Z&s zN;~_LM-CXMa?N=3($v0$wr)y5twdHbG*$B{-z561^N48V?;ea+41DumzeDDWR*gV) z*j+w=rMGHK%xRovLK)B(6DPv4$7xx&huUztj{zrCw>Oa!lZYi}!SayD1oj&NYYCgt zpF$G!OM~uM?VWH7P6Y+TIfS~gIIN_l4_T%FYfW(fs1>rZVlnX#cTTFt%n z$9#+kNtJon+KZ!@2w*Ov$%SqfQF^yXdNly5{R*5nf{+B;9b-bz_wV|9Eo)45*Arcd z4ORj636($qu!n-)MInf)x;l!k#A?6W8AhYMvESG_jB5eB&ZxdKZzKJW%-fY|(GwiE zvb23`wk90EjL^j|{99=oKllbO%j#I?4Jfk4L#0H-R{BefTHTGMwA7@q9y z9qV*>&2VrK4zP^5?h@wZ4bK6ZRrR4YRdRLa~Gj4VR0Z+bsZ(U0A37 zj3eEX5=6l87cNSmPWGo<5wlwkoC{CI{S#zj*KPu*HZ4_wi#Lr5E$4DCbqRTqmr!5XY%1d*lvs#N$Kc3hrYhjk;CVsqs?5TNOWx)u{&Y; z{vUFUEru_J$1S)p*SFX{QNWX`vSme`-iZ1wwrh+(f#v%V8> zTKmCYDmO1F_u|_cdr|Y0zJB2>Dz*rVQ&}WAvoG>HXVr^GU4KD4GUn3Xoa3!8dJSaL z>I@xOztlp>FMUDeJaoXX)=3Ht%|H1KU=x+RusZ$ToKauUu2GX-3cqLG9kcslpyl`U z{ZpkxBgM*{3mZEk#i{J(7F88~FiC6<|S9?nUd4_39Jc-vm9?Z6{Eoa_9 zqxBqJ^9Idyt%qFdXax+V#(7YV35Kt%+e3HVGXIfycN%u7giQo(ZO(JLPXt{ zm~H*L2Aol#CB(ek-&8b_FKR>K;Q~cVURH_gjrKkU`q~5&rsN=p9c>_3GsY`Uo=tQ4 z>oTs5d%lRUC0N`ol8yUE@*BC@VVkHxt;zs``(d}n9&E*!Fn6x^ycpL+2wmJ8KQb~} zOmQH=7jCj@^grxC&RuTDe@lyDe1K6PB8U?b5)c*wS6LYMVSOyLmbSpZy1DQC!e8`} z##xGA49~+7 zukI4vc8=Qh$960Ez+-v&`!Amwm2c_l`x3ojIt*^NaY9lZA)91#zep-CD*fhgzu=Ft zXoSTeNwxh-Z<08$aHjx?k2iQWm|rl|(keeBAMfuhoPLx zKjPKW)`6RP>84_n2vpQ*O4XrcJe@`4bsk}U_(%Gzd77OpAKV2Q~q81N1pqpUFs6Dxr!RjVu{K?8>{)dF2O5Bimxc4na+ zyZ9_7_zs}i{M69E2N!%TXG7{keOu9_1Jl0D=3jbHro0GzmvL_26_Swft?y`4MLkfe5tLURzoJ2w5NX&e zlPRX}Msqf&)vD^`%ESHn^b-vsCMtBe=*phFm(@BXM?NM4Om!jVlSOI{5pmIyv&kAr zrqn)p9v&@;Z{B~y(gsbzuabvvG4hm-z%K~)V=@|=pvK0xzb^O1;YhWE)NFe~L9l3) zysLB##IEn05xJ2_plT`ozYBlL#M;{WaB~#f3xVaq|1!z`jKBIb36D!umSh1ok*=<; z@lej!By!wOljN|-8(jOEMOBo;bf3@DAGM8#enojIXt9eQGhJ||i{mAqOK#I+t$h3` z4dWW0)fz{20)Ns&W0TV8C5?6Oq#!^o1FVVn{l#8TYrUJ@Ka4ieQK&#vrb4fcXzi_n|G%sE(o4e zt`IaT3IX1u!`I~&X6`)I`8B$zZyZS~q`|}5^x~^h z&flU8Z*0#O%j0Py3*NT(Ue$)GIJtcB23?nItJ*?{HP&()$G6nKHD93tkdz?ORJ1F0 zsPgO7d|luC!`hr%FRjw@mA&cN+|H(ijm>Kil_R|YM84c~G!$G)v*!uUr01~iY#h>a zzyl1Il9HCYKHTyFa4mmLPVR-a-|6aT)cEiBKeNe$mF8;rw91Vz^jp6F%_0>3RcZQO1BjtcC9*;^Q6@CeHvs1m&G*c(Do>Jf&06aE*ZRYT%{Kkrc zfIiW1IdFM7CduQVhkc#ficuAwYp@YNMR`+x9#si8uRCm^VVxzptTDAwxMWVd}$P+&u` zT7X)LxW}!exLpok0JwoN2S|Y3=15O3&b{XCG-DzN(^sK){r)4L2YGqzx6vH}3$w4` zufTe>uSV-%3jrU8q|mSgsI)|x?V-=@4p5Q&51=ih(&L9A__d&QR@UV}oZ`8NWQ z0Lc8c_CI_4x|J_gwXFF?FQoJ1gEi6c?{ig@{_4f{WZYZO#d4|$SOXu2k0dPkCKT$E+teLaNXXF<(T?% z?O1gf^oR=8VWi9~D9(_SZGW;#ctIfqq+6i*rV|(=&3I6DL$1nZ+1%F^Q|x?TFzM0; zhvj+pha{c(W1059maPZ3kUsg zwiM|@!Lu&g9V)Qr0!i)*Yn1{h$t}Elx8GSgH~^{3)1Azdprq-1$7^8$H^%McK0+M> z-|Av}rudn2PfORG@FAo?Dx1d4J`yF!zy;>tcDd0JO7_RQETLM?!&-tV6|iR60N22n;G? zR+24@_-ai8?iqBj(cqVa{Y#pd*J>pYjo@&26O*jLjY!Gla@gvCtugDv`2iBTS8c2SVzNuMC5yUyNg^S*@_GT@S&>@zKX(vRx@lYo7~` zq+WbJCvz?rurd8)b1gLKC@Pe11iREd zG=R0H2lf;zAPPj-QTWW+@MhG71Ma!ici|R-xsWygkGhC#?gIJzysBa%Yzn6YBjT3~ z6!l+;jbmS@$M-j7k4ZT?Qu%$&bOGiJ06P8w4h~riLrp8BSBXMBxjF0eM3AUp&zRGw zeb@;0@%7mTzN?E9L!d9rX6nwglK6DnFri)=B&>TBO+xA;j9M$;+&rJWQPQZf(*`&j z`~v~d#oZcNMh3G0Vmqa!aP^~qP_n6Cz6|K&z3BeOT)jgH5;iI>u6Urm_lM-9=2un% zL6qlIB;_h&G=Trd18;*T7=lI!KpvRkyy$;=rudwWJd);5 z?ob|1qhIxH$O3qLy$jhL>d0d-_&3qT@6Mx5vIZXHhF>ZrMPq#Ip(!Xj zkYFdeW5}D1XzpVPk6KZQ9U2uc`Ifu}v_2!N5PpZhy6YR0HhP$)MxG-hjUyvQu2HtR zSORNTO}9IyLpeA*b~|-rQ$5yl3ec@Q-Y{_F1Bo^iM0nOnnrRO~-`wQJFr+%^nG-A+B0a$M1d-vLU?N zhoCmiQXnTuz*Z)lslD7s?qW;GRZv(_%>pg#$SsthTZ0WR^p#1;Sugcr{k*7E{>bT_ zO6}1?lyWnfkn*mOkh75sW zg3xPus~aiYp~WxoX+?UzwfENb&eT-0`S1Pjq+QCY0pd~!RAIkKLZl21lg;M<`D zgcjt&{Tag8A5wVDxmTw2qrtR1-(&AqGSvc%QmG9qF$sl>-?x;r6e|UNt#Uu_Buqf% zvs`m#CM4(hQH02WSmLJ^w=>RIT`e#m8jay}7uBBnsQc5p(7@!eO}6s#6y zD;p$R#oN%{=(HD3A9}c9r1YeS*H1h}g8kNbxVY~i4+8+r#@X41TRU@C1!B>!U}Fn4 zLU@Oug#cgrlAiwD_484Y-PYJwL=Y2M^D%c77qGKjRw*adckU~ z4I;nuj>tvcGjV@PrW-xbl5FcKb8P*6rDTPGE>H6hd~g)nB1LF`iRhfq+KU^V^> zvQFgm4A>13+G>J2Om;;*zF^82G zSoV|Mk5!=8?rQQygDEF>P3<&n{m2E8Yw#)I&Ye5+i;KTOTmJIJi(=>{|BK6C@*zG* z2*d+}WGt&G0$uuaAq0auxXS0?0fk$>x(Fsv|B(W<*~>WMZzoE&sc`K#C`)fCX}Z z@?HEJJ>{!cfgsf6vRiAcb3PD}c&T@RgQ1K1xc?91H4gbjD2Sry5Hv@qV$+BYio!QfDAA-EY;nEuN`zlIV&`Bt+j6at?3n1 zA2ekNN7@1Fyu7K92x0K3lMoxaEfh@H3smD{76XjF5EIeJXOAH&3q)Cf8%p9-hJSp1OtO;Z!Gk; z>1Q=3@^EG*x!i1o(2$y1ZHihdS+Duj`t;%qKpZR(Vo32EJ%k@pyn2Q1=y)>e$$W!R zugSoL1ibqytgzts@Zbfk4qOcXduXviW(bfJQ1lD)^Gy+y;{Re}KKWP`al0PV0_NE^ zJ)Hyu==I`A4gAo`+8UN5e51J~oi5~)^$ez;qwXgox9eZxv0ew2ey@|WKO})>(c zFDk9`a$B_9L|Xn4?M5s1*C)MAHfDseg_Fmd|4&h3(kX_tq}u02NxFTmc}Qm3ZzGke zc@BcThu;$O_9lonRU3PIdmlLozf$bUdmVQCdTwQ<4LHC!5M3Xv_8A^;jU)U=g0pT? zK?e7YBGL0PI663)vCK%cf4CmiW@qx{*_80PrKmY~95kMgHI=Z?lm1c2pu@^xF%~dQQ zf>a4GdUyf)x!*w^c9~&&b$J=fY=~9wa#ZtjU>2GIuu*pdU`6xHn= zLDzTV_}2!mZN%=QH=sauJ73Q>o34q5KQI7FmHqwwYY!xDxIT#?1FX8bx)^qH$r#x^ zxepzMTy~$;))*5JtBHWI_S3vPT(HeU)X3fY_cKB2S9kHR)@V>++YjvBHK?+ezf4y&C z`%Cr^oXt>tdAO@q=LTM=vY3B@-+Ha^3T32k2v!ZGNlsQu#J!1wLlyD$D?cP({ps$8 zm8L6ULIObf?*^i6i3Efd!GzIxgbDxJ5E)kers;RZfUpa& zgkiU?z7eUqynT>3S%%tvTsL5zet+mz@wcVyeQ-60nA^|4Enms`_#yPeb&Uy9>;jk>&P84d+^dQ{?8`NsE&62S|$u7hqpjl{>i`J`SWpQ<+Z-= ze3zHmCm#UoaGR+GQDNKiK1 z)HH9l9l-lKQm7#gW9mAoM-rJY8UzHi!V|W4aF79ck?IYaBRpihg8HD`v<$YL`=)^p zdQaxXd|ZT3Kd%Xl)e=M&Ev=%@i{9!s+2BQzGMBm~#Nl8QTCv15cv$@M&pO~L8GU9+ZM}lfa z$;vw1ixUk=d6|~8zgSF1FumwCYOIbn6o}X^Pg>uAjTt)sU7*%&;X${#fJC2n;3A1* z)NZB=n4Cd$WFWh*XRw#=8ueoyZie(@IQlKLD6yy(nP4woR`IBoURnW zDcI$6Lu;@lV)`wtx=U1Vbp_Hx&&wKhWWdejPum5ElH}?!8Ni^$)@rxONl8A{CeQ>Z z$-bbc%QX0C4}K7;as>=fC8aqpqCz(5xK3wpZx&n{h+v|n)zyRD_b3P;z-YRLl}7PL z^f82Mc;zAi;VT)LJcz7d#Mu_REKbLHNdr5h_(3B>O_D5)+8XK!py$GnC>=q`IEv z%}edX%2ku^3QTY<-4&YK<%oYKrtf#rV=(Yo2t*(RO};yl60V)zLzs`s9-Ao8r0~9Y zHQ=Oe!K=EV2L9>&8l^#GKO|$>2W0W3(Fzz##=!yPDpY~~X!KwSWbbCJ?+XKSn5 z8TAt8@CQMyvKF(@v!B75;F|{RNexZSr?6%PaV5hjUDjNPhmS7=KLd;&D3CC4B0PcS z6Z|7)Cl^lJ6Re1!a`4(A0^Xoly8)ao5^;D!jq&J1jK0U|JQ|=opk*Te1&K53Ki+2# zepND}S`na~PpV-uJ*PR`p8N&^4usro{Ktn_6b#kIhq>1S~U_2(GQ<) zO`@p8fGvMz%TK)AVjv0UB|V#nwt^8dwqQsTix40vyId;4bqw4h8wh711# zi`>ggH}HQp!*FEI{P51a$2=5yMa2=Vf8XkUe@$@n%6gUajaT4!I~4Z-Q>3)w?z4Y* z<(uqO86vO2WB2?IDn&iF*p|SHvnyJS$KzFKHgk)qw(ml2i0MuQ#w!v{)3IlRZE_ft z(Z4>a^NTOOTflPPeKSRxzD`QPag%EpMueg-liEry^~oR|{|d9r{LW5TQ4s@tIm@6W ztJ`hHKnQyPOJR0)^PF+ceY{BR`R);Ez+2BaXF1}4FXhtGw=+BdayBNC6i)1oT5lmYWm;HHrnemPQ7(L+rg+nSoKMJZDgpL>* zb(^IQoRK1}J~UFdMu6TW0KR<8X+r`1W$rGAyaMI(=ROEZ4vHQgA)yEi*Bo%=!?4uW z-k#m8`{0^HG-7xESbIXvR=L4~B$Da_1q@!y**ZTXY(iwDLZ5ST#=Y{mwlgDWBS`5H zmidw7TWV{wcT^xC-~G~!M&UUxQ-J@`Mt`}C`()o2l+Wbr6J&|sm*mC7#P|00DZtBl z&R+rXq%YGzg$6j3H+(Nl0Agu6&EIizbBkrMe%>BN+CDluYH_kUEB(+9dTUR+AK#|~ zFzoO8NT=!O=pbqa7*n2caZNy==znLGU3T8fUD#WHrvK*6n=hfESpYizCvZ&s-@ku} ze4fdFBB=+Q2!YLbQsilxD_Dxa42YpOo7B_Nw(b~F~*hns9SAr99MMJ zN6`xFXH$a62jm=4J})x?ET?XVy7+QJn5m;=^%{PO@RD?DDA9-s@NknHCU+o{hu6LD za(G1vG^D`j99=(mMq+Yc;n^)ljn%^l1D3~;osdK!23Ho%<@K1{yLY#sd0&S6vJEUX zKr1uC@tCNx?7Pq+CNr5OXlWzPR4SS|9d=DDkvhy$;W72mQqUe?pn`0hxqq+r+I(;g zc)=)!^n~3)558p(1N!>t1fY?+N3Ta-$Ywi8BK=a7+N%`f+=9Y4>`7;e~c5wRG(g@D%e{xlSTq_ZHifrFd-W|L`fa1fes3Lc&$2uMYE z+Tk&;b~$$MqdsgPxYbt|0a-scg}M7$ib_MmTR^@@WQRl#BIh3YR1uoai79V0#%4S;=B$M9qyz4Txh4;geD#|YX8m0c(=9tMZrrp=< zy02+ce(xsQm7ics7>H4+NoeDa4MF+qF2lV4)E+6-6Gp2ML7w6&`}zAj9?4#fDU3)7 zCDCm$*!0I)U4N{$_nm(l@*KoJl|R4UIA|z#r9bg)MAbiLQ3t;%sUDf}Bs$|D)+Yu< z2eEZM%4=Z_@ys#0rd2mx-buPx04^a>QMVAl;SqMO>}m|m`@Uh$6yzd=?PzBs#2ajm zp4Qh_va;d^>$Gj#wvf2EASg7@RV2Zi1>DWf@amPioo9>edg4;zJnmqREzYF9fl&|g zK|oNDIl4{E^K?#<@gsrq`7ogGPz<{n7#A}_s7dYxtW63$4}V9}n1QkwI0at zBt#??x%*s;m7}+E^YNLs2eHimUNBy_Zhg$Li?sL@m2ckof)^4rd9U&`A-Flqe+M@Fq(blCSHN>p zR8w25Vx3rZB6SBQVFvKUg_(-XOiY~SkJe5_O7T)CkA+!kL*INTh?Aw?aar7mxA>OF zMf%*O3oR{?5EMJ4-aBWpMPH2_{-Zx1lx)g9GMG$#|CD3NZ=`w8zgZe56q<%JF;zn| zuAg%0_qKb{q$$nNGE*nCiMcqna$j*vLi_IyoW+GQ9?EpW&%Zob4$R6ratyA6+mhS| zu&D+fGJ--v#V&>?a}&cKJ+cvpFK3Ketk_|<-xtuUy>K`Z{87B2@3PANxHdWQu)*j5x z{x6k*tQ6g8l&ODYJHP%ZNP$PTv@aPWl2=q#KF2m}Pk&<^?HUB%X`=gprzQZBWY4Kl zoIM?#?MQ9tPO$ypm*?u&#?P;ec8P)^w%lFY+8T*6Rx{h^IBtL}+Hmxp#%uFEKf{be zH|GKk+@;JWel|(~YdTjNCvWd{@NT)`DU}KPTbo$xNkqqA*)B_MiTRsv`}MkNrweA# z80YBs$R1%7#OfG~I_;3WdJG#I$dEspbU|>2DDy4HqGzS%Fsbi`VOj{Bm*i``{v*AB z)HDDC5>4r6c;PJh|1>V%8t|-BP@p5q0Vr6X>E*p&hHq@q{R`WEi2xfeh!+PUV7Z~u z@z$eIE^xngn7D{;xwfJEtg@zBsoyzITI+d>4^Unv+sU?HSba=ePE>N2;f{m)KD+;>zX!XpBm51Y<;1wk-cyytNRl4nvp%O9z0<^D^j^@*Y3)R@~hZ} z4{3#*D*H9+74hGM5+b_OWjgVCx{RA6OZ36>7VkZI(|*{;AcMmv_ypkKJ5Rq+sW!FM&DJ>=6O!a~A6GR0xcL1h;EjUnyFClktCYbHD zO)Ic`I|vIqWA5L-zhqU6RZIZOdNHp12TK`F}sAV z-_{j%fBR156;5ZZxN{~$FV6u4$PD>}P%F^0N~(Z}B%SPN&CVt7ZM$|U;ol(#ltS{J z@9ZO99bG9nO(2=8ureVg#{F!;u`U16H~P)Jf%pb6=}rjB2SD;+R&CjxJloyFBNt;# z04-!8Qj%Jj2q&_~PH@g=?(I3gE1}WNk>>SArMhNIS?-^miyJ)Jdu&<#vqmp*tz9}X zNiVB9OJ6)F!9MUnmwr;V{fc?V7Fs2?nm>&nR3&AOi)=0aag05?O}gV>ejQ!QU&oN) zQeW-j)5X_@Vg|qMs*o3|`cqS6-ncN%s2|v@Jw5%u;68#0ShJaNDlw2Da>V~9xbOa7>5O%B7;*-C@_G3V zel!(CqI?5+JlU*@E_v&w?wfPmXEjx%h z#nD96C)X{)G0*L9rd~;&%o8N)N5HMydP3$tG#J#80k^TGY6}5AYEl);l`W2U{9Y*_ z>l5J?6bcpyXobJP^*jY%Q&ClQ8jBai<)+)=h6`X4lLp_CX;+_k5e?-znJep>&z4$I z6(<=ST(qWje&Mo->12Y{E_!-;;;x7M45{sG(Lk4Y(k3WsE@7yS_5zemEsKXIQ9oA<`gzoFg*s0nfMvcGKAr>3LPbNX3Q8x1omap2^XdoFfn zUHWrGzm0}HcB>Zkprw0C$fcl4XlWfX?Ex=GY~5NDWGA-i$k>Maxe%Ah2JAe$_;$7W zL@fP9(ZhR=xSJIlCi$Lzao;+2%4Koh9iig#pNz6{C+p0C;D3AU?lD$-_m1{``}P{b z=@_i+E&c@)Z$grJPTJtK^>E*jxx1a86a{2ng)ouplG=wC;aHTIxb4iDv4?PXesyTo znl-0Uta{JKEt~CK7y7Syxw= z0<1MLa#u6e>Kg!P#$MFTwKGxVy;G$My2dXL#2V+PcHFUcU3_ZpJRJ8 zrtX>O%3MBZuR%65da{n%FGenwbl%w86^O1%9ouaO0gx5{B34yIpbIX=<<3a<#~2}h z>{k%-u_wBWY#zx=ms$<#4UBKi1bU8MJ#>NKv`&UVc$uCQ>`C$2nupJs6{tmSw zhIs$0#_lb{GP9EpkJ@sflKcJb4WVPES{~fIenq4cO+893wu1-hOO4clT@sIrE*G-! z$S%SZ3s*2NKnF%_N_5rZO+D+}Q#67$+zQi{mKz`R(Y8ILROi;8{8v*=0wJtcA#Bf) ziwrM0SQ8$((Ps*O^jO<)p_k^qLX1`Z~e6@A4;?XE;pX2a$PKr4X9fGMAlqCG(a;|w8=Oq;n5|SHY z_j0>Zqz8Zz(ah=B7nC4b-^Rw4g6p@cZ&}$ZDFUX80&z<~vXx7a+<&jfs5X}xDh7flAF1t)Ljoxn4O{3~v{}zD ze-+!~@skbJzHR6EYyZxWL@i5|b@G#KMiBo{5`8?(Ne;)>!TtMn0x+q}O^6X2i;0Bz|Lo8I#JP~r3Aa3~9 zIV9gx>dgPD&h?9Jl^(n0;L!Auvu1U_2fDS=+YXnvgT-(ArRIyl(trmOEDo6#8hw%{ z^7{9Q#H~*n+`IczZt!D{g$u{Kf8NNdUIW7jnve}fTq#DYJ{rG@$m-y(B)qJ%O9>zPQqA0M=hP|&b!11d$B5%DFnVF^H~%i8N`Hj z>F2p+06?_VsKIaTRtlLBOiw3c!vDD@FT8OY{Ndk5CRD)`TYbq-TYey$$b{9dKuGuT zhyRn!LDpJUd@|7djf30Xh-BaHjn{s9#w}PeW{PX zJ?7NYw)Ki(A!ae)OkGC1^g@f|5inXXSWHTOL;#ALn_JF>H?(n{qoS`W^RM*Ec+LNj zXJ%nphQk*T4o4L=H8DA$y_S=clLE`yMSGFWs4pR?+^-mA$~9FpGu~$ef&s%iy!ljb5tu-utqUH)mwnFN%R? z^`;5KeVXBACre-YO7YT458OWJcG%$)d&~MUz37!1A3yv{-sgVvI?)nHqNW?}{g<-8 z-HTp(Yt+w_*I^bch?kO18;4@rLoWr%ivGFUA^F&)*5AfE9$5{_ChiB5oPX#*;0*%_ zz;^Zv^V5F{v3GXzmomT)s9k$)g9o5mAx3BLf*`$+exZ$frwLrW7Gb;y)}Zde5j<2b ze*Ode{HjPJ*MBzvQ)fZ-O@Xhcb>~-`RP@dfQe`wR|EVn!ky>1>wls zr?PhmiIU5=CuQwawv^{jJ{Zo9dC^l_O!FhCdv`awirtIKrKQ$Qlrt-5iP9I(q*xXo z=8|WnNusM3-yr%^wxxFI1aFC7l=(Ha1I0&9wg0eSXXo=@^MvtSPGG`%!}Z076|$T) zpT?gMvRLs6Xek^PVPYk0Mc^J;*{@%A<6<1|m0FtF^ykkXpr~7N4sOaZu3U%S?{>B6 z%gpCQQC3yw$^z!iP->wy$+fQu9MuNg4rz#$< z`|oi`b;GS93c2o5&%f|<`7F*1=Kn4vyPwLvy|*7c_W=u~tdVx#WMKgOMy2sgbIio7 zn!o1va%ogY6vJWG(0h?NBO7moWq6Cl_3Lgl{4Ih~qNdRw8oqs^ISR&U?)qz&}*mS6Rp}aZrVaAeVe8TSN}r0lAly zp0R_b)&iKF}4a?QQE&b_sG(ox|It6XisiGbLZs|M`$=nh=Z zYCOk2b}0W*A6KKgFZ=%Xhqtt;G%YP>QtjhE(?_2hdOVi1M)k-O=H2wx0nYx7yA6M? zPV}kDVcs3Yu2?et{VviTRB9){IiDI}!P#w!;6AOOruJiQtbEO5;M(_g3f&rPUh)tP zH@Jx39j2z6Ko4RD+8vwZd6u6`2k4;pFYNmusXEat45;uOIP3A?@TI+EPW8o!>~>-l zPD8@5_L%~Br2K~mzG6q5R!Clk4;&_Z8l9ClkJ7?i5$1ebSy)st6tmQNQsl%7riJHM zR9SPk9B|pEKiSx08=0B6u)Co$H(N-5R;T}gNeHOA4RNVf2AT~YS1x|KuM-xsB3_z) z2Yve51LhL?_JR)oE)w+lLCJ=HP4eZh?$MwwS6E!o2$G~tvMT)L@UQphNuAyiQ@9=n zzzdS*H!clX;2HApq35t2XlvKwXV0Dm=v4v14#+T!9)h23K$Jh`y{LXC`m!(X2E2@F z0g%N&M1e4zX8SY_s}XE#Qa){ZFzz>M%Zc#dN_i*30c(7 zFRcCVli54JADKN7YIGpI0PRn?xxH9Uw5-HQ5kxIvVacqOLYOH@Gg(;KcNkl*bt0Xg z-IDBQYH<{5%(CTkeb(Ph4_l6i&rF)(UeZ~A*uy;cYpe@N-*ghOhyU7cODg9R1+RtB z;qtXV_*%ll8wIEKx);i{yD$zFoYxd6F&K3#p1solQsAQKB$s|~QZ5VqnNPc!^~#JH zA<5z3=MRL`MCi)5*X!ax@}daIfLaI^Za0Bjugg(;^zfMd)69@59V`$eE)GOSL)J+7 zvwGc!i_6!hO9)U?A8Km8Cq)|(GP@G)nozW0@PSkw*e11P%||lv^+1$5=!z+T*jxXf zOT~s{i@Xn$YDAbs^>|6ryZL!@fzCmg%R{6m!&0Q6bD1(=y0?B#iua*?Hl#p>4F76^ z(Qfd$mh|gyx)z;anZ~OrQ0A}gd`OErni_Do@yHVG$4CASv+(y#lzMcQ3FCV{w>+rw z`5Bi&AEiIBD`Lq!L{FW_paxG3`jq`z7tv?(X}lq_lN8jzs^4RZ^0_V_(NjU`jj`XG z2A+<%3KnEq5;CS72;(WFf5GevnS;CBaRl`Q#GyD}YxJT-T}F1;l`l;4?>>)e=5DeT z(UB8SnDj107K0&fAml&F_o$Qftbx`>f`;_ZWE%eeg3i}WfOF6=hoJcb^nGy4?dw;f z18(1b3{);P-J|)-!1XIqO3Ha03-4yjRC?d^TApq9wO%dVZoV`1fNqe4wwF$MJtN{p z{@Yrv(ETA@51(X6ZLIBG6Uyr<+upywB~M86^jTkheU~^>>2{X``c}KF)r29Cly3f1 zaL&*4ccIb1)V;_?*Uy0<&-H&8K|H9rZI!I1E*36;;k^jiN0nUCy{`D5Fx~ zGKvCrE0}6_aq-sUb86*mBtW%|+eg$I-JG@2`F& z;T{H7gw_XDIkdTNkX{-Got8W#$RUyuJ)Nxb#!{I0_c-+5{=cClzgJP|>7c&-K>9Rh zpO+$Ug=Dt3TENS zGP~bpz3*5Pn{nw6Gm&lGMm;hCOfW>zONBA%X}t4{qP9S z+P4=B%NHf63}Yw%FRBCS0LcXac8$SH71GOx zCGx!l;sex<=oZGQwNlNoB4HCR?VX*eRjl4Czx8#&^jlazP;wsp`P=1-WEutf7yNb>wTXN?8d{XZlDDc z#Y`X$=+#Dl#jwQo3QSeiAk6c(SC`KY@s?ogsd5;A783}Uz zZJn^hM?YVUmG!;bB-M`=DvF^BXvQhnwGl;kT(MycWv`;vE+AQivjZF0(e^~)KTk<3 zs0F(vfkvo<7$azeQque^zoMwB1MTaYNM6_l#b>??O|&rqgx>%0AyYzi#p|9r_!rZ6 z-&b24vp-P4`@me}a8I1?%okVgHl4@6({H4=d2Hys_Ih(bZodT{A^QOau9noAMW43| z{P)%j-Efm=Da_S4f93Bg|Eu@a=3NKMzN}JB=-aXBsLZMJYR`8b$Z+jBnWUw6VC-{e zr{nyzJ=9z$p2*xJnh`*GJm&$w*NW?8 z?B;GZsiBwRhYe@9G32fxa~Y_=$vZP!>0^o-fDjc#yNQSNxY3fVJct!{l}l+ZefEw+ zQHvfZ)DV$7nLS!5w$x5_(|=!K;&lsBhVN0#KSLrrwdupCVx%lmy-Ole!?l)|b^BY|=cQutqO_p8qwbMne&jYgm z+P0`L$zPWo`Qzbk@Pto{vajKS*Kc*LC)bQx3Pdhqx^t%680#nfbu!El>vxWx?8r6K z!xSH$2XQkds8mt6$6cU-C7*8`P}B+%0eu0SssdHMRpZlDs7OQ34E z(W~a!_sL=Wb|GvZPMo;M^lvuY^c0{R=9wA5)0RJD#gQzuzz}M&M?mA%?*8)i0`*Z*D(V-Ptm^ig}TVa+$#N z8djT5@4INdmiL-iSWH!1)8N`h22P-zaR4bsu;y~Fg%9ykQYUY1xCqd%qN1WC#{_4t zz5r8G1XId7UtW(f<`0N`iYh8~EiaJ}aM*mIY)R1O*@+B+%*v)7Cm|qetr(Vj0}~B@ z!T+AlHC2Q*P}F^#9jSn5Y+%1c?0BLC&eua_IQKSrr6wI)MRSH?*hDg^OK`u*&=$ZO z!3(b`xT>YM{5FFi$2m986a6?`7iDJ7!nShx&z$_qmpOTZTYgE@I_eCUaU)PGB{dC+ z{u}O}N~;yjyX;XL-jku7F5IB*8JoKpUikM}ZIfqg>ue0avYmI6XW}0H(_!p`s=2Vz z*Uz`~m3i_Fr?@54puLR^pNAPrN_jk()U(-p#A1VRy~qWqiK?55-qPE0$AMbIuxF22 z@bTRPC3Fb&CQ9^?g!=FI!~z)Ct>TGXSD+uLpQ$$GJY-Rz zZ(E=msr0Yng^+$!q;pRqeS5!O=tLNMlX8e#sF_^CpMU^Xxr9a@D|WMmi0WP3tQ7OS zj$FG2SY`9h{L3r+lSm!>B|ntNn51PjW1x}IYAL)+c$ezSvG|Ec4lFliPCzdJi4YN2 zBWpCSF29Drr|tq<23*DfxrB?5X5nGKUOowr2#A9co#=`nA_4O+R!EOyUI5^SoMuQC zDv{oI9G%!IcSPcza$`N$UHX+bhwRNx3ctUx5w#jc?Plhe!?6pSZ+aFaI%`Fp-*Jad z+~;NTKQRzpU7x2`NY#IS9oM%>lWQ9g`zx4(uZb2|IyV;q-83Y4H9$%#p+{?G{}#=A zHVE>JMxZQrw(~0ie}shR>QL$mWLO)R3Vn}t?S2B1A82NP|0Y)R`&coCip|&qv__n@5+9UP=2gi4y!-%V}@Pp_n2&ArX za0FF7BizA#{I{0+P?wi})l_p^K~WJbx*Er!8_4m9Z`_~Y!Cf2s=n?+a?c%!*g|Jh% z_RE@@_uq{ea~5L(&x^Q(MYj)sh+-BUc8tn;Q0aM;?m^Z@ntS2BBq?)ygS?EwSbqb@2F3J~yuXWGPI<2)J3FNscq} z0sKY#8>-VSjErV|mh9XXWG(_d6BT8MIc`$dE7xF?^+RDP+Po2xV=IuZHPW;%+*=hX zWXhIiSaI%c;vWt%qc=bYAhO%DiB|>2t-|)2%;?e7(%g81Qu_TmkpWChyUsjDX2AY$}}M2LDMCVAC7M3+>0x|7(f1v zUocGE^)lHnbc>z^ncY3jzGfvwF-#Cf30%54djW}`O4A-kuihb+$lQmo|FJODQ`OzQ z2jDqcw`Pg3<(G)ry2{E`z)Wf{p-#vc=x?tcE>U##Qr1$7=s5kXe?udT#WDsWf+Nh* zqB+ak2c2TRbT>#>OONrrq-FRBhy@d?@Eok7K`s$?z5S$B_58WEnn$)Y{y22epnKwY`7F)aCxM51t8GQWu^}i=3@>N335AFCg=mbUAghy%8A}+l9 z^zSFsYreFr_)eN|suvG@`*z`o;eEA~ydjm^`UD-xT49^C=mX{IW1>Q4>Yd5(vz3(W z7IZclcm2`t|K@^%&{snTJp_r|h=%IyfB2?f`?>d>zRc^aH$IF_w<^y1$F0(<`l%hg zqRt)RLkEH%#YD;v4GMqm=NRB{^)4Snsm=^WAmd5 zN&LGV#+~xIu5LV{{d1hKJ~jNywWgz5<@I&5$UWo5k>jEB?$Jh=ea`1oo++Nn_s8$8 zw4=7p;+jsJkime}l4G1sRee3Zr>E!0uV2q$n(L8xPKkvyUp?s3sc~dlv&KI$G}uO(1m1~L#L^SGyRH)Va5V3|zkb;c zhdA8B3)}Cv7lXj~A-G9`$LANDhmx}X=5L*7ajdnVCN7$J#BH79EcM>073ahumGbEC z@qYnUL$loHQzSG!c0NzKiqAB5ZRkbwZly&E~(lTf@&o>t@~$Q%I8jq?GjSTQt*mzEQsZFzHXkp;%I_c zn)nH!A&RQrWFr0Nmd;w06x;FrW-_4@4yZKoVREnjt^V7U8YUQUfVWd|K}_hM*7-bT zuh?19JVdmltMP(XRK?hq82J9z^tpR&=l15%n&jj>k8R`u>1VUmq@^h+`c2T>VivNc zjg7iPKQoG`c`;I}1uOQ4{eP;4bHGUzSzm(-_bG70kC*voKJpQs<{8b8LH)BGgMIOc;!;gUT zjT8sUJ8XUcfiOi4iQ4GuwQKROJ_0l&Xe4R@vP%PLDu3|iBRtUYZ*NxYwW|MzvMF zo9nAo9vutC56QD^+((29z&VMR-7#P)$kk*Vn9xM{zU335g$U{IG~VKoWhUqXz{hQL^$&cud@u zxw-D*p&j=H&DMdPsOIe$1;+6H?rb+W3_VH`kwcQYK06<4O#t+tF5KjdyFTd+s^#{V-ssrT^ zlE9T3PC;v-uu#GR1N`{u$xb_X5o|=N9W2by=7Fg8yycgCZVDNNg}zjP{Re7Ld%P+9 z)pzKsc8WQ1fn(dx!NKu$7O>W;Z|G|{VA=r9LGSnPgem?BBqXUDH412qegq1%=vOcr zw2qOd0xbq8xsN#uBd=|ak_kN`MLlO|$gTNwCE~#aJY7oElS}6-QbmN!>mH*=e|Eln zvAEOYG@!)0pFVwJ3bR>88k>e|@yi1*ID{cGfGG_CmfxW&Bx^lTU;^^+$G|(tu#3YQ zc%bh-FrdNrNZ)cm>2a$y;H-4B5BuCXtD6sSQqbxz*F}jI{U~ zh!*xK9t@hPog$aJcsnQI8m>xq2kkndG)2FFM))~U?~GhL3_a(?AW&?uTiO7d1L0u| zukFG+dHVF}G)k+T2#i8i>FvY#gD`$Qg*+QPR7|`GK^+IQ%~yY6+MyzJzdVIFKIi1+ zC4dHr;L^PA+U@R-$BPA!B+Fw6X7yvz8@js_=9bJ^BGA5Ypss^gMK9dTpjJKpSF*$s znp(K~1maD0Y>|@AI#-P0)cyPR(Nbd7)<~m0jrV*AE}a52O|IT>X#)&&WaywhRDu}X zh6Riwj&H{!)N5!ZcnPztpV{URCyu=)6334NoM%3}>K8|DCZMZ5Hjv)NjO zHhTCrNP<+Ls-9+@+hxddJW3fxp`C^i0>Uxp0)0a@0WfgiEnCwWacFYx#S!5{?WdVE zs{;LQuQ^)^^3tW4yNi0p2tCMnT)4nO0N%Z^SQHS7<=gxQv$=s z5MBFGyKWVTgZSTRE2$^9T0<#=0FKVhk^~`-H>j)EkEDG?eSzbzk!z}muwv6%&1Rjz_V8aMFajz>~+vD%qkUm15Lo3ESFKqF+2(NqIzIP?h`*55Fz_AvdfG? z#PjLq_@HOjTUFbUb8YNjL_}w9h=mhv)+(LAkeEq+U?PYe=*MeWcs(Z!{Reey02U zI%25B7k{QC2yMYE|ApBPM(M~^0jx=+%(#SqMqiCR)!_ON$s29PLgsHcMNDt1rR>K> zc9blr%2%-mKEoDd8C(JGb1x39TE9IJC!?YKMs_h`zCC<>F%HP3of?8NncM>Ihp#|i zKv(huQgA|{NAVjpVTm5heeImfIMIgiRP&}GH7ntcK`|wUBx(YsUx$a+$B}*R>nPG{ zF(E6+5EJm%_O^=(qU7{m!Q3P=id7VzQ02J(6O9kA+T%|;a0W}yIA;ks)(Hy(rJ1w&lWxxLdc=p+1xB2^W8JT4~hhS9huSuE5oMzpj+HDdU@w!3!K+JzU{8&sw(6 zW!GL&7jO-gF-$_5EDR>^%!+>06#`i_pWR>lFYms7WkK&zQc?mAQ1<1&eEm%3CdsXc zN4__1tN?&!(?GP{{05{dRR-B45zB&TEet^!#SBC~wU#sFa#_ zNO?(gcXy+3Xlv^=^%SJpJ3#&u=%lAW?sn1#^D3s13ntpmvpeRXzX-5E96cT2&!;A z*p%&E2Ae9!%g68(^Q~LgiHpAy{)aZ7JVx|-Xpg!*$Btt}$s|CebOAC}5&R{b$SZyC zH-7uWJe4*xH|Gcet}qq#aSGxXLHW>&>xy-r{sL>WRqM9B#zVu5*1GL{AK(y>D{&`S z=1Go98YKtT{Fv@rNa4`n96K$ypE5l&^Bn%Z(5*b+pfSenX6N9*KAYx>S8&up@g$G= z13Xr;x?1l`yhtN5J#o;$pvLq6J$4f;auT-r973~V^`}(L{S<&0x71SJ*&G$J{Is`o zHJ4)3ZdD`2NOtaKoGTe*G*+)^HhgNdmK{dp+bDatQgUAbCR=gg=tDD36As9-iKr3| zGPW#tRDtEz-kgny<*UYLw02*B5UC{hg^_&qT8dzROjzPsGt`ZE!Pj5BI7S>fa3E~9 zE5`_$`6Q-M1RuC?=)tgvTPsoF!8q_r4Sa+Y4pCT8MI(}8W4uGHFcy!D)h~{AJQrOt^xl|0u}v8Xd!6Tjp*M@4G=y zlY;hwon3HgX{nh3W~k86>_!npPkVfMNQ_Dn#u)SXYGa! zrg-NOJZJ8J>oOa3BQ8t87B1rZp?g6iUa4toZwql*=>x3*ekpEb z1T{oV+&g+q4FZ#nhWjXmfUyWNI}RJKB<&w$Hk57LT7G0+pOuvr;0txBN#!<%{eT}q zzxFZNd+tA1$s21N_k({!A*ElRw|di_I%ryNB1z*2_o3=J8l)ALC=xkB4Z&+gGDU?{ zQC%GXg$GoFwg&zYnbiyw!R;_NL)l8yP&=_L5&Z!>wB@&N(*fDX1TzO48%?ZQ^r%LT zfPRgd4J&K>z~wQ#nyBKl=g%S+qyRvRx_Grm80h^waoU@kQKWT`mz@gOqHexXKBgh~|c%|K*pL;i}`btj?@Bc$5StZWS~0^>_ZrN}~zwt`fMG3nj_)O>{BJ&nV# zH7M$)jI#nzJ|ds2VC2#MhH9=*EWXTjY!5JTV_Z*E=^N2i19h$O94oEDv++snzm5;> z0Nez*ix?^(zo3~yvZ%!SUF)Nkl7P{-s6r;V8l#Gd!Adwt3BKcxdSzSB0Y#e`i0>O1 zsDf1F&-nO0q#8(%G|R5KlL29Tzh`ZVi`^j(ai7soHG=sChPhTcPQu-?DRC`Y4d7Cr zpuQvo4QUtf7p5^*g6Y+=1KqD*zviptCkP~A;1L3ow9~W*%ZbpMXk(3pSr;$WGbC1z z%w#m`2yg-Wh z#|Ih&h{OBvmE1^Vw{?4)HwDV=2Q{}JO(O@7cIsZZbg=;dJAeyuUO|F{8Jl7E&JA>Q zn|%Pv8-LEZ_|{T5IW5n7$*U31|FGBmv{UrJO57wyA6S|dIddIlz_2FT?Uj2`UsxCL zv9nhUO^F;n{3qZSQn|qpfX!($|F}Nl&`-OW)C?q$mA;4N4Ww-XEBBHQxktk7!f*&5 zx?FyH#3Kv$Yv7Xqz!q>VPz{>f#NGGZO8^Mle9*;MMHQkY*+;QQ_)KXtEV>*^@r#X( zMFF-Q7%T%M=v#eIoNdRuVnle_0{&EINgbJ&3m1!z(%aw9j&hTG*HIfT1SN|zcK9u2 z=-6PK%SAyc!&Ye9Y2M)l$@!|iFI~qD`fdIW)P{DmQj!+H`F0+i8*lcaj%0(bDmtkz zR9{8q9-_4rX&CwKzjSm2;2P--kuI5bGh!5j3>^ati*3Pq7#I0cAMNAeQNm2w7l(tn zq`mls`mH%P?#%_atj|5d_?cVL>Kc>5BYHTqk)tq#tOZL9dCa$WYPtOVp^D$#STtH_ z$A+hPP!nh6(XpcmQnc(ErgiTw}!7F;RI0Po9L)0)dAV9O;=b z$-szC$|Y!CTxm}Xo1kD*q3>sGSSm5&`qBT2H~ed3`n_3>?I(b_ykb#|`Yiv^vOfxp>dtZ2ZOkeU#aEhkc%&8!yCbPA%g zK5FyX2N*0E%wE*0gKesk7XKzD>Y9#1BJNw&%wyIMtb$CTQSxm%!T<}NPE=sP+oY2w zPR~#(VL0^woJI*vg2n@gh5|x2W_Ls=2qYKqXLJa`AY-=d*g>WlKVv;7KHql~tWk|_ z$csBsr~r6;ikcYur7b={S17v4v?%rR>b^cu$<%4iG!P-85wLj0w&;c^5!!#yL3f6@)g-}0Oi7{h9IP-tHJiA#Ut|LmOuLfB4rzw#`f+pzCv0qh^w=$tFt zF6xnC*1oD>`84fZM5+fzZIxi_5!H$52aNlx5OwRDHLQsSL{v}F+cVpRk$NALMZULh z-*y`QujbhScMHlf>;=s-i_ytg6_#A7n5@l1Fbt9$(MS?Lp@HLH@oAJeRv>cF5WK@c zJpfV$9~@?2Iq!kMkr2R-mbrZW@ywpEUAA`1%e16kXNBky;okJO%`B-OcOR zH43A7->Lzf>Hx~`V>A4sZ7nDx=_J(5RZER(Txf{Jq*4SITR5%YUO%MNT}fJsD7XHv zmH&kI&kf!OzvuG^5T1cJ?S~!u6*6AlT*qh(B%-U-Ga8AubZ8kdNsBi^V5$BBP~`(U z-?5Uv^bi9yOq`3iBi|V_DAon~9F)ZGLqj#-ui|x6NDEf#5s7xhVQKLiv}uT>Waa{$ zZWFkN+R7gTjC=Ob`$R{#I~GNX(NG8=fWCmi?P|215{lW5A+>iY&{#yh>Y+i@V+`MyGT|?DAyStMf;8{pgiC=VOs4PS+flJ zG~!xXwLRMj*7OHi^Nwd-NY8x|zi76aZdDI{XkP&M>)z=Us>6LLp(G z2^KFsEe3cFrJZb&mf7dtN1-U_KcVS{)KeLn0Wx<3HA*G6NblTPjZ47injtPK`W!-( z499o`0vd^b^Esx~VL%56NeQ(lSi8YC4ohBz!;$ za}@KyfE8?c_`l0?lEl+3QNG21^1@U-8(eT4!f!x>Rm(vzAnsZKdEnlNbp(Nu z-FGyJ1TfQl>#!2u^?G)+2c8eV(LWsq(#6y^vnhD&s3(K15#RyU4np2!H}7^3`DbX* zp{lF912QrbgcpKUqEfql;|71Nz_}O1d91ngp$&MgY{8ajB|6{D*U<)Ms4W0S4u!M_ zC3UKH+7<|RJ&nbIG9;s75fJZcmvgAP%}gU5mdFINi+jlBj^q7`@VoN2`8pGC1HO(_ zP0~#ovuHe{#J>QG3W7?RO!p$FqV20);wMs-7~D*7#w#cpcf+4Nd4lBoE_Vd*XEK^Y zy*rtRQOGM>d|>iSW-CyB!YC9W@O9J7*vLp7JZzqcePgqVSdpYvP5wnrO-VIV$6(7E zD2YNZmiDjo$I!xm$r0k9N<>P9v0Z-_`4V^%lN&9$5MU_qB7we>^pMCO(A`y{p+w7o zF{VshfWvV*u1t8Z`-wcdaxY2FYNM&V2*Xasu=^tdYMwkWMZ=b}KfuQ4o26&W8DobUE)0snF4NY&Cm0KEaIIvit6=5t|^ z*N5+^<~oDuflUAkw_JaDdU$NvwW|h~l-sDp?N%o@#10rHK`Y4rZha=gP9G{wREH1H zPw1)gN9qrW|Bm;E?g@&(c}S|kxNU+o-Bfo9RZbMcQ%X$~|N7Uc<)?w~`y^`Kz`rYl zw?eIU_Cfqdq)B(w0(=2^)mq(pKZb|z>Sq?RKSEwC?8HaR>&;92Y8_CpsC;nc>}iXWBH= zw4*|8TCEGH&NU8^N;l;a6pJL>W1l^n${%QXp4eWfpIQG~{1$s@^)e#b&=28EFInUP zH<541qY-NE^tu{w%SNd3lTpeG`6|Y)BOUsuXKXo}3Eq#(FQ;AwiAHsrV!562o z+uo6`M~O8QRGGN`Rp?MrlIhgqB#@P7vUul2I;TD04gK>DKnFP8gyz7LwitAk#BYTp zYCqU?0Kzg^)cV|pFDJIP&yZ3NXa?9jJUw*yjn)N3?SXSB+1#lXY?Pfs2n3^S#1bUp z_Hi%7zlqVWyA_m7^$;Y_q-zu9*M{9kH!giPBi% zWjJ^7-z6AfnO59p=&y;yDLu^-N!|)TYhv66;K{mR7xWrwOcjg6_p11?^7 z4A0vXtsC|R9egK7b_L>p9YC6o;L{63`PYm~x_3}56JjIcI01XuG{aX%aXC1EJPeD+ z(nA6SM+wZTZQHa*V(-C&^@1;b3Y`-}ot#b`%36q(d_Z($hTjf*<3ai}q?jvcRNnXa~7%XS}9YhY0+h&fSaXo>oP37NpQ6jga`S<2=8 zW^3jEK|`#8lTc@^H{?-m_IrBvXLA@&oDS!V^H5J`M7AnN^J$3cLctBzdggW56z~bz zCSDdOEfh=-H!_S>v*^e+O$)ZF>3LVju8?T7q&ODutS9Fj?mY?Z9`Y@4qg{dT!{|ci z#hmpV!YaduQnIEJC&S`@jg0UG_a6=Gsjav>qPi?!P!Zm>kvI$h_JIzX*o>hTvt`jK z3>!bC>T>S*t2*hQ&&+jB{aQ3VVy%Lryf2jM8~max?QF+bOhSf z$vS8C&rA$#COAukP$?8{TgY|6yk|J|{f9O*ld%ZWAzP`zDR zJN!GyaAHsUJyA2SqJ%D6!sUO1VSmT-QVKM-K;@gI$O-K`( zP`hqoIs)%S7Ked)Rlq~wN>_!+O$Xe&#}4%!lyYdyh(A2=Mkqt|3tF+qfs`6WMN{h^ z8e!V5TF~j54n{+pt>fnIK_!6Yny)$ANd3EhJ0BvD5;0y122<5f zomz?UUKme?g@pKky~xVS!Yv_UN{|J_EbD-B6nSn)!q6@+t5@SgLN2(+&bw(1h_WBZ zl*)AE6JoK9(c@eupExyY@Zp1{ zyLr2!7rX0riwOlosegTliBYBzAY2S;?3RV(YY;YTb22Bop(z)F?-UBbCcfIAyHb^s zJT9^{G<fk`_m=X+%!AW>|fA zn5)*3ox*Y2l|Fc{50=!?rlhtGZCLU?K1hClMHht`G0tl#VT7qUo`){C5$8s7_UC>8 z8U_$FSPqBeMk3idj1E0hmfTsRLi-vpBMIkCb*py6fXUL53&kj5FrPqp3;Lwp`b8^L zh`L5nV{{Z4QWTZcqXPr36;=Niof++X>eq0pK8Q&ED^-A$s#SE^5Lli?45tV$k}T2TqHC6J z3aLOrmH?za;-KLJ@`H1*Lu&#c=>KtLelSGqP2esJXIq?VC(sN;BYminLMC*wA5(s{`lTW}_6Zv7bN5F45N9e)_$K`F5e24`zbkDVVhk$u0N()k>6LM`0eXh|@eLVmIrrlC z>EZQkpc!M74t}uNC!;fzvZrxhspiVKqhfFVws~rW$;s6U?lNO9emAGsS*@aH-maW& zvS?+-J;l#^IB8y##wu)o-H$}gJMk@Q@XN@=z@r&We1nz;$m1c;e9;DIyDy}@x4;V@Ln+Y16h zkm`+&4FXHa1O??(djxX#qr>x~1&H_93AO@_Or&`XC?3#<-|%F0TuY!`BHjmBl0uP2 z562f<2#amJi`CE!FaR4t2MvF~{@UnZv^t-j%C1M{A9b0J-~wXy@q%V|%u?S@B$niP zfx{>5!eHC_wQHZ?FhGPT^p#poR+m5`(X_C;1i-|DSIoSaK*GJXDmFNjfu6b&gd62E zhWx^Tz*ke3Vu0gN7iTR)IzTU z3drc?FUEuC?t_(~$$_{FSN)$%-^W3c7to`UwLJ(Bg+#ALJw&F8AebROR51JCP>Q+baO)vMR6uPGTW&YE>Giueo($_|M-_#V_7Nmife_m_y|i+? zV^dOZpj>`}dZ7yKSwLW5B``wu$o-+sB9FO$1YXL1uk)*7v6Qv%#-Xm&MJIYEUYu6- zu8#jXBYtIf*J{;4T0l3yvE`DCGzYPYy$uE{&zT^1!NLN%*>AAXyNEgog0c5u3L$4O zt^(nTAeO(Pv6IKavUhO!j;0!;n-O@YTIF8Xh!;JS4Ao$x;Yko4OAVWN;D*5`yV8H# z2@+0RVj_AVfrTq>4q6h6UG9R14t*M*d5)OJv@;jeBMJcXK#Hzhv*s`Kfw04EK%6Xg z|I0yeQ)uWRG%B0T^Yl=eBwHn*n){9mA2kI}3x%SOfK7kjET7Ew(_IshnS!Q9B2Job?S4J{`y~LAR%3@U~;bNRfn`k9{!eNJ|(Bz_#HY9~GcWGy=6H^YdGHWo__#Y_t92N|Su7&s9F zE2{u1DL!=94@bxDL7YuGv7C<(A{z`LUM}5yKPN^emMImu$Y9YiFAMjQE%4LMonJyL zlbLyv=uOalL{a?i-o1;NNka8GFoQ}W%BCmUJeAN$@KJx9Dy18eI(@&-+3Kv|L)nRV z&r=^lir;=?wJA_={SigCbMn=pu8q0fLEg9NhK`(JXMSmX1|6lW<}vRR$d?9~c)HBJ z1o59Nh~s#xc$cwmegYoz=se?kAH>3oQ?(B)3vv3cz^9kh)@~6h;Kd->W{`L=!r=q8 zVS48B(TB2lQWXFY<1-duxVv5&X??2rODH?{9K33`0Iy{L1-%Uka;pzq=gV|4hs8U# z|35`UGz7uDLqj||*+%@3k@#Vx$XTHS`{8qvP76R!-qzcH4Edvwx%}S23XpUk(-T_j z>(wyp$My9k-YGO1HCpY})rwGJY9fC+iQ5+FXOi=T;GY9kP@aC&?8>j$;X+b*)ZFZ> ze$>VthxV#=;y!1-a2yCa6NBBq)zoU}XaVBJFs~mnezJN_>(CvM2 zXbK$%ZqI@D#t9Ku-1ye#Ms}h&CCSGioui$#0GM#s0uyTreY}- zOa))=elC#r6YaJ4FW0l!%(lW?S!%5`9YgujQ@SA$L-x3`1Af(}+#<5??}P*out%o6 zs5e~~%I+Y-QjEzlm3jJ@lf=b8tDSMA+s54^jak%PB;l zn?*%M(->b9oX_G{Y#Ls6=zUojJ%Uk;?FeFOjvECsP^`$}OUXZ(TcQz077%pq;=A5O ziN9{mnl*>POfhWPGOkoeRD+5hmod}68F&gK3UR^pVmDzCp_~$@7QGn#RU}d^Wh;{D zJ~1(|cZs19u0Q7H=28p>P#@^9WTB_@9CA7B(zbf#%G`@kEQDX$NF9WDrF?Qh$MS72!%-5q3o10LQ*1>QArwh{FC1m*3G!O(p(qRAI4Dj6P!kh00Jdn83 zGzI4L>r~#+yX%dv-Cu`SWQ0uv@u*|cXtanxAvN?SHw0L26mXvs#ZY^DyX^u|V&IT$ z80|+2PDs)omsg;vAbQ_lodF=30$ntjn&HK$V<(iDo0}_Tjp!HD5KS4}6o3nP0em!! z(X)$xw5^##rDAN>-r8!N7s&=_Vspf6(9C9==-B@oUIcnHb>_BFLilo~_Cf^I++k7Z zBZyfU%>mI2Kn)f2zpFtD5C(7y=vYakLtNXknvGdOGb77KAT!2%zikxb(M9*`%QkkV zoGstey7`W14(HBjSKiz?sj5episfuIw;q*yE0FZ-I$IR~4%Cs_yP+bcgQ{&jvo`)#H<>UvcbVJbXA?Mmp*^y@KmbPzl05u-WX`bDUic zYfIg0gwRI|Y=wEs;7sn|@GQt0`=_fsoCE3je|T5`*uSEsR6c(`Azk^~wh+0rFwMl# zTulK?JmR55sRlNcd|i@G^p%qc#y)LEwQ=L&_aIp=H$_B8f%CLD_&mKCHB?4HaA087 zADhI6;a6rt6?2H-gh<4tG6RDC1poMRCHKGtFmB(jfhw2+oEsuC8udvM@Z%QPC^1lD zU5tK;uxVb_N%wc}-huV0w>1wj1^G$zIr$7usU#YzvOpL?Uf+kQOVt7=^ z6^ImB)_?6%xtkZiuynZQCwy!RNTer(7XcYiVkk|i4h0^f?>?U*_Zl)W5-kDU*ab9i zkZshb8pw)^i+8$`kLp9{DsHFYE+z(z$Ts}o{a^#K!>XeWbAo|OqyL4MCVJx}<2AgB zkz2F-_3I?^;fQC^Nva&+N8rk|q&E=*Ye&bcZ~9#)#BsJbw@5}DH}W*VH-AEoC+NoY z8lIjg6_ZfdK0bv0B_m+BF?56=3m(rw`AoViAd+N$9zXe&#u0t11a>mh;}_ayPe{&Zu| zxVgb)tZgfNcTz3e@PRL&Oo8KRVKaA72DlFh>b&vA2C+&T1J(LB+na|Zj3O)l0dh%v zL9>vW1l*wYfvc07C-sHc;rCXV#M32ew(h~v3UA-ilJy-k%#;K%guYp`8e5h%G@7C0Vxxk%phLfJKZqid`K_r zqF&kn?lLB#Ov6FIj1O3cXRO8xJ$=J>ra z1=i{$_~LT#!QPBYq{1Q%$m5hI+exIdpSFJkoT(IrG|nxFYLg2})Nq!RqefDJVt4Nt zHke@N54W*4>l6bd185N+kBLX+v2KbS@o~B}PY~$IfR?VC$|H-972&-8|PXLb;e9&g{v_qgrYrm70@%-FwmB!J% zrLQEZdAyMQ&UPl<2##^>+%~y@9-eF7t>=z6G~?m~mZ|HcX$=}|m(<{MK#nTX9- z6u>?kx>**?(!71Cy({1C4KfIs3CeBKAId*mef3X|BAQWQ@XDW{w=+APFY!-!kOXCw zB{X!rP#*T-gUP0^7z~k3BvOTOsN_F$$nIvrQZK zu`231@0TxMChkEU#w#H)d@rcgZX3>|$JVfFZ^a2^e8?;%l)L`Hh_&X5PF?4>riw40v zG&D5XI5R0-875_+UAvA#O$AWvg2)52TrHL33+hk%4V)96SMgqL-ewWtx7F#nj^B^J z6<&-=<@SeoEFX`vJTj0@Xy5NZ8@%;V@hdi_Bk5axVwI{ zDA1+E?c>E&KhyGfJ(azd1ooK}(&hFFkCK9*;voGV)GCuE70|Q47=yM{fL@hx=~B1kX&l&r?#w)bg%{{A=mCK1wB+TCq#Qgq9hv_~0NaH>hV`Z(!=Zu?i0UggtX&Cm>Ow3q*tuS`Iz#os1qP*n(a5oQBMb8JIXOGj3 zRZ>bZ3;@knE5ZDLUqjz$?8iH03QMSa$ysvs1u7kLLN}kdQI)a6d;W_&cKut?;Afd9 z*p=f|OCMe4b*;AGsk*>a=$La%-qlp4x@}LD)1`?b=}-KAe$mYVehkMrr7x1%m<(2#dXKzj{p{Ndf4dp*{WAc46N4lL)!=MU!k@l~;JW~E z%uP&QKqVLbRO*ucGizX)Y-mvWaHohJeuy%kHc?*$ho%rw5*PO$tLOZU?M;@NVfvYq zkd_>AbH@^d5o>rO5Fa8YK>t^ppus|n6ENZ1aXqdE#(|!ds0ne2XCiKZ7!6=4FpknW z49zMI5ICW*_~H%*B<_bLf(05nj)v6N4)^J3;$Hqj|-646|b$z;S5Kn9wk&pl^HrmR>Dxxv0(Um;}3LB z#6=fDTHZ>W6P3vGvOk~>GHnA2y7Pn;Knqh{WP}J4=q!xCK{Z;6t0+Y`cDB7=G9F|%a0CffPvv?(WGwFAv9sG*)%4F~x6#atbeZC_ zY}YPI_K$l%JRkP7#WSVKUoo<$iTC%Tx%|=Y^WPPpi<&=k%w9_~ zuKw+o$+@w4MZj|h$4P#Rxf@?SwaVsDjUx!Q9*3=2vJ|1Z4Y!`S4ai86XG6}dZigqR) z`Tcki@22r(P7#gvap=SDTNGn>6GrzGhUywQO2$m)ZDuqw)^J?fnoW=&2#9x_MS;H>0 z;%vp%*oJj_(|QtiDTr2x&;<6q9yEoxiwDC0m6}BTg%vAHOv}&^9=;~`zmlP*vojJY zc2m$HID#F2*_FPWr#Y@WhLr#{-&+?M{ewbYJ&?3RjVDwYzw@UG%0+9a)Sp%l3N;%D zv*I&RSe>`RFrCSOxX_H}Xt*=b3m_Qj(m#8%w-+NKfP>MJ16?A)(oja{xSYoF*aT(- z-Yja8)DI`)KMXny{vc?i)J;rwK>u^+M>v(!us%ar@S8RMs9eR4q|hHvMDAz>t396j zCEM<3H~v60no3XdVDF?go04x;S%3R@xxK8nssAmR#=TE&W~v05ZxoblTenwlz(prh zerm`?#x3nY0$6be^Mc6KZ%QA zc;6d{O(8QSnM+`GzSOKqzkm)wGg00?JV9FH+__z70j#0g!UyCF4_rOY|lpa1YfWyV6zmtmaIo>AFRgCt;&30dNSD$52!BkLr`zVk@2!N zmW9fj1xbH}Y(ERWID1OD_vg`>Wv)xvy{ngm-nKC2&k7Iv{m80RcTsYix=`=8K~R$Z zfF|?rq|ZA-kK(~n5WN}nOc)6`yZ)gUuB|$FCDC$j1c#{Eu)y#Kg$Rj-IMublAdbo& z0r8rs`=M03%6eSmrZVC(a4b;qpJ14J^)nns60=;aylIjZ4xa&1lpXNcG}D@hon8}a z#E;-b@dZ{A5wNoL;UQt`z{4o(xhjv^11G&a+$U{^Oul}`(Sb@bn_d~d!Sj$w6L<-W zQ4+pxMAndunt#hujl3gptN$yKNO_1*(GwB%t#{!MIXY7?R_@>-e?B|u_&Sr-Q{0b| zg|7bIR8=<_W7sow#nh|jSZUOB?(7w%stLJC_6F~#-uZe%bLH=htxJoIZQpP4F5Hw; zlFl^gU%vdFjp@lnF0b|E$CMQ^vd-ED2J?QvIM*b@+dFKZ5PJ#!Lj@%&^h@*l4}*Y2 zm_xO635++|cg;*Yl=^UNw!P}_L{0&jZ2|0ugieFClX2|1bHefw|JQ7=ma3?H zFy9+}{zVIUV|TOTFhe8e1J|)agGu)r5T6fV2T3pcFTw6aieX%n?bqkhVz@-N*YJrH ze(++%+O37PIAkW|Ftj#!C2?p-@M52FTQ2~L!T^Wtf4*^Y(kNr7(9gGdAv1Kx1r72B zG}Bpjn!71`9=P)*y2bcg)5e03Bnt5e|5Xp6&vJ@*dbKg~zxx)B{czg!()EC!>1~P} z9(zC2Y;HWUxkp_7FY?wgE#=0W5BK@a#&4s(|0r3uopF=ZGnp6dMis0(9~>$cTGU#F zmT>bO0D~4%9E8;?&;!4!3wWPMl3*m?Rh@yYLW~cMnlf(0mDi9%pU@~eM(mztorrRv;I*s)zQdqmX@d>B)yeSDEN-g?4=m}##UOSCeqi;T^6go-Lhq;s@&VHPHQ_p z?cfStdh1%gr_*{WEv#RLT?HpWAk+&(YR+>oq%GH60txfp5Whbx0%N zKFo6+TMT&^h^5tvZD)*tUz9iHot=lPry#|g22N;z$g@-2CbTFvNze&mqGR`(JSM8^{)aPt#8dSN8TK@L^ zofC!XfRzzZkXtP`2fFAl?kxndazi(8d`_|lCsPhxaWofsIJmuVLsVefw!*eWF6Bk6 zhe9L3{lJtZ|C1VXwX?H(-2CKUeZ=bO>Pk#vSd#iNN7oSz;91}EEcn-{_1Rcb*O-;Q z;FP;rjZHp<7Q_HjMiihVD_r*0yf`E}+4N)Hn$QV5CZE^H5V&7*6e4z|gu-jHE;;a) zwPo8sFiLy>Z;6=}-#&c8c*fzZf43`btfxGGvB5}=*~o+&fsu&8~2M*?dFI?u(u zHRxAJW%<3{%qw_>)8L}>v)r&NHBY5zYhK8xLIF;KVzds4qF1%S2O1AH2T`d3g2Tdv)m?^N59}&~=00V6Q;r|UGGd{>FBgqsL z#!;J&@^qCuJ7Bv<5P1ireE4KP0tFW~ND53>zL^N+kP3Qtct>*Zs{@;PM?3@gfXInq zN2CkiVV9GHu9sjK5Lp|e9lk70<%V?*$r$En1mP@2oY|I-K%ww`nVsy{?XlrvQ z81I^4dV2CoGuw<84Xb9)Bx5D6GMP)xGPcv9a&A}dtnD+RW$Y_n3?63gr_OZ`S_-zj z4j|@F6-ST{LQe^gh`UVZ*PL9qA!8#5M?h+7lQF31LLQk9*@yCenNBY zS*J$oWy+JOCQO;z!-gA!=$LY`31gIEW5D&G;pPS^2524F6ASvi&Ks`~6-Nnjd9*zr zL7i`sj9>c&s^or9Ol7CN?jC>q5vxJ6ZfwGajh7f6*HNc1U&I~eE93EOUcwrKR&&CJ zAictAgV;i}K4g3s1w?bJH(Vtv5FO5k>kRQQ>|hZ7@pttYqVNU34vHkDh~H(>Q!Rgm z4Mj&&x0SuSeM9TRRk*`sE8YKs2CngCCg~(&#Z8v$+~0nb4vAD z>o@VWDW+@4J8rjf8`K*J=r8i;i5kNX< z*c;$GPtuNjW)=Dy=jTCerv$X>*ph@^OtV&^CYFN#k3RPi{kw4TAPF@^RD}mT?EZ+8 z08S1KCkbr(-$}sA#TA6PDJZ?+SVU5gh4>?%Ib}t&d`ZV#F-Ui_&vgc%f3~hMhgX6jnK7Ev{lSd@Hg)7uz zu3j&Alsrux&PV%1->7|CuezSh6N;sd#}e-Da#O$D^mA*Tu+E660EK~YhZ?HAPblah zYGa1o4-N{N&6};h{Kpef^xq$Tq<+{ygNZ`t0WN5aLA-;$cTkp3>8XhBx1)}Z;&>>~ zDqV?`epGo@(b4-?#__uB;nPG%J`aU{zhmhd^ zmc~sFc}{O(TUjOpN5EhEJz&%O`BnMD=OAKP1;t)cQ$s&}pNbO6Ey?L0H~$~_I< zp=g{)D)2`b!L(C)&1veugYYZr{O$<|xL2+c(D(`|7$=ZHO@en|VTQ3HNG(1V ztQISjdGLfv+BOQqm|;G@|H~Cp>Vk3kmp(~=-84|b9& z1%fk_C+lspWhEXaZ-0I?_GAC^E00h19o(Tl(5tju%;q@O%6?)pLHm$_BP&yAi{&ZN zBWRNLFH*7OFN%*xyRPT)L*YvA^6lKI*tdy$<2}VKtRDBQ8VikjJRpH?#JCAq_DiS` z1I{VW1*&6UB>Dkt;M^c2=!I6?F3GT^K-JZnG0hzk!WuHMx`s%u(^$a(K#2S2@+Pp#cqk*C2FWpszBZ8^Y4W~PsCOoTE4EzG`69=qkKfBz zI4`6Yr@!Oaes1lv_%dHM(|nHt81R>17Ex)C;C8$XxJ_b_IQ0EC1v3k*rl8yqLavRo z0kllKe+toc-_tGJ-xk; zU``F+o%Cdz>LcZX4qI0S(d27-6*2w%J@~3UuY_gfXo zQK`QEEZmW1+|0E96l3fX!r7_5t@(%_6tXN@la5g_&Si+9Fr^;6GaxK)8}sLt^?OsM z0(9=GUE6{;SKq8URS~!6uk1yAI;SfK&lN4Od<%)bS5JJISi=>c;xAupu;&Lg@e=yr z@A9;F(0pxPn`g0MJEa!5!*an`oial9eFc{6?8TkIja7V}l6lwf?pvpM^2F}~>0cX{ zn%R?E_3t-NJG~5PrCe(Mm0VX!&o-AZ^@Xuyz-m+Kfu^3GncOQq_AGO6o+o3zbv|o* z{#U8JV3JbJ&&c&1+L9XQvWOXF`qw>&?T1TX*Zu{+hggh9Z)FI_ZQ)d)4!n zjKlGDUnV3kFz#Sc1|}}yA*7-0f{ZH?CXNjVnD!qaxCfX7l(_=yc1`wdrOVmb)0PvF z_)z?*UsU1Nmep#FA0G6EnlIV~1XBHFxb&uT;;*`Wer;(n+AT_RXZ+_0XPcF`X}awF z_Z>~T(nAf8=;<_$rVH;Fg-n_Q(9M-jC&L?AG6&NGd$k!SIUmNaBzs$HQdo#T%AMfV zb~vRt(xYqls2zCQ1h)J5u13#`9h;2u!EzjYBd-iFbjU`9#u;-${K_aw}1cq^`J zJ}|IQjs`%xe%_Yre+{u4#6K`n5qsfvh<@=^W$3A!Eee^j7k)xQ-H)QajWoDwpy=AV`|YtS2Q+9T&>Zq4z(G9ag(mfXZLAP$OLvbd$rF zC*p$VDlXn(k(&uijT1a|H=%u@j634P?a#+P9pcbZqQLZ+H5bXmZvFnKC8ryIJS1Cm zI8$0m9)?M$Mg8E?dNEAD%Amu-%1Q(U|Bv=1fezf?3YG7jKC*|_jj8X|&0CW`;cv`%*pup^!;Y%G}^zcZMw}p?`@RTRW)9z?OD8M~d@!i-#xUPPUcLCJ(;R z0wsWT1Mo7e3MgiSp+p@(ijNXto{*Z${KE&3*uNfdehE<3Yoj)$xN){iU1Wq1hn%ox zd3d^H{sT$Khe80!A17{Hzs?`@kc}%cMQ^M58x1xA%)KDn7|g!$d)uwx@n0x-Jp0GE zygJ4EKeHSWU_gdNLvkkmL9#D#cP}IVlvqW;I9~+@ByE}8`TxOgxOOi!r8f1;s+^d7 zy5Ur;qvvMTaaOmm&3t|1N~|06&$_HW=rI+3p>eA7ATCznq>P#S(MQgGtTwP{T5ymkjVRo7 zl}*?9&>G*p&_iolC-v~Is5&KCwu%btD_S=#aI^5_IyW8*ecjA<-N9Vk!eQuW(59_- z10Vx*{L{k|xQPX%+UxOhqrj&t=Hd(o&!EuzQ@oD2ePLrXcnmfo)A&cW_A-PRJt8vLDwfvC#dK@on3_ zo#wv_MRr)Y{?9VQznC`@wC$PY!qF6>^{n$8Eick%+QDhG%Xa*rN%v(x$V0~}()|~v z&HVxeu9P_es8u(}qtMi01By`!L_F?-UFleVbBs}MeYyh8(mtliGR=-76M-vd1!}`; za8mD;X%zN$)xB)_WTGPF@7d(d*NrXRCw}UuHl{{d$t{{Cpe`d$EVKbQglUma*$dPH z*~PF-flz>t0}^tG1yDti!4-(Q13Lk__#qyxKJp}qzZA-qbT^F*jBERXlAEOcfLRwh zbm$qJ#t3D708bPs%rZ#m(Dn6&*=F&b%}1Oo@S^iLi)9xjV^Na->oYCRf5Sxqj++lY zWgvmj5TDmDA(JMc8YMA*yu49Aw246r0lV;#zTveK;Y2@96gwDV^%UJXW-^e#9zfUU zpCRq53&Syd@H@gJK~_SINYSaneLz7zxX zzl2-9cCz?9TFkCG{#n>0#792cdDXqSWQWrZ>!NJG`Fy{pmBKp;^sYB%*`DIx#5=xv zH)E6`MmOn4*BRD_3wpT=XQ+}lE2)^yJ80frd&(?s${n2AL8&Y3 zlyxaK&CC=Sl)C;Sf)BiLwJxfZJUX#SwB=NP-)yd=!hW+qzg};57jksekNK;=tIBb} ziz$>z)9A+X%2DdvTUCCHx*3}K>ITQXdfGHtRCgy&PTek;h-IiMSECu_;(7DEe`=n+ zB&E|wDsnA&xtg*|Lhof~|J3rr=05utnm1=EWN%&-LvBX{n-5$;w!?cw{yTEyWPiV2 zZhn073+ElxR19&#rx{g01zb9Hke!ijUyQIZv(vO-tXLhlKdGJ75zRQWxaBxS%c~j*WeaT-PH-tKi!=#Ae0?! zoa~p!Br2te;^-8QnoBrVH4xtenN~KzZR5AoZQ3QW?&I$o$;9>cYdQXAZy)4t5puNC zrC+iYps1boB*5(ciebmGI=^2>T>I+dd2AhYvbd#$ET_&$sZKLbx0spawWx=;Z)s6D z;r}de-IF;EW?3ti*Avoe;T2md)VAslx$xZ@mFV!vQXC82oKs>Cq-VWT8bkk-wte5h z=+dqnVb}0b=4eR2udf|gv8xQkp^OFg2?{eZPYb0Qq1q7yKn#dDN@Q+KB@Ea+e+AYL z1}0Ycdf~w%Q40tVgMTmc#`FdFt`tcyEtDpR`HKa%$hzC8O3huQ&<QOTnvBFPNMF^hvlnT&vf`hqxg?HfGejOa$4#pbo= zx)%Aom7=`abkD%SLPqyHsIyG$QdBNmx-xDZ}3XMv0SSCExIvBd&6zD zT|L+b+a>kF7e41ruhng1W!L+udBCkuWZiVx35@~UF;9%C>yGqIZL~46mno0r4Swmf z`L>#-S)K~Q1|&S_LMHlBAU~-8ZzVr){-9BR#E{dHAm6;HRPV32k9Ee1uh(mOZNKv3 zK%5pu_a+WQk=MHO<_BV1Xl%1;F`tp z@Xme{N0C*hKeg zxz(1sl~<~rzvqH#b+My`+jDfD9qd{4ICk9Y*F{#B5rwELAJ2U(aM}HBu$o6zBKu5A!rPKN z(V3FoW@W|ZJd{|;NuWExVf{m4&wxKPZIO z57$q5MFqmbIl=U+p*-`44FTSHAgtVk3B$Y`qW7tXVR(;DBA>Mmh6-QwV9^<%G2ICP>$COJ4HrXO>~+0Nu#v2Vz~N*rsM_Qek|%Zv;wL&J3rgKvx~iAo2+ zWC~(8fSQ!Hw?Bv4%O=kr6%#lBB8DNj7R-Pu>gvSSfZ+b%ujl)q*iV4)CE_=Ac*c?Y zB_9!i&yFzzO4M+|Dg#hmC*$~&(&e&I8I0t})(=E91=;N!2W`vCMT%=j3fGL^TT5|} z>2#NyU&W##Ba_$21P+LJfBt%|pO54ikbKjPOp+l39fuKltnTFOd>%junSBanwm4`E z5aX9d-kOPKw$klV#}?PPsZH+hjA)#mgiC>DCj}$e(CvBd_5D#9C>5=fM5&@g@y(Ju zG-Eot{AhWhs_<^@6F0>IpK@PO8SdQn$dP+=`i=q}lgD=E^?KWeZ3>HweOo+X%ypwPh)tn`pz0^kT~UUHR@+srb@Z^5Xycn;YM4cW=YUOrzZ;n{dN(wKP}E zk7@Ec{ylamyOd5^o6|pt*)a9ZAP_7HBQ72t1o~IQ_)NpV@N!~l!m8`thLhbz8um*| ze3q8F2koDY=k;~SvWD;GyLT_<*rSA+wQECr{3!;XOY1cs=UAuIQ5>A>#$cWLNU(NT ze4;DTS-qIiZ=|K<(?%v%uZ-Is$95*Q+kGx@mktm;`LUtl(8K8=kyFWx>b!iH!cq!r zj7!VPWZ}dk?tfguGi}UjEVS=wy(2ktblR8?T$6mpyx_kuVs6syKqo3d|F<&!h^KF| z>{+L!guQi}-FoZaD~O7nf-#%Nbz$A;ZZPDiCmkFTDG08ofbf9K zxyC(?qWo+4ZZ=|hf!bKep{wAc)Uf9)0so# zgwp~=?DFT^+#Xzr#PUVee+^^H;K|a4jDS%ID%IMNEhZmsMSNG^f-;?x#NI#)90KE3 z_GpJzgO5u?)=Q%_5)4I73v5zSffHRr((&zR1w)2aCPoEid3ZCWUwm6D0*5aFN!>2Y zq{N$j1h{w_lNrwX&gj$d{F6I`%oLDdFl%m*(~!O(l9RZB$h-__L$Ak+WfOH4W>Qsb z6jcxVw8^UJhC^V9^WUY4$hH3(BPKdGv2yt32bI>aFnO%&8;{vj&pq;Hq^~$-!?PGi z(U7>#(^Vz;dP&2}R2Tu7yhC^J@ZEP;zkz=GYu*jh$aedkvOW|MV(TKryGNz|Cz~=q}`vCnU&A&!uFHNQDvq z&1|}?Vh?MFk5aRmnQ2(B1+kt}xJsFbE4cH^u_|8%wC!x`8=^hqMr(P7$08*E+s&q; za@uWm!yx+J`T2zHhmV@%`_vBq8Ef>1yBl?>#QSd9uPC)J3^3;Wlp!q%kAX6oBCU5# zHSTVYnZf>!)d`MZ{k!dNU4OmltayEfhWF)_W6xSPzD+Y>9rN8wy*4G*p)$?hgig4X@M)?-&z5OecVk z&dP&^H##ThQ+I1~g)k>)H*wxM^LMd(gOJxQpkfH45WhKdk#wr42d)>}h^=;A=*d)Y zP5E8cR_JpazZomsyo`Y2Q%`BT$r%3^T*MOoC(k z5mq(`WC#D7jIacKe<)J>_V@313E>KPA>QZbbK-<>C3&JqZX^{mB+VK8w~4qJvN@8{ zhFL*)!(?Iw3UfLz0U&ubm6Yx^4c!OEjoa7;iZdj$o;kE;aJey=2|J&V=qT-x;%QuP z{oR0%yoe2eAF{8JZG*O*bOa>Qt*fgG%4lMnb#l5;tG+@?0Mz10t_i_RiWsA`;o3Y| zfdIC6m>ghNd5ma}NW5ueSKTvvW7FCe1krIlrn93kCYu~T4qtUpef2fe-ZsN+5B|}` z;h2V*0KkqQs*oa(`3PiB3q%)u5g{t2s9=0q!l)v(BF+$!>I>)XPnr}KkqJCd8sjcL{#c?Ov^PZEX72;+cZ+cbf za9qjQQsvnq?;`Jy8?jYQbcSoyInNmHPfCVbAi%s>9Ovm<=E0!vLS4%D1Z9pgM6;zDhsnnKi&_S|F@dRlDhquQtvVg z>=LD&8hqc?1y1zE3*7{cCPn$1Va^s^4J`79Nfaek-1nKHAyzOKN$`>(A?t7SnaN*cO(^_|_I2%X<)2w6a-CDO z`g+rT?d#b1G3AmqWKPM`dGKjSC*9Gd7`^qNujBQm#k9M^*Di-LQ&JdeUaj8grIgO= z{BylZX+Pz`#IyBl#kYryVvWuUB^6!)l|I~*#)b=kIQ9YD>Bg&VIXEgQ;hq?^ZX#)A znNv<(ePS)7`^x;0vqE21A8c&RwBCt@i-ASlkUK%XL|l!)iG*;M=qN0BVULrres}_} zjlGl3LBh6SP^1PWXRS*YEVN%C~&JYNuePbIfyYObg&mA_%{Z;8IoHvEOG>6 z%=R|FHq}B^|G#NYTOb;N4yO`r69s}U7$H%C*ngLS{USz>K+#7UX?Qe(b+k{JoxqeJZl2{M6mt@faGwT8#krjA#-kF^;rXEQCS~K(f*eRsEOpd2#8Cj zF|~*ghk;^YMVtOPNHxU%30&KkP|W}XBt<&(6wn0+)A`!nY9N3FsvsO?&s<}TeW|-P z%zn_j&cDUQmcH$wgt@zJK5r>~tnL_Nt~?X)~A1kr?TGKwm8o! zhgpBV%X{*rS|8sry7eA1GB<_7Bbddz&%Th7xZ77;Nm1%G+$LlA$>)w~{eOQj`JC?2 z(g8M1Mh382VG~=3n~85uf=tN#pIA>`Qaz`I2O>4PNRRrh4UG-FS6qleBd0@GfEMj?U1&?tC`Y_qH05Ga^sd7N}J7T{ZZqUSQ|j z8GDsbfG1&9J`=zx&A5NRMg#3>DtaCszvSeE@^vBm5;k__J{tD;P6IHb66YmJMk9$K zh1+00S@%j|nu2wfL%!I9Azxx64VMJNHRpO4#tNUUjyGN&R7%-3^~B|=Qi$4iJ-Bg$ zFX}9hPbW|1T1_zsf&!!%O$-0HnjZ2rzE4BuW@LGMOGtJEbc$5fwOj@Fl&kfFLn$a) z%z|JWc=Dglpj83O`K@QOYU96^Os^=AaA6=o2q2`#@+YbT0!%4Wj?8VpXkZ?soVvQ! zSrIg2|E9zs?Oc=ghPyl$j;{?+yvVid>#G(LqWrZ*{5Kod{k$KshWFIsOIn+rBmGbO zM%PIg092r)AP@}6yU%qU<3NK&g$aB-hl8<4iT776cxN>dfnW@l#r}=MI1?gS zhK-dd6Cvg&q1K2Mfkm9z)6)}LGcq_3|A6p&Z(we9NX`H%{d|zfZAm5`nZSsP76RdA zWJ8icZn(%CF$C-^H?0U4(t%O7BfIz(1yMBO*ugNWw@rTH5d%FvWH!i;57&fXs_F+| z6pNdbXlf9Ocm5{-zB_moA~;2LMcnE;I@+4G=j$OQ!)8(L0oKqK;c#QH^K01Lji`xfh401lw6G}si>{J6{Y_o#apudPyMls(V5SW1UYk@T zk>s1j3O?dMq^4YN*Ohl)toTe>^x85J*C6yymd(U*ors8t$+hYO@_!o*IA{ixfxkoNw#f9puCVefi9sIkS19iUcnNxRpn0PFt zcEw2D4q!h*=jYLTubCn5L%P>G(V+H+0#W|BHmw0Uq2f z_L^3Fnm0b2q3`*k%3I8BJz>V0&t>uB?U(Fl)z2@zNuKW-y3LC@_E$qfL$}wT)aF5F zvUX(Ep4nHJ@!HhUz6lx5iixg|OAGWAS+ZYGdmh+Q%3L#~Frp{AYqnsL`plLLu{#~Z zpIY7r-#)tU-B6%FZhna4iQQd^XB$ml3GO*{T;Of|dQ*!-T)`fUvIR~iN9^Xz{iYYb z|Jm2hv2a_Lb%1U6kz%|&X1RS2lJ5EfHA|av$xuqp1{3$}sO74t%C%}&?`^(HZ;n5* z>885r6 zjKZR#aY;!&X|MBW)0$B4uSelMSR)s_&wlH`{8fyaLA%a@Fm&+0{kVnImo{uPN=!%) zM;LQzYAT5*gvWqiQZn@UmAqm16raZC=BcIbwJ211PD7Dy5(qA^wY6o(&?pTJjaxHy z>PP${^;YE&A6Bi(W=aIZ0M{wkuVYp1h0YU&_+BzQAS+8sMepZ3p`mNtojH2?`lZlM z=8k2z|2nB#{$Z$!1Y0L0Bes|?fVPfkt2jIwuDIy4cN4mW*2b7pM*9*_1jnY-;* z@`ExmB)(Nv-slgK78Vvh2Lb^m<7c=xrqI$~9se*iRdHl>a{hR&=iSPI#V%`#xOEhI zaViTMOq6$RqHg%$9FE@orXZwvWz35hhSB-#M?Eqa>$|g$ z{K0y#y!gaGLQ}_N9~;!=BnNu~RpTDj(krY>H@$K#c5P&;S`e<`I>$xfW2k%nVmDpU znOgRer}Rk$JNrdzB84NVm>Mp)$c+!|nu>jC`1ST!j>4hbk!F!de!q2?QDth8?w%v6gjG=J3Fel+-6^o53gG8?xRa~ZsMuw9 zSF=%p(a4;_T(n%tx&KJe5=z3gp&IhRnaS*Uxb{Q+`Sas_p{JPlUH&NRxnEN}P*C>a zyj+diljngW;V5N>A1@u${^eWbaIiOZ8^15-g#O7qyL_a zzB7~MqFiUMEy1O8E3og>hMcTc4gL-{p4bdE+vr=|^8*_xM!Sonp{A9@y}U2)7jGBt znVs|os(015QErz$R`yi!@clk{1#jucn<8Wk%QjC?eVB3P+CbZ=&^`Jn`I4cGN8Dqf zF1q?3Nq?5z6(vFhMqTB?!^3y%Ib0sSnjJmrCAw+QP*3kR7SZnO3xx+?UASu&!MlOh z&`lXpL#H8P!M!O&E<4hTK(iwkEI&n!nFxMgJk(_s3KoB-Rl%~2!Ywo;q||(MiVd07 z8m6XPaQ63p7>eqTd$7+E3jdkkO%D*t!TpIIc07c+SK(A4prK8q%QnJ^A-Gn3;}bPm z1rv|IP3YX20lS0rLRjzT;Hx`Og+mW(TzK;UrYqB&IB^1sqN~tFsW;lJHr;+=a+L=B zo+XMCfQ4J&q=x#b`tqB?&j@kL{hgW?*V5qB1B=mlSbynu%0J!xI``fwbJLHty?i1f zt>4kAveEEfx^&56T;R!6Y3B3?IBAXFS#I?9_NFj2Yzeuj^^u$U?S&P=q!T+Unz`Jc z?ss9f-g>ee*7zY8&!I!6k)vI1|7lpM&%PBnawJ-7-v=*ZeLBUqdfz}>)|)gC*J@;a z$!15p;xt_?9nF|%iw(T3a?aq@(!g)h4iDH=!awoIIs&cNpO)g)aKM}xj2P2z-^`#b zMX?JPk=|`~R#pGxR8)ck6$7ZldZFjFLdCZ{n^Qc%;5lD5d2rZ9N0?P9ttqZyIB!P@ zt>%cR*F00IvVx1Crj5U7ze5d`s@j!hc14E`id1p*JXel=S1dd!9J$1#pL9RA{n4J5 z!l3_d3$jbzoIS-K*4z}MV>qzvGHJW^S%-;EgnF_pFF)HsX}BZ!$4Fk3a+ZhcOY;pr ztZnV>!8dl&i={Skg(S-g@4uw2Q`nmkX~DY7GiH-=3&&Mq3iBe)b8X&>$6SjtPb8f; zc#7+2(Ck8$=@{YW$bi6p%X4d#x&~cmmdqxa}51LlH zv^ToP2eyiy)7ZuA*yEgcDJ9T)uuyGwSyevCJK9mEW2Hz;L-=wFui15r?}6^o42fB+ z7Qb}E7H_^i(-b6A};v#i9(XGQqub{jt!o$52q+uAV9SIckgU%Wu~ zLM;7gw;1R5rIG#Lw{PZPY7kMTu@h6z$ah-x5PAF|v@Stl*Jlp@`~^C?i9J*?IkJ;N zFJieFj30kk7<>*Z3==H+NlAhPj7Lq|9N+!&a_Mys+DyUOTlBuAf2MqSKGU(@xTLpH zvLY+<%8AW;_wF@3DLvVGswtu3fKy9Qy4QMb6D}^{kkF$ai9XlvV88R`a$`!Bbx}$l zVIK}#+%gji;0=BD?D*BI5-BMutXsF%<(PZYZr*$YxAKSnhjMS*$2zW7?)^D1u;1mk zp{ADB*Q%;z8kyH_5OK_FTRXqyI901sBFxBI0)+V1*%En!MPP*enqv zuk#;(`0;ZuFfhQ)VIAKmhn1C`Q?!$)&FnJ^|nn(g|ZGXFsa}W6P?V!K* zJLP44bMxnG-w#)yLpkQ5ykTy2Qpd!-Y3s#cVZJPsF#8H;^l!3ldj~uF;*3b3U8Y>y z_QQ768F7F9E><7skPb*1Jg6VSlHy>OR6Txf^vn2!Np$!3`tPSgAIGKkW@XFP4ri5D z7zh7i+4aVa2fu!7LW0-j*DD8#mpUhM|2Cf$*}c2JV7kT%V=dT&K1ZKRddL<1-M4*g zwQKIv|B8l(n*rw>1~SZxQ82Cvw`;|MF4CZ>z^!HBCrY8>Kh=+6u!{SN<}C^Ks;Nnf z%1v)*ZLWOG6_#k;wI{oyHk_af7-Cdv+s^2EzgVw2`&6lLbu>>~o zJ3hECpx#aQN)&r9As#6@rSNSF1?SUgWs#Rd3 z?J0J#daXP!JOA?mUb@0?pof7n6(;sKfHvuep%3RmsQ-gZ%RG`iP|DXlvP`kj==^0PmzzGPG(!l5cXrz@ax zAYi7|R4~)_+S8@i`)eBwD?orPAn@Zl!Ym^^n!T6*x+%+C@_NL!b9?2$lkWp}sdo1G z+jLzRne9BEB=q02BcUQMyT5LgF8DH=emOZUO@N)QzOJss+$`0y#QkJrOOyWDvsx}L zdvjJkywEw-G&xQyiDoJPrWf!fM$>U)c-405h{ld?J5VjGcHLDlAPuRN(F(ErteExe3eopx zBf5-N`C=d+Qn2+mi)_?eLrJ~SfV+rM?!!;6L>?JIz48^=)S1bgv0fHI0uggQVjk;) z%*>dzWjlBP9s#}a4J5)tof!}ENSm7ODDvv}iGqPHEE^aj-?ZGzK!Y(N)K0s8C*55g z5Nsh^39!SxlWILBe{PSbVf@uV1n>uj7xe!olSdaWUS9hlmO8et?NG%fB@me;WFRLb zBz$q*Ven~erzD}Ac)Pvk6Jv)YJ}KwpH~9yZ(zSU1(fSFCQXj_dfCxvAMSP<>-9K)U%krc>@^n4*FG&n z>7)2^6gU1Y)27fogt@ELS{x;VPOG^S*H;vua(TW53PGj zL1Li>S%02Q<@%-lRd*^0U+OC=IR;{0DjXb&+`Oybiu^WE;2oqRMERP((hT(<%z^V| zVsSByMd-;i<;bpd{>TZ!Ad`+Hy2=og0ga!#vWm)8+EJ~DwrqWV=9&+s?3E%|TxAX#6V*<7$lcM-W7fMn{H?LhrKeGT~oW(FJ1 zZ4ab#+V-fvqCSatb;H$_9*Zt&2Pd0r|# z-}Z^vJ>Ttcf>>^VR^0FYT8QokYxSSJ)@0y8-vpn{7_1vz-jRVa2q234G3du`@DSm# z>JCGi0$?liN3$$~fSrm6#)gFNX4&4%+Ik1jl}y2kvu`F5ptApw4$HL%j z;X!%JmWgSl?GT#DhqUROm%)lzefbc>{ajlIKAlqSmUQIp)F=vsTj*DQ9T-~9hgsfh zuUi)y{U4VtTOZw^@zd?$AP{YKwjXenfaErSD-I3?Rp@4K695Yl0~yVN&@S zu>SEuJTQPMWH;~}jED&Ylm6GXV*}OBN@-93K0X6kI3rqy%r)Th z(gA`HL2^-nc^0E^c92W_i3~WO+8&|Qe&JEmL@CqWr>IfP_fqh(yc!5kb`=hhnXXUr z?;pwV9ra6oU=r6Slu|-yDmims#GEuC74qs+?OZg%aW2w)3q>*Q`{kLJE6}-w77hvr zvEfmpySYTvJwQ8?igKTyOb1e?x>qHrH-W~NxcCp*(}DgT(y046Ku z*UElrp{Pb3n(Zjb%-vd>$TzO>MI?6i{@}E`Fu!IqY=qb1K)QBRSSZTKB=K&KCSZPE zwyD`mXd_n+unwtvdN2`S*xcUzuw2Ie3A`Z4E9Qnkv3G0!2!1W^pj*N1%#q*gZvT+| z9?YW_zq{Xl#<7lxq8E`y48n{Mh<$cK&03kM7C9%AT{)m*r$?uHzUHsTE2sJ0@@|sM zd5VysmE!&tKi{!CGA+hF*-JD&w*BBIZDiGrHJeN=F)1a5+mjFgSxkYB8DwM3gYgKs z>udwUpM!$~8}OdDyKn=~!~)8&+1B&Ji_g%%?AF$FFzsG+uL*amI5DHNi9W#@dcQR*BjK9_x%x3$IYKAVY}g~(Men%Lxx^<3Qdnz3vT!(J@#x- z8%W4mC^-HWCy0I(!R{xcb6pnraIN zVHE*-dmfko0NfFLetw?OQu-``F4PU$Aqc%op&$-2S}>>Cwtd+@H~{D)s+zWK&)B|> zIpD!T9D@P0$PPeD0QBSH;>xUS)@ZT-D+@*lt_j&1}SWv-6RaZ+9MW9)yK@@!wBa{3fE7Vvememe_O2VXOzvJ7#&JlY^SUPYoX zzolZQOz+Lddq>9Vev@Q3O33MUsdqD?>Yz!gOc&X8a8YG90{8Zvsp^e7IdfLSOcV~9 z^4k0dT?IUSxokzTihhD@hDs221x6{Lkv0Va?Fa(yUo3)7Fa4ZOmf-n+NPfQ~`MDVsF;)6 zp7!ta!lOU<3N$7L-kRPkX)stX4;O4v4BcLH7DD%7XC|L@8(+vw0;K4Ahss|xn#wdf|s zi%&~9-32a^%8-@knC)jYR(x%v+sOXT%A+jvi?mz6JtSnSYx~^0fC%m8Onz8P|G4b% zBYi3392=zzn4|#TYhfsmLI;5eQUL@9^V*l(ZZO*c0sur1a=HM|K1k$wVR^aq<8{v| zr!m-HOwY|>qrho^WK*CQ=sCF|x32O!V+iX6blZWkv7JDCUB1rOeX^-WFnO{b$d=Jq5DThX&9AQ%xVpt z5p^q~t4z}tb52&D^7wSpQs3F4WWIToY9F*HqlBaD^^~FQa?N~wED{1Nm+{CIV_C#h zUiA;{v8#gh3-Aw$dh{MEPkTHnYJ1MKzkD$UJCeTKSr>!lBk&ZThZ_OE#;u(lWi>UB zsJa1=RH0S6r%aNcSLCSwzpG5|dIj}TJgNmwfL3zIbO4}JZO0jv$NRaq{JgwC;_vnA z0U9um4PX-V1yIq+P0keQ9~-hyq#N%qIJz%6m6qWt4y8km;0?xL?ekn=Sx70{4u*kFEQVlpPKdZdHZf{-KnHtWIhGGT;cK46JCH`52sTSD%rtCIfx&1^DH#Q;Tv9cc1 z32d=&P*2`+o_YeZCA^z(PJ9#^w^U_5912{3ZVO~#!Tk2v=^0F|p!6fW2HVLDfU_%l zT;awt+b>zw*^cQrQ+i)w=Eo^YFZ_YFv+uAB5`}V|qFbDtWME{}2b`;6@4v5BT^_8u z&8FOlb}l-OS3*oPP#3AG7&JAfzd}XAfy7`HjNh-0hszli+Wj&j_Q5nF_v%-hOzu6pA%RB7Q7f`p`UA}RIUSUN2@3Ub z;zG6V1IgqMT9|60SiR;~CrM2!TJCh0WfS3K5jmEoBu&14qV8%pddKVh1Z|;61zTxtHWJS*dAA>k3NjGAzmyu ziIKJS%y$7asDo4yHU{^0$`MZ08c}(M!hVEbh}`SI`TZdug)In|7#nPB!`>RKx?FzF z&7imD%;q;(HVN*;UkmujnOE`{^}5Y#zE7x)<8X-yxL7+`PQAc4+j=U2J^(J}@kF1CSpvSY?Jy z48sLXC=l;25V`w&#`c6ikf5eJ0My}%uCDwLIE1P7@mCf2pp0}uz?3F7EiH1DKtjqD zcFE~Tg4&$#W_W7c;~;%{@A|j?6c5f%Sq+~Ay)tSH6uWyy`WuFpx$)`F^w`e|DNa~W zOsbwE5k>Tr2*ugUGV;PZu@^h11Ex%WXuGiuxjxMcMt@v#ww)b5QSjLX{ie6aDJGyf zxB?8w6`(Dy0RHsK)ev;r-{1e={*vFe9{2+M*BAg$)dd8lM1!G#fnF2Q=PR2e-{Xoe z(#yV9Lqi#<_unSkdI6zT!`iIYz2LGAoM%{AU0IP+ZU-t>AIZBl2_+n$b0CD!ApW%( z0hSmJ4vr0II5EL z5-KYDbuw2@TYx&@S&AQFfZjR!Xx=9?a&H1IbDoz%{l4fQ)Fsc%_8`9P9aplIC(z@B z@LjZlX}+4{tj1)zzwX&6F6 zuHY_aV9Gk%e2$d0-sKAJcm`HhdX0ijKiw9-lPJNz{ZKeb$gXDw-Td}$0xv7yw}RjT zn%`}q5@W!j=WqTMi$H@PU2O?{fnCN=_qsPT(%1%VuE1(Oq?l|0UuhJQdCtPpryn;l zzXYK$9Edi)rv^ThanlI95vEy}(z*OL)gZLeMUqo|469wjmn3;uETyqTAIW=)4o9%u zD-y?kS>o?sg(6jSq>uV$A1mkbmnz0s0N+cjzLLN4V+6NBgU1`F=xY(%s&`Cp-qeE| z65w_3UY6~-W)nDoYbaoHU`qW^6G7Uz0t)4Yqg-VlHDQ#q)g8*?=Y&_An0^w^6pz=4 zB6wBp;C%RPr<^Ro&B3c@xw}5R)~vRX!z!_C+wjV2Zu24m-9MFAr!Pfb;_;3S`ab;( z&f_;n_l#+YQvj{YX&#h!{bK(Elf?W6%%y;O17m~;)L0w+*?y-~1Tk+ESbCCxXa1}Q zpAlT&LG(VomwPn`m(tjn78q>u>90Kj7wZMU$Oao{4xqPyN2{M`>MMvb=u@m#q8#ER<0|NSpApAY&BT+bxDR!)vfZB7tTRi|-*qncXqr zt6;2gMuo{&TXJbIL~s%jrI?-AuwTRve2c!#FS7WyF*T-Y_pHGQt}kv$tI1)Z{%#YQ zf}ZvKaJ-yhQF$3Xrx32cuFRTn%R*gS{BmgonIY+bsh)?Fm=ZfUa5pymohw+pK+6C57EH>9+KxPNl3b96L1dXh@~X@_gp<5GU8jfwyX8!3=2C2Ex(Cvn`o zJ11&t?<{+6FX|UDRB%;XfVciq+s3K&Q^ny!`Pz{lCYro#Y($4mwblCX0M(MasFr8$ zG%{Aa3+I>F7t~mqoQX#^_UM=W`&(y^xvM+nrPGQSn$YcS6TorxSvun)$_F zi*^G`%I?9j4cjFZ>7f;o^V;F+DGNy5A!|gB>khJ3h9~&D8UJL*(QlGUG$e`jPrT6+ zb=KZ=A6WhZkCcA^)XM+;J4WProwa%{m+k2B`;H7BLQNi)R9AKU3EGsk(T%;!5j~_6 z#3Wu5qOYN4SC3tp{~Y~fP0drg^AP_ytA*eXSG4dggF8S zw1}=tJOR^R@;{~xW+vh}S*{Fj;_LQHp$mtUHxL?X68$4X2YD|B?@j0$vn1{wF2*jJ z)}GKN4vg1Nykz#58xRHtFp1+E1>(6aFj$*Ep>ptjU<&~_(MB>mLu6!#=Y1v@*I#}Y zj02xn^$0_l$d z?NLaja}M%Pnia`G?h|u-UN3OAb;HteB*DEVlc+i393$+SI&9!Sn>#mVVaJ`KY6#L(_lCc2a?)q*P(^VC)v56W#t5flLQORO#>a z5K1n+qsW0DrORysR1TbZy-|J?YhA!Xv*O0FtB^lTZK7=RqkhPam+97NWxJoE_;7mZ zW=Wa;Xj*<3(b9#xLq{LwsECwI#48%Y=j$;FWoa{wbH|XPoEpn{My0t@n?ZG!3RX0` zJ4>OWTCQ}@YYhPIegJeeS65fa608~IVTlZSJXf6Ew7sOQy*%H5wHT1Txp#_STk#Ns zIcXapMgJf!@c@+trVUrgB>GCvLS)gwK>tRG@EV9rZ^r~5hj*t6(7^&}9kBJjg0o87 zrUMp2hlBqL;szo}_+!BYl?L4RpxFS?-p=0M_3mF%1K3(e7wpk91J=rhAM)F;5LXAy z>X6)A8tt_2Z%WJwdCh_C<%g`|#-JHJuL01a&O2wugLRN6vO6}psgQXyYD!&P;;+tr zeSd1Jj55uxo}J-dn*bsy9SLl5mb=u0iqvpv>hpy`ZWOX{bc>`4{fk;19NaBKE&{$! znhLQjzG6=glj{x>>-$C?ZNf67OFK7JZb#D1x*`P`b{qm_LBMYS++o9jHah?eHpjrg z5bgf;0|O2fC+CIxP4xeu;d%og{oM#-0&x3t2P*G!yS3K^F>pVFfIeb;b`zH_j8Nb( z4(QasK3?p=*YoHAXfUM2=~_EGd6}r<(JTIx7V{#5BCE*w#1J4|l9yGE=PwpLV*vMs zzMh$hrn_j=Bu!N!wdaGOA-0pugVLyzHw9P%CL2Ux27|D)+=e_{B@9t0tGVo4=Ar&d zKh+XS93<+H*P4+*iHV71*V3VORAAbN&wL+=D8A(;68n`QK%qnm0|uxws6Ti5iekvA zvh`e+VzLZyczI(~4G<8%u%&%x)hr82F2_v;r9iBLnk4aeH`sL=9Zcvc;iRU0ue7&g zT&TlO8IcCN2+)#tFU7{>vJto41&KWEDMBpl-2qNs5A&KK%#V3U2^K8jTj6ypk|5Y3 zeY%UE=fgEPru1~ZpBzU;UOZ6Xy1dCK2E8M}#4!I`-zOx-UL#k>>?8mCV!#ul3vFs_0B(($l5PPePl2pHeIQ zETVW%8X!Pgou?x^^lS&RTcGrL?Ru(0deD8xpB5_koc*YL>&V!x@E{Cgq(#2&>Yp~J zOi&=p6BLcLhhsDg)owpMH&G=xjqT%k3{LYX^SBSB`$c9`B6O|MaA%Ziwa3a*(PESJjA2iH$y;CWzV zU}*ow^|^ud!!s0WqzcD?2GOPT6@jrJ&Ez>(?vA72y|X7N)u>7LS5;`IuJu#)r!o>% zWImEpCJDlz=VXel8)_OG}b7u1APr17&Qk1uE3^Zln8=O~q z{!2Y@;}SQZTDM7lo%7?&ew(7N=a1nS+2eO!_JcX>5AqWI{SZ%7p!&l5H%=K(Du}^a zpILB04ki#xIMmnC@aE>9Vq@F)Xd3F1Bb%J`r_B{_p|XG!eR6!f9TXb({~_yP$#HO* zxBjw`M=ruE5xscfdl5?AD>ee<_(|BDj%@EK0uCCC)we%Nu@&8DOqf%hWy#wJH{t9| zWqfVe$EhadDFqCGyLQMqoLp~VN|8L{ZgnUXXVcaabS<+=eLPo?#q=4!qfG5X*TosH zL+c&xx|`s3Rj;I_)}j*z|8}?C^U{$M7^mdS&EN3z^ULtR+^F?iF4NjC(F$NS4sJQP z;pl3S7>N4HR-pXIrcQUPeC_Sc+aF?pYN{|>SthwLRnfWZ9GMl&>D~7#w2cMn{PDzK z+)RbLAC?^cr(fcDd_iKNCA6hatYrl5! zN*aCI=3bKJ#H$<*CU+5L(A=oJ>DZ0T{gKgRm@hJtc=nU7?kEs&1GA{{XuhU458;PH z&g13_1-}jo@4F#AzJKw`QFU@62*4a22=O8DP{3(_5DXKHEG$CDXbpNu5IQ1=wA|P) zsi4~yjfjYz@)sK{_7@-M4B?I0YQ-Dd3kvhXV9K$QTZmSzsHe zJNEgz(la##jZ@R}}&x zQH+&ia(ZX0R>xgVq4Gy^a=SY@D{U`9*DX6WhSZObOHQ_vw@NakOv3qZ*ZZ|jm~#q1svG5{&i-bI~6P&Jt|0i?hTA@_i|Aq8l zyc{d(sKco**GXhSB}9w)iL(JcUM+_K$rIVmolp6INNd-|mEhBK_~PQC?6~Mh3C>uW zxb)bNiOjdPZ^Fc-9HEOCc@{}Xa|>t;#t+W!9}iiD0v0m!dRC)!3#h^S*josm+5RCw zb>l9i!l7fQjJnUqC_zhf>LV5`g|n>s(F|1A2+kZxD*=rh!KHwaM%L7Usi>{q zvaZOiy0X#RMUv6-HfrDQfcg+rvWjShXmo!C8F3HiBRRRmqH>E)aJ@$jq%mL&3=QSp zD`aIgpo;mFR>y}>iD@-q=jcYabV*gRpeaXS%EcCzTV$&#z=5i#P!}xRav^(6l;ixJ zu98_US^iNacprTze?IyY;8vmRe6e#^;!toi~{ZWUsv`|cx zf>lt+&NHA3i!3ABseo7`QBJ;AKa72Hc{Yl9hyWE%ll16va8$Geld-Hche0I8Mm;;b z*4L#3Hb+d+zQn&%R|ATRYY8#93&BrRTJv+aiAEy7x{|zRC8}vt8nDp-5%F-4g8aL37wVbubFJ5D}B?MmBPiX&E2|P*Q8AIC4k5x0y5LMbokwg9@b0M$AEkL zmmTi~!;q(!<*CF`Pr#|0^~2zWkraAAgzpm(!*lV%4Ih@BId^~1jtB=S}7nN~A6XKM>@U5rJOF@n)lI8U5UqzT>;H2NXO>om3RNDv6(@nd{l z%Kymhy@6Qm>PkRNl&tba($bdw3UDdM17K9){F^WCcEFZ5=4ZOXaIF;rAz}UX@jswh z$j&bgV@E_}aC8hQNMBUGCjW#+>KBhQZvMmhSW@<5pePi~D=_7_@O}qFeEMnluAr;NPVHSJC7yNE2oRr-~)k?vHuUPiI*QnG1uCto81)P;uOIIi)A$GtKJtn6J<{_|nOR;)ZE_QTQ9QD$Xq zUkgXqMWi3sbie<@o?CAX6R3jS^I}9K3W54E495&e&6i>^9f3so5tBB*HMbjX{2rg* zLx;)~uE&4rfA{1R)Aya-AWVt#PR~Pcf{acf0N&TpP!nkUW-*1L9g^A8NBn4WKtiACzo-I&Q-PV~(_0EX_*%mNw*TFy| z1m>6LKl#`Ak=YE*09l79cL1^)6BpNIRkz{1Zc<}qV zVSIUc7%0!8fn_%kg7z(?zVQD1`Lp@u`C%A@DL@i-z|QO(EVY!seEBaw=IV=zPJVs8 zx(l}yuyz1WMv{ubF z!Zg*c>bTx2$38M?6hhI|89&a|-o2{PzkcCwx2why*-t9vA!KRSs~NRut#Jo7f_LU4 z>uZ=gh+$Z?gSH@BCsD45vDl5WX)dm~ zxF1M`3TASN10(NqvKk_7_|gOTn5@v~H_EuU>Xg9o%d#pS-jXKVQi%!ks!Te! zxsj|zRnf%lefNMyOp=`IN9Yl&Bcoh8=`XLViBMI;LwSO+>*j*z@Y`_=DVXJae57&C z;QhLtc}Ni zzf`QzdnV9hDrRLBMWzmaF^GU`78Y@gEHlcZSJs_s_YrQPvY*?x`8+rM-ozY>G4u>` zX`e{T%)90 z23np#*L}Y@hOo=gk*_ztNS6XEl{7M`d(!@708ynN=$x1cEn^a;BtXiUGDisR`ZB{E zo0blw##u!(zTC4PDp}Icf5BBw#)Y!)78P3Mva!bT$upNlq3!?D1POWlm8GL17}z;G zU!_nPkXr0#hVjc(CG>Ui{>`9r)d}pnOJayin^OfB?<30V8)<}2yBxMU3H%2c7oT0b?3&3 zmAo~N*q`a>C*GmVPj%ZmP33;z>aHCdAY_mKfo(!fY(Im7I}ogWfrg!*F;-kg8u5!Q zCUqER>%0IH`DEWLw*bozZ6-<-xf@0NkF`%GDdj;YnGUWqVc5>Y5E2#BgMikFl?L2i^QcOhl8RTy>7=)s)w~TY|-Q&fqoQ! z+JGCRI6)~#2dI@ma(pYe5tEw}yYoUD3=P$)lP&DgqMOZ5Ecrwx0<^{*Tuc#>M?s6k;UJb7PI7R*d z2U$TJ^8Gbz60-hZ(OW0guobj*HZTegMvZ!PBQ* zJ}Eq#26e4xmX<~Uq7!Kxm&t#d{z%tD zsXBG{#_(A%_p(b-ASI6x`DbFDRV>KY&ng@)7>eRd+zip3{bAa5xyRPN-p{ae^J3G~ z(t-%bD!{%9a4^;ZD<;x1&dB8!SHx6thCNr-^HajyfJZC)uv}M-@~Y2T*g)w1dkn=E;SPW4(4@{4vw`EwwPAfDAVMR+s|d{8QvU3-VA z5tJm1V7ux-s!-QilcbpGA^!#_Qvv6b$8x^?7q@Iu4{MO+ZB*TWh&vdq!Tie#A!z}hDT=!g zETg9DW7CC=XxFO@N29Xr-0 znKLuqBwt1V_AciO_!b20MidJ_39Fs?7HaAOeom<*w5&X!QG`(SNWp&jGBZBE+}X2J z{K}zRfma|cF`{~!tna(IneoN-$~SJJe8u$k*);dvwYnHdt=Cc4XtUp!I~(wI2b&vq zl(dvbfO!$bOd29+nbUDzLFcLH69+Xpzk2~&Yl#JL*4qdG(F6{1WkAZ>7(i;nmX3r0 zS*4Aij1~$=(tdL8SdQ}VkO0cfHw%j#Kr(b*y`yh(I{CLq{5@|9zcX{rFpT|c6#HbI zGa6Y&#^j);corKt0Jlu?ek8`qil+uv77KVhL)D8z{!<;>ZozU{?nb%N@I`H841&S$L6Ki$7% z%91rL!o9Hk_rDinT{JwnQKRcV5iGCdcL^xh$ zRzJMzD+dvHWyZm={L0!NL3o0$8fHJ%xag^IqvxV&1{f-_e)6O#$n!B9au604TXm8n zy#Bj?X>j}odt!P+)VG}Z!OhLMz>tdw46x#2j=5^HJfH;#qzL1xo>#*8=G=JZ;+acN z5BYV`OzA-f0SZlkK`7Yhr*QP$>O-6?P8Vw(m^lD91sk;S8*QTlhNLX*ro9LD{{`CH`b!t??IlX_}*QTZjq=ixH z-cdVvNA~>i!w6BUIU#J{?X}p<^RIin5A1NUR%c(PxyJXdvl)<#6C822jVB7!c)mv~ z*65q&!M&`+18iV0|PNjmWbz3CM z2afKLB7q3@7$869?E-WDhdRhd`)6fB-sM_;;lUghRHu`)1nbBFx=39SAPmy1lE$y0e;}pFI4!fJ1 zl-w#Kha$0`5b|pAvI=EYNZANPF+S}d0Lf{%q(HKv!R~x3X~%4p_IPNw(9N>{+Q|-3 zCxFHXLZ8?Mq-i0VMPnKIFkXziHnUzf3|Nnno3B54QFX)w%1x87+|T4>QPCcWuc^^h zq1!|}|Dp122oAZ;IKZJ1^!v7Xw@1O_f93f3?W$3IPF2pv8Mk~+H48?i2XBfnhYe+b zUVAO*V1|`qStX2iG~nnt{M4xU0LSf&;QPJ^bb0gnd#{_OHCqs{^;ubYVDK~a1TL&D zXugugH1i`>Gjf`SXoS}YO!gUAN_c{!?Vd$$?R+|M8 zm2-NjB~~t?CnHuHA_9fpOz?NM0Z#H2&mj`|zsH;V_0=}``dh9V&xJ;|j52ilGcy6{ z>$%^K8QqJd`o}GU?MYglM)^o6yPpwn%}WA$XvQm&`VA{!_cAKh0c{F|Dy-_SG&({k zeCxAX)=x@1w@Lkgi1{C<3gYenQNh;RPy$%|#kCJR9_*;efKeKVNOc7yP#2J%>kgpM zyXWeR!Fy*PS(xHK*wj@N#!*8 zIu5{}u2S_DyfW1;SJQMnr3zFye`bv{+Mk4GmnU=ZZ6fXT@@*mD1xOhr5{=<*PQs^V z24ct5>^o95erz(|d2n~ZU7gq--uIeZlQK{fpx7>6#?3d{qe}(e<HI9z0U*0g zQ&J|HSjH-jFpawDnn$F&4w|3}O)hU|{+W3t3J%<`?C}Z;cqt`@vO=ju+~R|jy@VVA zQ_j!WauIct+q+O%?z#`xqgh2g3MDx}@N!%#P$n&*U%|YevLyR)f3&kob9fh3fJUSD z;6yhSs?h8kWnPK3ijMM@_L~DDP?OQi6o#^j1-MI=7DAUcsaXXhZ~2fxxdzr3`BMk9 zg4B6K+caQS1)L4t3Tlkc_zVUk>|4fpN}3wM(l~!eeuV19BuOc!K$l5(+euQ0n;DZf z5Ek`U{u=PQt+aEf58)>KXMp3m!UxOoG`AhfHO;^=wI<&6WVFzZ$Uu1?8_zBg0{yKqhnt#Xl8C zlF~w2L7_l`9N*jkzF5mM$Ujt9CbqWNgC5{>9M%_j4`Y_Hs3=*J$Mu7*qbtnBT(TWd zkOb23#4z?Q?;9(8?@jDDKJV^27=pF#mCBHQF6|P-VM33u-POH=L;W`k=N~CQrNL5Q zn4O(ZPzV(N^)nYaMY-1$tW2o#mM9PJnFdDobtA^!%lDEP0u&;!^m2=g&>vA1x#JT; zL5U#3;O3PxlX*z-YjI;M>V%reT)L%Qy<{1CEx{Vpk=xf|=EPbIgJYDOEQnfu3Hhpm zw0Q}#m6FotSfPlvxwQ1-CmI7`_4>u#I5{T9%}HTJ1-+0^+UTePaI^e`qu$2XKd^JP+*bafca=u@-TUzI z8Z1nt&e4jl+hq>uzP)kBGGoAep*C+f!j%1R;-w&QA}tp(nE++pUe?vt0#)V3pQEFY24{vb{7=;7XQ{}^fips`jzqd={3Y>v6;AV+`B4;wz}ksS%$I6(0j_)+2`$zsQ_3L>C0tw9EYRi6rdf8tm>L^lq8Y=O`2^BSJ8oEw) z+4s-OF2MQ+tjsW}rAhunhov@d9@;JzD7t&c?XIe1%gSZ7hN~$pe%KZl!1r9CMrh}-e)CQ0K8n0Ud2>dz#$B~Fw44E3%qObN zBQw7fn!Nyp)nWIn}hMq|r(C!cP(u2zTkl*KZg-!zv->9gt#HF9ftR^VeQ3Jxqd)lTpH z!EqT&KrRCSJ@D^nDl-5>3=koppytL@!s#Cy0Hb*2%O`GtDH}9%Qx&}pp=v^pkMjaBd*~m0kn&t=>R{J~ppL~LivOmcYZpG#)F~&d9l8AG}qKA(1vKm+noTWZOOShmY z;V@Px;Y)xK_ILNMkg8vx7|^gR^tMjE-%nWt-+61ti78ldhSGbf58!*JOCHNi zJ@kIdnr{?~SV`cU^Ys_)vpZ-SI`yJsq@Y(ZLqb6aZD49t2M%Q*rKE46@kJ*>nmR2l?Z=^Wk0(?! zH-eezAy>TUkirtFQYrLJC9gV;)0Vyion_ zOXDrpq}lQ*XcXRU_;9?xKnmE}1EN6+8bv8v5C6#qa`v0B^x6h=@duNQIOJ0p$1kd? zWXe_`Fjj#^uG74x(p4&i65EmB0O+w!fSF!njU%m9olK3H8*jEh&$3 z6x|%Wyg&*pPVW!y82a4oMHIMrcqR)J$-#wK0DLgukpY*PrLta80e^S#aXw))b&l z<}mCiOxTo$`Uvx2iuWLA=zGVhmHw;tco5)w#j zjuPh9);H|Ox*u73R=%=zch;?%-Vb2QRrf&uJ`ArQ;3*V=^M6&6Cs?GvzH|a;o5`~( z!e5ViSw5t3q4`vvzj^GS@V~zak6?G&VzWftPi84|I=8P1723Gfaz3UcKryg(w%Z;y zvs`P5Za!p=#hMAI!`1VO?8w|{t}@xUfWJLIh{W02IR_kRh({r2^qjo3f;AZH>hBj1 zc5R$0GG69ypS*zI*#D@W-9|vn#Kq0c4w&Xf8YU3Y;8N}jzKp<=_wcvhK-Oh5K$}!( z$0wio`}>LjP*9(`Jk4)x&VzlKlofQ#1&7k7gWsm6xdtaT08~UMtU2({!U()u=+-dY zvuP}{u*|E?3RJHOoz8`UyWEq~Ft*Blb~CR;RgUm2y)=65i(#Bl!+=2JC{!N_Co~Xx zR=-RZAtO5#k}X?05Wvg?#){;|X*e1UIvg}Z9zrwc;>leA&Vdye(nTd8`}B{=^_`&q zy>KD#RSpC_sV5>i&?r43h~#zz=`%4*G;?}DOW9ATG+CM*h_=Ha3q;t0$Bq_PHRO7y%; z5aC1ckc5*o5`d4v#{=MIQ}g@>r-WD;;oPm$$+42PhbR96mz- zcoBMTu&@NSY8emJi#4j|O->3eiac zd^N`{6ryf+21(jp%%~$)n0IbTOW{ig)U@nmkNksvR;`__N{6C<&>R^8?LmQ@}He=*r&IepEi=x&B7OW2#rtEP) zW4nFNi9W|!{WwRrI(iEfAOb7898GI)qoDG+KmE&~uH zrl)x~ui!!VAW<*{@Z9%oI2)p_++b;|G$jgl0f(13;5Df2TPmuGZj-oMBBe1F$ff8>gD&g&en z=VRROw>xyknBiR;o(2r#onHe3a^AclN92qY4erV+A<3MZuEA!Ck&P`DV1p(wfg1T&GMI*ofj?0+0Wjs+SB+d50TR9}HB-0Pn?t~e!Ad5&@G+h;XESh(R z%+e)XeeG8ne5!$uTX~0qs;~Aw+JlWvVnKwVbxkj?6zcW3%5Yq}EPZTxN?JPpRq ztng35l&;UF&GFB4JNo1knRtuqH;}kV)Mlqq4Lw8F7BPO_!G_ z5{wH_r&_D;dyCeECXlgYTKWFrCf4`pHdneRl#x<#N&5|4ij%NV<~Y7H&_jKLnPoKe zV{jW)&g2UA%IKmt-9OxjM@K^zKWCNE{45`k-nm@!b8?^}>0jAIj9>o{cwY4igkLGw_7!c1 zR<5x%Y0&sEI<5e1sKXUDcI1I>s~#X5@Bs(8%s#la!Ym*+3wS#@s?31={E6rspW&l%ep*EN?YAd}VCJy0 z{wQdG&g%o9!&e;VM>Bw8mkvBWH|WBS>>jhQ6qLD`3b@StkiLhv*p`^#_U*O8{k{ux z5Qn(<@87+?Jf;;?(muL&{6ijbkLszlCxY7Hv1T6 zeDf9(2bSKI{U3@iC#k!q#|@}m|0+$dV;tT$M_hsw!YBLacDa-VS^fF*ub@lboNan6 z;>I>n^%fn6&v;wgI$D-JqNm5hK&rfjJ>P1y*WCT7*s! zE%)yj&2Jzt(lsVuTgygW?ChMl~Sz{@a4MSbnN z{(bXA?%lD7k+VgdFMBOF*egJfG_iT23Aeud&+J(4ym!Mw&_ZYx+3ca^jOtI_{%EjV zO}w+qS@-?9owGyVHeOQJn>Z9`wVwZIZPf~vqffDl{kr+@sBAkg6F)K}Wa3Y+6`#xn zTa1*o3jOAzXrkk9Yn_GUpT!%}{hg%QU|*)`2!|CiuFemOcz*L8_c`0rQgzszsYqk9 z$b?|A+Yq`D^BxaZgav>bL^^is6~|6mJi7v%mG%+;=_;Uum`pDPQ;Fw`I*Ucq)OzXs zw1TfaopWN7QgQ)mAea2!obuHT4a^ROgn#hh#1A{ITD{ZE%m`;6(1N|;#8@1|qgAUC z#4fbxYY!YBaMTbuxfvgpK)FC3{4@!-G@ID(*}Kod(^fJ>vR3+XrMe`G(&pvcpC3R0 z0jEaMa3Lp^V8cvwlL0iyV_STnmWHvjowt`oyms0>xKJPtzE~4L`GUVjO5Z4iomH;D zH#_@8%Qz7v#q|7l+rTM#MS;)?X=$tZD$?1cCPVkA&?tnf^V5A%oj?4gK4wS|ON>I%K2G=ts{6;&TN|C}X0nyzn)b`?g zF8b>%OuN?G8Wp=Si?u`~@SotW(Kj|PJwhHgVw zKmTshZkW@x{2zuIatsYw}3x#Dm_mK58qxKlb*geEBI2KcWu@JKC@^nbFLnMl3 zRi-vDK2rd@_~jF+?^Z-=cVcSLxleV)5-0Uh3UTp0qJ9Q5FcoV&e~$W8Q!Dv_L)3(r1|zYuAgYhKXRNS1B|o6TBW}bJT_BMud#fV8b=c}Nc&vqW2^F0%d{|} zh;-j9acO?BoKw!$kRpX%JAyK{)P7d=M2-KR&X<<4>2>aD=$#k@k!6sddp+8y@F$qQ zL(bzzM#qZVCk1kTrqsKI?NqcxQZF2yQ~rAJ) z7?b`;K)~;ITUm?>rv#`(n7BXwzkfj2o^E^Swkp5M@9^WgJ@?PlZ`?wB#s7HEt})BY zF%J&xlfIbitLn>{G+?12yY=H=;F7|}lGzHW6M`MwTIU&nE2^b+l|KltC#-v@+fqnhq)lulbPZnJ-jC?nXMOzl1{LJv08Zib&Vg9S;5aT}> z(~vg-84W;p3xin%?30M|JHJbahrU#!59YAm@GZ4$vlm|aW`mZBd^+p!R09FZZFOdn zilbWpZxAz0@TTnWz!EXm1mi@{tV_r6>qpO@#{G#F3{j>iVf;$GvX=inw9OGtgN;&zoF_-UX+Fxy-^V@M$J|zC+#xisWpCdIw1&DBMWE6Q zVj;*Bo29DJb9fSl%@H{}I9m_jkP3W~sCL2?o~Mez(z0#@HdRqS7GYn43_coQZTc6H zuj%&0CL8X>k@F_F{0tlxH1y_i+9DbOSg1VyKegcb_ZLR(F6H7;|JSP*hd*Snw(*x4dF;7yav5$QLSVOz}W)6iHV-i6|k)FzH||+QUN|ci;ng z6AUtw9)$N2i=gMG!!&niSJh&e3DK8%3JnY^%R-Bhkz4XJqUHq+KF;#-xk6)4YDQL! zWKC(^`@6Bzuw8_!3-ihs4cdx-u8Xp5K4pmG>lsNIC0}3sr&+pghF)P(47^4Pv@X`(ydf&Tp7A*Rd_d_A>2 zhSAX3mc59YBQcnnQBbNYoV43G@_FG{J2AHwh4%PpJ6Fm7zlViMzVYuEmlM`F|P-TKPTA{k5*c8c4`T)Ap^q=01kNC6)x*6}jTdT+w0V(R@zd{gpAl<$i@o&i^SQn~M^aMp}84{ERH9ai+kO1w~I@n;K`fkA8J zSqY6_PwwV;d{^KTpvwN*@7!dG@_KicmJPe`)2C+w?R8EM!@fop$py0)RhcHfouI{` z#0m+j^R6)ZWBb{wM;pOsD|XJ;z`Jd->3_CPL$!MXk12I((AgDk&A zMZ*bOJO1NG7!BsFfZjKuc53dGm~jR}H}QJXp2`5~jmve{85LdEwa!|EYcR)6&O3{u zO|?(#znZnj9WCL&J9^XS(vd;pd?p5Y>$b41<7;XJoP-w4c{MFXelr@5G4z?YZ4q5P z#4Q+B?K&SKc>NTU`-}1%hph#mp-xkWB`$KVl3<+%gq{A85gU-<;E)MKOl5@0s}KnV zwj}g8bgkOPZ1C;(2$~0Yrc+5z9e&@`@$ryMdTVs=aAb@g>qD!u#i2>`f=Nl=Trn|4 z`r<(#+Q2aW6Yn<(>ZCWdLjB>TTTxAzj&mG^_h}v%l@1nENS0K|t;H!A2<)Z#+{SVx zF4)V|gzAc8aevN_Fu&&ee5H}(6Whse>n*~p<8qH)0sN@PFFijfmax?LT|E|5+If`v z@@bSdYXy~6MtO8XIp*I3XS}qrYIt{a5kejyGcMm?!Lg1gjf7i_O-FgH(s1e{t zCxWLH*zWpW5OqyDcT5SKPEzq#LT-%=#BJ9i!G+IhQ85E! zH?MzPMN|`vg)>D1__7=ST3l@`ZM1&-kxo_d?&IeaUR%|kbo|KK73;lVO*vf3{=fbh zuouFiew$w9+9Evt;pIbSHHwE%e&(g1#!>RKYgTxlf8#&eGn98SMn#Q{-SF@$t9G&a z;d-3o_4g4VBS(H|NF#q}@-}PoPMR#naotzHb8_O97x3?HI;UYonf$7iwJR{9_eTS7 z4l#1~0{amC?f;UlRQ0NK*uZ`Uc{sn#K#L5U-$m$dO%E;8k@nAD4dsbtLsfvV>1yiE z?;cMSW2<@k6Tw8d)KU|ADdFoQJ%BK)JYjK3W35yI?|gE8amS z8T78;Hl5s*EuAE=HK9LfY688%#F|xFpkL zkHA9u{W-zp6CEN^Dr(_sVnYb99DizUOb^55H@yhi3usAT_esNs)ez4CCXB)<`+)&P z=@hFYUrOkJeEe^y{vde>bJQ}?e=)-hHHm%8!q*@yuMqRBX2Lw4f8y6$bZT(h9AJck z#2DJpkQz2SsT3=6!yV|2>UcGwv6QWxOnT#3RMB`pbwD0oD;(KxB8U;2!*8m(x!>;U zz|mr7=UfC8Ou@*5$$eHu6&*H!tZ9_0fW4G$@SE8YHQ%fcO=7rPBM}5wj@+Twk;g)y2lmzpurOs%0NF4Z430scb&taf zKT7BW9?X}MACN(BUB>3~v}Te{fq*kH<7x*7jvnz%1i2=l)x*kExxBz1u;_n z1M**1jtOxhUf4CJ&M``R{u<|0;4v{pf(BPF^-Vk(HA(COwTuqYQA-~7P7Hg|{X2Yb zN$=*_*?XGtZEU9HCKP>@l{4Uh3HJ|y#1qMLeC@>1k;_|vcd@PGU@8x4M7)i`GE z--?aP&tFMO!EFX_fS%4g9iAww$ag>Bi0za2FZq*FwVJOG}nKDEv2$G@8Qq$6?v zEH;jv;@V+HeyuWWO;?UYb-VHjtE!@!P0JP0m~-x~wbWj}O`ycbp;LYHjulOaqJll} zm8hah{MH8vsh5(|t%&$2IF2_qo|acgk@8_$&k8iCOx36JexU&kl38up3LBP`D(<&D zF|jlhVG#}ThZ^MMKY~IzmvPX)LHw{^?&WAWM5RF*8cHX?!`*YLfm@XUDLfY=Gt2G# ze)P8js_b%8%!E!hbR#6`gmZ@XI>EW|^0vN*J;&`H@E*l^TND#%2Ci-@9ySMT=^M?v z5Hv~L&}8I^M@G97!0lAJa?&D4FOJj}B|fNNYWc=YEMz+9xMgDBMJVXK3)2XRxU|>h zeGhU3jrHXF01{60=via4gc5i@#PjhWJqe~EW&a(obdd>198Z3Lw1+A!WJz_sR^#Pw zPbyxE2hCRQPfdlcT1OdqHqO|2v$CHFb#wf8+--=9%*VWfI!w*5EzQM1|C!_a-ocJfD zi-)5Uwp&{KNn)fp2>H(>th4R1od|0hU^Sfdy-Y1DIp*^)<@M-dk(x26eSx z7r%Z@wfrNMSuSN&z-6{aflv;h35*!uFwx++7Y&Me;|rXZ{*21@f{GGVjxRe0S624L z|1;se&DjUn5M<4DA+kSQ0xu1JgXxG+=ihlhQV_KP@*>Qk5yo+dF=FFuD#xvjSQE(M z6m0vCS0Sni&dc~}TxcWZ)f8lk#*(>?j`K|5Zd1Lch4urzW=duK}Q490sW!o z^93nOrH8WTPoMySfi*kZh7>1W%=#||K+Z~sxp;E+p{H`{DGNpK?TMZ1D8Eep{rA1x z#W^0W98P>F&a{-c>>55k`SCOzgt+AccNz3J;8n@J#-GW}Jwri;Yiz+s3d-w31Rj)2 z-pd!OgcPw6;A@ykW+Z#WUjF5t@sx|j=QJxr(R;&Q7Q^=0EHRitQzSHxxbtQrh+&2z zucF#Xf|=wPOW$j^sP2hs5*SohVSwdc58Z#iyvjSEUFhrJ%igbfic~jS&ge> z%xFTtIDn-W3VG8q<)f8+{(Fy>*?rakP1^lbJlN)SOCX(vtPctR$ODx4p z9sGDq;fL<$yisi~=o*3DwO*BycYj*EUbC*S4=%$1X8Iu>nz3lM0HxvZytuT5-tBNq zzu3$)Q~h2xmvlnX>9xekh^T5pi|XxOMf;xRMP}ZmJkOI)tMXD(n7WwA&3n>lGV1!C zZj;*H#?zC=Kk{2xD=VXdFu&3E(|Qx2pkXHAOZ^@4_c%TBUoh70kg)$g5cK}79m>d| z@34bSMQi(UwTtf5fq&lUAq$qHUvt7V>)3x~id2&h`#+UDDOBh<#K_~)+g3a5mue;@ ztKTjjDd)G4&T}m7;gilYa&L2aKGdZAd0q0YuHIwg$X;aDsg84P| zuEn@+5_$B$9)IWW`yC~DX88+JYm)6=H&b74@GVo7^j}&LxShK{M9=y*4lsB9{E;T55kzLTF~E^WpjPL+wuh>a!0iv)hZp~g8I@oq=$ ztkA9mBt1qWFK_0T1#bImTFu*+5}HzsYH@wT&t{ioR~nvA^F2Ke-z^AgY`DAqQmOxH zkz0ooR`)V| zjR{#H!uTr>wTN!tYo;8<2kqCftkvSd+atOIf%?IZ13J1oj2tp;R&H07JX7?;AHVt$ zgmN|Nugz&#oN^mIByRrYrfhQQCq|`Fo#tQ(B|(Q@;mDK=7tO$a>VlSjgVx}jb8@Na z`@;*b=ux8h#+7|B|J?+$t8vZjnu=E?yd*C-_R6 z_1LIJZh6jQLIpvsI66H&oGEddJ8mT(&g-CdxV*(K){OrDdLtxt&Wz?i>4s;2y*vDZ z{Ja*q{7xJZQ6-T2_6-tPN5D7gS2Vkjo zuN+9!^)65gIM4Ni=@j|wATe;LSH)#NwlyGn@X0*4h4(E!;Sy#Zlb~v3TkU7f@_%Rb zqqH$}%#TeKjVlg+J$DfD@41T;pRjVsH%_3AUGP1N-NAIObM8so(BDlIv*VUyc6}d) z))O2c1f7qWA0nrPp8fI1`)})R-_nOlsW8p*q4#2;>>?+mAdsG3|5<;wK~G9fwvzPH zf8$uVad&R`u9CLK^@tei=k}LuBzc~~Est)@Cf2Gct_!6BrXbkp_zYhb*N`hC!(?7Y zsXp;>z|!H>GKu$nH3p!j3Z_CJk0?b)pSt zJoO9)&8*EW*_v6&HWTLDVRQcMu_(F_nz5fd#YW{cbn4|l$=}|A=_FwHN+;mfk14k- zv+@Y$sJ^)gC&bi#5zH*Q6+0W$g-v2lZ7cI#`M|C$$&+g7VUaTO)227%K0Me;ogvv> zxiT$>Gg12ojT6p{BG!Y{YSu{%o|mvCwGxgUwwFs!ld~4BCFM_FrWzd(`C8Db|XAuo_yzrmZ^7>+u!dmFYMzXzJtmiV>$G z*AGOgmi81(@$yjhPOYM;_sLckb1G!2$hn6T#JuHm+{btp3c8;+ZJZv@PrzV=9y=}H z#>q__Gd4lh>Bz&yMczHN+HKFR0S_)q*k9MB#}YDc-@D?TH1J2^U7e7VPD(kucw(%F zK%x%4`^Fp)2Y+DSP|X+vC09 z`98j7>tR7qIv+D_)g>%DmUxFA(6bBMHazK$N$4?EU_u*@qBChqJ#a;n)v9c9g1TBp^OM3Dt0bVl-Wp#iYQ`hB5s}>W53Rj>PNufN7X=-e=Yn(iL^tc^O z_G7Nn)_ZU{iVZAX`J$%Hy0^W)?|81bJt3vSOC2N`Si?Gd^JBDO!1@_@lA}xQ9DdvW zshX^)y#wPI9x~UN>abGrDT8WN_GF7MJDhM`c$!g`^iaOO_It{yTcDG{l2!?yzIQEg zvoDwz9lBQB$0Ybv>znR=eZ5|!Qv|Z=EpI&{JM1%3GCX=OMUtIcmr{nx{TTsOSM3+GsSU--4l1;X4!A{hY9rR}VZ? z(VmNZPpF|8xs|iLESR~$4x#!YK+US*=|_&;&%ZhrTA}`BPF6TKK-Us%wy-F~xW2yr z;jq7k?Ma+lBJJ`^P@}`ak}{syhPv=K;Q#@SWBJ4rwb%jH#mqN965|GbR4%$=GcZ}t zT$TjlElZMVXssX%ufG^0`_V{ssUQ9zWAmXZ@~mK)MX3HWIFn66hqvwdyrX~3UtZ|; zvp6oO+n&eJBS`u;@2+)-w0)*#XGQ$aPj*?7F(HJt{5CPJ00*x!?e=+6le4VR2PDl$ zQ9bx4-o`8$$x~I5qQ_eJva2@%X;*mRon}vcvVS2xg-A8l2m6;AQncNvs#4P}58VD5 zeb~zmdoCx3uguh4j)~%_zJ&u*TH%@u-cgZ6ko2yo&D^5K*Vb}7w;k(%9tZ*I#Pjfm zTOgjOeZvfO9BrMcA8Q1Ci^6qZo0iEeYS2>VG-<|=Y`H?g{ek&r^kO4v$GWAnove?a z+aJ4I|Du2EbgEAMMkY`mBg-%#a;Z!o{x`i}Up>q7zg`hVHig;qQq>Y+Zf?}m`4Xr> z$1#NXD(HMFI_^zku{n=xJ-2m`+Mj$(xVT(QSIiB)e>P&w@7|2F>x4`dSewwtm^8%t z?msM&+!mMbrW%XsewAl)DEXhvOvA6&I}FzrB+NTo^ACp)v$q^)9vu`zrt+=Jyvb6_ zq1NoJh5k~ot;4IGIZ!%EoU5cI=2qIe8m%RJdbZy@LTwPfS`B8U7JIH%sb(fhcQ`g9 zY}z&w^g_`=r}C?9hw<8)+Cw+J$pE53tRnO9m-BS=v{|iGm4)M&In73cc^X-{UwKIK zE52?Yf1jL$)xHgIydnH!(kNiKr@8rive!xVQHSPJ$=Lr zD=W0p2?fYOqc&O(O-N(YMdT16iT_JZ{NQ)$42euHgZw;QR}jgNA1721!NxhRU3W2B zL@>~to3bH!H8b#{G_cE}_Vc!FK?jBv57||K1cJTmAUNOMv2aD1Aa3FNqrd+b+LAnu zetN0??bNm>mA^axaZC53!Hj|l`$KJP^xc=4n~WOidy)AA6uWzEMA_3AcQwo?GLEKW!D`b~pO%YS^bzBVw3OXj*x)LR{;qw4@NXt&=@oMn zasHG}8-9AMpttx+CFM#4*Yt#FpH)0QUO>uy;Q@ZKPl|hS%y@yxuQzSZ|9W@^l+&XJ zSdap0uwk4$^}3TLm2GT%nK}FMvYW^>KbKrXi9xAaSkcFeHlbY|VT0`HCtQkL1f>Hn zib{*Z-HSuM+AtDD!^tkc{?BJlwQC&BE0zq->_Kx&SJn=(H}d7$M;zi-`VCRkS(%Qj zBcr>U3|S)@)1%rnsl!Ftbs@S$*567@7>lrrc~n&W=xqaL^Oho>c3JV6ac$g}4Ni{c z+T{q)%*s)WYt^!tdr?oPP*E*Xc776W5pB_2rQ?K}>=UERsitcf^fv)1+Cyiz{w)_D zm5r*m$Z?BId6mxMI+kq3Bso)RuuV-iQ%0yBk4l3MR7NZLTc(u@q4V|{loT^>9=aj@ z({hwDRTD|ACF*g%duOt}ok@Wd$2Sgr(k5WEAMIS~{+&C>;rhnatmRbu_v5sYK)1LQL@ygqr`L#w(cYr|U%iw+j2i=@$7#MwQ{#hOKx9G&}T49!x}<7Q#aZIqZO zN5>y|g}I|av6VZRI_vI{!0L1+5Z#{~uI^EJyx@m8*t<7hE)UrTg;GxF;5G>eMu zBA?)>@fqFe{iZ4*N_jtjOLGql_C_we?<`No0uc$fi9Wlt>Ke^1Q8;_Y_jx zepM@n6?Lb)ASh?IDGL63g2Y*x_NuUd(JpGyDx^KntD=D%d{nupmPd(bFR;(JHatBV z&&Ta#n3yba62dcQXQQVbZ6+(Ukv`YC9r^`~&i#(R_3Zl*Q3Y zh|&JT`2IKW*?-+?K4TYdm87Gjr+ zY~oATcNISKBK@+EHf#UGP@$4XJVVIREAsA_9IA|zzV7Nlo4D$`jtIcZaquA*sZbe1<`m(&2qrp}a)3Z!He;SJH7rqP)$jQkG z>RdJ%ymg5K0}4i*Q`E2KU-feT689zODHTjTKOkh`ziaG*G;2NhsN*By#G}a6o4sf1 z-?Y(E^PUIZxIo;TBRZkKzduWT_POkl%8UeGK6b#Nn3w6P*vDZ`N8LQ-wa7$%8KjHz zUDCu{rKM!uS`$}_O&1BYn2>|BJEX*;vhTG9=Kg(R74@N(Ot(fuZuiZN*5n@S4Zt(W z=-i%Ve3LDss?T@*#^;irUq)kznp3BSBL<~OOlu{~^5e(zQ{&KuIgSJ9m)kGgQ0s6O z&aJ~An>ck^5F}Zj?fF$0dCiCn+=@_FyC0Mb)7=gC%YT0m{}AT~s#IiF z{>F99liYxZzd^iX-_{GpkvD}KkqZPUh$WGo_FmEYkf z65q0{jRn$;3l0vBnLxIi5QT@A*UiBL@q#l*m>BNvMr$^DZ(e&N3E+WvpDd}ur{ej{ z`J)}%FB^Q@Mqnx2KS_N{MBTdl_C*cj^A{J9a$MQ*?83eayzn`k^S`B+uNx|7rivW4 zUuq^XUHTRNOtV`1<-wmReV^AC0Sf6a?T@iZanMJEv^_Cc(-q(G>c2feGp2Mp^4g=n zF!edt?iy!drfB+9+cx#vC-y7x5VUc%HuFjp-aF%;|6SM+QyRZbmL6|OWHq7=Y} z`ySlD!|2?+?jm-WIb zS@80Wd~vf9mr{@A2xL!wz@bJ-nyWJK#%pxjss7#@GuL^(kli8mLXuQ{Y8YCke+xLftD0=0C ze%Vj^jpK?HNnHot*%O>ly0Tg{VqRmX;deYwK07muTs-@bsS+2_MIqc2X-6l^kVlt0 z{2bcghxcByX+tTqvqSx*U}EX?6AT}czO4cu8JNGr7H|X#`Q|z++7-mmgW{_-vRA8a zrku*s@(U(%AmRKjva%d1e32|Gv)hhq(Z#)R(tes(R8=&qu$3c7wNkQXf~T}KUaEY^ zPy{1uax#LZ;?nBPmL)^pTg~bvxALE=OY$N5xz#`HE0nrTN?`zuDQDrKmb~bf2xs7H z7-XOp(Gm}BO39ePSqwvsjqMkuMLo~x(qIbmefD>kJeL$--rJr%S$wV7TQecmW=-${ zrwpxa*gKRMY4o{SS)`nluS{47d?()Lp-QOmx}XzsBwjo5;t~>;)ZK1hxuYc|kvy_o zH_MX#O}>sb0`cq{42mlGgH%hcUD;)-IW=mOlzUS^bX{&#tj?Z1@BlMn<;bhiq9Jo* z+u7Fk7Bg8d#m0qjrI01JdicoWBYs@j0-gpD_l9RY_Y4$Sx~;xfL`v3jq4!&6P(BLy z{o?0RtTgpp&x~Uvqqw)^!Ma z$p}^6@mhxtGr1b=Al9&nAd**Qc+i4$yIxn>RN;6O+A9@O4+9X;6QQw}_%mq2XRJq) zEOGFQjNY}$Uc+q$D`VqnV$#Uk?GW$0NNoqw;x!9q_7|L|xqK3k2LA#iI*8RE6vS45 zFMxC8`zBxh<=i^12Z`;zEnW67z3;nJ2YI1%s9c`mAN{H}MO&GOi<>__lo`d(YbNAXrGIK!uqDh?D*3-YYOmFl} zGuw-g9u_o&RI52jT(MYG3csXJyusl^ml?hLy7IZe;2w|H@-NH8o=uMwEC*6dvG^vZ z1+7r6(CWuiLVwvQXUb`#69(;$xt`DoMxEK~(-P`#QEBP56I5vgp;MA4!6ZvN7LQC=A&{2T+3F-2a z^`M@&DYubSF)5=(aq+Z6xt9q}Y~w871|I{4yqV(T$IexTsAUn$yN%AWA7<~jw46BY zP3nHxwZYlg+S5jwlQr` zM;_-Wove>qbgU8a+#U)wjFYNn#9HwToK<%eaj>~wY?dBwKK2im=N{Z5nQ}Y3Hnlue z@usoG61qKb{C<0kq^R#7@!jj+AAB5&&)D;m^E*is#=^4YSn$54 zNMxD2P>;`+s5Y4j^e{~u4eTC^!bhCNM(5^~UfV|p`iJ(pD+yzo)ofQ+ORNi5aX@$6 z`>nn7pYBd;$N6et`q{F!FUqC<;q9_F|5i$u0`{q~TH{W#kZ*8A%e(!eOE#e@6p3ry zhG7idsZamDPSi~IQ3r~CElS;)Zi^f>(X`!6=@Eq-#=wye0*Fd*$4f+~@&QmbA*Riu z_Jiw^6ap%aX!-VGw+)B*jgMs@9fN`eY#Cnd`7Nw~{Hzd=&yt^fMX4?>E)a@vL{fSF z?Rh=|;qfSA(5hY^(pRe_emMsQ-tu*M!FK||W?gLi^zuH)#90c}qm-Etn8|(Lw$Bji zhtXn)!2u=lANQS)u{l4B3_Qa?@Lzi@ zzF5?n)la($QN-te`SL}g**Nq|C-BUqnKIUuOg|k^)^s#(a^RYBB`Ow2_^~D^X%PMW zv@V!AB&Yhs{E>4;YUxyae{v*s2n*)biT9}cT7`ctVaLnWyZDP$<<@ETX?|2b5IYtJ zeloQDrt*W|Q-U3`4C#w4i6%38#8}Tjdi$|~?{IaCL@!{D?xzb7sC)X%xU)a4{Ke2~ zSVr@7er1+9?6PaNlpG0nF~ONn>%q_TTQ=;dURv-gkx<4jY@1)c7t0)G@)e#X%f}>) zX4I?Omsb_uWN_{{B#Dbtbv;Usgk}4Ze3;%<%|lBL1p=nj&BP8jiK~Ex62!-!SEv*8 z@W27fpuSqfm~;Jl4(?eiWgp3Gmf44Tg{!eyDu0i-S48QF}!zLX|v^WX(%Tp6t(VuPFJM~)ee6U_e zI<)mfE=wE1vinHa^SwQ?N`_UbbDdU*OuFd4!n=CDC%YdTe8{M+mw# z<74{FGiZ{LXBW6Pnwb%ykwT>lyt z97_5e@4ru!@9JMFrePU71;9KXMjj0Yltq#f!>sc3F~6#9SSQJgz9{g8 z$m_N${!O@3#DemR!v@I7w(&ZU;pS}gMpIc6Q#gM@_WZE%Jrkeo$^Wg#uX}yJ}gpjDYIS$&r1`26cD_h6KC0! zS*Apg?1NP~9X4!lGqp(SFGg8`8vOQypIuygo^zGJLl?)H{+I`)1^s(Y#W!)NojAX1 z5X)S_IagE>ubsv0g)64FEdLU3&l0aDB(4dgW!_2(j4WQqUXLm<=ebgN-?Dy7?rK+X zX}-OMbN$=w@u~Z*_kBaaSa_?LQUwrg>38R;39_E0?H92q-mCSue=D7w92pSy%ah|@ zcU#o3ZO}-#sN3TZZ${D++3zgVlm!hmdXv%Sj4i$u-%TZ{F7HT9xF=zm{iWm99taqk zmEV-OLt^BLEOCCYt1Q!0Z!V2U4l6MU=Reif54dUHx{}hNeUSW!Wa-1oiHZV_V{&Gb z`e1_Y_R!7PQ8|`u@V&$-d>J9ftVYQjm%!qL3Fc*|dBMgSaqU0du{aYyYqZT~BgkV_ zCkV+?KB!vLW&FdMH7Q?lAlb~R46uPdZ7APJ<^7~RH+g&xB*MJ9ZC&^?gBM78T6oFM zody&UVPX7}?033ee`Q2B8CEk)q=3=1dD$Ew)&=o%;?#=l5x;)FZ$Wqb74*SzTBNaN z)tyC6EZpaBvwDqmZIoGkAw%jOJ~j(IJrpW2?PSuf5o zS%$xKL|7V}8IRHwVSH>e{00$^rC)1t$MpQ0E+!fZX}p{=6QNB#na0)NHgA*TbQi+) z(YJ7h{LI#t8SyI!HLmsfzds-o8w}JD!IQY+0#Dh!f|2Z)w^4FM=WTQY*QWn98-dju z7$~5uyH^O~IpRUnR{NnR>XPe-6zD$c%t8%i}1suPorqnR;LBvwj z1RPfm<%*EhPsiz0!aYl1L=ZP}MeTP04gTWEtP4y#{uWLPUXv{?G3Wh>d#+{$OZgn% z6xJZfvALwWpj2@tQ6ixZ@7rA-m>m6Y<4W8!w=aLG4*SGSbX(9-Mq?4a$8qR|_`#4c z9B6E61`(bTg7vT`z`==kzlb3$xqIog^LPE-O2+=`*7}ox0q)Nq%?V=@#ZlT!`X2xF zk5U?uAWR+a&--HTwL?|V)EfGv-j{s4hoTtqw1MP7l5zzCo!$2A#k><@FWfluF^BQ(k1*$D&zx6b%CH^X zM}sh7Py#$BUb_oZV@i&So=aHWPddtg*!|mT=KQ>zaCWIWiw1j|n;r*|(t|lwkx7Hx zpqlbpU>3qU;to>cpyYdXE1q;%3v*@TuwW@G_wMsI_pkl$Z zX0s8=TmReg*=z{2yldlKj^_XTz28E+zFF4t|_S{yyW?0hMz_%Ck2g^AlS4$vL zqo(gg)7$Z=jbo-0oJ1H_$E!1_wwbu#4-a^x%Zr9f)<~C5!s@(iieX1bTbcWQ3^dvU zP?jD0iteh0ERnXN_ior7tNGdC$PxTfHUdXv=bmKgBy3F% zK2c=Gp{I@g+>T=AB^rF-D8O4+W!9t*E%>*p5n|vT&Y+lpX2J=kF#|$`=GtZE*>&Yp z+a8%XzN`)8(w4L{Z`zYHO%F6TNt>>;}%apeaMVk#bP}2h4cu*eg5QOk}S+V zDBIwLwg?+OO8=%TM>zD?9!e9^>R}VUx4)tuMfFa=!TyV1sP{_W7$-gwM6BGz~Qaf5He6u zWK^xm%Uf!x!Dh@$-5XRzM~>H_QDV19Z_3XyHtJa>yTt{OWaaLPMg4kt&;5w3yYopw z!&{j=@V=s*zV+YLUQt)0{gfdvLt^wSxJ)|h6$Q5|uHwH3Xba^*1{!Pb&6$^eti!jY zkev2u`CF_HP>d1r_D)WeIF9?SJDb9BYyMMV(E-u!_lenm(nYa0-)^PyCp^*aYs}d8 zcsy3%_mJen_OWmXI26F11~g{sk}hz#ow*7B zepTChzpt`eQlOCQvvHTz{Y>vTVucihNnsG|;7?ARy0551zAkmHAJPu;u&Ii~-})bE zsB&aHSAa4ASor^FI_t2i({>9Zq9Uktr*tWu64EK%Qqo=0tw@)2 zgMf&DfJjSsN(cyuNJ~pe$3D-VbIx4X_s7gPv)!=a_rA}1*1A_S1wFomfSb=_##OF? zo|tf}oZ?!I(ub{=TuWoJl59P!dDxhQY;b5+!C_}p+i{1r1#z`|^Rw+_$w`b|`#sA& z@t}OB-H#u1Ff7#P{jg73jYBhz?qmd!t+g7+oj-3C_ZwGsP{BS7BQkA0)*N7Wyd&T4 z>_rp-G@1jkPYdleDmV5<84qxkWjB<8fBrepkN&lCR6x%8y}0$?9iDnvP=6!^w(E-X_{C59tv^p#3|`je7>lZP+TiK>eGJv72mtQdi1C^70pjc#E|%!7 zlw@dT%dt~+ZS8(8;^xnAz~zU+G#<~;o(%w>qP7j%c~PuJJR zaF<9LTxHIKN;aay~&9oH+#gpaBz8w5ed^AX#%2@RE1STmqZEaz#Ls#Q#p zDW_dXCeW2kvP{}0y9qtxN~gwqz$M&| z|8B&o274M!7PVqcXn?(mq(o&aCn^*IPUjLHGXYZ!jAwRo znC*Uecs?4QZ@b$1$D91Y6Iluu^xq8pc1B(y-YPjMg@YQ-w*Y%vzgS=iQM^|IWFViP?_ z=W`5*Du@+8EA8&UE1h!rGm!XM@6o-OXifnhg{t{ZwJ@shx?<@HP{rW26Y~i9)+`ev zuG?`UySb3>sNMNZ`=VOxbH979si7c?*6hW=DlR(eh-e)I)&ved-wF@3sLLa{6NHpe zsF+UuL}Nvcb^kV>7?n5tA$9z^!9WSmv72*cVkEw|P=vB$ZEMbRs?(K=Zlv-0^1*`Q zc5n`8(ij+3LeJBq={M{h7>wC)Vv2~dWNr4n;g+HMjA2`w7neGIfQv91yjy$q$_5XIFDhM=L6*P|(8+^eug#`z=}@3p+pbGP@8?aUjxhmHhtwNhvoTlKeZ+EU&VB@2a@Z50 z2Z1AUB7EIlNMV)h3r&@k*K63UH~7VW?eA+_M+0{Uyp#L+GdtDXB#H@gRxUq3p}!9A zbSzXahDJ?h{FhJfmT8ZNi*k@2F)`r}TA1tgtbWFc?BXQ5Ke6c!ha{YxbCS!TyIfKd zok>Kv@u>)qLZmTo$_C&8u+b9I_XEKObRz)MMpfKR?C&dY=R~FC?=(vUg^6tW!=&{n z**|;*3*iT3c!`|)SKxx=YPtAB&KOiGqKlBbI9(>L)%P#x;|IQvQ$EF)zAYOJ z4Llu{Xl36h;Xac#IT%k8? z7;Yz+Vx>r(5)*S`2|ez|0RNCVnRU8sGp7EVEdzc`!8_-ku#uHl7rXu{!=4vr2=pN2 zqVj=d(x3%0OLct`h#&$BS+D3Zaw|EAq-8LQoyj{k?(m_$YWvS6VK%+v0SIRlcL@)$ zTKsb9r7-XvQ(Gsv)!sPrTeaq-^?tiA>gyUO{U-`b(n-G_cg4m9$%yo_%#<;8znBy^ zsi3AlB7JR#qu^Jjf5vXDh9SvxcBAX>)`=6&5<3mrguDa&an>g4J0}VG=EwzMT*|2mEQ|NwUkwH?zwrv6E(H7S{%LGDFPM7aU)7-8fbK7A zN9ux-NXr#QS!Chmn|&;3S|Ak2F5{`VqK+Y({a%`1T6p`nW|4uWz$RVtkPg1?OQ(?K z;ooaKC~qTK-WPEB%i555l@4Eq=u6>9NOC17yq>u&qm#gx6OO%_3jdv_o>pKNYZMnC|4>j6 zT=u+9`T1krXkB()iKX1#t|NA$heDIKA~C(OTl0_UJ!iSm=K>C}n%*3B6cVclBn^DH z5gFGSd)QNTwr!!KQKsDhy;OR~8-xAVE6kt3TtL)%dP|nv1r2eTPS)URf;O0r*?;}d z<`dE5!us=PN{VZFnX{oLoJ*JA&sP2#L@n+)x_XL)?F`_NUHNic67o}C-bFEYBz(|X&J@$5^tVO`SZZ^ANnUlb+J}$q5)&ai@s{6X%P+NVz z^gOJR`dakM^*DOq-W`j_f(}0P{!AbChC)f#vOX>=ihbG>L)gqK#PD}IzN2K;jD$wa~if)7MV5gnkzuE}&r){*(YXz;X2Z(Dc zJNubOFTb1KX}vTG5fLZS&yJ{=|3YE>lHbW80A_`g-xSsQWnavCA3)l(gUqR^ePx9X z4-YRoE)MV%P%^drR`LN7r(ppFkkvIcV{mBC*fWsE;Ks;6t1Xx&uZ4c-CVDiYKa3w^bolk*mXDf_?}=YNA)tQi>_8++ns@Y~6`!QlRl zHYkZcvLZ4||7Q91I7;#Sd7gfm`s393eGQgmCc-HEsP3NmeXlGpVdcO;vChuUU44{2 z=#+7RKlgB%3X#^*(t_OSkoy{9%gD+i@;%@&0&fu`gqVWD!rD5#YFH==tjh|8xbUSx z+^^>%Gb1C7*N$**Zf+=pH>$k+v2BgMr0|!(5X5Zd$?0h;RM>(P*uLm|Idh2Z(U|?= z!v~u|j}t7HZB8nQqH*=|BpavupewMqT#}NCcjjT9@A--dBM$7U>v)z=ymb#d2Emot zC38cIFLf6u&4qRDg`w-Y<{7!|X1!aqLDE~<;7H8@|n)bMQ z$-Rl6*Y2$BN&Z=Mo1Xc+|14H;N%hxbM!0pr6$DzrG2?`rDg$b?&E^y73;k8#f)sEck4&%KtCg8Z>t}3XK&=qi?b zuR7O6PXx>BmoSi7FXZ+UZnX3K+6{&3-~f5uZsIk9hT_ZV^w(gU?`UcLq(QFm~O-pWF1r5o0r(I){_D;%&=hJbk%qzDfxtGAyJf~CX%J+2u;MK; zK{;gDx5&X4v^q@R#e`ppS0Gb_{i|U3TfwBy;$w|)BMeqnEL86{*Cg1sOuV5wy5-h| z%ZwU;1#5cPl)ypW(7a>*A8y>Ba-{)SN^un5bCUkMe?T>1d-5C3B&-;aSKyXZeLA^* zGlmH_qO^7k)*xj}CCf(Ts-=@;k+MM&mC;NX;CRT7z<g=EY9uUtJkkIEY`kzngJVyBXW(*y zjs~_44iZJa*!WmNu{^ogU|J2Gl@v~I-*v}}D8$B*v%&;xk33sCL;Qd#ABoEo2Ur$@ zW5|?~#OAmjnq%QZh-4TIM|GFaQ)OAsrqMQdKsMtS57#))3sLbQBNy2I)MtBqvc$`? zCI_Xh%&17abbRTAdXTG9Dnp?rG@l9ly^tC)%R^r+DRDdbN)Awm26YRrl=m}R3>k8e z`54=B&+=>!`E@>yu%N;3*>L6?77S!Ys+0oLI1y6ySI4tRjT;$2%Dr4+oT+!A<)8QC z&IMcA=~!IAn>7c$U)-k)?~02f?D=L-1HttXecy5Un94t!AT=;d@1j{JqD+S49LFH4 zs3wojSVrDSYP+7aTI1wvAvNeKrWeWYYKnIUcTfvM6Z|pr&N_oo)bG;&BXc)&5mWa* z@I?2$g<4sIUfW<|FW$d{5<|kgkI~sJN$lhCXtvIYF|pryLQU4C3rF@+dU@>U&w+tu zu^x?XE3cJw#6vaVSb)Q*vTn(}!GB;_;vj$u=FEobXiXxQGpVfK8_D4XjdV0-T)kN( zcGKioPN{u#__3lER4!u(%WOee8yD}B_y-N|C4?Bp{li07-bc^ToAO|8drL~-Cfd(_ zO_X!xFSN~&d?5IywESx%I3j_!1@#nx2iKH7>v_l-+IP1(72v|JRS=zGdlN*heJUL< zQoMxO4C=9i+k?17W2xoQF^M_?v{Mu~!nrtt>ZiWesoxNP>V5cZ@_yU9-%obj2{gc+ zS(GayyerI<`jA|P8Gkf&P&Ys&w6w{i?B{S&S?d=oEUnzQ__bWCX-BI;?hy{B-@?#j z`IIknb%3?vGV$c<>Ef0#LI|if4{;+#=jN_@b^d!+=DgEoyB5C)yjq-_4(e+C9kUg+ zmbariBAEZ}&GNWaLbKg{(zxJDe|FPmFA~U0+(13^{jon?t8+u(ByH6?!w@Nt{9POQ z?y)3{g4<2L*rodC5#dM%B{JO`R>`Bh+4s z;uqEii%HU?f4!=E7$2v#_y>Uzw9-!ZPd%26z%@E!p?ZjeBx+Hv|9#2ljusnBlGF7r z%efR-q!CiY#Kd6AeB0hGE+ZpDOi5{JViM#mpycGl-rU@bgzJF%G!B2AR&IOF1w0Wt zi8?#;?=$CmBk>S^eA-Rk`iZLKsZuBQzm=n7!pGS4F-8Xa<~--G%BL>gb=uY>jFiUa z?xxT^A^u$V9pS{eOA#Xji(U|3f{jkk>e@iGhyX_;m`7>j<(h)`t~KC_0~$^*#6f=j zQliFgTi*X{+KmA;K}c-;_@NDLtu{6`X9e*>cW!!YPji-SHtCmD`Z|Ymt2_xQ(iR(m zVB<-Hj?uhQgF&8?GJDw#vFD!K3=4dm)_$D_WF&`Kf;8f}=K}_*9 z(*TwQY?ie9NkRSA)B+?<*a>h(=|7XG82`TMWM<0M)8~~IG}7rMppZBo`|X3}G}%8P zKMlQ;m%dL(GQ)#V8cgGZDr`z#8-y8^TF?nZOznV)qJRlti;n`Z5-aJ7l^&#OG{1Ug zX$BsJ|>Xv+bU(BXa zI=Sy{U@aX_akF4(!%;TD9JWw!+E=QgMa_Kt89up*R%D8qu)#)|8VoWBHTrvQ@2@{I zaL^_l1K4#bGU&Pi@-IQ_v1786$XMnK0geu}$i1(#uwQV^&YD z{pF12nPa0uw91U?Ex%{@kruGDfr|>z8fdaDhLhIcH+T1f_mWW0R^OQzPa=VRVE4lyP=J@z|^p zy3v|x&JcwPcR#d=$N-ChH8q0IT_luP+!b|+1}$H;}A?}eig=qXAFM|p!Sr#`*8K36Jt1CM06Um|Kpq;}*?~JDbtG!HYlvyF((|@i zfu6>n88a z3rv*XrEKc!Q-S-%D*lm3e-&cw&Eh># zVrvHnkgNciVQr(f96oI9xIB4G=DqEyk~dzqI4X%%I2qr`?9|C|al3EznD_0>C+#&) zLrjbNNiV4h;_n0mO}mR^e7#Xc`O!j^D-)->mifRqg#7%G#5fnnwpGOQm|(SLO6K^Y z;@P8prKak1?FVy>(Q{jxH#@wZ|*BdkLxl> zE`08EwyPlr6c6&+^5>5w6pIdczE!?M21qz9vWU=H~($%92W2CeU~Y|;FS4dj_bjR@49N@pwv)Xnm=3tH5aCQp4>R zLMf(n_`AymZAoU(zC&t;v6z+_SKeM;1F80-Np-fUzQ_*dUb(bxcIUZ55x)&ZV^i4n z@7mf=ebVcERQIa`nogA;rxr9fXQ!kPg0CV?#Fq~c{Dg;VBUYeN?3T=V-h`Y&=c1QN zoeQ$7rZUbyTA@&)O#8`t#atjORF^08)4luZdi4ms;e?eX86!A2IA-9a)pPq-Lm~mI z_8#;i9>Q7e40WQ{dOJ$WlrX(}Mj-_#Jd{uKpX0f8a)xRt8q??bp+LT`G_&^YuzC92 zxV)zCI>lHnt zMZjXRbgCH*JbJemo;PB)G^nZavp2;%y963%Nt{(VPwfhn6IN(F2I%{>jy8UN(190kl zLx=2UFg6k%kh}C5M9VBUGWR%cigwTYTrh}+|FD8V8_#a5ygn@XV7)LnUM{?)@VL_9H7pBpALysN_HHBo1 zL{8q30QQKvu_Z|Aca3_bSym|0_i_s^_4DK~-{^?F#`*bN^MDElUXV-%9sdbE zI<)evbU(ocF#srtd8#tt4gC2@hM}IE4>%1O{*J0oE2iK?+4KDs9Y)LJn+ah(Seo!a z9_7QAp-o%*oFNVl-1ptPdD@A4yn@V0G$?%uo7E<~Brq!ueXslS(gZ)VTM7~Zl4dW4 z7&<4ux_e+1r<{eTr^EK5-;FRvMv_m5dcnyQ;1I{!RPORupAt97pD90Clta~1!(WU3mbT((m;jMN z7VotAZ)lV_Uqz&ZEpLRTXw(Z9QW#)pQFb1Q!gmC`rtrO6!IWlWz}v9T0ZfB9c6gTt zat$=I)nof(F(Z-yc1pzmzJ}tp{U#GFlK~d*#Fp#w?LH7-hE~=coF3*31fjx#s+VAM z5)Rj*=ayoSaj9BD~(6Tt#|qCF?cDG6=HH8Xi*5` z#6rvkmJamBOq@J~5(yTTo;7Nzbmy|K-D-I8Vv3NghXgMDnDG90MJzvNINxEAfYFAb z;6`cpM7X!}Od0HO^fQ|$ws0W>lx+kyW1RvP7J{Z{$`Br!pugMN`%EaY_Z_T#w&Xs% zgvvj14_5G^KuQOEy^tZwaHf-YDpK&$D&ft@5Dt&*GG~)_vfA=>{hNu@)Y{Rc)uwA5 z$o}Ie)HzZ>PMl@_w1@w6LnUzFq~%0J^r%>6AC7T~b;scaCv-J@y%s;glled~=x`xW zF2bukqKS0~F-y(v_&e9AceS1PY3_HAwv!J3PmR#4jv$s+lYaC{Uv5mwF##BeP+iZEO z7S_21{=D~wLyMIT_wKO3^1WEm^Ys&RVYFAFya+)2tgo*xEZ)hrq}j(f5&dVoENs-r zq8a&)JLns4^F0oshi1G_Hgc>?#}Ori3f>_TZ=yQmxS?Qj^NT(~ZL`6eB%}2V!j6B8 z-RyAHYkLMauTuEO2bxIKF_(1js<=N@zz>LeM$ggH+l%zb0ldL)g_f#)_oQ{p|Hp=# zflD_*(_RPWucK1q#ghxcmA8_*@J$-CP9h3CQ#AU%!2wa33xR&d;}rBu(7k_KXu_xz1Phi5D&PRPUL#Vz+=V&qWefk z+V9&}mk+WKUW6M!N;vv%K^&Lf`CHv-R#t_^bL3F9IN_pO-=0e3E=ptXGY-N9lO16v!y(~Qp2L+hT1Hh zb%AQD1y&n2hf&)k@yscU(H1Ps#%!f@h$8lMoQYhd>DA<6=QqfNB?AJq;o{xt=e-B;W*c>f;ePqAw*wr7MQMFo{JCyuL5JX8!qzq zK=-prdoG2G+voNTx9<}ZQlRHUJ|uMz$uv?0U91>q@YZfNCOBB`VPQcyLBGsHKlEZ2 zac!4pRH4Cy4>LaGKk~t+>{+T@<)K@83fn;)og4 zAS@buX@FUA3JXWK3O=pWl)y$)PiG4o-d(QYuYD``Vcpv9g@+S=VmvYJvUm#lOS)EH zpGU09!sRd4*yXuK%Ab!wh(n^SB66x~hCEKRA!A%wG$3QAHJ?fNe1#NNxxJ4>Ne?Gp zD(C4gO+Is4vT@}uEsC1rYwnN4ONB{kT7*MN)YyD%Ojfm6`)p3bj&ubg7OOKbhr`kB zd=_GLQ{(YXCr67_`9XN{DPKb`?6KjMn|t+(nl!z-fScN->6Wdz45=~3zx>(5u5-|9 zTW=o^N41dsKk(=ua*1|&xhAFpH_t_9=Xw7Vm;G)cs&&UJwQYA4FLqiKNO46AF9hp%&}+dN8IsQg zJk^8^a*xc07iVTLi4}2G-|>sS%?J@a7MDlp)pwfiMJvHOnD}&3T;d~+jg7`Z?3OgB zg}cG#7?v$jiG04c=;rrZ6sfW;yH2Z_rvn>_8e?d~hB1Cfwd@Dft3j?${ykT8n3;fm zicDd{?2BG|E^u5D6T^JA`p#ycotj(`a12(aT(Q(at^Lw@x*g`<&3M*rFul^zWhZS9 zXG+Q%0Ui)POw*4G1ASU_q4V@371W$Z^qez>VfP_kzLb;GV zVb5zdd+kHbC){}c%tz_hB#fE~?_IT)534=s9IN)9-#_EL+^!|$&0p3#eR606&8Ej1&!&0qV51_+WaL&PgPDDv3 z6A~PXR;5CskW=yEimB+AFVybjNM^K$N`Xdt;9D^yH%JzWe#XfbvXvFWx!zD=%Z9766eug}*0 zT}Tf*F**9cp}z%#9{GbiWsi2c!u+!K$2`cdA*aN=x)8I5PiUBiCN5DrNi&F=nOo}m zv)gu_KpLK_+@h*IRb!aK<#>kayCZ7Y+i7$aKoh3`&PNJB~1=PBuZ(pMS9p)CC^#-D4 zmW_qNN&i|Vc(2x$&Emo%NyHcXBsZEZH9p{wjEBhh-i3;QPHEqWC?=~mHKn_znaf5u z5y|*QUpNh1`2l4?jqPO||FELvoCWZc9{@!K`pMq^83f+Y$+y;(XftbJe^~lmE%L3t z#qCD=u2>|#!V5vWhrSNa?Nix{ngZ@jKB6^u@+ZA;XFaP6yx@P_{7)Tnfu-dQNkvq? zSu2RfxI5dNPiwjfseC9_O9)EP7pz8tsNacIjL(4F$SI+nFHaR1}xs;;&md!i+HAw?lpUo^mf-Tst= z<>d(;@AY!xMRP|3OT(*0 zTVCddG2AIOcfGo7W)a=HwgPOZ`;p(N5ytZm7hN!zzIqHB;mY8f>@RhLN%0H#f<1X_ zOSQ{P!49))n|*=__ynlJ!&FRte6{R!{p1bX;BGqJ^FsASI9ZF-)#ga(<dQD<8-U;5(bFa4WcdH2m)88GB@5I`sEk>7ksMtg=10TAL7c1|HXX_m>RS@Qr5bN$Li>UpH)WtHFWX-kW3DpHEBX zi1IFgQbo8i_V|aTK59VL19#;Xql&{N){3#ou~4H5v!g@y!}zFSnWAw{J{%hSSl6X5 z-Es+C8Y7}{gBuGgtBRwxCu0jC67iE;MoVT?4d_9(Wj(Yi!h`f{x&2bs39stn5wjkD z+s(O_4t>1-cIyBr-n+^wUfJrSCjDlza8Z`)fH5K%g;(1V$yh@|L2ftB4ZBYn_6A<| z2JU6=`h`L4B?)NFfj|2%Or(IU01=2H)SzY}d}+0`Xhx=Fmb930s#-MhHe||vWn#ce zPX-%pGeN4@Ag{4GB%+@f{O0&3)@;Ci9m*Yl#%F8-#w>7PMKBg^w@AY{`@Imf?ugo; zP4c(>1n_#`dt=i$77i|W2%&r~Vag8%#iK&fG*I@@Z4TEYaN)yFJZeS?*mv4GWPmY9 zg8>wMdzyQwX~r-*S_K)Q6)4uUyQiWAPBChHd6Nv90leaSFt)okC`7eJ0B@;A+v*4Y zz5oZQROuK*3tXljg})A}>z6NAHRVb%ftLmCUJbu*$0dzV8v(Ff{9-%qlg@XP9EOr*!D|q4&1KQ&$#JdhH__TXPgQ8%rW6z zxk3aH07S5#wv6)`r_809#M@v)ZW6N7G`W|TnZG_k>eGOWfbc;A4J``|VImGl_x<>JBx>-(DV>2^lF1Lv)vx{Udu^&pl<{j=ZsdkJ^Ncw=kJ|F^2vUGt z4GZe?;=5Z?EWya=WFIoE3EV-y%8xQkNrT|#j11v⩔1+jCwVO?ccluG)Hni%rBp? zV@vk$fTJ*I9~V0Wb%hnp+3D1+t{`X z#4dm%1vFjG*~UB`+l& z8-qgtrLFIjPB{$x68NoF#=Nh z<62X58j_UsG+(=PAK6AZ0%1GIe<34xc}y5O1}?X1^kZ_b{}t^<4Q-QL9=QxK7dd~I zXRLIX<>N}fhr$RZ7^+X7_O){)432Z%nj9Ju#8BIX{3&cszFD8Q9LDk`4dK3FR%3bGL!}Ul(cKs)d(n_oHdiv2fHrZ(pu*zGTMK_4!3G?y3pw zMe~J~*7KI(91<8H6wOoXedIMfQ-o)c_15c8TY+)fNMG72?gJ*_U@dPK_COE%F=~ zr)Nn8EZpGX|20ZId)!fNPcj}CenN$$(m6wf{v}kIK#aTYf7R*0R5^ykR{ko1>zUle z)84AF5uNNwo&d)3%*XUs*TK;{K#r1Cl z?4)i;ze2P$c63PcvI#0l3Ki{%ooJo?JkofBxwm{#nicqku+&@V?6lKZr@=WSHZwr~ z&xfrTx^WDwiqgZ1slT)cf?L-+y%vu}p2_+iwj4ot(aYv#Ho~(Y;)*=^Za035w`_=$ zf6vqMkE1x)-kYmJjO-KNNnj#FZ1$7RJ)-IU($PDT8F^QUN%+KvB1&!Hw0q}Gr)||`OSM`LUXW2kL8i)9yf$Nug0a4?uCCmFzg-)YiZ$~! zS(=x3J-Jg~a1ei<-2v9qw#S7ry|$y5S6WU^&PmhU39XUu4)1^7{D9LJ3=9m!WMp<- zyVuQTyPT)9^VcaV{+k*xt4}*!)Fq6q zPWX?nd0+65TK#X@Pc*F_IzY}Au>$*WWXZ+hTh-c%sK^*QG6MZ_Y2WMjIC@aCm)YQR z0JRhTbFX^o9337HyQ3$&1W{c z#gsGvXVz_*iFx>3ZH80PVY*af=q1bUc(_wn6I5GP+bw$UP)bP ze!DNz*Sgh-`6!K1cjX0XlrZLK91tm%Q7>;ei1LDZQ8*W-WA>9&o_UbPp%1BXGG9nB zX#ld9uwsBPC7Ah`m11hBp9*Bax)*l(eT#=aGu&LSwDC9o^S+B?qglGw()4oy_n=^W zGi|&q0g=F_k+?ZsMu^eP1w*so+z#w8fCUU-0r*I}7j)78i+aARD+rMjmT#Nw@?uhe zAsA@A_il@PVdE}sypKQqTTH1z6E38AXs51$Z8Qi-kv$k$vI3amWx8#zNn7u-q~o`1?{WfSR- zIPgtvg>z=~pa0oO>xsjPSdhR4X5jZ@EV@6G1>U_}alOI4%P$r?LW0gIk}ULM6jh&U zuk`qm;b4rWSkAPxUgT}Lt^E)(W|y0DX(~@}$M2R9l$(q$UjCXfcl3__cS70jy*xu* zGE~N}&~bKl7;cE5MFUF6ZcC%f^*j~PC*fc_gD@IvUV&CG(c+5vs4Fwp&;I|+4^CQ8 z^25t32-Z>`zpFle(*`V6)wj}!Gcdk;?K%kA(gJW(pns`Y19g4Mb(y|CPJ${ud zs=K;brR93rVzo)`TYnqhS!HHD@M0K4_MeS_cV3F0+G43#`CcdcUsl|PlVZ{$w(sBn z!%gDI;S1UEP9AvTkfKs-Ty9r242hqz-MYd?+VQqmZ>*Kew_XlOXt`j z5^2?8+pD1BH>pDZb?lt1=Rlg`FDL1_^>0V4{zK>_n`%j$jbhDsifl=9jtV$?M%z-OzL^J%WZ} zuSbB9tYv$x+?1^n6#kgOX~C%oplY+&J$Wi|OlZ0c7pp*s1@a#y2P=Z=oME~E(glw}Y|GHTcoP)-atf0$A_|NCc z_YVrn=#L*Vk24EO4QsZu_)8PLSURNyWZ2R_Jnpa0;jU;zq1(M3zO$*J#rw4r(wZJ3p~c8X%Aof3IC}a}(`y zxSbPh$)sv+ww%C*m4`8+v@CtW{9dz>VxFcBK0a8! z2ovE$$Tv#GzV*tBOH!wlDz#IF&$!K!b6iYV92mNJgj$P#~QhB)h&OkWYtfkAh5$n<1#4?@*YPiV1L+Nv9ZFU8nkgx+5u1p zS#elQtP3Y4%6REnKj&5H#H*%aChcvd#FdmvP_xwwCCYX4 z3no<%djd%O@g*g6ubT2D46KARj-$W=9Eb+WNGiVbp2(B6o$lY6>%mvQ#DY)6&Mz+X zp!E7d*iC6QudT8=fyFm?0{dNfXUBD|-<&}21NVhhj`yFTH~kPn(1D^)h?i+KeVF29 z(*iY#A+dPB$}`=l0%}4j{;i1ZP0_E51+BEQ4Q6@&N&n?J?~eqGal5d>B2FA#yr}(Z zH&xBw?#<;*_MPXB-_0P@{88~e;^M`MQr>>XI>&P3759WkG`FX3Yz!Iy0pvz1A?^a! z&Gymek%K6e`8P-Z3giPX#NW~x7&oUKY~4&U{vC0&WTDA-Uv*2TwCfooMd;PnaoG2; z6k74|*U}a=u@l2b8T#(f<Z2|k( z&}2nT)T`S0AE(D+q-2x(BaR=r8zs%fcmHj5I8FV~3wR$&by?C;v$n~kWpeShi}CTO zO6e!W?>Q~Pf01k=rc~)Tv8;Y;bFTF0wc+MeOX7oWc6w08_&0h zR>SACIqIX<3)S=)J+g-|2aVa>day%q*K4xy<=XHqlM^QEd$zA$pNgKcAqW9%Mh;GJ z;iCx+7(QSXi{+O3fONu^_kWYAF~Ove0g+E2Z8!q}Ot;b?6b| z?)^yIGT63G+m;}zhwKkuKmrk*TF(CD+PAvBQ)MHB1`^_!o)b+JsB8IyQ}kE?l>=ZJ z_IXcRRJ&}dOtQbM#4&sagjBT8P4K+z7YloeV*|+?F1zUH5Xha_#9g%|Xq-fLXu#8s z38T>URV2pAo}zR5Ks4Dyt``sfBEJq48*ZcggokLrK=zkU+j zjeFmWh`QAaRD1?+MI+A%dI`cz`EBhh7Ja%*! z3}4HipWl7f14s}^6_a2|M|5dSmodOuRUod_P7&P&+nfBUEqI;-F$Ad|t$1LoSfq)S zCc;QykoRWHr0qXI*9|Rl=%`UQln{TIA%~}8xyM(W{QJ>0!|OyBaDl|Lr+l2EPo_Su zAG4MQujiR*1W+)1%2GKm%9;ER&s|vR=yM>cySfTNRbXkN@y>xZ$*n?W>N=%s25=2WNivVBEv`!BqK&Kg5 zXtWXj^Kq^OMRO8``jQjjatUZ{u(wQ{&Fi5v2TyX}r*duZ>z<4X0PGC{QE*9}KI%Zr>B8UBBZ!!c=pGY}!IP?^nO<%O%ky7m6UNiDLE zU)gwj=+lRns`b*~QleirTszkZXn9CsgEqY7@sY1554~GGVpv;@vMzYRJWGyE8d_-l z^8Uq=iGkgyP`lTX6MnP6d;~;sMsBECF?hh*Y4%WkritY?b4ohAsFaZGV`BhxxQ5U$ z6yz;>?YXDTui))vjeh6%<=Z!pR4Q-t4FRGEENp%T z^dg__M&b!{fC3Kypdzw|2su@-1IPkylE^F|kPTd*!3Z`FkG0DUBKpmRMh$^Q0@t%v zGW+?KZ0NRtwnLo%s075Nod1mh5*+KF12^50Ohbkcm*tfPZnK{a3zZM70*~(8P99Y#bc(Pu+%+znU`Weu4%o7}aQ`cfwn ze3Y=vTsE)lkSZkhK6gHLYmi7?zb<)T>niY&^HgzR-|HQvG!jZUn1kJLdoU)@DD_&ugY-03rKha?B>L=-EnIv3rvdJ6v}21C=c2WbUSphtCl# z@R!3boCi4-xNqeI-#bg%{Or*<+2d z>e_&#^8-Q;JftF@DTKnEft_6i=m+GU44JA1R$+zX&kp&)6hK=*bg_H)=rcU20X6M) zFVtTeH%_=A!;NpmhLH*Y81y-Em1ku;JPqVol?QY~7vD;iWK8=aC1~Q8F1dC1`c4NGqg?|!lCSMRa-`;XD(6bvZin+IygfW zG2F4Asl5sM+xj|_WmNhgy8v0iqC-i2D=MV08jY;}ZzJO$3^g{SncsSleUeC<_Vr0? z;Je0uf%y<{sL`K4L`O*-crV~Wf;6G<)*#feof)7Q5#;k$g{=E6yT7poS5?zCrqEw+%C9Lr%~p0 zTu%<#z6=Ay#*QZIz?0`!Ni14YYIL}YNPf0&TiDpUuVlzGMRWJbG=1?8Tacx; z`I5=!xp{`c0l9Fot?oK9Y4R5+}{fsV&@n^Nh2quRxQM!ea(g z1nilhssOxkKR?)?u}-oi>&Gyp-b@?|0_o_{0&Q|XpQA}=PsdQ^T*Rb3O^O0725PtZ zEx6yp1rv$29uDRsv&j4sNCdk#Kw z;4v=uo(s~6Fy0uY;+1v}=|XV?6M!YaZVITY8^(ArC;(m?1EXIl_B<)*X*>a}q9CcN z$^*d;<&Jo%W#S0>@%xH?B+xP@3A&Sh1-w~_s;O@a)^E-A9mZ`r;j8)r1ydVEv6KG2 z&&yT2EIBopv^bP0WZ(7Zzvn)a!SC7gBG%JJ@5Kc|D5M(_Xv{`8uy4gf8WaRK7`A>` zYMFDuk6r3}(2}~Rq=Zt70?l8;kQr1Fv5@q(%dv`@CSl;?Fl`B@%w7BtfO{b`B|Yk3 zEI~Oq6J`j0WiF)1fJlu(IVKVWmco~rZR1~m?pb~&ikExiF!RX& z7jY9R0e5H_^Y;ZCcE%AsklmMdtY5tCc=|K-^nkT>k<9MzvRK8HY`|sqE)Z)^_5ka* z-!ZsFqW^n%p$|J)?D9UO&LL;^qr!NkjgXxC0OAZa>8EId7XqU*n83loAv(6?kQa_I{W2Lu+K`kK zL!t)h3c|3pbZi?(@Jb;0>HKG{4%^q3abM%0ZK&>?URl7$=U_KuYBL&*p!U9;KpEqdX~?pl$wgk zVnq)8q=t3PxoQvO_98$dyYqr4%e|w8uNG_IjVaeoKzt(6U%Y#MJ9frqO!li{j%B02 z*-SXM1ntGyiL&xLEd@`-Mp|z{TtNXD`1bMf@q&#I)J8oJP%p-ZoIC93UEs3c!rpRn z2g9d12U;trn-MvAOanZi9DXp{_+AJjf-of(5?Tbc?Nusb((Vs(qFq25_07rH%tF*bPO0HFfAio48NQi&*`#6}tV5^S*W( z2mP?ey!j`D-v2@F*39T&5X|&PG>BXf^s%ayzsW%A?5{0%8atL)`>w8&=R9ZA%X$lf z$T1nj*Rs(hlZL`D@21ewhKABs(D{wrB9@?2@vy?8&x=!37_{(Ao?34j6l@V2;O@iT z8Js-6bYs9{JAF^F}7q3^ETBBE8YzoB|o)4?x%+`!NHWgPj#E$ zj`np;i3Y-lbtH zO{>66#iz;akHcah`bV;yHbZ%FU%+aIx^=P2n2!t#;%fM8G!+@V{iVh{)0u-FdJtm^ z`l`R*RS`ubQ^x4{&CcHBb$`(rXT33l$3n&u`jedz9o5+GE(ACxB8s8SPa-hfjw4F7 zw9SlCV?=h5C1pbd?ZG2!N{Ys!ct!ZmZSvvD9E0x@(SZ-~ieG*AfOu zs&WxY)xAI~+udwUW&*Su`I7eqg^J=kV_CND z-dPXVY)a=${G``!HE#3u?D9ldbxe_A7eSUfuFM|OZOp?%fT^+Ls;G)#%);VW_rYrC zZ{gJH{nWnPNo$rw#g1dk7>}s9?3eMy_78E;AFT1(z)J(EOqqTJB<7gJ8q{625!gh8 z7I@$*<{x|ot9^>MLDi}7untkZes`<-hmbPd&bU~Lz~L;ZGDK|ua!j8Y;;==ZDET!$ z&aloZBSGkIC!O{EAgdzhYy|7te$Y_SGg4e3UUHJ{*-Q6ySCd`MS}JkvOqyzqD1DBe z&Ck`P)%dAhP}mu55G0N{_0qht$2>{b4yR6$v?xOfE9YAkLs+q`-NHX9JXE8X%;CLR z6<@gz4ls(Gp9C|;>hv=Lrw!7-w?3Hp2;bhK2T7a=_5_FxpXe6)dLHz1cvz~bMG?)sKX$TFQ zir@ILoW$yE&+@7T$v0==ulh^ZYinz*YrE#rZ+rkFq-;%R?;X`2W8DqQXGAI54fq5L zD=_?GyI|&i`L7LQ|J#Eoqgvhn%3&tzn!BtE)4O9Clzzfie0ADpGghRL+nv-MMDtc& zTl*eppv)nUNBIMn@kgl=_!lwt4RX2`CyIjosOY=UT^N=2!+c)opk>~Ac>e0#C3AFH z>g`dR+1$@}t`o$edY&x!(nC~D_f{ zX)Pf^VGLloy(at#JG)PAw!W_;ZDO4q#c>&I~2TZL3CD(gXubm^Ce}{W2bptVVBm+x_T*4J1&1uShgvG}*4dQG5@s>mj0gfG!H$)s$!c7y%aUZv@gNf{ zOJ5+4=M88PAcOzl=m;qWFVdV08= zL&y2wxwl7C&g4qk9~Yu;9-E$~9R5gmrmtu|`gAbcg{&I5 zj!e3N64W8VxL-k{JC26JXES!9`^a*D61p~pVd9HsMnS~FONgzBa?#0DdYsy2#O1Ys ziV`8)(b|fl7~rQVB$cgWVq$ArEA%r9_FV?Z_4_pmtiA_E5I`$Sw{J-kq8bgNv~Wpvlq2VBj?)cKNa>N+JWQB0!{UWb}ZYLJ8*pf4+sm}hB(nKbJsnQcT% zkYnxs@uCg0p#kHAP~|NRf~naDO&sV=ck?9`7?Z%2!%ct*#O-j=GkSzIngl&@NbaPK zyk-8}+cI1iC>4TpB9QLUNHQh-)_*=FPYq9nOU2o_-?-_ ziWfDLzm)NTE^f9wWoD1UPAh>;LM(bk*h_YaeCDj^J_SpsqGyUwsY_lPsxKNeZG6+n z>#4|-Cui!cBqxm=CLzSE`F`6Cp7avF2nog{IRa@f2d0LvXHe<`+QbdvframH>osff zA)a?*b_8FbZdOsdg{E{$Ueqw2Q2>pFQG!6D_*t<0_<;)tMu;YQfaC^0XPgoTR_|-_ zTm!>xPZ~^_5Qe8Z;phO)P>J{xD~+7zh!Mo2J@A)+8TdZ;l`=Zj3bI$`uEsDRsLK&;4I+4=wKu3lDIg-qNNNqhM zO32T=L#lX0~ezdWii6~^`wb()ZKa9kQ!r_HcYVqV( zjg4ew4|EyPhL5w?tAV@^baWg+B!&Hw+L zI82yUPa*SgnGb^$DrQ12{uL81(belm+6k#_bl@qXlMJVeaOFcA6cp^`K^58em0mZD zq0i%{+^K?pzbt;J>UmA<$tEvy#qnK>-=tZ3rHhYL^$BKQ?SAn5A<@?dNAz_Ere@>} z-&bNJ37aIbu66rn9!caQaO-;5awj*R|I_{OG_rbBg!bi@grb4rbH+sC`hR&bOJ{!k ziuxX*2iNY3OBEGKU)E!XBcJs>*3zjdkSv9YLL6s&M5U`z!2Z?!N*GJu4>R4~?|qp_L2<$FB3U|q z^P(DAbghT?Y)B`r^i(ZMCiiKFwt!X=ZT+_H z2#Ow|_(w2ea>f;p*H`*ek;biVft(5sxJOv=o(0r>ycF8|?T_ySZWP!QL%K|u6TYWV zw2E>0TzMe@rVNh_7NoZuiTZusw(8k@H9WG`VUe(h@j}GgtI;0AQzccBH9fnwJ_e?9 zWVyl7tfx@v+&mZSX{Ms3&UIVQ7N0lkwAw}IPV>^Bfm%Se~)!Kj*CKH_pe{kq?a4&@|A z59Q0OgGnW6+e;i6doTRsj{d&w=9|bxu)<|tz|I1qI=Y;aVfs($XINwgX)-cB!@8%q zweh#ZEZU=AjeNfIZ~0r;q{|`>UmxK7ex=xJq(QtJlmpMP)flUg2RhQRVQ<0eCoTl0 zgH;PXPky)^te|tQE93baDDfudX%^Gs=>;6$@>9+nQygq15UmKadtRub7`# zTZtE;3wh@GZbq1d|K6nI;K>vn3xF53vQ()=s4gWO&H}_bJ1``@i>Yh-7_Xk#!Cj=0 z1B0sRCd8#eU8qfMioU2?xw?3%hH3Q&xHSB6`Gxu36z(JMb6%x*KjVGg2V-h@!v5S&nkB`20Q zGs8UMN~W{Ho_+>OYCb5bGG8;dwC?Ceq^Jw1!a0#H8!@<%bABf7uw3?9X>J5g%akH3F!q)-Td zTXupS0Cf0d+y$yu7TT6H@Tp)XapEQCn^>T4={J!se3m<^$bk-K9xLJAyxoX{b169Y zmbl5by>BYp*#n)QGhPJixd#R+3jLo~8~UziH?wX0uI~^#2?DpJ&a3g_o?3xMY5WHq zS9H|->>bBhJ?4egmMCSifW~!ZjT2zXz7i~hOI(c&iA@4gn1UL8Ml2c`HyL>Zc6Bex zJ4A6-t|dXOqTq%%1iO)uV#5J&M~`BCBTkyaCSi<`K12TLjthdB7)xdmP9WFUN8H{f zx9czN9ltq_8q-A?JDKO=&W*S5B#%e0&!m3eqZMFfXecfdz zvfA!%ZKHp-)?!P}ug^R}5YLxJp30WE&g(y0S-TleddOY%CWgs?Z8!I}oc3Y)RZ)9s zpCYBGlmZu{(~zd1<41PqL&4lSCG)(cYn3FcPkOTGwjH0b9m~uA8EV@MdD;B3hm}8< z^KSiS5V@w}$@U!?V(c#$;lqFY?F^QVda%=6<3k@DQB)ptCM?Vqh6R87a@asFy6!<5 z9x8ljh~3S^%=u z2<{ZZ(DVVLT+zk5UVkOG>`CxJ11n}$#y-)k;k&AyIIi}DVa?h6HinbqeE2JA(S>0#Z_LJ0XWFG9YFpN zg{96+{Q3}=`%UA)A}4Xkok#{Qe5M&0jXYsHd;?bPpI@tQhfTffH4+aD2dk1gbLh0O zOrZvN0HQ;NGlT0&5LvmB0+RA|8^*z^8 zta^P_)237-0oh8;ytzdv!dxJAW%~D(lKv?fJL667Dv!+hSG+YINX(q?pucG2Gh%-FaeHD z7)a#EU7dGXF~BQZ-iP+v`@1yos{y(qwLm(h@B6kza~DsYJ%R8&wneUQn-Ot35^pz> zRJwVy4PG(+i_5piCkQ@yKU}FtbTU%SE@KyE?adFTjYaV-Od|L*2kiIM1mHG$C z?BG$0wxY=e*Qug-Hs~jTZ-3L5OE&^KrGh8TJGN`?$Mvpn9l{mHs>IEdk78P_*obljl7{{07}ks7QCgg4-$t6IvC#6}wjL7ohT(Z4X@+p)(;awNg6RG~o5E)PfO zVKCjPBv!J%tZ8~G>=tnvQMAGEF$nG?VkW_cd;0XRhAGn#0b=Z1TGcTomAo5sS2;zs zfc%K%>vQl32z-NatqAA@c=l<)Y<@$p&cPhIER4<$)+s%=Dv`xHkOb>cznloMJAlOU zHA_NhG!hU;`lEMqX)cR|9_b;UP>#Sj{8sk~c9ZKvx4phPtt4BE@E(u;-jfRNc;GQ0 znILAJPvET^)>NISxWz==*(Rb$-rHU&pNNu=>QEvM%U}=Vy_xISo0=eHnd)ma?uw6H zFWYg5jvG;Pg5M5^78UZ+S?nQ?2AzvP`n^|FPy>u6@w>dGGp1CT-GHz2c~8m7cAQMr z>2U8gf-FZN#(Hr%{`XDuD&qU2aCo0itkP6?qzMa-st`a$Qe7=CC_?w=DiKQospTf-2C32z41*5)A#Sahi!LebvuYk zIF6xDyZXA>Glx9GjWgl9@YUG7;9~sZ^F}*{AS(7}!`iGQ25-F|0lqp_jrWo9`j&*z z!nBg0`w9-bMRD1nI+Qua%qKP)JkPr*JvvHM^KObuN!`Ru(73GSvmx=LWthg6VX4>V(3YnkMMOCy_}Pi76>AoV5wwHbEwU?F zC_4Zq_iuHyI1M$%RH=$l@NNUX0oe!uOUU}?jm*yt_noYG=#n>LBE5V7ypN937{P`( zX488ln#wuEPnuU-963=SB;<)YpR}F3 z9|dSLdpQ{s=nJgaMPs>r>Lq83EArLUd@oz2R8+comZb+wC_b6I>6QD zrY`}ktMEFw;~3)l7OV6W=tqk*6_Lu$+R&H0kvmUAjv_N|kZ;ZatN>6ET!QA>?vroK z@{ZLjm{-AI!w#T>$&RT1K^>rR6w`Ul?f7;%VMQP0yXhmxcl(Q6Km_}_p}g99HHCQ zs*YKv8zEap7uRQR&7R_cZ6hvHP>d0TcSHBo0WOGjzJH<8qPz>Q`>g(W&Qqaq^kN~S;mu`P z7hr}PBC6eWC#2XtzW*QlHW<-$bk~c}4yjmAY8RU7> zm`7x9UnIqmR9&4Lg(y~rzN-ATFpY|i&ChXtByeWKk1Vw(1|S~OPhKdFA)$A8Vk@_8 zGp9ie`lg|vIktCMgEdbfv3yRVn{Vq8e=B;Wj5*^pGWJ*%AhQ*uX~VD;0tg31AvS0_AO+n?eF6f4q|8KugjN5@yM)6_os zr0d>(M}EzU>?6$1_6-a~j|Ym!C>p5Z4|8({kYBmK(!aRdL&HRg=iJMX6z=lnf7`yZ zJCyQ5bF4tR>d`M-PWHQfIzgH+;;(P~@@F)yUoorN@si*AY` zG-o^%?tIiqf9+_`)dypqpJUzs;Uc*ONFV|QAm#Nr#iKM2+REvSTq5d+s9KMODh+s| z{zYYWIvw_QzTHPfDcLpnC?M6SNwPq4$<~x4ARmeCWl8OFdN>$@Z_IMR7H*&v<(Un`W!%<)NUBaOAM5 z*RrG2%$_~&x!Tik-K%3aB{Wkyv9IDCOq@zwbbH@D&COa9_T|d+^%JGhs+o{c+uYV8 zdDo}!-uffo4S9MZ$a@cWVK!9+O9bS%%+5yiTXa}mIF-I?8PI=wx{Pvgco^}D-c(6V z4Ig4%z!|wQUOEMIKGN|G-MnjJ^QRQDwzh0ySG!TgnymV6vo0uh^Yy82iDiFk>YdeD z5*jN{LjQ%_Xq5IiR#eJ#n_q+YPd}G>mDwb7^pRKS~GX$fSOD8Mm=RAS5IC;7M7{^imDMlR>;l5j$<%~flc7-AEBE+=bd`}D^8M> ziT}|6zhI3+79E_xCAHVBd7lb-0J`qXiA)%K2kLWZ+u!&nbJUU8YnIDuB%=Y3D?fk% zA&cY^jCf4PfmYW4h}7~SHxxwiXEzZ!eB??aI;-G;8>Af+-Pb?-6|PGHP;?&mbHLFTtkNcAIsMzX z$|772@X#NT9pa$1!>0D~*TXy>!87zAb^Ph)+iPpNWnDoq=5w{~sO3Q)FEa@@Q7;a9 zQzF(w-1xo(o4soNass8impc97jz`1eJK&ZEUK(0MU6Ht6!<4j_MMH3Ddc>8_D zgIN|+v1EvJ`bTy>F;JKG8OR$EIf6uLIT~!%);*?58gUWY=-4fao|KNH7zy=I8rtkrXPdKC!Y>n zZ$myMX|$<*as8{-w$|bVGT=O;~n>WbHS;mIk&M3V7$ItV!^o5r>)2s8Yc0T>1P5P}YqO zPa1^IvEJtjY-@dFd|=^E+#;EsGZu1$1gUtTmA!wpw6|-+hVe4m*hS%w(;Xsxra)kpE2LZ z%x{%w;jgeo7E%Tip8b`=ET*It%)TLTKB9?Qxp8!)H<*1Qfe~e=|6_Un6c37@Fj0|l z8D}c@vQp-x&m8-{J)Gv=XynelJ9$%^;Jp9#hXIkaPVd|KBu-=#6l!vO1N-JUZAsr% z_-8vauMbFFB0TqwM#_NrEf+qg**dGHll!4*^yvGS)ndAPX?c>T#gykON7(%HhQGDv zlUNUu>_2i56Mk9sczFGdlZNH@=U#!`Pca;A`P}*!`;?ab8l%6sVujYIiQP?%V@&Zh zx$~c?GBWLrjBwA3K2s@md>y)QBzODbpBL_(6gf`AOO1diSAcx-Vi&yd+dh1=W&562 zO_!8jP#%iPEf}NZ(7!uvTO)iqc`gX%jcb0Xvxwh--WNc4JwR#y9RmWC-K+cHp8u>d zF~cX?$5NOE-k1fDPf~8GcgHJn)W3(+8oH#uH>b-<(33_9omZw*6WWOo0FV8yhuSk7 zirV89?9*QUSm?Z?aQESh-iuK+`1<`lik+7uujDG7-IyMynrr^KjOg0gEgN0g>&`hA z*c`hvyqULCASQMw?5UEeOqcxU=L5WAhwF{hotHl)&OvN`LqYssd#U5d$ev#Gv#29Nm=VeJYFSIO;JRUwzSU~LG zj^j!6)71gvsghbsr{8Z;*bS>A1?8v|MGcMG|DD3@j_6x9`Cenq{-QGQ4HG5&agSGX z)S3T>^C-lKzIgzvYD>8tTnx(T=2Llq{{%(n|vvQtSBdHHzbT#~UW=wrzy94?|&DEOoMEuKRhVZ9gBX-EQE zUyrz-A#ch}8q6>XN_=6})UiZbQWzJN_M1K9wh4Xa%2vCC)T)ZAcJ?4CTKJ@EwX*?9 z5z+D%MVP2g_4zCHwiRaV0bJ*$M;o_m*!SsWP4Vn_fVFV z#J9PP7H +!2%3G}O*F^AXA4KLzLNmqV}CH?w`X$N%t-AfuX^?gg6hhph*Z7 z&!;ouNd+H+^rAxSI?koP#Ak;oeQ$4A$>C!FUwR`LIOBd<;1Z~s-cmm1fy)Y5yp4Za zAj%L6pHi^`7Olo&NzG{qP{nhH|Gb*Q3e1f)>Y|a(*HnB+c6I@5HQZ!y4U(2#4sS`_ zPvRj4rwAilLxCL`V`Vs`Ue8dpbOMx0S@tHVmKw(2`!I(?^SSn<{5fv2GCDUId@94H z#D!z%c=YQl)W2EVe}r5$1oK_EJ_A<)NWPYy59Do=Z|LKkNWotausYF-JHU&BE1MuR znv)nvjR!kDoY5&a#Up&oxHEI^>j>BOP5;kj{@3 zby9n7X>7acAmw#@%_&H4Gv(_c3F%x7Mr8<>U!~ZL2IznHTEb_h`F(}Q`}V+?b{z92 zDgf+d9L8e2UK8fezAaEmMQ?rlNaEtJHuFcCIRlPdr?ab>vmXyM(R{thyZ2|D5ys+B zPC$`r_KnKo`}W(|C=PNLhh*K5?Htcd^c ztHfw-K@nT#%Vj0198H$l#+Tvs07DV2-|s|-Yxp7!n#-R4?s8M2^o)+hQW^OQO!D|K zB$lD>M`;^VIxMiGULUT~)fr^BEk^HNo$ok9cjdKVAQ;yIB-?i09zQ`^hXL?WhI}&S z1Vw1;Hn@XwaR)xCQ00Nl6E&s!{$X#r$@kXzqJ03MJpU z5d@1kaz2loGFu9k!Y}!Xk3t{IGN*ScTA1OE^FrjZlL*V}(7?cZ|Aob%(7T7zZhu?_ zD?g|l;GPWI@E#oDSMAJ=f>;Kb@%>sTq9QxSg+govl0HtV0?(a`1yPVp5=BQ{%cVV% zy+$@WX82s4a@Ksm3j5t>cOzO=>F5ume};~5NRPD?W1K?Bs(kpX{zGXK1!Z&*xE1f6 z($Ue;0>4)IJRD&%2%?!jV0l3_)1lDFJ$v&;%7*ZNj4eiGvI_61Fg=3B)%9TbmpbfK z9d^7pOlY(5-^~GtxjGuVGamY)VHRQM2*J;Vs;P)bs zk5eW%(Xl+q+OBb>pKKYUh%g6ea8kSEo2GqxvzQFo zN9Jauriv_?Df5BGjT%+dHbe>Q4c2Fb91NrC4@J&`KLG+~z6)K}>gAdMZ&=wMz_H7m z&;A$ubeS7y2%z2L36w8yt~ zBgjFQ0~pt-m#;^Zf1uF4T{GkaP-oU9i2@U*x_~T)cI)aZlQB4d+46XvE0FSfiK$Y} zOGAOVr6mo#(I6-K(k%oS6cEt6@ndszL`r`sT*JV}BCvfI2=-#ljoYUm&)@sl-B5Q% zDT?V^{sO@hQ_ukv8< zyA~{PwtT>26O-yfe|@1^TieP+x=Mqd6CBMJkPK*h6~q(tfRE_RA1&-)KcObZ7)p%QkQ)B8~LWBG8m>)LN*a%|Rz zkgthA^OwLA`2Fts@XOmf$afoHjQ1T2NGv~CHkX%=uhMPIX38feIr5&I4$p`epyx^?2{#cEzi+-kxfEOAJrdblAfnLO-!g zw~$gp zoedbR#??{3Kd4PzSvm{OdAN~*9P}zbLPQ?*oe#sH=AHjWy=@;PK}(5V{F4I59nc?0 z1{V}$P?`(UGN=)$(cNyz`*D^DyS74pb>~y-qYuK7psG+80O2+!JWsrdW=?u{zO@4A zJif}5-U}=3pBg>b?GdY=?_kP>FvJRMk0fPll>sPWJhr)Z8x4H@uomZFu`DDiD}=cD zaKXR?S;zw^$Seeyjx{vX=V;0%jXDYgG>o3wG%@Z(AZ+IbiBx5z6+0BSWQ!3LG>D0t zHrV17v^|EYbqy)PaZ+uq^fj3b5*_s| z&44PQ-!4<@RFv&zL)q%gpnwI)0_GeDU8T)x+zyTTC|(JE{D9*D=OQr1CdJawPOV@5 z>0f_c{qEP&N2$s)uA75DFNTAF89;$EOwHN(HW)8Ll|hIj%@Mb?W&fY;K$=x0y-y-r z6EOT-Ef$!4a8CoD*Q9~x!}GsSKzh^;&7(2(QbSHSFquI>hIta7{=^=byH0r{LZXy0;ebui? zF1k(~6Y=BcT*UWDLx9_j?lS{3GguIK&xIkF4_pZOPRa7@zOrTQ0HMv&WX&x~!o?)6 zmw=EZi6JIdtF&wN)yNal&Z0?EJ~9PMXLy5vE{f~ngX*0SM#Ng6r75r$Hu*&9T28Ac zoqmP{GC3i{7P-{P`jNDl@cMy$r31}Arpz4>zyinv8eB@(+p_+6us-xTP&sg_o45_4 z7}QbK>ITT>qe>&zaX1=CmOmBdO&dd8KT&8m*}VSX?`Yn7xGupP1S$k(OhCj3)Re2I z^qo;o(5v6qWM80`ULX{#Pdi`7s?q zY*8g%19IiJxwt*^dQ?T zQBf&<%@$hVwgEP4_oHmuxa^PBF-nY6)|cNMY$Xu(WnFkE z`}!QS<68P`D$kUVgj4WG9vvNR2uZwy@a%ux--!yTipk9zuK&am(QFZXxjZLda&lj8 zB5M24ddqqIr*dxGHBH^^i|sOQF=4|91+l8Xm;EGK&}#o{UzYer@yBmZ9pi^=epijs z-}7y0HfT3a*qyB&`zFTs5v&4qDa7?J|60{O)QH!=g*WqbV7HpySe*Oe-cmc!!$YS? z5%sQv&oMVePP}3mE5e(Y6b7^EW}I&Sud@D%M)pW76RzoSA?gmeXCN?<9gYQ5dVX11 zgeGejGz19j|KRBn%Bv8*_u@4niW;=T|3u#-`CTY>$#RELxfQyQ;F*YU5iqoVEkrda zypPkD_;Ar5Uu*@~S2}>g9mmfpolov#=6r)Q;)C;2HZ$>v<@xtx{-MX^0kABo?Vy0# z^GDS)+vn9+QY)EniKN=N7;~xX`h1x~KTe+zF6xpO9Y@yO?Ki9bXiyizvzq?y&(jHk z)VghH0_g%(o7{m-i%~hfsb**XeN_Q;IJi$U2h4On+dM>y_gNDxR{yHvi7lagwwv#Q zgGR}(t(^L3)>a6lc6IYFpcQ%fB-b*Em2s-s`w2MxD0~j2;5w0@7y!gTwZozurRb*k zQHtoY64iAIN)~*`=1D=Gj3kbHPWueM{0U_PvERg$a}P%n`$g_9{UJ4^%Ak{RFFD;xPRnE6 zImWR5_qEu0n&oaEAOv7Yat#Z4k?svGqhKy&O^AiMOAfwnrFYq~br2J2GDNMpx> z5H#sk@$w`xBU9{Mh~zdS($rp^h|;>cBoHa--mh}$y9=mA`)_fv*`~`S>|1lX2Y5an z2ZV`@)gRtCM_FEC(145TQFgws8KV-?`1|ZB;0Je=#%)y#szICx`zrkKEAe1EQO>3U z2PTlDF6f{_5}igxZc{@h7>_`%Qr+MwUSh#X49%d`!;sx(r2kZ8_VNZ$(n4h^1s?pD zR|K%6+HnK{A4?DpEuurVe8izLOW1<4>48ZG-sz)*0{)|F$$g`0Vk-6e<&kY*n*ZJr zb(I|8*wXPeg65O!s6~NtN-NOta4c)(qHAoJ&;R>)kI3dO;0zv`8&?Ac zANUoMN@-e37`xo%^%)*fVba`E0Ps<&a9Qg=fcL>~fs~dT-eF0DGv9(?D=;jB(*R6j z@LkJfJC0^W+$m#ieKo*>s5LykT8JQA!+#Iv0Z{!P=$4KB>7dDTqkV^9=~=8A)!YHo z8DRtiXh6T%-TlFM6O0sZTMnAz`{?VX zUZTf#KU>Kx)7qr)MLN^n<^=zz`FuJyISFOg-&XfCTYU8+*Z<@W{jhp)FdM|YO=GZtXmud{02j~bv@K-ja4A=sju3|1{Mkg^5LG2c%JaBgzVMps6X|yj z!kYx|UsI3M)AY*S)A)-Qi7mn7jM`m|m;KN9-s!Vxx&D5y7bnmy(gqqyDLp-k>AEOQ zB0L$HPzHf*W|GCA?L!$tYHv{INiTif&tIJ`zu{}1%9`fpmm;*fX1lX?&wZ_)ZeC1D zWwCJPrpHZGAJ6K}hx5Z)NeSOaDifRpJR`A76}q3;_8s70&$!2=sNrw!hbeh|s&021 zRNs%fSa6i_av-y$e28^3mP|qO!Zy#6-qh|aw|-{zc2*x76i#HE0yI`iIG2IXY#eBG%Ri!G}V=wW_wvMY?F3ocjB z_}_!cF7c(n52UYkPsz&iG_xBhg7&4}xDd2U0AIB#raQ0B{{|@LFU^{l3Z9Se~zidp)bSKn&M2NaTyTNE7GJ!($XL!$c-mtdVMw(+& z14PgY62!e)YN85*nwq^ z6sa#F`-dOXx4Jw2MyJXI7o&gX-aA|xy8?A~r;Fj-bo9_JqD_8>KR|tSLlr165<87` zp`e3AIwGY@$Gx6Q8ORiC{vXY|^78U<5{||}E^vJT^!2mx&A!cd@SY%{u-KHG27{6WG% zWp#B_w+BCVKpJE!_hw1uBS!9FDtP~VrU-7QbvwMIS>3Yt)Yv^>5(5DwvT%LFU+^6w z|9)b0Y#m-{GNNADx`=6G^H!Sn?N1RxK``S$eDvMo5!nGO zgI~`t!yQWsi|3@|22qwxGMby`Hbjpv{;D3OjRX#^E`E~2Tlr!dwe;Zu-twRQ!R%a5 zf&iPP+hVExPHpv;aed2#GBF-?{@}pM6qA11gyC<{W5fk<(DZTr-~)hFSg7uf2C?-H zc~<2>gg(ciW5uxm|N00Q$)q<_`(xetC9q%frt$21-?;6gWMyQ$6;jZ) z+?V7GEpi}D#d=Ry^LR`l;WB|cS)V#WR)sY#KYJhmsMF%Peh>$xxw0wd$ATYv+8B7x z6Z)ox-;O@Hj$HDPp4z*bjkp`klt0Nq9Fk3Qb>Dn!6R1?U+)2i`^G#icP&IYgby^MG zkL!c`AYCvWci=7RW$qW=Bb&CoYow&ucOn_pS(m`i4HR;0C1D)~N@>3=dq(@J%gEe# z)T~L)+%`RFq-mXeIj%)+2GWVZYfv`eHdTg}ySpJ$NLelv)4MEvM`6r{0VFxB385Lj zC}j}_aCpb)!~(tp2v7|Owq6bMdr^??+u%TcZ?u`pNelLYXy6m#Rz%cS#qExH0!Sfw_ z0>L7pa_9hFFoz%#AFfcaTLNt~!HN|NxDH-evYACqT>CxWDZ49KRST~>yXTj}M?X!K zQy)hlOQDOO7D6v%59s`nf|BPT?eyHPy@wQ%2wq?8Pc?b+LtF$zvg~HkH2!xV@M5Tk zCG5Z6O+eqf(*&KZY;FIK1!xg>8#4ebD5-PDkm3BygIVd}L=A@OpS;XVl%^|PLn|DfS>d6*!kUjj6a7C!+yqH{F)X8Iomm+;M8wE!NrOVrA zdQ=lM15)x9kNU(?qpwPyYHRo8Da7e~ea*5tS%HzI?<;4(yw_Ac7Va3G!8ZClW}5n; z5f2jG*G`(ep~AjF_w(#Rg9b^wh6h}*((%k`yB(4M0u zFfhm-W}tk>Jic7tPV+vRLPNdr_HxrO3Y0+IkdvcPI!+&_penc=Ux~MB+ks5@<7n98 zp3`lN-!1FcQD#gm)9=%$mb2T{u9$vbF&EZnZ8`o+rH1#^eZ97>mjpZQ&37!&_yO)B zh)hv{Y7y7Ppf72zx+&zc>9(gEF3O~nl2T1|Uq9+|?1R?FAM<=mFe0P*#+>2xjeB03 zyf~SxUJ%^bSZznrL-~rCM8fD2BUNzYlYfY`=|_Wx7>I9( za-(j}V^b#xE!-95gj*K{#f%%_;sK8Kd<582p`T)C&1HC)N1hf6OOwF%511=G&vRZI z67t&EhYepr*TQ?+2sRrKuK}t8#+%oSa-cSxTg(H(uwEaQsXx%FfJp*6#-=M;5SPF* z2Qx*VV-ZX(u=)6ChBH=&z*b!*8-d6-1FiZnJa6n0VDr>x;eqvTO8-WJq6{xJO>K)- zp&Vn}0stc$`>>}K00a9u5uGu@*2FC;KAx8XSHK>Mf!nVREa?;@@0A%FHw&aG4OeT1FLX6x?gFsg!1z#=vw{3)lIc-aR3!+L<@2j$k)zJY7jej+57n zO2wKI72{xYCJpTzDjJVfX_8gZuJDQfNe2m^BflAlLrfo$TmAbA1U(C}(%754x2ikM z=Nup=6H1~&3iIs?=`cZAnp;~*ovzgfA|;T!g7ydy8DQpAn&2M5Qn9s>SGj4h+cP<-p1WH#m9X8A>sR?!>F(ZQiu8fuH8 z$;yD2n*8^zm}?kWfMg_dD5hVqKF72D!VxmS;%B{Y!1%U-(oC{szZ;-Rjh~?kgf=f+q!(^e z1n0y*SUx8SIQ2cHq9LwiWxgA&K3tId4)YqUp%Sp*A|rk3=HEUM5fOnuMmqS?kVU>o z&nR@Y_ag(h0EMlGM1Y?Ezp(o(MfqGjfb%q&$8rxAs+ZIDbrCM|u*{#((B(N1i*8ZY zDo-u7znNb)yXzx;;l}w)4|OfIapq((6~*0ifkNM-O-7n77Ug`;FNAYFlSJe6Hng6aa9S;P>%3X0`N0D6a@_hQ5r^s&KtD` zR_~1>gD;S;Ia+P+8P}8f6NyiEy>H>o$~`ap$f^B1t%p|6CrSz4KED>Erqqc=* zUF#dBqJKi7n~y))nFbi@mrLZvQr@EWEdKVxS~1a))%i^0LGQI#3&LbJQSsR~l-SJ< znA6_m6coE2O=BmkkX#%oxI&67ydVtgx;n+XmH?q?;QDA`wbd{kAZwxjO=!rR0ksx# z#Eb0ue2BYb1jmp79JN!9Etdj7zF67WU93JCM=^pdj4%2e8JE*>CmX`A504%>cN>fg zZq`zWH`sU=>XQstPjd0)iq$_Mar_K# zEIeOoYZVoP)EZ53?RvN1|tIlnteX;jLUJ^9_y!u(Bi3X1vzVPM2 zyw!ZIZT$8n6X+9pbphymcmWA_a1=-2p!>1Auk3)swwr%EcOun&GyV8yqYD$5GW;$6 z_E1zbe!FqL(^Jfp`7N-v6#V$`5&-}P7xABB9P-+O)`qp&y(&GQcssaSy9!nvif_%s%Y>~nV64}uZcnxk|4Q65_AMje& zDS*x+J>{)Wv{&;tFnK_>jGH#6{X>MGFlE?HMD16_hbrIUyp3nQMfU*f1n~E1V@obl zxV(V(s~D3Gzy}baq|2&QYIa)<{Xd${JCN%B{r@VJLN=AXg@}aA$S9GK?3JCp_h{K6 zWMvg0Ssk)hR>(?{y^g&d>)>$C?>hJA`}^bmqdO<(ocH_vdR?#Q^?W|AfzdTcv4i3; zGBgl1X(ykb5K}z-d}W0U)^JGrQa<_&*8%=Dz%W3-`s2_SFymVar)mz%299aJiW7&; z*G*HSfLxIxTBQrZdn&oYhTfjfsy+ag2_j+?iWG?|5gHU%r?u6CSyVZArC+3}DFram zLCy-6;8m$BP$9o!(JCNIh!1F1Ejh(or;OeDr2SE1mYMODhc*QZi7 zGtLvP$rc62c_F}@x>&ReR5qzay zMR4mXNtSP*UBD?czKRSgB4M8w3OeyQK0Y3)ap#7SdKTD|BX~_o-KGSv!b%$su5WB? zIE;OdV2_n@E4p8*iFfIkLiWLBP)!VlU)?7?hw^6$gStV}dsORXN0uMu3*X&Ep*cw#vl=&n z*)1FeLJyg0lX>RH+CJNpUk|ZIjFIEVU&F+DCYcp8NLkYrW-Af`%nRNRV5yBm4(TL` za9sGp72YvyQ$~Sn!m1apC8x9X(l~68A0D-y4|1vez<$8|aIdMhNR@@(ZS;9g_xAZS zOumTl$Hco=m>nHY>$r*2eNy73$CL~%pDYYfCTElYz2UBh>G{OVQrku^k2<2gX!KBW zAs#yOGtx9qpS5r+P{m6&pNc&nc%C~ab7&NoI!IIBSSIrI7Y=I^wRe2psro^5uM`{G zNUbLyG-*$c`h%5nwKV5~f5IU27=vPK&|tuI(-i%HL*CzCa;pK&1ataIE-7Kp1DZKM zD@Y3j&Sa8)0RZRhz=TZ*uiJYI(;W%-r z5PF1N>E`t0fEs@sawn=^j(WDs#$+-3C_k2$go-fE#?&vPTwU@~qDohb;v?~c@cRI> zw<`SS`s-2$6#dE!QtlTZYa>VQ<2yT^F5!LUN}cE5Bb~;km!}7rebHj4Ddl|u*;J!U zJcU7QNRvOw_UrEN(c3S8-RyLS>TTO^GH%_{x`S0aReXggkgPpWJY@S0YFZM%dEvWI zZAIn^qbv=|-fUe(42dm&=aJnn{=W708i7G!| z3|Xyo8TT1IRI_CKlf=>DrLQ_WHMN~>ICm7JfhkYvP^MXv~iKa~?9o#S`CR#iL9PnPa896gL8ab^ncNb77QpAP zni@W2xA5`wV3rI$sSfCCYtdQPv6Xdt%bml#)w}0D@jOloQ5A?KxyGA&w%Fk5=}TES zov4JwcQ<--Iy$nSmOm+=9?Vmn|I$ik0oiWCSDGHzNr4-aBd1(oB$uCHM10{K>vy|5 z5FM`bt}0(L@&EmI#=__k`^Bv$G3YYg%h3WL9^64Fwf*YJKc}yFvbDa#2Y{H;HKI2V z5f~}{v+poEr1*MR+Gsx$KH#T;FZdp$8B3>{Y%OhCZ{(1jRsZt@Tf1%4;u`D*OI&IF z;5GyI(+duXhk50Ya}r_YtV&oRIg)qP4mmRM--;SCZ~ZtN_rPie9R{c>U?i|){(^}^ zzt<{&hVm*Ot5Oc~3B|bDnUf-txkLt7AX+(!ysiD|*U#hty$b}4_QcG9aF9+~YcTWvPTVN&NNjyulg#&_OXdh2!H$H_Hf#qsi$ z6d5))AGc=_Io7;BCqr!cS9F)BQA=m3KFl*9aN#~?HdXgTxwXFjn8ifX(##XPEI99+ zptvPueE9>8izzW`PoeQEEPHzJfWcc@zwKj!>(nLQ5A|+ICCP1zQ0?CC6F*n1)BOy> zBh5y0xLDv>z?f@m*iWC*wctErq!099QVDOS|K?pV9Dcyp^BvGcfNXkRT^)u#kx-p0 z!N_kVm%(rOQ%~ycFw|T}tmgw$KADO7Xx#CT;}>mSfRWXIeccAKxOw@9=XPI~3YJVh z6<0(^#L|Caa~WyAeTIkjQIQGm4Aswy?fKh+!kC%VYU7+GU$8>Oac9o{rHWO|r$2Iv z`iTTfd}jIMyG$=IOunxk^-0g&p5ArlH~6KWpAKi!nNd1Aw?FAQaJ+#0E}!Gvhtc{# zI)O0K@`q(xxV3UY_qIOpz`qlS(U`5Vfg z-j_QKS_{Z*@pGv=%*~`?{`~p#3s|)31x9&={tCUpr=G)z@+V)BmHKM%OJSX;@pvVT zo=A|+dkl1A2%@CcY-Of<{=5P1yl;l8^E>B_9@Bra#oIHD1Z+UUHkC!Mm-md=^V;i_ zvL&h@th#AqBsaymzDL^(MWUg#X5`shfybREs7q(inFK}W@(q`{4Pk~x_;rY{q*WZ_ zGK>RNo$R^N_};156rd$Wrv_TTU%Cm2CofM(UoGY5=H}L_+vw5(;Xnf#x_7RDGen#B zELTn@4WG`pXQvVbP;KcHhFrQS19P}0Cyhl8ulqperfZ=LDq_%aQ&m4OPrFE0o$l!T zfeEmYKR&xg7fwX})r?LK z#&&(o$sKqE9-;#6Z%&gES7gjC^vk08M;)Mt`T(KFlJMC5$STvVr63^SA$1`HxU|^_>#H-Jr?D`7Ro{t9&TBt&-FO{fpFYm&(a&7NLY;* zvwW{N)h}?3)g6PD59|1cKqio+vWZj@<26JcL+VV`X4}^onsFe&&3#yCbA!MM|_B%#4U+yCpzR+tZ$7~`|>Qpd|V zM((|LCP1686){!s-n@PXJ9vcS8dz}SAMU*`jmX?1E_^71?fWuxY;o}OWW%DfVV4y( zeUbq&qtCUQM@yL&uiEeLEd}4_fB!DcBfxvu{>7eD0Ca9%s)nN0?+f*0JVXvub=B8; zaR?M&$MpBJ?jo;?+~$t2#?ZX6q%9rBR6Kq$<CMa>B* z+1tOIWHTv`HmL9gd9SS3GtAs{RHvoO1V7DJ^@SBoD4~}ZF@yh1XG{JxuTX)0ADD9v zXKSuo{kOJ8w}Mdhy_Mj=hBgzbh3|jCC;9pH-bF>iGgH1xjAP^DH6Uba7Ai|#2hB%n`^UKv z2T>CkUXHl3@Q6E61qBi4+kiy+{(MjzKstDr4kP|L;L+M@YHAa%9gOQRU#;l=an&|x zK#b+1Jy28hKvP@<%$^(Ki{bb2tcDbsKnbU+v?uD)iKNU_$ddwxkhDPZ5-6fYJ zui1iWUI#}upN3+By!6TfNR=7&8ED#?JV_iK|NdDMWcnYG zTKA2t-QKw9cYHyw#_QaiVVZC7f}|I&}^XhMwoN=jsa-4db|;Z_4kVF*n<`~6%fZ4WdL z`GKp3GGm?nPBrHV;9G?PDUNS`=;FP?_BTcX_qoo2=41xNvqM#2Igj2;N+yAUaHy4VEF3Jt3O=K!1oK7IeX# zk_iVPsbR-xUATr|y-afD&%o^$JS`=CG9T6&AZ-A&Nx*y#E!h}ZFy?1@(t2Z$y?9^! z^_mHjJ$W`1$)BjgmfUr9FAQPWD{?fkckva_E;F%fa|9rlK?(=30hTP}N_=$Uj#Eaj z(?4ShlOT-W|DjaHgI&9K^k=c0`^n|ZWrsRZ>z3YAGu5JlXm->@m_TZQVm^64& zKderkEDkpyLF{q>iz9C`IxmACmH zdz7y+3()JiEM@BEwK8xy!8roR9rzD{Pp6s#spejBJQPf@BOX1oi7+pamrDF;`S+1I zONR69K!ucp%$W;1)c>eTXanF?j3(E`8Rz z5hHVK3-KaXaOvEGYXsu61)~l)JF)cGRMI-tf*(H3aXhOVJgO4NCjV3wT)4#xKRc?* ztbX`1Q9ITx%mll+9r(ou96Y^-W z;A{2^-6_67Ey4T_!CU^J@vx)B(Tn!I?Z`3m8nsI4wT96Lp@L={O}A9jOk3UbX&2CPg#y~v8yC>wAM8;;5r^T_<|7C5JZ-MIz2y4W&{CKqMLUxpZJo%g8 z#<}D+S&9F2AerW(;cTXZCkfqFw%WXr3zxMHjN6#s{?_2$KZxNR{4VMPUU+~V#;e$% z#2u?ML@&Ii04>I?Cf!Bku|yg;>mCG7R9uIhKsqpJ;tC%Kv0#V+@YrOAF<6dsfwN%L$tu)nH9P!G=$ik%>M;}u zE1~UIG#vZsl3OF?WX%<+)pKFR{qB9RJ=QJgiVH-?sVIO9K-4j4gfi~>DaaHB!)I?@ z^#D?RJn{XSma^O}o#HEGGCs7T$@pNaN!9BbclDyLCs!nVXo0o8*nE+BCY8=)wlgQ@ zYm6s9C4`m+n2F+g7Z7BUJbPCNUG;h)(R>g7H(;b!7)fCB`Eh8?3@NBC?a)eVW|Mr< zjOL!vCF9P%Nqk+{x?X{n8M2=Q>2oPdGArv8N!Pg$-)RnJVm#=kFp-u!Bj9q~nB>Uh zX!~G)lXe4lZEY>kI$x@5UYr7vH8;_-Bn1lz3~^~2A90KV#A zR9EWC^q53Za?gd%wxvA z!H=GRX}tY&956G2hZfFGP|m-@#cbVBDEzIJfsh+Mw|XbSU_bgg?mEo=n}f#M zk2%!#Vw$rXbZG~sruzDWvF?z&y~jd6V~S;xJ`@Ry~uOtDMu$;lEYLrqpH=YKvSBh0w4@sg^m zc;fNmRv|@He5d|wnOVashov27*|5yL@<#elD1KrlsJcD#$(t}%Q^%7QU|Dg(VX`P% z9wGtf)>gkKm1~DG!*r!;n_ojV1-feUkG1AKtKH_A_o7Noo?2S#!}~;Ml{0^Iq$O={ zB8O$lu_4}&z!Hv;w*5|uL!!HhvPE<~=6m8iSJFGx(2F%oevU(#xQq%b=o)04J1cvb zbiMfcl*tq8)=FXBUfrBwm0iiBvr;I7xJ8M(OAL@l!@a4}O}>#>@H4(AyuP*`-%@Cv zh})Ppc&b$RGS=(q;X#4jroP6^s~TRv3+F$q&Es3=duX)#X3bKJZ}!4k5B|%7U41Dl z7z<{dTnKwYr0bmc%I5?hWCVBhN?sZ;`N#Ca_A7F0YEs9Y>Y~BNB+Pz(@n^Y3jhtL4 zteKyn->cnvfBuf2O87_N{L+<^l1CG>v~kRiz)0MZcZ zkvw&~;N%&oB`70RRHk5XyFPi!pMPLl18yQSm?LV8l)!2D=rvt=*Nb7Re8A9xRlLUN z84tD9pK2~bxsPyu|8LwWw8o%(1TVXXXjo|<-s*RKb)xljHSy$Rk^U0l=ZmLVlWxQb zd8(XoeeYCrAClo77*#9=P6hZ=fb1R=h^-8a;insP zgx@^7J0QmkGn$wKx6kMrAHAMk(?JBaDhX8YU0%%cXxRyJJLrcA8Iqj-o*-1pU1XM*qDA&rO)ntAn9UXgR%*@Pe`{<8Y zYz010kaw8)9-+)spM29$R5x~DWJCdd?}dL5W^GlxS8_pn$_K-UJzTa}A3?l^_A}21 zmOBnZWZSE2&g_x#uNB#3c2eDr&?RjSe_$C0wl++~1aZC06C+>}fI5G#tdm439!@C0 z$YfD@XrH6+=e!tdeC^kQF!*8o_eY-OtG1g9e*-B`r$n#sXtzK6mQ>z8sTyeiK#^D* zj#0H`bf1VIA>=R&N_YrumXClxUn{{;#LA*{1?souLCyhMXpV_8fHpjo?2O?M-MMfX z971r9l7wUfo2dXTu2JwF#2;uTWb?%LS>k=h?^Ph}MY`_fH-nmlH8P% zYyxcvSyfmvWnk+n6xgDt)T^s$InNFBOYZ|>g#a zd0P63vGnlkewrJ)kkZn=y8p*D$+76&uLzI5O943kJHdW6ohNgDuX+e46$vJgmd=PZ z9g*TC8Ypv3_cVg$s#m`j`asL18ub%T*B5*bZ5MEq!|Yyg-G@N=0dwB(06}b{=47WE zFh_tV!Gjmvf8EH}Hq^&*<3qgI?J!x50&ReyHtU7$R&+oU2P7c;^iB9jows35&bOc; z|E$pxU?>Ao0g=+f;NHfX<da#MN1>>WQwoNH_d_B&7uctt zzx??AL>BYC=0)ep(XKCy8^iPYnka8Rhg@y*Ow)`D$A7u*U?O834DJm+vSG z15^@G=&Z56x*N5iqPV_77M|;-k2er1a%40vyVbShms!+T`yif7zvGo`mE4jN%=~of zv_##px4V0JO8=6gCUrXu2x%*>t8I%Fbw2Hp!Ss!&_bd@-^F$({x?nvEucE{qtnVTL ztuMr&7S6o2On!ehd<2M1cGKs`q%?ZyE}Q%7HZ7AWwYk->2^d!*Z!e9EPO-vgzd|T) zzBThs+@k4J@!ns%3Uhz&CcP5vHdRr<3-X!>4IXWDn)vgwYERu{9+FsN_|e0u9U3H( zv-|5DK-P7tOuw1EOi{H`2)@Nu>AfF88>L9t25n?@YC^e*(E+~A)pe?+p1UQr{lTZl zV)$QmH9Eg7;1kc7!$E%=&KD<8yUeCZt?0@_P2=sc7YQf`{8yLKOVcKg;PFCZ+JVxE z(2h63DDd7?ntO=QDhOwG-Xdp5SndbG7G^G-NO3WAD;mCMO>#nnEIH-5=5IcIHA}kv zG;WGI;owx{v74HM7i`Z`VbLYWT$#-1iO)hrt$yv-i0Z=$2%mWe!)hYzh; zf4o{)IIf!37*<_asJ>255_uFLH{py+Wn(ef+1Lul-&_T%Iz5X!NV1ap<6sZQ>(NR^ z2)BxAE1Tqzp+Pab$%6wYL_?h^14$_UMu}tz55#^<>0M}75WabZu*KYF>!Th8p-@6^ z6ijLhKgGTqncPa=9M;fw*jlBV>-s)2aGS+z|EOJ6$1m&hE0zNi!ZtlNPIx7JLzCtL zrsRAya_h^C)a-~iSBVtcCnrzM?bpO9$Z;oaPjFRawdw#iX;Tj>%Nu;IDUci$B0ozk z85AuTvoT&3C5svDSBULk--I0s3B4|Z(|S5ZP5$I=PT@BtQ6)JaW;F*sa+CQqOYdhRBrLaKPvn8D(G%;M* z!=#jI=TAIW`aS1$oTysGqy&Nq)$sUSuod|5`k$72`YZu(e6?dpynu_@+OW-Vm-xiLiK>v7&^i#u(3jds}TuD z2yW~0(N)o?Bz`*G;>kmtbRX2+ZS9YA!_Q&ke6!tZ_+<`;ZQ3*HB-=rkXKSo;*ta<_ zY+A8!?LIIzXG^7|yJ;-MJB_gS&ad`r4MWB+*ne$0FHz^;ei66#Ua#OS_6n)81(ID| ziaqboHKByA@TE-u5~6T8ptj=dj2cOt8jq||Z>Sz0GCP|B`U4Oi!6e{D#0PFp)@@|x zfIGAVea)*+HL+XsgEt8p`7aU>+-X50L@yu`Hp=E1iIKw7jg zwSzhEUDb3_$!;6fk{g_n*t7|?V0g=McS4)F*8Wk=l4&O>f!Y;+a3adgfh2fE0Z91( zM|hHvc2FV%L4nV*weXMR;HH z>?{u?I9I3)0YwASezUdE>^J?I1-hm-+xJE3@m(AnlY!cyeC3Q+E?SpgOOT7zx=}-7 z`&R{kc962WUIG6$L?MbQLwJ(A9gYqy2o5Ifou~XTH{Ws*D>J*R?q8>CJm7_?1e=XZ znO`Z7UPTE!LwSptd&XE*8SE+TR5OmmHyn?$V~M8%c+A`AVo|jy{MH zcwi9ELPSKTC`X}3c%D74Fgy92vx%bw?h zT|Cc(V5RzH%Aa_C?OPcfx?{X_+Bog)ay6}8fgj=9zzrgO&Or8nEzG3{W+2J&ge5Vj z$XHCyYlGbEAv=mluB*%kgGn4?@v3zu4BbM7zPrb$?szsJ@ic-`+39L{cPBh_u%4KzDsaIuUV|) zyONtM4y{@|q9-*a#S9Kt^ovz#9eyda} z$1+L#1-R6Jumh!N0A_sJdbG?6h6GQWIXJAJGU>qIPhF@QKP5w_$~MrA6^C|acoJ*n z{hf3g3HJZ}wO|6xR{4ZXsjl^E4e_Qzq*tc1=ObD(>_j8a9p1<$-|(M&PbMvj^O@dS zk3ZAOHw}5u`*GTxugFv?e0{X)w~uex_~W`=@lC$R?EafK|C*-w431D=ZV#?+*riXJ z05UO8-Fg`@F0gr!p|&a>@lx#(CiT_KuW@>Q%;!+Yr9u};(T2saO#6gB;znsPZiS5Co_B)vN~;yQmZ1peSsE|J$KS0Q4%(OjVHy!khg@3KG*qN_~Z z&UKVV#Bh7u|HGH}+Z%J9J(>g2(?+#5 z1v05)!rcyKU#oUrsX&2)>7|@BMzOn8_lh+Kw|2K%y7`Vq8vQ+odx+a9>GRJ*gfVHe zh7fpC4!3&lcjJBuDF>OC_uveQ`?8p^#L9t47PnCnX53*1zR`V+iD{~*V*k|I!{hrB z!S5X9STaFAog#9tI%6W%H~mXgfQwzm2*>zH3G`T(lgkr5jE8Ba6Xr`;!;3Ep%oN`9 z-4SAZ8!<#RXzo^-BGzSPpvOfL4t2JI`5s8zG|P6Le3~20>euBdzB0$(d?M{Y4vuQ-?#P2IZrYv{u}E7rXGny5eBg2crKz1Of8Udur*dUpS~ z4Ze6jk?YX}Z3mK@9G3dDR8&-xm7uc~g>&9E?`?bgAa~Hb>GWW2bKJEfLqQ^e3hGt@ zEqeEIQNs%Ly5>@uz+&yW+l0^|iAN|w7B{Vg81c(FV4gOf7w9JFs@ zB!un^N(V1_ZgK2{yaNBtS#mfbWRS~;k8hq>R7D#U(~?OaeAYwQz1EWy;&f_UND?zP z1?W^;{BuMQFArrw(a5uo3__MGGKVvLOCqM!FCWM+@B5tzFAX|PlaJ>ZaH?weY-%zI zSWU5LTav+@M;*?^66+@^N+r_N$UiK~x*VBU_|NIz@CVW-**71Wi!#tP+se;h&fQ!a zhg)JZ&rF#vP>|=5PKsWL2-1bwdt*tVm5DnM(GkOZ^R?-+{tb;R7y&=?+cw@&G)u z!56u>xZZSGQbHKu1OB`pGQNS0?eEzbm}`~ipgan@Lg1{?i1v-c_H8M+TX>XjL_aOh zcG3~I&#zyT(_g@+qd!L&uz2&8OGMSSdfOensk|!F_qo+n$2vGNuX6#?NVMrMv zXd>_n-LRd9b(d{=^xo;H@LifC3g&TpFcQcR0el5)cI{TlID?d%0;p)`0El5`EHzp` z3Xck!CJ%|{@2j=jb(m~w9HU*r<_YK&ZS_2_;f)&XD~%|4$FXTLb%e$4MOId$Ba*5Q ze~L*KX9PPMpX@A#hUbbl=?HQ0XP3b)!(+|lZ^8$>t1y!si{ZFiv=BND_dK1$;Mu(-wcY8Y11c*X=Nt3 zNr+h>fo=C4#s?xH1WNr$;{m`oUAu2q)lc8Y++*u58v%bsJ6=@act{B?m<)W33p-bm z)?~Br3vN_U@W%OXH*_T7B~fz^50CP744|1K!9jC>0zHG1m0|BL^I1l4w?FA`;{Q5i z;EkbDmlZ6jUcBInVcw!c%kCVe#qd!-Le?ak(7^hE8lmcUs4O*c4v#9W*O>@dYM8Y5 z<|3!mHT5n}NuLU6Fio;TjjuUVz0}YC_8uHT0G;kmYWN=KDe8TNn>^Ija57sqrTb2ZW%CGZ8^a?eTSr80WG^!`niz zdS3u*H5E#`vQFBRCt`S5Jwm7OCYQB*_Q1s6N6v@wU>4FUN=>1Fl}6;lP0fOTiu|b* zRJU2<_};fUh;tciRQfEY+SI=ExyOL&VsF=Nat+mM@T*86QFuenNRN8DmfsGw|Cr;j zgQz0U_Zs7)*ghHMbm+aLIqss-_@&W&;X`x_T(1{b85ybUxvbkdu1MIHe_W-1IofoJ z?%d%9Y8kDeW5tm5c$QzS1?k>y6c(-4L11oXb}!v;Ot)=$SGV9ygj3Z!4t-$dX~N@KM!z=&UrF0>*NX?%Zt_G3eOVY*LrrhflC52ESwE@0 zgY2lyFlGrRAHkX{pHVaM`-Uc{eXrD(^ea>x(#sP~WogLm#rnKD#Nhk{>t$RwonnE} zMvR3y7Y8i0H$L^s?99!jh2ljL^4ODw zQZ{k!&cnkFNlqp&aQ|&CIrCoKFzM$`cX^d^sa*jUQqw9J7C$R2ojlO~=L7Snpw#ZGi0=Kj zgbF})M*|Y5x*}Hsg+Z|n-Z#00e&@Hhj0z@@Son(#)@Jh2pBJb#)Mg6sE#D}u+86zK1PiLS1mG{DR_UlE(qY@j1)xAPd$x6nq@V{a~>?&2xl-F(g8x$@3YLC=T zhcos0D7j7kGnt;mpeEOGswgWf<18E|vtXF)ry;v`;I0d-?$_#7 z*d;X$+etVsU5Od|IJ()^NrAlsKY1yo>j$@w{B$mVy#fyJKXKP-cy)73n&>6 z2k&>cTYvx}`nb38Ic_r8PtVAEkkR-P>Sw_<*R{!DT0WiPz$fE|o5u-0s0A8xDB@`W zJ|3_nIZg3G4`JRe8Mv}_jb6-&8mumiRa=(`@R#}JT>Wkz6@GoU6}jFr>kYmKvU18c(nacqLGeLa znP&(&+f}tl&xA8j9ebWDcm8`{9m)h=a*A9x#R)%5M%sCL!PA8WC^M*_v?xj=2|t() zr~f)sdo@d}Y5YK)Op5JmgLb1P=f7X28i0hPIwdYOy zyn}*5!ZjhP)xTb`9DlMWo$!^{ElRo}KV#MN+2&l-R3Ya2MU&!u@`%qVSG~UHPS~e!(k}Cq$?Y|1Fx*;z=Hr z#INEoCc1VbNx+ylGB>hw{&KcfVblY*o7U{E6w&Rqj`)m9lwy?FCF#eK8zOOnNs1D> zIaf1nm)?I|Pxsue;e8Obl@U%x- zC*33b4gUJd-92U#$>EbtGGvnE=BAS*Ae!LSw1dsz;>b=M#iKGU|5wR*5#<$t8alyXH$u~LJw zv^0%!U#Wb2c;2A{k!?^)9M;N8fN)#*wW)`DI;BGJu79bjv&O&vHk_*P)YbOR&Ty9+ zBP7^-CsjRmE`V(^Wdt^skk(mpa5+Pl$!+V@%pbpOb~N48HTFz|D-{9H48Ekw8v$8( zdW&^`d0-CLN#EZR9BYJ2+rOeuJEW`|zT>DKagvL^j1ucVzv1(EUZC3j$#@dSPVxkw zikS3(bS1XsO($+gCJr-dEYIus>yJZ-7Ta#?n&M`5og_WS!}m^p@@=v>V%A|%Q|CC9 zJE$3(;KP?x1f6C4cvs7_3IjzJa`Gono|sRyWQ4S@l6N<{A;Ph0NkPQ|QTCfok$le1 z;foX9P4hov9{unZEQs18rLU2o@`KaT`XPAHkhuTY!?%0Q!g!uT5-uYF6V3$blxN)Gw}~B{=LkP1VMpZ zBI7Y$A~$sV>UToA;bpiEn-3DkQPV{$-A{M>b)&_pDk|AD=xBp1DCuW_zS=0WDz~^Y&#{K=V+DO}l1QAJ-ZxAhi7k@0tcWMpv)IsHNigbc<=!Wp=ZD z)+21D=A|m_dM_Z*e@@>#&Zh!MWt=$ts*vd|x=$LmQ9WHrdGLhe8)sF{{C;O&;N>X- zpdwGc9Z?V3NkGz1AO7^02{pUL13qH3ceK)OjB}532mRR>YtGejMWf>EtI?q;q#cTe z@d`X+m*%+!?^49%*UHVLYN%@4dkVoia^Vn;^|06aqlwQVqzy^prG*@E>=O3Iap=1-rTFm;=ah6?64Q1egv z>#ct8<%59lRuh0`i08)_p~#eek5^OO$%1rLgL1Bz3~%DJ-ha}M^9LigaN8Zo#Npv^ zTeh2#3z2Y8)vDpmvwmehzEQbVdHPeVkyYU2NHOC@MJpV(GpNa=%Z%Pe$2xm8CN%j( z`pl#9--w{C?tLiNhWe3uE&mk~#hrN=);!lL>-!#GFOshiIu%yb`mEL@zk$p4|#EMLWkFMn6@`;1_*5qr;~+E#M} zcLqPKEud+_DqQ^}n{~uj(7qzH8lz4YAv@ucP>q{iFrICmZj?o7>sY!eg683Y?tWUL zK}|Kt3hQkol!Qh4RVMh_61l7%sIp!?*w?a!CT^M*ybu=JrYy5Vff_0edOCs5$F%R3 z7E#XXX^AZAP_spR=SeI?miTYtt)nMlyZcH%PELq%x>DOS_s^1Bm+Kc9HJH4yxG~^R zp$J=189(v!&9TOr7UJ_PSEQEBP`?UGJ-apGEQV^pVVkW>-6t*YQFlM3h{(s+nX&xA z@^~9I5*ix8-H7y~xpIYD3^a*$`GiVvQ^55?)y&1xz3_n{3yO^zlyEpf*iF8#UP&@+ z7aezkX|TB?Ahxy-HPtAc?AIhODzJOH9zAWMU}tAXCK)bn%EpS2}!G6n7l!RZXklC<9FkFM61 zV9z^CG4P@McTu0Qc}v;ZY>~ zk&w+5zuVp85P(zX5$uj|UZGY=TSwon>06S8pGnp5xP9hF5}?yOyTAOky94!@Kbhke z#o*weIq1JFYx|LqI?*ETPU{n!Te3rD@9^b(#e-LpijtXBpduL2QKm9u<=#65$^XqXYFHUj z)r6)0{=6V2!Ipc6vH3^g6Sn=KLPfr^nscM>{Ziu;1j{7ZPcde9wN3$|K?l{Os;X+Z zK{2aS6Xu~%qpi1)xHu22Bw69ss_zeG8o zW^^c-uz`30ce6 zUxw_Ee5OZ{SDBcwu@8QMOCAX*C<>jzr|VFbHfY6fQ2<>a9WbVFF~cu1`)AXGJiGhf zjso8gJe^!(A7!Phjoz>cA&~nqwlM1Kztw&-cwP`0&sR#5_-k?Dx?M$W_0AZ5`>tT| zNYTU(dO%>f?q;Ap(bD_tkIY4{(t1x|?s@lW6S0CjThD8!tHo9%0tWI%44VcUGX!im z<2M}2F3TOYrBm+~vs?9ZLcDLtU;TIg5U_R&$dQW6)$=Fm*KcRx)fRA7Xu$J!Zg;~5>lN% zW>|o?%-4uzbLl4Y~1&5Z@ew&*NbmG|_Qua4(vq6o0Gfv@y zU9LPYE!1?`+V_+c^AeMrd94V2N<}dV;%YH=6i^LZ8^Y+ zTOcgjFXy4CY1@`7!jiy6!O_Mf3hL&wIYZNdf-FzWs)05^s9VTg?CL zcF4;1gnAbG2DbA_Vb;q(YZdgCEYUmF$2BA|-IGW($Bx_@&@S0Eu^S>eoB&@P0ZbfW zcZDW5?UlQS_l%gBZLl+#iS4kp+NH5)9gIioHZrXvCyJ0)V)8#JmBOtL78%xsS_)tK z0U|!$(bZn;jiu1Hn{$!4#>{ME>2tMiNL4t&;`b6)`~v{rTvAN1wMDqhp8ZpEb`;?F z2EB0O9GR!;7`AMDM?9109MO>7_j^6+!T_*rjz{8-v|6cBuY<@~s@GzJ+2S6!fY+Dx z5~pa?wfx>TI5UH9Y!cH>q0G=MgV_ulmc7TCJPW;{g$f!zy^2-O54~Ij^xXM1=vZ$M z@G!STB!E7X_sn=|p_=8pvbmFxIGUy%Dp? z0G*F?&wu(`=x_gND)`PCSE?%b&BVq9lI`XN>xpv*Uo9L4YSuWid_4Zm7p_Bp<3qd8 za>OxoC=}8FGH(nkCG`z z{2Bz9MQVYXJ-4qv`(2OoY4A5BKJG77dGb)kcRp^t8tr}9cf)YEgL~}lA&n}brPuW1 zcE;FBaV(X=TR%{yY4aI!7o3}Z(*1f>YOJ@UI}PIi9Q`g-?;>cKN0k(Lxv%cr@Lhcx z5iJ){%3RoPx%|CQY$mwm>zpgE{=q}i$A&hTHtqB$jee~5rj3vTCUx+WsTy_InrsUU z*KZ7f0ePO((4I4!s`rIKjEzUV&SijS?vup@$n2)}*E?H8Ddn6X>6##|;xq$*;YFbT zSh~27zYq*udG!CI!2Cwl+ky0RVW46JCf>ht_YJJ?sD!MOH}<6sNq8z7zCjsqJeU;( zhYzmReF^Aamj0J)0xmI}OTg|w;e^KPJfvVMxhQ)wO9u7;*)kX;;88`)>H{sMyKv9I z;++}Vfczy&Y!BA>5I5rKfN(h+rU@-yj zgR;_a$hBJgS4(Iw1^Os2a-EY zbf@We>~(f0w}O%r9!o@Q1>-Oy8wY6f57_6xO~S8aSLx~f$1A1J+=E(2<{f^R7K3|_ z5WWS>`6y`{=EkPtzoIRD9CUHhXV ztm`PMHUsR|=3OMcI(1YtP8BI)|BtEn4#)C;|HrLlWR#H+8bYFwB#KBx5|UjBAsLlf zSrL*IvI$8@Hc7IwLr4grLiP&T{2tfq{rMip@1NJ}=t#HwzV7QfpXd2lr&#N086AP0 z7ft_*{FC}L>|5sIl(0$^ZWs;FHr1;&Y9%tc3VYe?rN*22TBJYG^b;}2#i{jO+bN%8Bi2f{uyj(I!7udN&bjXUfFFsqb_uX;Fm-#!wJG6t$-48bL?kz$yMdkF|FvTGu`XiVOov!`wM1cOYf=b zE{_((8yz}R*pEf4Z6gJj1$EMobB-433l>V@Q!FZyV`E}z+y0pIb|cm&=G*OBrYSwP z+|dku*=V?QaS6^H(VhYPOWm)I>Bj)ng_sVvNa@FiU=&LHTUp~Up4Ww;so*_|7K!ihT;|mYFQJ*PEp?*!h2Ses-JK=&JYcqM#J=(PL zR6U`OucZ#h*|~qZ8M!_qy*df1@krT%82zbb#??|+Tq*r#Rcf9+Q#Wwn7#K*sU zuEVj*9qIKT`H_C+{_6tAMxBz|EQpPUrAI6mDIcBpTsc6F9_MohN4H@J8kUd0+%UJT z_shh+`(iE|-EcYM#4}#M+$6V8bG&&%VNdv%f9EmjF0LZ6yb!?|Pp)KWC$(98=*7Vm z${W_e!Zy3uW_f}6Da$%H|FI-C?fjj|Z61yNE2>$sG{*T+v+ws{9`;$cF>Vlg5Fepk zaxTBSws+)Jo8`^`UXgK!sAkI5JrcOR1I=i$U#8rtJ@RgH+Y=b^`q!q-%TpefCU^aH zTrVZ5uqnbRfj-t?X#KR*TiskwqK*)z)y$&)n07VkKraRvw*2modL{fXoIQ!SVZcZs zkUqx1NfmAVuKN#9T;YErmIaSvEvLee37zk<@;A$Q63a2pWOM*Jo&eUg6=D)}(g%8| ze;3I}NZPC}m#rx4hiNQ4vmRIBT@7A3*O0T({~(=` zsOp;+zZU&!rAO%X)~bL?#-mRi3`zHTL&EMi@lOVSeWi3n>(IuHlW$0O3&!rOdL+Gn zLa z?Vd=?&d27}oyydL$1BsfQqCoL0CD=ea>+u1_HQFuncdy^%fX#_&RX4lnf+eEzobkHwFA(}$|7q+8w);|9WLZofwamH?|ii4|lH zE?a8@#BOQVV6!)MoIf0Ut>f&!Diiq*d+ycmBOzw>_2oKQGn^A~*qTQaw3%*a`6Na{ zefl1{j}>fMZzh)Ou?p*>e}{b_3e1ByK}r$8y>qE8{qmx-ZB$g$PiP4%hVR`S3_S4{ z&-}gd^i(44pV%a`?zXjdT0uMeTJvJp^%lbU0blYhagL9hs}n?W-LGH2ie%-2g+C{3 zt(ynQZaTZ+Q2864?Q%08K5;(`?~z1I+MX@%qjx?j^XHp(#ou3;OYJrfKKSEam#gH| zKefF)=U;z)t1m4l6f!ipT_~kx*!i-1u0kkqthyu~gQse#!`Urm)ix#56{~)SY88E= zdL%0mi&{yfo=WsiRj?1NvSvU$zmm#hZf54wc%&pC*mcrmrs2nW#tsICd8{9co+=vH zwC)ugcz2E%gx9y{;irI#Qd5u-%WfdNnx10l@vP7LsQ3C6R{Vv&67E+Lj7`-{&o#1o zrc#JCi@YyB8@En~j5?eTeLOG-WPI=AM19+5M-Zjo_0$%W=xB%iUy_;WV&YVW>Gj@+376 ztuvNR<*n?`bmSxnSE=#nx|pP;nBBE2D7Ed@apa22=knU5mX=fqD2uHavuD&Ol=Rh_ z9cOw{4Rbnb5~_s0HFcQWCBazp`hmNAv+pOVNEa3Tp`P+MD|?sC%%3hik4bBLjzP2Y z>K;{9U(^o^aZ8*zUT!BYS5nD8v62hHbHtWnuSslkC!!)e|j03BUFBfh}7`Bk@t?5gMF?pK_V` z!XA+(#LA?`pWh$YZgz0bg(U5CRP`MU&VJSHcn5+VQY)J5MeRPlx0I69pEa;RSpvz@ z9tm2(l+fILJbv~;mYG2friq;v_Dy4t=6<<*|FN!n(BI$cmI7RVnui zFk_CMExwAk_UhjKuidAl;wj_RyxJ1^cU^LtCbv$XU?#J_7OQeGqx+R%`d5p0{SBFu z);s0xXHs!~d&@r(YBU)7!Q+VVO{69AuT2&4DDAmRfqNQy!6(G_YMc)rIM=haGYF7x z62qIzV#f8Ga83@wXfTY}uIX)C|05x`B6Y6a_dcBcF%)=Sq`g%0aOCwbP;&gB-CP=- z<9UkoF61(9E(sJ^3h{G$*mP|xID3nx-0hj8WZ)cA$$k!DEj3!rGWjtE_pt(&IWss#j~gEUW|e zcNu6+yV0lAc+AHQo?$h0cVYTmYY;r|^|+>#*8FNgT!WuZHz+y63CjLAV>$c4zu}mH zzKR5%`*cO_25xJONA13jF#LWYZEL(WMJ~X6r}^r|pdR=6pRyGnZbjes`rPcf#UR_I znCuh3JL5a!vspf^tj+oeCZ)~d~A=Gl>PKna#Y(rvklM^k`#8Ord+@s5mcoFGI{vFrav6s&SlStP$)>7 z!b|`4`1|=$cV=DfrQIyQL`;tVO=4I+w&!m3PU8La6`u01`E+jg)oNtzL`T!zX=+oN zE6h%){MaVP{qV1zj9Y~!F+3nYWpj1%LX*!+wtxr!dR)7Tc5Et{P;=j3za_I8!li6} z4=Elh^!@I(G~T=1U4K){JZwr``Lw&N;^)u3U7MSfZuh2(f6T58++Ade%)@}51Kw;a z>ZhzngDE6j<*sKR(323@DarhltvhE)-xr8I9ejL!iD3;NYvSS8rJ*5SG(!)&I-87h zMWm(u8;2jn9vFDBIMJ2gp=~htyD|Is-0IxJH4F{k#_512EGjY~)21!4%2Wi^QdDdd zb#31$MLeAp7wUJQ{yFjJ%3DRoVxyw5JI?T*=Z<5dLnT2<&9NlOc_m8dFj_6Nz6+cE ztr3Gv-C5AT#q}a};UOCX{xnkl~pVu^G^=eQuu|2eDNQ8ai8^^_%OxCFg zg$!X)sUeel5}*DmaSt6%bp^;JcPhka!}Qr)h}+YY&xRuTfpxv{3jsu)}5z%z~jPkELERUTUP42hQM3!(@Y?LbH z>ZHUDO0HIFE4A&(IBcpz^=f_A1;U?x*^_ZaJk}cHk$;_UG>zij@4^KJ*7#US7M|VF z>@?lH(Jp=mFUWlQ*K2i=_XTjxpN>7iLm*$0&t+y!exzFFp^{?Uwyf^dgz3RAnh~)r z-tuw@Rf>W-M)fmT@1${CT&i;LBv}ZBRHlz{@z)04-DE}WYRP!%2B){b8?TUHJh@gB zW-8@d)Y4!j*|}5s{T_fbVs^vZzIKxZnlYtldaI?r*1iq!4wnH~Q(A`#Orfr7()qvG zOuXFi(g_V=SL=3oUF%|T!O`)ikkVC+6EAKg4&0~;NSaEx zAm#)jY7_6JBfWDREdr%0S8uNUyEA3Fu>WDbf;2scmkegLUw5JkZ+C`%c{R5}NpQh! zoC_A>$rah|BbR;PxCVOj_0^64Wzp%el+F@Y2Zs3Rsn za;><$*nQ2#FL?69;it`Ap__-E*6hc+YsV5`!_~j;h=__-R#a5<^86UQcs{fm2*rCm zcFTAg8%IY409JFLI{l9I6irfpf>6t7|05rs%+1~X0$L|Be%|8mo|Q&bigNxVRyR6Nll8eT5ZR-7Rhr z$Q-8iO0p|G7^cka{FAf{SQC+XN5tIOhPXYn(j(wHHOXHTxYr{M?@{9xCi0E&rz7uw zlHr(J&7vaGgz)`3Omq@9RzG5I&HFwZ2-NtAse@j2{SNvJ;-tcTti*la)cU?^~d~MtIXxa<2oP9ELnVAe&6}BITy_6)+ekAJG6;0ZI8n@4cp##{3L|5VtWMa$gm(ZO->3C zZyjlI7qFZ8^w>jVVUC+h8Swy4$Q$z_Kt#>s=VTO;Jn~T}|2v1ED>f^2yr! z8a)fc@|XLvvJRig?5wa9MLRj$bZD8+yy{!5Fp~b=ikRZnE~IOws0)P&KhM=uMT>sl zMA@jyo+?ZuxZ7g%%efSN6TUrlX%AW|3WE zdxlk5yslkJ*Q^|IOzKE$7z-F6DQu@9YUx5DTjsnU1ro{JytA^##t^4y$UM z%f~rGZdPiEc&ZR(f<}>ODh9_xp(h|B5!;X`A4M*`?Wd z+^p>AixbG-KNxwm&7x5Elr39n)}W<`D3|fv`VAbh-7eJoD9*Yu;FR7+Vd+peR$(dO zpujt3l`5nluAHt}HR7mgE6%p?5G!Pz`;OpPP14jw9PYC4w%j#EU=qqK+CXbsQegsUUK*S?A-zMnph+tu8#%A zzte?~smfK<_Y`?epTiG#-|*$y9A4u(lcb)QogEBT1hIEEMfvDTeSCcSH-B$dJI~F*g7m+=TGD6c=om5oMH=$NV&9FWfdt|)Zfk2>GXMEMzC)JS z0F1&G*gN$Hp)SKZ<*O{RpctTeb|`vnOPnC{%cyp>2;s?zCfWP&pD-Ik5A_CjtaNlC z-{7|w&RkVxy&AdVa|G`|aEJwjMHk;c@$LnA@UE;_TGXY?F=8{OT4b zx)&S01#1RQJzg9NZlgC(D?jCDm}Yd+=8#}Emq(LRR+V#WP(j#zp}Hg6!b9l}Yv($T zOkt4}tOcrk?Ip5uGsnOt5ho#{DdP}Dfn)^hg^jxT@D{qSBjOMeG_X?a-dJUq_}EJA ziN&nohzRCEYU#%E>d0KQmji;1KOf%~EOS)SQWl4{q1)vU`6HjM^;Q#cwZ#Cc&_IH5 z5;8lcov{sFF(DFBlN#g22yzro(XOrH8abXTgLaFLtG8@UE&Z@_0mx-O)l;~~zEzF@ zUE&>ks0;O*^KLd0aC0}JjZPwl9tOBJ=_4t~C1@C#$LkQHudLF4`d`Pyr6+U9felKm zO_$gfS&Kyvts>+L4Ehaxg@5q*npq7ljm}LIS#EcTh4<3ZjmrG-)%4Pn6*%>kq~iY+ zw5ttHT;Gzqmc6v4MPGC+`*L;{yaBjq+HZ9$k*ZXKt(MBkZ53!bPMvCk)>RoF=~Y&1 zVxP@C>hEmIQCB}mAiW5|X5WKuG!4AU5OW_HB<=N zrqjKd(@wM4q4INS?Ih@??%VMH#Zl+Ba9@y+P9S-0PNv#Sp_}i>byz3M$ZSyRn%mex z+_T?rk8K*djUBBw%O_%=h`nJzpv$&4X3A4RS{Z+|KbEHvpz6M8RWh!Qc{EV(o{;^eBR!L@dEiCU&jJIKT}U_q z4hS&YPkzfZvkc?|jJ+5}+LKrPDR&<*>OaBsyDr@X4uK;fN>-wN$SO+v!l$&Z-}CQV zTUOq0`$SFQXsy3Zm~ZQxtQ+#90L=8sTCXx$<<5A0$G2foEp;DwP{0R?y&N9f5c+9xXl%M=Hbo7|S zP;b+xQ`d!<0(cQKFT}<*dF}A+6O@edW@!TeG|UH8qiW`T^p1%(&U!=|7keu9QAd^S zwAl9SX5~0tJJQ~Ns6A9V(>Aj&_NYeIHvoD7+ydxVKiGY66ss8Z2$nAnP)T{I^`_OL z;Mq>0JzNS&TDvZoVcJXU5Tr?=(asOiFMU*Hxld>EIjzM7F&Mk7^6xO~QSq4{0J>tD zw@p>?bYh|wKoki*@&4SZm>mjtbcH>jfOm{1;|OHE&2Qdq*-w>g?~_}3G5fB>p5!*G z8=|qTZ#teIzGAwsb;5z!|K^`Sdd5p;(l;5&G!%VZfF>cR5Bb+A_gG+lP135IxeFoe zt)hdgaT4%;?ilK zVMofnZFp3dpCw>s%8aP~y;6np@S1$*!h@^)YvqPh`h(I|tlF6giYeN87FKNGD(Fl0 zhJ3k^A4OFcW>sXtP12r(WE#$Bmz@c{4fX=Ho%2c<%&16#9A;|Q-S}iB#u&hR=cEJwWo zN?gYdd!ysAz~|2A1vFd2e@KW&`m~oy%(h@@jiv}NX((D)+$$i~v!WFC$sfqgdSi{l-o z9t+Zd^RVfV*l0)O$RX94I6tuOc;lkAK_=_bkF#Xg!PHVydVz42{aJ^ebxmZP(9w*} z$^FEF#6QGBB^*1Jg-=^XOyDeX89KmozudQz5Mwc4RZ`W@toMmpiEY4Fr`zNL%KLwO2jWKn!58QY z*VfibjwX5$XG?TU%qL{*{PN!Yx46YMtK=G=@-BHQK#CbU*?)-1B0crRZvcenlG_s7 zES9VJWh#lRIYKf!mEC4ROpB3a3kQ+5b&y{2Ckc`A|DttE_L(YQ6_(i$=~b%v-LJZ> z{ub@43klg-_OCGeE_7YL=bYgiIOj9~EYcT{80c++PDSw=U4m~jzt;G|=1K8&= z;p{-O{DyDen78w|h!pAv_S%-m!$C1~`hMT(&HIlTiHc1O9&Zhc^I~TYJg%5oA`+#6 zIJskfx>cx8@Dk~lTxr4@6a<*}v+dZmgOb`?zM$87;lXKq@dp5p3MFhVq^``9Gu-Xs zDw=OS%kZ*&>^p!vB4BajQE_>cDj%GXM$=GU{Qv5IQQ4yKEz`U=kSC4r5*c#x2CsY5(MkI zk}{9=Q;Y{y*}#OTpDEM?z9S|!YA(elm#)75K-7sJvgW1S#uMHq0v>mSyfv~97zibG zS+XybD8I@FHw=hcJD(a`6ib9PY{hZG64gb1PMuAk)UDZ8b@y{2CHh608iIGlumH#` zUfT=4{7#Ejn+roU2LWePxiQd}u#hCTo#$muHB5JS+w35PnXf%bve}F_T;E&hg4JpF z=idxV?K8E#x-Ak7(?#9w6FGm-DHx|+vHY7N)+@x;x4_8TNMJFJJ!u`W_4ky{Fb42p zUV$dyZ_VG>$wl9_(~pf0fgi{9xpT_9*;xLGTB1&pN1L2%rekd2v3;vp!YS;hT~hP? znR}L-E19=u<+Fjadv~R_S&5C>VN;6q+5u{GIfeT1=jMGz^4jvDHo_#3|4R~cz_4W| zlh%>v$R2C*F~+en9w9armNR{$5eMySJpqmt%iD?BEZ7EV!zM*Sx;^aP) zdL`dYuzvJef@)TMeFfiL)nfevyDs3wuqe=HThw2W;wp9t0jg6P0dm66o*p&%Lpq6< zTT9&ND;3NcGj|=%@mUtqfDS6wH<|l*0~Z;f&fE>zydTj_z8wK5&3MU(Ci_*33qow1 zlQh?*hWEY1pt%3I&fkjRoW!m%sYlD{B>p48j8t{(Jk#dJQcr}8tm+4fgTyTZ9&iE2 zbj$Xa@ul;4$15tzRcnQLZApYBAo0p=5q1YK9YFlF-Z+RA@5h55+^CjhSUKak>j^I? zZFDL)C?uy?zjn*(=z`q1B_8ui>^}ZMpC%8Q1-K^`^o5W3viLT8GJ2+;VTSrEOP7q4 zww4k3J|=g8ThijB|mVwW-W8cVRlK!iSH9QD)S?ar;xW2h>m$@i#JQmzJ-M*BIEOAXrgW)) zjodn<2h|+u3|EO36Y)M3SgDlTJLOz@jqs*?K}EU79*yCl>ONG=Cs6~XTE^V{g~Ft_ z=$ZA8|C`||?LN{LTE8($X*Fij6xU4p@Y(dWNf zg^1*tr1sl4g&w>B#|kwO zfzSRiQ7*x`TpfK?ydg%Jk^pupwSxHU6AxW zuDbE*mq?O&+Sfs!O1d-0sQ@pCFYI-7F|xJM_twBhorJ(7Q6z97M@O4=55r(2^Dsz?I+ciT8L=sLdbI^wp|R zgk$0Tr1p*nT)&)m6{-Z*n4<4YJX87I&}ZEqLW+aA7A{X54G4>>mf-OhGSZ-t+xgg% zJ1s)<*;7M2^flHE0-eu(9&^d|Z* zNRF~wluO%;D+YYYwlN)(Q>7$7yAhwq5dUE2;71$&277wFif02C#o|rCnW_q?ly_Y$ z?N1bLM5X@Z$K2b*b2jv#e48nMd%r1K;Z_TAcFw3c#0AR`L-oM+(@Wg*t=Q51aV<9xo! z+&~31Jl&Q|mu{waj5qOc^2VHp7O3PVb)NUG=Hd@ailte08;xR+n9&T=tjPPBVj;Xvq|wKwHbw!_~7CCesyC;e9Si? zF@s<%xlKM(13U#p${#%0N1yLgKNNoT#uK%SeIZ}=D~kBT+~DjR=Q5m?Kj?KIRK}fr1|EILF2xYX?Lnq!cCh{9f68&9vyC0bkXO?KYB7 z-d}elouOpKvW898lUosHP=O$2Ed-eA-_tYF@oMw=2^#^iHXo>!L;qWKhh1ELy;E-O zUUonckUu0tBsUQ&J`l8ZWPYT@y)Gi?Kh!6=&Oig1VK*bMF8?bkZzlRmqF?+k{1N=L zTO#WKn8BRC)l8RiVAYw^Zfrn@eW#)Hi6U*?mW$)OYanaNz)_|0u1*DfRZ=jZm+?3AujnkM(?nwZLJanU_*O3}aNHM}{&c{4`1GRwfJ^EofO zmI@=;wQGlyx(*3kp?El%(-8ejNk@G7SA$dl;`BPXdBmY+N9XE%>I(p{&!~kL_`=3M zC<}ytwkbyY{&hnyT?+yLqAU}XEJ4p3m@c7VF^#V_Wgg}I^S4X8(#@;z}uii+)o`>_0>`_JUyZ>k7HODb9q$In#6I}9f z0C@+)RZ1sZ%h0Y9{sn|okU$bjxPXz|eG=xF`v`nqZoIxJ5A8g44Hkqz)s9prnIb00 z-o)aQ^|KEs|26De;I%A#6V`S;3tc4O2SdlhDk+}!S@j-83?48l0BTRx#vNb6`V*4= zGm?*lC^ootbWV{l`lUAPz5s$2XaxpFt#=6Kecf`4KTJ5WGi|OUUSSs@!sEzfO|sYp zZWkgvi^=DtFdHyE;YGB4`Pcc? zwl6$f0}j{p*u?5bPEfM!eKwNpkE?Yr= z)=+Xvo1+9m>l=g1}^fhDeSH;RU@gBR6MYLwgq(zNtd^(W! z?+oCZE8+6+)KA8Cb9i@?2ildA2+Ia|;&TvIY6)71_ab=A#b$T6JTPY!-WSFdBYcd? zttTRz3FW_hl2O|-W4F8?S}3&)>_I*DV`g7;1FoI`bE8$E(Q~Y0tTDyXlCL%w6qs&3 z_Ruf=^0AFn-0pf>RjxRs?ZJ&cD(NqEv%Xpug?OJyS5MT)08&wKkoaYp{YGMpmACRm ze%gTuFgX@`{-#|3!;+Krgf;6iOdwWe_4OXX#>J3LnOID(PAy60MqSjhTvR*cA3_Rr zyI;ni8@pI+KURd-j)>e~bzOb^gizHAVut@HVlHOSoT z$eGaq?G<0#dC{TyXJ1%=KP^05g>V&OBWSA*koO0@KbQ2XrEO;!)jnwVZ;2bQElB-P z<5cu|VU_`cMIn@ThZNV@js7p@nQaR7V&r8qxCh%5I7pJ^f}d?ee|`VJM&HrwsdoyJ zFJ2m^|FVy2RgHrKWE`B;}4g?5Z|MHyw=CGaZE#T*nhj0=q5XtehOmSKJ&R+OP)%8jog?p{?n+<445}8JiZm#Yd(qx&>z0lS) zZ{2gK<>o9c`(R~Bt3J4-DRsydxEl6)jmA3JBanvSf8d_Alxz<@Mc%=K;jB6P-rDZ7&Olu|l#nQh|0wfAYO~~l1UBhU^<4}Jk-Y>D#D6hK_JsQOpV?TT*&V*7G%SMRb#R7yWfRy zVt>lzb-R?hqrytE_D=b3>}=N0#FC;4tJcA#_fPoRF*h1dRqvp!^?~MDSF8#@6DHSk z)ZKw)PB(z--2gW!MsH~187=DnTma{U_I!VMjq&(5bN2%iSF_t+b~#Ko8vE_IEzy)? zWW^Y$7+7T{m#o2b!8EOtGk|VY;^6=Yyh80dQ}I@dcj!EzJAj_>h;O)k7Fv+X_7Ni~ zwwvz<{N=LGW@^H5Y+OIkus>mk0!izHl*C{E+U+>{`>k$^$K9AmYH(mAt-U^@7O#vi$B+UM4C#~RjWQB&5kw9eTq_F-9(FJf&)lA45nxFKxfu6wWK4El8**4<38JM*>%hl*{NCrQrpj0+6MsT`|9d@48^S#XFoVP_$y{E|_L?zPiGSKw}(Z z+2M~f2xSy2NRgns8Ah7GnWN7@gmb zzVvtg*KK|uv*we!K|(l31df;fbPwj{+1vCdsA37PYYxGqd%K&rn%CmO*+YG^k~O!v zEA5w+T4#!{n5Sw!n)?}_CR_62cQ#M;gPwg0R~-dYWQF*hI)x>_oT4mjksIq#5_NaG z&MZau`andh^UbG7R=5FC3G;Z6fd1*q1*^G-O7g97&lUB$k3d>1tZC%fox>I!V28B0S+4?PE)K7>WBmJ9mQ0w$-V6SGp?qsBZ0rya0%u zqQ78lA%R*A)*hSV&=CtnNTh2I)Q@xjEdd_(eKGRzaUnBuurqp8m~t3{05X;wWS|)q zNT# zjHIn;$H9W2v%p6HrQc!oX*efIOMpsYQ1_zZ+dpa~QqWy%3gT;vQdMNOn;&*KDG-qn zK)(!0fo`*+W;dBsypMeDr~1T5tFrTaI+I`W89mR4fBBx6V4sCeo+g#`BoLBWX0{#T z6t$($qe7Ts%~Sydi;!}wHc@B@N(WJo89vf1hLjh3L|ZK{I2$^eLYBykiJlhA!sR1w z#yK8tQ;4~jq9`6xX(Kv#?>n^U1--cHL1q~>9z`ZK)Azo&-a1~ri~YZ)&p!6=M0B(( z18j~sZz?LfmDkm%N+LFZ05`M*)zhz_XS!E|Bi!PfrHWmO5M+cZ-k#24Ro|Z_oUl=% zN$j>s?y!8u8*NqgM2G^Qcaqx)gR}(zO`|56$%N#$+DlESC;3KXo@CtssUi>v;+xOh5 zC|_Sn1;tFvJJ=i<0h?VAAtfS=G?CXH_<@;TyjZzqBvMa=^v=$T6!kwiY##5xN=U#6 ze+4AD|M5z>=kAXt`4U?X-eZk{Xlyeohf_P!<2DoU{m~M+w^qTOfFkhrL@M!aXDs6;dj|CWE(I18?N5jP7_8y z7*Y~a{*(Td0yjzDk@?ms@!;R#40|JQ_ss)Se0zOUawTCPaF}*2dE80k!!JLmV}Y{Qdg7z3lyPZV`u+`YD{i4 zQ@X^*w&San#6XA4z(vwC0?funccMV5_TDH zt7%sz@azb7a2&8BZBr95eAH&N_qyN#T0|ksrE9^P(7^h1`A3#o;G%X(7?;ebN&hTK zh{Droqa9GR%J$?swd|IrrNvqyuWP|VjRh%c6{Q_EVV{S_mOg-e7Vi}{tGWw`Jya~r zD%SI)(2_wk_67NjWU^WyW@3rKI?#(K6gj~ zm+C%GmQYvYnOmubRoHRhT%=-C2YTxgTh5lYA=WOmITFsi^PvF)?bHk71ss$c+`Gnv zEWIvLfP z-NirMK^Z*iHciyb#Qrr#GMp9Fm)XBrvY_$xCIRp-5b>(tBD8sTZj>sxCR|{UErp6+ zM&feGZKkkiV20F8d79mV{_J(PWysNRRP5Al%jEW3(bRQV@(xk*8Kp!_!MEP^u-Si@ zUi@M4PC)0)cWX1)Bp@(@0Z=pPl~xuwo?}1lE+5f>#}e>>;&vBRUt@T^W11!0iu|=p z0vi%j^)r*((%LM)aq)ZK3%7q)u{=uxs7`O1qV4P)Gqwgp9HHB!h(-r_g;zsrJhB;5 zt2%d*-~5Xs2vxj_{F-M71#5nYW{QtS_Sut8!R?mEx#WU8t~#wj$+voywzYg8$+JJx zq+3S-X~X=*Glxx2NwF;3Wf!_oBL-la8xg_6sHwtcTp)PQ3h4u{n_N64Z8q zxzq$#jQ2)VaIf!g7KXhkG$=~;9FqfUN1y7P@)(}Kpi!L3d}NL3UhaoOz1NdSygf6p z5dLa>?0*H6h+1_D(@a-L*LqM#fBXN;#9ETrmMb+b00Vi1Q(=NrJ_S*Mtol!ZbaFNf zQd7JqqEgz1#iu4>!Qe@0LHBLneOkniLSoGt86?6%tXxMIRlGqdxS@-Cay z_%pm}QU(>L%|oUgd+thGqh>r7GCA%b#^ba7W7-u#`8HlkFM9v(Cn_nXR`p5-p(u}_ zCqKH<+xMvb%nZAcaC7eC&X<17l=1ZWZ63T0cxJ(``q@V)=J!!MpD;i}kKx3uN>qsN z06x=Pk{fc|5(ibD9exg-jieQ;JqMlqbe!#?&n^7M0EQJjO_(`^_V554-xa{eK3iPk zFmSkogDS0)o041->n~*BD&{n= z>RYmC9eJ&ljt$L7WPmKn%FbajTxA=OKBSE_+L&x;F$=zgo z!y7JVpU@MLz(poXm+A=8HPgNc`mA$Py4rMF?=X%!OuDy0?5iP3-^g8!>=TC~1kcOY znzmlHl_)U|z#tZ{)(Ae>vb624hRAp{T@6G`fY{*!iKeTguG1`tv8Hs5Z<>+uEgZZ9 zuASwH80dCHdtAwYWC-%4!dkHjRVEYK zFvMCsAXi8F({?6RDn3XtIk)D^R<>rQ_DRTmPyUHAfsFk0vzyVd#mp13a3YrgB0xXl z&t5$1i4%(RY9Q#|UMr1ZQj>gi)v?) zgSh0c0|S=;M&pm>fS!849xeHw(^cRg-wO^HqxagAy*XWx(d^FiGyK|Zx|&g;_U z)v9|jx6fJ>aa6Cgw_INCNVt_ndt<9JM2qM8x!e26$h2IwC%0V1!%Gs9i}?!lw%*yg zhW;)LzOnLYGfnB#keegFVB38khzM}-z*gIrsnP!*KTL>%5){NaYOpBGxQ6CkpJT6+ zz3TsE`*K}h>*0`I0mWJi_*))QHQNBroTmaS}dd=J7q}!Uc`JE3v28?+u(2Bzd&chXm1*6VN0gennLp*|1lN=;OAkXlbQ z(<^S}ztlS9SAALQHkUaSG%*8%@|oK3!_*Hwy{b`JZ&de2Amod2ehl>0kT!tjg>N-i zPBz`@#eeGOaVpNYtbll%%{Wsul@qE|BqJ19$qMzouVw7h5kLZ|*iQ#INl8(yG=y|8 z1+gvYo9d}2po1MN7!~K}vG!BR^kMutW!DOnnk1G75y40E_)QiG~n(oYWDj5DIL(1?-iguO#Xe zgu!ZHLOy-cWy2rc_^Khm9}8t`s15UkfS;o}VfIeO&>1O+2@%N`0St?SIr`|cFtK%B z-1dyOcuk*BI&$*xN1V46ZEGJek%&##60{wdJx#=;5t}{{A+mD^B~g3`JwR1e6?z~- zbFp>#XE}Bw0^1Rjkf#7bh%*k ziYfo+CprN`F8nSYL^KXz%-71eLa3kNANpPNmX3I)8Hj|e2kjExa{hBU!HInj(!!>; zPV6U{>a5Z&&yf{i5x*Aya&=T?Io5nzU36V*59902uHypYmWI3I6&72LYLtyBzvbsT zbK4|f{BM`hmyV{@l;CzlCg9djn`@0$ew01moZXET9aM|YVm}>cAcY$4!$4hE*42`f z)c;pg+6J$VKzcj`<+F*U<+yOy_JOA_NC;CDj}+D!L;LWJh;UrL^w)-(1Q0wS{fo7> zy|9ZTkV82%U26itiFg`hio)a)VpURnz>kMFhc)aL9ba5nTSYo!Zf_H&34Gb)+`A#m zv?{~qg8hsae>+;gQIZuKP1c>J$e@$5=9fG|fmZdkm*_ivkP2C5E0J{9MRka@MWZXBEdDj&uB$mzsz4M5G8t>5h={{`lC# zm!6rx5278mlcLFo>G9wpN>^_1L{%Hy%um%^@B4N%sfs`<0B1;2#~V@hMTA{8s+UJT zNIu3FTT>0?vkD<-`1Eq_1}=rl-cay5$HZD2{!s!}!l4ivy=UHMEjo(+Mp4w5UC#%- zKk#5+ei}2{0qmi$6TF&8@yUtJbfd?nQWwg*kU8DI6*y68ECvMwluYOInwFm-cnBU& zikY}=ZeZqI>Vx%D5G?B?yIuatNdp&QM1vJ_l2?c637H*GD+yQNYaC4sw%FfZ__}3? zhFD~?SaK*L1K%s^KCXazdZ(_4#N);rDe7!=um_}F>Iq5MDs4BJ8ps74@gVk=|_ zL#dfpgfaqUr7eON_eAnpx0mT~gWSuqlB=^6i#?j#@o^_Ie$h6oHY=ue#e$P0)F)RR z;Z%g??TByY#5&1dl5Iv}98`+ZdVj%InB~{3w~^u>#bmOgLj%UwxAy^}4brs$;{fu4 zP=Y2!MdvMfY@Fg_4?w}aH&Z)o5TQk__!~rH!R6XSL&zA6#YZ*L!@Mxish$?d*8!Dq>Vtz-N1a*fV&HZ16S@s1kL>dqfFe?ko3JKww zn8bKRz^*g%^+da1lZWIL4I$Qnf~!ct8ZxGvr(U~UHhf6`^4zqHw{q}?w9-EBuX}0p z%x$MXTS$#33t030jqKm zQg2NAr^ze`zH6tKdv3WLA3qW^7%a=}0po);ZL>TCZU1BP9^x1#8Z=_}nCiug1atK< zdGP=6n^6RgxggFCVbJBSgYKc_yGcIY2;a*7tJpdLTjTh`0SaszagLLyRa!+z1!fW$ znasj=#%rI=HN5%jcc5`&6=>pnsuoM`6f)VC)0r57w;ID*0T^bMO~+Sl}_k zE)Wzz=!WocfPg#0U*BU{2!sbH#bAzuSlfkg4zn;)7~)F1jX^WP(_~fT9M`QL@Qak8 z!Ttq=ndfmi!rz2>3qMZmupC6aTq?8Z7g;g(Tf8hs>0%>pc^og0P}gyn+2WmFidf%$ zSJw?@{}H-lKb=epqrQu<;JI!YO-z8^5$_fQ5k|d_u3x4X7ZqQ-3x@GWdfVZ z#Q5|eJLfb%^M$XLh2JeVsbJ`(px94+>d4v;7j0c}8En)cfu2*sKl%(WVLY}fHF|uU zDPP<`__e170)_~`M!TiDAj`Lr$v^Rqv|eru$0`L>yvg!40CStQ&Wv5-L5&L)I>z+W zq;-sBTg|<2mBKIo44N?v@td3uJ8WE}XYo#-`7=wc&l+4B{taoVT3KH$#OEHyUES%c zm}xduEd;L}2EKg>v197bNw4!%ZCvZLiL05ohbqCAz;xw&9+J3{G&9NV9LC2?YmUbX z-}4SStRST`8bN-iWD;#&r)m=k&8Z{Oc$Lf%r|LZbfsn?BHVlL(?u9>eOD3(ifPg>$ z%<}H~sbJCs-z!YV;FX6s4d#?s?_^1<`1`x_fG>N>KhjJ$s_0cp(M~nYG&#-*4kMts zSQK6lC|^q3ULALGcNsk0jcYlq5`IT$+nFqNbc47^8W~qqRh_C&g0uM5Ci=o{{Lq{5 zud)saYY#WeC_K8Y`ak{|U8^QUsDv#Tdwc?V`=Uz>kJKG?UuoQGzfUCG!G>FbR6@Ov z&Z5z(K7=+h^IgxlaBntZn_wdHMOr`VPP*39Iv)2u2`xnjuMykBh=LL%2|JZ;z8PJM z?A9p~!dNXB{&Zkt+UfrloIz;h^UW?UEv+Bjd}26uidIji{AjpzLd}D5xq_-&pNrNV z@=c}&HjXli#oTwxlYQ@YjGttRK3O-Y*LU69iIngds3vs#4>d$4anBn|n(n44|4`n0 z?eyp;^fqDZpAJ5Gpv0Eh=f=EGS-i_T$93ZT$8oj$m4ty9+pY-gXnb+C+%LS{;{ zIs#w;Au*bTPpiz5)CJLv1Kk`g)c=N7JX zv*U0zQY%Pw)W^~*w%sa(9TTSE5N8C0`hxw5Z%oz_Xub6 zfXEs8!U}#F;&2>Z*1aiVqABDcVZu>LM~j5#!=6rBETG9VOOTLKfF2L$n;sX z`g3gq$lEQ7q${mccQyIcFmCK8Qt&*NJq?sG@Rojbvem3BxmRi$3lBk~n z0Vgz6$&is{Xu(hQoUVNz-u@9KrN=O7;cCTCI-gUbO4JmT{=`m!gx1I^Fh|{!^RtTo zig6Nvw0}|!tOmSR_k&8R5a*{*J7Js(Z)l$xJDZ`AZ_&?qTjC2D4UQ6pu=x41Un(fn z0ui?{hJKn8_pkK@g$}|HK-cedwTIhUEee6?#CQFFM16TYR(trZCL&Zw5{;&0CYeIY zOvWV1JkOCxh9qPxWX?P%AyX0(kz^(zBuNOF62e{k{C@Y|Kh8NFpYXo>-TV1I&sytQ z>sq0k(!QN1(BGC{leko70N@zirI=}4$=5!cGuU<6c~2(qiz^ozQd}dJ_%_6S5%--C z)zD^mxLQs( z5rK!{Y7w+~B*4>Pz!6t1ll;3Uzx5w=6#Ms(wqJB@V^+y(_&a~SV@B+#|D_9)ug0MV`jxCa?qak) z#r{6oRZ#QQ1LKhyrzf{5`vo0Iu40q+D8-d8duQ4be7lSrHxdigU;o)ZP{JH|ru%r6 zx{v1piZY62C;6`qCH=ow#o8tJO8xtNM@Z}c6`n0Igpx`T*?-A*W=F+GR@=lS|EKN% zK;EVpiI~Qrk&4G8U#UJ7T$AUcX|k?>EZf&<`_yXUxV_b{E2AwiyaM zIRb#QR>xE0aR0RwK-V`kkKW3iw^hf9Jq&%zvmT zj`F&)S4B$z7dd0AFx+`np5vU5j%0<8q1kcB(_p`amRBXEz7ZckM`VL_$+wl43`;Bf zp?W^@d%v1F6p_EHk^t`nkcq|j{ijy*?@Y}Zwj$YHt>b4UBiDqK$@1>ivy*lP#OV3b!DNr*l6|N^YL>Ue(p}%r1DPJ5HxO8V?nS2}0P>738 zL_-hWq+Ux_Z6IHx%Z5qsz=LnM20*Pe*$6{b{iyk%M!q;J;}hXMY<}T+{TysjX|{Wh z2D3qn4M-$Z?%X@kJ1d@}hzcDUy`g<&N1)H(olg3K%*$n8_>9ETJf%6(V)xnntd8xF zA>T)%pm2xZ~XQJ(w z73+-X$`_Ub)nzLon}F@6OMA)kR$g}4JZ)h%bGiM5`f#Uvq>^micu`c_4`j(gR%sGM zcWy_$(5I?xe2n>ml`MA_Y?-TqTI&PqZN8u*;>zbCArBmS8@XvrCwsg?$TzEiyrfxS%jvGG*dPN%s0LRI zZo;28#pq;>6}R{{<#?4%GfhVs7|R&_&a264{O%EDpt#RKeZPrzrycw z>x;afu^!{}4RK}=ejb6_%ldYQ$<=3;v&r2vUKmr}@wp&I&z~|SC!8%MaC6kg`I12N zHrX`&yhqoc|MET&pIxUV|NN1qm5@-4owVzr0R6PPE!LJS400ND^vjfdd1bFu?lq04 zHcuEUj_kvf7?X--$$EKR!SY3B*P$^??CSc))FH&M*U z%gOzPX{_S4Ylqb&=6Cu4evT7&owZrlra$R=kXd@IxPrs8E5t#Gw#{%e z^z-Gd>>{JxgK1K1-iyy{-fl~U^6tqivelK?N8Ss}9X3V(?3BOtNi}CGM%ipQ6u+Yv zI(yayrp(@CR}>Nbx$oOxzBVVZvGfb_@5e3X!b3N^3;*dVk|64$H&ve}qnBsT{ellW zNh$_g*a%=hCcKcQ;f`gE+FFkXXPmDh8j!|5U94}||Q^t;- z%R3S>OpUpI*hqJc7iDx$GB6!7oyv@v;@Z#*681S+>FGL2;&(o9NQXNuw&mdai9MyE zWL=4j5~XL>e@-zsfT!Y*SI8zY+w3y71fq>5JZZyEj}3YmD_H%cpEGbeaZg({)WHFj?KG(c@2}#?k;hD@fW7Y%(II?0ZEmylYn2bIuVpRHpOtH4I0(MM#L<>+l z-gfTHFx~%z`ZRWSj6{N4tW7m-UC+6f^xto_X&m`_JgQZTC-xM52we!PTvIxJ%!^sc z!jR2~WA(oKy2Y<`PP<;DReVt?0a?(fG(%O{t6%rz3fK}`v^Zkkk1wJu6ve6jJuI>F z0zdRI^>zD#)4w_{X+GEHV_fd-U#;98p&Kd95sdA?3?16A94m*Vbj|#qN2zIc%btq) zAi}Jpw?TDRX6`FTwO{YqAU($Fo25dJ6tWdE1uvQ?{tGks$hKp1-u$=_BNa-AWMRHX zt(8#2H%0u#rMx_p0rlXND-K*T$k{BgtW7wcP3neWcsJ*aa)Pu%_^_o^wGPtNhv)JK6;Qu&fMt zI+w(^LwMvE@0i+SctK9!IX8I^zP$2ewuluGP~}!lNQ(H#S2=}Fdn{e z;r46CT$j%{;vRFF(=-bKelV#-D)0pjIws-M^B2p^{^*JX^jk@5&`G#nl8F%UJK3wb zjGB~vONcg%^+uoQ1NLqtX{gAZq%R3_61s#BP?uAXK4>)0WnU0${BZvVC!Msy^)F7h416fk_am1zSp@KF`A&uRYGYZSD5Dunk2F zcVsKoa;D~?&;avsrDYFpfbj)7$`TPW8gw}J!I4K4@5X(7^OgyTKvJu*U?R92%CPV> z4vKf0K_dJ?nswWr6 z6lwB=(iENu`|LpL7h!ZCFO0D)SlGukJK7*8>k%*bH;rxHDT%rg{>^4A(QWA(`HGe< zcsG{ZiggPMv4w1B zcM|PV@zsx|LLSX#E@m%80diqDKJaz%<$JS5mhP*1iJpMKomaQ(D4*dl;EA=hK8!Ww zFue<$NiZs+4Q~X!Bl1f}f`>h$Zg-vvJ$BfNb<^FP9?KOThQ9Gab%<}oG3xwXfOfzB zW?1H7Uv^-Zd>FDTWc?7u{rMhIn-x>PK3Re2<4nq3!(ZMw@u`SKVSxPRDt<$B=6=SF z;CD3RO8#PjVKOxe!0%Bej6~$bqE6BUq1^s{HT`~k&req|xyBPql%cqWlUCLZW*t#& z-*-3BuxL&WxRCWWFh_O1iL5z}&#+lLL6JKQ^^_)zor6vpDjqSt1cs1?X0!Yj8I&~4 zpt*f$wc@_~V5e83p?vq#%9^P*+XnLxbyalDsLk7!@8uRJig%_dFqIT1e2HxPZX=cQ zPBT8H(L+p+k-X{K&U*f}fbF?=@r49W2i*1{9%Pe8DN;4^;9T!M#Z<zKRy@#QJ>V*sS6XjRv{2xkf#sOssne5J41mQK0rMQ}MdvjY? z*$j=@*Nd0Rzc?#jlG?b55D0PXR=u3ePW)`-N2N0{iCCO{hEtF=nxJfb{gLSS1O9BM zCT~LHG#Cm4`9$n}-T9x=BfuDTTE$8I(6a8{e(JKJiwiYZIBrp8pwk$B+XA?RPZQRa2nzI4B2%ERnzHh3yX z6_ih(M8(-%{FHFfhRvLqnAm5V{DPWK|F_&IWxW{SBth?N;ZStE{bRx|OvDyOBJWET z3vjUDBk}UfxLLlYm@OFBDmOKBN!tcP$XSYbyJQbFW=TipO;TQellaO#Rg-p$eILdG zGq6;7nOl13$4Y0HZhLz2HV-oST(>@cG5Wj$>)FMZIBwC>KeM2%^xPKJjPyE!i~L~LX;W6Z>ctVPscMue<|no<*o6Oza+sw{ z@4}qlNmD$5Ot(&>z?g|4F5-Du_sG|*bdLKOK3| z%4!h5QNaRNu2Wn@iO0~$h~+_NY7?QS&wHd`-fXd0diF}6^3v_=s83=^F}sx4ZVWst zW8EfqZ&Yt`-CaPHCFY_AU4UGEd-r2usXbNe&EFS7;Xy$m)LVrULHDzubee9an-Lw{ znQgVsWoeRUoLAd&dZd{bilx=~W{9Vp9`c^V!bGv&Q^E8yMk=Z!2Qy3~Y*u~f7W_|; zcvMqIGir|+@sMsyHWRoSImspYa1TjNJ(6{l#s{3qTWp`YnF=vtY_nok zqkjKmp-0rNEQKtZ;-BQ+2;FNmM-~U*U%KM?AT=4(qo_;TSxUO?*LqZ^6WFoC~ z4x%1p9KAmnjENsNvRsC*aP32PPF1vkXpY$o%U-|M$v4(iK8d5UtDmVrRZ=KnD&-*x zDHN&~1yo(4DsCMaRkzC8HtT3(et={F{8y>LGCUo9icu8=Y(H^5uUWGRmt6bVZV&p5 zpvjPQd;K$fRW$OnJs~*!m~bWp`WE}Fhm*6jKY~v`H~R509!b5-`pAhdbN(QlXz!X7 zLyT9^UeG2I*7jdK7h9PSzlHvgfSTaJhHr~|l*@kq?`VlG7afKFz{upgcPB}#u1Un* z`a)9HtFTrq`Z(}pZW29H&ad+ev#}hWJt5|Wt(}E~N((U)>I?7mQnHza-@IN7$hEVG ztf-aF<9%s2B0Qa*IK_Uu#YJ1-_LGTOWzmS^_#ctBFNq(0hBICo6@4-o{8o3nO-*rV zg+kEnUoE+*6x0Q|`T1kmkVq;D^~LKx-ms!7!t?`zqb4K1-)QYr9-Q?$5J__E{ysuN ztjA4GGF@0JdHwqJU&ZSV62ISE!1P57x~`ha+`kL3I(zr|@OjMB{E4BI;o-(-4xge^ zzJKBdld!UVV^zO4x~OjJLk#4adD>e+E3cSwa-l(|GPvTJU6iLsj>oA>Ttvpy>&)L2 zUb;i%a#ky6_o-z+XpF+$71?r7dYey1EDlOZHRh1@0PB^rJ)aH$npBywV&kJXX4%&` z{*3hXg8z0`>eC$i)H2^64AE`1Kyiv1Oi;+|$`rWzMEEoso$werj2!27oCI>fXX<>o z-dFvM!n8{U2St4yUzFYklzZsEJKaqrQmLZ`H$we0mZ-z=l5vib;Hb+?=lFQ7Cq!x2 zOevIJslbQuY1RB599A|v(pweig zxF!}D?IaW)Y!i82I3=gDQ<;P2LY5|G2DIXMlx~)v*&p0sm5aqfJ{jS}jMlk~Hiwh3 zyqX9HUheBgd!Np+L>=1I|7mCNJ%G|ebZPp}_bF#$2t&clb_-|EXrSeqJeroSox9Rw zEO01}KP!pV>jKVnTtK#rZ3sgNto2npxH5K26yG!!#IcME`b&6^Uxq2y)FC71Yc6NL zo}+276H1klQ^Vkx%RGiFs|cc(y}|CKHDQYB=Nc5KueGtDnQ5P|fe<6JkuStP4R2 zvNVcI_T8U-Q>go^YjLjSz5~5RRdNM+tgamD%ARNrcgjQz9QFd-rkQuYwzDOD_Iz*_ zO{J%=f5&T9N1k`eggYFyZ!B<2f>01vTbcxUWEKWK8?MAZ>T?Y4oeG z%>CnC72RXIz9gu}H}iUb^I&pLWSg1P)l9eLq%$v#)JD14OnXXD&rDzl*jCMD?V0f`jIsR~5b*$I7J zxgA%Pdk|;gHJ2S}R?o^mxw~3aM@%N!TlLz!@*o8u8M^Z=8WXXE!w(v07$`mHZ|&{) z>YCO!4fx}t2|3`+jrwhDWx|n4maxtha=*@{+a+JA-#B(lfF_hLFDqj8diBk6;Z8hP zy81dtUaP|dYLf2|)e_Y%21e$3>xxn50gb#=bxw$3Y?XI?U)a^dc>rsuIM}ThF8j@S zB$dOs7PEC-YM;h)ZJUy{=_X58dD>H)(Nn*EkEg_9K z+#9c-Y?7zPSR-OEtG`XI>4eIhoos;y!uGJG!!>)@1jk>Jy*H)-vyz^=#TOY7yqD9a;l*>3e_?EVDRy{AgnOi-y*I!p z^jj)PJ#QmxjIow9Z$<%AD}ksqzD*{fv!7^#8&1WxV4}e}@0W2NA%tC>Dtt^Cbz>%* z=>`}DA=MMk_a`k(bgq1hA{IGGIcjyUac%13{HJUWd|IoXHG0)XufK1qAG_$+rz%nX z>n};MSweUWDXDy;&5EFQOKbjJZ87o9*BroW)NOlus-`)jTgukm&A38GCbsR$E?sXn zb7lxz)4L}vue<}xdYIVIUHrXy<0zyU4_mfL3&ofL{~fNIdO_R&<;B$>?=%m=IHKPy z(}zRa{9lbG-L52^kiqd|hFqUw+yN=aRbFBTSac~d8f6Z_jxhQb`9(kQPtv*a1aLzQ zo+}*m%9PY(@m-yvLpEu>bEd{CB$F!TI{fLy!(l@<#>VX!K@(irDm3P9bOErb)KAmq zO4G?#&wuJOJXtC>HPoA#=v12U*Oc!kqE~*p4%jSQllDE2?1MQ!>C* z0{s$EdWooi9g{avH6N*1K4^%T^pS(SNRU(XS#I^Rro$>Sy<{uzn0m49v{q2$|8dPRG-@yO_I5i^6d9@wVvsme)sLgR@S+`Bl5qdpcsvR-q|uK~tBmeRh6 z^11+Y`&r}Z|EVvn4P!S5rQ@yP+m_hdFLLY&WFbMv1A#|$2iL&{?fl++JMz2X;nc7~ z^7f%U7xJT1is{+LSoKTC-C!MFGP&zt0ycmGUsI>ynLHIbZ<2N~vDUY)^O$cjM?Cbw zIgCyYwhnJlBpfw(F^>z^?yWXx%KXH(7Pv#mV+bc($0y_vBpwZuYkfPrjjdu8Ba+$T zE;s>EKuw1R0U>FgN7z$%CYyk;qUXk6H!=&=e-3j(W08I7APRoS@%HN@tOxd=Ic`pL zfEZl-VrB9|o3*c>tj>3P)b_{1dG$nV*fM$)Wk5rYxd{*W3QD4ZrKWSJSUY0mVSuvN zASc7oiVkqYE3Lu#_19ceM@Wvvf8e7J5_DP1v{BrwvpH1nF3<{0vw5b~eaN-!58h9) z(_=hjgp)GWE+7h_J7v&$;NQ>0v;4%WqKV@6%&kwU-ByTk>YVGx=_&Jh_bgJ9C}evX zn}PtzsN(}FQq`-^Y$yh^EF9DO)h^Q>{}v=<;Znv~|3!A_v6A^(px<9Fg8;cEUB>|y zf>=_=OYmMJmA}y|>Lfm+{ow$5rOBHxZc^n;GWM`MjU(2K`13m$TnqgKzjKT(Y^>~P<`$O{FY=^VI+=x< zJ$t6!_iow2*MpuTTqAmmIi@>wiD}1_Y0jcGOk90^WObue#ZY z>m>!CNOks#zXvB!HovRBc>d~NPR%sg&Wm#vKzmk+0Mh*hv}%O16v zWXBu!M%NjoN+dt1emv0_wL8hnw%}Rq2fNhGazQ=^{;;3Xd$s=NlI6xsxEbEw8FVd$ zedW^$i9ROzwfTzoK3(EzQj5v6{V9oVf zv*gM)CC-q@(~oX1%;*gw#fBkT$4{y1*G9bI#?%$q1+f#N>R+{6oB3-C1Xm^FX^UkL z7Q%6xu#2v}arA%3sVvQ_+{z>*b~UR%UjO|T@&tX2{_6S&Kc3aqkaZv^U&ExkkxX5LEA ze#$12_XmQ7&U(aGsJm?!w)ztVod(^@?KLw*R=BNl=8ox{OoJm=&b0?#lP0HdP zuxjDawwX>^*-JKQXP;hB>024%p-xiON*s8Vb;?=kFazT|OafTnHaWwBz>OlLWU`TD709x_r{`90^@5mQSCVIEp4@2lkTW7JLLGi ztG+-3o!OJU?`ZNHEvBMV*_aIH{GroHE_4*N~g57876 z7bAT~jb`-mi}0Z9gv9HTPRcV zX#{kA-?>r+ryt(n54UAZ$A51sN-Zagz37*izZl%;adt<&DF*2e<2LtN*Sgs;aJ~ID zTsJ>;=n_Usz>E}1_P2M3^ef1_oGiW=4D7!%)$q7l;4wzv= z7)VVlonyC%*f(dO42b@7m4i+M0~YLpB>sN&lR%MKT3RYXmR|b0O4fd&*sPl%olqr7 zESjL8kPW{7E>c@Zhzty@gl-upAuPE5B%RDNMN+?Q6`dcF`nIaeRkOOkR!(uqbui*6XuR zE{evp-W;Ccn4D)ZDr0PFu8O$h%CovR=BtF8;ldnm-OFW4=fZY*jXZ_nWg7MCjR>Pw zahHF1#K8IgrKF;{w-r65 zA^9`pW=MD#5y-gY=3mo*K+<^r$}k+IHR{KbntMO2=WE+wfo|UHsm_UEhcl)nSHXqX zW8~7wBM*35*RwfFGOk3NUm&Ig>2{;uD44&y(WjozrymCPzzR^ZF z{W{P32sX4iqajeW5^C}09(ZO!@s5VEfHCUERVT+ut|_G~&C^naow>07fq$;%Bv;J* zja@Ptp`<$?sS)a_Vql<-RIr56{1O#u9Ig*>etVEUV8?Q#PI?d>790*~4Uh+z0y@U@ zp+V=L?a}Uj`L2zT+p6X&V;E^|d+Qs8eNb0g{Cl_qFP)j`ot>~IPpp#|I;WE$->_zj zSDfGc9&ZDW6j#LC$s(+9b=c%bQ;`Pj02^ZFyer#H$1+deg#6WdAh1Y~=irwcqb{WH zl-bXzlturu&qfFvh@8<%_N^W$mq0}>EiKsziQRnD(VU8kSj2K{xwOxtGMOSIuzq?uor9vVc2lgxqSXYO zTu8et>?+=a!js?iMAYRoAdpH}0&f8UBX`*c*pv^;crp}xbNDKu%p4b@{?8HIh-!&fcdm!|6a zG$>>#1qT83{JTGvJN@isp#)9+$6GSBualfPG=|c8jX znH}$FvW7|KYtPlgpa|S4BvhAvu3%QQ2;nX#BERik&o`FcP7Q2`I{{NpBLZ$#?rFbTMI>dFBaU0-sy`|Jr%LHPZ_=w;yDZ#z0SXi zL4cd$3pid?QhGjMZVJxQ4Zi9gRS*iyajCV?)5r=ztb9_vf{=ydF(SQNs^P9!UhIS+v2RIn64vyZcT zR5#K&3(gOwTud42T!@PR!toK@Y>;-saGl0L!eM}*m_@iOXsdT*fWwA1xi~=;Onroi zURVsFTrp|1uF!<}b2u9+S*Bp+)UTNvLmMX{vP0ui^3rYhT_ss#jg@1nl4rT8mhrN* z`*e?{L`zlfT@hy$ajOc)Re;(POP) zy(ER;lFWo2wT`3tI2HQ=B?Fz2*tNstD7u1I6BwV?ZF#1>S3gF~3NNRAam_NP>vBdk zTYy9=6><-Hrxm(0pLK~$z6pa0$T^`;cNoa;qj4f8pd(bpvpv*bU=QmaylbrGFj^e& z(M4CefoTabo`Kj$b@>P;81ArTaBh)#l-LR1aM#3f-h@iQ`#AUUY3;@qq40ncxt9sB zozyS1KUlFwR3;kEq3?+u&j&pc3z!w5fr1xI#}1NqmiOv8I51F*&By5-P&_U zQZ5%3b1 z5Mm<4wWsY5b$FNV9Mn(1%lVmh?^k6ZzkdRF`REPN-<8u0)B#t!&W1Lp9#y+JAKCT; zIEz_{=s!tE?*7!`;R{*%X<+Gh4Q+gk`-0`T#_3zgrw(d}{9N#3eqrnKycuJdUYJMa zvADQuYkDWds@^}=mTPG}R05>zs~uj0r-ua)gq4znks#BnjzkHBS=uX0EY z)BEhEF^0X)pDtsrqL44bkA!x&ECzto%lY%4L~U2>l7_H0`zEz7N--x+%11Iy-O4|5 zb)<3X8Od)X`V?cAYY~Y+i@QW}=3}Wm$ zoiuOAMNc^He6~om{|wK6g6R6Y)U8ICH^&Rx58P)V{jU%D382~x+v#J3w-^*VJD%2_ z-ivU8O_-3Cpb9=4-A{46hWFET$8%E>4woc&J}bo}J5_T<;miC9uCx2! zzFueh{P2tK_aWYhmy=R-lE38a1&zf$@05BU>^~^k$d?h_z}=ofr_@57y?ro3+-TU| zG(A>I+L}+ooA1nz>u1kr^+nt<-L|$o*jzp2Z$83Z{HylNM#!C?dM_UU1v9Q(AxYI! zoK^E#qb2eyh!wUdHdPa-kka$>ZOaVK5~J9IXdq$nS-|!g`^STz;$}%6ducx_tmVi`eK?zq-O0p~b9eM@hULHDt$N zrPHw)=~x~l-pWGd z{OxIlOaU$`R~m~8 z9qGF>m8%WP`~pX?%$7Qe2f8<^f=ewxqoE9_)KC7=xMmZ5kCYd(B>$lec=s`S@s#RO zi50};F>_;FLp#S2dzujb#Hpg5E{u`WD_1_bMJJngxr!tDX)kC4=4j_#Alf;TSZVUx z_JKGL4fnUUT`(46;Ghek3=4pa9m2>rtElJbO-FU_K}ZR`Z<}#FydC)@t7;tir0=+x z+B4ls+y#k~SYiR|-rQbJO0gk#&5-0`g=e=^mp!hjonjLrdt z>=GcwQ`N32({|=&T;RI$4f;HNp{cP@v4#d(q5aKnSf2k`+-X#nppY%drQf9{Z%Zp? zzx)@PRQ)t*LeHoJ-U6@Es0 zH!;^7g3ywnBA6!Hm#+%*x}4UPbaxP9yyCuNE>R4GvO6rVeojsrL$3S7(P#ZI5=PEp zVM~ILA5&3LeT)|jBtl12&ko(p^17IzL3rg0e>U3jymr%rSS=+wD59&vCKx=!?z}`n zTJ0_Len-y!JsM(u0F@ZLDr$>r}Byty-4RnJ~R|2Lk-S1ff-$@-Y~pyl6AA*P(! zZJZoU0xX;JTv?m^OVbsmQm$@F7mPV#9&9b2;w)aeZ>sm_3)7&6IL+3y)?<(DyLedmNg_S zr9!=eG@%d%^B)fDsk)Bhb#HsTKI&3PU24&x!?+;Q2VR}hdU{XkpH?|fll>p+0b*VkE{aHc*^r@SwO6PAZPg1+jHhv%o`FIjE_T^=UQ zp#-oBoR_+gTvB~N-}tl3vT?O9_d@KaY59k|YoCb!@U`@6@!->}sPr3WcGQE8Bb!vA_VFW&R+3E|X-c2nhH>xkv+k>h6Lm7c z`=h{OtnKLkA)lcU)jHmMkV3`5_W5S0dV%mhWfsiXLI$T!f-V5>iJlHnd>kly1+Mc~ z=8l1cU!n^8mgn;AhNw-@kxH2e1+MVEact*o@8}@{b8+AE}@Vq;B^@SepN0eT5v z$sH0TsrEaG!4@fB_z=aa{E;ntcXZSka_6(ydH7&ZUQ^tzL&C=xmj7(K^pz_IH#Rm1 z00?*v*0zXV>6(4n0s0VsNckS3-Q5+7zm@U-qT62ae1cvQ9t_Bjb4u!F`T|p3oi03o)~8`wP3# zesE`mg<(I$(4T(q_TM)mk79WkKj7q+i(tt))tj6YjRk$%q|Lju+(x%GJomo;K5NAU zolxktLcb5G7eK~`4a$9snJ7vrIt^)Z>McF~Ewc+;>-Msu4D2`77s0OY=sOUYG0(G} zGE~&bB4i*wKb_ogMwF))m}#9vEB>6A)D=4diQGbhGEo;#1~N0_%V>wL#LXZ z&H=Hm$vlgy7@1;3BrU>yMsw9cX zOs`R1Ra;$$)uZrn3p-mO8(3%goA|n)kZqfJwkzjCr^xt=x0#>hARPdY1AjEx5%s%b zfmG-R=DzM`ZVj^%Vx)K7i_BF5Nq6K>KOHjC`Q-s9BfWB=VsrJZ3;;R)sADevvT>F!KUmg?R$=7KEFjo8M|i zr+#f|#$)RuYB2G<#^2KXht15b7phW|rK>$5VWf0)Fq^4q3XK z{EWdJ-xpyhiCG3(393uI;ZM@T=wn+twNQ?P0CE*MfPtVqNxU5-76U)>~; ztiL~;4WkX*P;?z_#7WSz0nqVRCdG*S@_P7y*S?@Dw2CO+P6M$8Igguvs6U=sEXs6@ zQ!EGrYPbeGOrIKhVQ>roiH@p3?`vd-8V*9T@XCWMExm(Cr@B01Ccucy1~LXFW@w0psD5Tju(8Bz0JeYI6rVx`e~kpkz&i58vR9&7=uug-V{*Kn-%p1G2_R&GtBu znoNW-B?8owDSptaSv^6>7{1+I^$7DK?se@l?o$IFcT>E!bz9I{>7o_q{!1$0Q zyp2L@AYW?lM*Vq+{29wAsLYYGw%fpK`7%~|klQh0;}d0fnP=N?j95kJbOZzhWU#Wu z{={iJ9GH&+utwT5J&*o-Y_hh%rnw0|7Hwee_ecvT%I0)KR?s|=-@^L~_-z>pnzFYx zWhNDaozu%m+q@~ptXDu^rgtv+YsRDY2|#RR_R6$k#0WGeQ>^%xrHtVi4a&xOIZ)q*Q0JvG0W8c zWlzq3DA)b{DYMq#ue90p9S`)KdD}v$rNwL7i*B%UCdFU|ys=*RykUCz1>`}VlOz4a z$_au^EC))hKPI0`M7eTm*f0NI$r799BGS7P5)-dXn9~uM_tqn%g3MHmwD})2cyDlL z<{g5I8j)+*ioB-_T3X$)ZodySSA{~yz=*nk^Od=+d*KuX2?-7eQI{V@qXCDVXRlg1 zeG6j$&p`Y4c`_q@9lu6wXrO0-6c)jC-~Y(<8=oWqt*r}bvCb)JeTX0nP*s}oTvWqTQj1`|Ap!80i70joeKmiq zc6(;mRmQ37&_U}qou7r(7xYGvvI1%geZH?@B2cp*5H6U zE!HQQR6jj;t<-<6v(rgbJ?#WtQ15J*LMh{$6lkip)bD!Myiwr0-#5;}6gK8krOrul zvTipTPei=n7OAaMJ`sI9mRB7Je?t`dUCiiE4gt37jTDQqS-AcFVPR~g(BGrVnd<6% zKehj@*ZFh~3@9DjcPw7T86xG9tFw?}O}Q zeWZaAUi1I3!C;Q5(HK0Q@!Ha4X?b~9M+d@y-idPM6n;$VX#5vNRea5 zq?=h~WzmG_0k!8h)Di?)*VWx!@*&^<>Ep|yte1Bm-$_DTz}w&mUgBfg7KgqSekq2Z z0qN-s@}KQDqm=}MsdsD@0@Ck=#31n?48+^!=hE# z*!bUVghB=AMRjFbId@`<%^|6%+J{epEydJh zqG1NBv)G}1Cgr6$lR!=Owyf+IZ8II19gLl@kmroHxqMG8c8R|rBie{1c+i@iMh&W8 z3l@fKT`mYOq5R3p$l4EbIZyvb)UF}?f=?-u)-wBU&c1ah)tjL(y#`c?C)TpsI*QFM zWu$kHv|r!lM2%roU7c&(eOt|Mk9PC(lF_5HGILQ2V_rBuDebwYPAp>_=J)kBB&090IN9N!~L&xIM_1~>$i zRJ|&N)licPTQ=y5=bBUr$Nrz3-+pI*nVI1fFmGs`0_0&^xb*k%YZA&`jS4v~ z+UmgAZ-ibM!R+h#IyEuATl`Q+19Tr{X33q!eDh|nZq2G+zm$PJ_|_jqdaV$*0hxN` z5vc2X$mIN;WhpBMxidZzp*KD@A&E@@)B0CpRw%*H3scw z;el`Km)=U7xIXGTf4n9R(@e=#ET~ukwLbpOTWasQrF{g=2z#}$2Peb7wu#T_b2r88 zBsqRx_rH;rkbz=4Jwoyy!iMb@h#x0%9<_ z(lfA36^^EO{?!xOBeh0~Qebw<;-}Q9LL0xGf+V{4URH0pSLxbeo6EaWUkSf#A4}k? zKlIdBW7jS}1H)^B%&GZq4?Nt;o~>0DC|vwTKJ2~$C+hY4QWkz`90z3U zCh{`UIbN@~iL+=OB8Wh~J-uJ;1frpWHa=Xa^c!i)k!=>IVmVW{cRNRKM~eWn2?!7k zt`vMt<#?o!Qykyw`EOp{tfYT_hGN{=^@p!cm0ey?)?n<2j(EG~p#mMPL{;2gNDd!z zA9T*z{sB7^JuyYP12wvwSebvSOwZ*ub9aF9@A$*4R4|Fwwze60c`$Ie$Ya=G)}v-+ zn5T`UP;9b+Y=^a=e$CxFBeQ&NpPg9II+4_DBFv~@@uJKOVgM(RCq1{lKx2_Bo<<+& zJ4%90awv1hR@O0gL6eDv7y(n8JNCDC-Gx%B=gySPW!{)06^>Sq6$?};DR)^r=8;a3 z$zRY_@t9(|g~z6>-s!}OVjP;*RNJZdp&L$9~i669hx{dTk>!`S7onVJ1`S7vU0g*^R?G$@c^BC*}}dn zBX_j(udJ;u)@;7GrCe)RGt|Y(So>=}WyW%n=AlP*jqnd-G2MI>J$uc2ePQGb`R#GSFFk5ITj7A_u6s2vG&Yr;|~PK6CB&d%Po+SorWiNweA3PSUN z@p~c4JD%q!HjRy9p==^OgvVqvXV2}Wd3N`uQL@jsCzA5=^E0xt{8fdALeH%XB&KZK zoEsQJC3wrafyGf;3sLs&L!iHs>}mbB2z2P_ipuUy%NMt`IbG2QqT7RX8%mr8E$b;J zomcccdp`STkW-Q}XVhO{iBe2mlXn+N&Wvjp5E9b1P2{E1pxadzOm6FQkU~~Hk0Umi zSibvXlBd0AHd_;m7ZkzUDL&jM`P1c1!K~O}SC-e$ z<7kLFr6N~ZnV9j1(|0-yurc$bBRL;ju`e7T0@3EnCs@h0rS7ABg0F+qKRf_s!6L$}j7Zvy z0w1iU-%Tgz3VSP#@Xc^jSv5wT3J3^bU|>2yqpXtqkK|GN!64SCzY@>K%MU!j4DgKY zM|(=`7ZD}EYFMVNi!fhlv^w@)+Gp$jI_crK3Mf|qPG_we8Cb6y9_M&P8JE8Hik}BK5k)&JcR4c z$jE48EB=nAcfE7urzxaydCORfc{5WgSY~?lg)|nt+Naa2eD3E(>Dff{#PHC<6({cx z6ZnaaPW^(Kh?av88|PlW@cUH~ZJYRhsv8sMR`}P$`X=XQSoblQLn+MU^!5C|wTdgi zoScAAlni@J&jY1+f-%HXa%93_@Ur=;4>c8O{r!}c%Xx4|?p8f}1F|^dlKz*AZQ`r7 zTkEyt?RmuhLb({!egv=N4%mgr5wscHEB5mk0Rem5xnrJW3*nr_=gcVKui%=#!pvcO z(lU7z5R2&m_ryZQ^M{|;e~t0ayw)fy*7G+GvBtI&%cf+=N5MIhVJ|Hb!h)jY&+(N@ z2b+DWWybF_6UI9s2YIt=Yg0b`uwWZm7rB=Er}1}&_0pd|&fwn{0Lr?cXce{7Fn6ov zvC$FM%sZ$>?m{#C-FDa3HaH#c zQ>GFx;Q#8CSVBU=XRN6bt{Z=eH$NwltCgp%kfn)QiZWBFAFj_+(KT0Vw|>xzCt3*< z)%?oU$K)wj%4hO_t-la9oG@KoamVJjedU=jxy3RmpCo&+BnO#CPOHMbD@-*WhK5TU zUG1cP#&21GGF{JCnm0eHJjm;l6tiMJ;xjVf+3j5$)Lv6j^R%6{p&@_wAW&4sEG>Km$I>0vaVBZsPUM5!>hL@_}z(B&rO#V-W0A< z&o1>e^|(i7rAv#JgWi7@yPE16Qbr3Gmao{as2*9oy&mK9&~wr`Mme^BHT_}MOpKbx z-|U%~nrxpDm}VQekM1I@ZGWQ|;ujIYMHtP^2q#2Uz-I}_yL`m^4mLiB-jGNIA@v(= z5#OTE{GaVJpTG4-e@;z(!?d=`2x5Y7EAYL&e|Df|Pw;lm1CW{0^BR0?DXgyhS9x^t z$FF`$Dyq)UpA}3^Px4fdX>($A5<%R5nsnK)huEUQ#dA=fm!3SZp4NL3GvL7gA?rQh zv3}n+;G3ofMN~2>t58xA677s=8reHpk-ZfXLJ=w=O0ri-8BvjBC7Xn-Y_hW6<5Iu> z`+VN_^ZEB5&-07>{(i6PJdg7@j`KX(`4fU_jXvtdSB%y5{XYBnAcl;%mm9n{Jg8U} z)-%RnaLZAg4c#11zZpmg#C+`qEs^csgno`h{f|0rf+JCdo%ZRe!(a6}{`OpC46i{03|JZDL7T4vQY| zq5SXL#7dV6rgWUNP6DGvosZ{o%SAD5WotVwXP^6>MtYAAwMI(Mfvt5$xf*nhYWX$+ zkG?P+Jr$?+;$GQ4aq%x$TZMr%e!|~9Oc$;jGjBr+eUe9!wubZI`acdK8~jd-V!p8m zr0)i}*9t~o7x-gRsX;}!G(Xd_ct$GJIKDK}$J-P!ECLsMgM$m~$@t0dt+_1dSF_6rZ3kqo;nD3!w{Otm-@+-Z zK{b$}ws$ie&wEK=*{or=1DzUTpg}u|c39QSku^GVs9KitHl}LKEB0JMYuK>sVi{pq zA#D}k-e&e`@7sYQNbj=7#+gR#4Gmi`iEKAQjxc=LBIG}PqN~1sJ?`jum@VL{=hl|4 z8#9on}UjS)+V)-?u~KcDB-bgg#rJAGgWCG+6wiBplGE~wv{evZ>KDfXPn3^d7^ zw4X7Jl&etrGuu7ek$^7z=)tybV+@ty4cNoWuxAyj;s)2D204`F#W4T?fS zH+4OAr`5rNL|ATJOSdL7w-sE5);>)Ir%!LlTCMoY@dDx3^55|LQPtJgU((b(*Zh2U z=GLxKy}5iZ=nhO2wrv%}*l?@_U!gbJsHv#HzS zGopDOvxq>1mBv@c)8NB@XOOvXf4Yey?8}B#x>fzCU+DlZm-(?X#prR6rb(bu;t15(WlD%a;xVA4tM)hD#Tqr zopj;cYFbwfc3D2o#C6Qj^L79Lsv^O-YO8FkL*t*a%&&r->&z`xny5#OT zytuGN81ktQ{ia2e6>d%v1)u-qBU2NrKIulx!Ag`y5C=+E+LnrZ^a)jBzxNq)NSPLx z)2WrIxF{S@(n-7(-|FkfVU4Kc-A1$HL$(E0`2HBR69Ez@Y4MtoQM9g?X|K=Sk-7DU zHmvc?zbF|bcG%)=DA)V!tgO24y4Orhu;ja{rsjR(m5kM1?++e4I0A3UDC+C;nwpgu znfDHTlXFRxXJTv8%J#lZ#h-g+U1TTm($oli%B@?s;woYklpwQT=HiuH4FS-a(m~vU zy2b0_5_jUt&%FZ#%g~dyj(z}7LP|Z5%5W8Oho$8KmjZSO)x=j!JjNLMm-(6Zz7y-k zzt&kY!+kk571=>d!2@H!HQWy*o+2ck&ZwWB!$>`lAQPviq3t|fd1+Tjl`D4xi|ljr zhD4{ysm?L3g;1Fn84uU}VWfV=cj>~Wz_>4Ky2d2W?GJvmFJa#I=a?sr{$i+^FV{tg z{|s45ytVt4%T|5o^C=oJywhFm?Y%B_?4)m0wcul?2*!mw%2!4tn9u1{^()%tGoHFK zIvPU{oz-FL!}Ggj5JZ?0LotRsL9IJ@NIqPMXx1Rj!d!aJ=x0mfu znP(`23E|WYPimX~4?RXu1qS!{@wI5)Ja6TBG7#4Yjc?kGo+iofs(QD8-sXPrPGFNP z%2{#iviL&}_!u4joPpe_Ir?YS@oCA72Sq^5@(Jou)E|EIZjrEU_2}UWk-9IXZ)p11 zZje51olkV4d4SCXEoNKls_qrs9wJ3A`yA~?aw4H%>y{LSkXG4JtJ_li(#=OnQbQ}rRVGg+9YD7W*z7J`jB*E})ww5{@qHpjU% zC#`CLtz=VK-6YHjj!Pr)5jcsvCw=ne2AyEvlRI0`8Fx@fXfz{o-XxQAW2g+5LHXlT z=g!fA;L$F=yOoSO%;c(5%*Ku{#J-@?d*uc=Tgv=Q12%8J9NkRr1Tzl}g@{{f4t+7` zc15NULy2N_X>Z=|i?9a`3yJUFk^dx^Yi(i@FQfkPcXPgp{4;2Ff!$xWMv1TSFq3!Z z{8^o%sn)9u6ksCEGYM9tB1QlgDGXr}#Y6({lRJrv)K0us@8iX|52OzmjeYM)ZSBny z@%r`|UL0-0a)Yz7I!Sn_rBcw~Wl9_u#Y4H)Wmlo=55854H-}9}*rz`ix@sk@T@c8e zbB?X=-@R##vH3yrMW!yLDeD2E0TOv*C2b@|))p|$or}2`4wpC?wgN3s_15CjFM=^@ zSyspKBK(j1%!_dccTmB+vot)wX&E1%VvHjCYReL09=s&@Ul{4T698MQy~vX|VxcSu zXF3;Z-GlL}1dl(>ZnQ&4M27gW$d_na5%+iYNsij-?3O>>OE)n8QShF_J5u7v2i*@F z3I}W?iI_nJr3B@vB&HZZ*&sXV#u1^C3ZYK?{b|$5QY)62If=su6k;wWSurIb@uytB zw!*E8AqoG?okbzk3I;eYJYnTy#M%3iN%h<0Zl59|Y`D`&d`d{k{|u!df93xkbH`gw z#wGLW^Z-3&zbZ5XMOJ$`Mm&^)#LK*+37Ydh9%dow#P?Y&CTzT{|gSW#Qg6| zmBKo0Y$d4x_V`fYH`v{>pKRy{t?6uP-Zt?{0OIlmW5VAkN5=7Ay!Sl-dqfR(_$;9VI-Y=2AuA`tHp@0oJx_#e{>6Be-2+3`2Z4JxK zDkK)!|9!^~YnECr@O={%ma3XN?2pOji)kl`TZP-(%^v~=(ms(_YF>o5XTSZJ9`cSz zgu}1h-(44-O@y17S^nQQ$@ky4GiF@67#&_6^#uz;m{0~LJ`gza=4MT&jj=UXn%j^( ziTX~G7+Q9vH8t7Egh!*y+$T#T%<%kmx_(NYUV0Csd70^4l)oP?vlYy^f7@^m7R|Pk z^^BNxqF1&jJ8@8e$(M}f{t_-4R2sxHj)aUYk5@sN*Zp|u`R@fJuK)jj*~|-KPrBn1 zlB2=VVp?}Fbh#jW$ol}Y`HDl>r8Gv^$jM2Yl`B`;VV}u#zviNhrR6v?Ti$;!HPh2Z zjtgNKI9u(3)F z5^cE90j<9m&SQh*uBHCxMGg#d!>~PK=z!g29K8oMH8soJ7i>~IRn zPfSf6*tc&a*nd_bX+UDZB(VHzWM}12%4J>zl^Z!M_qc`PZoDo%4 z43+Csre38@`bG6V(dYMuVv?#&(jc11b}{)`2e*)aIW^Jrw;n(rm z_>dHb5z{@ca`h|AB!?4JVlK`UJEv4wB$?I3o-65}sLQW6$|TKO3{8mz3x7nOR5VQ? zxRC$O(Bw2Jp_ow1FqBmJ-59LTyy%`_nHFRhHOy}%(1ZMeX%i--fBEz}kir2=< zvbmNbwNsn5t*zy!9bUxuWDX6V!TY4*%a9edpDDyBY1vbO!gFSAEFw_7{COm@zXvD( z?p*9{oSyT#y1FcuUj#AEWGyle>b}5U+hM;-Dsa}F+!6@Cy{`xL*kA~VGnl7PUV z;fwijayo8Y_hO5+ily>1P&2d9QvqIWfT5&?u9A zE<${3c5_}491J$7*_9fQkHS(?<&qRr&U3r!*PlwLP2R9ivoTA^^x6a^NwJKeC;T}f z9L9xARe5A;B&==jwy!|`L$X9WB!r2y^Uz|5ztE9^V6<5;-5wo3N8pPbJOkjdxgpzv z&a0BMTWP&?J#E%*-usiRy@y+@JH2dgXYaEBjcbShyL~ze_lP<;P4xz$i8T9^cHMv= zx`PbUCNEdhapl~v$;PAcA9ZFv2RuP15HBmHi0niQu9fCN=J}!$gZ*1?ReYr% zFg_4lGZUP!RfNS~K+Zq4c~G;V*eElZQJTS+BdFA)RIdYz!MPx23X|6g#M?>wbI!0> zEGLu$dT(~Q&ULcaH*7)a&^OOb!0V2>2ivaa-X}RJth|^kPO)jwUWGKYW>CXrMnCy= z$C~c)uoArQ60(7crq=CW3glz8MKRiRtag4T4JQ){MaE0^~OkawmVY_etZL8_f84 zE?aP!DB!&;BCC@5T3Mi^gB9mUiAp**Nm@s4YQaUHcFGspXY}9WU-Ye;J4`fc z#Yf0L5x6sC9!9z?r2(g4bCR50v}?a-i=Yu?WsJMic%8D|j=)JdKL-tpoRc}o1j zJd!h|%-cd`VgkuQ+>lTXb&Z z0;icn=(ONb5E2Q@#O0!nzy8g4xDM+$q~Tyk{OuuHJ{_oM=54-YM3KjI7T%=0ySp0d z$=9uBhz8~^lNphM2M>NA^Oq3ed0t_?3DM(7R|#3SMo>~z)DH4cFq{-P-Mh_f5Ci~8*+4KoRA)=FJHanS{MItx8mMVV3!1wx<(kgzT8@|wgE@T zC}XIA?rYz{)Up0uijo+wPbR!*d$AT1P=Wz%?{6i@x;n=5fM96d(vJ5v=-R8Mj{NXk%e@{OjN_7% z_xg);tcH7b7~@qQv-wX`lXA(6)#hFQM2F2=63%WK9e?+Ad-vzM2lHZt`TzCn*K621 z^9Rju{(5+*$&iBT0;fN)j!q4IhMRy4c4QE-o%i6_{WI&Ou>edf$($J&&)W zrIq=A0r=G8r9o3FzdRV^F;YwT-SCv`_TP@Xf=bQ77}Yd3mcF`HV+VAH9hPJwj_aJo zQ?X|z*QkMpW)(};dcB952`rIz{je2f-O3Yt8X{$4UU!A4lG(qAy2Qs%uz?h3x{UvX z^+z2+%7D!*vuOTHEIwQE--9aF2ZxZbNA%?1fNvVW{@;7haqA!s&Xpl)A%HGX%~}Cv6MadxPc zAqlvl0Hs3~Us-wjAN=r`c6ep%K=M<5KC;Eqj%Nh>fIOTzTpSAN<`30y*nK^BbfxEU zs+ScPCHkHRiJ;|+3{g}3^E&3(*`&$lc^O)-_Z0X~N=R!2NIXOk7`46L7sA->TdMa* z#d)FkjAXp?Ji>b_M$HPs%s|wueV9&*r$<_L?YbXbeT~RWU}Qn9sFT8s6Wb5t$N=Rd zkp1ksf|aJL16)3kVY6jjDlzkXKIz8;gZ}1+{UlP^ez)f#^HM+~h!PnQ@%Y@xj1!+} zTW#$}Y?ysdg0YG4Hlhf?U{uwU^}nlFw{i)67(c?JjIC zoDo0-oJW&*umTh^=8^oN5I_SXI=*#;HZoBU`A@8P`_oi1^CrEKe>)AOjVH_`od!G8 zb|Z9bt=Dv&H!bS58M5hw{?W>FPW;!$Wa?W8f$S~oM!zf~aH!pJDk)NF_1WOtEIM9{+ce%? z-bv(b)ceH-8EL>;W7u$>22X9$Z$bDw$PK{L^d|K@5P}!(NC=vsdo35g|5v{5s8jqU zbc7`EzjQdRP#Lkn#KlIudEv!Ttd3n;cxe(S15h=Q0aykZk07tQBSw8KfqN1UNh8I! zXcCRp=*#kW-~KcwXKO)Q05BtrBJnG_&>wo~x5E)cAM(6eT&CNP3MnsQpr1LB%R!7e zCB&>Cb8DVHt};j6J_evm)*|0%(Dotr#$%dlB5E?;D%nCqIyr~joi`?T+vkMB??=5#5-sb!9 zxUB42+@U$!8{$i(*&-`~2YgFQhMwx`LIMyd8V-G21gpYN-NSfm5!PrN!+SyUV%{hl zQJ;L``{#nV%|-8I5-V&_I4c{oUOHIc2V6 za{RVeungXMhP5<&X&RBiUT0wOvMi zHSd+`HNb<_2vNnD1{1kAx);8KVnUkX8UajDTO1X7h^nGL1M!M}ji*shLQUUm6wo9n ziXi=fna#AAn4GNp!`LQ3`tj?rgpGI5Qvaq4DqvC%j4lf;>*~d{?FX0S3%hQl)heoL z@7tpM+I$CEWQj%@m62gtqzin6bc`gp9Ks2S=jbH>y@)$C4nBw3_hb6)3aS6aaxHkxEgNQN1X$mM}1;WIgmbOy0kRT*_Z4uUfO zX;)Xtr8tLn<$Wpew33TNO`3I ztAEXKOGpOEi_Q%Q#SiT4rc?KEy9K@8AG>skVJ)_i>N~q=EK$~y7;55)TzMN^TdXPd zWrWlO=$fZ?9i_g_fh$#>qk#`84>IvZZQRtSe5fYl9fTMTPFi86S_b$$YS z7L9uo|5vdJo%uU>Lir8tpZF}eg`1k1>~N%JAo1sZp55FA!^6%Es}rpvv7rkLSr42` z%e1t#+%VB>E#@qy9-ocTAH-kK1jbitD8RQNRNsO25ETijH%DY*4Ay?I6}+PDo|yNEqpvg`c}q4bYuXdjscXK#^@V>!F0&O)sdxo zZbtfEF(E6zC}l~&>m823hVKDRhUYRZ%+(EpHuT=h)0Ef6*3i7e6)HyA_@%eESB7DY zMCGSXQb}zox_#4f+%+;&i@1;a3v)A+L_GIQ%lbYmK2ulRk^R@di=$r!X=ZCG{|L6s z=hroSuzlZkqc);bp?#c}@;A(=e|H*}BLV_R2soAmaFJ6&0yy5tbW2Gd&iveL+oAy^ zFM@Xg{QL-@Cp|ZHb#=od>;89SGTr|Ut$lhi4k8!j9VCcG!)JbqvTYqPN;M9-W;(nmMot947~KLO$pla2OQ1WxEfPW9<9!lYhpFoOcX^blL=6H%>-By#v*KFc*mA&z8#!~qvb}6z zlgN7}ej9jWeLxQ|29=(~Um-LEPN)Fx65&n(oqf-8zsGm{jQmnU7U7~IJqLmT?K2%w zUThs$`m*8&vRZU6+-^f&_5giuCVg`S1ZDUvkn6S<{;=8W#B9kC*_y0Ljh>j#PJsyd{sl5zx&`A z!xUoWr~vGI8f$J_2mupv)zeu%Ht6O}stWP(v*f-I0*L}J@+{>EIp0)ZC{OO(n5!|Y zRf~1)MeZasP#_zfRyI?O1?KsG# zpi(^%Jfecp;Sv+!GLUe#M1m%C$;=VHf9F_o-@nHVomo+Vf`Q}=PZigwJ4r%nBU}mc zTY$u$UBt-)+JmB;q==^UBFth-J>GG<+&~xL7*0E4=R0v?8QRQ%?L9~*6&u z5|~Z0udnaeg-vI<8E_jU;lBLpv9$y71$CY`nG1>P#LHadZuBm+tyPG0uEQ&iWOT6^ z%A3+4XaIC&SBlWy(N^T7*J{Z^N7muYCOnGXg|ZJ`)1^`lNfV*SIL%q4vDy8tEnhoS*hf7(&W7$JeG1 zKzsw@m{UrMEmq><_#PqsFW{Cm!Q%bRy{38n%4Gy};LljS*g{`yWCiI=Ch>!q!3fG#`TD88_WSpWPoF*= za^Sxlx}TuKf{Us+tswiq;LlueQze)wq>$_4!f6iWu>7O-#VB|p+sb=Ao{zDH*b{Dg z?-7g`Fh{UT(<`NiQEiE>L6gAdkRC7sDD_F_A>mzj-d~N{GINbI7=yOhtwWlE4{UHx*ah{_Y3m`MFDejkVhHw9)j21$PMsz4W#SJ);pE$|^b zGd}1+eq{!|C)Bw_EJoS2$&M_P0eB~j4Sc!;b&x6xSP-T))LQLp3SFUcG5I3vZX+ZK zf-X^+4^)bS$Y?o=38eFH|2yIe_FaP`b1-5^4v|5I0&IkDmVak zC#6sOA12JF`sdvw^#k^)pFrPSL8bewe_bZHnY1bKGp&VeXUV?J?=%6+ zWVpr_Ha0Z%-N4zjK*sGDwZp8Bz=jDSV&NpSSgC%%8!^{n0|?DaM1`YC^PdUJ(2jEO zbVBWZ*99jm$?C-(4PYl-sI~KTotVi(aBaBj!pw)xQ5vd6ygV-{&auqUP?a~Y>aK>? zs3?RiB&DU#?vJcoHUbU9^9d;ROcvKTJ8^k&HjwhEcB4ArW45xg;w#|Y;++zJ@w#=W zR*d###MECxd(^tGlKcRW0mI_217h(riS|8T zygHo!{@Kf>m9&cnHiz=2?s=T2-yStZ?{ML0s@p9e#*J%#@iBe4rJlxj=T<}6+OHSx z!RED{=HBw-QXDjWd1j9u4F`CxUwh}Ncb8+OvwDtIWktB?RbiSi=ZNv}Raeqs>bi?vv zo$2xNy_mec6s3I|iCE`+bt1U1Q!RYyyj^#CB29N4dGVW%(sEG&DOzL>@kU zx_#YB_ar(xx&TSRWv#($6EcW~G@naPB?uy&(BXni!8*7f8JE^aPEEa8@zb;}w#t9U5ETw+YOv|OG z;t7R}45irF*{NMfeT_F`HPCQhx5R%GDw(=9^}BrM*@j&EjKNWY?g4Y96BLt@<0H#N zZ~R3X>I{6SdA-Jivu|eLcM@91_XabGi!WH!eRmt*M=+ zEaLEc7tV@ae_hf*L)x-4XU;fG4wT;6$U#c<-mfub`m-X=e-3)lvg*9Qy&nIFdwTl* zH`e{d_ja5hfT3r{=qcY;h{Qy}c=CAMR_9PZ&MqGc2Rgh_77xprx-YV7D}g&(cPw?cHP>G+8i_hMv210y3Vi#*qcaSERJcbh#V z&me-{+J5lnizaGZ@%FZN<>h`I9UTRZTW}s18RFyPMXY~ql9rZE)GUa|G_LRH?mm6_ z@+Q34ZUE#L&a;!DgB+5Qk*{7I%ra~J{8-3X1UCpx#oF)Os0am&qdA?GI_d z-}4-|ikSaU@Uy^9Zz9J`VYjQ-7w0D={o!uS0%T zRtR3>@#Dt_cq8ZU4<9>q;huiuWcI^qLrw{ah>9nNI|l}wsz*7_`0Gvls(oeJ9745h z<-JFbNn4pEctK8}Y<_li z!3nB^=AN^$@=vBwm-aQJJsWHrhYf2SGFQ4qXR$0dg$Qq+Yd7COKzOTVfBZPL7acpd zfB+p@{=k0pHl%UvQF)a*k!MiOjC(76{`>|AmfgTTF}B0skuVq(6yO`9L&POAH^2Iu zO{r!nS-K^(E9uz2phP!56as{Xum1UT+>lL9@EF3(yV|^-E15=&q)_pRZpak4lJ0@V zi@xao@<#_snC8Gvh~pj+tQLA!N~FzoVVvEc?TC@%Ku1Rr><}Np%F80BrP$>zMQ{?I ztv_%(`rcZMkH4s(a7spI4GbRde7tL+F?C3`B@ws?n1_O`z$(6Zk$_O)z|v=5QX%UT)A>3RQ$!&%81u* z-*$F)-wp_%%X6GEKtP1Yzb4PomO>aL{94y>F{CvLQuSh##jgnmvfm!nw{p53&BDS0 zNI^UV8Qj?!^aYqKzKEO+<0|i9cnRO2K2mr5!i_+$A-1> zjJTj#tiY+`J!nn5yV%^^ z@Qx0h$>Fxjs;CndR#u$C!Xe4(Z>iN@y_t8y(|5DBo{ALGE?N%2_Nlx)GgA($74Xa{ z4b;PkM|r(1xbA@=AuH7!^2;9!b>fB@;E4Y2ucvcczfCw5_orD9w}hz1HqteGiiujN zglwD#Ti90|algh!7FdLjREUH0#RTZySZd*HnZ z@8KkMFcMBd%!?Ofb7*vQ^vL))rkK0q(0aoi5RsV3&&$UritJ3pR(z`yu{G*rQyn*# zt$HYKzh7Hh8z;KvjkPhSv<^=5rcIl)_CJC=aNm?Mb-MM>@Q=a4axzu^-J6Sf0ED69 z@*!Qa`N@Htjnso&TwDeAPdJvv@v5s9Le6Ts2frVwARyLipw)PIx`B8%)YFSBe z(UJ?^kt0uXEPHO03)OF6W%UaU-Avv!w${Va zbcmCa^SruxInwbvFqg6zvA;J*()QRfsz}!bQGDzjVq(na&YdIOGr+r8>dG!9BU1zk z0(E3nf|9`8+?)y3>GrlB`0TrZ;;DZC;+;Hm=8}ennWKh+f{d)J>-gNQThtT)puICr zMn=4f+1K|pQdYf=4xVJgmntco`GuVSiYfgZX?I2RILXyzBl<}CNj;?p7~93i=L?fi2vE=l28I(jjKCt(xNsw)LM3T>w#Vbc(_tGC;24D$Lj#(lax+QWNPX@ARqE+1O)}j(p?tO z8y>m2;uv(a459NR;(|cV87v=O^l^2!JarEKF2RqfT9!1>V9QB&L8$+yWx~ z7?LejLl(72Y<#>Aa6&cSFlbw173!;sde0b8T2{8!4af``>^l z{L$5Q>e{v4{{H^N!bBk_iXK!B(6wQ~f8EQF{=U=Lq-YnLBv4TtZF1us|=_xa5*avsd z34MJo^|v-Lsj2t88F;AcOqVFF=BbYeKwoed%7BTg`X_Mvy#S%otos!mriSG3moB|H z+y0}_9cnlgUfX-XF>`Zst>N!^2d{q*hZLfitk$^8@?Y{<9yQvxI7CJ6qbp}wy87-f z#mN+Nu9zJwa6a~;f-ykG^7|V45*ezf;Fyh#u!V&Mi1jKp`&4EA-@j^oBaS$d zE`8hvI_hQ11eyaHO}cTgOT)ljg@5ji%^+{!vxSmR z>0gLhhQEv$Hvj@Gap{91BK)Iwp%)7E#e2NUt;po4e+OFM)-zpZ->-LP&j>BaefSwh zYuCMf^JWZ6fky4arcC1!(!?R@DuGYV&BN0Z$kld5%W%;k9f2hmO$T%R7_d9UUE*CSd zd2ltdc6K7IZ*6%{7Xmk^a@{g(eS@-H`)%^uw}Wq!HA-?Z19~jy~nP zYwunHk(P+W?|EUQ?%uq)1c<%ijWr+c^iS|_ocLW=LWq)qDuu)y8%EZh92}#QlLjbQ z#Q*d@+r-Ft0fh_-mMw?O)Si9O{I3ct=AtJQ$>VpPDEf^VJX{%ocrRY0Pv^(g@N;*6 z-*$wvT{Ai>iwPvj-_p!azUClm4pP6P1RX$_L~;NiKM^KuddJ@%^H~JR5CdS3A=}xq zPoI8Fk9L9x0G^_q9BlSVNfE$luQW~GKbS`WW*fyAsGZzdwY9Q z#Df^BOVO;v9l#;-Xtjrkl3+S9D~e$0J2)Jy{r_A+qvGALb?ddm-)n1wQS1}M zIn??#4%m15cDF3k<}s8aOPh`MCg00P5j zgaR-usfR02(ZVtvLYh^;WKh*{x+AB;12)$aKUCkfeC0~|ty__ly_1d?ULPGD-Ertp zD9*ci_m{0s&dxcI61Pe?y&l|hC8ROe{s0*f1@;A>f1MjKR)Z_M|M)REBLnoSD=co@ z=zxaw473h@N9W++`gh=+WzU{n2Ux~+;6M;Wm89x!fqyXQcxT|jv_u(Hbk$(vRl~)s zePEZRcEh(GhqYuM-lAmf@YfjGMU`m^XHr(1Ks1}r)Ou; zu`PqT9N_Tv>(>W=2{E3UY0cXUi6M~lf|}ahk2-~oja#o@A1>gEi$Q~I7s@Shd#|d>{luex>&3>#mPMz-%EB@r(Nf<(VZ(m(#q;MQc)S^ia}a6C zs+}Sanu46r{>b|%l@GN!y+^f(4ocj-j=sK9#GE^K?il{=pI1fSS$sl*z)UzcM+B|N z-NUrjxGqt<>vQn)=g;@)d{_;Q&rM}ttz;vPgJ#|% zZU6%Liwll_CI@F@XAwT9_n>OP5xZe&IjC4JcK9#@J^gXS1u!bw$oQbHM$?_a#{;Z@ zj|91pX3=?C*rb7?Jf(lV=EeGFr~Nvxhl4{zWIE+x#E5{BqGB=f0*M8tG1KGY<5yC( zsDJ^E#nx13U)RKtuqau@h<%UF&OrkslpEYgiyxmY+Qua%b-`4HCV6)SR$&QaN?u!bU^}k;8;?IE(xU&n)*5hu?-@1^*q1TUD=4u2^CK zQTVH`a4Y^!y?xJ~r0PFnQ2$_N7iVE%*&=N6&~&%~)xJOL;bJ^__bYKFl|1*5>YKgV z9>2I?^yNc+sl=8zLj$Y8!Yr?t_;{2&+KR^axsJ?S^`Z3VeeKh`DPXLJJJUl#lSBkav zDVM}wd6{s)!bSKA3L~Jkl8+x(T)lb~_6FZZlj7>?jk*eZnHtxox=g>=wI(w&Q|Vw9 zK6JY6um-+}9Z}a|jA_iVUZqSD(U;pVxy52!R_-&-d?}j!aEG zLtQ8$CiXSQ^6D?+gDRP3$E1`gT5>j`Q&C+pfv1d&jiqH|5Gg}&;VvjDo(7!M*VlK; z{MpxagF*kw3$V*TOSg&Q0`otHlFnuTy?J_iURF}duC&2M-?14b6b< z$tp0DlDgwOqv*Thw*x-BLGvZX)L`$0BgC8O`BF}sL2 z;0d4Bp&9Fi_$e!3E#eSCK%3*L2iLHaa@tdP{w5qZPU+1nk`_u;DTa@Wu_40S-@h9c z_RLIt8fj@7e9+$C^k(WABfP%b1qCdokKFI$5;}6dQ%lDrR6)s8f&OgffUcaGn zKXai0GPUn^jpjmqzTt(Y=@ zzf3dIHgId%3n?qXFe#}dZD1_aE9N_Ab|H}OdQ7XJVA{4eUWZDW`g29WzLk0F0%<82 zM9*LL>RR8q$WJO_{&tTZKHQ?FuKq|+cO}Tok9zwBZ~T-E#)-6WNj;5{{7h-jA|MK+MGJ70)z(zD(ZwAD^1yzCp5RI5UQJ4@sA@*eGTsipM z2)hm%V#sXnkE!n$+_KK<*EjDKx+VK3OL;_<6^lVCeX97RH5|xgXmH?SnDBbmI9?Sa z-iyjxaLt~-c!9LBc|QZg?!zuDyZ3G2p`()z7tv1Q*4p3R`$J1s&-?CO&VAb{^)<15 zHl2R!SdKh?`Er+8-h?7)3nE1nH0o7kx;Ej?+^Ci{fc%Y&aJ^)h*k(`UAdlkb*O-}n zSHQdp8j4I!gsMzRVNgS$=|P`&LZ8+{C4y+DdBK%C$a{}!)(E7wG*FMKnjo#n%=}!K z_H=?X|0`1oH$d~RQq{Y1`oaUC9i>DZ47#&! zbAo{RJ$2AzM0qf2m|bR8A@SN_rYh0;0BSMR6hDtvQ@@zbyZh{?G|EEopn7^SX%3Y< zcJZb&7gT{mu*SIJ*r|yM{gXYjcbGSO0&u4DXK?ih-4@ck>t0bVT(O(3^`@RhH=yj! zBS%FMQb*2>)RF?)~Rw@=W#Psqg-hnE=jZqd~mD4+kdEfoGRR?Ym2@&*eUDigfh42;1t=_{;e^kKyefGWVVwd3W)n=VF+o-%RBV<;?qR>3+p1Vd0k6D=K zvVH~bf`OaauJ2`Dc09{Nd){@thYk<{1|XG)j0Y@@o9rVgZtHfRO1zIH?50wrIlZwl7S!b`YlIfw*IG(J*qDytnOS{oENxx^XAfq z7{GSpwQzD9Lxgg7hLGR3iRa5L-lDoeaygMui+bCy?Ro+{t;bH^2acdPj!Jq zW@gDAC0ib`9azsUcK*VJLVT9S@qRY{&1O5R(l|W-r~~WbmyL|v_&DI~v(YpqO18le zi8WEvpO!N?si@52VEa&>X=kI;AM2I3=S%iZM{0(hywRs}+e4xpCAEx#0waQaV(T9c zTnRNeD(xH{7uy}l*w3+n&vPI7xr*gJ>_q#Dlu|?GB?x|I_`S5g`Go>_1dqs|7 zn7970)Nl2I)U^x7Gsac%m#X?X#l^*!zyO#A=__JxyV*|wGxMB__|Vq27~iu1!C)M6 zQ{cmU6%)GG^;%wAknN3^Lk#&{QX=Y}Sl`WSV~$FRyEi>!xmD@v_3txdOuu_R3i%HS za-5RV47ThhF^8|nuR5)*@15w)Kd+sB|ezl1fDpI5x7;vY^M^@Zz%qXp>D6X%s zpUmwV%&tc73gy@2;CXdFm+rFos;$hq9+HX(a4hp{&REcn5u0O-xMVg3Egr5x)_Pb|LK0P!$oC9z) z15GRkj*Z2txv(P1dD{q3r>FZ^ks+qQgd-U97iD=~^kVLV)s=-J{Yf{CPaUv$Wyh%DAN6Xa z*Vp52WF-3@6`>yUcDj}yqTASGnX(puLS-7cIUQ;$961BOosSpKyRn%zeVevfYbBep zLFJ>I-!yO7m7GsdCar3ZtuYFyk(}PhnMifn*4DOZa)cwNAt;2mjZC(Ya|1JQ}h zREL`zH`{uLv1f)z?iEQ&cYZ{3hR2&`(IS0qHxGNNR=k(nKJTi!{Sr6V7tUI1)>^(_ zHd%6gzWPpV%E=z8_191B4`LkKuaF+d$bF<{!=}adBrykd-R%d9^1sZ72rO&Jab{<) zh%6-?jgUeXtMbRue(OqlH|)m1c3_w4wp)vjZ|ncDvN|VDm7M&tEcC?r4P%-0(YXs^ zZ$dTF6)aP8f78D?JnS(}zv@+daG+gFDMhK81)_HE2_JcNS$XLomCSeS2E*fwbXKd~tdB_*wgr{_j`dYxvYu56T* zL?S`X5(VKx_$H(<7;M2kK%?M)B4)jlpZ_k3A_T;R=tpb*N)Jo_uEzQw8{R_wG;NtO zYBImNP^k2Ye6{#ssEa#M&z%r{8NRF8E%wB&J>oi)^OqS$Pf5}bG~G8adtzE>=WFy+ zc~pj4J$l4qP4+}m)cq?tbOI&aBEQrIZ!I<|c}ee6@uC8 zZOJ)c+5M#*EzSn>O^m;*1diS?bN}gTeBx<=1Rd9?G-r>&rzEz_{u6Tc3|8`=E~GSN z@}*_(6rJAb)uLmXdRf;yk5&C`=)GJ=VHU%C&xiW4%&%Nhc|azIpKXy1qn%H-8^Ukbu6lY5sxT>p^Z)$f6F+jK@wfX%>D!|< z;Z=Q6B1-yYJ2}`~_Sb)JcPmV5pB3#&^_=4mHaqd#MAfL{_A2IEQ>Cl;AKu+ioztEv z$!5Hx=fUT}1I&{O6=j!2v+9q(jd0j0)@L#=fPkLWI&pF;%e5@s;n)%Nm0MR|H~s;u zNjqp@o)>LLX@^9;U3ZwRfzHE52^ipxaIlFU8<2y`ObzyLZ9AysRk|QXL??Bp7Kr81RgT* zhmw+@z`!DqM$)pf9OB~PD0WC80!rr0TbsevYu7SJOZ}e%JeTkfPdu&fH+3vnthvI^ z1iT!`NlX}|0nGi`O$PmKDEK#OxKo`M@|<6xS@rpuWMT3XlODduk6uTAe7$Xs#&%e@ zeyT6teOfFt`sEO3dBx$_&kq8kADIhI|LFWi@2gUIzoYAw`LJ*3GwU{Uxl5m3Hm!0C z)!l-oy60|2m#9i)6l)HQu(4R1{T^x!10($=V6|R*?j9v)I(h~MZ$CdSSVKS_+~Vuz zu$Nmd?e+8O`qh^ry#mL00~hykUZ1r$I@Mh5ll`neqc~Z;T51>9d*bBe z#2mveSA&0p=FkYUDUcWFGEXpp{w$)9kEmfAIC=W?*JSm>P-eFraX1JHoSWFl@|}v5 zfHrpT-W@JtaSIYE{z4JLJI-_zKru1X0CJC_AF>(+q-19Bz5k0TBt8F`w#1%P?4|k< zsaB+-pn%5~ACDQ}gdb~ENZS>EzfQyb9@h~c1B0nY0ewF#m$_6Vnf9nCO)vWRRs`*e z>>v13p4j9Spew>PvDL%z;DZl`60CpLbL%P^U+Wa22KY$S6VmY4UY?_pEplFHZ7i=1 zPWLSpOPa!DR@0t(6&X>rS9i=7vfE!~tdxB_)mz+Tx)^gM!0fl-p<7?Rd`aMyg!mz^ zrL`Sbv8X}i%K?R7lNw`M6^GrT2}%DvV}H#0TsEK4#fdU!i@1>aXjZp#@h zyzO{V;l*f|Z)VF>{8}@Ox@8G`2W=Mh3Vtu+bLdu74|?< ztn}8zRnytd>2K(egD|3lS(z;oI6|Nrz3R&4HviB&F>`itl zBD;)~Ei+{AolW-Md+)vf&*QqTKL79Uc00SBor9F)c)gzE@wh)whbdSYFey(@&xL^p zY(GBS?o)j3_?}0%&bsSFUv*_LwkX(LlyER?61yo|CaE@3it%F0VeKtc`%oPRicKyb?~=;_YOYI9qRDMd)c;^0B3OMXFuNOzt-i}=Ku ztn-!OS{M;!JB_EOC-f{yOG}HWs5tK}*H0I%kijv0D>Jhuyu$RE`cra8!3{G<3%AUM zac-khLeAgorgR78#uuj@v3))E!7*_+`un~$@CS6nWX_eI{4g_5b`HoCj=CvOAHw+t zPju=*=%stYX^mYnFBdPJ(7JDth0z!dY|W-v+@y?8Rx}vnmE3i9dZ9}*f4}bI!t@G&`^Qg-7FdTnm*OMY9fw|leaP{V=^}hm{@+Rah3$ck5d_-w) zcOb~L8Z93KyYpGpC2#L#?B(QVsWLMCNuE1;jZ^}~KVQr14pzm&{JJZ9BFqx4OPM_H z^uiK(p3db+F4Y{{YAGm)LdOQG6=atWCvmRn!2Ru=9auFsfKC)N4jfjKA_%Gkp1R$# zv+#%rC1~=X4S~%SB5H*XQPj~1fF&RZb@DAIVnFr&5ex{#)YRu?ZvStgeCk2>Z#7AR zBG&?yP2%32)(0@?1U(MJ(@4)R<6&9n#2ih3d$pK{VlOZSF*eGX4r7(vBBx80^0!xm zbWL!2RljX3DB#%nF(ldqvHm6`Pcziov^VojQNsD_%+}59fZkbi;9O098pL)SBIiEC zr&0OQc*8p#UGk@&LH)fg`#{?2lITpq#A!9=XKzRO&9H12T%W=!1umG1>7$gNKSO@} zfDYOG*FoZ*0lvHmbGR3A=fNQwwWa@mEPVQ3oOK6a;|2;{wynC#&^+9$)W=2IAHZ5Oimph;iAeXDFQ|5JaLVQ_)1NQR(5rGc8!5!EIB%I4 zhn@dR_M2d35~yy+hn-x1->ISDbG@KPc1%dotD@3BS(O~WKKO1l86Whzi+awusLbD2 z?K`OLlv0L|`yDnU9XBfU+2dTdFc<9>8+hqkQ>6mq`UdFg|8V^vCiN+`WwbfjmtLc& z%+61gQixy`J2+n1CuX`Qrjw6zHd>N@JV)AQv_sNSaThc@=?W)h%E(6R+#QH&fP{we zD{MK;hH^h5QZ(}s9?*|iJ2*5nH~SJ$@v}qGi+Bt99af!&V75@OrvY0z=}`Hy8G9#>l(EiR{37GS~kJoC4cpLISg<5bNJOI?n`Q3RbzaTKcl9Pl@0}pYFatk)=O$jzvS;zN-HjD z4m;uJ6)f~o){Sx8jDA&JceU+XRHE`t+r=B~xF}eFFljz1@|2-$y6~e$bEK8&51YPG z5c4NAwyPDsYevfJKSv`a1Y#8)E=MEZltb|FW|`Dalnl%%%3!oixpX?}d{T|T2*;zJ zI4@kt-+q|C%te1uv%f(3_DXZ&Wt&#)%pm!4f_c1I)u17&l;q0KqbB6@?Q87*ZEqHf ztS+0(lP+Sf=Wg&W-3{Jd9InDG&`>9s_&91t8R}@Mh5i_Bt1Ng%5z!*J$3PVO6;ZPc zh8^l1$p{t+uKBx)&dv4htl;8=-TkXXcJawAt};Q|@W6z#XQ5@P|U*mPo31u)|bSN!W(VFZ2t5#`J1By7TU&LW%hP{99O$!eCjVvRB1?wzZM)A z9}0H00WK6`LV^3$qwA>Z&fUAdh(=mo-t;Hdu5)Te-S%IB#n&B@XGgD-JW-~bG%L&= z$|T%E)Mt4e!^aA0S>gwiUiegMstL&{i4^RpR$_VDgH_+&c#Izvfe!OmmfvG8E_4({ zC;Uu`TOSX2#ZIf|;U=b6P{4FgHO^%kH)*H!WT5Yb`zps3J$GES;<*~VpGR*=@XuKk zV2^RG0Z-i@SfkZKujLyQB-O(;GLWi}SPZ5yWd95AHmt2Agfn2Eh6p!7qmly;Ei4VA zl7XCn*eU?6K@2h~I?%uRogiAqe}5h7{S$WlWJYl{_(0Z1?wN^%l+?(%UKdM}+G>}wjt$Z@jre9OhqZh<&86DGLy|U zDerO!c;-#8Mfn*{+P=e{lU?ACbpX_)*SM2Qy>ntvD@ZT(>0lDbMvT~AszM|bL z=Aw=;gzXYM-8_nZ`Av}GY;}kU#7y|}6amt&)hbMmj+izoKcCM8W%gmASOqBT4=hZ> zJm2(?A1kVSl@}6SA0|oD5u3Y=g3Q@?kHuSMlO8W$P9uk7NpbP%xv0lu14b3x_xjm! zah{;s%>{Wyq2+{}xq0SV20UwsPzIK_h>}uUTiZ;p>cIBx>L3S?<#-z`Ii#OHMZ|LO zI|1c1EN+nl5pnW?!VQ#5ez20yW%>V}jF)O+{?~Gms-leB+`m7DE(>O7$slAwy;5F% z%qGy?GBX|4>w8C-4lP8tF8D828cBOw&UBdFb{W}clO(w`+Ri(8o|>|EQ!)r9jd1AO zG6npMrk~e1{^4@Dq>=lVvEn7;wHUAFv_QIUQ>MUDZ<5KSu(B>0g8c18OM%KHp=jry zM3#;7hYt&WQ)#KYyw{Al?kDS){Sb|ojcsGPyC#_^-;b{8MNuOCBhRcl7pwi@-dIt- zhiG-b6KT0GwHKTg82>!!I?%+EM*f~#DKcPLF=lPDPSjbJ*k?pn^aIN~&lfxW+MRtK zyJ#WIh`fxHFn$&nZ`z{-UL-A<2e>5w!}7 zBG^R_7yP7BNo7$XvyDyEad}bLQuB-7VSG?(O}D){m0ISQSaxEu(sSnylVydW z;Dr9zZP9ydQe)+0ZANwnqXwCeQTkjj-_GFE)6mRv7??Gqdlozp-ILR8^E6Zm7$@6qT*vHDnsd!q2Te}N+X!w zaq#eJ2L?$0y9B#GVhM7{lY@-~Q1=;LoS%UV1^FOrYYy17(J?T1gB}f$d&9b63*_F7 zjSFm1|L-oi_QK=;e~(YbR%-U?vm?eF5Pd0de>AbhZ6D>2k606)?f5l}EfVQMNNLxv zGEUB%(o%G7jT1|wJ4CV2muI{V4fl(p*pnfna!Ro`Q(dIbl!n632;97As%2j5yp7c; zi7>Lzxtk_m*O-PWckEs5rdcnitWq<Qmt#~D3y0K<5>o6v$B^io^8&x7F_whXq<5~_;+jVme>ufAlvHJj$s9v zm?y^^mj_1Q;D>MMn%CBg-Cj$rwf_=r=8Lglre~B;u1=Sb7Xz)C({VG?jWE*<2k}zc z!z#CLzRNaLz@h@b!s*#sQRX2lSWv<8bi69g0t>G9xsp5olo#W$FyobejUuehmCT=~uuOfssh<>lbp_Y& zJuG)4+nb8Uuq)ElSWt}el+CTKTXE9sNMv0m}>`up*V z;rBOlWL|O);@&XdHKefSz0}=Jhoz!4`2FB+#ryU8c76Q6Mk;HK9eklrl&Eu)8tZNi z_FHP!6HwZ%FYXaI34FxRR|t5eypBe%>AWk&Pl5S2eOO+!-P##@tkEHR3~FKOXEN~Yn> z`Z=I?;LY`)>AMx{)0tSI`&#T2+>grd(MEn!htOYy9kq z%+sgk(|9=gI+=KJKU3(^m`ky#jd9K~mJ=ivElM!Cd2VRtqvYlDf3Yy#6(xW8d#ae* zG*sfync44_m^ZR}fLpt}94$hR#nn9Eu} zD$Biib#&tjznZ2C0XkP;1cVANh&-bDbuU(4t$uo{)1P0K40^5kbwTs9WR=xF?YFMx zDa!uNu%ED#;IvJDNs*hB5-?yj;(+=VcEZGK)YEmrK>3Okd?eDPekz?uM@RW)L*4u8 zGVauHO4`K}uLUXvR-tke3vO2*9hi5p+v$+fmJ~kDXovaOiFv-DCC~S+@3soI?&72z zRCBW&rq$xPBe?KXHgR zV!0$EIbbMs^N|&FsWmn4K*2r%cuiJT%->kcZhCtsb6G<#0-!dkDN^x z;g_j?`yDwoFO}JMdz({*z=EqQw<2(y4j*(zLri(=6m-NfR#>#W&*JIj#d1XK*eY19 zCx47`g>}1zo%S#X%5}c}iqYs^aB8$EO#74p4&&l9v`72Qpr_gKBSt}cEdBCjf_+v= zNz;HHp}TwGLKuyQc%ZMS()IjNZ96z8xw%t6lO}%WsB`#H>MnSANXu;*m}`73w5X_n z$n!e|Q9nI;OUti}qngrMj~aA1Ws^Yd#*B_y?$3O#nmsK$@U3w169ws-Prk6Qad(5JA zz9)Q>VVwwMS^Bh1ZLVdnrx*)5vRh3*1-a*m{8kiwcge9toaZdsOjxO0EfGsF5%;(1 z8~3?v3&P7gvY*u|EMOTcfB$T}bPdDPTT*pS_YsvC6Yu=hDZase_tIrl?yN|v*1*pD z3zxJ9q(n>H*I!4I>a#AnTSy#)F)=gey8Ht8OoBo03|oBIy3~ZJhFD^9!k>wttT|=+ zgMFBc`j(Np6|sp9Ts`&o^LshaX#R$0RRHRPt^$v;_|NGxsVU=P!umy--JVfp*L$7w zr|T9=-9CnHUbQJBeaeSH?Pb5~53#O{qb^}%+k%f4?AAwRw!#sp zt0W|b(Pd~B?rrWPJ{T+Q#c!X$5n8->dbZN`>L=6aS-E{B$sKYE4|bwKZKG=Z{v!}N z{$OM@*KZq|bTbcYNTzw}g#83QY`3b$eeaEgZ^Kk31IvvSC+~^%rpf}Nk)wr?f)?>S z)dx}s>No9+=~}=DUA5YkWH1kvBzk>*Y41uN#$O&`H_xC&@*V9}kz2uJzt! zzZu*iKlxkzxzI7rnA?HI2`az#@mD@uM0!9i^kcp%kL^IOd8Zr?&m%nfO@tXSE~@r| z7xuOYxZ%H5Xk!o!BW%QAaS4x|wRH|d$w;B5fNEAIyO#B(7-qygocab6#@uf3RI7#y(ZNEpTkv8?B8=r zjJi)!eazd*6}IcA)2q?4Naj?OSs|Ae*KOw=PJL!>cqDK*juLTyMpSSA#%}k=ry;ZC z?nrrE74o{WDjbo35*!K6nWawugwih`9`5C0?%d<$K|0p0&C%gDQi3Io5 z+Eu^L5{RtShD#xjN2fiAZ9Nc=+veFd7S-rK|Vy1LATu_W@#o?yM z5E<^uK8EI7N~XOg1-5t+3sC}cFXveWyEAvR6@^qj;rAwzc%=l)UqVJHo*W7Y2taS@ zEHOe03Ty5CqeChq8@aorcQ8?u!!PGMRXM-t6=GeOGILkD=G@poeS0^P>2`8>7#ySc zwBs5UWi&k(iI7Kp_@pX77m4@SvWTJKPjh<{ctClx`@7-LWSn6g-KJsWaW$Q7@VQ@k zK3>otc7ZFvdZnZujnom>s_0Itb{cq8^m)FsxhGUo+t*>i;4{JXFrIreu%N9f^Mi1l z#qcEyi>9+<4Yl1OGZ}k(Zg4s~p~yxGZ&mD*&(7T{up~_PD|O{l%utQ|>=)b+IHzs$ z(P8?M=3{heqSEFsYu<_Frw>ZY)*qX6CA@H*aQ}RsS@gU7h+$inZOUz5J)%l~?`)1s z;v>9;$?+)Y{&@z8whbyu6x47SF2O&my_r5Fsk% z`uaKE6IGQkX#A0n%F3FD^Ae0#J|I7~ovcRt5giRBX(Kpaz|HgWvvcpWO^IDLtMrGVS3C^WPp5KM52SqpPrq zU_I0`z01gBOCo{&e#ma#ZP+o8{`<2lZy;sf&**kGPY8R_`fM+K=db}Slo?=*xpC=Z zSz@ZugL?;Ebes1_m@n{@?tiqu3)84g#~iFu+jN2#il(^sD}iNV!RBT#JFUAjfg<-y zI9(F_)Hiy^SYo8J;4MLO#z~2QXVez1Nx#}n$sg#-ELFN*S?y0*sNDv0-f{wPAUQw( zgSl-&7uRK*W9+7}6qxo4jC_1=*g-pMYV&4lT@Ji+<8ms-N2VxTzEf&;wfOn-Pny54 z)fJG@3TS3WA5%NoiP)HWIZu<=ja8XCnYR#3DIlFqdMlRSUvL~^6we+QIVU_Gb>KXx zu(1}Fa?lv8Z2P0b_HbtU{)=Bdkrz){=kC}WvtGzP-(P_UObpHy)q4oUWI5sKNOqDe3K&nEvwZ^4(pSK+W~3bqVg@JUmXI;zN&# zk$&B}ok78S`{?MF{YIc+Gm+mFtQdgx-k=hCYHZ9TRdq_Vv1yf0C16H^(QCg_Jw(RJ z`DL)seCTMs46%jxn`7>;=3rgWI5$~543E%#?AXdGZ9Y8Mkm7oDr!*y2Qm2UEY^MVw z&MeZI`#|lPPFxC1CbNrz)M>Zha zf(lkil{~ZR#VmFn-5%*CB|#%h}Tcz&T6?5nl+b+;0mBQ9tVe%{)lUK`mB; z@LIqijqro8>grXr;VC{3xGs@%R+XM!_02dlojE;S*II9(Dc7}cgX5J(JbO9Nc35w2Ee8b_OuwS{Dr#&r>owXG-sDiz) zN5|W&`CHTte|+n3@Gl#JOc%GUgcQ6kEa@QzcQTe%9XXds%3v@13C#YW=N)v>XDoPv zdu{DI$K3pa8L-GWzo=GXNL6673jP{^X+w%!=AVwV0-epOe#MrqX%eo3#gB;=jU*jC zK<*eF9sN99Fs+mNW~b9W#%Ffu>Q{xlO3`+`pcjX@!FEj8jkkQpTZYvGMl5!cVONNZ zttLZ<9@-t@tkw5F^qzP>+t=RjL*z)<&_p;`ZiJAAVNK#?@~RKJj?DGUBlr=20_9Dg z48EVW|j zc{Os`eM6I22Ak!B>zB`@9yrZE4ZK0RGbKgC;W5Gk@|KqFo~`CK zarcqZHLt?ggqkWO!%IE%SEyQYFU})?&bf(%Nb=HuC^#7Z?0PWfV4wiLWKcgQ?Vx+L zMCc`yEl@MDNeQ@-?>hGqGfHdc&8IJnE1Q^@M4ggPW7ti_#+8;vA!=`+GyDecC0Jeq z!HZ|c1H-g8L%Darsx}3jlxYxhG{V@dLN^Fz(`KPj>FM+g48!QBU{?b7R!cvxND#H) zO?G`049gP$006b|;Bh7UB0WqR)KW7t=Ua07)ZO54|I(SSAYb>#M*dS|DXWOm$W3w75H7THNzT4u)cxGhW1Em_;FD<+1 zk@27KTmkw(uP^fd78(-_|JK1YC~hNk5$Hw1B^uB@00uig2O8!u3R~%W6EdV1z7JO@ z_P2!;%l%;q0W!3-xe**tLwfxUgTZmx&O#T*A%Ewuj5h?=S(3D>Gb!7fswd-S)SKmR za@IOI9zB)ZtMba9wgV6gqa8tKmJ7Q=w1M#qjltz}j3kxS+P+Dt4+GCd68ifG#%*lMiEUmS$~L`A3~|Mdqx|Nb z)$=IWE>7e5z8;~4d8fS-VUZX1yV#|^jDd~6XQRw(m#biV`!l?mtt_r=a{nje-Bopq zm%9~mAEY$AWLK&=Ir%3?kJU^wrun>FZR-J_ zb>i_20HV}^VGQ2$oj*DdBg7xfX4JR6``W@x%jjjN6ZUD;&_&y|V%jar6uvNe>egX{xBL_#<{pZPZvvIIhyH1+uvGdI8q05EGG42=! zAu29Q+qbt;P;sQtSSl#N(5Snm&TG~W5833lwLETZNl;eVsycTDElQk}ot!v+z&0xz?T`PcZ~)t|MFgU7=eBsJpPD$1n{LI@MfAd8b?VY z*dhX|Y0##Ez4f`a_EWIg00;uELipoJ%g(NP#eQZ31sZtmu;O5dQ^*<*;0*A}{-zoL z%NFz)tA8`IvloDf;9w5AItO2LQgGenomH{47MGR|GaK>o^AEq@gP|)Z`9@B})YQ~Q z9(F8s)m2U_L$A#?j4q?1vV1?aMOjPBG~%n(i#vd*VTMr>YB1OEbEf71?0-|!(nLe) z%gnX@wl@Pjbl9k7@0}xj$Ix4;G4?H zrnQN$j~HiqyTqRxu63U_7mDXOs7gvoPJf?MeKmPq`gE`6)sy(aQ1M|4;*%iPn#6;@ zw*^_FYinOoyRK5JnjaB)iOP%XTb8}@QX&yZ<00*?S#Y)sMN3qa(5%1z13%_NsXfLo zv>Hxssk9>H$vE4@pHd!Ln_EFxI0fZ>so!D^iW#UVT{}!Ka5If5LLJ%YtKRj$^326p zJU+b=8Urr4+}zy3?K&=o_Cb-I&WGV`>t5+QfAp>5q=KmP_x~<*mA!Owu||WG0Y;&9 zb_s1{ANq`sj2=j6}RKvBe5O5Q(Zw`p~(;5BiyRFPm*fGB_y!|9BC8s_At6KWuNvc zo9tGNzni?l%c-wV0L+;rg9YPpryU9YzM!Op&JdCb`mF{Jm+kFeyo>#Lg5aw%s`n$Y zTOa*qbZ)-X#rs~TwQZEM)wlQ3<@S+9oPghY6`HP+hBF6}uc$`P-g8>S^lzPQ7a-E9 zFM19Qlc=GgA#mC1Zn?I$wr*{2=L)I{on9RiJfO8-9qioS8|bGkPo!(fHEhx;rInCO zAh{j~8mX)-R|*PQ8xv+d$l5sk%$93C#_y+^^@okY`z>_zN?>suI2aYvSok{%4GE0V zo+~QWx?x~-p01aL)Yb+kB#>v7*xXT8qm%z`WU3UwCWS0dRqo~Q+R+OO|5pF~=!pIp zSk{a8vbDF$zjtqB7MijopnGNF1`CyC>*Cv<-WMM4@O%gPilN%fI@KOM7j1I}G z0oNgzIr{ql=2!)CNS8)n%M7DCBV;sx>Gf zCm!+e@eS>}CnqEzKGBiAGcckJ3d+IEdhvK+-X)Nw)xpD3XMNBXai#%d1W+NK+|9?@ zik|f^;}a5UdCqN{3Bdz{Dx5sC+}hn0&mI6MK>)z%EA9i|M_E-BI2KZ`UeW8HIX-~# zFp@(9>^tp3y?-KxbF%+c%Ej#LKhsSTu`;4w_URcLr>1Z zv#!v%YFaSTOUbD`l5^($v5ApL{JqH12n_sRDe~@S$|pu2NK@KnMfUN_g z&>tzz%^-+Rh)d{fF{q{~6xgvnJfxEn`ly;u!hM9f zTc;#a=$^_4M*OUmcNm43g%TYSg2j|e61313OaOdA5(&h_55j*yW@ zK2c{Fr34%3{e8*+1uIJk6O6DKQ6k0uA(z5*y8SbRn(m7DUVf$hV9uzxMf&3n1uSj1 z^tbCJMUjw8|E}e_YSwCv;KA6FA~~eT@c1A1YlxxcTYt?hEoUFYWK zbyc>dU}B7x`F6No55na9N34~pEqv3VGFhZTgE^60*wauD$T!*Xr zqPGCD4J1@9E32HfGh4tfeAHeEX=!FM1|H%sUc|F1Olpf<902bz6yD?>;O`(LeK1NR zOd?>g2q477Fc0bOw0}43zXc+iOlZ_EbP**W2A;H&%#b8&r?-F1-(8fqc!nOxPk8#{ zc52R78?e`nVXRk+I!L7SY@PYZGiC8V`)&KW_Ho@7{(xA#Z<11uKU~MH5t+A{m_EU@ z$vNuwF@o+Wm-wVy(t^+Yfjdozx*@$tN4Aa&@;zrfjjZlPSrC0LC4t;e`L z_c%m*Q;ka6v`WpEI@%QLT#tcSL;w5hXAIlGa*ij$yNj-mkm$A}{Bl^suL1F{nQ~p!T ziz2B`x=EEQxx{-luuO;vlxx_ypPa*Cdzkoqco6aFlM-F zg|u95v~6UgGN|723d%bjBlWK3I~NL}ixVU!&%eBmU)d5w zJpmoFLU#OIwiv7Mhx^mGUz5JUUelC!-s{8xz6_7-ay#dQ+GES*H$$Qwow)szF7Dw^ zR*g8!`r;mW!;LGVESh}Gb?>vL~*I{vtCp*+xto*}@2frN z4_&kTmwz(Fx_JXXub&mbCQ^8K=!w@1YtT(MFPa%eQxbKaUoA_Kli+>(V7bywT1ZHG zZTQ8}Bt{Gkx8FR3g4^`7z<9;V-d;jhTIAW2+P&R3<1Pm}<$!7e3h?tHGtrSE!n4b4 zgAH+WbMaJs@7LGMGJ|AgyX0cT?=k&W#d(lRegwDCCOv#$#R7wO zc*_u^CKQE8-KCsPSS3%^Bur&w}Jc834I|1-h_ClNZ|ahFgw)yKVd#i(pe8N3RU)C z5gEXjEnf|Z5o<~x4N5V<84qmdVD6*8&YGAw5l}3kt`-$RktdTeD3RY`fAbaN_4cQ# zb7-%cMPy^gAI3f-LNRgOc^Rw&6ZBgaI%qe!jIYLpXjbCOHyR;5)#TH5g(u5BpLVGO zz06o^v@O%(@`b+e%iir0GU$vhdO4A*oFi~x@WUQ|po|E3Es)A~q92%clY5DbzGGeb z-2g2gX5ydmz)`bDp&nv)&{g#2veE;uE1wO6X0vW@i^1aU*vB%?f1^HGQD<5a zr)`;9;jTMM>}OAc@{SERhA7bFKx|@qnI>?a6d_c!KNhwPW6i4|!F>y&F|h;9{i~Bu}$9 zEqf6W_VucrZ&=#}jrOu*`sRhte~3KKfk7-G0fsG1pydFUS_&MiP&?Ju)nNe26OL_M zz(NBg^=ZtbZ@|^l$@N3t4}h8h3JzADP>Im}cWa9LLsJtl!|{=id-MpHt%<f2Sg!wpn*;hxVUXlpdkSjk(mL;pKBsRR@RMZ7IlwA2 z4JXTM&g83|yt-1k&^Ld3+ms0Wv0i00QN|#rqP(jcDEFX~b<*Kg+EgsZXQOH3ttYPj z6`lra_Ei|@=t#AhsacTA6WPjPx$?fU?!-6>vpIvJC0?MqhNfiCz`+q_%FerjH(u}^ z-#nJH6%qO_9(PxLKM>uksK6QJsTgFlYfbM(T6+Jr*FhAS;lXpeGu{zr7260}UJC3` zf7MdL=1lzFq(qgJIMbagEnQ2C{BEy=eV4y5HRmy<5+?VNWX&FDRy?=uzRu(RI>}y2 zkZz+8YocB9@bpNt^a@Ys#j&@(8yKOYTc3??{o-uU3<%p2pGA5cw@F)FrgD?yHp_b+nYq1YTJY%X+lWNX1?LiT z>%6+heMHa9$%Ce)Mgq_(ep~N^S4$a^oX9XqUqmohu3l zYhrN9w;W7(b$vS54z1=>QAsr&cwZ(^icMoytn2zHoW{L+bKJ#hz3#A-|0|)e_{IK6 z;Y=^9Rwo?h)SOr)j+SLveRpKzvI1Sd_yW&Md+YQ9UmVlZL-<(_1e@Fpkc62VZ0_g) zpbUG{9FGfkWMywaV30QhxY{Jp^^u;*sj2k!zXoO;3=j&>VPho) ze53zPhM~(wLK|QWf+(**um%{ig^VXRAfMr|NC42p(E5WB=(LL9oaUjGCy?boGqYB~ zwEs(NY|!V=t(NDkR(FAnWB`yN#4yb}?&Jj+8Qv)-fPMaF#(tRhuiaLTOUNNY!LQi0 z@_m26(d3Bdz0QbjuGu{**KF)xGH53|RrIyC?3Hq?XunhUlD?Y%5q0<(eZYi*VOwGZ z6^>0np$ZO){Vq2cVdLK{!?()$=X){C366xfVoN)_eq-SIt{o_O5w_F3&Dsi8b!u+lPj(v&r z3rEkKt02L)Y4z#)%OYm6!vb$fuz(v*?iny;vu<7)NU}Z7$A;ZVNkW=#I*WrwygjBz znXWTdP?4@!PBnIrGI3u|btf+VKtZJb8L#BJ>t8a1=ms^mfvG9RYh4mS@^Mdh3h`?S zN=*VRbu)f~O#n*@myr}V@7hy_PpdC!K4=M6QC~bCx__wmHDi0v)V}SsTS%Soxx%ll zy^MnVWkqb+l4aE4(Sc}dhm=kQIVB}I`k7~`Dd}>JS3e1iUM~8z7GqtB9tkSU|CKlx zG)T=*jgRItct%<5xxJJ!IrZY+0%6?T{7di@T+wqyTVKa-{2LpzUNAr|r>fF>M>hC) zSCpE&rP3hr+S;rPOJHCzpSHHl)CJwM9&z+);mGv#Qve_%xETNsTW?N^#@+f>2M#G1 zW3xk600Qp?4B}@%gBt4fzz`cKM0G%lfJPJ>rj!y2*UN+;=K^qJUXhUP2rD+J?19f4 z#88Xx0A&+-K=mL|7Ww?}1S3(e=VNtflYj(S0BQk?R^e(o`uek(mtlb0)t31slN{x#kvXT{Y@&TdNc?S3`C=6IcGXMi zMWw<j9E{`$Z@{(nz{qarzk4Q7SQbBab8VklcarA;-qx*dHK089GdCCUk zz36qDGjT;&!Gc-8uegF(4Fv(M_uxdSQ_jvYlcO5*Az$Rlzjau-{l!^Ra)r#LDsk*+ zrIO>pmJB2F6Rv&_9r+H^E5$Cdv&m`sD??TnrWst7un+o9W6MA%1LgJ4F==dvOr8hx zLMMG1!fvWTiJ!%*R>;wZ-jXr53Z>7Clm1|f^2#CTcUO>CPYSd!pqk`c9L$*^X{V_k zDT{ynQC0oZ_VEgsR&NK98JMaW7%1>shzpyBXFk&?zj=m1=$VpIfLFa%l(;_PbA$Bt z*n>&*7Acx{x&+Fc7Lj7>mH55(8#2ufTw_ zyE;UStT0SmA&#YNKw07Xf6)VBT0vpaGpm!l*4A7gBfiOm^ z4+TuzR)1AEKNsOB2ieHfj-Kn*f3j#H6~lg||Cdf#ICP{u(>NDDfTSJ#^{Y5YzAyGG zmR6RRBSGZ_yzv)62lA&92!|t^6T-9sd3+EN#CZRHAXS+IvEw0oR5B8W0dpEzzyO=~ zp|X0BP$Gc#iGv;DyVak6O0QFm|67Z=DxgYe!3Kh9*za{MJ05Bn_~?lJQCS_E4iTOn)HINr0zlFYOIp^UoElu9K^Cg3Ae} z)GUWlft9=o!dm>o1+MpKyM}GqiTaa#K}`eY&()sQ7g{5Hux7R5KBSe!vJ1t4#Vj!D zR|(to_|9)X^hr>G>e`LEEleHc-4nJPH_OTAmCLE<$++F_+kF2fy{=HW(5plf-?wnf zXA~W)BhQ@16RY(8j%Y4cf5DIX=ife64s?6tq4~a;(jY(1ywI?(t? z1cRCtsgTh`3M@0``rLb1XPpLCx{4y=;(tim2fDgKfa_yB+jI>Ds;tV`g@ z=&2{24>2>?`u`J4z~uqkDiLVyzJ@1&PHUUWk@jHawTmvG677#{z(|97E!&)~RY>c& zgf1kMy3!v{!Hw6_@;u_c1g|}__U}^Jw;O`_%O-Zj474A;H7R7DunV7Yd1GF#?#Y`P zl=vKsO<*>NC(?r~ENrB4KoB-{8alVPrw&lBx<(oxQp=?zs%7h5_)Tzo^jFO}E>~%Q z$P7jch@l(y_Rvep0S)xO?@J^&;3We&Sx%XI`a zj>MKCuM~V8km2p{B>?+B(WQVun+1LujOPJuz69tU=pJCHvITg_O~@A1vCcg%ne}_q zlgvH$`~T{wFaPQ&cJ{tO64ZT_(zBokH?M%6@gs-NmpQG)Nf!*=!2YnQ8+}c{xH)|2 z-P29i;2;NAib;EY{-%G8sDo%<8???fb+BPOs-zjmL^RQ{wbfwgq zwsnwwmr$yVXCmJzVEiEW)fr>@Uqw;ZuAo1c7s zW7n@%8S{car5_!x?~`Rk2bQlTiL9uHdChd%pQ4(}Qr6Tn8K}5Ds6uU-Y!)-6W1`JF z()ffEfU{PTFOGN9!n`#yNcEe4*3P5g%zMfOxfrP*zlWS+zw_c&{}ppiFysM1XI3$r+za&dI86Z4npv zm-oDD%EGtAC$QxH;voKFyLJEGMiDE=wILNLiJXR|>mMUms1TC|v=1Y%A0`THjMewk zJVQQ=JC=f@uTG~!AZL2US#oUVQ{50|Svnp?C8y1C(np^O>w-I&KV;PSiii9iJ=;7f z*zTKO>@4i{Dx3OOj^Ij>!h4ONP1@I{&C%UuVP96j$EOaNxnsLafWrZ6^r$wddpkQ~ z%(ddP*GF-Ida8U~)8;{F#yuyO#I~3K*Aa7R1qFw*d#w@BQLKAY1Se!RQ)r5!wX6;; zPPmblmzy=Fn-1JAFn-s2yjn;qNf$n#hy59IY$m7<^WKlJq6tY}bMhaVhgT^npMnDT zNFZ9a5;pK}@`XI^4YCoT&dWhJig54YooWE<8f35BVPgw|ZzTwAd_R4ng%uj?kAb=% z4#JrKuKT7J7JTGV009jks21Qm4OO}cfXovlfDmf+>Lirj7l5{iCj)u{cr;KjZ9;I~ zuxyBpja9D!RCwo~gv3Np)t4;qz={&7zJ_uPe9Y^sii&u#yoi0KLsPR^A=B+W$e7`C zy%J#K;`$0dL0&71vb93V?~O=e0U(c`LF*k75(3FBa0&haN*?Sx=YUr}1DUteU@$?J zr{Jyw&Kk&^=YgY!1Qx)+wi^thk`%<%kYwNlkzU9#08oE@Ab7(C4Yfsfo=G3GTr__@ zysB+rS4EP;H@CMD7V9I3u6?Fi@dnU>NP;m;j4?JR#d;Qd)98$uL4LikH=wt0LUZ0K zWh-pd%UzdUJOL92?!MrEZ?f^F|J|t4p|8}?L{$1DWsxANWbnoYbd!o`Xps3C%^NpA zdHI;S8l2M9rcqCx3`vgSzbl;NuLh%jY80m40*=C9*ZLrqjdqBgkL&m?@mU(p>4@oMCuG-G)zxQ7-h zZ%OZiqTleYVC#DyU~Gt8!+|PUv+Z|L&!u49U zDd_6Tb&&;h9q@OT8i>7@kPNLI*K2#UnVe_NEhMsycDb^KsDYoUO?8pC;hmxFO0XgD z2kmREbbq*?JRlzPCU1I_701lJeobmWi=*dtzI_G?{DYaY{@0B%Mmc)wZ=ABVDDdTu z_{(97G3rvD>Ty?58)P>8#tn2j){q}AabuH{Bef*9;)5NJ^6B>FQW$udxa-dOr89pW z&?~~a=&&G06h+Z|Uh9P;o!dJKMV3;I9fIn*uGfx|zppZ5d)lv{y1Rp={6&_B`-Y|5 zg<8rs5?uB*l6IIejeQ8cKFf;H*xRd*d(g1bVVt>a)L_csFL(c(TzVt>5L$k?{vIS* zPGm%R7zh+)!u?{gZR&f;b3zT87-wgGGBPqqwE6-MIRu3@x3v71 zJV?Hh4i$HBC|vsiRVp}gp8<(=Edzi+u&M5WNif`|xP*i$AQFPyOCn(JOG`Hizw9tT z&=dAZHWh3j)a(P3-}sYrBUjT4p6^EhG3B#cyt=c{(U_20TvEEZNi4s)NNj1n`iIPY z>`^RqX?eg!hm@lw;P?Igt9CHHY6B4fHIM=r3AqY%lK!+X$AB+<2a+m!(v%ocaLvq^`Lgw^WkPU&joQ&U&2_}G$ z9F_<9by*NP54j}1U>S$s{G-#&I)v4T&?4Zghk(0bNwUU|V3HsM(Jd@DUqIoG#1&kl zle*SZ=Cc;LN5KWfziuF{ zPhz1p7wwCtPl^~0L2dz;o$AIR1lMn<|@LYu&dLrp&2Jo!+f?iHcA<&fGcIbeUQ z&;%=;Z0^s!LeH}6PMMj6nbSC$&G(eJ^=iGLvN&X-;`|C!gS_kce;HvlHH&{^qLlP!gE%*t-sKD&*X;tHUY{x z<=)5v7kH*%*^ESH4r5+?wpHOd*Pk6HjkdNj!O5LBO#e!f-hb5Ed4UDZAlK9sYmfaW zYPAk;1IKKhYt)8ow($DS&{!vwKCQd=Jo&AbX<2%8p`j!&^0CHdJyWvxPW@g6L}xrb zywn71(HRHbswgZ|A>Nn2J_;pL;^5@DcQ8=1%s}Fa)a&2c)^86Xl@#2sH;;Rj9BMq4 zmw!WQw)c+eqA4b3st-Z@cdlm5^$Q;wSnXd5hijAi!KJsUZrh)Mz*y?Qy4@KsUGWQ- z`8r9w>*1|Z#uyR2D`pRQIEh(#NdcLdi4T?Ve7hIt>&}j-k9IXad&_}T#u^%nG48|t zJnP3R;Ab#8jN|}uug7=UikO#|J+XjwgxDJ4MKm&EKwLWzf&{rMgXa}E1O&;j8-&;J z0TBKma0PUA3~&L$Py|$Sd@z;;1;H<^qQI|TuR}hHjJ$j!q>4di5EPTJh`9~_D`XdT zA}h{9GZ`nRVkcHaPmT#-GyNjL)(uF6nT4x!4)lC*+2z9b4Ag&b-yU5d-@!A;qG%8# zc9~vc*cx7tjHwj(kyH0;!;6se$^5xb43c7tBo8CWnFutvZ;x#bl6DLrT@87!Ak>T+ z8dxOd1Y#=98j)}raJY>SC08BIQy|M;Na7P2$iw{x!A*|m$VhLFwBo0KM&2$&4;ho2jqPQu6~C{2nJl>fP-&y=`-o*(fWUgO7aUM}Jl5IC z$ylCHYW#RqquQzef?6jySJlx%w043+_#*ayRwsVao1HS{$ZD`H-eZSEaEC z4;Da; z&U4D`Jgo|@ri@x3WG61GS`AKegO+)`kl_fA#6RO*(N zNa?dS8~+^&8<$>lnl06yzZ4@=vTF@bR8n;eQ~K!dY{tzwl&8Rc_V#10&c*38+}a48+^$ZAJU zmDlp`uAaq`f-BQMX;>fJkp;xYU+e1bR_xN>ohSGp)nn%|r+FK(`Ct1HffCeH**AWd zi@UWj!=a<$8Jc#VOK7TD@p>Wrz_RI(#ltJjAFtmVIDTHKHkj zB9H`M3^_g)IV7-qBjy+mh=>Hk1)=_fiPC*KmeWb9*2CRwuT`qlpUXryyf>eaEekj- zJtNc+0rM}yy@B~5n^n*dJma~QEoIJ#;f+>S`FX^;owHM0CI7Cx&$)xu)(Bor$`}zNeO39QEItb9mS=^pumH?|AoE0Dg)vWj zn@b>2VwvCg8He|q{@9C5e5jRr`{}9UDz~%Jeg+;Lt&!4_x1HCj`JGZX=b)z9L|ZFT z6S~L!_3+0WC#@xx7zd-t+l^7LPp+Xj-FM@B-rcA>j5e3qBVY6L#FgDu5}nN05&e2u zvqQ{$*_(C$R{zf1GgmBS-6nSVPBE8vK&p6o1hy4L!n4C3LMv>Gav3_A?!_XWGB{2AD-SJ1E&M2VI)m zeZ6cUQC*P7HKVwVYgP8b2=`#9e_LK;w7O47_`1*?&f!lbMrwBqE{i;-TQj`0xQjNW ze*{jL5m&VZ{8w95G1#k?#P|jcZkqFco8)oKTB0D(MJ?Nw#{%n&z^yLyOY+rvGkTQz z9h#wBxpl%O)hHT=|GczBhVhoy1+KMJ>-5&soW`CdJCRG5&I(zxKl*q}!Sia=$S*Yg+#m<6tEfGeJNhj=<~_=G#H*CD^)#^lCG#5rK+nr&BxzTPZrN0ypXkSV6KnB%@I3)_RNL- zz)n9RAtVj*YFkZBO)P)FzV9(5?G=z`Qn+4skbBf1qDJ$qVE*1uf9iceN}vpnUi-Rv zxFh!hSj2ek>}e&Z>8}QQdIjA@j|u6p^YFKKe>gP7Z1_4P=)hQzP-c*$A7_>JlcBe4LtjcF*>2bKDz?>C zXKEXL?X&u<`Z{~E>w5R$MW)sW<@nNXZ1nyt_ccspuf}i3qZ6pQc<3Q&u+_qRk0zhF zMBg-j`1#{vE@g}5u6>syQj$XWzQF2?@}>56Ys!@iQ>XQMJeI5MCojiEjxX$R^PkBz z&ClNA7PE6c{t`zyZ3VP|YC?W_J+)!>J5`FLcU`rOlz*xBQ0tk+Xy?6!-cWovTTDkTDa=)uO^Dp*K-L4qrcCl#8mDp4sYt)U_VQN_H zojY8$zja3*nT53b^E>tA^={h8)bjGu(sXP{dmO8mrRKZ?T-9AGo}s&QIO9Fn zH`M02RPD_#pfGi*>UEV4s9+3~P0SeY*qmo8D|Ph=%S;{T>u=RrUzq3<`pZmsN_Uu^ zzoa}q=STME9R3rqi=oPSQ#bjOQ6<h(Z(lW;YqRKrVjdxV4de&rbQ7zP~v_Llx`VBk)mfW3Tt zeAQWjJeue5b-lvnhWn?h9U_dz;QnSgGptPk7sDCl4GjPaWbt})Yb)6cPW1H<;E>`y z858W-X=i8r`Kkm+gXwqS+G1ET1v35&glZ5y8{!Y56(-m@KuPR3&?N~72q<1VQZt~} z+|rzKdqqKmze@f7pt&;bq0)rO!A6!CMbCtUL!|AA=Gd3Iy63p;-Xgb$YYxsVy&+c} z;c>+q?gy-Aw%kcsefLVT3tJ!sMZDTO)G9MF4MN;Uw-q_Lu$Z&&Vb=_lNV;kft=vmX z>rrS?plot>L*jM2`R|NJy^7h~^%I;&-9zu~zwJ`e8MD)n6T5WgX{$<-VjfxV?3ugD zCu#X^l$~{A$CdO|cJ`UiZ0?rHhCZyOjm|V!r*Cjq^H52OqKL1*v3FLMaC&;W+^r?D z!VSphAU8K{j3$+`SkL%iQ`n!!pWkmQJ2t~(9nlog-nrJm)12diiyz;ti-Ul>h^hQG z3$M>fUvkqw^3!_8ox0)mIggS$F+@UlX)0WyN!!QJg6CUWz&%N?AH(NpO@!nd=J`4Y z;za5;-0JE(sd<`LNB3)l&-&aQk39Vuh0NPNwYCP|AD^0HLs&OUY zto@^VEc>~Sq<&?%xnwFs$!S@D)Fqe`&$)QruhWiy;y2(N8xa2!oXvf7M4tm-$0&mxs0T9fZo*FYV=`S&BLv zkKD3(kI!(c#nB)Up^+}}h+sadiBdwfyLT^#}FYMGS&_7PYzry%kw=kXg0hg8a6Gp!{Wm+z4}!=KQHv<$kh$# zEu}whZfSWp5wdAyGW-;V0AYJ)gdr^4K(dwm?UmdzY|NI19FfRB(ZrtX-i<}E)`4L< zSeOo2eGR_j@X+s$yaAex812=^UW!Khnp|PqK{X19Ptn%jkLn&jy$65?S-XiwMA)IQ z@t@_^$j<;REFV{p(sP#Q2e3(L4>8#tXi6qN0~mLl#RjD(L+B+y-FFTB?wJ6=HE*@D z3p!k-5;VDtHe_VizB6mtex|ddJ#!`0&*L(jQH5oFgce3yjKZJ2cmXRk8q9@N+i3s8 z><7$i0}-zypVA)A-qAHU-XaXRdhJ z#i~nCyWJ-nG@KXjz~Rj`dm+~<;`@@9d~?$&^)XHPwuPB`?b)?%r#Je=h@a6L@?y28 za?fNs{j0D?&MRFZKn$7&RxCO^L&5cO3FZ z3(F@NZH&|HuhvZ}hYc}EXBc-Jc-m5bP`BrWQnC2)$Dk@I-#5q8MH%I5q5>Iif0>NG zwC&E^GT+xE*NqeJQ{Gp$Sm=dX=f!KJ1w6X3xb&m42oY*S2)wGmoc}HMF+u6BsyYGK4 zzd6;A;A4pr8j>6ThtJ+^2<=|A2tc{SUiXCY3oL{i`zBsj$f-Fs^ zVX)zikGu5Q?)gd}8wJeuTY$)66mk_)dc~{k{AJF9gC`v(%+pGuwp?Dcs|pvd29{KV z12hU*HrVexc|yyn`sO^AV}O$~8t*M_{{Fr1(2teZ+8H>Imvunnz5AmgK{7Lz&r#H5 zTJl{NSh=6h8`enuUNVKfE7rjab#gk&5<-aV~v%lCNANT%|AOD*E!&*g+ z4dG=18e@7wO5By(?{2lQus}=g47da&Hf$vehsgu6vznvo5c&D1K5i5CM0VpIqPKSP z!zBr_QXu=b#Y=vZ`>?9Ut-+W@WUcX6Y?HvMcC^&7xyUCeiHDt?{Y6laC0dFYBh

N0Vaxrt0)5gzZ)*KmEytXsP!k|qJ4ylY%_qUKu(~3!BQrGyA)7g zE$#YtO4oajRw|CjHk`2*{Y|m_R`iHMNAT}lUMu#gcBxJMIIMKe5rh|n77Q! zW>!#FxQaLUWAP26tINez-FI9{n6NaB@2a6(4)Wa^Y`KvpjhSQH zgE*$R5GQ&H^8;>h5#tMV+h!=kZe5N$rMgWf{2j-ea{cqi@3H3~5nYAU;=&az6%QllMRdvTF#P=ze zWUblVvGRSZQmVs^nYR_)UV42@TH#)C$&Y$kQ%&kBpU{g4d@~OHG&gZnE8EuVw5G4+ zO@*k6BTt>x)_Fu~2)G;`<~}=B%#NyFp>3dJ#rMT|Q7)ISQNsi5-|U7u@(cciA4}*g z@UswauPAP}E-jPoUz+LcM>Q*#7MAGcwGq3&G18ENt}XJ2f9ig8(muQGvN@rgoa^2l z&S3p9+~r(LR>&?@fLr zRjKQs>hp4F_3C&p4zA?G)gSMS?ZY`emvp#!DE#qG zUQFLsh>YZ%E^NPuuu{^uM^Ld$Tl$V)*azA*8%pJ)?^D;iEMGG)q1Ey^(xs_U(5|+I zd3U08HUDT)h;p4`l(5q^9vbJBx$uG&2h;@kepu)9)Zf1dO}|RO2(o3by*(6_Lmei_ zFcG}A_kaafQ>Q{R(=S7jX*a=+C>#e>Cpo01rlw?J6S8=XF+VuNBdGLIc`w)eOf&Du zk;d4FS9NQKMKwTu_z7>I#hN$je5humWo4^UdBMYB&?Ikp-W2NVaHDX=1ni){i$Ovw z1WhwDvu5m3BeMRovPT$+?55v({LbWo?M^v|`noH^mZWZ`WE2$_Qy(nJY3QY667}rR z-5X(etD>?3mSzVphub1$b9aYa(DKZ`AXlM0L-|VX3{TW+>z(Ak)qPVftVK=4 z5gwjDdnSRGi*0Y9Xl{xQbJSX9!1MG}in%BzYYtpq#ZL7RB#u3rhJLNh}$VE)VA^X?4EEt-v@6r?`Sj?7JUvzD^lOwyb4v@ zhK(ByU_5cs3)HD%mX!gBSYq%dUhz_bKHOMr*^qacTB+y$`uPEIz06!Xm8mQaGcH#_ zhNVioyQ@@+Mot-+3@W+w9DL8Yc=1=?a$5w?vx9;3d-ha&O$2-TyXl=aw*=&b2&>NvR(KJ)U53bHYmY^`6H zH;fgjV*|0CZLvDKjeCEl?V(3L`B|3~t@!D@>41W}u8nwlerNK38Pem^-`0H6rO!I5 z$6U{%M^<>x?VKGBwXapUH280nxQD+8B{VjnWX>D@wyT9Uvz$gy&ttutn99r7eZ(T;N$R{c^CI-fB z`f_j0uhXX$HZiR3>q9%n+Pa=yvMKrC2iqINPgpKt?UbzDOlyWTHO>Cg!Us@ zcMOy?kp7`M>zuFD6i8D{P~a1rz8|009lnzDP$Q`rHhnHGE`-IBks-P{C{+c`+rEYf zTVtFohQyL24S#YzUMnBM|G8ynXIFIZM1k-b1lU!>0Hu%lK-5|}cWPgZE*t?4IB?|1 z<|Nt5cX79VNsDK+6(x($U{Z5k2KaHH0r&;Jvln63$Ga%k?`uc`1nG! z15QrugYgywa7f}j5HJ~Ms6v9ki!a3Tgjd$V}(J28 zf@#r(v1CP0#*EJ<@)@dOJxvxZo?dAJVQi0je(9EgPf^dgbM!>0w?JakO=vuOy2PLM zJN#q$3KK8evCLQc_dDeE69gbl8ExW5|>Q<)R{t#H-ZdHt~`NJptnws^f2$Ru-6>78}U0tVQTP{6nvD3fG zC^=! zZa)VPH5!Y8&&>B=kW?RamS%SYhwR2gfx_*KMGXd0553yYH+#CZJ*kV}J9N(MDj#u! zIPuLii^X;zk8g45X==x*NO}3P`t5so9YSP;a0J{?w?)fyJlrU9v3cr3&eHFrhAU29 zAXOeib#-heTg;JfVZCY7jbn80-QC^Cv0k|j3J@Hc@IT)>(fuL&uizg5-3cw@30jq_ zrzu@jzBHc;-YD)8aISo&=eUid(9XkJU8wOl>*5_ATVV!ei4pkp%16)ZkU_f46ydCuNWuH3zvNVD97ZMt|TVo6O^$#+Oe2|EF zAX<{A2L?raxTK@nAha;{0ys+{Nl2~8fvoq7H4b>LrzCb>Rvf6kmui zQL4>AqyK==7n5CT!EfG}!3pPGYNhnB;4a!h(VKo(-yY`hr#>7eUfy+Yk0pF2Utdo4 z_V*VE_hOdAuC+b(d7rhO%l>WXtyh@c=I+t1vK`@53#-;(B6F-t?Oj^hPk^Lq?PHeE z5fgy?Qoz&-li{D&Z~q$PDQew!u`Gb+0(`8%DgQ)-BjD=!^ZPOW>vl#7Dt-TWv{AJY znM5B8jzVpBP^BuW#SbA%eraqBAS>ODy}e&>H}$Nn|7viOb;=j6{W0OpN;{LF3>4rk zX|h*+k9Y3bqqK%5ezs*n&(Oy${2A|bfc=24f7D%CMj{VsDx}|S{ersarEb*w!j6tG z#TXe>v96M#UR$?sFM<{_)8_Y92$Xwl&+HmIx}}iag!F`kh4o%%yM~%rUNNOXQQ=0M=0d-j;Hklqx-?M%nn+cyVb8$*j` zGX^BDe)f*2A9z@vLS%twDc%m7$^!d6%NvrBu3+0=7)vH=623r?&cyS$_WZ z&Clv*ndQ$90V%1fv_T;m5B>zWslQ@lG!{QqnKp3QtJNjs<^%3}n!-4(gJ5&uGydGS z{?5Xb-RxZU=$Fr{V94x>oniWxE~Se`{C4BlFvv+(He%5b!E#6_CvO0%PSE~}Wk2W} z78KOq$7?V8D(;JT`DTu7k3yDOTAl`NU|?u=u)o^ZMR9`Jo$Ipv?ChsCr4sHn^4qGa zUd1cpRnvomhPGVwx`4$8i)yOqa}bczkTFt}$>9t0H27CAX=+1|8qu+)VQW|ejuaVs z!U|SF^VE!tg9i?*6%xvj6s97?9<0<~;q9Ot2Sktn)eX`54~(DUbt$~RCSB%t=rs76 zc=YI1fC9cs;&>4p46G;T t@et|w0h0S2|GosdlFz&xP|1(jq5B_ZVHC2=0 zx}h;Zc*H`9Pq)G+$mV)ftaMUH!O9vV8z#}@9^Z_MK`2#eb{_7<_bon9+MkBk*x|fz zJ>9_L`i_qSx7a2RzFfQezhfSZbOKviP$*a3>}y+B=Eu3=Qy;T~Ne4%bT1!`P zCG25)gl%SzSUbP))=26@orvNj-Epe@+FP9sRTCXtP3Q@5Ao%4kO*7&fFI$&z|0@MJ zb}OzrwVw6SDFvQY^66&kAZRlk=Z@|6imj1x34ZzV7*=_a;oil$38K2Hs`5r-m7AM8 z<^6}g(8it(>(?Kl%e*4p<|b47;=Qfcy^!%J7&sNaU@A~n-?_)bfzS5+EtA2=A=0i} z%a16jGh}q07kj9xL+xMZcPxF%t2Oz+;HxKzYdWUeQ;uZ?^1lP&&~sss$IsL`1emcF zRiBhl&4K84l~pQUH#oFjwq=N}8uIMmsBltH$rpP3^(oVn!}dIX!X$L(t6v7sbAM

79q(~d3O8Z8RH^UoW#2xHpzgB%w`vVI%ZecXAyn0jt0Rgq&G z8L@!$`5lB|WDMfw%_AsWKu2}=^%Z{@IY^Q;8o)2jW_wy`LoRUqu35Q~=C(gLb;G2) zq3E!-4?Cma($nHr{Rz5naqPR-3%@n^bmqdv%5BB4H*?_U-@tt-qhMw$0~~Hl+A=px z2<5nyZ`lc3fzL>CVFNEmgs|=dMf_Q~pCBylKYqM9vl^`{oB;)7vL33m~r<*_84wYFX3TV;Q zPn>n(>*>nBvWYLyP$Btc84A0=Yr5@MHY$0%MDWM5lRc3UH z<1p7#F4V=4u>Unqh`q%4Rx3*qlDi-wiw^69n^hAO-C#+~5H7B^G-L_-5y!CWF>fB} z!ho=biYLrbcNH-)fmD408@JzMW;kFa%@04X^?ch|P8ALdrbQpG$NhtDn-vx7Xa77t z`efJ9`!I93$<-~GJ;U7BmDkR#{k}J{Uw&nx<1#1OUgW-vUH4r}!TNw>^(M{L5h-OV zu))^X*C#`pWKC)0WDw?XI#!=bYFC#6Ia8E8(&}5jaZaaO2BTetQd`PjVN0M3 z_6VO~$$fZ^<;xKTjlLH11lz*W9NOzYb0vDqH?-FL;*0OAWB$7EQh@vDr10Cfqq`LE zE6hJB7n$9p+kLleL1|l+*MwPP_-~Jo67pg9oGD^DZ_lOw_RhWcdVYj-p15zcC)ED% zkj!?M`V9{>yn1J|;Hi8=#jLQG-+3n3BbtwMBs3fw(R^HfFzuUD<=Vilb{6v*tz5r^ zKd|lvLA;zVor4jlC1rBh|0pPfBAuT-qZvQ%D4b|i;+7apK zYxRN_h{$bkqVK-e`rLv_qpSPoh{Z^QroFk8>8@rKDZ^NXzP`Q+wda*JVZtK= z$vXKkUZcg&a1}Yn&0Z88#5Y%RUt)|x?0qFDs8TQ`bju6#^T*%?*V7-`uaKz9)8RCu zO>8Ma<J7xIGBpU_oWz1DUbEub=M|LBi#8MM1- zlB^hdxjnYSCl4lFY%+d#jDztu-&=Hw6?-#Z0MC!kcSMuxhM?*nm}PlI9B{L;Y9V$< z_Ma}C5eGZy$D>5t|_m;6}s(U>^)%9+T2XGMghI8r=>j;DZkm~*RE%` zhwBnAH~?;zqriG7dmep-P|AC0#+~T`I1`T7>{hVCqN1{rW-2%AUlTiORGJ5Z6zr#HP8i6<9Ra4W){^OMRG#JI%%FOH)l*F4W5`d)#C$Qht z3$9qc=sRF0Kq20B<*CyBPVChH$N2Qwv#}aMd^LjVkPP0z>p+ruk?L1N;!aR&klQ_n zqzAEWbaeEqZ^|9zZjur~#|%_?_F9DojbwCMe3LKXoPojVMd*)Eh(71WvZh!005x&} zDn%pYCm_#yaIGL?Wl_6_rSln9>=hAd%`_{(X*upVdjuj%i5SK0(vBASOM+zkBid3Z z6ZwV{N19WdnV}n!NN$R58fba1#}il;zE6;y&PW?*LCc*@@1{#oJwpcKeRFFWQg0SNiJPyQ05o; z&yXVXQwgT5TtTZGvFtw6hha`sh?OAyJs)R#C4R|0ljEPwe_Xw!CQfb-J`=%exJPUp z9Pu27-9CLf)mI%}J=RlV0x%84`MIRTV~_LfJ)AdC(WS^gb6u{RWL|t8oF;(KI753TB=pk`1$~3ymVbIT#9j=m!282? z+*sTeB2h}ySWI zeB+V@6oX+PQF1C!^j83+QL}2`whAIeAe4gWp+ntN+>&a9N}cn=^5r*b1#n0lZ!=C$ zN!fei#BN5$U`Y`le5)~?F4WDa1Y{9H!m@nm3`xIQA4UK7JVMU$NIuqcKHK6EQ9g&4?S4$~eN<54>7j zo})rSE-jG^qH>D!SRx6#%4@I_bldgp-{Fl)Dcwq%CugkaDI!Lj5?KVzb|G|pfnyl1 z!cL8RJY9Z};7bcLBnFY&jF6&#{klec|DA8T`9wnXV6$8@!aI|=bB_G{bq!4~nGM(d z7cb@M-~cv%yu}6EK1{m|`|ngaiZ?7u1|MYt+`@*{NEFMSn~8Q65R(e>)8N` z8hBCK4!4zJ+>@A-J!{Z}}jxBngm)npc?0}?6oWdaA&vH5`^2qR9*Ya9C69j4rOvQOdu>k2J zYEkOk6*=)GKWtb<_lg14HR3$Sq%}TZG@0B(7oZjeT;BX2uJ7qL9*2f%M}|xNL-cN@ zkP#EifdEOWT+p}gAWuTd1ofnt;P|JSm+eBpE6}HMW?36!%mCL0m3-o zY?A6u>9m9Y9`2bR|0;`pGnXki9<8NdWMs4*Y?c6MLNM+F5aNDbAcO~716d-(!bf-V zUO9kdg((PY5N-Q{{du?l?UjC3ioM8yyqV>^bOJ94V-WaZz}T?~3p0>OKFmC>!nD=7 z#0Gf8mxC|77b4M*Pp==2ugE=~$))<{p<+TW%6%U*eMoq-vgY>Bh@3bfHZA%9Mm6-E z`j9e-EA1l^gNc*X=zZby+l9r&#W9A1q}fH!T3=W9Bsrr%cP}gJkA!Lq6BB8aMyp1# z*1oKa8g@-2+d~+PyFx`qtua&u^D?88jh{c?2Z%vfn*7#}P(U8b{{{*Q0c`QAG5J94 zwBfE6LXg2&cM+Oy9>{2XH@iOV6Gl+${fJ~p#Onyu`|%q_1^3s1`XwVA2l^ebzFZnY zCxq5ee}9#{Rlkk=gSTzov&VwzRTV}^#l$+A**Q6>LuILYY0XLs3n?O+2GDP;tk*#k zHca2z?ZE^5%7k{wl5`E=*d|fIrk-ihO=$i?4nlE6K;SGw(w{%q+bA(xUhAl`6@P7X za&oN|WsB&^lPq?0X&PRMiKcl&=r+GVn=HJwrA5!cU|p{xds;?@%bx)|toISN9oz*6 zL13oMm>mxeZdIDu|2H!{xFdYvFHiH(-+w!%+0Ux3+S?RtktYqO{OEIDpR22V0HSxa zWC2awz}bpHd5p7PHk)mP+euQl8{OA(61P|jCywr-C8j~h*Z|bhV4v+E;yM7tGr-pINk`;JO>~$}>Jb zUTWIZ!Xn&a?g225E?glU-zp}I7xm#$UOay}LD}v(ZAfUSSLy1krZlGoTLLVQ2?$sr zRqgAhbN_{g@qdVQ%Y_T&`?Oc@4U~)|M%o1A0=pk2lRFq<-zOReXbPd|6F?<0ZUbQ9 z2IK$=A_gO}Zi23(3M>Iyd;i@fBcWS=l${sHq)5^QX*35lUJ$w?jg5`P>f=zSK`JDE zxla4~_1Bh%?rA{=gt?-=7X0Wh`z;oh_w4B@Z8BVqk!+~n^uaH{d4b8b z;1k@q=Aa#fEcDE?l2P5&r)+I)!xS?}jN>?-q*@N-gp&{pt-H-gM=0LDN$-uBET(8@;Wu@Nf4ZOJ@)<+dv1hdAE6y*cD@<(C2xCR|Qt!C&Ck$qRuoUQY}> zgY_h^Nl)kd+}Q%cC?I*RN&%21KQs9U`K%Z$L9cbpKX{eQA>FhcdQ`NVPFq=x(`e!@ z$Hg54Evk>2k9IfzL%b)VQpLy*IEGK~)5O|7+|m-q0ZVGu0hy3Qz%G?Y^0XAk{Ku**7)3x##YZi-elyL^_S|S%s z`HgGHHa8+!0z)JfF6MN__B?reC{m=Jl7d758%xa2Ly*A3Gvv0(4|ISxS_7oBgQ`~s zRWlL&B9ImfoZQK+rmLl2*V4iOPkjueoCB@-Z*8^*&_{kvrrT!TbJ(>AxbXuc4P;QL zr^JVT`}VVtBLh%UxAhqf0M zhR{IFe;6O!X?PjzMnSwn^smMwMyLpQu1}Tc!&?=K6}0W}=;RZWaaXLC=5GP(5I;<) zpB-9Q^Y{_n&Axx|#K-EwS`mm0*RNlHCpHg|X45Gk5SpWs{{b02`?>HM06ZPejMb;k zoh!Z&bjZ}u@EnBE#FT+pj(|o(UX~{U4Pb$o7&fd1g_q+qJY!3IEchlZf9}~9m4bl_ zSSRY(?y6=(Q3?b3o%XUIuke-*>ExZn3x?RLCf~(HQc@D4@;Jz`X_<~cy?gg=V0vRB z_o}}YK{ke*0f%Y_rcj1J>$%+KWQ(%*m_eC5h&=+Cjl!LZ2X1`aYIR!blDG=%8SR^- z{l1%jJuffsxhJ~_=VAOUehanRYOix&FyxKq5i{O{OQdBkXNSv*6Op1rGI$m?_tt$C zdWaxsFS8)qgEfcS1KAF)Jv(_O4s5Sr?dRqeUqvMt2iA6_6jhodEOkrwQOi-1c-uI+_Mn;Mb8puGH3BHkS#F1zgpMG$OD%j zH6&oQ62hJ1O5hcDDlUqz|L{${rH#!iNe=$gjgM-RGZ0tW#P&OfdaALEJi+NXT8A92G@ z-ar>>Vq;qeEQ~roT^$~S%+KyxNNj8@x~HNO&M4lowvHuh#SUogez9`lvPhrmd5(7X ze1KNT_$ELrC7-_Dx1+9GB*ZahA@1G^95+CVgtU>Z$QUR)$14qaIV>zjr0BxLd2vgT2==rVL-b^1 z)Gsv*{Hz==ADMERKKb~66iGe|`zbc>-mMW)FbZCQjHTitlF$qqjCl3;wEGy{>>Irr zur#$3GiG`itRqIb%X?fTe49F0Pb?mHIjmEX9$F%tX|YCrb^`^-4}DMz(6?e+ zOP^4fLmj9>-+TC{e5q03X{fAwQK5EHs;6Z)-Y?dgHC%sTx~iuj$K<1Hu1;k7*|e19B--R<YhE8nw6-1#IZ zgE*)h$-)vJvH6G46)T{Mr$8&Z3J`z{{NX`~-uV8Zps;W)+_nMbLM%$(8TmhbdI~SR z1rlK(XmDCju~q8idnd34NXgM4Q~^zckoQSi1v^;?>`yZ^aTWF}ps|U5nWXe49T(;# zRVjqG%$q+}3ta!!g>U*QHs&4Iug#35JIR7;@Y>1i)qohXl5_jQgu`%KTS( z-TW^;&72D&+*V;KW*+PUC;({#2t~EI`_NC-Z+2}~7qTcfE`(sMm13MMGf7b`sUvH{ zU9z8<8ubzHef^lXFV?Q%n)bgo(%}Se{`%CW-~(rpdeEdcs(U3(xM2VnkFbMKx~FMq zN8}ISwm1U#aRb2t%0v1_N>Fde+C@;6lGYdT7e&7YVXz25*3!xfb$*_FfKehQztPeD z_Ewt_Ll_veNp2?9fnW68Bs|7RkY2S1tMOt1G5LBh6 z(;7GIsMA+7alL~z93U=Ykgf1h#BH8-c66}dL7wwrsf3PhPtf`r47sQ#X}Fz>{?UHB z(*U6Wtht!&(-TLIJVjoHjDV5!Yze0Y9X4F=z-YFq1z>FI1zZ+z1#a5f7DKm!(q10I z9Mm3gl(YY3l!dOI0`T7>kWLW=D_W?f@#yN%tGWIy{vJZee@JR>D?KnNyd#QqkL=o$ zm8@z02E!qW|4#84g3G$+{wogVe$uh*JcFIh@k()Y z7<}4D0ZzU`XM0I2^Wc*k=4bx2*q$#BRTJHFi|^+`QOk=UNsnSf4u2j7A#)}jr>X6L zFl2lW>Azn@4$UesH*nUYBs0p;S>csUeDL(~F#=cA8;K9<``g&g!=b)z_Nk)G9{CPDE&0{HcF@R&Y;GJ#B9kXefSJX8ZSV2Bz);I4gX2&EbU2;MbU^o6VTZtq;oj z8KxnC@rA0DAe|2~q4z?NCCQ$l4eSMU7W&G=H5iaEfX2UN;%9HL6iLQvhkc6UD zdko3kv6ONXq$%fRW!c5V!tm_Tz*`lsm30h8+y~&>hQ6Vgk`}y%Suv!6`ipGcfhe{) z$Bqvz(A8FN+uPf7KRNZO9tBx)I!4iQ$#8A77(>}7A941*dnxXwjGo67M}k7qF+;=H zVts!Zwk0v9LNso&9Di>GOBZ2KzQgBaV1E9JlFon0sk=xaZ;%|yP8mX47fv|bEWV1u z8;H%vn1?)O54hM-Sv5wwY>iPYon+uc|A$w}Ir~YBn{bPI0>zm#zgw)eJlwMGJBMC; zk$#-fYsdWUqXmO`izh(~Xx!T69<9P8*w{!!Pz~7vvjK{tlO)pjAp~;x%V4g>u+8~r z*>>V4f%Z~Z&mX<7PgyC5+y|e9ZjYE&L=Wl)kQ#o6wRfYG?lC$E$@RDE0+^MZ<14vl zVNs2$hOiExW#q)>9)Nr!E1*d+1z|FA0%K?IOPFs5B*m=#kBqr|*DK|aMm<@0>){q% z|NEw`)mQ6^^!5d5>B5vosoGEVfti^ZbOnk&n6<#|CV^!2l4MIXyjz50LmdU{)*+LUl(Jpncyg=1qdQ0?Vy-3Zqw0GJ01Kr(si%CNWJ6|G?O6FAoe_2L z?_kiG;be-xBSK?lEN>_<8O8RG5sllnb0^hZA0N4U-!(Nkf6XjwqK!v#@4k`P@g~)c zEJc3J8eB&dzKjp;cApXa^8MIIm&>&$_k3ATeApn8pqL~WftEDYCLvQ`&IDKokT5kn zTZ1V}=eeIdA&b6%C~2Q7CZ~fUytr-}ZZP(5n~1in3zkUTMC0k;z-QI_)$s-~fg*J` zf*QP<2zDZp2c2vjQIdmUtS|@lSUA^P)PSqWejY^s(^$YvWTmE!Z@}BpAQD>kS206~ z`o-^%(k?zjTH<}s(IKXzgI-iG(9*?lfyeKEIvUqRtfJWM6%gSnWtU{Pe{cU%Lrh|m zTACi$4GsRjWVXl6I0@{ru^&dh1zlej2oiU3&Rh0<{`^U}wcCHR*5RpgI)ugLBnSy9 zU03B^;U!OcjOi6Yp6kaPvC@Jf9=Z0OPw5g9IxnTkE>aUMdv)l-8fxFYe%x8?|@$A!ZO$I!Tx+ydYhm{C{Ka%~Gny&~U}z*2xV{U@Vn% z*nYYkHYk?Mj}+P^zdC!Gxv;i?j>Eg|w_a#)jy^Tll2(3VLlUYKVR zIq}4Y)c5Z{V?G#NiuVpP<|sO`-mnXwgh+HKf==Vg$C` z6;v!5mJp-=bnpr*LUi#2MnwDlTFu?t3W>^tEiStN)?=u}e%aWkkF>~H$#Axdq!oPr z972Re;ET~0FCVM#Pq-XMDX^4C&%)xZn&>qAA0fP%H|(Em?Oj-)w zBsO~&ti#UP5Ed2|q7~s*OSub_MLNJlnWTy{go)j{E#Yw^@P8hM_CD;U9zmFT=K%OFxygfs&}9t|9DJs=3Saj8B57etTQR=7_c2a|+l)z#Y}eWSyoHpnF(fHEQS zFl@a+l0{8pV78N<-W6YuRKK9C&{9psX$3h|w7j%{%D2+GgeX9h0GUG;jdzw075f?6f;lj-Cg`3~!U0C~)X)1@pO zU_+Ai0#FFr)oj2d5b5-Qk9LHgG;x}LvXsfhu$wD{iMx3*dmX4g^usI%yu%*0*1q7R zul-H?CM>iNKc%I$HR%{tGoMDhC6>Bg8}53T zuMhjp2;#{{UMqc$X44i_Zcr7eCXtz{_iC)Nm$7*A-^|Y-jC*I4Ojx8Q(H>jsB#J;ogbuaI?#doR1g-FoM zIE=H!>6fXkt&Le>c7PTd8XAxfQYc6WB@M*}06!6Eo9;is-TvnDH zry4Gk9bjN2I6B9wIbnqX{{*5h!h)I>0Nx=T3%XBZO8rpU9Agu0yBAXEhK49THmSdE zFg=?3KJskO{3f0S7jZecYDrI(e{6xmhkqT(uS>4We^N3D7d;rgWAMt z+i|@O4U1@$mk?S#gKY$?q)>A1V`q0qZmrF<)CXZiWbf(#o}iMxKxL2esvh($g70a_ zK0$wjeiY+JAu8XK`tBVo7A01vql=Jk*76yjPuQ}1E%`Ix2ter)geMy=R*7g2?ocMO z7;7y_FAszOM#Dp3(u~PBjJ|LltO_yuBqF*BMKPK01_GvII{8q-M6$>JR`~JRO;E1T z2pImWMbFg!r|tP$JynksP*dPlzh_AzK*0|@Da^6rz&CNLzkhYAWb#kLW80zZr!lv6 z4ZjP3Urta?*a|7iRfv?Ul~uY-4sD}`D$plCUmVf`CQ*C8-2&HB5GObvn6 zA0%w|;NYS%-%f%Lp8ajWnl?UxTAHlRfOXe&q5k(IioFghoXqyBKuLO~`)obBRdY%P(;Db%)mh{0Fb$ zA$$N$a0+VRiFtGI1yG&sXc72#VOXg>K96>N;Kb3W9s;OyockF_L9B=IQX&)+YJS^* z?4PvH9b2FZ7dH>b0PX)F??3!{?%)0o{9R;JR1_f+r`3sMk5m%bh3wHnR#ryQq-AAq zS~$r{Mr5RjjFU36M?=|7rS<;Zk2t^A=lA;;e&_AFKG)}Tq4)dsdOnZwcs!2Bad?aq z!X0Qe`iNV>=g$|r7{oq)Z8l<5kN45coa`zt}MzOKoNhD>*@0D-jSL5r3Je>CN!HJdPn);$ri%32u3p#FIO84E(me0QekT3<_fIY6c6IF3$#CBy zyUpJ5sf(a7)GL^H_B69omrNzMhZ$yPd^oG&-x^kz6LUmh3xIb_)3 zyJMIARiYg8eM-vfi7l>lrgbJziv`84TeYAO$@;YV{LoBIpN0(^0;*B1_cBNKCNq=3 zW2INeAI}Ef+z#t3Bjgm4QUZut@a46G5;K&uPJa#c*t6w#Ku&2LoM?Qv9au$zXjV^h+^h(tcwlk zMSgdSR&vsPUvf70bf41zz)7MA+|f4X_R_`_XS{3OYfHb9{*%5xT4H+xCAL5xY8s8i zAB4*IepH?Ra$-nzPONIvVte+Kc zzL&(08xU4>yVE=6(cr&sNK`QiV|5YVNJuSmWEZ*_I-jAy97tVE`kCdgYl`I>U5l); zl@OSgW;e7cE^GYR45iV8uJ}->cP574}yBp=CP~xcD1eEVuYEpd?la+Pg< zD}w~tdhT3f3?C5-o1$*uf_o|}ON8sASi0}N;p)}?qHY+fv%awr6$hd`ii5N-#Xt;W zQKpj3ZZf%(KNezp74d8$=|BY%R7wo(V%1nPlU$0Sws+=!@llrsO>hEBO$ET(8Z(fgV(;5!^QCYdHm^I zu*HI+ACQ9KCzt+g$z8=1E8mmhFG`4e|p9T{P6MZ}dU-qwF|H&+0{+4o8Uux|Pi_g@UBo(5N;j8bQXZ|V$ zVuQ8o)^+`UdFiLE`Q_9#;1z!R|5{CWaB0AW+PT;;OLG?Lp=ROp&QEqxy}pgRP6>{B zeXHxfVTYrp#xK$OhR<})aSuEDY65EtZ-)>1D=s)FC`uHwW5PU)NNJ zj5~h(cr2>Pzo{)iUV(M|>18;hG(_)nb3@jw8JO_7zGPRl(bx|?!fB{c=cz+@#bLiL z{f81%cf7wN77dFR-vmmT2HXjp7vp05CipKSMs$zkE$`1S$8Q3kzr9Do#l|2bBg`X# zZ%eDOZQ&{sQlY(q>MM1^9?K(-`S4p@_l=1bc`AA~4o{dzl`F)dw6V_pn3sXG+)$$= zo+pW>JoW5Jyt8pOY8Jk4&z}C>KM9*RZ{8Q1u*c7y^>iqP@(ND5)kIHk=7`=h0(bA; z-ww`yzC^~;HiB^czZ2&~fjJX#wy+ET{yPF2vB^g(X%1iZ?$#fTlMGSDx#L{(_;I&) zKQR;fxJSo9n>3;}`Rw$Q<1L=Pj~f&HZ?B2Uzc{(X)08;Ys943MTddgUI$GE{P7A-{ z3|j8_lWChm25QDq&{$dgnc!fGsv5WY{ehBw1CkthAh>>u=`o2zMufzsm8y%;_iv1? zdA5wF0;l8EU0|t{@^IC$pNu*qV%EDIU0=jFbJI`V$LOD}VLdqnuxv6lge+@{uF)U2 zVf8J)#q+-(M8&V z5kt^GFqh-~;=`X;5@z||h^Rb~iuK45-GR(SE*3NW>cq3N>d|YiMBJ`y{o_hueS{v@ znJ9^fh}hLU^v|_3{S99Id7{@QlB5U)66U;!TB-d@eFHIw_LPN$hJHK}?h{1^$UIpfgBL={QB9&rVfJdhXW>V^C0S~{*wZ0))_pC#K~M*W z8r*?paqx1C@slU!n%Y9-^mugLn9~39hs?|y41U167k#ntyHQ(9|5a^=`uEdq;7LQ) zt`&UJr0dc@U%Xp&@AKy)hWN(Pa2AjP@aiZTk)i}EkO4}>Ah^d|)b^^!;gp9=D;GWe zYF}zY85*pf*D$Ee{Is+$Xj}^an_p5rjN$nc@!nqeQQ6f+!M?sD#ya<~s1T_t3NNlK zY6hl^TfmbITK?9CanKp`QDT!oahir(el5*XKbLl+Mw@ie=69pXqlW!hXzJ2_(wV;y zq5Re6i1y-s%9g(T?H$IS(5HSeshIpH`qywGiL=qFK$9u${j#Gw3DY^J%j_I-Rja_gUKgl3=o9olj0q5#LmfzibJS&X7HQs5N44JI9M+p3yfXW;4?w1{}3FrWC&q>dCtz zP$8|itfi^%w{hdhEjPFP4Z{svPG3MUYsY-HbkxUf9ro{^Hv45nXS@k!;0Gd564f+4 zrZt{m4nE!1_B^?F%a%QB1LHlzd?18eMfC|?Y3l392%$WRjHlJ`?e7DCfZdru7G)Kv zZg#&@OOPMR(-+_w&_pSS1Q~~T6fs)EXIi77>5baHj&B|c09JSCC;P&M*0Vl$lJ0oO z5ZP+R={=klBSw?dTzNcGO7Fp3DmW+Su4gn8>!FZ~l1XCuQ#UPTJ`BQp~A%utw1 zVq#2W_<+huf%;C&N_;?Stv0opK(6QHMu1w#xHW>8%n47!1CnP9+tJa1Zkv7MU7eOYeu-d1a_UpD{D?|0II2@ajZxTW~onV~EbBU(cq> za8yXE&HMNNi(IDi%ndnR$r=cxgO1F#Kz!ZcU~25LFFoNZYmfS(-+O`9z6Vif7SMb}V02!$P@iA2YXr>Vqavo*XBAi*Ix#z#$*ru& zyP-w*h=NdicQ%EO3aNS_BK~E?X+S4o;y#!EVj?)Vpt_O&@4t*THjpQ+8{>fT@|qj*)?rlose2?n3Z=LI@7RBAZ90QUWu_Z+TKg zVcT}?WEl*kS*oFLMUwn4Gql?e_8{g;JN5nhU0xG|6k%L9Xt&nymd zJLEbeWAycnrO7kqK8?NyN_KH+kL@8LdBL~P_}LwQS`cqneL+Je?En45U-MqYy&hO2 zb(>~{UHhXzNQ7WlXh^BYsj?gD>J~LPsApx?iID*ipxjp%ObBBK30Ay=K(1##ut&nz zJV_>_lLa%8sn{nyiNrGQN{2(ZQhB~?#J#oN?rS&PopJr5mI{XNZ}-lB`(wtsF?Fw< zxZAeklz-&HK3#mi4oL_v)1yAMIy^?FVZ(t_r|!=;-~%1;$j|Utxl)EsSwsOL*%VFD zA;f7~-o8SGfg?#4LbsD8%QjHHVM&!8nGR?zQf-i03w>W&d^C&BCP_iP!&3zCsF_kCkA<< zn&-AHqPKbdy4ofN$OF*!0p%O4M?ITR1Jlm08X=j71nw~In69~=@tGfai-8l0KR>ZP zIlf6IL3GxQod;?@`FX_2u@eztLd=`@mls}3y7eqH(Xi+Z(c;J)C@m6y$%q}N7RNZS;( z5AfJS!`Elgdj+-H43GzuwQ%?$Fn&CASryNG#P^xOjEcjnIAU}Ox^IdDM%Yq~*?bet zW<57>+7*wwI;Q(~dHZGD`k8*^orI<;8d3wZ=N)9qN2~5&yanuFD+Y7h!dkrclU#|g`OEJ ze_SoWW5AIwAkdp^tEU81y;h4xE0Q+1`-0c{D43AJ;s}%juP64l)L3KLvLGZgMSy9y zZui_PcxC$F2;?3Li&m{#k(d*2-3mH=y35Ya)ApKtv|&-iZN5Rsbmj~+uunW;NJ>7LN@O>a6fzau036s8Tb4- z>dA@eYE$;D8r~3e=d7ozw%g>MPEX5DF8%U+@3aB6qo9{CnRV&LIW{}o zQBPeB(&{h;fz;%fxr%iGri|+kAI=`WVR{{vK4)V^YfRdd^*X4fk(AbhOA$c=u&{{6 z8L@^|9{yVkV7Qppf#I2Yz?}~$JSZ)*TECK>nJ3ybe4)jOnb&j8i&oBySe)mZqSI;l zGxgP@=FYJnc>hcB_crz4g*v{c6kIoMXsh9a?`o z-z2n7Vj@C%%mXXlIx%yHl3#Xn0wM(F^h48*{{!Be z@nQOY_r@lpPcBj0YPNpc&YkO?Mf}yj(?$X|r?YUs1r=+*ewU9VEcnx0tZyykWk8p? zjJF}lxy5PZeE}KlUU+pBkRN?j3elyMhT|yo8&-=RLv=Lr-_7izUq?#|j=w*ur)Iz{ zl5z(hbV9H!!V^AkRQeZGALb;m?d@A?oVG_80tUPsIPhM++l%_N*4S*$hyh%)3)p1D(Zb1 z=Ti0BM2@HcW6MV*t+V}Hyb?eoo*8b6l57V0XVI(3XF`98m(xDi`{hTrOu5>zZ{NPD zk6ZyquM|G$CkwrZB)oX>1d}WecO}Y5DA#=CMp1U{_07F{95+0?7_)v2|fGGzac^vDi)Hycp)aIO0jid?fr@UN$u1Isc z`sp)gW_3Y4!uI)zL?=)oSsKIW|>(;KVOVNL~u?;G+1oi4CMi88^zFKm< zC3x833`9<%OBMmf?<0~vZy_h&hF^eoD88G`0M(C-L1tP?_ zNB4uCCQnCVlmZ~t=(54FuWS9f!_i{HonV?6KY+)wXb1kh|lu`vMHlCFima!K*$ zaco>mMsRi!4uS@i)#Jc5j0UvEwN`@g{P`D|^k%e-p`)s3hhW4H3Z`g>&6@SG^{W@J zUb*L{88Ocx0!2oz-gCeiy;8i-&K`so2>rBe^MRcy(}4^`ipLv|1p43P7S08R=KaM* z5V9o~kJ_+Gnj}|5K0t#C)w$&d$3$-^U-)_*&|9-+^Wwg1M!D`cA+O&LZdtFI*L?N8Ow1}wx(m#7wq3xz`(YT&{Qy+Hh>mV^w=al?-{SCE@ZVYJystq4wBxziL^Ey{-|5Mj_>|qEOqeJ zZ7cE|j;sl6*8S+XwCL!)V-^exZK3rKA#ML<{Rd~y&@`8f)KXVdr z2ZIKdHegnj`Lk%lh|bgMA6{H}>BPHB>%*qK+_K}qNb9WT!#q9hqOC@C6)|}3oo>1& z(>KuBEcx`P4Gvv~D?9&(z$YIcS>3SaPsE1dJ$Fp>PAKj7B_}6)xdALn1;kN8kzp<) z3ygp_HawxNY)2Odu4cp-$Du=i5!6KUz`uZJ$gBWEj5spgYIXu9u?;*6f1BARexpkp zzt_nM_8d|#;YE(ZaG zi+L|N(e4Ram<#JeU&y>U1(V6U_ZEpXouzO^3_>{~bV1&bhij|Z(|O1daDmkIi``IF z38zZ4J_9gE#@slN7O|;xY;2p0II^yFI{lFNsgP|qxuwUsO{wXNx9&-<6?5CGi%M?A`-fdiSwf|G&ae&!NGBoSenCB8T9@lzvYAJ%1Pp;>G zNPozLebp_rM@LXB@N)J=MnS)bQcs%a$Zt?w13Wx>%b*(6746HlfRMmarK7X+YG(5q zHEIOiUclVEj=ug7UXFYdj;Cy|U^n)*u&@U-lR{!*`asZvil8RU6hki_KK+au^Toil zOk^1`OCWPV1V1^l>l<~FNTi~oVp9G{Adq}jC{~v_rR_YK@8WZ40pKJpJd`)Ne~(O` zyVoB~P6~cABHnlYq=SJ@kGn=s8ya|FSA%O^jT?;798v0{1tOL{<=Wu#pt-tP|L!iy z&rd=g%O~QAs2eB+8Z0cVvE-OxCpr(fUW=vPWi|qf0f3A+(02#)(PONl&B|fWdsC$_ z|GX;eL`nIt(o#Esa`HxL?!is!1~#VxH5$tRL($4gmsckzoeiQ^dZqTX`+YL!*;CW)9%2#EpKwn9NGU-UR(?+;M29xT7ws2+c{EJirqY+NiYhysv z7cqS{37*jJxbm9r6>xEKE9-jUEJ0_Zo{9ir;F%}^@DW{&4Pj1{5FSqC#gm_PsenDjU&ti53N&27&bBix|81SQeQXk4tj?{^+oV z@ItiLoU$PLSH0dli4F}>veHy@;P1M;B2YGN{||5X3_oXo_F-_{RGp=H?ax;pUzC?S zbZ@U?!O?3mx7~B%-I;AI4ivVvJn{1M#+B!fcLV&EeK!12;Tcwhy-uzd-L{Q1=0=1+ z86mMU+d(@cV@(JYSv2`<;BghEb2k%@stIJELW=0=iw;>`*-NvnPt@X0$^3FeL`&Yf zz`^<&Vq zDEU0ojj&L9FRl07%uhikIDqUO>ev0Eb^~gK1FLze#cYR3;oh-JZ-O= z_v$mECbNH^YD^KSfO@AY5PG2Xrf6^m;80XbNg#82MDSU(v9@OB$g6N@`rUY9i<%RaWgaDkiooyzcJ! z@I2bR`#jx{7Z%-qUHH+*^6ahR0)xv=UphZHU_NS1S+NMXUVvpY14~GwORAss*N*Cv zg6})+|2p(iB^`BVA$CxO+~zX?20R(~Dlj-$1Yg8&lqNg3w``ozq`8I9eu`Tfx+ZiT zk{$W4cP^NJ;SWjGflNg93LtmNnxZvcew4Bv#YGg*Ao_KwAZ6pn^PiYCA<*A)G{4Q~ zvc8hKf%}+Eul+u+Pjen}^!I~NF5C8>DO%riZpem5XIBhr9BS|~g<*>gYi)E0aymr7?Be&5{H!} z3F5bZ60Ht$%lYVfaL3RsyFm#flU1djB6P?&d+!yd0k>bRh<3Q}PkxU+82*1 zY!3x|PsLT|HvFNFvz{*_=cR~U7_^3=XDHOD;DsbmY8I1o&;LNxfz@jQtju;H*2VJi z9@xh%gk^JHU(6qP&%UWW)5EYq7XGPayF_`V$|1-L2I#S0;J(wyHeR> zyU^0%l?0*7%iWkp`j=mS!PyxeBF>fCF zN*zcLapE!Hiw_I@q17+V?q5>+Y|u9kV2m}7*ysWtgjk|n%P{l#me%v9UK3bJ@wK_Z zob~(S{EI-#;vy_ zPYiuB(WO`H1^rd)uO=_Bj*V#9$o_QekkvzebUfYfhhf4hpHsRjqo3rI$9FzuGt2ek z>zuEXuHWzu-=n(e01$mhh=468kMBMuEz(EzG3gTo>hqNxpEF&CWD)my&QMcPKA=IAi@+D<`W&K zbkJrHQD=0Liwg6TdhLyB@=0nyB{C6JL*i^)j8&v1v5Gd}MyM z(3&c)y!sIbjkEdAj2EF4m4hHL?O3Bg*7td4(Gf}4!wW|^LTQg5-{2S(bnnb6&}pkQ z{Zfn9BZb)PV~D5HX3-~V)7{@I-j)@PY$HIs)xDE<%y$8Qict@Wikk#9Q-~rnp9=k^ zO`Fn)ClY&2K|f6z*zoyu5aAQgT;NaJkG|fh*N@k0+>4Jrk1Nq$x@>8eo~W?2whq5Z!<0rTe|tt(1I`5(a?(#)Qt9APD+RzXC#I+O zO7#tCRTf5BSC8v55P79wiKM#g|Ni^+U?!QY&iiRbityX~fGDkv27N?%#-vzkVGncg zE@I5?vqp6-lo5eh_DwRjF0;G;P#?ZW&EVUGp>_O0*<3h3y@9Mjy@TpEUngOQCN;3p zF0)Sc-X{3W&du0xZ@T%>FH@VZjDM}3;ms^{Fd`RNVe^68e}5ml(%JetI^K=IGNnfW z4buYWgW2nQ&u8!6z4#*CTXh{sb4ox5|_$uS^U&J=}GH(wPx$75=pR2?|Cv zq&u5J#~SYtsI028#KFuXPk}J4QuSd2Igqyz)Z_gzfG_}UTztKLaKh@1xO>Z(%<6zqem~ znjHed!dk!}`g1zB)$G^CS?&gywEaOByClO<9EdmTw_p>H#AO6m8 z+O+1~7i_+qe3ipVm^1aQ@)GAAZWZi>C4Mm=I9RMcPHool_gn9}K!scAuwPr1Z*!fG z9g9H|A|K<1t-U69D`@4t#_z7ZR!{Slxt>)5ThxTGZ5Dzda-K*qErD_W4k z?*Aa>G5qG@=2nj)t|F-TsmOJNnN6|q$g}*`KQtsHk${1Jp}cqj&=h_wuxV*(ssmA{ z%-YTVn2?a%Lg(tJx);#nIywXZz+}eKoMz0}?)DDUUYz;y>$%A_x1(9OJ>!gtey4{= zP$|n26x32{=`6&RmIQQvY=Jq8Hi)Ew+NvPNuz6J*bR@i%dZl8Jc?AiFT zgdtm6En2!%`~zb>@|{7uqG(@pu>qaYwTVw<@r0=fUQJ|uM_OS^xf4;VQ&c%$#`dh2 z;40S!POpbU6K~u+rAks3<17@38&L5NoHC_cW6-eezTlg}qB5eqLF{}1SK4!6!#+wQ z&wC8Qp+!2xy!`wD*P*X5L)P%%nl66%58Hfag?KYQi zUaV2qu4gPmSz3ACYvbdmPXo`MEo^7#U}GcJ&Ua#Mp(mUi9rGT>6H5<2)gq6nczajc zhHjFB6Ov#cU*_S2*4?}75;HuCa&?H#L7d9$=VG^~j|_xba08tkwB1LZXwhP;I+7|W zf(P()Vrf>q-W^m{6l$TBHs#zi#?S!m-up)SJjm^?=%I3vjv(5yK>B{{>OYJ-?aaMD zXI1C-Z?@*gAGT{>aIT{dvmKdit+zMssA$}u&Ey2aFgapd#$p<`Q$J_l zAIevD0{%k7R2xHPHC;(2l`-OMipyGF(WkMH6FSi4)*!8R=Vp)AM`Tvl8yPFg;yOm) zEA@^m*(1Xcz=5Jm5#l*$^w9#lpxLi`X zDk3m&(5W)-A<_1&xSnVcSCe+>m_nexuHU&+mzz_TyVI@uS=pEV4(x|P2*mXp%1g9! z;<{o`9^P+7eXL6SwrtVCmnM{7_l+E~w#t?rYEnfA1%|jsoEJC?7M^cqzZM+{Xs3Og zZ=H7B;Jc!Mf(s0#gqP>qr_pnt1xRQonGaA#^6lm{DLbLBub+sYl`X=aXAgP~uzjIH zOr&Px#IFJ8ai(GnY)*RRpKL!htPP6BO*!qTf)Xu|)8PW6+NblNz1z13z{Wa(M~VfH z(0e%+>J74qzllufws~_9e%^|y6!jV3u`bCVoLpoc+F!RJb?QT97!&#*pfx=Tvg-PV zM|g?)Nw8YT7^0LeGgUt$!6~-D${5mZpU4!Mp$*#T^{)_$tI_LRO4hJR<;%PIz<8*N|DP z3h&hgCt)D)q!=tf>+f2k`UWpal#p2VnFCMZ*qGVu*>A7(fjd89Q1ZPaw+H$7X}+0+ zcie*p*x98uUcalV7njjl#+9horYCrB)*2CE_JRO!2 zH~XO~Wrm0(!w?iSiQ?)>Out_J1=`M@JJ*@?>rwu*6YhUv*O0{b8FskqD^u9XM5r)3 zfJl)6@J=BKeytw;xMWBje|kQSycD!D)gNZ&xti#uw|us|ph;bO^^%O6#4T$B?aq z3N-Gdramdk9|e6fao#`zQ{hi)xdGD6aE}B$$P4%WvKbopJh9l^y?3uqg5U3Sy)9Nj zX_wG}2vSE=GnxK^)nqk*a|#c-i0L+d7)H+0KX)@aOfqgWg_Rra_)t%{7dQ zipu->Z3?fEt!s^V#sowo;EW)StMAW=Z`%b}5=*;y(N4$^w1J$)*cm=EDMV4zCY zpW1$!^rhQrvMxxn09aj|i+YnPYsrTN1f1IRVg@%-p?KG?#GwQ8sekK1*cD+4i?d_@6z1OE=mFdqU082ykNc$LljE51iBX4ZnUBDd4d zcn6(i+LI?3A=ILu3c)gRwYtSNo~BjXZAR2rsDEWN0Cg{am+W(mLh9yG{8>-3JCRLV zn($7Ic2FL|m^7j!*^MU%Bu6Y-aYlh}aQVq~ln-l&mssztg~s*T*_$w8y5Mzhrqs2N zS={7#*+p5`-EaQGU&VV>@t>faMgzg%dtXii!k8!bb>ojb>ex?6aa zYBLaAb}Vl4?~3T(-(AdEa^#|~&rPLc#0}Ak;D!crznd?^Xy@EsHR7?z`De+;CJG)S zEJ`&7k9@b66vUvE>_?OaX8IKme;!S@U)U5`zuN!|guna8RbI>{;%zWWQ$O!}-MqvJ z>J=BeXiiUGXld(UNR2UosqE?3eCi>UHRIDk7IjSQR%#*T@}^B%=)?1KAGXf@{5c9- zy7$stZHPQ6n=#v{MQ#XYloFGM^%PsaJ15!`DBiz+?}s5czToQo;2YpWc~DHk*hsSk z2`w>}S0RF1I^xcGy}{1KM8IriQ5}2%W+NPATvKgT?b@2Tk2M;7{OD0u)ByxuTW?oD z(wbTyV(!2Cu&S^Y0+h0czcy~zpl&ux0#PetkLr#qL4$|2n{Y1Y4PBlg64}*3gC8tg zK)$-ctW;iE@jQkvb>T}YtR6nPz<_l_M(GTYvIPQ9y2p>@FDq*c(A@~mfr~KYR0?db z0386by?~rSNa~4$u#L$q%PAVO-`Q!no|+l4+taxIx9I`Z2IzOTN!ba|>4-fz9DfCH zm)hjssBETzOp3dpJ}CZ%jT@b2&8kJxx3;ml!Sf*d-b5(CNMreR1WZg zU#vl$ox*dM=Q(p_C1IyMhTqUvhvInGQn=_D+b2>a@#n*0Tv&iay)dFoX-ZpgbDD*P zf!QJjPdo_O@cu77taV&mPE5QcZdi{dmJAq}pL6DL9ZjFL-P(mp=47S7z`CO>HrCqcF*%Sdtcx`7?WGyC15CP8+SNF~~ZZjdiqg9(m) zn9(LUW)<~V`KzD}Zq(3>3moY=SsxZ08am1;m;%bWXwcM4OY2e|izg*{3*lt-ldi2C zZgaEi-lJ1T)ksiL&;5Mc?qHP8+={AxOP4McgEQR@&O)W!z>=4`3Kq-fdywvw>DEg5DwCd;vN~$em5{t$^hfZ`}%oBYBpf)cKj(WKUTmT4+A!#un@*!W{#Z5AVAS$;d$|^M zhZA0ZqWgnT3FTiisD}+V_Q2*A$@!L-QXWtYIib*=)K)p}-1+md5L0-vHU(=*qMGZ1 zWuzFJ$M;ok+Hx=3qpG40DAO90JMuFqNTTth7JUz-H#Y|Ss&)VV(@u`8 z`wOG?omi)sj(q9hbHdVea(;u5GlecVt-FvuA@r_eWnF zFtDPpTdhXQ_@)G}fp7XQTU3xW6ieKQc^4MUetE7gk@A4cSCWONxkGi-DO++;O>n3S zKsmuAs@6SwHo{wM0Pv=H-;ex)U@E*6*($@?(|-B#r9;cf#v|x!ODWjof&eA58LELa zhhI-iQK+6wUG|XcHMP?l3c^r=>kT}zxv8>jdFUp`LmfswygNvPt5CD1@rSnqDpzE! zQ6A15bky%!N*#}t)31lEncjQx8(o`TPj2{CdE@pR#1(iC5tdh2xaQQU&XNORKIO%D zl;%Yi6=&Sk5D0k6lxAdIOfwr$Sn1%<4K8^Q)FQs`LVQ}pijfJj>-@>hd-tkOoA$CsJ)uQX zQj$Ky1E3s%oJL2c_x_Y|{$unMX1-Mh^ep#d7oxDr#|4QUM0M-euP@6&H{2U}>u6Zt zi%YxJnA61aRjtqch7C^9s7M$Zs+L(45v@|xtb6k?*Ipm)Y&$|x6L5Z+Db|0H6k^iS z7Zn#f(WAA=q|%b~3M?qwkB+h&wMgmvsHp;cU9fca_^NR57lu9P&)#8f{zYGa^K}(l*U)QM9I*#Z;-U9q~BYb{TDRohoWpxNE+2rEr=*RCT zgl2&5?q+L(U=ZdMR=qv)T1p2ACO9D4UJ5H|(s+qA86%{?@5;)`TKAh)I;bjlzN>M2Z1(JzN6jUC($YF3quS#bAVRE0^yvA zuBmt3CGiI26dPKvNPA-)+_NYa8X`Zhp~!kBMEhzs0%pe%I=9t4ki8q3jSKD`v3oY8~4ItuwgaQFoVo9?1TaKgNUG7(K~eagnW_KW6IQw$t9 zkVoVU#)+l96|qc?IJNUZAiG>>%vt~bYVL2c`h#79rewWb>pp$s=A7DD;ak{K^`U9( z#jnf8cKYS}CZ}h|`}cM98m>1U)gzBCNwy~bGRLr-e~i7V>#ZtRNhG`~sU_*Uq{l|? z;Jb7xNSa`i;Hk2Mh=Ql0amY^d#zr-fj;|xa-M|SG0BjKIsmRErOGEfO%gm$4D@t*C zz)tRmLAmjcVZvQe6^ibBb*EwNbrWxg^*!BTujil>K99$oUb3#my1jQ!mxXulej>7N zyn$UEf2Yc0ANZun;5mpW*HOnN;s%%Ea^%n7ZhY<9L==}V5D55D~hF%o%$_u zn8PlP(Lv<+rIe}UUElZQ1ynJ!s_TNPc4o*@4tjU*-aQYVrgf)I^;ERNKNL^#PnveF zTN~xr$9_83jyP9MoTl4j+lSLOeLqil9H4Y>)BT)Z+0%2vmA!_bAnzSTv17CAUV2)Z zbgKCB`=hT?@wlMrgIbb~DS)e(Q~sK{;`M+PvuDp1sd?`e-wo)N*AS%#+S+OfB7gpT zAV5B_rY7Wv^dYiCg2qBcfjJI{q92+S>6C~iAN$lNpvm3lj@}Tfw}R~9(r7@w@7$_H zsAk2`mfU}49!5 zYE-=leMtLCoxqZqneb-y;LDT5ERKiof+BZBOW$l{U#s~jH&RozfOmH@)QVH{iCS4Y8GZWaE`?hcPw5k8DuZbV2%xC+qdwx7(*Lp8x!`3O&s& zhRS6$fg0O>o&3p!IVymSD~p0S#N9Um^b{%vFUo_ z2X$&0ka+WWFGq(2O|KtsAvLYb$KS`~Wj!Hs;PCan#;A~qBhO5y46wFVp5?~{OEM?M z>}DvP@y@mAR&#m}24IJ+3#7%ZswlLTYLf$h3r-}#?#cGXkS>FH5Qi+nz<8X=U?Zvo z&{uB1QuSR{YLQf1PK&IO@~Fw&7`22k&iOt2yzV>upx4lPw;%g|Dbe+4($+5KSleNZ zwk5&FgW-!gJCs;2_no>E?cluxoY1@M>?b z1&Wa)>&<_YP=ktQ;ILuC!X5-mff|xvUR^vXJ?VA>D6kytC8uDPU0B=C#?8269FRIqvODV zH_TFOB&1tARO?Yf6dVIbikXY6N;(Tt0@%dtN6!hiiGsIL7$;JknKC_b5kn0yOox_p z%k02(=xt?i2UjHvL8N4Kh2XO1-X3ALm1f<#ot;|Jqw|=Wz6)OlfWaE{E-*Cs`)T68 zccy7KUfJl>xF{8cYmBR7(v!01m%ptzWbOF@NRwv4NC+n4GgcRw1Z801)SlptlsA5SQbXiwkfO1c-n0C=g+Gzme$-~^`ik))5D7crfu&qtwH+2w*&S( ziVLwFlKA3mPD|ZSf7nM|^+BO2cjg(@mda7bdy*X2Rx3qrgvQ*DPvgWnq3pU2AnJae zjX~^4%KI=7$(AN9y=;k=2AU7$wX~zR>>rh53@t6XX5rvC*urv?dD?%?lN+Iqy^)?` zNcr`g@f2O~Ev+ZM=xvnoty$66X9kTo-M6Ipe3&rM)j(O&J;QmrQ&q(~>95d>yvJHx z;z2@~v(gUH7I`*;ODn4osg5uS9DV9S4MBpXKBkd`t~qhQt-XC-)~H?bEeXXF+b7l~CtKqd zeJRo9HxXbGEM$)LBxy@PE!G1OC9%YUkaZDVj!k%w%3D3vD<1i(;#erb0i$o4oLKGm z-2iNEfwGK-Dwa+diDoiYyde639@py&*O_}fVps(k-8|LBWcrnm%e{VPM5Rm;IfuY{ z_tHlXdA1;QTC-a%JD4^iKw1Hk%cOtw^N5zS=ggr?X=P-jVpRtvaV;Q8frnZK++IAN z!Bqh%_((ORLg8?tUmNrOW+W9+ilo2*ztGV75>3IVuOkP6e!EVLUBpZd*RT;4%uU{( z;LAR7GmB21I3bN#_jzv(Iz%t&z1^e+)L~HP^O47o*SJ<&YxLvj28#Cnolw z)$e`9t>%Ue_;)1I1A)fuU9G97z!h*kcryNEs>nA)z=A5(9~X6zBKHaX#ts~@bLALd z2@`>ofVHRd{8Q>&`TI1mWe*q>m_dKo@Va8_$;^|e;pHseyg8^$y-uCd`N!=sS!vCS zxifTam%pPnO^A7y7-=<$dg6UX8{4Jx(`&VH8*WJn+-YCWqjyOF5rpz z|IagPEH{M8mq$|@7*Iy@gvRL+hq_$!6BOKCoUYeoB-e?mk4N@nEK~z@W_?EESbJbV zbt!TeK%R|sm3qrP$jeswfOSgT-AQBY*5$y`lVZ!xl!r4?x>=QSo) zsicNkr1;-RG4*=h<%-v@2BM6kEQ>`=WRUZ-s3^;LO7(Gjrw{>wPB!LmIs(cP>0~yT zxgu~Y%TwcPqVkdvBpiv?^XD@uv)fNTS4|h=CgJ6ud(DkHX*H&8!jqxZ1on}_nF`aI zt73ky#WDFg@0u~v@3jdfW$&O(${J_eUAi9Y+@>P_lM$w6b!*gEqq?EXJf1!_z}*2e z1Nkum9tG^&*@#&!c5DX9P@KmKA3WO=xo;0S2#mOkehQZD*H=@YA)FGZJ1O$yxpJ$L zZr`qRt)f6>vK&a{ z5!C`vkSK$k9rv;Y{SZ<%wL!1yjG||-r}X(hbNFi850km2A(PG$x%dXEt(;Gr)`NT9 zKp`Is2;940KbPwPbqVtKT{o1z!ggyl?>&=O5Iy&C!|& z3oEJ{wD3+qVT4BKQzY|ASq_A+w3jDzf6igE7Z5^UFs-r|q zc{%OLS{4;TZ6%^~IekO=jIvqa*!4&pbe)OxX)NxT_T1>9Yon^34d#6M)P7P_X}kN( z9!-IQ22a;e({1v*9vM~roQ)lsNtg%`T?J?-`XCZcFb43D8cX`y&=d<6Pr&)Wb&z_w z7CX}ElM|DZv$_pdW(x*J1@`>n;?{VZt%3TD0*HV@Mg`EBh*48KCX*y<*ek%Ff-rnY zeEG6zv!3%4SC zH_=HT6>BDHCK8)^zmC=C_->o(L*T7KgOfzrD$)~*-zLBx!3haIVanT0xq}oaPPBHQ zI+k2kSq6vZ-_ph==Xm)+HwrNkj1=XLmf{FtK|+&6NDiPJ=j${W92gC%zytRx1fcXD zOlk~3!Y0pJjvGl+v@~#BYh=s=cWn32Ost3tE6YC~*-*JC?D{|U#`zz@AJgk@A2xHR z_u09%#czj6iX4v<)Z_rlX3C_56cVuB-{b!+=;7+>3KJ`;OgRqSd-NF1`mCBYYkKwi zdvo={s*G_ICE>%yjDO3MhsdkVR~$(DCV>brtzWA9;Uh;{V`NQI6h8ynl#LYQE@(zX z9LTTyZ1SLGi5xu%QcQ^xLnQg*&Jjumb%rOAGpi8tPW}X-fkuz%JohN2npGX30cTc= zU^3dC&ledTy%rF5MBQ&EPAlFYSZ%B*Z90h)B!84|pjofpwM(il)$gpV_wS3lnz>y> zt4_2$B3?phCrdlUV$|~#bEKF{Nd%zR+FHBF_T*}fpAGNKIAiA4fi``Mi*LPp^{PgV;ucsJ9dMEtIofCy z9Ty=ukUNuvIt4J_5+R06XL#Ljx9c8Y_J&|B!wiy-sJKN6L<~scnj^IuOjeR=pI;gJ z?b4-7dp-`cwY6dBn%J#_l#SjS2Ok+jhm+@F3pXY+UvzjPS zXCS2r@}CH1GQ74*=kw>E+-|>*-ULC0Bw8XJk<~gBd}8d)Ok!p`i==SOQkagDHb4?2 zFq}4md`MRwA4#o2bTE&QM8%w}KjQtP zZQf`Mb9Qb;mmLuuO;u6@DuZ~??p4*W!wnb-y148MLbQv1B9x=Jlm*L@@KGyd_YV8- zIq-kg8_6U|NJ4e^!S7=cGZ;b+ktYWO#fw-7Yg?=%v(_i;XlpBwUhMC?OkMJywkjOH zB@oTStE*Pv%T>5O864~X{j;Z9d6e4>OQ-;ng$uopN8WoCUVB|R;g>E&QQ%3W!r=_w zYfH?`5*#^^EpFryU-7AIO09B}=5X=1mpw%IL;q?CZ6~`H_(a}8-Kx>E>igI#s^a$x zT_K(gGF;C;TqS=FKkgSK!3G4S?dq!hsKdU0?-+JvW$~~w^}2PFkb2>XQ-|kAF?fUA zNoEn{)x$G0hoh&|`x*UP{Z}2aM)#~1EEY9iMJXhVI(i&5J*~QS9aDCJ^C)FyzkdB_ zG$)rC^P-%BNB+O3VQu}ZT0Qss^ZxnNKl}ze)vgA`|L6CGhgN5?|M?Z`cPiEKruswt z`48{f)t}{me^*iS>WA>Zzw6#Fs?4@pNzI{YCP_I`9=odcDBr>i_ep zvURHeRvoDhu!;e-3KW$B1|fIS%^>aQ4r(f(Pn<-^;Rg={I#l~t=nwu6Lts*90GzZD zP*CC=eVK;$GPxFvV>RUO?C_+wz33Id`E{4;Pbz~nQBe?K0}Y1$q~Q*wPy{iFN^{1s zHW~Xp#_uQ2Hsxyl_m6jooYr?CP|OO2_|*fg&Uj^8#~<{^5h7z!0xgcW+~4ZKg|tKr zgCmX{sjZ;XT#>M?81c>Ks!HH2SyTrt=TB7)pD(*txw%>MzS|bRPpTml6>8x2+cSM< z(-`bF_wEa@rrvM+2+&s*#jIJg9!()xnSroch~uZsEi5d& z=kA3u3hE~{1tlN0s`W%#Mu#A%ohnH}AZ^OZ%9|5v+I@2k-@G|*(58LO+4O8@XXpQK z3F-xrkD#dtH`_J0ezRu#Y{R~cMdT<}nOompE{SeXe#|-Wgml8l($R(9bX{w^?0|jv zkrON;qqtbSgZ%K{KV{D6)cf)8WlVLIsVHDvR9##$=a=8-%g9`dwkkE%nxLSS2hyS9 z8UEYSYET-g%IK?3V?qeS|Wc~e7i-T z3Jxk&F4Ez3tp=^j`qcZ^;bE<81`p1l^!KPJHU0kXnwC*bUZN>~fEv(e%fz=U;7r82 z5y>t&q9)Qss$ngNhPJka)YAeIvZm)IRkwMEf0z(aSfW5RpA%+FSEfdtsrWH|IBn4J zmtT&lO5G_mo!|tdH%BNiIt|(Tpm=*grev71nlaxlmMK2v z-+Hktrq|eT%Cihw1=%4Z{LJ}uX-iPgDKI{84BMI}f-?Et3K15c8PNjoY*BIP_g%{S zR1pO5-CSs|?VJ3AEcgh}Z~flJHJD5zYyB4FGVHf@u+T>HPBgEDlG) zbLVk|zz19aO+_jA9uwlHy;3G4VrHWC26`9H_1zM)a2Cebqw!AWTB|59=n;86H%P>0 z`^~+Z((6tNrRpCvVua@P8#ibXH;Q}WSd{@j)6lxnX*0W$1euJGD}1KKo)K;J>LA6D zmBe$W%=q7^(7F0@-DPX8AO=WhMHqE=XQzELEy$4I@f@!fLMWX(*VUxJ8PJ4WDi)3O z<$I=Iz~~6H3%q45pERfMEtG1Bh{hDsCrEVydPws=?n}D>Ay6rT?AUyaLd+yRr8Eu9 z^faE9q7an<1b=@OR96A=mT0fUS}1NxJ+>cd64m|bV1<4KMI8D{&ubl(2a1ySylcXe8&$GvzfRhWT`x`7Ao8P*7M(m7n7oRKa zuC%FmX#%t7h1%*t=Fr6qEy-N1rg6y4_{^BIvqoZBtt*9=3~=zTFfvvS4u*uV)Y#J2 z7o>?HF3%`mDR}&`aevE2p;VV$JhWl-GLW7ky`3{CRM`eEP?@k2HFI`q+yBA?O@e-f zMn@e+fOq_Re}UF2CgQ-&6p38DGb2tLO^Uil{9a#^h_8h1rOkMHTVmg9k zkvqpv4W+D)

UKF*jns!Py-4y6Qu)Vb}W&X-@mNe|(PtKp`9@~hqc^uit$<%v zl7Efbi%0__GC{G|$vdD?8&iNl%aD-^f}0vGS00IF{`&T3*0n>BMx856Hmz@*_Ya@~ z9605D%>BmOuo^RFYmqmj1o>ZIbWrR(<2*U;1pc?&eMi%ME@EGo-0Sh?@ zP)s~^!-+$SF~1RoB0>Yn<9DkquW!sG9hNsc-=^$Wcc+`&vURI!{M4|_%EAP~ne-%# zJCg#mMY>C-ck!d6hnvNJFH&z{!o_i%c+x0RR_Bc9|E^0`?qHC0*p8we?%E#^=use>lYW*I6j;Ggnc}g@RT;hVKJ<>n zP#SvS5`~q)m_G6H%=`oU_oqdVZ3pftC>CtCj9%pa66Gh~_!d;gp`I=-E_B>tJhG>( z-Cuu2@L$I01fkEH(RI&uuby4H_fz@)d`;zvVe17TPq?qJsJg}Dj> z#QJQVkrojUA2@3;uS^bS1r{=5ZG%NBqrNIKC(DOQ30TAW71X?Iw2-h}idDNvwo8f&Ww6MEU^{7={7Nw&;AK7k0VVoADRhah3r zifyt1XY2^| zK&!n2QPC62XT7kEwqM6+^04&9=mf2pI2ML`PZ<;)ABe#JApbdG!>aiBC@-%oi`?DE z_B-;`e0cnp&6`^i7HcYCUH8mnh}q~SnWY0~GZ{GT=_xq(fF~BFrlZRYzQA=53D$zE ztb{zj3-=#+tEqUD5wBa}DAd+Dh4^>eWVbnhw z5txdl6k{oww)=XH>>JL~lbjCA|1XK$y{vq5fA(6s8-~h)VLH%%0IqD$eq1bV1!q|H zV^9Pg_r}e1XeFN>MsDA?SjGlaT28RNdmOB7n6i(#_R8>Hks)98~UnLGpkY_|BmXWpSX?pAD z##gRx0MCt%v?D9t6;I+?NFj_yB9`053_yKi-C#mE^q@9r55dK(bv7Wnj6}+G15a0! z(=zo#l8X-=3i%B=rFi>h3hE~*fcO(ZRB_SaSfopgQi$=g+;Flo$9dM?vCAB;`ua>7 zeX=`yTtxNk%oUjXsx|$ONqSCxWJ%8GH{p|tAEtSf93Kz-!?#(@?X+yWZ27nc7T& z^=^BKU7}Y9Ui-UvbaRLE8+_s3(J~1s2dqxRJH(rlvw`A72v0!z&EISM-(e~95@@2R zo{U>>HX5i<&apu+CpI2{%gXtc5Zraoh{)sb88SszBoh3xLh&1EJboXO z2a}EWLXCCykL^gxd$Hia=4SXLl43`BFt%+}Y`;4op^3jAU>G71EuYR~4bZ zj1|#J%N!RPtiPsh0`kl5(0N?a4K9j`0y^d*16C*+HZM*kMb%mr8;4qM>5Ab!nSx=;JM6vZ~)s_hvSHJ1L^-T@Xo@;vaZ3&v@ z3^)tndUyn}o`tg%i9{mtf9B0|_XjskXH-Jc2~FmsxW>SEV^`U_XWl6Mo7z=IsaQCo zVrOFIsjLzkVZc?p|4(!0;@9*2{{LqYDcfSToE1Vz&YOlx3LUl%O3ESPgR*9hi;7ru zGUv0AQmB@5StN}NGpE$DOvR9p2&Iy~_bc1%_aFS+KHKN>nW^5d!}Ywb$LS(7BWWVl zy3oRg@Y?7B$cCxX=Lr_OhD=Un^9U!D_of?Z3duF}+<(BYnzLY<@w0QdDZ-1ZY~)Lb zIc`o^^Qy8^Q+9igFHd^&(Xd68c8kqUB52Ck#jex4xu=U=1}E?OwKB;1j|psX@?Q{1Eq-9y-E;Y$IDF_Zwv3&EJj;#{ zJQ3rROq>$-sKxyJ`fnjWon3BnA)6?SD|v-f{=6-W<-RCDJk)ipsNM;0zQ1rV&Imfe5j5-NnZRNF zh^k2{{@Y+IT4!|)lr$;qKjX8Nbt8m^#ebk|Bw(whHr*2(`*$%hF)v>(Vtb#T+TUmV z+c=8sz*)bySDS<+6*|A0n{97pwuK)vH6>&*bCHTreaRz zY_Uk8dJ=ss$zena?JEoc(i4)Jv&9w6iAJr2BTythQsha{ENge_AaV><)O+56BMTlA zLA7@CX3U(;jGK}(ahRRm&{?7g5Hf@L zKQW!!iOP1^jWOkm5*>d);d@qsTyLJp$VeKttrHU;gR!*MN_80gjpA0W-R$3hd^2%V zM-Fp&&YsM3TCh~K7sL^-FC^Mqw8a=tNZI=N{P|HQMj=^^+qW-*W<9OCrILsd{>eLc zf647o=J116St&XL)MSJYxoQ0fx73+}i15XH3o|+%_g2dug?jbW#|*jlSnFkawy8l= zK)A&brgn4d`}Xato;|gagHY{eyp;?FE`S7=nE zH4Jj&7WDct;6%~AGNULwoY@DD)iJMClhJq{kf$q+YBSz z9X>jOpLFJ|S$nYHi=CN1Mr8&Eyg9q?EHP}@u=>9g5`shQn~I`gs$xAJk;M|W6t1`R z^95#rx{a3QoE*CMR!oUGe9**wheE-BXlTDm(loLmf#QDjTK2KM$2(u>3Lh)JY)8->xf>qlN3ax1{en z4`ZMA%RwM#(nl|SlJx9nbVS)F&uoLQl?({F`3r}GdVGee|9aer=4t7il`6<9WRICe zBSDlIq`^xiN^ag58#XT_#2T9=+qUx5u;=?GJ|@Aa8Ihm}#=6}Iey2Ze#NcDM;WL~q ztu)O3L1m;O!Org8Y+&j~XE>f7a760FBo7@HefW1`pj~ri(o+vMGa_jx_I-sLAA(IG>%yTKkhgzD?8pbgH-n?VmEqr7Gq?wfB8 z94O!nJ+dm1wGy@#TS@~~Kf$v-Qt2V2VtE)qfiZp*GZ;oQI^X%`jX?)8DeFI&*Gv!{=5U2QzP~6-mYg? zhdFOV`03M2_dW+Mu|=-J3B&vU<>+}p-ab9`qs`Z}?a;xT;Dq&$+W@eiLoj4cZ?v&2 zjd6Pl))n5#9{1*Ez^i_u>;RMO00TC^AVD*RRyLZm$LaZrlP7fQSZymPjRRrcZLrlmh&n0NIh~nNyfVAb{B0QE+3^IGT@bW(LsVY0JaZmCH zPPQIGe+VqDp)iu@)5ed&D+`<>=!NLm1d*o%Ss?4g@FpEMSk6y#@9T(9i0|WAmwejA z$w^lvnLuKxpEz+dwtpHZtK;Rk1rd?@(b&7J<{@G%iswlB!@~D11q*Xx9Jihri}TVT z@!t7wt68&+Hjjh7xdME&C#hWGKg&+s^CI?uXHN_K!9Q4Q69+2s+Tx029I>>xYysh} zrx%9AkS24a&v4_@AT5}^n&P}OE^s-jYb;fX3S)j_mZauFjU{*Gm7X=!;q zz?ER*1IBw~0>;e)kQvSef3e_8E|bg%xJ4y;hTKz0Yq0^yOt2zI57HpuQt9cd1PU=I zG>VbHI3_a%t1v_b<@eSVl;)+S1fhyFh!K$2UFI*0 zw76u&r~eImfL_Sgx(8%!mY8f{rIWr0*vgzCKku8brwN<&fmz=yWVrl6`u78O~5M$!l0#J(Hnr)VqkNcOPD-`8!EMKN=YL zhILuKNO*)!0$3Jwp6N-c6}E1pTm6@tpHf1_;Dz5$3kEM^C_5d;Hdt0ySC4|a)iLh> zzT47<%4~O&!&t_H-nlG)2r$8KWjDR(}hIH-P1QEap0&MnaIn?;!%|j9Ou^1Ro z#@%`Hq$RC!oA&J`38vV&(wae!kf+xP!Ry}NzQ~`*{JZ{b9|W2}+A0haADu=MC4ON6 z+QIZOS-EFDU@=KQx@%g+58xtCX8Y}ZL*Cc5Yp>HE)ya*AP%UAzkvPJ1hje)VDdJ~u z9^0rMh~47ceZ$-OB^qI74sR%MF1v;hy76%r^A0??j2_`2h4f5u2M_w*G&YEw-Mi+a zH^rg)9Xd?7uMi5w8b#b48FDq%!)(Ciy55pd!U3@sU|Y#( z&#@|*qh~&NvUKAktHr#>#-juq`1{w(z`CkVA9w2#Kh({{OXF3(^0+uVKU&qqIleh< zU2$BrpSd-cruCN&UWOc|Qyt4A%ezvx3=O?*PPrF7!2*`};X{3&mb^g4zrBk8(R7m| z=@W(vCGP9*Un`;;`R9ykT~*{#m9oB?xCCw>B-V}0;^knjno9aS25Br?P1QAIa1%~1 zIi=UN*I?m)V4M5*?+=l@#{C?xwfD?^lG+IhMJ7hCV{LSt*vABFQ*}A@-t#y(E6zeq zq;uv^T;Jbmoevr*`gs+GYNmgE@s&Qw`wsLrH?JIaJYaQoDI2^4ZhaUMNO9oY(^TV5 zEgr-%`HUJc1);498~J^^JKVJuZT*%><`U-x9>NWQu|oGjmg|n(G|1D~LG*jWR>GMt zeLl6bRme}S)&r>8a(W>x^qtvH+c+W6p|TfftrAjmx%PWW(Zd}%GJRJ}Sg5764r_)5 ze3+jJSSfHfdYCzAWJ9jjSzf$9*YuBo$&-~5(2K%yh?qPI$Zm9KI9hxD*4KL5qoi6| z|F7!U59jVvzq@aej)TrTE#+BhX=e5)rt?Xf<-ve7hSmq7AI-3t61Yq(j4;+eXJ;Fo zU%qLL+wB(=PA!9W|5lajpEWu$iZiNX&40i4`gG^ITKenj`cz#;%)XfLGxZhlN z)Qf6`mr@FfJ*;Uhr!4)N%sb+JDEaHJ??hxf{|9hS zRB6&ZBM9!&z5B@WK?cliOuNIi*Od*BFrat8UY-(7p__%AuYLQ<3kgSQ`m6?EU^U4$ zhI~ZXh9kTZ=m746$_)=ZP0FlSd8g%seyOD}%>Be>YH(;_;yt)|mn?y86Vidr_MpuT zpwl-2A{As;^7tU}a%Vvu=72X;F7c}oe-b~UV{uBfQBmoWC#!@j;mW8m=F6!_ROJFA zDt3*t)p}We%aB^))}G@8feq-rLcO2V^RAP3R!GLN=CySYE7u^lgzn{gub{Eo*woWt z5`1@QO%UH&0z`T5IRzyAE~JN-r%SP+hlb6#acxAL13+0q5pzFDh?p1#I7-;=IMK28 z^i|3cjreeJJpk%T3o$qMHHe>R;3#Zj@PRN%K6L7{3hKSDPLqP z4!z~Kf^O_Do?FzUo5l{l@M~SKl@!~l!16z_pF)-%QlgV5LXb={LPo$V_e+U78WNl0nX;0y`yeO_fi08q&6{md?Mui}W@hG*Cy$>#l`uca_yTwGy7e~v z-$ME@_?OcwTerG@1|&L$T;;nU3QB-fA*FG=UfI~egKYt5#v>w{!=W@`kQ4kEK?7vY z*PCB7kZviv8xkuz4-E>FSQVOpY-o||^e3i2SJ7dM`;=e|x+Hw`ACNu6^tbi?pUIkG z)f--BH<(QHtXp~-T1uaia26d#;-`S-ba#@Jxqg@4l`rmA!3T(%5_&(c5WWeK%+{11 zq1Mi;iWuZ;Yiswr#+?~;KI+o?mLE6_DK8xJ8nJ#}z3TGF3&2FYW7K{Tgs1mQOiZNBAAdp)BN0R(3+U=n z`rzKZaJt$!-aUbmITbzvMHFQih?)0lx70@QL|PRmAys`^M*r_@ekQ#*Aw{rgg`lC7q@qPm8P~e$+PZOZ)(ohD&O_EQ6(`O04j_ zSjJCQX61$4C?@7ox{Ua z+T%g{MG_a&+P7^x1+xYG-zeT8PS_P(YK@?3?@7mIw&Kt4uCn}sbRqB_`BBI>ax8}v z6MHMGLtb>y?F0xVqzq^>bGU?43Oqv(D#lBBjjJn)M! z?a*q?v`L#=7ZDMm)|NSS-w=hyiY@b|ukh$dvZQxaED!)~)#ZT=l<07Z9Ms|_GZS*` zEf`TH3^DOAa3nAU&&O)>Qr3*bphCFZ|1GP`wUvn11IlT*4V2xqX$d~T9v&MRtMn8~ zr*!!rNGx&>({*%tH8K6GZOVvFrh8gqLW0b$QH+ADHNZ`{r*O~?uym%*t5u6RFk?_s zMWIrTbM0wIVOfDyiDIz|I+U4N;yW~Ks9ZCKxQt~0*mVEZm{7#E&Hz=;A~e@FwxbYe zDF|jGjLJQk;JN{4(S#mWM(ob8U>Z*TGy>@&>h`r+$B!Nr1qC|?0H4nRD%$(`#WOxi zpcv_RQPeV!2n64TV0ii}hor$5^S|!Aeej%kWwZn@JlzNoN%ZsjPXUcIY*-%an|j(k zNuRQ~?n)}_9%a7HttLLdGDP41%VNBK^?RQoT7 zvP{Fj6Ba#@3X}~?@kvXbKaq|kf5~csMx5$7GVBnsa4q36bP!Nou6{cR?@#jJkk^Ow zM=@Px9Qp9RCPGGFEP>JLY;V+{ytGt#^K3r!WEb`L^KSLjvfwD%}I ziF-P!f{Wc$u`+pVQaRnRfy!YLgQWLS7)c5+-N%pp!d z59~j{fq$|2be>C4JYs)#>#sqdK}b@oX3d&m%^MN;lm^Ds6cg>Fv_&|+?3u?`aP54S zzW}6PTZ^#W(ea6<%tUu7N?>w>0ydIe;yz^3+_^`v5v_h1%GlQU{0*3~S{aGhDlnE4 z(Q(B`yrW2K(ze1r95M91@S~ZZ+NbcbOPMmLyY}T-mC11X!&*Hcy6H{$QS!@ybt9~f(NMc?=ZmzqM z4=;}|$_%gaQ2FF})!YpCU}mhuDj@EpdfHK1T{4t>Mh7yg>NzI5bGYkrQ8a826*_6h zO;XR6O!=9D*MDaLrnmtH+M+iy`?c)mVcJ{$cyKBA&b<8CG8d=K+Cz|nilnHwpr9cX z-c^?6WiVWn7PYHR`R;x;#rQg)T)x%&j?S(jrg6S(Rbib*6M!<6W-*dh!0F;O%D1Vd zjt$Db3GHGxZtRwZtij3ULh}CBn?=Kt-ft-3#g5kkZyp*Rz=CF1r(Y0a4!LoYs+nwZ z&y(R+ij>bS*z|-9KBO5tEqdV$*<01^pqYQq@U^YyA^q4;h>4C^CO}Z9z|L0NQkgG^ z@c!X`v^M8IWz@l?q=_EVB-h|RVc)v+D;x{Zs*?%8vP8dEV_P5bzwZ2kF)o#M*c7_ zAr~H7PyJ<;^Wjr{K%~nHf1TDJa$|S;T>G&)N0bqr^@;AFWdT~+JYt{l{0E%VO@18u z=JlK;J$sr5$GllHXO4C`c6w{Rtn*k>CPR;j~~x_T3XseK;+P+^y1DhxJr((Gsv+?)07fr9PT=P zk)7vg97*P>e$U>T>DXqDUtp8(wczW2>JrH(vx)+HLKb`L=<1HS)euU^rBV=`Z`cLP zEDBJBX?yFj&QGB^1J-zO$zfjS9r>DuWC5e^^RZLVV zui6uA1uq^b!nMOzUwM@eIoKa>by+blx^U=#gXZM}M5m5>{fkF?Jk0Xn8ipOadV+-` zdF1#GB4SncIf&1SrioJ&&Bki@R$~GKZdiVVt(32nTaC751db&_0#SkKGOV#&P%yr2 zzbsh=1{3ye@J6n>NG8D8q=yu}JTQtx8!M4r&73){KK4>|^~uhkeLZ3C<0PULgd0)x znh8~s^3J~;5PEgzkmgEE*^-vdtAo6`+tt#&@rzxoGSdN=CFfPyL<`GPxNiwhnmF7^ zeAiWy%b_N}dHoy|Fn1`HGaHO(hp4^U#g(zgFb=lkN6}CWh8oW-GzQ6Nj@m#a zgx>_qPrO%A=V0ck*{2u7NaKt%`rTo z3F1;o@Zy6A_JwnD=lu*s`QCx6*-R{~Gwi!c*!n_!!G^h<@aQ zByyUgUz0F0ox!X%6^G)NdkYME`y^}*x{bM)dLF?H*(B)O4_>5@(xt-aa~uQd_HL4L zFB?uu8lGC_*Ib6FC2oM%+m1&N(seeY!=X8UjU?mAuoKf8MfxlOnj(_nafX1`W|ct5 zx$U5j^yQr6Dw{_EiW9uVjTshl_7oG$Xcw2}&z=_Q8uvHot}0qTBGM29G>0Qv(iG6! z3rr4WgEn)I&vhNKa5xq`3>Eh!>}0}U4^|>>3;VWU0bllyB5Skc3}`BWX6X%dD5JT- z+6qiYa^1n@(FCmI%$CGl@>wLFOY-u#;*uMg-C(efW%UwJYuTxRzvsRZAE|0w#Kh#$ z|CML`Pz^<`#8jaX^zCKOilq}K{4n2lY0yqj>C_FT62j?PUc~-!(Dq{kkMG>s5xJ*X z{gcNiOH~P%C}3VYLjN0UZ1~aeJZ&DB(j8Z>vH7 z)(@YDv&#!KZyo6^H)Z(}1Asj6BZYumbFF2It?iLCXel^@X}3mgdI)$B;4?Q`jm#XKpo5+b@l3Hfp{K^I!qV-#HE)k7tY$0(Tp{h@184?>Ckf_t zDbXKF%q@UKnST&$m!w8!p9~)EX++Q7TvnL^Mu4wLN(nudYs%Q({#UqcfV>h=B^&C% zRS>?dOIj<5n^>ikLZu*|8%Ecb$|%1NT!BcQHm=i=39}J;JB{+e>eBi8@6D` zNh^SLA9rl#x??U?G)Cbp203=V4z)u<{f~U%9|Ak@sn6u(bh~l5dCN9Nj}4=fXfRfb^jyP+ zU~y}&!&!Z-tR^i!2+@D^$pCg2nI1sRI(ujJyD{RLAx-wsF6uB%_%NAs9FXf_ha!lH6sWd`}*4@~S;{#gT` z$ax++Z9SFVfEicxd2n;HUZC}?S>3A_FWql1+dhQS+WXj>>McD*(pBss3DwvC{By#e zr7+Kg+7#&-2eUu(eKK2KD zAPml+lye7Z@M*s7U^vc!>*oMRF?B%m{4clH~Bl__e6fr_zKaSbADV$Ui<;|F(cJ)IJ~}nK~NJ zgXq1)LPauB#qbm*ol7|}3H582&p-D8x*XeB;6q#665e6TRxmA9c9gZQoR+zvJ|N5D zLM|(POoY+Ah6l8mfBya7h$CxjIZR_<5_0Ek>dt7hND3X6OyId>2oehj(xSgTRK?^Enk-)34;t;(;QQWrRGfq5^@n z7n>E-;Q}{yk*5tNw0QUCL-4d-PMOnRC9-`1QogsSRmd!Zz&%!AME$L<}G?g;WL#sX2mQlax|;gbYg zj;8r@3Y?oUbd0re_|42kM+wcD%j0a)wQJWgnuSc&qLF>{0*QoZlo;)E%}c3x28ag? z;Pg{~?APja6o3n+##|Ct@8W3mqi6Gj)$_xGx+wY;?Rq>J0A~o1Dz9gUV$#FBqzw!b z5#ut04nrbc8n}?dO@5mXGLpeR8VvM0Ki*d{{XKlqkX#*D5MkfkK3ZR534g8Xbn$x{ zq9~3nT>`D0`yoq>gbe6M5bX8SOB>89=J6Z>N*F;n$Ri*!PGux!3DjHpe!(vud@ud{ zR;$nx^D7u?V%R?x$r(R=(HGdGdQ#211CAPT?GrLkD3J)Vt>>i7pO{ut^A4CPriQz` z)$vGbCF(?;iMzaL0nbs{KL?$N;tIr|GKu6bZ5d#(D4JK)@K!ZGf(Va!Yrp~jSW-pH zX^L6$s&jdAd_r}M1<{1!Z?LKw`_Ds8NQvPz+mLBfeF*EAg7xBzWCzklr~U7h?sigX z4;*+pV~2I0Ib=YM3v_58)(TZ}dn5<}p`sB+f;{0W1$Z@A><)HIXa*a=V=*3_x`Jr! zVEUg81w#K_y1R51@5D=ayuq94v*XAQjWA!xU)MrdjNbQ5?BMK7hj=+Ssx=Fk^ zJworW0T)iwTuNxL=)B|i>;a1rjR@tWF-LXiq9ey#PW=kEa($mmXF9z&+7)D$c3B4t zA}Gd^I>Z-{a2x>GQ3O=c6#O>y@;pZMmhA?PF0_zXT?v~`sqme#KN11 zKG+P?R&<9n`0(<}C5Ml$D#I#^%EzI&cHKJUqL+uBdut*e4Y*kRx$vT4kPGqXKUZ{= zbsg`1T{mnLX?KvleKJMj8;EYSg%9>^Tq}%#*7V|jKa|u2XJaAr4s2LlTPy^cG&{lp zq39@N$;D(`T};h)B+_!-Iyd8yHKDpXI<^=9xwcJ6EKZt8qA7q4-AxYq3t9n>lJkUU zb9D^H&1w424G3Bv8>oe((KIpU0y5LmzvO1i zLAk$1BJh6u?Z-nSYGb>GOURkbXA&w6%&~3kpq?{9H6*i62ox*t_b}B<1HZ^iS)VQe?0`TGXx-EYON953l>|NZs3 literal 0 HcmV?d00001 From 30e8b3de05c1d6b75d8f27b9188a1d34f1589ac5 Mon Sep 17 00:00:00 2001 From: tzy123-123 Date: Thu, 12 Oct 2023 12:45:33 -0400 Subject: [PATCH 015/104] [FLINK-32896] [Runtime/Coordination] Incorrect `Map.computeIfAbsent(..., ...::new)` usage which misinterprets key as initial capacity [FLINK-32896] [Runtime/Coordination] Incorrect `Map.computeIfAbsent(..., ...::new)` usage which misinterprets key as initial capacity --- docs/themes/book | 2 +- .../partition/hybrid/HsFileDataIndexImpl.java | 2 +- .../partition/hybrid/HsSpillingStrategy.java | 24 ++++++++++++++----- .../hybrid/HsSpillingStrategyUtils.java | 2 +- .../ProducerMergedPartitionFileIndex.java | 2 +- .../hybrid/TestingSpillingInfoProvider.java | 8 ++++--- ...urceCoordinatorConcurrentAttemptsTest.java | 4 +++- 7 files changed, 30 insertions(+), 14 deletions(-) diff --git a/docs/themes/book b/docs/themes/book index a486adf8462c0..3f1bcccbfb247 160000 --- a/docs/themes/book +++ b/docs/themes/book @@ -1 +1 @@ -Subproject commit a486adf8462c0abfc9034436ddd72927d6656809 +Subproject commit 3f1bcccbfb247da44ab5410a97576c0bf6da103b diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java index 25012ffa8f1cd..d4c0fed0a0b62 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java @@ -160,7 +160,7 @@ private static void addInternalRegionToMap( checkArgument(firstBufferInRegion.subpartitionId == lastBufferInRegion.subpartitionId); checkArgument(firstBufferInRegion.bufferIndex <= lastBufferInRegion.bufferIndex); internalRegionsBySubpartition - .computeIfAbsent(firstBufferInRegion.subpartitionId, ArrayList::new) + .computeIfAbsent(firstBufferInRegion.subpartitionId, k -> new ArrayList<>()) .add( new InternalRegion( firstBufferInRegion.bufferIndex, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java index 6d3a15d427a63..70a1c0b985e05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java @@ -125,36 +125,48 @@ static class Builder { private Builder() {} public Builder addBufferToSpill(BufferIndexAndChannel buffer) { - bufferToSpill.computeIfAbsent(buffer.getChannel(), ArrayList::new).add(buffer); + bufferToSpill + .computeIfAbsent(buffer.getChannel(), k -> new ArrayList<>()) + .add(buffer); return this; } public Builder addBufferToSpill( int subpartitionId, List buffers) { - bufferToSpill.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers); + bufferToSpill + .computeIfAbsent(subpartitionId, k -> new ArrayList<>()) + .addAll(buffers); return this; } public Builder addBufferToSpill( int subpartitionId, Deque buffers) { - bufferToSpill.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers); + bufferToSpill + .computeIfAbsent(subpartitionId, k -> new ArrayList<>()) + .addAll(buffers); return this; } public Builder addBufferToRelease(BufferIndexAndChannel buffer) { - bufferToRelease.computeIfAbsent(buffer.getChannel(), ArrayList::new).add(buffer); + bufferToRelease + .computeIfAbsent(buffer.getChannel(), k -> new ArrayList<>()) + .add(buffer); return this; } public Builder addBufferToRelease( int subpartitionId, List buffers) { - bufferToRelease.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers); + bufferToRelease + .computeIfAbsent(subpartitionId, k -> new ArrayList<>()) + .addAll(buffers); return this; } public Builder addBufferToRelease( int subpartitionId, Deque buffers) { - bufferToRelease.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers); + bufferToRelease + .computeIfAbsent(subpartitionId, k -> new ArrayList<>()) + .addAll(buffers); return this; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java index c166f4a1c4b46..5e3786daf0c9e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java @@ -73,7 +73,7 @@ public class HsSpillingStrategyUtils { BufferConsumptionPriorityIterator bufferConsumptionPriorityIterator = heap.poll(); BufferIndexAndChannel bufferIndexAndChannel = bufferConsumptionPriorityIterator.next(); subpartitionToHighPriorityBuffers - .computeIfAbsent(bufferIndexAndChannel.getChannel(), ArrayList::new) + .computeIfAbsent(bufferIndexAndChannel.getChannel(), k -> new ArrayList<>()) .add(bufferIndexAndChannel); // if this iterator has next, re-added it. if (bufferConsumptionPriorityIterator.hasNext()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java index d6a08da5fb772..3046683f47535 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java @@ -168,7 +168,7 @@ private static void addRegionToMap( checkArgument(firstBufferInRegion.getBufferIndex() <= lastBufferInRegion.getBufferIndex()); subpartitionRegionMap - .computeIfAbsent(firstBufferInRegion.getSubpartitionId(), ArrayList::new) + .computeIfAbsent(firstBufferInRegion.getSubpartitionId(), k -> new ArrayList<>()) .add( new FixedSizeRegion( firstBufferInRegion.getBufferIndex(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java index 9980f4176fa52..53b9df0f55a3d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java @@ -198,14 +198,16 @@ public Builder setGetNumSubpartitionsSupplier( public Builder addSubpartitionBuffers( int subpartitionId, List subpartitionBuffers) { - allBuffers.computeIfAbsent(subpartitionId, ArrayList::new).addAll(subpartitionBuffers); + allBuffers + .computeIfAbsent(subpartitionId, k -> new ArrayList<>()) + .addAll(subpartitionBuffers); return this; } public Builder addSpillBuffers( int subpartitionId, List subpartitionSpillBufferIndexes) { spillBufferIndexes - .computeIfAbsent(subpartitionId, HashSet::new) + .computeIfAbsent(subpartitionId, k -> new HashSet<>()) .addAll(subpartitionSpillBufferIndexes); return this; } @@ -213,7 +215,7 @@ public Builder addSpillBuffers( public Builder addConsumedBuffers( int subpartitionId, List subpartitionConsumedBufferIndexes) { consumedBufferIndexes - .computeIfAbsent(subpartitionId, HashSet::new) + .computeIfAbsent(subpartitionId, k -> new HashSet<>()) .addAll(subpartitionConsumedBufferIndexes); return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorConcurrentAttemptsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorConcurrentAttemptsTest.java index 65a82027c13a7..3f3d0a1f63e43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorConcurrentAttemptsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorConcurrentAttemptsTest.java @@ -266,7 +266,9 @@ private TestEnumerator( @Override public void handleSourceEvent(int subtaskId, int attemptNumber, SourceEvent sourceEvent) { - sourceEvents.computeIfAbsent(subtaskId, HashMap::new).put(attemptNumber, sourceEvent); + sourceEvents + .computeIfAbsent(subtaskId, k -> new HashMap<>()) + .put(attemptNumber, sourceEvent); handleSourceEvent(subtaskId, sourceEvent); } From 22c1eb44df7226c8e9045789e45c71c93668c644 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 24 Oct 2023 12:05:02 +0800 Subject: [PATCH 016/104] [FLINK-33344][rpc] Replace Time with Duration in RpcInputSplitProvider --- .../apache/flink/runtime/taskexecutor/TaskExecutor.java | 4 ++-- .../runtime/taskexecutor/TaskManagerConfiguration.java | 9 ++++----- .../runtime/taskexecutor/rpc/RpcInputSplitProvider.java | 9 +++++---- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index d42a56585980a..680514f09a6f1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -1470,7 +1470,7 @@ private void establishResourceManagerConnection( getResourceID(), taskExecutorRegistrationId, taskSlotTable.createSlotReport(getResourceID()), - taskManagerConfiguration.getRpcTimeout()); + Time.fromDuration(taskManagerConfiguration.getRpcTimeout())); slotReportResponseFuture.whenCompleteAsync( (acknowledge, throwable) -> { @@ -1610,7 +1610,7 @@ private void internalOfferSlotsToJobManager(JobTable.Connection jobManagerConnec jobMasterGateway.offerSlots( getResourceID(), reservedSlots, - taskManagerConfiguration.getRpcTimeout()); + Time.fromDuration(taskManagerConfiguration.getRpcTimeout())); acceptedSlotsFuture.whenCompleteAsync( handleAcceptedSlotOffers( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index 2ec461b40cd23..c3c76220c1050 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -51,7 +51,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { private final String[] tmpDirectories; - private final Time rpcTimeout; + private final Duration rpcTimeout; private final Time slotTimeout; @@ -79,7 +79,7 @@ public TaskManagerConfiguration( ResourceProfile defaultSlotResourceProfile, ResourceProfile totalResourceProfile, String[] tmpDirectories, - Time rpcTimeout, + Duration rpcTimeout, Time slotTimeout, @Nullable Duration maxRegistrationDuration, Configuration configuration, @@ -121,7 +121,7 @@ public ResourceProfile getTotalResourceProfile() { return totalResourceProfile; } - public Time getRpcTimeout() { + public Duration getRpcTimeout() { return rpcTimeout; } @@ -195,8 +195,7 @@ public static TaskManagerConfiguration fromConfiguration( final String[] tmpDirPaths = ConfigurationUtils.parseTempDirectories(configuration); - final Time rpcTimeout = - Time.fromDuration(configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION)); + final Duration rpcTimeout = configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION); LOG.debug("Messages have a max timeout of " + rpcTimeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java index 31b4ae67ec8aa..0afdf05f0ebb8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.taskexecutor.rpc; -import org.apache.flink.api.common.time.Time; import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -29,19 +28,21 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; +import java.time.Duration; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; public class RpcInputSplitProvider implements InputSplitProvider { private final JobMasterGateway jobMasterGateway; private final JobVertexID jobVertexID; private final ExecutionAttemptID executionAttemptID; - private final Time timeout; + private final Duration timeout; public RpcInputSplitProvider( JobMasterGateway jobMasterGateway, JobVertexID jobVertexID, ExecutionAttemptID executionAttemptID, - Time timeout) { + Duration timeout) { this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); this.jobVertexID = Preconditions.checkNotNull(jobVertexID); this.executionAttemptID = Preconditions.checkNotNull(executionAttemptID); @@ -58,7 +59,7 @@ public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) try { SerializedInputSplit serializedInputSplit = - futureInputSplit.get(timeout.getSize(), timeout.getUnit()); + futureInputSplit.get(timeout.toMillis(), TimeUnit.MILLISECONDS); if (serializedInputSplit.isEmpty()) { return null; From 6ebd4dc3e4f902701d64d3709d47d971f1ac4753 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 23 Oct 2023 15:43:20 +0200 Subject: [PATCH 017/104] [FLINK-33342][ci] Adds target version to Java 17 CI build --- tools/azure-pipelines/build-apache-repo.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index 2375fbe984a49..892d2a519dd58 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -146,7 +146,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-20.04' - environment: PROFILE="-Dflink.hadoop.version=2.10.2 -Dscala-2.12 -Djdk11 -Djdk17" + environment: PROFILE="-Dflink.hadoop.version=2.10.2 -Dscala-2.12 -Djdk11 -Djdk17 -Pjava17-target" run_end_to_end: true container: flink-build-container jdk: 17 From 6b0215a213491493edc590834ccfededa3f28ec6 Mon Sep 17 00:00:00 2001 From: polaris6 Date: Tue, 24 Oct 2023 20:05:06 +0800 Subject: [PATCH 018/104] [hotfix][docs] Correct query results in DataStream API examples This closes #23542. --- .../docs/dev/table/data_stream_api.md | 18 +++++++++--------- docs/content/docs/dev/table/data_stream_api.md | 18 +++++++++--------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/docs/content.zh/docs/dev/table/data_stream_api.md b/docs/content.zh/docs/dev/table/data_stream_api.md index 1b1b958cfa0d3..968066c13a7e3 100644 --- a/docs/content.zh/docs/dev/table/data_stream_api.md +++ b/docs/content.zh/docs/dev/table/data_stream_api.md @@ -107,9 +107,9 @@ resultStream.print(); env.execute(); // prints: -// +I[Alice] -// +I[Bob] -// +I[John] +// +I[ALICE] +// +I[BOB] +// +I[JOHN] ``` {{< /tab >}} {{< tab "Scala" >}} @@ -140,9 +140,9 @@ resultStream.print() env.execute() // prints: -// +I[Alice] -// +I[Bob] -// +I[John] +// +I[ALICE] +// +I[BOB] +// +I[JOHN] ``` {{< /tab >}} {{< tab "Python" >}} @@ -173,9 +173,9 @@ res_ds.print() env.execute() # prints: -# +I[Alice] -# +I[Bob] -# +I[John] +# +I[ALICE] +# +I[BOB] +# +I[JOHN] ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content/docs/dev/table/data_stream_api.md b/docs/content/docs/dev/table/data_stream_api.md index a5061b5981769..cb67469f376e7 100644 --- a/docs/content/docs/dev/table/data_stream_api.md +++ b/docs/content/docs/dev/table/data_stream_api.md @@ -105,9 +105,9 @@ resultStream.print(); env.execute(); // prints: -// +I[Alice] -// +I[Bob] -// +I[John] +// +I[ALICE] +// +I[BOB] +// +I[JOHN] ``` {{< /tab >}} {{< tab "Scala" >}} @@ -138,9 +138,9 @@ resultStream.print() env.execute() // prints: -// +I[Alice] -// +I[Bob] -// +I[John] +// +I[ALICE] +// +I[BOB] +// +I[JOHN] ``` {{< /tab >}} {{< tab "Python" >}} @@ -171,9 +171,9 @@ res_ds.print() env.execute() # prints: -# +I[Alice] -# +I[Bob] -# +I[John] +# +I[ALICE] +# +I[BOB] +# +I[JOHN] ``` {{< /tab >}} {{< /tabs >}} From 597ebf1c9f7325858456849cf8e4dade29065f57 Mon Sep 17 00:00:00 2001 From: jingge Date: Mon, 16 Oct 2023 12:30:40 +0200 Subject: [PATCH 019/104] [FLINK-33274][release] Add release note for version 1.18 --- docs/content.zh/_index.md | 3 +- docs/content.zh/release-notes/flink-1.18.md | 151 ++++++++++++++++++++ docs/content/_index.md | 1 + docs/content/release-notes/flink-1.18.md | 151 ++++++++++++++++++++ 4 files changed, 305 insertions(+), 1 deletion(-) create mode 100644 docs/content.zh/release-notes/flink-1.18.md create mode 100644 docs/content/release-notes/flink-1.18.md diff --git a/docs/content.zh/_index.md b/docs/content.zh/_index.md index 6ce0a0ab09180..f25de48eb0f8a 100644 --- a/docs/content.zh/_index.md +++ b/docs/content.zh/_index.md @@ -85,7 +85,8 @@ under the License. For some reason Hugo will only allow linking to the release notes if there is a leading '/' and file extension. --> -请参阅 [Flink 1.17]({{< ref "/release-notes/flink-1.17.md" >}}), +请参阅 [Flink 1.18]({{< ref "/release-notes/flink-1.18.md" >}}), +[Flink 1.17]({{< ref "/release-notes/flink-1.17.md" >}}), [Flink 1.16]({{< ref "/release-notes/flink-1.16.md" >}}), [Flink 1.15]({{< ref "/release-notes/flink-1.15.md" >}}), [Flink 1.14]({{< ref "/release-notes/flink-1.14.md" >}}), diff --git a/docs/content.zh/release-notes/flink-1.18.md b/docs/content.zh/release-notes/flink-1.18.md new file mode 100644 index 0000000000000..0dc95975fcc0e --- /dev/null +++ b/docs/content.zh/release-notes/flink-1.18.md @@ -0,0 +1,151 @@ +--- +title: "Release Notes - Flink 1.18" +--- + + +# Release notes - Flink 1.18 + +These release notes discuss important aspects, such as configuration, behavior or dependencies, +that changed between Flink 1.17 and Flink 1.18. Please read these notes carefully if you are +planning to upgrade your Flink version to 1.18. + + +### Build System + +#### Support Java 17 (LTS) + +##### [FLINK-15736](https://issues.apache.org/jira/browse/FLINK-15736) +Apache Flink was made ready to compile and run with Java 17 (LTS). This feature is still in beta mode. +Issues should be reported in Flink's bug tracker. + + +### Table API & SQL + +#### Unified the max display column width for SQL Client and Table APi in both Streaming and Batch execution Mode + +##### [FLINK-30025](https://issues.apache.org/jira/browse/FLINK-30025) +Introduction of the new ConfigOption `DISPLAY_MAX_COLUMN_WIDTH` (`table.display.max-column-width`) +in the TableConfigOptions class is now in place. +This option is utilized when displaying table results through the Table API and SQL Client. +As SQL Client relies on the Table API underneath, and both SQL Client and the Table API serve distinct +and isolated scenarios, it is a rational choice to maintain a centralized configuration. +This approach also simplifies matters for users, as they only need to manage one ConfigOption for display control. + +During the migration phase, while `sql-client.display.max-column-width` is deprecated, +any changes made to `sql-client.display.max-column-width` will be automatically transferred to `table.display.max-column-width`. +Caution is advised when using the CLI, as it is not recommended to switch back and forth between these two options. + +#### Introduce Flink JDBC Driver For SQL Gateway +##### [FLINK-31496](https://issues.apache.org/jira/browse/FLINK-31496) +Apache Flink now supports JDBC driver to access SQL Gateway, you can use the driver in any cases that +support standard JDBC extension to connect to Flink cluster. + +#### Extend watermark-related features for SQL +##### [FLINK-31535](https://issues.apache.org/jira/browse/FLINK-31535) +Flink now enables user config watermark emit strategy/watermark alignment/watermark idle-timeout +in Flink SQL job with dynamic table options and 'OPTIONS' hint. + +#### Support configuring CatalogStore in Table API +##### [FLINK-32431](https://issues.apache.org/jira/browse/FLINK-32431) +Support lazy initialization of catalog and persistence of catalog configuration. + +#### Deprecate ManagedTable related APIs +##### [FLINK-32656](https://issues.apache.org/jira/browse/FLINK-32656) +ManagedTable related APIs are deprecated and will be removed in a future major release. + +### Connectors & Libraries + +#### SplitReader implements AutoCloseable instead of providing its own close method +##### [FLINK-31015](https://issues.apache.org/jira/browse/FLINK-31015) +SplitReader interface now extends `AutoCloseable` instead of providing its own method signature. + +#### JSON format supports projection push down +##### [FLINK-32610](https://issues.apache.org/jira/browse/FLINK-32610) +The JSON format introduced JsonParser as a new default way to deserialize JSON data. +JsonParser is a Jackson JSON streaming API to read JSON data which is much faster +and consumes less memory compared to the previous JsonNode approach. +This should be a compatible change, if you encounter any issues after upgrading, +you can fallback to the previous JsonNode approach by setting `json.decode.json-parser.enabled` to `false`. + + + +### Runtime & Coordination + +#### Unifying the Implementation of SlotManager +##### [FLINK-31439](https://issues.apache.org/jira/browse/FLINK-31439) +Fine-grained resource management are now enabled by default. You can use it by specifying the resource requirement. +More details can be found at https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/finegrained_resource/#usage. + +#### Watermark aggregation performance is poor when watermark alignment is enabled and parallelism is high +##### [FLINK-32420](https://issues.apache.org/jira/browse/FLINK-32420) +This performance improvement would be good to mention in the release blog post. + +As proven by the micro benchmarks (screenshots attached in the ticket), with 5000 subtasks, +the time to calculate the watermark alignment on the JobManager by a factor of 76x (7664%). +Previously such large jobs were actually at large risk of overloading JobManager, now that's far less likely to happen. + +#### Replace Akka by Pekko +##### [FLINK-32468](https://issues.apache.org/jira/browse/32468) +Flink's RPC framework is now based on Apache Pekko instead of Akka. Any Akka dependencies were removed. + +#### Introduce Runtime Filter for Flink Batch Jobs +##### [FLINK-32486](https://issues.apache.org/jira/browse/FLINK-32486) +We introduced a runtime filter for batch jobs in 1.18, which is designed to improve join performance. +It will dynamically generate filter conditions for certain Join queries at runtime to reduce the amount of scanned or shuffled data, +avoid unnecessary I/O and network transmission, and speed up the query. +Its working principle is building a filter(e.g. bloom filter) based on the data on the small table side(build side) first, +then passing this filter to the large table side(probe side) to filter the irrelevant data on it, +this can reduce the data reaching the join and improve performance. + +#### Make watermark alignment ready for production use +##### [FLINK-32548](https://issues.apache.org/jira/browse/FLINK-32548) +The watermark alignment is ready for production since Flink 1.18, +which completed a series of bug fixes and improvements related to watermark alignment. +Please refer to [FLINK-32420](https://issues.apache.org/jira/browse/FLINK-32420) for more information. + +#### Redundant TaskManagers should always be fulfilled in FineGrainedSlotManager +##### [FLINK-32880](https://issues.apache.org/jira/browse/FLINK-32880) +Fix the issue that redundant TaskManagers will not be fulfilled in FineGrainedSlotManager periodically. + +#### RestClient can deadlock if request made after Netty event executor terminated +##### [FLINK-32583](https://issues.apache.org/jira/browse/FLINK-32583) +Fix a bug in the RestClient where making a request after the client was closed returns a future that never completes. + +#### Deprecate Queryable State +##### [FLINK-32559](https://issues.apache.org/jira/browse/FLINK-32559) +The Queryable State feature is formally deprecated. It will be removed in future major version bumps. + + +### SDK + +#### Properly deprecate DataSet API +##### [FLINK-32558](https://issues.apache.org/jira/browse/FLINK-32558) +DataSet API is formally deprecated, and will be removed in the next major release. + + +### Dependency upgrades + +#### Upgrade Calcite version to 1.32.0 +##### [FLINK-29319](https://issues.apache.org/jira/browse/FLINK-29319) and related tickets [FLINK-27998](https://issues.apache.org/jira/browse/FLINK-27998), [FLINK-28744](https://issues.apache.org/jira/browse/FLINK-28744) + +Due to CALCITE-4861 (Optimization of chained CAST calls can lead to unexpected behavior), +also Flink's casting behavior has slightly changed. Some corner cases might behave differently now: For example, +casting from `FLOAT`/`DOUBLE` 9234567891.12 to `INT`/`BIGINT` has now Java behavior for overflows. + diff --git a/docs/content/_index.md b/docs/content/_index.md index 30e64fe67fbeb..1e6e1ab0751ad 100644 --- a/docs/content/_index.md +++ b/docs/content/_index.md @@ -86,6 +86,7 @@ For some reason Hugo will only allow linking to the release notes if there is a leading '/' and file extension. --> See the release notes for +[Flink 1.18]({{< ref "/release-notes/flink-1.18.md" >}}), [Flink 1.17]({{< ref "/release-notes/flink-1.17.md" >}}), [Flink 1.16]({{< ref "/release-notes/flink-1.16.md" >}}), [Flink 1.15]({{< ref "/release-notes/flink-1.15.md" >}}), diff --git a/docs/content/release-notes/flink-1.18.md b/docs/content/release-notes/flink-1.18.md new file mode 100644 index 0000000000000..d7a985a678070 --- /dev/null +++ b/docs/content/release-notes/flink-1.18.md @@ -0,0 +1,151 @@ +--- +title: "Release Notes - Flink 1.18" +--- + + +# Release notes - Flink 1.18 + +These release notes discuss important aspects, such as configuration, behavior or dependencies, +that changed between Flink 1.17 and Flink 1.18. Please read these notes carefully if you are +planning to upgrade your Flink version to 1.18. + + +### Build System + +#### Support Java 17 (LTS) + +##### [FLINK-15736](https://issues.apache.org/jira/browse/FLINK-15736) +Apache Flink was made ready to compile and run with Java 17 (LTS). This feature is still in beta mode. +Issues should be reported in Flink's bug tracker. + + +### Table API & SQL + +#### Unified the max display column width for SQL Client and Table APi in both Streaming and Batch execution Mode + +##### [FLINK-30025](https://issues.apache.org/jira/browse/FLINK-30025) +Introduction of the new ConfigOption `DISPLAY_MAX_COLUMN_WIDTH` (`table.display.max-column-width`) +in the TableConfigOptions class is now in place. +This option is utilized when displaying table results through the Table API and SQL Client. +As SQL Client relies on the Table API underneath, and both SQL Client and the Table API serve distinct +and isolated scenarios, it is a rational choice to maintain a centralized configuration. +This approach also simplifies matters for users, as they only need to manage one ConfigOption for display control. + +During the migration phase, while `sql-client.display.max-column-width` is deprecated, +any changes made to `sql-client.display.max-column-width` will be automatically transferred to `table.display.max-column-width`. +Caution is advised when using the CLI, as it is not recommended to switch back and forth between these two options. + +#### Introduce Flink JDBC Driver For SQL Gateway +##### [FLINK-31496](https://issues.apache.org/jira/browse/FLINK-31496) +Apache Flink now supports JDBC driver to access SQL Gateway, you can use the driver in any cases that +support standard JDBC extension to connect to Flink cluster. + +#### Extend watermark-related features for SQL +##### [FLINK-31535](https://issues.apache.org/jira/browse/FLINK-31535) +Flink now enables user config watermark emit strategy/watermark alignment/watermark idle-timeout +in Flink SQL job with dynamic table options and 'OPTIONS' hint. + +#### Support configuring CatalogStore in Table API +##### [FLINK-32431](https://issues.apache.org/jira/browse/FLINK-32431) +Support lazy initialization of catalog and persistence of catalog configuration. + +#### Deprecate ManagedTable related APIs +##### [FLINK-32656](https://issues.apache.org/jira/browse/FLINK-32656) +ManagedTable related APIs are deprecated and will be removed in a future major release. + +### Connectors & Libraries + +#### SplitReader implements AutoCloseable instead of providing its own close method +##### [FLINK-31015](https://issues.apache.org/jira/browse/FLINK-31015) +SplitReader interface now extends `AutoCloseable` instead of providing its own method signature. + +#### JSON format supports projection push down +##### [FLINK-32610](https://issues.apache.org/jira/browse/FLINK-32610) +The JSON format introduced JsonParser as a new default way to deserialize JSON data. +JsonParser is a Jackson JSON streaming API to read JSON data which is much faster +and consumes less memory compared to the previous JsonNode approach. +This should be a compatible change, if you encounter any issues after upgrading, +you can fallback to the previous JsonNode approach by setting `json.decode.json-parser.enabled` to `false`. + + + +### Runtime & Coordination + +#### Unifying the Implementation of SlotManager +##### [FLINK-31439](https://issues.apache.org/jira/browse/FLINK-31439) +Fine-grained resource management are now enabled by default. You can use it by specifying the resource requirement. +More details can be found at https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/finegrained_resource/#usage. + +#### Watermark aggregation performance is poor when watermark alignment is enabled and parallelism is high +##### [FLINK-32420](https://issues.apache.org/jira/browse/FLINK-32420) +This performance improvement would be good to mention in the release blog post. + +As proven by the micro benchmarks (screenshots attached in the ticket), with 5000 subtasks, +the time to calculate the watermark alignment on the JobManager by a factor of 76x (7664%). +Previously such large jobs were actually at large risk of overloading JobManager, now that's far less likely to happen. + +#### Replace Akka by Pekko +##### [FLINK-32468](https://issues.apache.org/jira/browse/32468) +Flink's RPC framework is now based on Apache Pekko instead of Akka. Any Akka dependencies were removed. + +#### Introduce Runtime Filter for Flink Batch Jobs +##### [FLINK-32486](https://issues.apache.org/jira/browse/FLINK-32486) +We introduced a runtime filter for batch jobs in 1.18, which is designed to improve join performance. +It will dynamically generate filter conditions for certain Join queries at runtime to reduce the amount of scanned or shuffled data, +avoid unnecessary I/O and network transmission, and speed up the query. +Its working principle is building a filter(e.g. bloom filter) based on the data on the small table side(build side) first, +then passing this filter to the large table side(probe side) to filter the irrelevant data on it, +this can reduce the data reaching the join and improve performance. + +#### Make watermark alignment ready for production use +##### [FLINK-32548](https://issues.apache.org/jira/browse/FLINK-32548) +The watermark alignment is ready for production since Flink 1.18, +which completed a series of bug fixes and improvements related to watermark alignment. +Please refer to [FLINK-32420](https://issues.apache.org/jira/browse/FLINK-32420) for more information. + +#### Redundant TaskManagers should always be fulfilled in FineGrainedSlotManager +##### [FLINK-32880](https://issues.apache.org/jira/browse/FLINK-32880) +Fix the issue that redundant TaskManagers will not be fulfilled in FineGrainedSlotManager periodically. + +#### RestClient can deadlock if request made after Netty event executor terminated +##### [FLINK-32583](https://issues.apache.org/jira/browse/FLINK-32583) +Fix a bug in the RestClient where making a request after the client was closed returns a future that never completes. + +#### Deprecate Queryable State +##### [FLINK-32559](https://issues.apache.org/jira/browse/FLINK-32559) +The Queryable State feature is formally deprecated. It will be removed in future major version bumps. + + +### SDK + +#### Properly deprecate DataSet API +##### [FLINK-32558](https://issues.apache.org/jira/browse/FLINK-32558) +DataSet API is formally deprecated, and will be removed in the next major release. + + +### Dependency upgrades + +#### Upgrade Calcite version to 1.32.0 +##### [FLINK-29319](https://issues.apache.org/jira/browse/FLINK-29319) and related tickets [FLINK-27998](https://issues.apache.org/jira/browse/FLINK-27998), [FLINK-28744](https://issues.apache.org/jira/browse/FLINK-28744) + +Due to CALCITE-4861 (Optimization of chained CAST calls can lead to unexpected behavior), +also Flink's casting behavior has slightly changed. Some corner cases might behave differently now: For example, +casting from `FLOAT`/`DOUBLE` 9234567891.12 to `INT`/`BIGINT` has now Java behavior for overflows. + From 6f9238a8d6a3c4d7dbb46652c2c621c8c6e433d0 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Tue, 24 Oct 2023 21:27:53 +0200 Subject: [PATCH 020/104] [FLINK-33321][hotfix] VertexFlameGraphFactoryTest#verifyRecursively doesn't work on java 21 --- .../webmonitor/threadinfo/VertexFlameGraphFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java index 070daa901cf0a..6a3f173a12c5e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexFlameGraphFactoryTest.java @@ -79,7 +79,7 @@ public boolean matches(String value) { && value.endsWith("$Lambda$0/0") || javaVersion.compareTo("21") < 0 && value.endsWith("$Lambda$0/0x0") - || value.endsWith("$$Lambda0/0"); + || value.endsWith("$$Lambda0/0x0"); } }); } From d722bf7d1fa1c267784b5103438e7a53be919d77 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 24 Oct 2023 09:55:43 +0200 Subject: [PATCH 021/104] [FLINK-33346][runtime][test] Removes timeout The instability is caused by the timeout. The fatal error hides the actual cause (i.e. the TimeoutException). It can be reproduced by setting the timeout of the test to 0ms. The actual cause is revealed when disabling the fatal error handling or not throwing a AssertionError in case of the fatal error. --- .../runtime/dispatcher/DispatcherResourceCleanupTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 4954b1ba2c3eb..122441459d8e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -575,9 +575,7 @@ public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exce final CompletableFuture errorFuture = this.testingFatalErrorHandlerResource.getFatalErrorHandler().getErrorFuture(); - assertThat( - errorFuture.get(100, TimeUnit.MILLISECONDS), - IsInstanceOf.instanceOf(FlinkException.class)); + assertThat(errorFuture.get(), IsInstanceOf.instanceOf(FlinkException.class)); testingFatalErrorHandlerResource.getFatalErrorHandler().clearError(); } From e9bf7b5499976791e388c384570db9453c661ffb Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 24 Oct 2023 16:33:21 +0200 Subject: [PATCH 022/104] [hotfix] Update OpenAPI spec version --- docs/static/generated/rest_v1_dispatcher.yml | 2 +- docs/static/generated/rest_v1_sql_gateway.yml | 2 +- docs/static/generated/rest_v2_sql_gateway.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/static/generated/rest_v1_dispatcher.yml b/docs/static/generated/rest_v1_dispatcher.yml index 4c510e9374259..62fc1122535d6 100644 --- a/docs/static/generated/rest_v1_dispatcher.yml +++ b/docs/static/generated/rest_v1_dispatcher.yml @@ -6,7 +6,7 @@ info: license: name: Apache 2.0 url: https://www.apache.org/licenses/LICENSE-2.0.html - version: v1/1.18-SNAPSHOT + version: v1/1.19-SNAPSHOT paths: /cluster: delete: diff --git a/docs/static/generated/rest_v1_sql_gateway.yml b/docs/static/generated/rest_v1_sql_gateway.yml index d91e75cafff76..71c7db934bca7 100644 --- a/docs/static/generated/rest_v1_sql_gateway.yml +++ b/docs/static/generated/rest_v1_sql_gateway.yml @@ -6,7 +6,7 @@ info: license: name: Apache 2.0 url: https://www.apache.org/licenses/LICENSE-2.0.html - version: v1/1.18-SNAPSHOT + version: v1/1.19-SNAPSHOT paths: /api_versions: get: diff --git a/docs/static/generated/rest_v2_sql_gateway.yml b/docs/static/generated/rest_v2_sql_gateway.yml index 42ca8d5bd599b..15008070ef94f 100644 --- a/docs/static/generated/rest_v2_sql_gateway.yml +++ b/docs/static/generated/rest_v2_sql_gateway.yml @@ -6,7 +6,7 @@ info: license: name: Apache 2.0 url: https://www.apache.org/licenses/LICENSE-2.0.html - version: v2/1.18-SNAPSHOT + version: v2/1.19-SNAPSHOT paths: /api_versions: get: From 100cc20f40bb09890dac3708387cc54bdcd91654 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 24 Oct 2023 16:36:10 +0200 Subject: [PATCH 023/104] [FLINK-33352][rest][docs] Add schema mappings to discriminator properties --- docs/static/generated/rest_v1_dispatcher.yml | 7 +++++++ .../checkpoints/CheckpointStatistics.java | 16 ++++++++++++++++ .../checkpoints/SubtaskCheckpointStatistics.java | 14 ++++++++++++++ 3 files changed, 37 insertions(+) diff --git a/docs/static/generated/rest_v1_dispatcher.yml b/docs/static/generated/rest_v1_dispatcher.yml index 62fc1122535d6..4c5bdd89d2918 100644 --- a/docs/static/generated/rest_v1_dispatcher.yml +++ b/docs/static/generated/rest_v1_dispatcher.yml @@ -1819,6 +1819,10 @@ components: format: int64 discriminator: propertyName: className + mapping: + completed: '#/components/schemas/CompletedCheckpointStatistics' + failed: '#/components/schemas/FailedCheckpointStatistics' + in_progress: '#/components/schemas/PendingCheckpointStatistics' CheckpointStatisticsSummary: type: object properties: @@ -2935,6 +2939,9 @@ components: type: string discriminator: propertyName: className + mapping: + completed: '#/components/schemas/CompletedSubtaskCheckpointStatistics' + pending: '#/components/schemas/PendingSubtaskCheckpointStatistics' SubtaskExecutionAttemptAccumulatorsInfo: type: object properties: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java index fd8ed6f087be5..b38f57c987896 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java @@ -41,6 +41,9 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; +import io.swagger.v3.oas.annotations.media.DiscriminatorMapping; +import io.swagger.v3.oas.annotations.media.Schema; + import javax.annotation.Nullable; import java.util.Collection; @@ -64,6 +67,19 @@ value = CheckpointStatistics.PendingCheckpointStatistics.class, name = "in_progress") }) +@Schema( + discriminatorProperty = "className", + discriminatorMapping = { + @DiscriminatorMapping( + value = "completed", + schema = CheckpointStatistics.CompletedCheckpointStatistics.class), + @DiscriminatorMapping( + value = "failed", + schema = CheckpointStatistics.FailedCheckpointStatistics.class), + @DiscriminatorMapping( + value = "in_progress", + schema = CheckpointStatistics.PendingCheckpointStatistics.class), + }) public class CheckpointStatistics implements ResponseBody { public static final String FIELD_NAME_ID = "id"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java index 642509a965c7e..613916eb3d762 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java @@ -23,6 +23,9 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.swagger.v3.oas.annotations.media.DiscriminatorMapping; +import io.swagger.v3.oas.annotations.media.Schema; + import java.util.Objects; /** Checkpoint statistics for a subtask. */ @@ -38,6 +41,17 @@ value = SubtaskCheckpointStatistics.PendingSubtaskCheckpointStatistics.class, name = "pending") }) +@Schema( + discriminatorProperty = "className", + discriminatorMapping = { + @DiscriminatorMapping( + value = "completed", + schema = + SubtaskCheckpointStatistics.CompletedSubtaskCheckpointStatistics.class), + @DiscriminatorMapping( + value = "pending", + schema = SubtaskCheckpointStatistics.PendingSubtaskCheckpointStatistics.class), + }) public class SubtaskCheckpointStatistics { public static final String FIELD_NAME_INDEX = "index"; From b1bbafddc98c76653a5733ec345f79b1ee4eee71 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Thu, 12 Oct 2023 19:25:10 -0700 Subject: [PATCH 024/104] [FLINK-33255] [table] Validate argument count during type inference - Previously, TypeInferenceOperandInference was not validating argument counts which led to bugs like FLINK-33248 - This commit adds a check to validate argument count and throws an exception if the argument count doesn't match the expected number of arguments --- .../types/inference/TypeInferenceUtil.java | 75 +++++++++++-------- .../sql/validate/ExtraCalciteResource.java | 33 ++++++++ .../sql/validate/SqlValidatorImpl.java | 28 ++++++- .../TypeInferenceOperandInference.java | 7 +- .../functions/CollectionFunctionsITCase.java | 22 +++++- .../runtime/stream/sql/CalcITCase.scala | 9 ++- 6 files changed, 130 insertions(+), 44 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ExtraCalciteResource.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java index dcaf52a71b5cb..8247ccad1b06f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java @@ -209,6 +209,48 @@ public static TableException createUnexpectedException( cause); } + /** + * Validates argument counts. + * + * @param argumentCount expected argument count + * @param actualCount actual argument count + * @param throwOnFailure if true, the function throws a {@link ValidationException} if the + * actual value does not meet the expected argument count + * @return a boolean indicating if expected argument counts match the actual counts + */ + public static boolean validateArgumentCount( + ArgumentCount argumentCount, int actualCount, boolean throwOnFailure) { + final int minCount = argumentCount.getMinCount().orElse(0); + if (actualCount < minCount) { + if (throwOnFailure) { + throw new ValidationException( + String.format( + "Invalid number of arguments. At least %d arguments expected but %d passed.", + minCount, actualCount)); + } + return false; + } + final int maxCount = argumentCount.getMaxCount().orElse(Integer.MAX_VALUE); + if (actualCount > maxCount) { + if (throwOnFailure) { + throw new ValidationException( + String.format( + "Invalid number of arguments. At most %d arguments expected but %d passed.", + maxCount, actualCount)); + } + return false; + } + if (!argumentCount.isValidCount(actualCount)) { + if (throwOnFailure) { + throw new ValidationException( + String.format( + "Invalid number of arguments. %d arguments passed.", actualCount)); + } + return false; + } + return true; + } + /** * Information what the outer world (i.e. an outer wrapping call) expects from the current * function call. This can be helpful for an {@link InputTypeStrategy}. @@ -385,39 +427,6 @@ private static String formatArgument(Signature.Argument arg) { return stringBuilder.toString(); } - private static boolean validateArgumentCount( - ArgumentCount argumentCount, int actualCount, boolean throwOnFailure) { - final int minCount = argumentCount.getMinCount().orElse(0); - if (actualCount < minCount) { - if (throwOnFailure) { - throw new ValidationException( - String.format( - "Invalid number of arguments. At least %d arguments expected but %d passed.", - minCount, actualCount)); - } - return false; - } - final int maxCount = argumentCount.getMaxCount().orElse(Integer.MAX_VALUE); - if (actualCount > maxCount) { - if (throwOnFailure) { - throw new ValidationException( - String.format( - "Invalid number of arguments. At most %d arguments expected but %d passed.", - maxCount, actualCount)); - } - return false; - } - if (!argumentCount.isValidCount(actualCount)) { - if (throwOnFailure) { - throw new ValidationException( - String.format( - "Invalid number of arguments. %d arguments passed.", actualCount)); - } - return false; - } - return true; - } - private static AdaptedCallContext inferInputTypes( TypeInference typeInference, CallContext callContext, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ExtraCalciteResource.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ExtraCalciteResource.java new file mode 100644 index 0000000000000..92c65c17825e4 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ExtraCalciteResource.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.calcite.sql.validate; + +import org.apache.calcite.runtime.Resources; + +/** + * Compiler-checked resources similar to CalciteResource in Calcite project. These are extra + * exceptions we want to extend Calcite with. Ref: + * https://issues.apache.org/jira/browse/CALCITE-6069 + */ +public interface ExtraCalciteResource { + + @Resources.BaseMessage( + "No match found for function signature {0}.\nSupported signatures are:\n{1}") + Resources.ExInst validatorNoFunctionMatch( + String invocation, String allowedSignatures); +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java index 8b2dd917962aa..28c3504f97e73 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java @@ -158,10 +158,13 @@ * Default implementation of {@link SqlValidator}, the class was copied over because of * CALCITE-4554. * - *

Lines 5079 ~ 5092, Flink enables TIMESTAMP and TIMESTAMP_LTZ for system time period + *

Lines 1954 ~ 1977, Flink improves error message for functions without appropriate arguments in + * handleUnresolvedFunction at {@link SqlValidatorImpl#handleUnresolvedFunction}. + * + *

Lines 5101 ~ 5114, Flink enables TIMESTAMP and TIMESTAMP_LTZ for system time period * specification type at {@link org.apache.calcite.sql.validate.SqlValidatorImpl#validateSnapshot}. * - *

Lines 5436 ~ 5442, Flink enables TIMESTAMP and TIMESTAMP_LTZ for first orderBy column in + *

Lines 5458 ~ 5464, Flink enables TIMESTAMP and TIMESTAMP_LTZ for first orderBy column in * matchRecognize at {@link SqlValidatorImpl#validateMatchRecognize}. */ public class SqlValidatorImpl implements SqlValidatorWithHints { @@ -181,6 +184,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints { /** Alias prefix generated for source columns when rewriting UPDATE to MERGE. */ public static final String UPDATE_ANON_PREFIX = "SYS$ANON"; + private static final ExtraCalciteResource EXTRA_RESOURCE = + Resources.create(ExtraCalciteResource.class); + // ~ Instance fields -------------------------------------------------------- private final SqlOperatorTable opTab; @@ -1946,11 +1952,27 @@ public CalciteException handleUnresolvedFunction( final String signature; if (unresolvedFunction instanceof SqlFunction) { + // ----- FLINK MODIFICATION BEGIN ----- final SqlOperandTypeChecker typeChecking = new AssignableOperandTypeChecker(argTypes, argNames); - signature = + final String invocation = typeChecking.getAllowedSignatures( unresolvedFunction, unresolvedFunction.getName()); + if (unresolvedFunction.getOperandTypeChecker() != null) { + final String allowedSignatures = + unresolvedFunction + .getOperandTypeChecker() + .getAllowedSignatures( + unresolvedFunction, unresolvedFunction.getName()); + throw newValidationError( + call, + EXTRA_RESOURCE.validatorNoFunctionMatch(invocation, allowedSignatures)); + } else { + signature = + typeChecking.getAllowedSignatures( + unresolvedFunction, unresolvedFunction.getName()); + } + // ----- FLINK MODIFICATION END ----- } else { signature = unresolvedFunction.getName(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java index 5e6e3d83b4d24..b59ad75f8d433 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java @@ -68,7 +68,12 @@ public void inferOperandTypes( final CallContext callContext = new CallBindingCallContext(dataTypeFactory, definition, callBinding, returnType); try { - inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); + if (TypeInferenceUtil.validateArgumentCount( + typeInference.getInputTypeStrategy().getArgumentCount(), + callContext.getArgumentDataTypes().size(), + false)) { + inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); + } } catch (ValidationException | CalciteContextException e) { // let operand checker fail } catch (Throwable t) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java index 61b8128d3f748..a758d7cdf3b4b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java @@ -975,11 +975,16 @@ private Stream arrayJoinTestCases() { DataTypes.STRING().nullable()) .testSqlValidationError( "ARRAY_JOIN(f0)", - "No match found for function " - + "signature ARRAY_JOIN()") + "No match found for function signature ARRAY_JOIN().\n" + + "Supported signatures are:\n" + + "ARRAY_JOIN(ARRAY, )\n" + + "ARRAY_JOIN(ARRAY, , )") .testSqlValidationError( "ARRAY_JOIN()", - "No match found for function signature ARRAY_JOIN()") + "No match found for function signature ARRAY_JOIN().\n" + + "Supported signatures are:\n" + + "ARRAY_JOIN(ARRAY, )\n" + + "ARRAY_JOIN(ARRAY, , )") .testSqlValidationError( "ARRAY_JOIN(f5, '+')", "Invalid input arguments. Expected signatures are:\n" @@ -1190,7 +1195,16 @@ private Stream arraySliceTestCases() { + "ARRAY_SLICE(, , )") .testSqlValidationError( "ARRAY_SLICE()", - " No match found for function signature ARRAY_SLICE()") + "No match found for function signature ARRAY_SLICE().\n" + + "Supported signatures are:\n" + + "ARRAY_SLICE(, , )\n" + + "ARRAY_SLICE(, )") + .testSqlValidationError( + "ARRAY_SLICE(1)", + "No match found for function signature ARRAY_SLICE().\n" + + "Supported signatures are:\n" + + "ARRAY_SLICE(, , )\n" + + "ARRAY_SLICE(, )") .testSqlValidationError("ARRAY_SLICE(null)", "Illegal use of 'NULL'")); } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala index 301496e37b6e6..84e025d230772 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala @@ -659,9 +659,12 @@ class CalcITCase extends StreamingTestBase { } catch { case e: Exception => assertEquals( - "SQL validation failed. From line 1, column 12 to line 1, column 30: " + - "No match found for function signature CURRENT_WATERMARK()", - e.getMessage) + "SQL validation failed. From line 1, column 12 to line 1, column 30: No match found for function signature CURRENT_WATERMARK().\n" + + "Supported signatures are:\n" + + "CURRENT_WATERMARK()\n" + + "CURRENT_WATERMARK()", + e.getMessage + ) } } From fa961ca2699523fad0093d309667123cebb5b322 Mon Sep 17 00:00:00 2001 From: jingge Date: Wed, 25 Oct 2023 14:48:32 +0200 Subject: [PATCH 025/104] [FLINK-32912][release] Build 1.18 docs in GitHub Action and mark 1.18 as stable in docs --- .github/workflows/docs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index 1d17d0057a076..3353fdc693b6d 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -43,7 +43,7 @@ jobs: if [ "${currentBranch}" = "master" ]; then echo "flink_alias=release-1.19" >> ${GITHUB_ENV} - elif [ "${currentBranch}" = "release-1.17" ]; then + elif [ "${currentBranch}" = "release-1.18" ]; then echo "flink_alias=stable" >> ${GITHUB_ENV} fi - name: Build documentation From 4de0f76d0bcfc4953d2aef1f06ae490e46b03d37 Mon Sep 17 00:00:00 2001 From: Yi Zhang Date: Fri, 20 Oct 2023 16:01:46 -0700 Subject: [PATCH 026/104] [FLINK-33090][checkpointing] CheckpointsCleaner clean individual checkpoint states in parallel --- .../checkpointing_configuration.html | 6 + .../common_state_backends_section.html | 6 + .../configuration/CheckpointingOptions.java | 13 ++ .../flink/runtime/checkpoint/Checkpoint.java | 9 ++ .../checkpoint/CheckpointsCleaner.java | 68 +++++---- .../checkpoint/CompletedCheckpoint.java | 36 ++++- .../runtime/checkpoint/OperatorState.java | 15 ++ .../checkpoint/OperatorSubtaskState.java | 31 ++-- .../runtime/checkpoint/PendingCheckpoint.java | 38 +++++ .../CheckpointResourcesCleanupRunner.java | 6 +- .../scheduler/DefaultSchedulerFactory.java | 8 +- .../CheckpointCoordinatorFailureTest.java | 1 - .../checkpoint/CheckpointCoordinatorTest.java | 138 +++++++++++++++--- .../CheckpointCoordinatorTestingUtils.java | 55 +++++++ .../CompletedCheckpointStoreTest.java | 20 ++- .../FullyFinishedOperatorStateTest.java | 35 +++++ .../checkpoint/OperatorSubtaskStateTest.java | 64 ++++---- .../checkpoint/PendingCheckpointTest.java | 27 ++-- .../streaming/util/TestStreamEnvironment.java | 2 + 19 files changed, 453 insertions(+), 125 deletions(-) diff --git a/docs/layouts/shortcodes/generated/checkpointing_configuration.html b/docs/layouts/shortcodes/generated/checkpointing_configuration.html index b817d7802f5d1..92e2ae126a44e 100644 --- a/docs/layouts/shortcodes/generated/checkpointing_configuration.html +++ b/docs/layouts/shortcodes/generated/checkpointing_configuration.html @@ -26,6 +26,12 @@

7F<( zhE}DjvY!M$d+0es;Q>8>wi&y)nmzTp@xG)dsR&9ai6GP9yGD5929|KxoK@O~k&&Xp z`ih0SF~Oy-+z=0yiC7EDU~he`lhJU9rT-Y4y%_FA1S5;Cfalyj3^;h0VvuEdt2l?i z326vPD~1IV!ZzgC_-4^em&6PQ2d-z7zZPZj2q*$gX)>eK(5DBITcv4Rhi$Zvz+v7I zN|F^4O6hboM^5$2&Cg934l@B!%f0)v-(V+CzkvT;b8`+@nmS$c`BAl?6qBhQ#Cr}j zJETdYDZH-4z*m^avfM99_QUC!xsegBjFl`$6W+bt%UoPwnpryTBsbWoDj)?l*)Y;L z_H1+#;ypq~*VacGza<18Y!Ey$Ft`suFkGe5{VM2fKY#1de^rVGttS?B_wGMKs{mmy z_^lwG8~?uPG|s}pA}%38K~CoSW8TT+ zFPo)yd$8bZ1XIteE`haomoEJ-qmP#&(Ss!9sD|*>Jg15?Qi%mJuZf!SscZ7YhgW)f zl$&A{EmSQvI|=t`c|TsU#rgdCb57__4J`)_Jjnpin%fQMJ+u9<{P+cqjAuamyG zpr2Q|LQ&KV&}24JON1p3N)?EZj*;8skQ?i05GdXWi4b~wBHDL&+1=O?whq7|rS5w* zPviM~2xz!@ z1SxEG7pGik1@H(W@)8~DKD5pKxIdd$$POqrI9{~9yy8rv9(G!G6@`*oI_tkHD2+mp z=GA)_%gd8<+>X;XB>DVVwg$W02NC5Yi@$~sIrMNazUUzZ zQ^l!|ma@|?0+KyWD(Cig4vaCSKX3bOVgk``xCV=@E23Y&ivT~gpqn-F==_ciU?+|f zUPPm)B1d_7EEwIOgL(GnbT%OtT!U!Y1o0)2s2@LmaMWM?KhvAoU!Gou8L>9f=Dkth z@j!GOM;0RTz>}Q#;?utp^+N$pA)y3_$sr^pjM;ridQVG>`rhZJV%}p3&`9LxG=L5Q zGCGv1yI`*ZCaoP;?iSJ0-=W}q*Q>y|;>#*e#$fz~L6qG_2In^gd6d?m1Ta^-)#6nS z-PfK9wA`bM&NYP0J6W%$U*x*M!1U`AVT6EtLDFq%B&;)az3{OQ)o}b(9g31|rq*6> zM?_B(p_D7yCs&kt$nY4 zN})Y&Q_nBK31<|g;|4NltM}xTbSW)!StdBqrxjy7Eyj9HRh15#0*#1b){qugW=3zW zex6+RLuCJYHm0{i>zZF8w$&QE?>_%SF4>b)QJEu&e=x|&^VW9hPpqI~@RvCf;G?f1+oFDb6e{EUZ=dvjnupc;&tluU-M|MGGqp2`-=o zLIh+;^X}AWF_aHvM|}<2!e_w%Jus`N!%)!HST$skeoOdJO6sPJ59BN;nnM6%qokq& zK>njA=-b07bg}XM&Y~scVRXk1$j{H`sH6Pf>FD@pwrATxr@8HPLu_(_+t7{ma}~ya z-_qhwOHWU~3O3xe(Mp!3kBqM%Glrd=9iuF6`EywKeW1D6N(vRgUy{uv;eLDbMXHy@ zZ~4|Dd?&;?CJ#GymINz-yK?otI7_0YlHwg^q+PTz>xzsL@v)ubB(xme_sH1-FtSN_ zwmdBMPB6I&3s~x~c$|3ehyu!db{<#js)<;BJ3{}(2QpglroVi7V@m#8nW$IQyQtP{ zA*xAOJVebTq}A+WawxSZBec5Dvv(a8Ae|wQDMad!w7y`Ox;Hs?dw}hDB4BKXnEgz0 zvx3z|8!QhCjd$>~H=RJYsPVv{G-EgqUgJZgA@pVMCpy4R zz8Ua9-Ba0de|7AodP}J`^$g;mUe>vAxjkHd*Q?AXfmls1Dd|UdCI29UEtRc*Ta-9R zd_J9FK@n6%tt>C!zghn8%P4%fL)0M?%5jUy4$|`7UqpO#VP&V%+G5z>lwmvQ#*z3G zMn%{3drx__PG;J+mWk)+qM^6Pu!0nq;}|?L3=^_?oyzhg_AX->Z^fz#D=$=UTv4aw znIlHatO6Tvub(|``WBf@Q@vpvKA{5zM1HrPLhlqbIV^6xt{&FuBBcrbwEJ#OH4V;rV48ktImv===qvqJZp+zCUB|-xnNml^_Q!6hH>LSB(BFca>SS# zntMW^cW@z`TydEm9r~sqULsoJBNw;0K06sGHOp+VB)OW5Ut2zTS6BBkz@U>|!C*A_ z45KDFSa5O+hJscwNlh`}ijI5x!tIA(XIo_XjHn3B!X6U4{Ka5TN=?PaZ!7#X@CHon znHjrMxI(NkbG4roJZE4hIt^=!Vx(xQpFL8?aa|Z2N|ytdNw7gZXu!Ofu3XV42h3SA zb&B9=87r^qE!?B6j^4gLgm7+OsJKtma7iR!G@9-fCCEN}qc4Fb1_#18 zTc2g)8Z-lkw&_6*i>2Tc?!VSympM`-YKI1ggucH#Rj+jS4L-pRy^+8dHnstT%N~mz zY_1J^`I7R{|51x4{xkJZ3e;_4(HeQMVdD{Flyx0u%ba~alDGe-QQb6Fg|r_a&ADV` zPHyLwu(U>eqMkS!U!J_8A_j~W^ew;+y zl~S~ScGZJiVu4L_T)DS`ixth9a~vB4LzZf7bP6QetmBOt3a-wVtMsp)RteFfw^1j3m@UyErgJLvFtM9|~>d7FOG;}XR)sx#Gv z0vosO4f)+#J>!ld{MvRfeQWF9Fs0D=$Is?Z=e{+Cjt8TnQh+=&AeUVYwlUGEeNMl! zQ*%R*S(l-SToY+ny)pi|LOZfaAFQ!@Yk>{iBj^xS(8#A6F?s87!*j5pZ0*uEu1a2q(9jAHDtC-P@r&&lV>kWbSh<^5D zKtxu1-pr>rxCam@@1ET1S?4MallidlJ(t5@^UAU>odFDof(bhR~YfX)KZ+;<7G4H?fN9O zRPP)^6l8(<4K7a@pz7Xkua zbq*VST_5!o2-@Av5#^0c4I4Jl9M291GHkBUEQFJ_gnvb_U~G$A+k!^>^lH8$U-Pr1 zwFl(y$On+UPE=Hd>VWbxb0eXm$R=K;a~Q! zX04Gav1iiFP<9o1nk~?WEvsl^X^=jOkc_$`Qy4~f%^gJ_f1U7EFf0OyccRF(2#S}VF10_ns^gTtUQ~L<4DVyTT(ENdm zRhL6i*hC)g3}&Ug$Gg6)gSqOe8SKOalQ^S)hE?Q}G zzfobBL0mJkEIZ{B)AwLJQ<6(GHya^{V}vRB{U!1gi4dc~;czLpYuoh)tP+=w+Gyp9 z2XfOf${koR%sR;pt&XCEYVaKFZx)dpQ}SB(4Nm?W zFGCLxnJI>p-!TC!?7FCae7qv@KcR4F%RjEhN%Z2Php-73LWD0g)owII&WI(uPk2IX z>A$UODC*+dw{Mdjn=j-W*6!aC_=uk(9F3`Y<^{FbyVFq4S9820Xa36wn{CMAuvG{3 z#rm-qy#CZ#w1YK7eH&Iac9SZ99cr?!4oe0ED)QiX3bNnD2>+p|4LNDf+3Sx)Smb9k z?7Yb@WPYXW{M22;4^cp$@9RbqIMQ+2AC#MZu=ff9(TO;gNDxpZ0kuTbZhDYyQj zUa?|im#6;aq!2aMhiV-W-Anp8un6oF>nhp)CRY=3HMaL$CYgzBW#l`$i$P&oEpUWm zR?5Djnbcfkc*^&D^P=MjMSXcyPVQvbkRc?*2R?O}yT&f^LrH!l-NA*2?B1O_1aQR$ znwR@Yfz`F@zL;9)9p=-mpW;7g6ECz))YC?q<>S~q?{P=4K?>r2uA_%`q zM@}oJNgkT~HWjp5I^zc?nKik!H62<*gHGjsKZRm@)*ib#jOUtq9gn#f6$#q}*pFjm zQrMn(DZSF_0kZBT)nT$rlk}-NeVFeq#ntyA{3v&N)(f?Z{kXPmgs{WIzG$xNL+$Rv zB^0RsPLCE;1bR zxn6(Yjd6K_XL1kgW9aob_u(4apgab`EQ2XtDYhnN)-3}`639@fe zcqr*Mc-^rl7z1N^J}i9&#oGGdAW%rGNi>yk+m~SYI5(na64DAjv5 zCT!DGVIQ*IukvwA`VhmQ&?w!o7Ng#X!ec2?0E6Ap_aulXf`?|(1KZ$WeM&h>eWY^@ z4=UN|Z>wO)lr||38najD$3IVg=`~zxrBeJOp5SmMh9_Ak#$Ec<@$+PRbow1dXYww$ zxE)fn^P-l3dT%-ASk25Tf~`FyCPqycf5I|FhuKuD1ZkOP>AkD?S>bk7PI#+P2D;oTYK#2n;(Jzpan5)C?Y%Y2kvjVPkGM6ZKef99MI7)D>#vfB$7`qy ze7{i=FOE=`kJs|JMfbU41@ zhmWBP+d;Go3_kn(zcaHk1v}Z>N6KwX%k_)x$7_00V+a$z;lLlKU)WQ*p@Dgl8JlMO zSQVhyBjF0h)d{u{SI)&Y@VzWnI}(4VINBuj;BsKq&+dnhryjA^VEPoU4EVMf2jgl7 z++bcLJ><|mk@w13ZuccQ1NC=FEbJIIgZt`A)yP4bWH&&}zz*Veo zSr2ED5bHRj6(pS&O)Q=krhbKuqy+`4hL_$J2NfN+(aK&Yc+>qd{qbxkE^D>lrAuE$ z9JB|OX|h#uk-ZoN%4BfV?#UzIkkue?buPZVCoVq=EB)T z=SpjcGXgu{yJ)yW#VIi#S3jiqj)_rBH{5uRtNUf@L#EZiblUOX&l}%gY6Vpn2atHo;hE> zoc(^$zU9FP5w?Ab=0I=$^A*h!&dFf&^r;U|NuKU`ud)k0s^mgg%bm-Vopa9cCq;_s z-#4aFgfTi341IYV2D@<1trkS=ZEc^wkeu2T+3a5UY(!NliPhjo+=-%a#b(AC-J=vU zC#kw!-a@6vGMsuB%1;xLnW$%sph!!1 zry$)S-5?#3f^ZJ*r4!p9m z7&YXPN8V4CoiVDl0JY<%INN9z%QULUgZss;Qa1W>^G*n_BjVv4T%{`2c!R@uc3?*t zmH!zQE_SHrp+=v49p5VAY&mh>Y$zG8QrSo04?I7&H2U;nG{1>?jS3`AT}RRdX(>GW zJhI}00_l3%=9_%&cbV}rj^8Qg4=yhW`c9@0@jcLQsi1HkI}%xsYSEVU*-jPeI@Ue^ z+kVw`v5y4igf<^RtOjrHrj>RKhkpzHCvaV7A1@FKb^kW-?s9gJ0OqJWjzVbd>W*eK zon~6`@i{Df=o8RpO;1mkl$Rg;-Lv`M<9@8*_%9mx>%AW=94VF45FLX0QNkPS{MA^G z<*4MO^am=?CUorppRO^{Cvwg-497;YfKDpVDV%rUi4^ixKnHnoALqM+ zk9}Lq4DFi{{Fc%3))Y=2sKVx4i?x`pgY*?g#c&%%&^k`p)ofljg(sbHF3#qW-T!D| z8^{LF}RqRoYSXF*vREuL0De1?rH#^Zi-25)zx+^ z$){E(s?_m0$biHd6Zu92f`+z>-Sm9p(mJ2)e5W8rUVh_ntr*HH$}FuDF%u-ZMkJ;b zav1Qu1v*wz+_L2|`1*Tea5))P&6di^n=1H%12}H%jsqlIK1IAGa6EQpv=WCeo-l#0 zsr^Ovv5jyMbw-dZkKFy?i0ypYn~#5oMkDMsq;(maXc2uE2kEv0}~Uu=)F&$=9i<(7z*m(9h&(=*UGaode_u4SoXG z=AV)5d5fsPZbXGX3M#xi0V83YaU03*Jb{e^Qs9b@hQk*4kh@(vZBds6o)e}lh|7xb zZhvejzi-0^GBWkF#J0~M_tH4EK-k$p%v5nG)f+t*RD}4jz0J6!rGeUNVGmY!^8t89 zg1PVS)^LO{4czMfR92R0^tr@1wZV<;@-RKWcyD%0eV0nhUIpkipK# zN#~P@^S{A0WUT$QxQBB;;2F!moo+sEKqy`L|2sOLYzj_4%KV_bzH-L657S8WkQaYxA0G zvl_T&0Sm{&$48emNDZEuBdkxW6FR(^AE735P80JMFU5`ElYT`++}Bk~SlJ+HAjC z^|Xf8+~gB9p6Y4w_Nnz$P$ED^N?j`_W#c$zvGd^RY58E@9$QLjI(2wUkkB_~v3CAD z{=2fNJ>1U&G^p_3-?G{BX6~1N<&E2nS15{CVoVsc0Ajyzkd*`6<|J_uVfUTyK!;h+2EC*u993v`^DR3K~kieNC@1Rem z^JqNnwYj`@uHt;b6taW?osC1D&S~r_kb*@6fNrvH&UOH zBeHt1JyEFX%ELtH35Oc41sWLKwElJ%`Kjp72g;zzCw~*6N94+XTwGTL-3ENiIEGIs z+7I(bIuLy->O;8IGuFsV%J}y>I1XQnbpZ=|FapJ1s+JyQ8xyLzvv?gA!eDBfmpP@%+*oQ6x!uc>v${$#_3@r z6o;HyAi}ctT{VMZ;hB+|Rx3~%1Z|25d5U=rGMFf)1|2Mu{Y_H+{a`f$A9%fP0wVn$ zzt_=2n|_um8T-LW(FQ>wN7X&lDmj>jk6-2rq}OnSaSg_DKAv$HVRjgc+L8-|f7sVL zVah?((w28STzIyWCvH+_C2!^~pUjIHUaL2<_ILy7_qr=_Fww6SFU4}}#22=t`PW1j?o zDaa?VXP|fU2xe|5pa_IqWJp(uX}iHoZr@9LcHL?Su*a5UOoD?*|9*3S(gTy%ZFC{v zNP)7}d6ub!j~5Uldf-68%p3`y@qD2qLlMa>Z67CPjw=oKAsf4ZS%Og7 zqV`?yro1i(W05kUbQrC=?uZ7NIc5PfEs<&vLf+J=LZ>}%ggYa`SMe%AksKvx$!uwy zYSGl1fXYn{H*x|uE*%M`v;fr?sG2PA$3jcXUf>=gvH02NyeZSWHwqWa&H>&XTK zm7g&SZU62x@m-`z&in=95a52MqE3(x&?JqBS51i*v;C5{VDD6ENj!Bc0Tz4?8TQy# znn>p#+G&>v;M!DBV&)5fglx*F=iY!#5MkbzQ(3=$M5htpnw ztR)Va&y=x!F~pk(Rh{AU_WeH-=wKOSpPs)$21Z-s;Bjrd^(RTSALP+HpY{=(Z~w~h z{*xdp=&OK>2nW&|x;0gwAsRJ2ymM{{5Ob%y4!8=s&kdS%SW^tsrUrt+Q+*aQ!Q;mi zgN8=9td7EC!!>&p`0)4Cn+WGbT`N@fJ5Ma?Blc&KYptJ<79Y^BDRqJi$bx!3OIJ?A z7n8XmL5KHj?x8s$bFL^|nb4~yjj||%jWDalT%=WR!E|4|yvC55G&W8P7f^wGjj(nx zMs;T{dAnTmc4c;vg%KMLZBJgyhQ9Lp*mU?DRF~%){k{QItLB)o36^6Kq^m5_h9F@ZT(^R~uo73sq{M8B2jzC=CGa+rR=>G938 z`X51_e<5h_?rBD>0x1Bisn+d!TLBN5_Om(E6T|JAsnXde`_K6;fN!8lU^C4M{^Ms7 z8iBAR`TAG4v3wA9t@jdav-}QU=;=R`BBd)$>H@nnF!r8I1?07tHVh@8;sYo5n6-BwSI2FK-Smm9+)aPUY1 z88x^2?zl=$Mt>(bfngqq{jbJ^L)UvE=(1IM_T0NIUanWR9(99=BZQc~e@&)QzMC&hgN6uP_B#09wx}^=~#*LQ1mFD^Vmc@ZGmrBU%I5I7aK~wOjGrvjQd>-H1yqXT|=I_7X zYHK4QQz9T^OB7Az^QR^4u>&WaaW$Q<{AoVf*a^Dm?Jazz0TInhBH(Q$t6msra^WLz zhks!$o=jD;#Et78|E*U3L;dmk+b!v|z+xNRT!qzK#m9OaX|taeZe_2Pzr942DcIG} zsc=0!vUBS(tqa-7`*9OWWAXBxu9A$!%jbntw)9EzP5yAY$}-?}(eeR~Sk-32uvN;X z8b3Tu8iG0PmLjfb35Kdk`R79NmWj10-awlC#X%`fF(Hp-N*F9V@O?$w z)Gp4$Hv{AeaXl^r6vZFfi;9q_0CXpjkGpZoFB3-|MWDhD7emC+&C$3Nf`B8eh8G`! zLxugH9)&?gJBTfG59bWemRec!KLNu7|!T+nW13`tz(SpOu}8%`5R~6 z3||*tHbOf3pva#)HT zhKC_<1Ai!fjRiRb5}7LS?8}|>&NiC&>2Q+2EJ5MF3uJ21C6t!tv3j~CHsL9wOV+7Ggq(yT2Olt^ zt^F`ZnOmse3aMGdYH*}*x6WbdlU7ZqZnlq=(WWpK!?zHKL0STvWdCXg{+!B%aN@ad z4k%7c13^J~DO_L|Tr^ucK;!ae5qz&gsDw#Q?t;r$VSvicd^;I1{RlQt`O@?0`Pyee zbb~a`yu-z!mLcF@hmP1aQ<+lf_KT#;Bz~hF=MAC3@k*X@W?{(@#tTG;W%Pw+f)-jn zZv7u}_{}lY?92O8~CTU~ArdL(86!mCk#OsH<9`@+nKr4CW z6BP@HuIzN){WDIY0eBF5x0mm_@OFd8& zs!ygp;U1UnJOR`oythBB07nAeO=9&o+Rm7xUdxJQ8p z1vovw^*ko%!mG4uPj6#{4Vy(G@LxM30nLt|GuYue!Ys4ylYR`m+H07uD ze7$~Ffck=pRCMH*5i97H;ZNbQ^4p=E>} z;?vEO&Vo0ka8>BMy#}H_z$@WVB9@V3`U-wv(1uR!1x;9d>~T!n&LjZUMomY*g52(V ziI{MPOyf71bfu4H;k0Nx9;FdIj+#pYk7|g*1Eoirf=SZURB9g)bS&<0P`3TmCcKf@lfE>O?`hkRE`_Nbh6Tl!R@pn z44X$`8KmI%MGm$M2m`NssBp0(xC8FJ4FYbqc4sQcU<^RO5kEOLdEou2je~=9qvLY5 z#bBb7f7Pib7`>85TK@P`5iUbIPTiefYq(c7opL(Ig?3Hn77D5?ol{>A(Lre7(!W5p zDOsl}DuuAK&ZxDG8_u2}J5aIjNB}Y@;@^di$<@rDtT0^I`jSCVhNR)#_)5c_MXdB- zUtrs1jXjZhpg{zwW1X)27fS;U`8s5t6x;1xK?KoBd@*f|ZpWe``cI8rar6cJ)cQSYXN7QtT3uBgQ_(%uj z1W1SNcX}ynKU^XKjbgb(a#rekd%}*#oC2J<+9^4i1Vq& z5mA#e`S$@TawCPGGbV&%e`$Xr87?=vtR@#d{P=J@ede=6;r#mbT8kIT>~F*dXPPDi zYzUtXuFa|lV2}z0M<ssgtpj{Is20bsV8`=DUKVdBSzlZtL{EG?i(~Z9WRGqr{ zPg}!jn*iQ_G#c1)6#x+}*v6n2A``h@L-<7bh7|PM1H-_;39Pv}27dbpVjDLu7mFxH z8I#ae|=qb;4?qs_-cyrT@A5iSTCxY<>t+jyLV?cG z=zAL{YQfNJ47t&H1I}Xn*u{3ZU3bCX!~$XdP%eIFR+krqQ8X!|c`ZZv_`y>d>>E+^ ztcSeX`pl#eeOmeZ2UuaI+?iS-E|~*3?JC9iv;?I*8-qNzR~%(*DDsiQmrWC8O#IL>OlKMgn|+By0D^2WSZY0iSEQip+d2$Bd;g#FGjtiB#IHUXiD(3ND`NxAh;3YKB`{h*oKf z5emhI0!-8xmr<@Xzc*puBfc_dQpi1HhSvX{Mo>qa4Pftu>zznT_+&B#8NDetWg!*C zR}{njUFoK7aS}mmsXoTpi1RuHJB9yX#7T)c;MD70y?>(1U5I5!-SB3_ycS$}k+brIE_{Z8O7mC{VLL^%`I<`f7Fsg~}XUW&Ar?Zd`8Lzf_20w{9;8-=S<9!W-shrZibLZ2ENR zVsu<#m6?OqqbmbYh?H!78aN;D?Z@nbicp$b2&Vn@R}sAtn17JA1bz%D9igD6hNo_+i=-N_rqAU`U&(zholEAMu*-Xkr(dJYM#dC0MYtYc-=;PWa?KKKng>DZ{c$Y;@(5u0N5f1Pe zska|i5X>@%Kvq0?*V3oP?fd6D#xApg(;orOjHIeh)&`|+S@Jet|EP=kcS;rdnosd` z;-3K#d%Qsp1Depr@6z#X-%5It&r~?P0HHtUC=FH24dj%TBa2PEW~CA}M}`o-M^nwk zehee%5_!2MoQUreq z08t=_EKqljAcwzPj}UwGMhrC>PJOn0KBK>Odx7A!7Ch;vrw2*Huz9bs5EMf2#q)^0 z*Ds<6d{D6HwMiddth*lGCxR%%6|gssLgn!rhZpC*QMbkn>5~i{*jx-lq4duCzh|zo zfcB|QV8I9{2fyxx*6#))*}=TN?ktCw|F--YtcrLZExnt$+Wq>uAExPtc!O)_S@HCh zC6L{CgXtZNHvqkdZR)_5>vm_WsB!{sZLc359|TteF`C17tHgP*BYGf(a!0WGRFr1a zP4#pwdw#e`42vWGrkokB#X8-b^!wUKDM$Eu=go-II54x@x>HmXk(;>o;d0rKBKJR^ zkSm(L+b(IVd1*F z{FRLy8wEVGRG9Gh$Zs!?qCkinkXYvfqvkzwNzF8w^WB+eK=lZiJ0zzPF-z@d$d^J8 zFwBCsGpNrhV%}vAlk@fywjy+iFWzqX#3pdoT0_QpQvb(=nVNjCN{?AOBkxUv}o2?HN7^4i> zdlrJBy0oCNlum#WW2T|?AFe>`i7u>A-bq!l2r`-9Y%0#o+th64_#Q*~R=@XvsJWBt z&Ci>UD3o0nVAlxz&b#;A3jlG{1RP+&XbhY8Pu;d)x%Df)$8U-%U96unx19Mw2b!>Q z8#s>j7(+{f%3-*fC@bKjaf+XTw5f<7hvu z=t>kh2zz^%a14~s7(#T730kM=v!z?a>+Q3y1^`3{cm>zDV|A=9=Uo$Vt)je#>PCt$ zoeFoIuK60`pyE~gPtilY0l`i-mb^6B`e?*R5^CAy`AEfY=hU@?V9+xNQIuSAX1hKj z=+d)(hPb(=!1W|+S)UKOBe}m~g%67C`psl?WFoC)TfiO8Nf1x6=A00-94`4r_ zuX#hBMx4+c07v~eXAVtCZZ~u4KnwUI7x~Ic41hc)AcRbTKDoqt3usIri}e`P@S-2w z*W0GP!9if7*J|e}^0oTct`*;^H4SXUuGAMOfos8H^ze_p1PrENSFLDv(}us49r|!z%U5+=SL3eKlCLkw9L4?sH!|%ys$e6%4+q ze#X|7%5fLKcOq+XwA9T1bbmGh?)LLvvY~*o{(m3$JAO_Rz(BjRSI5{NMx+?{9@alK z9D@QB2Gqb*xghFRV~j6DX#-|8T{A!#1NfO_3b$E9FQ{XqIY5fj`J7e)jF6psu=q^I zfX~y#v`$Lo0IUG12Hp~YgtJg^HDZTQ|Gn}FM4kUmKXX5Mun&VUk2~%-Vcn1{r4OQ0 z+bOb<->I~L$Fz1ySy?xTSp*w;X9TBq+{0CzGZgMuXQ|7MWwWlI}$cQsRxV5r#{W$M-`^Y|>RX z6*KfGnj8aF3$hv%{oGsXC=<%C@nzcV3JczFoX3 z`u*s(E2o>~dfA4(+$GD9|64>1Kir%Qu+1`p`M5wJMUZ8VDn`rPLg_bF4!u+u zDDA&&Lm??EY8iYu`q;F^y^AaYkLpsr{1oK4wpsSLv5f+k{S1gJy56-Ynq9~5bTz!e z>i9qc+AdtaB}TNVz7TE*fTBau^M<3NpWWcdf|Ag20<<`;f z#BD!_pXci}i03!04e%I9Bia3WM5rCB-e(&vS5Ey3%bkzZavLmTp+Jn#l=G$t^>16` zCne+!q2nMbK|%l~UDu7m>o+Ljo3^?qkhi*R2T}22q9W1*TUM07ZnQ}fr#6~tRgsOz z?uFY26a9uD;0SWG5hJeO7~-&qEMlyxo5hXLf>Gq#|Mok?vTkimNL7$O6G^{C3%_&h zO{4Q@wh)xp9@1wvRP(R7iPNy_GxRDDxlR}4Hy|DVTc%$jdZ|Gx-0P+lkiO50Jt3N{=gh=I*N#2N4w8!)yQ zyx%Ffyt?dpzt-133%nUN9*Qe3ErrJ3l$2WM_*ivvO7!o!Zwvt$t1zXkr6mKS@|Q4R zQ9(@_y8$+8p)?WXAsqS*_p&>Ws@L`WvBse8{9S4$sL{prTJ|93Y&(%y0@dsqj9+k# zi?Oy--|_>W9#JEDU;tL~j^yUf0JvzP2-g?*k22 zzm|k9H6apRNNG2!FUm$40D7k!HTN$X?h3!ur!ZVtD%!08G{uA=q=q{B1WBT#s#2?O zr5@leq*#_7>B}9|`!R^eQmnBy>~WkP@1X)DQGolVNl;)jd(Zr0jwq)5A1Rm@`4u)= z4~goAELmkah+7-XPw6)XlNEj@ZzqeTOd*>;iWI$ihe*eboA6%Qz>(@ZQQf{El6aUX zN$`l-TLCZgD8k8%EQJ%ND6=-zk&>FF<4 zwE|&ur?qY^!U0-B6lHSW^M=OvS(dc|QUxx;&QI-gwU!tikGFQ@9>3Lf&n^VpfMK9+ zi#v=U7!!VfztnDYn2G{_AweU#IF)7SehF$nMQ#vFs&ve9v=4=G7HXyO zjUEE3t2$F@^SAW+{(We>AmYzd#b1#xSW-8^WpzfSm5J=(d7 z2gHQdH6>xLE<=R7DxwtCi{knXYlDMHE+fEWz+sFM^A$<5k2_-w@e7$MqjoMpiEkb2 zz*zPFK4Pi*uZUuaISo)DB-(^_zw!Q|ObU;gueAB_A2n^l2B#%(a48^O09SuO;Y*yr zYXmU*0spXAV6E@JTAD%=P20WB#o4q54-=kJ1>SQbW08iXrV@dzZcN2A&WF!q?V>+l z#|9p$Y&tA9tY>?zK!K6%)7MRu?WZ{qn#?K=4*_42v;ADdey$o0c2-8B0@3_VfNXqP zq8$CndIZ%*500qT`;YJMoudu=#l(4LcKv|UvLbAZOXqi?J!-xZG4R@?%A4IYX(XZz zqN{2814>RRTE4>ygx#ucqr{`xzdD^&O;a0pr!ANZQC+KXbaRIoWd`3Iav-Q`0!~q=j>fp!Dng$aIMw4ewKjOE(cD!swfESB zRPD;y@~55!wD;l5Ths~APuWT<#q}q_7_(~!G5l-SkP`e6|6s09$e{{S*kSZLb~S4p z1F~K$(W)v%uOg*3tMz^<{C4{f%#5UV#KoJ*s&$|w<$M>)Mfy9$2NVhRoS=CNE%N#m zF_PZ&{YUvT!nGgz+Lg9}%cHUw;?8nn zVCtz)z}pt6Y=8K5=$Cj=49gYP}WdM0PG61iwsUV|1TtGbx z$b9_qXDc|799{}FlQSzYZtQ^F7;Rhsix;*rV$Bw|^Y+o&zN$uSPs77=zY8wv8qKJF zBEBW;jw@wcFYu-(QyK$)=k=AV{*dCwYcVhoLXe$hgm0~|b>gTZLdkQ@__7P{W3N}c zY`Okg+$4i#l@x#Njd zl)r^&@&yVv8;k>Ybd$Q^Hlu2%j)aXI6l}yaRel2FK#G|iWb8|biY939`b&nzsUL%0b2oeh;h2R%CSRKw$Sv~(;~|MJhA`aGWh-PyI=0oizjRHGoFSYB&pWeOcXrXJgz&FRJC7{f*^nhpav)RUIUrh z-|L*`Od}p-@m5~nd{G487ErB%c??W`Z<0699aB0|z;7 zV32?q3ay<5cT23$Q4>N>IF=;n-c$_%Z~_Q3THP=p@=nB!^* z;{$&X8#Ejml~y_zeH}-Zqpj|-QlFl{3Zuo9u4#*_Pw{(Ox#-o+Z3|K`7h#mRg;M=Dpvx*%LN<(^Va-n1_-a>{HFQipuWax) z9sp_pVdC0#zqz*a4>+Ws_1Y6`6%IHduZl)zN|cf%oaRp5%#1lU<|5Z5-{6f|P|h0J z*21=Zw{-YZK%dl@Bc`q|#EltWQZQ4y`5Qo?s4x+NG+yW%C2f2d_{9C=t}yWGi{cCq zdD4n6VO2~UcfRgYER&{O(R7HHFirhCeWQQ;RwQ5Lac>2Ksk{X8g155zRy$$BrATw!5QNn8^6_ zbKpG!DR_ty!+NmW+Yi4o`EVMSJ@q5xHAESR4c+lTjPvQZBu;RZ0IA>SRU#vLIQt#r z`~#-ufcceK=qv>e5EhMJG(Q)cSYGlFRm8N7~D2l5Ww=yVqi`nHBIM-adJ zcP*^Iuii0=IUy`wB!bpyOH~2O1Y#m&40c9%<~sJqFK*rd#gZ8Ob$Aj3v=O!8#?%=| z0?OhL-5KS_4+S}m>28z&jrXzCu>~vzZDVQ}{^9r>R7HhQcqun<(L%KG?GAEcIk9NJ z>Yi7C2PlA!!x$yXi8J@o0* zXOY|8PR@;3ashl|182C$W>W+(*Ri(Yq6b-8FR{9%e8j`ml;Y5 znSEyY;B6H>?g0VrUoF8%xITSLPgGp@!M z8)1~X-QZ1RwWK` zwF31GcYZ!e912rZ36T z4nK;aq<&d3gt~0uD&?2vhey9Mx0n4k#;s|3i~h;OXbQfZfvSo-3kyedfrD;|qZY%r z;9GWW_WyHPTD5wJkvy`xiIaMfH1MKh?B8{%yUa8Zrfg}&t`B=h<%S(?Vt|T5u4a{} z*5xaoOYqzHNNo7s?`c}ccp?Z6VHb3ZbYNhK&Bf3xRw>d{5%?GxsUmzT^+njnwRm&~ zhQWe)Bz7g>&Oy7i`jOY{lwy+476*;vD!yrt#w<86C0kNAkPYp?iD5{tv$Ibb2N$C% zW#M1-8Y@2E8!3t5H@)4)&u*o8LZQyz{p>@vlbcU5?LdYx>%! z8V7QHfs{L{UOU$sMb&KK9Pc_lD#qRiBEKIvh;V zfA1`^C*OXE@8wVh-%W7$O~V_>U+w8(7mX?32M&8_$%bZadnhKG`!MSRT{*BQh2wKN z52Us(08YKo;$b)MA{qv6{K#-@3I)k$W`#VqkZ>niVii*dFVYX9p-VYbG z9UDx!@mwBdx87+nVY9|e4Nk_US%XPYhjZfHzcN0W@BE@^aLndWk@*lv4YDh092S$! ze;zXdk5w~ce+K^-Npj$`S)-ori$4GZOW0Qhj`n+gRDNRiTMBzm#O&1EPft=(8wJwz zoQ8V*o@ZA0)+4ko6SNnkBU$qHi@f+TB5Am*fyOP4f$MKD?{4kzi8 z)_-g64Pv|~WUFi{uzwJYv8!wjhbfcxXRC~@Ri*#Gf=Ap3X40z8#}Dvg{xy(s|OY0)4oL&LLhHO0%-vaqxRv|YGCKf0bo5p z33Qq&$f3cI@u!E?Czgk!mIKZrV`XJ5usX}UN{rD7^M8lSL&6{`>3`SiS+SqK!7Zeh z3D7tlJX5FEL*6ehEzy?&`~AD#P6u8L#wB)K-A7=NeLEKbQgYk*TFd0&t;eq(sW+WZ zccRiHhV38fX%~nCGn5zw0TOJh|A=FgvE1sVd3Cgm2GQ2i8h>t`B@S%y7$LvM$HfYm zC&7Ca3(f*ZYI(5kL@~;cHazi>*r;TGwcXe2J!pM9L^mSolCcngx|I%Y_vG`4J+C5I zRdSz{XTrl%=OkLW(|WUZwa9MU;$G^wR)UU>5A0Arg9{Fk0!d)160xD9iCPDK1l<+a&o`^Nb3YL7epF^ z>INoaP#v{H^h(objHWt;<^tcw+nUDdMO`Fp^&uFr(#kBp%QvN-4f;qhm!4 zrkGeb(vF%L)N1+eM4LgIrmv^@yq~1LWm=+yp4L};!-6qWJM3cPa>mm*;%am&6>)Pi z5XHbzw0S8&`qw9>#NP`?U+ok4pGWupJj6&Gt`MnWuT0HxGe!xi;=^Sn&|sL*Z)ZHU zVD01<=O`SSeKDqK8JvSgHB@ zyH83%prRx--Kd)x$$uwUe3akF8Kk#wOVcemX(W&FlkD#IEpkIg!*3K;VQ>1m zVK=1G+=G+LZhpihpw|?^-t?7fx?qa#;43DKjz0K`CTG)#joRy*9s0K0vKDS4I@3vd zGa)ac+xAoBnhJgSbl!^b%N1YxfZI|Y6}8(3&PrYJ4OuFAw@l%T^hIU+WkLK^A6e4Q z-G^u{+eAGnPRCqs$rb=Z;qQ#5j6O`rz7O{9LB`+t#X?RJ70+d3UEBD}l%Gl>B)F^003I;)A1Gy@6qIQ|e%pv24=vAW&PbQ;=Td=B%W0;s(W zn5fPt6<@%#m;ZYNsKbaxFnXrZhAo3;>h3kbGV+5FQ1BswV6m*DE)dL^Xb^|h4*$-6 znnV$?)SVlLf#ap-B>p%cB$tKWQ+Z<$v8$1P4+O(`N(WP}-%Gf>)rTW5DXAYe6SQ0R zItDICAA3yAK24mg_n<-AGgtAHq}+k70v~bEA>9)ruk?P}i#RcF2Hm6jux^;3)|fNn{#VU6_g{Guqr5sMTQoR`^%TQF+?I zcIKFtyM^}O+o$DA6;icHv$FH{rfy>2Bw%Bw~u`+usoQ0c*d!94O^UI6Z!Rc%YdU9ENwV{YgO)*I&9! z#!Mh<$>gyoAmln4PFqv!gGLcq_wS38*?XGlt;+;H+yZBw%1<1kp+)9*@axzHJ4UW| zA%u#AQu_1rMnfH*FD#>%x(a>{k&jAACFr*Ookx@1T&2BEZCzPx53coHO{g>Kc%1Bb z1RQH`TrWOF%SsHzP;s^==`!Gh2&3ff<`)(e09{T006-HCAr){ge|-hgi=Ha&P1=Dn za2H(khre%NK%F~as}hZ>ikn{;sr)i@Hh6hu2AU;mmqkEU5r9zg{4K|VKSrfayNaO8 zO83Gh?v*rssGvXcE2S1JHc0gKKG5tHK7OW>9up6mHvE}YmE~4T<+4PtnRzQ6`HWGOk#zz0#jkXl`jq=vcF=L5xlU^G zD|Hjv62R%#D;&zj$qY+5n}T+Z7bGKFh6pUyMTG55X;+%xcw17|Sx-@bbg|t~?1Eml zs_}Q$+0x(^a-P@oAc7TvWuKjF9>438(VS>t;QZf5|BJX|4pM?=OTa2o9Iscx_Ml_B({JQUiS)+p&yeESk>V{{@o2Pi>{(X^oT=jG*DWjq9WmggI@Ln zV``SPVRf+fk|F?lhzXYBtO6-lgpZN7^L-ycDpk4Ea4Z?$WbV_wwE`msmFuvCbB(WV z9LM9KQ(*ZI?*3h{V5imp0j%03gY|1QDWi9~?3RJss#0J@i%}4WG@i*eOxP5Hg|3`S zFyiI1ah!q$E__H?S((NHP(Ne?C@k9hZY^|6XunuF^V=`g;;9J$CMy8ylQCmt`{k7r zZ)OoFTxH}vp~zSTAda(?A~7oL-v?R8?@El4{&$xQkRM>1aBCiHW)KhG4kgS1VIvc5 z0Z^<3LB@fqNYEextd$Z#CJ!#o!P0k$Q6|4j!N$h&s@)Sy({4eG)|jE2+p7w;2V9v= zrcIFg{B24_D3-JJ@5`3$mIIcAWGUzj?ZhZCswOqzo|-$TEYW8(n@I&;ip%X2-XROZ z0SZKD;bum<9zcp2r{Lo!&WxG|bxyvAR^tb9B#mro!GC3UOhb2fpbVV9x1@Rbj@DwR zz`924_E>7xn^qc18(d$%T<%&i#{WL9qqh; zK4(~JbY1hkn;L2P()Ot)zn*gv+vya;Rq6EMS2-S|ZDB``v;i+)g#0-37NPV`;kTy` zNd;vOW!6loaslOQ`YmWM_11_cs3ft^9F42l8I9s_UN9yodGH5-v2&}1NID@-bRDr} zy+1R#0i46G{Nxm=Rxqr07ZYsM$ZP)Fm;u`hO0Qxze1$^68dpYt9EBoL9k62EXDKqg zzSYCm8{!j<$)rqP>MHe1Oo&fLTup6_ltw>}_?YS|Z||SfYe<0M046$@D?3YPn-0k*lsv>>sl@2X%vGYYzx#R?7sLVjSap|7k$I2V&@3wo|L7;+{0)0)gg7zG`aXTr^jl z-Ds}l0U#FRV8Af){2PeTPjUf*WmH5T+&|Q4^(qz1r}IShzN12!~M)}oN zgCT*ws};J_+$Z`5m#^>b!L*e^_U^O~iE9($A0c(JmdNW>waGyUCqz4hTY1ndG7;mE>JeE7U* z3=XToLt3ji#_&x*9uV2>tQ&M`JO2bM>VEpl!00RUgS zKfi{n3%`hdCZr0MYHmU)F#0ESCGgreq6mEiaX)RPZ)xDES7I!w|E^t_4Cn*Bp+^c1X+TvtS!WN1?ilYI4b3fe&1YvXLJl7x;EV`@^9AD#p}8~8Z9 z!{J#slnD`ZjcjZY@*GVS^qb1C&&bpT>DOwqu3mpmF0UmC)>*o=7fEcPZUCqblj-I|FMRH^ z+ruomU$0fSG>G^_WxhLzL@iG5;YbWGgE>f5z-TUB)VRq{@>})eTQJ>xyESrUSuNXV zih+XBNc*htN9#D>v`TllXiC(`WY!UlX-_UXRz?v0C#NREcFH@e#HNFke;x}$*`Qb? z%^noNE?;D9a=F}2AC{VvS%qBCfhGYfhd}4(X|GdoQBJ_2kc-KDP2eR$tLiS4SD;wi z;C-v1pHW{gUtl|UtY*5vV^)r?zWr+_$)LNhz>SthAjA3>*zxX8X3-En2X$aLaFj+V zmvb6_rMAd<)Z;)7m{7_abQruT9uw6;o7(t0-+OsT*Wgo!uU+x!wFhfWLFcnR%i+S3 zy%(MA?$Ld6>B8^Isto>oA!1ZctgMqWAWGw)J2lq8^Z^kBw)b;2?!U(K|M(xiZn+v_ zv)vufCqzvyzI7*hxV`uaFoJk++|~Zy4MySver(kLq3S*0sqVwS@nd9U&t#L3t?W(5 z%FZr(L{>J*3S|?8><}4Q$qv~e*~!Y@ghXWYT;J~h@A>^7FR%N)bIWng`Hbs&U+=+| zQnr;m1{WtxC9b*k#t3#UKl+G)JBYv-V8?>3B7E1@jT|+x?S1FHeZt0cf=XE6e)=ru z(7@ko?q_h*%cKGeTon|Rxu6zNi2RhidW9es(UD-hntG6OZhyW}QoaT_$SV-`rh2dX z;P)X4OYC5Hb?={_9{=@bd}q%yZsKtNz7D!GiQ6y{-VOw{F;2m!V1G8_V%X7PbG-sIC> z^6RZvPOYm{& z?WP)UcjhfVp?$V`XW=K&lH$^Wohdb|B=y?>u9ThH$g=rdzn-6)iE^oYn{)#Nm7SP* zecQTUjSH>}jZ8h3G%nLH;bH5o4hfy-Jl@YOmGaTi`DBV8+Dk7f+e>GypjG zeU<3jk>B{Ju1`sw6K7WC>n#8LV|T0fiYL#PWXbTPSiArlla+_@W5?;#KNALPGO3piWsR))nYoht8iBHRztyY$XuklS~N9`E*p-zR~bZm-b zO8(?m8GUD)`MXwalEpq5Ju5Ff${?H_W*+un`taT{Td!fRw&iR)FoocsP^Iz+>QgnJ zS>8^@S@%9(&oO;Nub(jyd@ZN$63Z`i*OF5HYGV0~5ANl~=RMqCxmF(C$f1hqShKqW zen2`!hpsxKLi>DIp}%=S+lEGet_thO4B{`*;&C}4h)ic>T}irK-@(9t%Wb&SqmU9o zZCw0b?FRIjqKgcAq|b6a-m7Kt||syNOz&6kCr1rZ|h2wDkG zP5_QyDjcggKr2x!(>7XU>r6>p(qVjguV1t;%mnb1p2v6DLbw&266lQmH+ijic@yh0 zd^V$)0#z2K9#C-KqPJ!w!PkA0__DmV^!ZXlVy7)waGQN8qsn20DulVsqMTh#!atvxWvZGWu8 zU^X5#C_HX*6FnUM6OO{o!2n*#luP_(>Ijxxg!7~A2|-fegO)SzaF5CCV+t|`5MP4p z4;nPP9~StoCZe~yx|A?q`LFErLJXM5TG_ttBsT&|#OMAEO3M-Bp@QIhcpNC=)#O6ltBCk=ZK%H!&AQnIq%r|5u@ zNbiqGvvR}w*MKYLI?;$7&4spjWl^xJYsVE87w19(^-wf_vkJGtTkYJw-@hk=oPeW# zv*c&vMhg%GH^8HfV_Ygz*on#a$**f7$r5GS7`74$FCT=fW8bx1qP~o=g{QPtCcK;0PV;)*9Powg> z>d>@WbmB`Ro-j4yS@#rsx6EZINMP0v!!U-1{Qa);XNrL+nRgca5&RNnRGmMBy zCIeYse%ljt`(GB@xp_`Da!E9!PM$fv$1cg#2qosbFMgmy>en`^jV##hm?kiD)G_yb4f z>xOAk5B|h38-t?&p7+yt`Dk!u4P#lJf2(@Nc(zD`6ys|AO)rpz|G6;+${4Z)J@-|( z#Pg9agXO}+o-8|nwX(=rqA9h(c`^CL7#SqPb_};vy)#x7;G6iy{UzJ{tD*0iSI_oc zPqV#P(bp2aC5GJ2P6rO;DLe`2O_BS$m!|4<$#Z1SjRL-?KJVRt7Wox<`ZdOs<_C_t z8<)8kgLdVOiSA8^zuge^DWkk^`fK8Gozvw?{$@@eUaeFGI>WNp(eV;dIKMWPtLz2} zeE$C8pdBk`ngJ@}FAuBb-XxGgDW;hDt=F;_-Txnv2N=w`o?jVysX{)50E~*KlPQ01 z+KmJ=mn=2d{9XLWN zn=YRn0~r}l8?+|(VcWmpCc7DR{Iie02=tfrEgYGiE4Tnt6Yi9$~!J8}JKWIclbO;(72eVx@IYOG?DhSL>0YE(W%P*{rtu_7FO0V0I7=(qL&&y8B5V_Nq3U!HQop`ayZGF<*3JXGr-6%N*aEk7a3ccnQf#cubQ8#_X@7(?+p#(;2flVlBLLB z@Gh$3uTAO-3(u>3Q0i2G_Opbb1TFbuK4v2`(L;v&HiDn2om|?cM^D9y#`;j zL14?aEEY0l1F$~`Qp$fg9(c-)v|WxL>ACVq7rOQ}NUNP#0cgq_I1EU}{ z=;?}yPC4}H2m<973UF1N0`_MQudtH)oM+#8-+!cP51X`7cbg$bUoTv&QY|Wtk30rO z+-#MTzjORFhct%8qjPIJP0L>lT)UL?r8`Qbqu%7#uBoAJQac(!AWO|#vLYgGM{)-2 zPG0cb$gkFshsMtL)1hkVyV5SU~NHU%#y6Z((zDd8ly@>JWXN8)OMb=OwYdWr$b3jV(3h9#i5YPfYDogbBAT>-(S{@&*V@|5P|WvKTL|1eScSb>pOy zpL%ZJ0s`Ni2@}b5>v-v8gj`4~WUu}Ij-82=(P4i{bW*VbWfxljjPRMqW;iWy`Aj?-?KK)i0TABX39$ zT^Q{DTp;@)W(~jm&U_eyfJbAw&=H9W2IG$T5CT3q7<)iB6baxiDPnjflHLgy&J)?Q zbuCmd68f6${}Uf!T=Ro%YoqrzCfPg`EA4P%W221fTv>oTttqk2X3N9c21^A@dv)&K zotpOtRyN|q3aNLNGYwvdX4eJUj=#O9e|N)W=MYyA1Zk_AaT$B)DZ!vAxh?boj=d8v zCTM+pv%)psoYbBHbo-+_nfI>isE#rVdE`f{%DLi_vNkeh`o~Tp(Xb#95doMWlYqNd zVhMKuY z2|<$ow83G`OX4+1KorQp`={s+K&W#l{YRYSp`U2piDT#!(|sx6*f&~6rEp!U!gLC4!4TOHB$9E?2#XkTaVZ`=y^l+g+6z7*E(5<25x z&~<z!i^VtbcoUj^x7uYj~=7E2DKYpCMN+v|i12{cM zj6rz0oW(1tJu7hZy;^+u?d?5KA-;cjI=JZt#M1fqcawy_EMp6>8mUuNOuasc=C~bk zHBmsHrZqwv2$5OOFXoX|k7QJBR9+B|vEAtS^5Q%#u3 zTMlmO7B=OTeZgB;AEz*3Wy4&?E zJAi5y1`~Qhy9_rRWT*>VPW~2pkkTUD-91Ey43xNTAWBS6j9{W*0*nQxE|<6SY;ZJx zBdr~9un`K<=hPdxNd?$nWcKx9WX^8I&Ffp7iyhIt`9=JX(pr8XpszCH!I@BjzIbe1feG_=nZ4~Mn)~h@xu!0yrm1AwQKH)_kzjSLKMD3G7>p!GfIhzm3#Ya!&GFlTCpK9sUR6*kbKvAbxDgf1p)h87op zq`~TT8*Ujx;aZKYX<8FVy?4k1%e^$`>!w-MYPmF3^7|zdsw=90XF|f6D?SHG& zQ>$b7+&zAL);5s$MInnURlT5H1_te&RRc27W=(!xM}9oiq2rs1aH9u)F7o~qbp0-m9cF4dir+g5nTS}6>U4W&@Vf~bpPKNYx8*a~23 z6Aq3Ae>QHR7x3Y_P^3xfoP0XT6?#t&{UCbPW{*vlQ*djNhxq|39GV}E za!je`Grj3@_rh2kY8wIu_dfiTW#X?<+Ytk9pw|72 z>oV4a8j5rjOOK4U-?D+QRUWB!O_akzq#DRQ09+G4E8*l%N92t~@iToUZWA zmh+o~qno7uV37B%(T87?{wq;adZ~)>Q`Q6(74z0@6||Jfo@r8z&Gkt1XgvB3_QzI- z>NYFER;y2K8-?NhI#~bwRdLb3g5EU( zdy;JWI+o7n15!Nf3hQoSM6vCB;{B?uEAEPNOe>s^V8CoqZ8YaR6=#%YC}KszBtzdG zFG1LjYiwC4*--VsG91lHCIxB{UkIZF{R#Lqx&j>jN{#DD0n5$7Z{z!?%wz+wHG7+6Xg%%UNxqC@5%de>{yWLYV!9yGk{wkan%K^=_6uk4Ll$z z@MO=FMgr0Q`-n9cO0zc!fu9FAVI@rbfz*cBJ|pP>z}oG2;C10|#*ul3EZIDC)$MSd z#sDR<&82n`XagO<4?C&46dJ^t`EVQPMxsCm&w99W8TC;|nc)cxB}m_7xcA136ex)! z>`fECOD-SwlrmfwBEP+Rc;mKl&ycaDkcD|8iPS)nxZC3px-kbqZsPvit$oXTSSW{m zg=?tJ1jnkE2|X)%Kh)&N+d=52!ln!7-?|g=;fLp*@qM9;{ZA-E;CLYt*8jAE4QdxX z!$~ML^f6QWcq2VJzvz@hiDFaeUd-?%?1QUaRhvr{XKtQZ`Auj2dEbXw`cQ>&$JqbI z(O$1$p^5Wa>pEO!sHRfFqW3OY_{2y3?JNfmN^EAl!t!f)k4s&cKpwczMWnn454Pkk zuITFW8;6pklAq@3kcFJY^CxC+bTc|S>wdB}Fh><{Ms-59d1LOOCfOa37SlSEz*_i8 zyvv+Mw@GPft`yksQdNvzLw@^eeF4gR7ZCY{vAT`j5$qa-W2d?Bfe<_XSBWNKVa0J~ z{9guay5MzV&GVb#R6GXjxSpHgBRCEJ_O9aFF~j@A?2#cah1g`=g{0 z;^+x%)TetMyH8l>UU}OqFRDVX4no4-l@K}35Kocud2#V9L;MwBf%4pY?qIFJl_@Vk zu6uTh3hrhKW4=E2Ff@T?c%7%1d%HKozg6JfM23TP2jzI0&PMUvOZuIWCa%*(pb0x zV0vvwJV-O_tn}Oa!z)f+f=-Z^0s?rIA7QBktj?FCfSJ46;;qn)zc$v$3S` zjU($bx6qxx7V}3nwXGpA3ANka3C8RjTg7H}XPeAloaXU^xwlP~kAAzn zy;FTLJE#G#PsRvSi$-idj6%Tan}8?SX3umSuRUBkDBN@#4Z5Ecbce5D)T%iwgh zLic`WDx4?SUg#p&xhaKXoo#swF+FmU@1Vk76LAU4d0jD-pDRh+mT}$lb$=YL_C(b@z`a|k;I`iz@totPPZy#dns*vv{b+>&zArh>UwO2z z9y1~d0vpHG7Hwy^xduZ1ltWL@n^ubv8FfDLd;<0)4>u$HCSP1smPa) z%cuCv^jf{+v9F&_tTK4B2zy(HKRWoT&H#OPk<~+(*`t*NmM=_O0k>G4N@-M2{Mn~Q z)#Jp)<%hjr48cq#R6gDaK%+XwxGokv)(AW=;iRaKW_Ky&=@FqEVSUD{I}|BXxXquA zY1P=sv|ck`e_HTiZu$p7+dUCgRv(_naDBT5o4pBTj^g6rX2w6(*K8zj;eFm?Lx!h3 zUtye}l5bi4k~H?Jc9KtC@9%m?YX%cW*^Odus?}0|^L{F-sB}ZKfsI?$>B1?OQq;za z$F^^JC5Ul1K2+Pu?%TfU>~$4-%rshu`Gat<)O@CH`=VNHb+Kx*n0uaxGGo1ST${%a zRCC(6g3GW=-zx46f%NSmVr|K_TD4`u=Y6hZ6-k_ppVCe8LhLuZTs`Fc>{iqZ`0?3kaTHsFlPDkLiz+A;QV}H zc5bh353m3fD$92Pe`~5%>sl;gh%q#QSTF-ag9Lzvr=WSr0I|^;%(sv#+UIceoEO|H zYaZZ$!M1iaIQuIW{<*c~qDCFIAWwj7^aZ|_8FdWc<(J{Ro zi7Lv2@KeTWuY=onO&JI!f8fr#O4LnSzi=~}l45SA0$W~c9IYRZ>-L|xyZN?n4<59T z3|6q1L&746m?0J1=1fx7#{QH%NBR-?BZdEX*Qd9=u+%fB9FH?z^=7zhmtda+g-oT3IAh+eF1Bv{li`7~n}2{M@_@1K{H!%VxhOvy%dGqoqNYc{A)Y($ z@xY1b$ACA*hyR_k@#AXA@XvdUeO;Qp{Mv+}xz3+9R zZM^g;u8Tq?|2k=u0}fWAm@>BFu6n@@JY_3p%8aS+awY2cz*AR_T6ou_&*sjAmzC`BO^36{t*mXEvwob1gUTW<9IH`f&tmWL}Dq$!+_ z9X(G%3QN4(>;;yrd&dMvf|pjtHh*!EeHhyg9do*^XxevBD_*wy7M@=itDq27C%SKw z|9U}G+Tn$EQtZktv+4$LFim}rB=gT1KV-~|ehuL+T2w17EvEfsK?trw zyA1i)^KS>&SG2}_F#EWS>9dxz;emrW=Px#FqKOQli22kHSKQlVs5t-jaboKmve^H15F3eewhXM2B(}?KzvM z|G)AVc~zM%{I|8-7a}WEr8yh<^MvE;@2mdT;Ev|*cEhRo`$tw?Qxyh2o?pCL1fcK2;k8Wr+#mujE z7h%mC(Q6zCLv#3P#XX97g+%cq&CLLfB-V?)!Y^%EG~A8v>9yJOWT9D;aDuP@e$7v+ zWI#GNt)1@U`xNDBt21)jzl!p42F2@_*7cf|^!Pki##z}Z>EH6zG8w3`gc!wX^HGt5 z?GD&xjFS;7ecaC=zsZ27n6!4k{zqA-6DGyw>-9fXY>X}9I-lFS=#6)S+lKwy^2bwj za@J;g)h{)oHi3AWlT$+rt+rp2CY2*lPPv6kY^P;ZO?arK9)IW)H}^3vklvY2GTm@* zvPE&V8ph4Vp%>w&$O|P0b6$U4^@R2VtF`MREc-9A6bIEww??bC^hlH!D_&|;-0j~z z3MEx6h)TtI9g8uM@yc85(W&cXDQtq`LxWl^VMFoe2G8~ClO=(pBH{_ zHDcpkF|k#*M-qg#HWfx{Mu?}d{uqpblr~58yCB6{?7U>&aeBNj2)529e(U)Vl=f2C z(ZEitbR<$K4*-{_XY`Tii^3lJUJj7}r-rN^KRA1Lr_@yPVyCv0djh}qBR zX5JM9?t#1k3K$%UQ$v(0kW+F%QkyBw`+wJ5tU1x2ogd&bwWnqUU+{7ZG>#00j1|F#v8DNwadzCUJmyMb{ zjqW0=-u0w}unAT>UKHBOZOv0E*RAKwh=(Sc^(H5MQkYCU(}A#ll&(-_4{L%tcU-r< zRHTzZlq)m~_HRqRL_a206iA+F=?t;Ho}|Q+fJv4VX$c^BDwwj-ZOWl{aoQe|Y0U7$ z-%eKHf`QnD)_tiA;=8F9>d#D4>BWRrp0xY}WX#A+z~`5>QVdTnKqH@Z0jEtTA~%B1 zOSt#XQHbTs7$vAq0D%t6a}%&CEGhn+Es09+G%?sTVyB5y=tj_gZ9}1Csq!=~4jA8o zS2@5nmO^XM*wzpgzzxM#YXtaSPF=$0j`T`)xZzy1+!#4XVg=Ji?{;&;K~j1d%bHprJw zezswo^1TF}Kh&hy*8lJ}qtEP$0yx3m*-LRL7Z$;hN`O9e0_=oiq|ccnj8fB$8P&c& zOX$bP3(K=jnB21|cA|iT=h8x=8he{Xm#{I3$ZKaa+C$33Se4^w0CG_WgavgP$MQ9J-TMyuUbRhepTdH#DKlk)z%`m z+mIEI|CC3Yl)$kh<6;>BY0NwUNXfH&WX%U`u=_{n-t zBS|+hDjE9$62(Hb|p=7_?enbZqu^328?yP7U+vs&Nj6f*S6 zFpi%=GMxf~Lx%KzR*jSzGI=_?be-(9kf6lQj}|LE*KZ@8`RQ~WxRd;w10sv8SifW! zZbM`sN(0Ip6Fx&bE$~EKwEKt*93a%^6pH?C5Q$V+v|@Vl#^H`5C+dG6N2-I6N*QZ@ z`G(z>4me#2|4l!E#vlNGjG9?vYQ8O-eDkvlaGKLjsGfRI#AXX2+L?AEa)Zx|#dSrqZRF;<_16mO2Ndk$_& znw8Y}tG{o36+ULA#7z14(?ve5E*hzGQZ6n+fpo4H)vDnB#4*gZTGF?zoYWo%5hV>9 zw|WFKc{;A(D(FCA3}`PY_EX4bNs2Zi@6;7o^AeyX?7f4LXy3Ol6@!BX@A+^lVQ`3J z^$Z>O=RJ`xR|dlq{5^NYq2Gt#ALTopF#CW9`iDCQn?`jL~C# z%Nzj*OL6Q4m=^+m1SHR^agR<}V9CK~kIHQl9D(&L!$!<}*>4}m5(iUWAy%Qo37LQS zD}#E5TQQ8^STc-2ajIs_~3@U6RTtlSKSi_wSZn%qMh|8lTo)&O5F;-4xgGa+&d8VFSQvDL`2!2s=8e!=m<9mvSPU=96JWB3l3DTd|kUFWE!U!TeEIEJQA-^In8IVQ^gv}Vg) zi!I-cr{LleHW@ezJ=Vrn3T3P5Bi@1Y6aGw(+d-kKCr0R9P_mBqLOm=NuhmXZ3{@ zDem0saZzQ#MIxQ|M91Ilt{dw+|4h3T zzHyu|6|Lxhe22H-BFwGpfSv#f7sB^7^{tz+G!W7V-UC=jX`g)svB(F~_Z|ql=K;~^ zKy32CwS@`>AktuH@c2ts*3+j-9J`=mKnyxNAakV>1`Gb+?X)&l0!N-2j8XpkSTyV_ z*A#-`pRk$l`~{n;350FmrKMTf=L;MXqZ+4Fc3L0I9muIfEcTElJ;l`f^7RQ5VVM>A%$KV4ugeAy;R3>I=q{P3#3U?3KwQav+kk8U9{zj6`cfVNMkt$^P6#C?6*fAUfj3+9Q`Y?(5_Y% z@1_oerH`3Eaaf+cOQE9XvxQ3AJ%RM+t8E|m>75L!HYP78ERmLaLl#~!_GAV#XW88e z2JWcb;cnnIyfCLrnTmg!TXZ9Jai8LGr_=4u@YSEJ{<`P1 zn^64qFOy#EmJ55gszau+FF4}vB#!zP{Z)ioi#4T><#`WRZj~r0j`drf;puIX7019! z%6HUQo!m_AjKh4sjmpX%6Jpw$b7zRmdNJ@~`NTPt@c{^ZVNLV)*nOC$Ex5MJH6iIgj z1y$?LZ|nXv;Ugf_xq;Z$3OF*E(u&9`3xg~2K+w6r2KxN6+hVNf0syc(TlY^FU?%ob z(E2GWtriUN3{AbLeqH^c70Z+jered?R@RObiZ0V)MR~7mLJT4B zIwb58DRd67iYS1INESZBJ& zh>~KWkXws}FeLc8t$!BKwN5R~{L(U&LuKfgD7a}(}vGY;`8q-b9e zB}83dNm~_9%KHlLkPZZwzD}5X@7Nkvu(qvip53bwSM;PpU%UMwN2yqAdQ>|aO7|q< zyD|lLY3~-de1;Me>Pzj|2p$g5du#?h7XS`*cL>R#|gZb7*WA< zA)HB}XsdA(2Hy1<8{a?ueu4FdJdt2;?8&M+^*ncs(%3#-EXV7$HBf6rzFs1w#?~yS|e+vver>b+HBm5`4C#H;zWr z$XHPtl%5iz23vxH{_6>{=a9w~Owr?2pR><{191n_`7Z7$xRiAeiB1QDcMB#!qMKXS z9H6+vgDwmqd;uu)7)%_II2RB_&H?3bW9$D<%$+l2r?rEGxZBrI5dGAds~EitTpDZW zA?zI-{>3*SaQ}0=3Bj1kFIjC`UC0Fu${@@hn|D=jenu;JUpCZNW$gg5oq)@99a8MY z!|dh?MDJM$yecgv3OFGN0}jEHp;sH1qPkkCB5Sw5GuI^^&FHa5qpuUetggtNbGtqx z*oYPGil1(cPaFzat?`uDRx78NfT|@APMaD@SisR*S2wr}bzraxxe{16Ih15@iCzO9 zK*BqT&)^1HOz+)6(en;(LC%cO@okBc7;X3`m}JG=qQ|mP<}fV$y|ITS7ame+5^ZDy zr6FQw1O z=KEDIS*|O1MRGzOh0a}5{94gh?&4V58SRx}zjp3kPkK7z@OZ)o%}L(ZZ6Y~GqNEoh z&WsNspG6(UkYV$tbWvqz;}(bKRh?yEF(}Syu5O-1DYEh~9lPtIaQj&+ z9^UO=o_hli8c)@eq)H$yj5z%Q{XnaWFgNkZaKkikc6lBGCj*X~7x+KmCH^A$+lj~c z9d?Z1ubo+@bkPS0UfuNf>B%C_)NE=!2P3H~uePLPV5m9&7)ZrXAQf1SrJ!Q4o?r5K0$` z^e$s7F52pJ^aKes#B%!4Sw_HdGJ>w~NNarHAD{gs3er zTZdVg1@p#UJ;YXk8BB%oXIm?)Mnm_ohUq{kaB`mmql8s4*8)09K)sVjlH+U3^wH?c z^(#(An`8HcuTVxNPa4h$o!_VDrz>2e1A3U_^z-8GT12J%%g1UMdcZ zv^N;~oD=QSE?HjCA=@2eeNPfwJ!aoW(Bf{oy3mgqsYeU-0*t8vzbiKS1!cd!rsO+g z`H7f0_Y;7Epf)aU_zKqr^KJYZ=VRJwXgTy}ziq=v*--o{ixKq)m}lvyvrfxgyEpA1 za_x)l1LoIGdYlFQSNcnQtwTAZ@Sf%ku3sb@Osu0%cA#6$PI46@#%#3_Dr6lq{gFGl zpg7>rC>ngLkqR0;h*H7}d;OcHlU0Pdu}HUAgVIXI_LWeqTwqp%&*sy>vp@5&R6_gj z-4vpQ0C`xGGkCdLgtm7?9_M!n`;IX417KV>KSr21-Z;G&Fm76&SQL6C=kus`^Kan!(&l|DMfG|z-UB0(L}CI}ACwb4b}TZd|GRhn;Z zdgWtPqA&KmU~6O7y$nsvC{^ST{CMr9iVMNs=Gx-TG(iF(f9-Z!7M+VOHU^#Eu>EbO z*?8${ZbaKahypWp#h?|7MFm^@Rl($MiOSZjI?vEo6D-Ihg-P&NhLlyDf|*6I6$!nu z(P_DOs~M*E?rCLD=mrJ7HFg}x2vGcNC_QJoGrsxZhj`;>_?pP{Bbm+p<4-^17)|<( z$TgIsa2`jcr!s4n)rLkEBfet5m7+OgqDBF~MXG_w{SwLxCG zP3{1vSgIGdZs%M{FDpsey!s_OTgggytpI;G(?p)0f-LFc38io}74F9}^AU|O%lGdE zEj`|%NRt=xYTwW@<6UU=b5swr;F`QMA)Q)qpT#eLdz0Da#~QVv7sc4E%Qr^4vZ`4; zN0dmF)XE&Wh-=fmUZ_Ps3V1Qoe(acenYwqGe2NCCP8a&jkGYm^Ur4rn4BSJ;hzz%` z%ScdI&ZY$8@49YFYiq0r5Bu)!jt8BM`)=rFM``&lP&7)TiNp5^c1s@w5b!_jQQIvx zY`RDiuy#|BHg>l*=%n^&;P0uH)UO5i3zAjQb?2k zev0i5H3~8JK&}^rNA3o~miaG|JEy6{uH9~~pcI`4>bup>D$l3J7^3#M+&#W`)zwdc zx3sW1dxR4wvr$VhFjP0)E~+e=Q#Lq}WWgsA;+q`0%y;w@-dfq6RqXpYwED!1=jNIurh0h2{j6QCFPM${ zq|k1|y+CtLjGf_qzh4smZpI z&DztDdEc{=IU-eMtNop+(_3OvMZtSd^-1&Iq{UHkqItE1vF<1GM8-ATzY>w#Hn~|4 z`%pVZ;yXPzr!B*`Uw>*ixFkw()TiXD44!QJ78-I3@NjteQe5I|^6uLs4P&7wT|a>3 z<4y8FOK&)Zms-)OK`_^KKtuR_8WHD>yt#X`z1Smcs^1DCx7XcCDlEk#o>P98vKW6b zNo*3-%trG-Yc#}BOqT!SyY%!!F2mmZ&+}W%s!;*$*0xXmsuN4k+Kx4>7OZ{G=}XS| zB1aET^NDa#WZ^#E+z()b^yRmmE%|vI?FTFRV13MEJfrc4!AKj%BT<3uft0v@SXU&J z;71cp61W?BG!!2gmXt*G;ryVy8-)D9DIb=uBHRmN|Q8{S<7TE&XWX*{bQy954urW>c3?RtJi~nOEkBV8E@@(sP{Y4zyD%K zEce&E8-4$bWXHV$iHa}sTd6hG%wqq{-BXtRqjTEIbc%5z)Jswu~}uwEw2h`9lql(yNfXMa z?&q+nz2v;{DOJ+TE`Hy>A)=)q;bXN!HFH~~wWi*$M&6ZYjNCWyP#$ZV!VJraF+v&y zIs^N`bs78P=<4>EmfWCFO;iXDmUhHC+GqC2CdWwV2;z?`O&1xagnXn_OO|gem?H;1Cgsc>5^yA( z^D$1*mK5suawgfZYijB-9NeLjr;#VW8sS|#mZp-NI-F9SIBJjK= z&5uJ@@r<)95KYoTRkOKQ5iU&#nxqp;V`{%9#;mg~JH&FSD}DC#p2`G{%Kz@3zqaC= z`=T@(&G)PC_%71T;`P0A1^ry!$hsUUHqF!4j?p*L+xLATFUQRHZ_8<4%N%46aCvUa zEML=VwM{X7davr6s-HRXS0jLEgd6nRCny{&zY2FvXr6*AvZO0G!Ti{7iprfs^1vbR z;^yJE-z|RwOx$~}`tC)W$NX;o-DF>V=kKE1NprkIb#LJLuQ#%>i}A97swypT+wJul zHvcQb9=$c;Tg;8-YxBt3r$>D}Ud<+%{k_*P-b4&P90}0zY2IhZ@c-FUx0I;nI(h%i ze41lfE3ySaqDGBuLt>4Q-7h}P6!8x~TPJr>KV^xNJ@@rr<~FyC#}5g{l|5>Oy@(`H z9X`}J;YO}G#>0y6%7@2m%Cb?lAC`F=rHao|N=tb?8>FT7BDHv=nx%DyHidFF*wH$2 z^xcIo)C!aPo?Vw8e`m_3im6IYAYJ2@At%c4S|tVh+7z7FIxBLAEUK&&EVb-he%tI6 z+4o=Uf36sG7Pg!ub{MqBia>C6?9lzH6VkKq;+*#-L z3VM}9GENsq|dgrk$$u|?76fx-mt5d-?0lv2ZX^PE0 zcQwSa;tlc%6-R!r3&5epc0UTseP^7IV4wM8=M^h>yAwRq8T0yj#nn3EuMrIe^cb|&R0aDZ9k$qw!wP`#s zxjpYLc0z>E3;q>B-V?TnAcz?cQ)-|c;G8J1T)qIs(y0qc0#bhctHMr0wnR0pEm&xy zb0eU)jg)qw!8x(rY-c)mh3jdj6%>?_egP~FNauaDFZe0Ck^`hI|vJfAmd0L_LEg4AUju&Yd-dY{}uTDb1cLzj#z9C8y|9$Dp-I@0Z9r$i5fU48IW9 zk`5vs@vdmN`M$vVQ+YsQ{d?8b2jc0X1n<<*MqD9j$)TK@gN6E*6MSh0v{FIDs^mI} zwlx^cEizLSPx_%^ZK&YbAneC+u?^HtSf50$KkH_vT{kGubh98+{q8VIptJgPnORkLSTr+$`rgwE z7?>FubKrjJuHQ66u7rDK8+>!obbi zK!Py`D$yBFiZ3t_eOq3>C|v%K=wFr_?3YGq8{$>bsK%XUI;7Cs%e~Zu2YdeSQT*SC z9sgCyXSSD>nF55)&v*VZrl+U-&Ypblz5vI?e;axgqc?C}C~lDkIUtCocI7^7n(knt z5ULx3DKH*?Qs!&g0Y`%Fsqcp(cp<@%0*xDVoDzhOI+~e1Gxh)BkqzmX(;v+q0jp#Y zZrwIOl0TzIq;feI;HtvtS`>V=6>%lb$Zv*2W)I){1BcW+{!w(x5y9Ki(#n~ACU^uC z>Va=N9dhSAZ`uJBb~kuHMTo`i?AL*d&x1uUtdnnUslo(IPzK9fgz@+MxgeY^+n-$)ZFGk8%_%*pOP zCb97HqC(o$i_*pP3p9wBd9VEMpK?@0-_a9(|9QT6hrjncJSb+*4{P^^Q<^DJWD#uw z3Y6VO>pt_oGO+KUt6KBUbdbQdP>rC|5dQQo#;^2t#G?Yu?gL9EXw^zW`Ghimk(NkA zj^w-CX?G`8BI8QBj{Qv2%uYG}=T{^7CIk5@P5K7aL<_hbMYb$+*VICaByIp48Qt)s zCT)=SQMQ7S43}T-hRVom*KNu^@BhQrdq88|{(sn;fix4k2q zNJjRo>`}LsY_dsqnMqbwcp^I)WmNz7$MgGs|Nry*|IX<==Q%n@_qaaS^?JY7MSTUM zl1lRBqIS2{e36|rJpcM(^1vMINyYE}>Y<^CfnVZSTBTecqXF!!prI5(gB?dCmbZS)Hd^ zKRz$^|Z*T>xR)dNTqa9#I013Z(ZWOWz#W!@M>RUQ|L9a_SMG9;{iMZAW0l zXi;@#We~UG{(J8a!R#S`aH&w#MjypG9j`Qyg|(kUASZp75A-;RDnH%$P@>2C3)G^% znQLAHEe9iIf=#t1WeZl9{La9)nZv&O@=(mV=iyQJL%R9W7PusA0R~7xl$j8d(yd#t zNzER=uegAn9AtPPLe;I;OuE2zg=ybzVA&qLlsz?da%;Lap9Dg2(n06gDKp zJs>1zb4RnozE`!d^rE@TmGzQ5F)Hr{sVN-RuB}9jU%(h5i=4GhJ9(EnkKqJRPJBr5 zfgrmL!m%RexDTt;C%yq1;duiuZFA%Ko&^lbXu`VP(L2bsP#n>mO!>hWo$t?MM*8la zLD@x)u+8V8%q*&T(#|9i7$$Kj8ASTcFwTXm1SB_Wb}g5hD+5Y7}8k;|Fnity=B@3mNoiRd=<%!Z;d@pbD8M1mZCi1kEhqH)X za&f+lyGyQBChZ18AM6DWUcuD*_%~2Cln>dTkJ%{?>*R98dR2TlpHbpgGZ!dh zDO+;Nc+h55g=qVJeaQX75O|$1U>|JfInVc3v}L^Cf9a}0N6z*Z%X)c;LWEm7%#W@! zWx2!fjRb3Xy@*p-->n*)$`Hj<%QKby!b>6^Qc-vVbmDTgg#E2=`7r^tG5yr^-=ltB zV@k+c^DSu`71y~LJ#q#!ohhK_n-5!j;Vy~e{Qx@KzwAxkvzeWR+O%&! z50TCp~2C{<{v@iR!ii$51mc@ADE7y3D%8=fA8;k?a1h%FR=2@0@N&Cs|eenH#bV zx!t^SYi(E0`>v^Jx3^pua{o4LhFmZ8dDWI!zA~PX{0ev1;sy`j44PNX{_2vPhZ4jt zyMoNejT;T*DC(3GUl11|kW;}>9nFsTZBSp{osa8)+c#PWI5;8Ok6ZGFO(X)knopys z`|R{Q*bjQ0yA3-WV08GyhZOW4Z!$>H!;QiaV2hu5zt=JeLKr`U+;c&&3v|mVuy2K5 z1(x2IF>AliX2Z>Sk6ZA~FLR{S02PR^ecqpP*gNN2q+0@1u^}8Jbx<0&^_*Cq1dfjQ z3#5Lx!}^LK;;kYg;$Jz_j9C>B3EP~dXhX-YD${iaoo6zrGrns0aBWYu^Lem0TTX=KJP#hbLK+KTZ0Q-o02l5_HnkfNW-!I|ZJ@?86|Atchydwg&P{COgR=yOQXo#+fGz%)GnVdy z-6(nKQY!9*9hPa9nUBeGVDYHw-Zk;p%pL$6bcLh#a1QMNQR~$smlz2OXA+AHQy0)6?V2b&xyzNJsKShikNb zp7sWjq4~*xESUW5maars1o>7QjX#IYN2N@(IY1em;=s-!cu2thfMes}ecZ&QB577i zn}^o^{%jyOtj)U9b-Sj-BekLPNEc3+!p=EBpo))5S6}M!8o;_nxMEmZ)>RSuMWOIt z>nc)>L^94xzZL#gmvwKhSKP1fdt+3>&?Xdu(P)WR+E?_C!Dt+6cYkJV34i!Qj(JzquU6F_^{hN(A zyyX830b%X?U#m=i_df(W?}JzjY4+1@g}sVj;F&xgZ|k5TXsgdg;W&qo1$*EE02Crr z6CBH3ibw5=U%;l+J@#f#{AsThvK2@{Fnq$b$I$B^#F1&Eu$_uPgJ{LS+lt-5#dVon zCYtM(-*ANyT!@0;lJ8KI!KF6VLwLI$QN_70Qz6!OJ9wT&jn9i)>{)k;mH?>ige6vwttA<=4$e^0^>8C3T!P zAu>;wPtO1x(*6A@TelOn|_^&{x(QJs2qN!q=7C*E^B~8qc> zx+=1jn6LfyVSe1#=_)|Ifh z&5xT86S{l8{`lyyx0}@@`)lS;U*nU9XH4~`W_+^OR+s$z_#-*FenjHKVfTSYpsrh< za^17>dv!&{BOtg*ii(O9-JD=E`}bKN_|qQ1%{=QqtaCUuA;(1V9Y$u#GVe}47X&OL zY7%nO=(h$+)Gr{;?xBzyTY(Q(e2@)S#{XK_>c1C23)jxDBkTi1Z&OW8mrQh=+z)k> zQp;?&^ioDJ>;u(xa6^tO)Y(-ou=@OyU|T!+3ufUEq__sq3@hyoD?{t8F_a5zrW?c3 z2T{?ogMPDzCNPPdL>(W%m(L!cQD2bxD42bRqGPXFnx>41JoM{P&F@ zK%h~+_;*%7OI(=TKhQYynLlXsHS1F?5wU3_nv;BAX=F;w^*E_hLh@$DQ*p|Ps^hUy z0kxeO=#+H7&ul5?!ym?QAwb<3TQ8srS_5w=_boTWMsQvbm>5=_g(I zWSye-`fPdr?enqo)L-}rfM%Gdq-+Zc+~_ArNxD;) zT`L+^DI!X?P;jy)KYfiI$aJ3MNJ@wPa*l|TtYa2quLiUMS;ah?mvf4t&o$_HuryeW z8Hl5UccO}o_AF8!{m9|pNrY2mDR7nvt&W>#N^`by{Z97k8JS@m+fx40wFHqcQ6&kr zHM)Z31=#j}x0y;h0gGA3ZCm?8a}lEm_*po6P*}7$C@K}#t3vmuLJvQ}^@N;^B`B4ifdme;&GL94#Eug#7n00zvmLuI1qN=bPCFdNA7q|74Xt zsFfH3#{lr<-4FZB5$CKW{8gF%x`WhqpXkr};LonCQxGCNR-qIRauikSjl0}sIw4wq ze3XPK9-Ui#+E1QR7O-37I$@POck!AJnPwhMEIeQcFV}`ZV1yd15kUQ!B_@P9K!PnEnKh@m#khePMJ{Op#uWfu}*{`AdcNW_HVq_t549D=jyo1aBbq)MOvEqM$J+FUg}-?)ur zmn4d5qd(z>OSV1Y(y>yp4sKS|P0SM>FTE-g=teygyVUO4tg^JZr>fS7y~@-Hs5-RX z-Iz*+tBKN=4aU^RzXf6CD^fkbBO#$5F?>7ojAIFK+LK!qGHVHkd1I}%uU?GVweCIu zvD#5vY{h5kRJeUT;jkz9*tN`d(RTC9=})-kG}(NsMv@@(A300^sSXWxvk$9Ei3|@j zC$e6u-z+cv=ZML+MT~w4KBn*%L9(0owkwgW!kj!vt-4E9aXNc{1aec$?&DkE z1`iKDFPr6=mgqO1HaMdeyc?W1rU6fi`zY#V%b+OKn6?g zaJhqTIa`PY!*6e`HPdS*qNuci3~@ieB1euS1nNQC;CFxcr?Y?4`~_7^1_=-u`+R+d zSQ`E)G9P-&Sz|vTlO=KkNYjQ#*_7l4;0S- z4QQKSVxZIM21l1`GG$}&c^_FfmjmU%XC}mH2oozJX`hx4_zOhKzUrbg42dD`sB1sZ zy^PIgcHJL!VB%wj3Dr9Pu8Z!+1@`juD@N#$3z7m#s^gmV(LU(sMuk9;aS)wqmXIBt zsXhk`(6CX5`tZ`|V|mYhK(L4a)IZgpo^x^Zu|HShomg_<#GE8SklrgjP5xJaqAh;+ z_eS*hQw%EQ{95Cz7u6HYH$S73*4xsFwK3CKFTcHJ*1j$&Ks+O-l$Ou`$552tJ1cJY zS-^{APj}7UWfpq&Z&&+6Ub|6xH@bc#gOCR4~a^MYc%{m!bvc7 z=+Xde2ZqT$FM2(qfv$OdzeAdjV0{eJoXy^@plpq;G#2eJ&|kbz0|C{P$NFAS}^jKfA!R*^>4pLofunjScKunmn`7rjBiArEx75@%K$ zn}QOqDCFUR!D{2Lf!Kf-<7{%>vZ>|N&yh!(rd}`~eL?gp1b;ya*!eBuwBT>{1X1Lk z-{j=RFf4d5KrsQy^f$28?7OJxc1pM0Da4@w>=SS;XQ7qm#k8}9lhbJgMEzWyUS%|W zAPntbsyn{&Q&2#MNIcy{3x}i#HG?go8M3uDP$GjZPg^L9_*@k6z``JbRB4z=qaix_ zk+zw@Gmc&WTc>Cx3WHO~3}|mby%q_cBvYRtX#syjP`H%|wQ7_fRADDe(oW*gKBGW% zQ%4dW2slqRKC{x|TxIUi(7T`o5%$lglx-l*qh%Fnj%As;`_w zi^h2HYLQ`Ze){1WHTz=MrX}`kA`jlMNThY#Qw|Uyp6B_*%N(PNK&y6WRHUcWCFy*! zSKV>`^a|7<*Q(omM6CwE)?WPJ_=zlN3j`EdRsI4uot)Xzy%&vZ`Fp|Pp)DlAPwYx} zK6)W&n_X%2=5*k?F96PzR0-eA#YD5U{S%Je0;&zPOEr!EB!=h62cGr@nCZ41d^rtOg;knR~ii>Zi_=Qz=eIMf^`yB-m< zha;2(!d?VeYm0q-R`%e60%&{l+u48RW)4Pc{}>DJ{t?;mJ2v1UO;X3gB-}l2bp$>g zwW5(W#GruSlE2xZlhAs8c+>zV&hO9%V!H(jyBFn_qz5OyQoHgfqs?G-Vp(lmt4e}M zifvCFOzX~7T_$Sttg85Q*eK1pH2@0Im|calIj=2tOM6)3r$e6hJZeEoIU|>8hoQMZ zPb^1x5LSh!KN#QmJ9!B@UdRs0=);GwSA1GH&3?SG=CV2~AJZPA3Lbw(ect=MhECj> z5NhzmQS@GEoCw5CXs?|YoMTtTzTML^y)Wn`E1)MjFI%nFXfL@Q~ zn{lCUf^mo$=DJ-De8;6jjq?&HFE^#>!a%-Y4?E zm^O9{0>b5He;}&7epH9WF0n0ha6(>QUkqlU4%fC2+t2+1Bp8{D8A+6DNV1j);iHb~ ztFBFhCL2<^&zDbOG|U%bI6pp`vS%0r>CrniJ+Uv(`%4NOw z0{;aT|B=y_7;!C81v5HtK)#wc9Q0+-9`1lU#Dc->yRjE9)$(X9I$r^`r7u)en&pJ&jHy-QhHYi_V>ca>-G&j{;6mr?pM@rhFSt{1a-h4v4Y zk6p<#9taswE8Ya)Xj1fi8sm1=;;F!7XV3HCR9ud}K-E%?RG0TFWwh!$5o#5$0a;i3 zTAVDqr6I4BJh)7lIzhA+f`c|7(5NIxpJ7O+Tf6kVwR{~&&kp#Ls3;jwW^djCBh!_Z zNx^8GTQ2Hk($uGYgL@77Cj*tsf;H<2LDc?i8MAptXUItsTpBX#Tsy|@-T}Je(}XCA zQ_KYd8Z=K+ANf`XN$H|Ppcni|=6278y%~&7g}D9)JMk}d3bZK;QhVk|f=xy>Tw%92yaM%!--~tZ+Ua^+AGU{n*)^Ov$)WNc z{Gu!(JT9#~g%da@N@&7b*8e@;HeeVP5UKAY49>7W0#gnoAYNuO;bm)n1TtqYDf z3!oKE-}dxWw#T;bTgAua|3SRPckVSmBk%fo-Wl#0*Lzj15+|g@ncsWCN26{k>m^zZ zr-7KhJ~_~t!Fmoy z&;3F@bhMrEj~Vfgu9J?RKmi9CURfyXRY!!@tb;5&>sWXB>|L7ZcAlX8I~^s(w&1vV z-bW#k4`RPbL8xYZ-~5w8Gi~1o7r01<)vddCKdg%3q)q000>kg2t`{Ypi){PY1g!h# z{!@#};bKcYd2TEFZpAyYV8o4GqQ%yayyf9$qD?M*PiTP}PkV{xC}xFz&fzDE4p zdZTd}XK$zBKg`KtJ=F2xo=@sEMsQy9983qY`;*J={iO`ph zGBqXz5fw5PV=dA*IFxuuy0(2#)0GMN%ZX-9^4N1Es_Kkt!s(2}7YwbJ0B&JbjsNaT zvlWTHX7NzP!dVq86qHb3+)d`19EGpD*6Iz32_CXP>F0?mG4zBBQX0<(D8yLACCc2_ ze8MVV`##}5*{{~rpct0mR_&KXtTX(7tqesfLFT_{%b?x?hlUm09#9@Ad#oi`A9D5P z>@{H>S}=SE2b6KB96Oo-x-*!R0R|VlyCOMCL0-uZJ_!RhcU&WTkdO)h+d%>@A!5TL z5$b0G#Z-PNi^l1wi0a*?6B0iX>I^Ye3%;pEuWw2H-P7yrS_#}a(@PW8?7Vnrb($Q{ zBO*LH*wL)p$h1vv$(gC*8*g{-yZTVM3eOoFL-*&j1Pbk7IJ7$$sze+!P`OWq=DSC*}4!e`bqU71- z3(NRv?}eH|9DW7$?zhHag3V(h)XS%u#f(1eArtu_o#XO@?Y45(WlR5iXUI4tV~ml` znB{Nb}QTb&HhE^T#xzcLoehyLT%-ZmQ4`qR3nMBWS-Jek++~)qx+q zb@dOyM&H-C!gPg5GT*KdSmhIk}SpU^08KUxzER1 z`(6*d8be=E+sc&b-qy8ewv!)mUwiMd;e1UGP=&TQ8zevoPiTAh-%A4G1~p*`#|W4n z#|NfHR%J6fata^Q@sPYnMZ{5F{y44e_Y`&zm~1l~X_e@@`D6V)td@IxW_~iZPSfPclDJgvHv`hhL}y7+u0`@vy`t5hg|E#vDmU(xD?9K z$IX3T>%&921TCY~m>#*T)g#QM@HbO2({1C%=4R%^A<$;iBS`D;AOL7Yg2tICc-U2|L`g;JxkGL&uiZd z$HXJp$}VS~0bNY1^v(+F;9$bBH1$sw%Z^gDHuFvA3Bf2Kkd9FyZ3j|@Yqk6uJjs?o z7sFbxO9bb(@QF-gKG&#Cy5H^m3Nku37wfpE-^tTF%*wncHga9?$+!nIsr-BV=&fmE zBSz-rb#Db#WmR;Dmbs#mgm#DP4Ya?_<4_KYtV^i z-sr>uQ#0x?t?A;(e8!H({)9ICtHyOMx_+hFX-ofhfQ|U&m$!?6nOQV$@^O7yDUXFj z$Muq@FP6FB`nQ#P!Gd)iIH)WoWm3Yb?$$W)a>>>9Q3W=vQ3J=2W;-Vbq+q&2at-S}KSnDIgH z%LoPeeGO~>kypCL&HGGiprEJa6NO2l7`y}`PewW9FW8U1dO5~BVU)^~tR*B+@L_*y zvj!bP4fLWArc@99`Nj!XM!b|i)p9a}x}Lj)O&U%)36oM2T>VVpHQPy955uKRYH$&A zhOQ1q)ZI1e;3dZgN^zC$Sm!*3O(~HfC%;`N$ax&x3ox%DOw{9N^=~w4MXdOi7zzfQ zdkiv9*!>a314_h!E7Hb)C}zre^bwtQNA?*8*tXW|v{qF*U9fa>bqYa39p$rzD_{4H zP&Esj8gA(<##E(}Hf*E&7F-%ZyN(&|>oEPONUiVd-ft6W{-axjO`(-M)XvqQxOEFI zlK5X=a0}1zqkQ?h92G)yAc_(S_IE`UwFNK%z(>`HHC_2R&6?o-ttnVYgGT$)84mu2 zJ8%kG=VXy~MKOITsUTBEUr)TC$7>~(b*;fc6ibV$;a1hl$JV|ku7g37Y3^trY|M!m zczagOb$t9c(h~D8x^i!Ro19+ey36{c?wZyG(#PwNgdsucSH@ZoW?NW&P>ZK^JHbJM zU_+AupMT9H)LwflW`$bbui$pSp6^G}i@5L+HarJPVsAYn(!^`-8+giKGUx=8MLf)! ztwK(VA*aHXm)GLw2Ea*MTaqGjj1JAV0@|iW<&$Gb(v%*jH&M3vzMk zlIUD|LoAUphkr*B)85pM8P*vC zsU9*cCJ;Ue7(JU9vMI23e><}vAeUP!ou+Gr!l9kr{Yi8}gR_-}`P~*2^(GNkwQ<6X z5=RzwXmY3SdwLABb-vm`MTHB+WNf#@x$)9&);A?=tLC366UEdDM&6BvmEPT&a%H_d zd^*>lE*r4t@O?72QB(voH+I-b9sh2A@`;!g3!Cj=Eywfn3QLGqnG{v(kfQdr``l?+ z_y~ZLy`u&4tDEdtGqj$Z*!)>uGid?rxYP+cUaieCbQ9 zo4+4{22jw<|JX@Kj|83a+|q!O@DYiTPph}wNPIr&=MtG9p_3?hNmEbX;)jzXJz-eP z-2L;a!4RDI;Ws5BX^Sa+`yOfgn##uKcK%iIjC?nF=A!S9OtR>KhMAi)jv?Ck^o zEA?4!F3b$nryJL;7ty-F2|=t>y~5?`DP{Gk!BjK#^LaTu<|k|HXsPQnR;}`*^O;d} z=exSeEH%uhKfsG!{}n!;QjfC>@q&U@_7IGwpk@CZ;S#ckkFeGy5!282RaZ|Vf_$rzUT+V|8m9f9Uzi^c$DOe~v#X|OB z-!4vZGG53sp$|1)O<=hVpcq^9GGFSPHu-b)vpp3z`~W{R7=eQ&f`PBgc7RS-fDSDX(M1Tmc_1V#+g~PU4PR8ZG`RfMelk++^ zUmFRCt;(vEouSV4l+(q>w}#w&$o_WsL0_=>)mI{-=q_RKV}rl+N{Oa%ne>p;&OEh6kk9ph-tnWso~U9h)9AcMAJ zS^88nrDJz6#K9j;Z!fu2O4;a0CSXXxbgruWR*nRqh@TMkGYO7L`IRO=jS`CncqS~$|1mhj?Hyo2J>IDOYl(!w-#2O)_wSgT4=Til z3k^o)`HUToh7?bzX5v7=PUusrC4;#s%eyhOEOX1_9Sp;dsl6QC6mtV$xwd`6N0g}` zAs0|f)ay!)pVS+M??6JIQDs}o&2u0)erIe^)uekHCoQ;FN+moJep!@6CEQR<$jT2- zLoMod6-i{HVR?#DMPs6RQu5Ke$v8R6yc=IdJ@HNYuyYXn8) z6I?Wib7D#;waHz+!M{DK8j8{EsL%BH2b7|Z8Ff@>NnU!I^j)KKZGLXKc;Mf|fZ!MS<$}R7vjvu2q%qYjjVlWbgPVKhQnC<2;d!RXFLQ>25;kB1Dna-b zIdz6rj&}A`iBQE+#ZDa}zWStV{K1 z?}Dt+G-mo`B|kMzoSRu!y!lIP4I5PrNJz#~XD=$NhYUkZd6c?19Y}fx%}Q-feAQNT zzR`;+Pq(X*BGjjrAbvfqPn)!mXgkNhfv~mwET9#PV00HV&CkbND%I-=Ej1-u|B|tKth3Tk$$8`A4C!brE8{5gK~36ol0-0N z$9?C5dNhv{_pGp!I|0iz8mwM3(&VV5h03fs;GXQOPZ}o013R-|lnSBcTn>tRvGf#& z(-6a208;1*?P&l4cpgjh!oZPM^V5*L!qP!f45Oi1gynWNdS1>DtEKvdCpLMMC)Q$7 zgx!5IcT%#kU`?CVwBuSbK;%@kuH*%tw2Ie)KyZMSZ014iz8oP0oUUG~l@V1p_N;D; z)hfd|h4luwiCyS^KcIG)R9N26UNlC=S%W_l8tw*z@*@t0%fSyjzxX9@9tVfyI=aFWzY@xhn>;6|^)W_sKTa)PQ@Q;=J?XRaVR1EcTrunhaL^XMZanwrdm&<(P)P+wYG2h_qo(4n$ zoT2*RN-2R4e3Qn!^u`UK-2zFR5xgU#6^P>ckd{R7kezPoo!(Q4N|P@SLVhB?hKWEK z(Q+@Rluox=(Ac;PAKp99ZG9uTUXdzA*1Nb5L{RE((w?s$6Bp9z#)E@wk%`eML5ECM zQXB$@DM?RA=PaFwo^!m0s2oHH0D#4iycYFNsmCu%dGuaM1aDuJ{0Qn%cpownt&AHi zSM6)i&?rvZNr)u73MbylJVUf3j{;W)Agx4Ik*4w9fe z;s0?WOl5!YV9Wk{wB+T-^xqZ0O?*u6pup-n_yv&;P=d^}rm*)zUr$7rXuAp>_o4gS zQ_2SDNOd^@c#>26#3RGK)n#q0G(zm8iB60oLJ9G+`w<>FAJxAYRjc?$%91BLCg!SM zcNhhU(>nFKoW6Ch%8s1bJSV*f80Zojd}_MMVEX%B>Ak`c4IpdCPP{zRjUG@Eh8soe zN}VoM};(r^2P=^{v0v+<_TtS###x7axAk#@bJ*+Jh6okqACh*A@!| zn@df>$_-1Aa`Xepey>!Mt+JkCXzl=|Q!RoZwO5jay!qYh?Ue9+kvm0KO2~91Cp@kb zi+T90CdI^5pEDdgWjF?gx1N~bhKFic9;sOI_8@*Omu6|uXrud@wOx(pSqI6Bim!ft zO_n**)5nqU0o#|+1N7>}>nXJyZhuNCeN&EyTK-&h4Ujk! zk6yjf7NZP8L#(djdLSg$ADW<}BkEW!J~)UfpXOkzg$fpIS?l&v3Nhr*tWUbI(z;%! zQ{(!FMi}Osk-)@y*VSsZ=$%nj<&qmX{KU*Ij=`eY<;{(AzUR4hfsO}b+sY~tgK(E! zqROy9?nPjuAb~^8!Dl7rcK5ZrF4rr{z#a5W+yi)whwaVSL%xZzXn(R^ILn8`>}5Ut zuU>Pa>oj!#X;vtDhGXah^#1hwo`$aj%>Co5W@5L1%BnJO9m}SzkO4OZ8TI^<^1J!Q zpBF+t?pHZ7y0VJ?6j+zb$KMMmaqPBeI~PMYrbB)bW{PX7*x|<*O_+r;sHyl0mv<0-$35jD;lVD|JZRKDwb&c?Xu-luub#z~Qr=dpDp z&2k7^Hcuqej%;y33XRR(F}pOOg0gjQm*)M%w^~H@pA8*9TRH`CnE%XQkN4h(v-O(? zzz^2JuyyZH=~M3>C-Ov+yJ>Dl=@vz3)^`AZb8McaVk)hhCu4;RSx~59vEz6hs&EJ4 zbG+Pa|Fahi1dtX85NzdJ`pErNtUF(gFXbg0EAFUn`wT`>`Lsd<)XVmWD zNwFUWX3je2+^#N~R>|Kwq0AG0Q%_sWiBTi5+~DqgZBQL>uvT7pLr8McPGreJfw^9a zVNF&+YV{LXIGHXo6JpI=+hF5Dagq_^^w5DLsiauC>m~XhUOw^u3gk%{IB|H3)P8O+ zhLd#BdYn38Cfx6HCGv6eYJ2gu8k3Yu!=8pM5Jw4xxW^|4XeSDuQIU(DXA+$@iWXnJ zQgb)1zgk8VEBGFKnku$4^689?V5%9hKeKL~Q?9Sf^debW&(5ntk(-4uC|pjGs@?JK z_*6KU*rAMW8C*D9vHlX?cyBZCui z9Bwxg^DaD+iKVqoUG>2app0`CT#pWA(Q zrh4S2sEVa~00kmu^je79<$K{Ze=^i@Q3oTm_&QU@{W}7NwNKoeLxw=q5_Fzh{~dn= zO$_5dGY%PFy)u_#g`xrhc^iaU6$pQhI7^(|KlcZ*z!%I(wET-r=ayd2*9+8+ZhFxh zd2OQynrHvW1+Ld=YZKT;i$EL3AZJm!5Pw*BjUl(7Qcglx=+E`&v-#Rj&#Zoepgqrf zk!3p*77@v{^4W76!x<K`muj)Qx4CE=naFPuU*C#wE1m{6Zq53gpgfNh!w3wjkTM$Z@;MEfX#PaG zNON0h<=MW)gvbgYP(jY{%@JqQNHgC&5OVv+lOrOkh0^ibEwL4Gv6X+a$W-oM>Jd3Z z+dO95vIBw^=?_)@7&3G5yE%|}glp)F$Ko-h;D4d(H0XDF4j#66E7FoHPIjk3%z@OD z>d@b1tvf35Tt(#S2U5`y0w+dEpLaMF~qX6 zF&AqVTFWzn+L<^uhCQoH&`xl6FN@w~0~HQ$Sawd$2;Ex4)qzu28+j1sl$L+pQz}&B zo=Xot$N?Fb1jB1kDTgV4%x#7wn^?~lFLPlGRlt-6Y4d8MD|Bjrv)Z?#oW~!O3DI}s zdCc(=>y-KUgHZ#4I+k_iZ~3QBoNSVg*ee11 zkQ7n4t@=V$VY@^$r$lzCWLPHY3)w~&aK-R`rD5JxBNT^oq;y;A9OW#~RG168F3?s{ z_!Ntf+zgw4;%1}sFV%2 zft(Y05`$GmKZaSSLp~FOLKzx`kHX~8?A$2C{S*e4X92{=l^bZaM{3cVO<@=;Mnx>< zAj|&cI@;Tg|3P9tWLPX!T#B$)Gtgn~X&(2yj821hP`NfnOh@Oom8_NY;`3&8CH$o7?}fgtj{`1yCb(s;Dnuc@;y-mNtKcODU2w zUlUlG_jDQy#>oy%mBuFbSV}DEc!9Jn$?3&B?xzt}ancn?p56DE>f zG#CO~EX{Y~X3F%3pe&sHR-@kSUSw$D)+leHs=1Uo_WEq<-ug$T6)->aCnR>Jc|juJ zLKrOSP#&^SKYNF!Zwb&8WX>zU1d~neNFXKxRtd^XK(gD2VfHTXdl%!g?Iy&u9*^nl z=H6t#>_}zjCOweNcn>fl$FRNUWB626Pjir9}s1)=Et7Y5*V*Z{W>K_(c3}-C=EIipzN~Mdug|xd#EU zg)GuUWNh>lM=3j4#6#_*EqzF*oAtcYR>A&fR6pF9^ zcqjGqDnxV2=8)Do)ED^qS(ELwu`h6n@y=|;dYM5@BV?Qo1>@rXdyXmodyXwaAHYeZ zg9`QbYqIHp+QOl*SXNrYeONMf(pqhUqpE6PCn8CL4?|4j!)End|C0L6AAg$Os%0c| zMzs(zu+`u>gyxdF&=3O^a@+9@Q1Z2FT+bVr&r~S^0`v-XS`4o&Rp+rQvKklX@mnS< z&N?SnGcUTLGTbv?aLhihZz@G&(fbs`bgTN!(T9A!)+_|uP5J! z@e{w?Ywj$eh-Ff)h3C8yE$CLNdrLCHETxn8njR<}LPK&yLi#qS8;->cgGiy==qndv zD!x=L!x5bjtnr#9J0Dn;Cypj#i9tlkn?1KRnaSXo+$TUg3GrsPVV#qoev=2178bR$ zL)`@7iEkcvz8?ijy>|;*GDsEh5O`uyOFrQ8L#jTRqW3)>&q%p=#5(+P#kuj>UB+cm zU>pT%FG4IHJiBAtsZV6D{Tct%-z4)*Wyqbi{7n!-yfKVER;4HyQ!;O6NwM$x*hMk@ z`dlO@xaVrR4F5S)FdseLtDKk%`3usHJG7joo9+U@mjGpHQcGl*Se{CHxIg#_T&v}? z0(BjdS5?u2gRY772{&^?T@(80H*RnrEUGIPOeiIGXpJ3{R|92f93O(w!Sfxij5AI#We#i$34_1n&H;2PsKa7<1jHXI|CYngxcx zhAiW>qY^5jin6Vo12a(y5fYfj{4RL}>i1lLxxU|+zY-@6(^5kTm#VUv$|TQ{=Baqn z77vSMt4Q!?j#1zFxfQ#s1t_kQ(PKW`$2tSx6Ke1MQoGq+?D~ zZFfj7m?r{86q1~j&R-{0qbbhMlk6QyANiDvCBq2?Zii*B3VCc2Yp&jP{%4ljVr+J4 z_+jTZXy}~#cz^0bYq>G+3zIwHi+6IM8Rc#H>?0mQd;^TU7|n84uOV?qNNqm;fYbOzXwLz7`&DRl8DgNXMW>=nl(mJ}ZVp?J@C9seF}{tm+e7JrsBj^pY=`l&pE$ z$e9VHct-u@iuc0zW6i7{TjxabL{fJOy9&k8lc1u`5HmEt$>mP(`+Gd_0N=@DMd<;p zF`dHjCaxuy$%q0tlF&a0ucVxVJor3%Frppwf$=PR{tcrM=xe!yK!)I3o5#$5DDwiSY8- z*|Eo`PNRS2&y)oJS6# z8E7}z09y3tO{DZoD0vH5Q`e?>`Z5%1)DCL$xEE3^Vv+EFReJkB|6$$swCZg8=H!(2 zAzu-WWQmz5PW^)A81>|uf=_K&M?IshTs2Q5U07vUTA(=wCTIly$ zeX9Ip!pFFv)%nh(pns}W)*>v2$6DnVI9kfbV9EUPDp=;~xp1cB%IQhNHP;;dP&?_+ z`b^?;{F7E~)9}E;Pk6}sNZ*L-b5v|yYIr2jt%eTyj)&kGJQlk#!y=XS;IfzJH_Oe#_)zj0TgA zIM{XGZ;U{233GQcnEj3oxThxs@jQg1s|i1a*>NN33Y@nTrI-KxfTHv~E7ya)Ye2vEO@S`PS{AdGXhZ0t~a^Pjb{ zGUBN9y#Dwlmtg7${lms|j*Ib&v+1m@S&FINukuWIkO%PU5aakZ?u8J~?^O>iFZmlCNJk-V@ztdmG+ z&QJ^8enZX)O!nQ|bG~j2!%gD)*NgO11IaDh3`!#DE*kawp8G5r%V>^SzsJ6Y4mlxo zE~73&LrkLcz8sh8jisUtwew4%M}RZ?)r{`xd{V2uSECD`pM9P&lVUl#=84#apaLGw z=egPhg~D0>c09$sIV%IH`vy!4?#Q#1*Frjhl-yF z|JnwHKm~o>Wvt5YyxaNonqZ;G6bCN4HD8o&6&4W(jyPhrSHN6Tq zAs|tRY2RI3dq8{3hH^y?jD^oMBOF)WMYO-ZES;?aFH+UDZ)!w|i@E!7BlP%-UrLUF z3tWEh3~xc%0b3Bq7lC^|E=W6iqrBL_?-ZnR1TIjNI=*u1M?zdjSJx?|k;q(* zeE~^WURC%(&Fa=CjMS1f92W!sO9k!8{<}rhBQ*q24h40FTM^t@b-m~3r&yA#UYL|s z{j!S0Nq;}`d6DzUGn^$Q9Pb1KW-k9L=}Rt7v6Tj2GM+q(tn6@ds9I_h3X%}5VzQ;_ zom>|KBPdKmMary25dIxzY%UMbotjK$c}n(FWidYlMZ)3Q2)^uQBjqZS%jm}+`a8X$ zpc$sLAR)^Q`_F9J4G$tv$&B}F{~xB_JD%&lfB%20NU~?yWR($-O_7;VRQBE?`)!Nt zEtEY%sI2TwnaK*-o6I*!LRNl{m+Sgm-{0-lALrQ_op`>Uug5r!`;q@+1xnxA_(bgH zUcReD5|oWRexe~^HsDu+h5gIC>n$8N+MLeIG~}FPLWiY@be6*)=z3z(OQkH57VyGF zDgR>W{08D+?eF{IWC>t`fkd|vH&DTFb++@e7oTTYp4?f%_2-Qs zWS926BDcYxn<@zJiof&F2+wh=!#`iN*G12GcWi}|TGJhHTKRVwi{@E~*5ind2c&Vct{tU=3f=8X19*kAMqC>*~pq2_h zz@>ayzZ7}`@c(&@?yT@Wqw;4oLXQO)i@YpTz*zM4)=4l)lcFaE33W$yQfi-JF)FG@ zK4D(9DK*2Qpuqa{w`rlak6}IIaKVe5_ zyKWZ6EVDl28-EXXP}lv2pC`6E4PlsLznD;-=l)zaot8Q&mDUD76_;>zH|LbR!hK}z z#L}&-ry55NDj*@uL;5{2E`y0(ZyD_n=j!1=E~TDvrI9hS#Wy5l0rVm~6zbh&k;!&g z=RihyQ&#g#>H9|G{YR0v{sjQtu1ZmMSB|?7o*=7Y$+rmoJ4jRjnHkNXa+~e@M0A8j zqufG%&O3)wU{!>mOQvONE4Uo&qA5cCLtjfJ*1pT(l}=+>Me87vQrIcN1L$0A|KN#> zj*8QR{cw`?iX3gyu9UA%e0Y-0d%1CKQ$ydGX6$a`Mel;SKFN%~9ekItZi?dq`N-dK z%dcQ~{ztpybjgvlymY9tw=?;{?R34EMU&grFU$aDk*8?9b3x@Clyk~$@TLZI5OH8` z)b+D_<@g2?S^`vG?{f$Tl?dfNlsXfLP#J+!Q!jq2?|A%*FNCd(;G3QU>X-_l zDmehw8v(?~(|fQO94}DuSM$ZZ``_#DdEwn4$C zkCfTmsAXpCP%U8%3*rmJAxqJ?tEJfhJjpUBGICbVcDBA#2s80{J5Dz}m1$wyEKdW|o)M&7|%A`ni!iLmO+LKe_$!D|5{gT>pnx`u740iO~KJ zZW#+GK3d?Vx&Y=Pf%PZ0mIJf4QKOo8&T^yuUjn~_1X&Pwql-?AVF+sYm*&asy60sa zxL=#O&a^k)%g#e&337C0T+q^0VZD$J`YYEM{yxuiY|F%gOYFOKk5i+D`E-Ye+#~J? zU52|uDqqS1JQa1M-3racekn^$W^BU=CX%1C>dQCGNk%x&!vhh$VhWeJ zurC&K9l?=48=*a-k5lm7h{m;|ZU#Xov|=Fhialp_JG6BKDxRrlOVLVRW+w;PH-!|{ z-|?SU9O+ip{}!eEHs?-=+66wc5m&sHyiG<$(h8b(LNh${7aS>Ez+21leO)RP5@erQ z%Od~lh_Op~LsCt#cb{U%v1AQB zC8sxBb2FY$rM#!#Y4w2#o8;}z4nupJV*PCmm=kr%^oqW4KKFA(i#2;Adgq}aMv~Q*Zy;SqbWOL)>y@jdE##ewy0g!UkOO{w!`A1eUHz=-z{beU_ zQ-tUHugL_F(!$Fy&P165(*2uaa+JpD%;pP|@Maf{ZHh?iwx5JiT_lK-oJfo06yemY<8} zy80`#OW;Bf$4$6!r=Q43tgTB?EM|u{q>x2wh?~XQU#<}kOIixwY{4Wmw7a~gb`m3y z9N}5WlQw#gv<)VLa*g&LUgzkigSx4PmejF@>ueeqB`9wKg9tk4i6~KLt4B-~>O96? z*=3F>4n4~xkq3Z}W5oVgDykP=oVS0Tkz8g2>SK+@VBTnH9rTYGW^$(0|WSoS|5#bN$W5%w55KzP|RtnBl9S9xLCeQ1JfS z1Cx@khh-5q4DAsHTkxsL*Ceq`C%1X~+YIZTL+PB&0jGw$!4=Gm8U@*pU&}(xSI=ELFY!yPHQgEBt3eCL<(l070vM1R9Ed2NALDq+I#9-nu~E zx94EgRYCSklovWV!uj9_)|G9j#a~!BS=~|RHrXpGmd6pNF!MAwO_qkJ^mfNo+ZTs@NoUe-v;1(P_IBrDPTPJGfHHVqDm%ju(Vp_8(LnprOkU3>kPC;GN3YA zJiunsN=7h2Fc;|=7yBAG2VS{2Nx}@WhZ8EHx`NPlMy28YW&Mm*P zSa4a+n^|Mam-ZwMgvB!W#P`hnT}tkW=78N^Py=V<0jar!Za7E2+MhD6tk+h7 zDU=eQ(DutCNcPD>uf)CMXbJ)t??ZW4Bqu>e?e$QAE@~os*409b7~UI{j#GnHrc&gW zW5-Ys__0b!LJT8a+Rk4ms4p@F-u9(-q#+K~(=5fE8x7eUAK(256AggX{KyB`?UFrv z5#;6iY^?S15SCG@!p6w|7ZpVf4fpnb7vBAr%d(0kwj% zP%F`!l~03YQ7io<7nnf9dolXNkhv*jS*HhKLd_15f9-h`L7lJ&5^D z2aO@s8ZzyQz=_GnD;xQM_+mk-A(aF(-!c@<9-nnJOc{n%5;A*PL z$SitPdQ&vq6va^!|KD552?2!VrP{S5TV`BBxp(W%_%|QbH%~69InslO3CYBP1?(Jx zV?tm93(xGmXdog1xaSXvqp;>c?0X$y4$Rp64{UHu{h!^*BB$X2UbMuX)&;3g>2NKH z_utDO(k6|9pEH30X#&=dYAd+7YmTMj| z@&Az6KBms}9bwhK6d{GBD+nM{D36|EA&lWTk>ZP^J0yW55l6lIOkzA?9(m_Rk*(FV z2}vi4qyS8spv0-B#tyml(53%AVr#XB_$Vo~MRp42E)ELR=w*rx+c52`<${2NHZd7N zPugV*I!^c)E@tXC+q1yp1W?$q2<&0kr0fKfyefv@@f%CxN`r08qA`IkCup+=3ksQU zTRgNAG7ftmMK?-4-Y(2P9=c4|^PSJGo*wma#Z}G)u-(Qohr#dfCDqjgFmm;TU}FF7 z0B5N;!dOIMsDhYAb>%7gTPpx4L|~?PH&>52c5Ga{c80bJb8U=dDBTihit0Q$lXkaA zytCs}aCV=vMo^Ui1rGRX!^SZpY2D^mhkJT6EJ1Aebc95X1fEM+O-(V7uXnKjv<~}o zzJFz`yh`1jZdwn#KzksGH}(qJjd0UZSrD`n0wp>mQi1ZrWNrbJBXvo|FupM!aP~<|tN1t<2chX48Lj+%qz`2)^6p#PZ)GMQ*4U^{~ z5@0`_7vHh>_UPQ8(#`gjJEtB9!osRL@kmN!|7HvC1wU!=O3cAWx)Mf?C^qW*tjV>G zSj>K5{(h?UFVsh@#m;aO{*3tWrETc5GmrLsMETW<4s#)wQB6~K8^;ZIbw@{mVTG-a zcRH~!vY+u$u=xcl54en1lS4UBTK(A`Fyth|M|WFDv!Dl5A4^abU|KUT5@Tc8)beQI zMKOWA8w~X?WgHcM(U|)*gY-I_yfH25YA4_K3TB>si?Z2SRu%oP2E@|a` zCU_IZb-8*sdMNJ7cmcf@$W|X|1xv2U-fPs6<#Zm;vkni-Gp(F$cd;m<;AC8@nk%rg2wXNS4&Yp1t`JF})#4!-)jym|p@Q zWwAW%0;W~+cd%<+Eps$kA%Pc{>jW4p#F?2NJWi)=&4h_)*6=&?lYzCk0w2zZvOJq! zdInB~SV~FXszoN;d(a62^^6fet55mXolP9)vZ4JF0xu;y@Bege+6chsj3JNUq@XvW zpE>#Iwg~po$xUNN`nyw3-P}mz$l0Q<1_BC@*W-B{5>(rUyHrw59}Hb_Q)%M2%fF`M84qdR%5oyNregtZV~22Wap zn(AY0g$=t(_29)ru52M9bXtewIkW}J9|axUR#jFX{&yP|4PnL5yZ1Ze-3A6bTX>mc zyU%^B5;ZDqeSDmK=fRV7G!v0myMeii5UANerCEH(@Mn7O1-k_c$~kvQxSx43Gz1ZY4~jMX=lRTO`BR4$3HtLR=L6_{oj9ogcN)vW(VTeX;_o{(Hm%a{sYJy2lc z(S0%vDkR&)FMf3&s%n(={NeD%0Zb8q6g91SnGKZS2J*$m3B6uaaTOcSVScKnoHlD3 z6#B_gTILtHM{Gl6JWERcl!~O7TqCR58qdg{m`G#LU_bacY zKL$-7$ZH#7+xJC~Liq{Z>$GnwT;i}n!K-ZAE3|R)F=715(kL*}B zAgno2zQcpj5jw%UaV*DuTgQZm3u5u*JwhzhS^5poIrk1SkPw}-w=P2XeBiVG$Tz+9 ze+qkK*LQrGSeE20GRJ3sI!QHO$q+J)K761IoOh~?1UD8;F(hH?aLLRLaY(zKm05E6+3l{2nm zv=VKQM`G3;qPauoS4%;wLc{n1>y=!Gu3x7z4cLg;to?zCtu>2*;{WGWmCK!U=I7Y% z9_q>}0zE<7+mVkn8n3yG%M|#vEaC8l!pNX_1(F;Dt2V=0I__Odzdd~ecfb^#uvcO<#D+wBDQnvSAo^P#0Ug!le;&CyjNsep**0)b#%VAkAw-m^UIcn@D zgDz9jt_0tc>xUd|>*hhDtbeP zJ%EWGZ|;fKSPEeKwa^YVElQOr3f}x9_RxZDP>)@x(VhjQ&Dw_R)On-ew-z6x+cw4+`{=wMlDhh?!ko!R88e2~DJF$| z0Fb+^|4EJ9Qq`Gxa6K>Rc{NZ_WG?)xWGYl1vlGH7w`DxViTK@TC^p$P=XqsMRC(%& zB^&C^ke-WEzV%nn4{LJ%Ado#{0w<)Vc_MLQPEO7Z3_{yS;F{kt=vUoK{$om~J-&wE)0=k9~9N#vNF@Q$aK4(lJ`v|b(iANt7I%ia_#)0r5c5y z3osKy<3cJJZ{(Xb794kV} z7Z;E{kuO&Zu$x##EBq*9#Dnxhz{F`30aM0BCo1ipmc_a2YY9A}(JR_QzhS)(&#L0o z-h{*TI#F83i7)OOr--NjzhbOv)di(s72dH zpC0p2QTth$oJhBdW8#7r!2ed2W<^VOyJLxPkyb?w}K{D<_>(H6^ zU`g;7`N+L{M3UJie*WS{&b{RDnVmE*Z>W5S-*N9U$K2ZA5yzcBaISF941!I{bWH1W zs&ESX4++xMcoUFc=jOVkgg=QLLRg~BGo|xp@4v;JM){e$?^5hodrbTGcrO6fWCbS z`eq2r-P)b4d;lm~h?Q%=vdr|^EAb3HD}~`R^*!$g&U9*C7(#tcH1NER9LKX<2m8}K z`$g*qUeu8Ih!cPXDj<*)kxfC86_T^|3!!1Ek#qZqM4xZvMG(=<;n}$gYd-DD7p1^h z8Ozc&sF{)n3M$mzb+*~mZ)vA-EY_Kwy}r#QozN^1U*KvHWw1;?&5jYGCZKg}BXM6K z%lwj2F#Htqv=+Coz^|0Nd8F4vaz2hn>`y2WHsPNf24mp9b<#THBi$NOWdqfJc1djU z#b<;`f0l~_BLT&xp+l?-6mabaZXE@!fC_A)NY;=K#WtaqXhR%asF@fZ2{xWu7DYXQ zun~mCQYvWH*!*4Xe*0&b{a4UJssTs18c!nLb~`HR*W}ahA4Oov4^wU2Tbj9cB0cRQ z8ecp>5^Is{ceV2p*pVZ?0sj)^2Zqj?pz<+el(x_YE$|KVWSJ}7!u`K{czim%|Jhh2 zOLbBwSGkWB7iK#OgXFDgocui(<{L3@Gsi3FQ0G#qs0;MIF8J>B4Z%DTk(h|t8r+_i zz9Jj6J*cqFhzmf8mv32|lJ~}3bySiB9dmC?Ft2+Q5;G-aICHKD&v=}mNFwP-+%bN2Z5ZOjwE;;>Bsf(2?<@ZLHtzlzGf!k_+!4ic-?!=~;5i}1BG z8Q^n3Kh&mJ)=zINngDuO#z>?3woR{otFMQz-<$eaXh8v0WFZmJtUBM*=ug~N>`{p9 zJNWvqkKy=<7t70R2`l2IPY&M)rio)A47#B`<8KXyr*ZTHyc2)$58K{-C8L~AO)}PL zc0SCf4T&uy}$$z>~a31D0TA$YH)h#sLe z6xhcy-od6gX!q@u%OU|UfobN>e|EYzzrtuB%&lRO4yC}gyOXgmj_myF#c%YT2VL)3 zY&4?HM*hRzUnDpbxo-D$qISiacf=s&Kxo#RpHOV$qhY7Re{4lCx;#lZkrM!a{+4^E zm;;g}06kzwf#JzhM41BHma)w&?pP`pH@t>BN>(!muc_kqcTbw=<5@1fzQ43W^oxoG z|K61QVia%?pZjqZ*PKK6wP*NcmV;J1DzjIQh@NR=-r#v5?3DZe;T7gj$??bPS+PRHSrNHVw7L=S7>^ zJy2jN6XaKW|0>z*76bM(fQ$9ZrvQqLg!9DCxbS=J1f~MorGcm4=MJy{rhmUV;Qdox zDqfPrE)TBGR&V+@8bu$AwGLARR0e<((#gj)Y3y%7OQ!kf>zj!80BA~}9sMH;d1Tn~ z&d(dpf$&7dHad@(^^(G)Kw~`z^K(WgUn?@<<-sgy**95K+M-u;aGR;Il5`;{bmNDo zK8Qnz`QAs=sui^K%b21qjLg5a$^heP6}U|RN@pn^fX*@-rFJt;sS<*0KS z5Rjn;YRk)nmg~ziUnJyAV;5Q4pB6Ku=#i1IugL{~8$yo<7P0<2uj|~Og|xi`JM{fr zd`0qc3ArFIU;2$lW4VhqLNw3@k_GjgNB<1Z{zO}uamRAH17RW@+)XL_x6+bwgTEpL zJr*4z96f-kP>YN3t-N$?P#Yf$bf-+XEbBT6tbp)qv9V^U&r5}X9O4V$+cMLiUcAcQ)#|W_4vQlt{ z>=japz3gVa+NfH(<%L9(%Lq^tAkmn?#h88S-7~L2U;iTAmit9Gp}Ytb_V4e+=12h6 zTmjtiwNV^I)$F7oegE~IusG>G`w1Vr|xCkb?!l8{3-j|c;c z8-Cw!SbtK|f>3FZ*9f9e{e=LTOCSu_&1-C53{ecuL05}|((~CJMv{onYI8g^wGt@_ zJr?Oc%>^~7F#)cNYUC7YIjN?DSxW79*71!xCWXn1GWiL0QE~j zmj=osa%>s3I+hPQ(cLo5VYluzLauq7-K+$h6_v-!ABG`gCUkWTAV5|34lPttMP9sj zt{HzvHC6a(;IX-#iX!=dH5J|Hr)(Pc?7nZ)11cLem-9D)T`qURh#IVLpf?(ryo{QB z^g68d`+LNSV+W{1S*9_k(0~f(QgbfCJ z8hzhBQMAe=w~Wt_9qvhYTFjRVXFler-HLVhP?Be3#oYq)&BMGQE7SC8xClTeA)`n%V;IAUV@~}2SZJB z>CtAQ`ycGGhu;~~%K*4eJ8|#uerE4Q{X~c?xLL))MQ0E0{%5%{+b$h~I^SGSsIhtt zmv@)_{XB?GOwoSu*Q?eepi5D$!Zn#;rf})30%{b@l2}xj!p~`FYMup@tG|aN*W)F_ ze@l|1fymS>&1(@kq>9yiDXZ*6INHeA8RlizhE2 z9QMxNfIFWM`C+Pk)77eHixeg9*crQpmh-SZXXI%Ob#Xw3JG2VKDT|(M#wjA!V6#g{NTKq?a#t**2M~&9 zkr}fI8OWd_nZePk*WdZ|&1IDG%MG{5q>ET!bp;qVZ+N0;K-e#giBB%k6CtK<(ls?z zQZ^GczA;Bca{`w7mQY+T!08A~o0Wi7(stC+f7dFTlJ+C;)2KP2czCz6PRsyn{z9ZWHAs2vEro6PWBb`;5Dt0kg^{z@myPlBWAlw(_zETx{ zfL(=yMfvesc=QW)hb7@B2ijwJasWXBV9%tn8p)0FcPsETfy}RJ@m8r?da^c`yM)-X ziC&xxGpJS1zn0S+a)u$15f&+HvE_81$LFN*W(=p#@?g~B75J6_)o$Jywi~0?(g`dJ z(mbz2b*FbqQ>IP92(`cR>-eCbJTgCp5CPRymEKN)??!)G7wj9JE-yU0{WR$v|8fQ( zO1^)1Eq!{s9eU**)GoDSzb5BWZW7urbOw5Akv3qU5>pCUppr4odd!$u7W=QJQoWz2hpt2W#ox=(RXBp(0w5N}qF{`(Nz`G4J}!>#A0 zvWpN21FBc>9xgBudElPSFyJ4)z@=UufV7qj;28OZtqI;svLkB5BJz8}`Gnbd`zeTl=xm0`>=nRz@6 zzAZi?X?!nebnshUv@UW|w1zUS89y7l2Rs9TfJ@(OT1~upDW2~a#*Y0ER)x7q=s8CL z5FbYbWmua00H0qC8wCL(RqqoucpW<1EnD)s!W#b7+mseR0A4r7wu z+N~L0VA*T+F&UFt46+8=tD>Ny!wel0QflohH$}HE7Th& zltxF$w(w8=H+wnJ$X;J-!ml)aH>`j^#{n@BX?onLey%2y_SG@Y2(mB00;zYy+~1ud z%Jy1&&&Q;9X`MS=D-Dp|NUWf)2z020ckFY4WU8MSjz2TZ5kO9_)C1xDK-RqDkYpR{#2p!$O8^QlGIKm;l zh8$9dXo40UPyXFGs%GP5UvdgXBzda(2lvf z8D?pZw^n22l5JZ0GIOL-gKaRH1KhP%axAXNq!cXh6%Ax&SAq;7pVCvZg}29~1Owu~ zbS}!2@aV@b-EIUc?WLRV@ubMXAWtG>17;;vd@5`(x96mXjd7l2&&k)nC>Xv42)y~| z73z7`F!0_!Wi!)_j^GW4nJm!W zA>be>Z2uq%TaHa&DCyVnF$%)3NTw%~-#vpVo?mDwWcx63yI=+lFko;8PorJFY}!ZT znnn6+KEocPw)Fj(t*&|?@hB^mhpeHotxw$|Mac!)30k(hsw+aesT6lI=PoGMzm#3jPB7Bx8$2m8?>iDudC7L&laAS8%LV_uQR)@i zQTBZw*E{g6nHuxOj~Sr56+d3LMXm5H?8t>xlRYyy*# zIFXl-h9eRJk3^agT!zpBFgyk+~*th$w21Q@k!P!XZ&+Y{7G< zIMK((zFD_X9LmArj&+UM--IzV)L_m(JC1}^QD+R>)iTeyZ~DpZ(cxDe|M^~dghZk2O0?KdBd`+V%)!EE5ezlc z%rOevdVnO!9};1qo;16hZGW32m5g~O7q83xahEsqXTwW|j<@qL$= zuD)!nt2IeB`gDgBi#w2SU`V%eA@cll2A=$QyApQ2*BV@ZB3VUS^p5R{wb(=%N5cQ= zN4?*EyOBnqJJ6!S*>Q!SfnMmfP5}CWcpr?5q$_o>12MIa-^Yi_zQe~|4p>y_$Z-*s zxbAS|V<2ufuxV>5ZyTPFYb`_!*jtvg$ul zRaC9;&Fyy?CKgm;FBa$XdvuWT0 zvLw>G7SRz~n)O3DqhusIjZHk$!%-FyI*lR=7QwZ>R|9x#$VwGi=k{}~sj2iFu=Zhe z_K-dI>{OkG+xs3(+BO|5(>i@&ItGO7LLH{qboj%Q2nnwC?pNR*uD5MI*}FW#DFLB5 z04Ct6nf{r7@P`*ksXBoD@_`BdqqFmg98)b@mu#elR>@VLM@nmC#%Dq#zb%gJ6>A|R z1Z)t`KYw^q@E3Apd+X2`N-wR5ukd7kmOR>M{qsj4S3N8pl%u5~Z`}#UR83<#?^RE} zF44pg9QybuM5gxrNLFC|LLeNn*)-oIIVx)#@u}S@N=7`qd+k*w!-I|iKWioBj7rad z6}P5|bA{I7Nfs(Yp@9J|brO0!?7>SPD*DANWFPliTyA4#qVA=nuhtE(T+<_I70?z< z&K@97FYl@$Zg$akIlwuV#BY5&@O1HQKh4Z99Eg&LiM^lej?ta{sy!=7lbA;&lJd;9 z`oqxLo5$~sQ`kQHW!jb8JPP~#Q;c%KtZ|^MZLc^|^5@>+s~hMv)r(ngeBY(Iv*|L@ z6Fe%u9e8hSg;kfsn7VcP)3DC@R`PRMt!tln@s*YFTR*Ki%(>ReHdn9~!o%SIJd->| z!j8(%#pC>HFw?cDh^3SbOZrF$9HouIku{8HjKpuoT=g!|#J{K8X85CisZ-HPdo}tk zv68VSg1VnJok;QeL=wGDnj(d7YPS^79dXmHoAeB=i4%qdxU&fS*q}EP)HLHF$JjkV87Kj*2FH3TsU5~a-wX~d?#b)i{6e#HR4UORh|K6eqTBvmz z(YAC1;mynb6?vJ(i}db(NCs5@p&oL8JwSG0xXN^c{J)oMIF{gzfxqxb(17oh)Sx8s zNr6Tw5*oVqlDt12`4^|9`zEDV5Jfx^vQFux5b+hFM0t)Pp9t)B6X^=L)=%HEYn;TW z6$pj2mD*-0vyd~!XJRf^rN0Yc)-IZ969^)v!0TzBo+xdi@9q^%h>Y(|gz@)1yT>1~ zwdRRcS0dac=#M3D!fmM_tdKKoA%2xcv}AOVE&e9SAJ{tv7?j)$>M^MeFUBZGv##zQ zs+*N3_fmw(-%AkFa+!*biQ&I}oAqb;=pwvq$WDC*skG``uFLyuI3PYXWu+wUS0CKP zTdY&N5*=#OYNm^`-q~IGDOM%sM$pPg5}`%X$~GbY{&qYO@-RTFgUy@FYOF&gy&vn#p7Kj(I9)^^d^^NhLoRphMB!;>{YW*gRN&&e^+TUAu4KQC-rnJ!3_3>EiX#!1AWY$D$v*q*`;K5$z4^E2?ChaPlup2*;SbZZC?pv6AxB%R zJbmq0t9nD^0f3^~j6>);3Ji8{=#AjWq*_i|+U|f`UxPIt3WNXD2RCogytX<<25TU& z4pC-nrE31$-cBFAFbk|HLZhw5{bk>OE55?Nzg(+5{@qQ5oLsngnPg2VR8D@IGUYo~ zJ)~dPEE-O=8aaIs&HC0Vq101%lVi%R?+mg$IYp?8`$LVHfy!smfX=&l(5fe%?^NJByDN8DWhhv0j`6}q*JFREe(0LB z-h>KA2M7Gv@wj!sOdm6L%*!E26xT;_=`To}?bdXvuI19Dun92=(-DR*eTa6RO-R7cIM3+P?S4&!)jf)({iS_ecMoP!L(&_)lKG1V%%yax*SP z*Ez8q3{I_dSm_U|KW?_p2f= z^b%XU<`Y+*`i)x}g<4XjT4o2om%Mii4D~mRuN!n+3I0sv`}dx~e%)wPP2>y9X*iKc z-Dy%PtiGRv)T?L5(m+^{df+cogZ8G}MsO2f70Y8CdiXjWw%e)7LA;^PMcHKmmFipB z`JeS$76nCB%#;yG&j`Il0L~{!0;y%IIjbdd0V-`Gw)}`e?(B{uVt+ zy+(;a@1@LB+LCE6q9*5`2Qn>K@X`u4j;!I`tl6ZoZ;16Vs3yt;ZvFk-Z1R!KeWlmN zR29=ybP>UCUs}6eW?yWL9vM$t4ek3O+v8mI(1MGpPsIH4+pfP4zn62%O)#UY9x{{v z>GnwKr3~cN{qRP6A#-w*nv`kD^P^a3I-!gH#lZSHg94KB3)aLZ168_HKiLG8<2CB< z}z>WG)K@}E{%)5(4w zd=`n_rzQx?FfW;$5GQ#viFs74&Fm)HX?EjtBY6U)6H_1rM= zJ<3%aoil7MiIl34?Ra|Jf9l$hm-Td+YI;eIOO_bF!1c0Tr5{gCUXLyF+Ba(0_4asrG zLOgLYIRkV5jBA~P z@d(X@6Uj8fi?JR&y-yp%vS=F6r>Ap+hR1QU|MseXH(I% zJWB^^{Z%yXY>$kXZwpb4OM7TBqU+xtJ^sVlANIW}XWVt2@;-&P0nx6Uk_-Nk&xPX| zl5L+31ETXhlTEDD=T{r6+&iLhh$R2;y3HI2{ZWk{S0>CK3hT#Y>0L!PE#sN{Cwvoy zg09^3YXO1CvdMdY>y$L#nc8c7D}uXn5w6wU?zh;M&BYaqc~{gKBG;;PaP>F@VjsYe>TdnC z#z_Zq*z-V`mY3?ZoNW1f%rvHvQPmVx zJt?7Xn*IO3DDeiy1XPfy`JFbh2On+6%aFG>_K)-}Z+0Y`6n4SqZAOdrH|DwU_Gi3q zBALJD0+pSM9?L}61%)Rz_a5e#_CnQR70GIHD?vz;YtiXaR=?|$3BpB_T4uTw*TSvu zHKEl6e|_MsbYnM|RjarC*wss7oIkBALUq%~vc&G7p?MZ+!r%FKP86F? zn@{)Ud1`-CZJ*}sL43DPjJb8O)u7p1MV+YdV*NaSBf1uEqr1baSQgm zbMgA^0ViKkm2wfP)$Hv(Pm5mIUHNn9BTipU9N#KsQI0(?Se0SOX4ryp?N+t)Q_a6u zsuj2B1%#~LA3s>K9)_m7PQ9q!*vAo^wB@gwb+v(=*tqm1f#9Fb(7*g$OSXMs^oiZ{ z>oeop@uxX7I4B~?zx~*YsQMAlWa-?eBvVK7L z(7pl&88!Mm_lHIEjHBD$T&}K(am|nDUShT_dcCG>uopn&D)}Hk=p#%T-rY_a+@z&Z zAW2ePxrBZGn!kO{aNdCJ&=%%*Q%aHGu;;yEZeo&`dB$@)9`ejh&t~AVaPGKj$;;2* z`g^IHN?h*jO)sudtap8=ZbNJRuF?DrJm{LrURps=G*LN7&K7qLGL|1*UfgycGmBR- z_%eU}|4L5imhcAt!fef`Ut<3Uj0P5=xYgR2=Pq}Isi0dK4aqy#>v(+AxnEtv z_WA2$GSs4+=hr%KBz`qV$37c_<7m_1qo4nn#NSMq{N=enjKRZFGnU3}ds{%##+XrcVR%3@WAYfyP|SK|A*8&ODj0)mGG6*j`H1 zhgO4$nvgKedQDGFP9-D&tAU;Gv6>T~o{gLyez=tpIb>D2 ziow~oa=-fvO;}qNlvB$z{h`P;1TL`dwYD{v!OFHD&^mr{qB&1~UwWfqqs(xpolLS5 zI!;NLj<8E-5f?C|M=W<+l$+g3j;$QsboE$xe-*wJ1K*}jh~z9RxvNWw4~~>|7Hq58 znX9HXl%49Qm*K0R6K%E*=dhTxrn__zrU%b3Q_OkwY@RQ=Z%^4VeVkAm-jfSH7+Y`_ z)w~-(GC(O4Q{BJvT6NnqM>c1eKHM`{B(b=2F>iV;oWm7!r5;nO{%mf$W^ErWz(|Y# z@@4{`iZjc?v}=SIooeaT7NG*I&S)nX=ZKjN7}?6bb7Ka78pn)?|7el&zEe6cXB@r4O`3;_4jVZJb^-> zy`$ecKfS}iYqK2$k6*B<^!$vkAo3byXeHMsoxQ!|c=h~+0I6Z?!vDKr8$!8kWj&G+ zaeFyjpQb5nj~+cj#;2FRU|ppB1ZUOQb@G{z@{k^K!B$xvZ`%8&LMY!N@@IBFQASD0 zd_dRhf-&?DFHKYj2F+V>2{~@8Vk@5m$|7 zHM{7RGal!BE0lYIs{4tuQJ0oKES;h{>AfN?%DcXQCsz`hPE!$@j?6gF(cw zoVD%1j7z%dG1kzl&av3#X#@^qBQ_olwGW;*lEU_3bGv8>~>Fa<0;30Am(-1w7*d>QK)(@ENvq#AVEOjUDAo?yS zA;v;^Rh*IPcJJzBPft(7!+VeZugN06{825mgP)&2627Y-cX?hKlI>11=DpMV{_zc# z_#C^6sPy3i&3Hf|v5<=Y`gHzjRI{?9X!@$gX<_GL)+k1;P*9;GN4;VgeJ?yXV4$v6 z9UErL;*<$IYGNbRaIlkths#r8PaN5H421y9ymg8Q;RW~QBsDZRBrRhoS$whU3Sr1< zO1d(uSx<$Ja;;AOB&M|MM-rT3JT!l@IE)Wn>y=7|5>+P;Oz(H(kl!CBp|Z>wPH6t= z;stLzJ-bBW%*4o@@&4xt)y*fH4ZB75=B};@@RNJ}0Ol6IckR=HiZQ=ErwuZ7a(Z=V zjxE-EDZ|Fn829s=XIf4TcEou8t;t@lb5_bFMf}0`)2Dad4b#9h>c;JX?x~-yaNVhf zbd@~!xV}Gg`oN%ZC%R!?#hxuzft284*N=#M)2-#TEe|6dL=DNE4Q-65ag?~#DyKI~ z+ej~NDpAPG`>+|Q9>+I&Ra=x>2H2d{eH(+F^vl&fJry^7Fn=RgS z${X_xk>nk(#Rv^JHocep@!8PjTBUrCQ1cmY$xTKSf{`ZAy~TOUVY) z*Ox9;jH3Z`L(`jk@3hpu_PT~}7B|FY8anZn^A9O2E1#LgKu+7Z0U#mrqIcVJ1yPr3 zZpSz7O+4piUt-)0K!`JQ5||J|V_V5LyQ zJb^bTYUdJk?L0Ix)y}SO7}IhJuf2^QgDY2LK9CF%^11^lxdsMawlam<<-tW&6L^U> zf8`$#N&F#3b`(_BkN-RAg8F1_jAP;b<)CxECm2JIoz&Tev3qI_xX2pA?AFz*w~y|( zxLz&MuTOa1+#koP1XJ4ZmCxS!j_>uDu13u5_w7E)cg7Zo%K9?C=0acHEz2>N=-BRB zXx}`VKZ@Z`2#xJZDaL6! zcvZfh9~kV_NR=yR_08;~2q{kPN|a%@Emtud&4m~NKmy!+LK%6oS+OExIz+Bt7xf}q z-w#H{8HW1LSr-=(BP$CT73=(~+4=fp^+Y^9xWCvfX_5Kca8y_9*x zwJ-6D(QZ_4|08$XWI8hTCz{zOl63tMOY8K_OJw6^fICtLLucFlC*B6d`MgFaKDO;5 zgEw!9=WqdP%oFoW_~6G)$K7E~qLYVirzfWY({5h7dif4TFIwl{%-9l4pDOG8*?)e5 zvHUnoGYiUWZ>$G}`rU`SnA)FpHX>jj5I!X9G)m#kh@5v34qk}1sigLctPmuK&l^@0 zPvuq!DJ6c@3L7mFLy6c0$eb6me?F9|wYMF!-AQ9VmlZ;f*lZV?Pmh|%!O2tmH!gEv zbMF&*D#sKdoSJ=Zjw|P+L+YIHw(17V=lwcl?Y~euSG3K*jjQZq>6~wqQDIdh^!pk8<@AOs{BQ zXhNB#S42h-5k+5Rq{pQ>fcE)Zyu4PidYK5v`>&4yrjjKTu3A(Q?J6|&gHQHM(uDJVAt~z!b zg$9i&$hZ`W6^4zbqn~FLi`oCj)Ky1S)i!&uP`ac;Q3<6*NH~%bS-U1;X-GBc`5L%gS+Pp9h1VlUMe-c_NSq0 zhCE+=o>Uw7w%kbOip_RMt#T;c@QV_T;UbN^Z=*(5Q)yqWZaxbSHlb9D2B z*tFxWl3k=b0>#QO@6O`^lU@o7WVvvV`rh=1XJIHy+zeH z^gaSaN7^Kk4agj9Z2k7@6`iA_(=7DzLn)D8oBS~xydrVG(%e^P3auy8Ay~lvtND(b zcpRfAWvY{cle~@{_DRk0&iDe~n|nUuyJN{!^^V@NwtPahMVp1Uo4W8MY=Vv=M$HX? zvwU->`_5k{ksJ*dmdk7!?4<@`R}UHsm5i#kC%SV&FzSR!c|6Ji&}Y>f9bW0 z7_AJwutdzQnaygzsSP%7m~TowzOSlkMg-p=YDWa`tOUcD=8Js?*2vDeIIS*EEw)si zS*N-O?HJAPN_iB4tq%nl=tS0}|AMeU)2{0VBB(`@6aTl!fBpXcmdN(tb6917Xdi$* zYhXET$Z@fKRb|DPY*E(rZ8=hWhX!u=6_&6E|H~mC-GZr74ejUz!VP1_u8%IRGfs67 z1|vUWR!~Ug&dwu(bWX{Zl{3tzdw5{G=={epC$0&sZo6$XC>!=Z6WvQ1Uy&9fq%E-M zynNW8O&90>Wz-C1;TSOcS*mSe%=T-dNAL@lM}<>Bx{_nr;a3J?A*~s&&V0Bpz>xX$ z6Mnn}`xV(L+(0u?)$4SZX1A-msBfJ+);^NB$||UCIhI;Pd8nzB=P))ln9K!JO{$f1 z+nb3cr+oE@U()%UI@WiRp*&Bl)(F>qxe*3xJf{gmA&Kp!xpomz#ztsuVD!127P7fg z-qhN7__yb3Aj6q^g#$495!D#sPGNhUZQ*6uj3)6`yZ>w43V70QvNx4jby(;wDHjf_ zRp{*pjnIYP50+)oeYxs!nF>da0AKwRDQ7w}$9el;ULtTBwyo z6Q6i*s0|lsDU^@S0||9L&-<8gv|B8O3pln{n|pJAn@&$&+?*ZFRp#KXU1WKbLk{p( zxI;P!JiMSF&f3-m2lyfh=I_yy-lxZ|>(QNi)8@q(Ayq9o!Q=%z$pgM*%SI8Lrqkm! z(+YJTQnly7G&Rz+ffaAQu5Ielu)27 zmMY(%BgT(kbx{`eo*Rgwq{gLtJRp@4BA-|~5nZE48;^R(PV-^Pgq(11-Bn!q+v6LV zFU*J>WZ1|EG_ZruQdA=UD6;bC+ZJrI2ADo#(=RT5UNksB1eyWaI@Ar?NWw|)Ds-N* zIS%QizR=@4gd6Sn&nYB1_t;ojI#9a7@+ z952l%LtPb7^8g)vP%Z?jd{nwy5PmZpNBr=3OiZRuZ8rl@;YxFQhW|oLfG3A!V_}arJ%Cb9Z36MXwCM6cX%o z@!gj@V)%%~|MH$-Qmk*M-)`C`673PK$+V#d`Yu9bM>wRN1xLfO;xc$7=n(}$sahD6 z_egB;rTu-gitjG`9Syrv-AQ*?ipJ-#C3-{XAU_!s;?l2Ibw1%6@BA5>fNmM7vVPK6 zmsX4JB!T>?KCR~C`sm-3L*e%&g6FT#z4+uP5O}})fSXWn_qlVeI3X(b?FJm+zU6j| zZFjZF;kKH9Y|hj@^kec;-o!UFyKrV|)FF`qC)-AgHpnl%hl*G6`N6*_(38SoRz8x zkp3};kuT{`zpfY={U1rD>l*A`4_2k#5XtVYyu-!@?cY)KXcP(Du&kQk^wug6TH57$ zc30yH)ujOo#}UK46s$wYmC0u&3aZa zG>cTtm&IpjE2an($6YyH}_U8fm;YVr`)5z|9UL`{7$!0+Q; z8-;nz?<8}RFJm6yQ>k87BI{0}qo1qPur@qQ)qX6%nKJ%FV*fhSavw1akJQJohUC-kr-Y?Q~USQ0?}8e@nIO^yIJ{79ErV?YJ}T?F}ENKs)MPGt6-L)5+|u8T^m`L4d~g z3*X^rTi2~rJ{j-%=(3Npsv6Y`<)U(@;Sm9`CoFgdyk*F&6iOG@KS_~|uwtRg+21K0 z0HNf&IIz2oI$b8-=~pt|01}7t(aZeCq5&d!D zgy{US^7;^wOpq!CM{R>dP6%m|!h1TUJo_S<8g+RzQ3pAnBI>%a+#zv9&yo#kq~AAn zbXuQ-dUY&y$Z^G5;pF9rW$2p=bQ(fEP63m7YTavkaR` z##eHSm!YR&y)aR*|ATYL?Rj-Z)@#8@J#(ugRQf*R$@)T@_2Oo!S$qE9((!_Qn8zYA zb~$2iaYBX(uOgGjVnzlQdFlU_Y!;~uR{8c+jl4ZQIJ&fPj9lcgw;VxgpsMG=M53zc zCDN!%la9W2e(Q3<{Yi1QG#3_ltCM8tgPGGyXX6jgGB$#u!9-ZL3LP)~8yMmmP}#nF zAts%pf&r``S?uHssJK-bIWbkD^&^8^m53y>qdNu+qRjK z=u$SulNg$jrxw*>Al^)|?Vv@}$cJl9>4-)M8gj0k5=wf$@Ba@auODHT)sOFPhd{H6 zkye*i2>ciVjdo|Kc-VPFE{AA7L6_vx^{U*6sP1nWL8REmMa1KTbm7*d`2HW!%gre$ zVcJl_b^*Zu6Pj}sek)V_tYH~=8s$;SIT<)AO{@EDg`lPM?wvFP(Nt(y^3?Ll@elF` z=@iw;hAMdl&9|$QayjZ$EUWTYOe`zz4r*9Jl(_-EBC#+Zh&4cU{9#%?UzwRqaAB|Y z>v?@|C5qRk1B-L&n1n|gEpcm1_W|Nr>$!vKtt6) zEWxCNZJ2E}`&YlYtlhFaBy~7_P1pDIBXMoM0n}Giw-#*rLqylx z4ck|+iCyy|8|#s?m2RGEO>5eZ z7H-0Q@C2kZ+BkyK9`;BA3NdUOjJ9`e&_-@>uXSJMZ|IxvLyjtnk`*6kn+S{xsHaAk z4%0mron!)f*M-*R4@Ti~JyU67b6Yl$bh~cru9MI@mp;_Dvsm;dW~7(eW2negh=(zG2l1eYMSOAk|>9lVdo&hYLz?WQ&Q zz28EO+}>*&y>d6F0qTH<3*GCML#hw@&h5;qTmhWBgB$J`BGIwvOx&)bt!5q3lWMZrs@ zlb_z3!G_H7-m6WpN?|VCI)x&&$>C=(VlUs=J+FWRR})kxi>67zjfTA8LVCLoCnw&; z$%mSyepWPpOOAIoDq!_SVxQ;9IfqG_S5JIlxDrj>`L@iK?!k}8VB-_!8PT5kxAKq) z$DvN^WG#SxALDE9ZRNcv!>m`_urrI4?N#rI)*6g=%e* z;tIChJoRQswIfMqKh^3se5YP?D2Ee2U{(WJ9|^=otXGvH3*ir2Ke1Wm@= z+S!4bf5W1yJy_+CXcz1Pk9VTZ=>b{{aFZ11rTUJf{AXQ%npJ!pGQ0Lt@BdY5i)ssj zOsvw~m8~*rswZMt*_N60sKs@?f1e6mAoT`lcL_0I9kNlTwUeuQBnvkTp&D8=Y!5#Y zn7T+q8IE^192Ye3D^Bsj1`K&OR0NY+w#Ut!vr%6rmz1pqzPJVC*V(2-i4;>byB9J%Li53 z59;Q+39R8p=CVI|9r2i z-39u@HV!uLXrcPoI^6Z^()&|Ibt`YWSAHW;>k3Q!blEihh$C7nF(qZ5-q<6x@y1PN zK%_Li_7{$3%o?Ra%aU8ci)ywAhKqxZ4rn&b)hVTg6zrd6Z; zP8Cjc8&Gs>3M_QLWtjKbCDL6~a!u`~PCWv5o7S{E0PUX#1ebpF%L{hZ0Rgq9)%*-J zyx}^a(0B;=4+G(^O}B*J`TcM55}c7W3%{E(6Ya61PR3A#4eh#=kxdj4CvBzmkltq4 zHNQ?mF6Zndb8-TDHfnPRI~T6)L(z8SA#xXcnWEQqd3WiEm5YWhoj-OcRGLpzBo54t zMs(0AGMJPeDm8wI-3Hq_t)E>JOFJ}A+H6dc#JoDjN}gF)E^fKWN=uD4a8B`%67T2C zjdbw(-4>@=_q|k#2*m@6=vCNuQ)Fl1fk>hPp0Z4N|wIry&nT2+mI8Mo}{==io^7b>0-9}DznIb0m;h^FP( zN9HMZqZP9`u(rFjd-&6oPoYjmM(UUcTfR{A& zDb%R2(MH1Du%OO}@{{G1u2Jd4Y?E#s8ho*8Ub~ihi<}@dXzo?Fr3J{+W5VQs&Hocl zb^Inm6Gds;LhomF-odhltR$6I-75iuvAg%hIb)(AF?m)b*%rR2c9- z%u$a&b960IOd8Ar#*&A$)??1hjV?dC=6|BZ4o2u|CyD3g^cmcL@U_jg7>+GD>`ktn zs)m$Vd!W2hTxNsqI3vX?lORWm1(E&kkG*C-BE@SuJ*${YE+ZLR_~Pb-^R(v|b*}v} z*h%sYN#Ie$2~1utYEnA!DFg_>Vw4{d{XnzbWxQ%9f(>N(nR4Me&w zehm!rml3P6N)?l!#7Fwur-edrO8sPTX=S|r@^$iBwZD}->i4y{v)f^?lRh@y$j5%sWT|K0h-CY(*tfeqTEu8p^;5CX*y%2k3*<1>wANEj8 zR=RcfB5zC&INIiSzd8|_k7@4if*9`}3suK$Qu9fZiu@rBnvN0$4^ggFE+)p*0!(Xs z9fNEolFfRC@O&lUGy#FDHS$B{M%r`{g5VC21lZNoe@OUWRSP>+ zdiZ?-IPQRnnXJur2Nx98hRj7ursd~zAX&x6C(UA1z zARp~|05NX9Lqv(CfL0*FL-qleYt2sdB%iOA3#WWjsVFyi^ zEM>Ar&0>N=7RQ12>RbTXat8ecafS8E(w_i7&sHm%#a3UsaKx`Z;v@_sfZrLfBMA~9 z?jJRLZM*kn3r5;p)3&VF@OsR3%`oXt^0LR>Ovn}aX**HwSAS#f;YJR=S9-ih>9-!C z(`{Nz?;N)5aMc~zpBoa&T^p2q;&v-iyfNU3=kKNwvu-Yfu;P`A;2@k2xeWF1B2MMB zsj>3e;}Rq$XTAn#Vq@<^&YclYXQy#P`N|0k3Pk0Hq*6uqVHRx1EydY<0%W8R{G(R_ z3etr=y&(|w;Sc+dY{{VWld0W^SJE-AZ=4kcp0p}NoQj9b0;rnV?w zsf2=W{Ep{kc8teis?$s@{Ye`zo*{z263)ZQJVbGfxS|=H|FXHvH6E_}1MK*%Rq{J| z!P+`x>0ijlUV;uISk3QkMVd6O({o+y4!@GZ>!P-XorORQzSM6FTw@4Bc51T1AIR(IT53M0GbN1BaD3N}a9 zHoy0f^PH1}fTD!dR6}qa@QjH3gBgGRG!e+|-h(E=5c<~7>J^A4#^rp8)nx7Y!?5?= zYX9YrMr>S9a-8W&D^0}2#T!Os_Nzt!fe{ANqX>MQF4!(PBw?}GtDUbQ^D&gG67j14 z0@U(*dF>SVXbqtk5MG6cbd}6dCR3$g2I~U&`-_61cBWp1kQISO0E1qNZQu9dq8nG5 zgIB^tm|Z_zD(E2)@udceJ9u#3T^g8J&f1oT8zrmMU7LuECWXVi^d-n`by?J_IR5ix z5le#n82w9d{2RQ|gI~N=Yl4%VMol9a$qK zPeyJ_)MdqU%{8ATiOnapssixgtGtApaQg9&JXi4Dn>yFZ&R$AUqPB_;_2~&H>EQq*=Fn<@ z)^eIfH=W#xlsI5VTr18~u57@T;5`w)!yrS{Z&`&sf9+^x)jKLN)##AXDf>k}pJ8EP zV?~->J3n`P43IJ*dJ^5gldU6Ql0Mq(Lu{&K;-l>>#om(=4A|00`?eY8ucauZ#oAuD zQCk52koR)m9ul)TJx<6ChQBni+0=JORj1uFkEqjlESd2;h;>cLiTVol2e?wBS}YpX zjFv$BWa8{D{ti1sM6}~(E6OlPPqbA31QX)ti;KXSS&iz}dW#oU8o3#!C3yCg<-Bwr zy{7|5$<__u%WTY-K<7Ma&_%Ot6zWzE@wzf^%Ufr_&`A_LE=|fa*UxZ0^L2_+V1<)i zLi5ky>q>c>=P!rjBJC?AgE1q+KeR?WZ__l8H-kq4`YI&9<~4k)GcwEvqy;L4bYls* zPwUk<=LNjrp^hT-s;Mk8kRZ2_|7_yON7=bYIc4w;#c#QZGXV4T1Tg_aoFt4Dk$J~I zkW5ra$4}q?Vf+oVi3FfdfJOu9Jr_O|6una_RR9wIQ4at0>yq@)j2WOmi5C$C-_Md_ zr=$zn^1J~vu1SUZyqyS8$aR<_&CB#m112wuFh&KBd4DmJ`<3~VCqzAxHK4T2xGwl; zfa?QcXMiawQ6RhO zvIi`9+dEjw!|gY72C0qWxDAhm&@&K7P1OxBL#3Yq4=Y}$|svKoyvtOV6h1G+R*1d=!%pd z?!Pi%5(d!*2~RvX5;D*$B9_jz2?;X9y6#qG{`{pj51bt1Z{C^t&b)ZBifma9NXM9- zzHQbgd7E&k_fyN39OSB8n%88|M2d=L+qslM1=|-^l$^8O8KTIr3y)HRONBvl^=Sxe zY2CseTOceD=x~>WALPWgb--qB+!aji%`m0QccFZY$~H;qy-cgnmR)kRRswSNP2gqy zAkZ#>t8(P+@$U5Lv_u|q;nnHxD!uuPpVgZS5rx3ff7+_BRQ%}o?Fk5j-Ee}5a}00 zU`Hg*%pxr_GX|D(3*qFYdx^sQeh$giL8@K#;c6W&va)L*{&y#`O9`i5n+F;a;0o77 zB2iS~LTw3u;3eXuib*XgrZh4#QY$mK1Tbun9j*2Cb7ycr4c+9957n-^J-DEre`isn zt2-~v)8t+n)k}Q* zJNVNT&F*a@h1%;YUg~xFuYx{Xc0A{U>u9JzjKMsmD}LQ&!cNooVxA=>(afU28=pQz zOl!x#>zJxn)A6&55_A<$`=8_tN0ojD!;tKgP8I`#g$!4)E58zr&P1hBbb+y)smlF< z3kCWkc~u+8sJmvEkJ<4Ux4c{>#%U+aoOuaj3^X-)D7PLCR8KBF_&UP-c!qFDK>Dg= zp%ePg&q6i6ZsuEi*u0%2BJXx3}vP}TlhGV-o6SxU?=ljB{se|)( ztNtB*F>rna2VaWg(dmWg(jnuMC~52&K5_9LP{#9ajV0YNzwc-8-+Mv2bmoKX9Cb7z zC7%FLq<+QHdd^(ukgh^K9O$#u(a~2^5HKIpWq`8>ohO^z9EFnb8~F4bbn+}@IK<*x zPmvU44+&<_@X#VxI8>oz%J6s?keJ=yEd=ZD8$NjLK=XpbPslAusk|cF1V;j3ziLeg@d$9xajF&nwA~R@ zjn_gZnii&?#G>b(??~LroA>~v1bopUF+Btsle~mx;kjZu|7hrbwBy($lR^i1>?cPC z;_wIT4_Kl1c5 zYwa-{zZ~lTeE|6)G!G89TQy0FWl&%rQsa-yMpKIq0n4^ja$)|$d6Zva8o47j|rs$!d55Okt+1RO-?6QLsu^`n;%_;TDB z5kP7bgS{9~1a<*lMf>?W1Db$|7vSgX#scgrjmz9_=eTuf>>0^l!@{b^g?APPMV?fv z$g&Ds{R~&&=V7p7Uj*TK6ik4x(CwC@U5xGq@|%zFEXe7(yw9;SIFuf5k(N56%5c5Enl zaKN_a)(|nh#yo}-&GQn_k#l&p;gDl6*Y$e90s<129LU$*F9n`ynHgOL^mVRFILC|; z(flkjlYfimZp<=)cRoNK7>Hmc6 z4VP#jpxO*^3h0^si+g8f7+WjXH3~yN{?cyZX@AtpYrwPe%d=-v<6`9a4Zk1Q>tHrf z$-6a~yg?VY2y$rn^1w%NaNNk?T%+DO;{bsXh3t!zP|7NYt`YJIN_+*m@WVXR@X7Nh044giDb`s|?@I1ONm?mtMwgRzZt2_QZX9?~A;F(lI zM*QbR0!AHhi7aL3{Y2nlEoY#X$SVaI{#MULARl(bFr(ELX$;JA{R#j5O}z}`AV$lU zO2Id?b3YI<8ZNqsOPCRr(q7XOSwM}R`M_6muWCJPA*m7_HNHSMQqmubsY0}5axw;X za`#%-wtv}}_!U0BP`T+QQRpQ9!?)8n_s#V3RSotGQ)0d0u|gpYfh5Z?Ty*KU+qW!ZKV(w*S!rwF&@%%`m3INaLhsxe8DQfLdqX;52B%m$Uiq_1*AvkBx_wFqr?dxa#(sT7-6h zP%QfCJtqQRQa`=usK0kE%=kEpGODauMtSLgX^(qOP3@?0auuj5&SyM`BT(nF8dbM5uH)}l%6rYAMZoxD9{(Zwp%#3)(L z7K@@F85FN#%)guys&q>}CA8z8a8^$($cTzz*;K3qk4CImujR3b;chEr0db@`xqE&( z8$Im1OHOni#GoM0c(I&0?aWEmx?}@1C-6zv^~pwk5dc{g^V@(`Ll^b9oupy7Sf}q= z0duGE&>1+-aZ0&R)MOZ|A@xTKZCWud(_#i;I?9ESVdYbl(ewE(FD9(js^y^FRm$`I z;9bNc8bfXS10tZ76q$%-b{IV(Dhb>&ytI-L7wI^R)i9vYD6}Q?T+TMpVSl;q>USun z!A?;oT9Aiw^*NtfQ5h;kxrMTFUUw6hu~H+=v>W>KDlO^3fSL2h-er;t16M7YBq05( zM7+|dKxQ)JCzbvD^^Jk$-c&|h}F zp1FYZZMuC_kN%zU0aCNFW5iN#akHoF9#nBiyOuN^-`v2l z3Kx?4*<=DE{!DX-0P9&+Uwu||2ikU?+OvBt1U&OFI?jT^)bm*}k|Pm<>4QKi2qm!W z0O__D(xY^-rT>72x(P`~dAsOTM;R)|%ElI`t(sZ-pV?Kq^mX~O;0~-yigNze4`#J< z@Yj^CT=6nl#ky4g`BX3hpUz-w9e@D*=C-zTILI0j@J}t)5;sE!|MjYr3n-I6JY`@1 zb}bU8n)NO(WgkT(0p>ln|4K?8s6&=*yP4)48iV?lOHVNG&%9&w{k!YxPimOd#*&d? z@|sk-uzP`ys^P-90_mEN@s)RzB?Q3{i5{0|9Pd}&)iMibrN}?$ul(|2M}QPRRmQ-d zZx3GI3mMa{O4h3ZE-p@&`-HQsy+JuAq;L7Xl|>!1ARY1di}0n&HNK!F?~uCa#Z;B7 zp7js}m8zV5TAg!D=S(-q+D5KS=V^yak*x_Gy|9FGJu8;guj7OvCAZ^tj9v7 zIvSLxFlHNY@^5)H6@xp0@A2=0s_r2&T=gHO{`tX!K@TS3U3+N;nN!jum7*S&ftBM6 z@235wKxzWQ8WoMB2IIR+%W(=Kl!Szyxl{s5#c!JDW*OysQ3c@Sgrh=gKLIVD1+Ju75!+2>3k+y|P~atT_YObhj>yLNpT~y~@`3 zue+T_$87um#ibekDChd2?gsdFs&?@cekxk>t9Yg7>-WzT7J*hpdcI)T-+TnL^`$|J zos6Wkw<<2MN)JM|RosJ>n3fW*RmVDUg=)%O9rGX&@bZ^ogc$GvVI~xbUD2;=kf_a= z+jTWG-aK07OjUeXRfby96-|wg?vVcAM0yDiF%`|%ctiM?T#FnM{l;L2adKBd^NALc@dQGKHzc8cPK`)Q|8K8y%uS06I@1>-4%|8TPUbkLrbK8i? z3p-IAS$h-sVNGze!3meb9Rn+*1ukzOOP1v`Z$S-G@j=g21GsG}eWsV+!_kD0K&D9g zK$)1LKWPgWcDU9P;r+@P1M_tM$~K^?UoNmrgAgFPL9ACL;xNvQG0$V=%mf?9Zx|Np@i|Gpe_7#u$ zhic=-W!O$Jl2Pr6jBlJsgD?*QCnKPgjR9SM#H+%IPayXkymBfvVgJ`0Fx;SP3PPNz zfL0-hq9!*rvfQNxRs;wGd@;|5fE6QP$TlcmhZzy483aeYNIaF-0V;h1_GAQsGAF|x z(k|0knbMbc#K48qFAxsB?s1=M$_~j%Ojn+BHWw8v*1)>(K7KA*p>qjfMewzksu8C5 z?fzhXYOy||GdL!DY@OuSj=u_KqFxHi;+peH1ht z&YE-qB*dUBo_DdV923y61i>V2jOFfLOG>q|bIsUsZ;R)st{o@fa+%{R(P451odw2M zxH7;)Qy{c{D7KM=+w?>lze!Wu`|f<3I(^-;Fdgx`TNl4N@ve?Xyo8JV=W?G&Y4?z5L!;HD?yN3K zT7|``yYDb{#!^A~fX+W{mTz7&>~OoVdEUhR*pR@G_c2HBI3|HXAqc~CiGa=erc$q6 z(e8mb`Q}7VgtS7isJ75xvs7r+l znFG8*`8E%m#>&I;B>-ll659nUc%tI{+W2Yk&p;x^?Vq$;7JEqC!^j8?A_DGB_blZ@ z0{7%=G$9_TpN9${4jbXmnjQ3HFo1bTL{6{8ci~)Nae+LpWWs7`%Nf{f_bAT@rko`Q z$gdbNunSAMtYMMB3h)}K$tt>4(NKLDRQx3b0r zy+Rx3fTd#Z^Nbg-fp#DrT+XZ$Mtb1AKv@?QW$&b{0H>ex<7*7~{U((Qj{L4O%|$}v zKMPZMMj87h25K~{9{|cRC%G|jP5AiB%j!w1x44qGM_c+mMp&w?Xk?T&fbetC-YWI< z_-nm2{PU!%E$B_LZ4m^vJji^yJhimh}4ZV%ESpS5I4z>I{@QW8_ z;a3MFQhB~_C-hLi5*1+;61-UI>~JkKNWi&!Uz?rEoi(O&ZBy?|bW~7ugj}nO*s5!1 zu=c8YL-1=F*$Zv2G@^Pge+SR)_ljBv6JMBCwT1uG3>cWPv>!MqO^rLKUo4VNc+x}f zOIi-506N{b4l;0n&1HPj_d!+Gl5xdf+5d5TShv-mM}^$Q06)xB9>imiraH~%j{HE0 zaAfyFo514~rOu~9TNag9l+Ru8TQ({PFQUH|{fmbR5r%`Ln;w&5Pkg$qbGrRb^IzoY zlRmiM5mg54D?}8D0INCv8@I|pY#8m`{YgUqzo403t8tbcUESm=(IbHXW9t3Cb~OQ+ ztd?K*5)-DSjy%rk`&d?v1|X4?^z)Lx{u+;42={HRmAA||9DCZA86|3m6DB`88 zBajw=9eT>yW)*i~R2NO(rxnNZaMr%xw@qNgRLWWwpn0s=}UOICT3(WNai)8&TX#^1W`1%b_R>V)GD*2ld6O(@dnL8($ z5c=||59|J?_qx>}PPTz4>C zXctQgOf5O2UFsecPwfRu9WW$?HCu#^Hsd4G-g)EaiImUIG1a4=^4_z)a^J0SgrudA zf%sr)`w&Sg>#QO*I158IG_DI~;W-o>ba5ZRdVjx7$G!Z`WT-`F#1rF<>v}WM7{a)$ z=`q<~4#M|PvTW6@S6RLS!$1TH{qf-axB9ewK|h#gU)dj$BWpz2X!;B`Sn$W^m|l^< zo$R=#7xDe&49$+Kc*`3J>&N@`Oo=QwdioEG$8S)&EdAuDoUl{qy}Krca0b9OG=XyO zU&sl2ypEShd3m|VPI)>)g(ErVdVrN`AYfI7SSrdz<|xtke`{faK4@Jn#h6I`P1zYP zxU`hvzF(h}>ghu$)w7%LwVz4vcPNQ589O;SJrGEPPNsu}--+7K=3H*r*C8VC*Myi# zvrv6)%T_k$jSeAkBqkse^yyw;uu~di$?99aO1ZADhABU9+REES`N*Zg3i5G_>D%z! z-b)1yZ{HvPmxby5y>YW6t#>r1tzAY;@ z`i-x!kP$?tJg-f=L-d%!YGFUqu~F^oGWMx;<$UdS_2e}@?~E7iE4FhEqM&gWP9?7K z!q2(MR7I~PghQ6o=wKHH;gGXgM85gK1H_n7%{Mp7ibWv!=(>4(G&PkC&3MF#Q2`fb znrarh;|4EpgC5)$pBxoivnQ9xL5L)zQJun~J$Vw*YeaDFtd~P8%^z;+UAD%oZKLGi z7b)<%^iKBD>SzB`7o-dtI` zue!ZDC;CBq*5Ds$zl!!Y1oBZgW_b@Wn78jlLmvOts{BcNi#*BfLSl@87)xZqFkP7p zou3VNO7V7vyV;j9ndlJ+%z`lH*`qHvF7CRC1NNsKDKDXLcn}>$ss{k5fG!JI6^g)I z&5`xvSfVfJ;@t-j5X(z=#*3L1r0m{{jVtcJOAp`2A(Me}H0K~p^Q7Z`uPMRtY|2G4 z4{ZZ`=nTH-XLZIc}XVEL|wCQ_JBrpJJ_r((D{ z!}MnZsKIkgjGgh!i7mepkAWy(1`5s?zySC;__-V%`pa)CZ@cpzxb2}cy@yvvxMB-H z58C!8PK!Sc;DhJZ6$>7S!(ZS}&v1W2Z3EN7BTIvne<5C$j)DZvW#bFibv}_2;RWgi z;Z1(4R1l_(eAi2zu?)VB>XaGQmvtwuVvKP-%MU2&iat4R+0B&VD-+G|nU$(Ke|E(#YOZc2JRl+{cKWiT#_}llsWd8T_OrMS_TwTC z1cJU`ffzpn(QD@{#~H;Rh^oeJn%@0j8Sh$!rtWk7F#DCNSw>0{w0J6Yp`9x%bi_X^ z+^VS$yuf1jy+Ra*)2i-Z)Pd5@LG9l-?$`p*$iw&dO>=A)y|QDV<5)4QYbM6gM7?vc z;^bf8w$1>j&8+Y$TBd25dg;u<%^oM?ne$kdUv@RZr$?LnATw%4D3bWUwf~Ngh2Sl% z4uk~9?t3se4nF(Gtf%ZNfy_j@ZqjS`rRM+2PxNe`W|r#tb}ZO|OjpMiJGkwpB@!;g zZ!s`}*_`PY=(v#b7MP+>L$8SJ#7uD}7jpzp( zCn6E=ISH{mDi;qh2mSzf_4~=#dwG|H+fe_$m5G+r~<7cGs0d#mpqh*M&1| z{110p@5T5H-K}lK+$-LecP&@AjqCEzYn#f2NH@IY$Tz*?0WBN&ZBniJ$n*Sbjn#-p z_U*^kmceENl;2r2#oKk+n|zcqnuS6TTZA=9@F;!TI&%e5OxRx)08qO13N9hiKqE`Dep^vS@RcsjS2oG}26?UELl(3aTF%?Hhx}LT)luuu%aDaA zf4{eTTeoJ9=y^85);^>dv#}oedczmua>N=< z@kA^dnFqm?(~yn!YNW*1)AxgWsy^jajol3}e=`7woajJ%i!!vwBM%E)+o)YP7q#vG zQ!$N}01#|dr4iggPg`t|XInmcJ(?z5`enV?r3=15FV;;mk#PXR1EQ4s{9yA?d@jt( z)`94Opu?p51qH~#w!{gul7LhMjmXP#J_G`MIqmW+%Cq3xg+4+!#j1ygY zG9k-|Aj1IDV6bCCXAZSJy%WhuSFr}tAm{0=*$l^}2&Xztzt0J}q)OV6o#B1UwB(A* zV8XzqE}vu+V|N0owD_pAUVGR!PHxNEvQ-_sf%Tx34&>|NbU8KL5M>UyAbsjCm#5puZtH@yMk&H z$c8+$M*l1G;&2o82S`X71KC^0NsuV%p2?Se5wtQiwy4fe$d^aK0a`0?zt}0^RHV*8 z>=!@~y7#JkVJbru3F}B)H7pUC|K3CONM*WQWU!pbeqd=|kGAKJQ6=>6iAQgv2X#eI zSyw@b5u6aj<9s)@g^_uE;EPsU1F+@tn4b|6=?n_Gdfw@s2E}D7z0_zrCiyxPNVLI- zNfWZL>L8jBP^!DOq41R>FQo<9-GHoT{WSDP!Rr`I_kJMxNxyfc8Bl^#o=EW9yT1bQ z3<9o>TI}4Qo3TeMgIkBk?Mxl_-y|utcYFhLK^Mfqm@A2Izd+cBK#8rZfKR0V115Tb z508n7ftzxF3{Kxmfr|h3<5+%O=*L0psx+rvTs6}AtlnoNvlJA33wGk0c!FpCDt*`F zmIs-`ZxVKG!LZ0k)bjSai`Yg5OfxpTS5hMzw9_FnvKSp7p`!8kuY-FTffH2qLFF8Y z>uE!XTMyLl)EOTZzaW-M&KY~r7`9)VGY%rbBXeI;>Vr*O9`IK9_k#W!qzqAu7S8k#VopHz zW%SR7H@LADN0y$R%b+D?~j_& z!&)oGx$Bf$B-_d`49T{9jTQlSDlG3X_`Ivea&t1<-|nT;;21hR=|S+wZzsG-US6W- zq6P#x_+XC}R>dRZ9w}RhExIidcLnwyl(>cB@}Fg?K3>ipY86{dNox|UXR2}-!LASB@TLFqZU$If{v zHO}hQaor!sgvkHStidb2`vv}GuXO5G-Uabdy)yG(cxi$tQ!++N0u;dA>3xvC3fZEzMI)SP!W4r0wNgR;{x!CM47mO|8hYr3us}E|N zy^^OMApSyjbnE;*u!^yfCS^8Y#TZ~R|i6wg_R(4o195?B&e zu@D=<4i;aJFAWdLo7$tHr_UN+c{JbxY1M0+gf%yA`;!w2djMIe98B|GT?XIUNyoej zoi&+AkpTBog&XiC3hYb}m(JU9ban5Tpc@={O6j>YB0K zn2o*FVhD^IpW;kZ;UL`?QLQ*7ZU%aBVXF9Uxw^@#WT8*<@3grTRv3krZJ_lpEs3SA zr7&sILvSpl|C){kEssmYqvtx-D9m{gva=qs>a*!les75uqq!jw(-jls-xYrof@zn2 zQ$6bwg@(vL#urEtuF}ayW_KnCv|Ld4oWFh9eACCqCJ?N08}34g11LP?bR%UTO0xj` z{jl0VD_z4K$f!V>a?Q!r=sxp%oa7zLwxNzGVfqQbqvpJ<80_iq9(bl|X2^=s#SIKy znZ*#bHei>Ho$;Yqqkb)wdG|A(pX0H^wY-?y`~LiVOIi|kD(t5Vr} zmc3_WkF1nEqEIR!d+$T!kgV)Y_9oH)dH4PPe!u^9<+?uC2j?8;yvFm~_x;>ALSBiV zAJ2;)&xD-74E$*(+GRT62qJH?`tFQqksvv2V936kvY-Ahwi{W&kT#4fdu0ACUt!05 zcg^H)It))VeF4Ssy}zE?_GG>B1$eq&tYax+|Cez{!K4rK7^IFlJv+bF84LN;o^%5H zZPT%r{IHd347uoxG9&MH#I!;C06D}};~HQt14wHkfhpI04AUsiUF$tg7yz#U)go_V ziO&1kooYz)Sh#a(wV)jL5&XQ=Rfb=%UDs)fe}uyN-U5%8_rrATZ0U?K_D7|p*E9IK zMjV#kOg2dX3l)_LZquOe>Y@qr`vThDS$)yaBu@ZVshS+_Ismi(*bu)UKRWUT|AWWL z&D3<4wWR1?3=D2)LWM5y=7mQa*aQN8l^?_>5!K=2pt3bny+OLH#4E+&d$AAYE}JIMrfc)VX<7R(78E@My(;3Iw~+H0{( zgPoigry|-Y+!R|dz@pS>B!7H@`koO#_l-0xgsLCx{%sg%D<-^v9FgiJH|cRy45#6H zm`yZBqLIo9>V@=~O;8mmSGHVMQI|$tc~N5M z*7QFO7GCaB?{JW50l^ePM+}=V$guF^xzvtfQ*MwPspWyv-Uw$FfbxF z3Ssbw(Z)YjjE&6RW#~DaT)=J}vECz9wv%B-Sj9E2@Tn{wg<)-8~h2L1bwz>?urs+bM{hfkJ7o4P@vqXbr1vVZqvA$5fix1{7B zlYzLN=)EQXKR0cgo?E#kR|mZ3J3Q%{USxmZqq|nP(}{vH1*FP=E6%Y>l;bD$?FZ_E zmTZ?%xpq2hBw+TF|Ins8RjhCGX0~dFO6boqd(3aM3B{A@jXbAe^xtlNOJ2Fhbf>+` z;H)xa)ZEdQxmqsZkhow<*KbAtOR}J0`S0frD*NuY z+Rq-RS5JYZDc}R(3l~}K5c4emcGgQy!VE6O5j7hk@IO_%*qg*>9pb!C+9^5C*avv@ zTerCz01Pz;wT)31^;CpEC%r)aY|`j8b~S8GcYLocknFz)cb(~C(MbJ)(U3S@$$S11 zS&G2WOoR-mupYy@3wLifNv5}e_# z57K^2v<~b;d`}o)m3r)8i~^eDFGJR@ye;k$Dl9Y1-k&cV&K9M{2xVc}kZsO7X(z!2du?2t5iKnzm4KMLq55CoRN{Cc(smTOO zRN_IFiso<-H%AXHLZ>K{_&llaQdC$7m=Dz?Ltq}jh+Mp*RnYMO#Ro;4sK#+UlQrJ3 zY@|p&efVe-PmnGQMZv)Nr8O zFpNa3U?56>;Fg&#P`u_K!4)2B!?E>8`2qn^%;(z?h@~4MK8NMJ?-M%YqtIvP0sVu3FcmOfC=z79Q8H?b{ zc>_GJZYw)xo<0JjVE}S=qL(tJ|8Plelv6tSjgNyX=%*7%imM$}yvM^qBpwT)1U<;e zr7HWq|G^Qd(8;lE**`ahpXNu!2;2-2ii=U^p~^G<+?3aT-#~ct=v0X<(dIm8&Z>Pr@fhsy$*R(1IqCgrtqjaeX2cH22*7E>ycj|_jWR~o2mb*b*Vd?|( z*M57BBVCr!KFNhY(1MY-(5}C;--_&iz)LiH19UIPi;v%cuRLegwHJ}fZw(x{t{G;- z$kl%{Qk*u4L#F|G*53`yxRn03yC$MA&FY==sLvrFkNk zqpdoZI%1daB!3Z^vEfRhiCe5roxW1%CT7B6{%v3orhLFw4e_!WXNUNgJ|l6j^pz&+ z;L4E514C&h(VebV#$R7*g56Di3XO%p{L=(~mAibQTL&)M2K^ppyY!tP9_{Uoy2l=d zrvWk`5Nc@TUlvXhcm|xrfTO8_y;J#5F^0UtS|p&rW98sCH0xtp4zO}T?fK7Hi7C;U ztZBr;p?F1A<&b5lmiB((AUiqH{i0z2Ghcpi3oa$=!v?-2a(-gdz_T_J&h2Mi9^|u( z1mC0${1`$s4=-QC^3%rW2OER~@fOuA@th=3xxOza4AeyK#MV$+&H87E2_Uyfb&rRpPp3?zdQxJuHyTu#& z{RNO`VItEKh{qTd9ZkI;3Oir+-!$aE06K+bP;`BThwA`!`&V)m2BCf94?~bNfW6(V z_4I{T!pV%P>8C2YB#xx7Y1|*y9dQ%A4vUg16Vry*j?@yT|B$0r5mg-w)m_2}mH&<~!%&MENS^n=8ntjpKmj}gXYY)9A@=V08 zw)>niL3IOzqB}~@eedHpR=zVR)-N8ieqyT*`Ae=9H|H!j+?5~g8rFOC|0@0lLz4J| zhdxCWc;RLV(JI{w(a=0&$p}D0=#Iy(wCUV0yC9gQReJBuIARxbm*+vMD(71ZU*}CY z0bnfSdrKDV7~$B`_SU<{Yc_5`MAc5pq$?(AdS)Ol^RzIkmaG{EzxVb{0GWZ37H50F z<8&_N0oe5r%Q&ouNxAR`>L4}+2`XChUZUjD;%D_aPfe5(>I_M-p8i!8K77vj$+#Fi zKdq!Z3dbq2wz251uJ4F5q_1y*4YM%bO(Oi1@)+MYob2Bn+$Ue_LKa0GaB2Un?pdd3 zZksbyve%8%GeV^Y;tTqp{deCFt@hrzhOslPNYv1%TMyw9OJ8!%cxjB>$(;h$417?M zQ#-*X^N0c0$l6Z0Xgy{QT6Fg*6b(?tC*4I6dHg(wM3ukC!&QwP2p)m>1j2at?~k7Y zAxmrf$9*l#xu;sdA0pmlGz(i>>Ri~{!GEhZs zc3=@*Fyj))dksN_<~4*iZp!8ZWMc|Zx?-Uy3Uu`1)w4#{4$(nZb^&l46V~{C!Uj@F zNOFk)E64D+x{oblTJ}1ctv|3cl+UY`C5H^HdCt3v3uz!c;q~lHvm!#ojOWny(XLLE~T^E?MB3=jlXE~UgQ)iTEF z-d^U%S~K09X^Zb!4CM^%m42g*qIHE{kJ+v$sZlO8J@5V zt9%){Odbx*aDO>2dnZlOrE3qt>qFETOxzlI@?0ADZ+b`>rMR(>nR zZW-5J(DFtwxZ*?dp+Gf+9Ofk1*2Bo2+JN_Dtp~U^(b%^=x|xA%%#gbWNi{R@1^&N( zp2|)-dM%`4UzAF~CKUsx<%uLCuSpM{v$ux)ZXK5%IiDSjAli-xhoam`YTsA)iIV?V zpcij=cvR{0G5IWzQ$)1~=N-g|rx14B=|*-L;wb||Q^#P^(h43YFGGw=TZE?VKsfY2 z^hQTdM65z$nex1Osp|C_v+5_teSbdCqEgbG#r{Oc6ymu|gGtZvD1g~|b)I~VRb&AG zp&G{f`uXg=YrU4l?Dt@6pQF9zB7S-7FBtyGz7F z5)=dBOcg`6$Td$fa-trgXOT028iMLwaqbsX-fVPMSy%DOL92^6QZ;(2EFZ3`UD* z(+>i01QDPvWx#gxmg4BiDUmPV2@S?%RR_;3BhBBnWOoX=UR+PF8DFzVFYtbd$$XNx zLay1FBU3O%eH37P@zS+um@~Iw6QOC!x)Nz@IZ2@c31FZ<6HebKy!3p-=KEtJR=Y;p z$C7R&ZI*1WfGxid9!GsPN32>JZ(;sPrSDZYh?COHC6ZJWEu=1^R-F^RrQqY$rxLZB z5tTOa4QrZd3PQFUph>RMq~?2Qxy$Z)RwVf&=@=9!OoUo4kK`54ars!J73Q0xApEX2X&3)IqVA3*grR-i?TNCE+aMAYGi@M zVkV>K=rj0!1~~ZmOf=nqtrH%~cqibS$Ow{61^xEPzni4hzp(WGhlwuIhq2|aNyy~H zo0swg_UDtW)ShR?F+X>iAbVPJczazKc1~@vz;TE}M^6gO9O{3fW{Ovdj=pk@8cLe0 zh)$9pHy28Qy1|ZFXVdH4*&emhS1Tkio8p5waXwyp zRy^TZC&{*ZIb3a+;&KdR^6AHg+f|0DrgW#gRULu(kMf_-3+IyAhhss4N{R+<{?mXZ z+ocrgHV7PRmlz3cwu|urb0nyfwEG3Jns0huaCv_Z41|DTa`Br`JS&EoK2Rx{j6|ul zx%&lx)XmvlKcGBhZrr{&61{wBzBF{PoYVBDDJNSrxf_fFRjaj@+bis=+)PHm-F;UN@gjRDd^`vmcqOVYgRF*MqEd%$FecLz z6b!m4EZ9HX7diTJ~CZ3>Z6?#Ai;EM`cA4ExD3XT0c7%MJ)Kz1H=WxDJzG+L45snDE)at zcHd|Q)h*@XH1#-J4ka+U1Uj>L$VH4HG#JMIKD(=B6bMn=CG9#*hxwEdk1N^W;Cg%U zXM!8Z1G$A}`d|hg#(Q_s7s#ehTQo=%M@nmpHb_+H>HIb-Xdun;YKpP%Kc{Dtez&c< ztwy+%p@Pbb;s2e28h*_7U51u8uD%K_b2{TL;R9o^i!SJJTj8&uV(x@9zYS(j!_dp~ zFaxz{!F);!M$JR2N1Ko>)eLR+g|Q(?)z2dqi18faq5!P|0vd@n?vNn9-rLY7yhJJ? zL-zq?h-2_wY1EWCvD`h80@qh$$wQv6$cP~}#CjZw1Lky83XQz#&H-%fnI4n#>mVS6 z#GmDl232v)XDccUPmgDj)OE!@k(@y-S))$>EeKCXto5h}aRJGiUFNNor951!IVqh3 zPykSZHU{Tr%n3;R0k1qH#|}ayzP4SI?N#H>YJYmVp(2s>RMYLfwtd$h=$T+(Vo*r^C-kI5)T_Nyj*F)03kf^37j@b{s^Qu5H@}%>tt~Y*n4B7zcJ(& zja|wdBo7@Qz`4>z_wa{16351Y?;;M7g?rwXC_4b|*LKLZPkDE)sDGLQ2odb)A|{@* zv`^grROFgLp%Mp3Go0I_LFKH&_EKjk^RsZUBHL6gxZ+p9%B=>2t?>Wb>5E zpNHw5b1G|lv**wVupP#}-JWF)%az3vi&eih#62X{~|c>MQJ-)6i1)b!io zX{~boQ0@0TV-EWWfTEE4JUSC=TWu zi!QJoWHDLaqL(3qjU9FoP_(A19MS*`h;7yHRqublbF%5x)C0S=16-?Fly@OC%E=!x zgl3Q#vbyOrez5zw2vHfxu=`%0WIR~jULbQEM+TZP{v1{{Eh01P;2v`j?1rgobLig) zMdMcu2jsWFEbH0xm8ZF-pKnZyH^3`=deUP8{_?*6WlNAjVacufB+-Ud4J0RSp3liA z#(#6W_um#nx*}w2fhjuT?p4o|_?NJwspk*4*=hc-Ab4)E{Z=l-4nionz~JCu`@GiH zSKli>m8C&7hH(fW0`3{17yW0j5!L4$%yRhu-1}AA2Q5|$v|8$=x(q&l0plYAvXBR~ z=t#XCt6E2n2}}uth;j)>WdL`gV}rH~CljmFyLU z0s1s0h8M=L=;(~&oxQ3gCs|XDUTZrMl{;45k(m!WH{8+0CtzvM`$+jK;}aZ^O;Wu zmJ!7fZ1YxK)0K+x+0eJ?nJ+B!(M_7ZFquNV6t-%xJFeu7M=lH&Cr)QqmR z@Eo3yFAn)8fp}xfjO%_v-tX6Y^q{MNRKF=b8VCf$=05g{nq)yw|~+@%ZdF*=5soe()K4yRxdmVdX&M=2$4= z6ODu?Oa%TtLyR62M?B(W&%IY)<5t5gkZ!4Y*q<|q(rZWVHBe^;Qiyq03s_+lr$6Gu`v zal__L&kGXW-8 zhfC5PY#x#S{%ZAc)HID*1N8R)r%KcWajQ$!ILtW1%pEmn_V1H+tl*W3aVPrF~SSIv!(JNSaY;3J;wbfVWZr{B^l#8Rz ze)J=4u;6H#IVY{gX0wLHbY{VL3XNiq*Wj|Tp4v+j_ZIyOrp-ul%g(PyO?09Y&e#YT z3g*9RkxFddaMzyIe)h$!$U(PxOqTX-n?op|s<3e$oI%%*Rh%)jJ+NzA%-t-q z6GrU4w688HDDX6giaFye-5K}9s&Y!&x^BjwrsUKdT4hDlf$MknDbc66-}jDOO+tv7 ze!>Hi9dBMTkJCKjxOptI4@)(Pmoy^s%49~Y_VPTruX5Vf&USFoD2^MAkQ8gYqRygfR$<8U_tM-y@Cyuh)=zV; zpFR5Pi0^bCk3zT&KXXPV;BGJxO^5$Yv7EK*cLTWB zbtjrzw=nk?9*5pDyPv7WagDk}ak0#6oGKr6n9fdRcF@4h)IKgBp2VLXw2Py*oxv#{E( zVsHh^f&o~|GGb{2MrbcONUI3ZnG?$K*&Cdg=Ohwjtv` z@TN4KWHk90j~u82u*n6YGI6gP0-oTx`F&NGO8N2;Pdh1BQg>)OaqhSZe-dGy-Gvvh z?0$Ll^@($zXfjvMhbr~lGHBjiy?U(4On?>{vn84l8-68aLVL(qu{R%XkUg`W!)!YD zC|@UNQ9L;S)1$K6qUc`meD#Vi>Fl;TBbBS35<9v3sX0r~kO3B9_WiVj6HSF7_kB(u zrN~zex;EpU325v7)zTj!W|+DU^_)t6DSNpwTxcA>I8pi~*0bxAIGNRo1Z6-#m6OR! zY)lH%yet8oi=b%ElYdy=zk{)~`%s6Aeo&81C2R$KW0+C<)UM zgpQAOW=6`${u*ogT=`9D#Z?GxURW34%z$DmUlz|gi(AWHQ5LQM<~YANRGWY%F&?%**qCEsAStH-!vuUerxVN01IjeB zuR};%kvLHvB5&`&(XCEH|JljVGB;V+yt+CHq|0dPmp|h zb$i5+N*Q?p$Q|1Fd+75VsEo)lxD<$!R5^lal!3|~HmzH$^MbQ4&hhCMYN(BGUQ(Kie2T z*n7D8P7u9Q{eU!BolJsn=c0x9UpOwV(FD~&hhM$pJ%Kxe<$MwHY(%MJYA_}qSNy_q zMh5k3i`Q!5PyI~P902z`4Fk%B!3!jxE$0ZMm~)tkaOxrGlO|2sGxtX$2aKLZ-Im{R zXJp{qlUz?lM*u|JeD?KG`wzYHMmJiR|fbsY|s@frWc5~X2Bm@>sUQb?*R2~)$l^Ylbq8`J3Vk+9sg8?^XMg& zQ>m#emmbz%ski5s6_DL7d8{yadBaAY5zMR-ZJpJn~{4*GS z+|#4YxoTv*IB#{+{!PP-I`Y8Yo&o-aTa{r`3K7Ihfkjihn|5$Po!MB{dGQAxo)B}4 zeX0tbVIpOY`9yHKcmLpF3DbGSh(T*G+Qx~cMx$}8$ zHuMhg7s30)H{!x@_#9o=B=e#bto_!1q}N~Te!L=nnA1HgawMb?BW)2!*}L(%DX=NP zy^Z)AD?}mUtbMnQICPt=S$3~%$;M{Zu$i7lGQc-gz>*pFvy06fx9myOY4oG|*0cRW z*@m|GGq8yNwkXzzJG0~}egu7*JMziEl-0~~zW%2rSOm9UEEJnGO&K-%*AO*pa3!Hv zZL-k;n{Mo~F?{=L05$2B#IxXU%ABs>*iF;rU4lT!nwA`9$W%J|dhIju*;$!YkAkd? zG2{uuiaKK;Y(4|DX~7uG=lL(@!%65T($X%& zPJ%I3n}%cnh~J)q4+VM?kff>^I!z6Y*!4Xi)UO|(f|`07hE`41l6w|S$H58!xKQL# z;n#3Vzqmk!QXRqQncA~gFDWj*3aCI6MZ1A8d+5{8Sxk_QT#(Tus&=XZ#^-$Z7mS7@{KqJ7{X056ZYrz{cumeu`&vk}dz4j2wng&E>i6p&RvIqRCWmk1lppN129@9#7l_nipb5`Q$SrffIYeNHd1 z=-9`BH^AW;zSUpxtfJ(BU;8t)j?TP;W&ImpikH)m){n_2HZ(7Gdox7gW@|kdWc@(* zwS%E{iVmx{JFDldV(CcR>AQ5&Nx{V1oN0qKhN1IfWVL6BHJHtU8vDN~nq7Dn%E+v6 z)7%LgS~Bk&zr3WIO!8n*Wum##;Z2K|pxzbMRu_5uQ_BjRyANN}a}`-p%Wu^3>nn(g zg*&FM>?`x;5Hf!mhQ~21$!Y4uR=dSZt<0ZstJSTiwm8PKF#ev*g%J-ST0xtqfh_Xd zhQB50k=zl)rV~-AG=o8CwPO5PV!S^Ngj15d=;q*1Yen2NFx)hUy`|#(vuoM~s%Nke zyceQ5x~#t(BRixy@zh?o>8Uhw4I^<&qfSj{aB+o!Re*VEB=oD7kY_>Aq&dh|AvuJ6 z_G&PJa|Tky6f&`SQhL0%{H9~3!9Ni(l!+987n7Qv9-PO6WQWIt&WnPQlCp%C2(Hiq zJ9F^zZ2?$01ns|c10x7Rz=^soDUej49b}_z`Jnx_A()E+EFC{<^Y#TC$sWAdVr7bd zGreU2VIv(tQ946@WxU}CKw5r#SH?S_E^De_8&QSB4&`;E@2#e>Xf>CLs`9I~-^spp^t0G5*U^I!tqid@%`d6Dq0zJlnQrYPXHuB{Y4t$dS)ia2J(65r*=U6oir z-eB)r);2bZJ3pYNFA($iSJC4K>nb@H*B$qme+F6e1e{RtSs;JG zmkyx^WI$qsZCV;6I_1rN(|$Av28=Gk-7SXbZPo#;eg_{0xb%bs^3(ygF(XEbOne|I z6J*Nd$j1gJti;{~V)aJHe@)#@+xvc_fK=Vee#l|DqSh z>!PdIFCSh!HmJ0}{DB41Nl^0#p97^45k~2^ZF7hu@)XCZmod|dqzH}AbU)jfV#?-lk%XJ+qS6pOL zt`LRO%QdZhGf9CziL|e@ms0&p8rAL?aduIKHm~K~t}xstsCkE6$+M;~#?-L5lnY7g$rp07tttyc56s&G8#K&ntVITDmG$fJcx`j3 zmo!vlGQtZkr>4WHMBiL)udXE(5OH;_RwPlAljZi+f+9}gFn-U=(E>-3>1nEcn_+L? ztz29nXB~x{v=%`=B{9;td2`=9hW(8)8zmL3m++ZBRM%1%@^JSH(YBXp4m~NL5pQgm z{cJ@Lcfb7W}TYKLzAf9o=eY#^ZBA* z$G+0LJ3}rEKMWYD_VgCY>~xbE-3=}V3WVombw9ow<;|-r`WG#u(MZ%&rg#*~SQVlm zL-b%!NaV+?)VM8shT0pQu33--wGDqV*!koXc%dv$dZS_bGDNN+v1TsN6wHGVb8HqB zh}7E1i6pk$#qI(*_68rV;&9>IxeJy#*y}t0`wb2qY zY~r?NM95>>UTBvN4@14ZiPY;R0%}jlLeL0@`r`mi$ zDgj8E|A)PyT6Pbl#m&Dv87atZ9wA!NX;SNY|2_X?DY28|=gh*}v`;&!^-7#Vac^Yo+W+(bAH8dC7Lq*KT>fE<7m; z$5MGOvvacl4n_q$tA{wwk-{L!KHfc5-R!^isRgl?Y@}_LXZ3BtLUn;`7&JGMF6mZ= zIXacRzF#%4X$sr3_omO5-d7y9chd(Gt0bmlp4$`Gib$0QXKMj9u@SQlK|LQi9bY9$ zh(uAZw;XPPs6&p8Jc8#Mpr-N>7_EpE3>qlN&|~sg_t8(V?8en-DBmTC7Q%k?;)HN9kD`XLCM6q zuP|clM4-ws$8*UQV-&5J$0PsnhmC)+CmyuV;po8gJ;cx(3A>pCQ|}{E>2ql1beFQUhEZSJvxb=+Qis4aybz zM%$xs(?il3_4#VfIRUcrZaPlj!|Q&p^KZ3VTQBK-+e@LONWC!1hQKR2s+#IUO)U0D;5hq?6i=k(!Ua1o~+%?th@dGo>V;KM8+=uL5om3 z3)0R2An|Ydab^pF^ZsoUpyV(x5GG=$VJqO`m_0QNRSNJo{c+6%_TZT*B=|g~n*Kdp zg(TV_5p~jrz?SE6)fDW7TX*@sbqwAdXH?gTmj5Bp9LhsOOFp2ar>8CE z%5dQmDP{I!KFn21iuaaS4VaPzA0PL7V86P@6kVYB;sM-ULJ2RMcNU3s&NF)oxS-ap z=8w@jWcPm5tNvQxtMNxyCXL1zRSt$4B0(D~{tV3N4GZhCU`?l>Q7M$O6ANprGRtk( zm7gCL*>ooH=Yk@ZZ~Fz;2}li(E7(du*1+8HCWiVo&HZY*fZ=oexA0j0H?iA<&Gc^; zEn=w;Dts0&E4!yqC4hVS!vO1c2ZL~1PyxLlF=Y?e2eR4#-W+TGAJ5)Ny(vM-v}FTU z4VT^kqiQN`w15+VwB_KHxA)bArK&#Ljgi%2t83{{OBD+!Etg#pi=$lMnxB?+ZdS?< z)fJAWo+j!@JQa+%;Z{&cLGbnkfn`JL#u!(8)NO+30{J0gcpCsghYPq$pw(pp7uP~| zs(zRXOUU5mCE5bqj+pLj*e-7pdV=P~RLclQaINc8*siZyt}KT1%bf7Kw<+;wg!6Pq z-eOlRe90y&EnKEf|llfwhwSUBx0|uIAR-1o`Cal^SVi?$S(DNqeizcS&OVudW zC7~NP_Gmg*l-QmD7=!Vz-bLVKu%TTirF>2>m;apkP5_{3W*a{%#8%%XbxKpn|53Ko z3oj~q_i*`X<9NBzTl{gtFrZh_&V%lqFNpzxFX^so)a8Rb(H+E*?$do5*# zLji&DW?(Qqt|ejKBUS%S^X_|s-kU#pmcEqO)5EEJvX5gwehW=0Lo#SCT{bxM`o^fs zr1k~3ZSTvqk8mRK=t%^CErH&~oZ&WuQQWDX7=RvzY8rm!l3g z_VFT7d9Yv-sQAbZ|1ix$AAC3~`O!_H)rE``H*Ws5K0ls9-iFvrT>!6voZvyJJ-tE| zw9;7IMMbN{Y0fWl{*T$%CQ>_;O>sdFgylnhOUuOU}*JS}gNd z4nhl?%{KER^^;U2UMZPp&oi1P_LIjEWrx7qA(W#i#YCj!^jD9v7Mn}TZ%t}^CQ+?0 z0sYNG?8~SRW4<}RE_Yx!>`4DY=HHtKcocBEd{S{@kzc`&?3l|ClbUHL7QWXp19Yb= z!x2qgFvxR^VXQVs2?AwqGY^?Yg4Fm89_Ox`w6} z^tPO@rpYyMvTP}bo}!kUW8G=7)eSW`=lVTD@(mZR%S*Y3&TI40kY^pK!@`3D6p!tW zRitm?k4c%eRO7C0zI{9>E>>b(`_l}+F0)*Dye(D2voEak0JUFF!w4@713zv<)@DpS zTD*6!hUJ$(%U4158{P#^_%NfCcsVRDC_I%2Tkj9Bm@Xx@R93#)Z8(A9=OcIThYz{| z$4In)!&6A4Gbb$j;%8V0|5z>)FhX?x^}aBUPb8cL-`vJ@ywc$xvH>}LNdAkwx|BRG z9<<|X!d(BU#v_&cUbxUP2kZ~bGtJ|212G9gIX$YxBuTIrKYmfS1WO3CV-&F|EJ&vh zftFFH%-$H`mpuv7lQ(LtmqYoLhXBfP!}iM&QKx+Ik?n@SL#Pp;Dho5p&7QC1D@iqL zDjcGzhO3p*>dgThRL$@Pf2gz z^^;K6nuX$6jK$dV)9kTE6c=3vJZk8=n-ALK3K0*@o~@TJ&*B#V;Td%2(5x?UNuC zLq$k7owS9aYtmG+mvZ^+Ni@nFZqqKC8-UQYo0pOo7H`#7I?&II-jLvZ%{Avj<$9S~ zPx?pqX-C^zKc%#ffdWHwah z@GiabwW(AB6@Y2VLnY?cSWx0qF0v`<5oAY9$ti$vcVL}8Sm+`~i&we8S7!#Ad<9az zTpEf);Qd#m<1yS&r;}YW$sG)frM6Uyxuaf9P2x)p6Ek%=XDK|%B;ST;IBnB02P+5G zJcWcx-MlUfv?Ijq%iEmL7IMxrg~3w#2(Hv2j!0m`L*^U;MHl&_k_Cn$U)Q5Ni_tp}AQCD+QoW#&va zhDO~c_b-gFWFxO;2Lcw>5g8L`87V8A^osD~=s2*pvJ6da(mvb@(VXNae(myBy#j$OxBa!MY1y|B2SzS)yOounfaTisoR zkAA?GD%=ie-kHlQB!7(8q$EvvsoRU;OcKMWu|HP65G3xPtS1kka^9~NanMtATekWY zx1xn9`_fG5UYttMQeZkpPNVKtjB=LKBGG^;LvJlQI&-I#{AC}Y00@5x`uhy2v3a#H&Z z+ct5&t5}Vu>(%_XPk13YHh6V)HSx*x7-VuB10NF$L2W2+fHYCkq|m_V9s@}c45b=I z;AvH0j24U=?z54Uu0XUP$ypt7&2xhPK#1l$r7Ox7cd2$j$jFwZ`FHUmWy}kJIc+(t zz1}CObKT{oZrS@&t-|Ulh@4#6zQ3;g0334siAMIp5UIHK8XqPGC6-0@jW5FsxmR<` zH+%fF{W3K(D;_}Sf1#(r)&XER{tUBI-@JTgxAodtexM9fgBas;b@3(L19H792ReH# zaT!tte_34gIR2f5wD?Iw6u-0a9VCmn)k!f}Qf*j&@v(Z#zaF-IbFWwN-Vbg66Z;TU zuVn|NnciQ=m@TA7IDL8+5L}Tp@`}Ve*fBK3Vf`0eeC_nK%< za=&uue8O!`viJ_61S;rGSlv(GCC9YH(6k*D4X%B$sny4uwrUM5lx^O%JG>rbRjTQP za=|JLC<)hKDy4Ok;-+-j3JbVt~dc&CMM^YxC#XfPnCjx@RN{5Ue zDvq^vd>cB933uM0F1#St5FP8G>Bh0Hk`057_v}Fh)%?Q#pcV>Lt*#VmKsk52w&2pt zfS$Y+%!A^=e}xKadnSa=%1oWMtG65ELLexM%*g+WO(K`S<~^D_2OX=0wRO1CJZCR} z0ST%c=erkDXc0eI7`UiLv3LqL+Wsk&JtI$E{%ud>7I!zt_YMZi<=sm{RqwfXus=OY zO+Pw^&fhZ3oovJp>R5OfjT`wQ*q-ZBq+ht58Os%@(Fdq~yWXblRS(kRNVo0Ku3>sg z4|tpiQ#zbQT5)kaHhxqV^Ui0^yEi87p$Oxc zFr~&B)BMaAV;t{0E4hYUiVxF>JXo%HM`keMDUt5p3ImKZ4%I8)Mmp%Nv^X$08?c4kHT;YF*os=Hk~{v+NsscmvE z8>Z%o?{zDcpy9d(E-ZP`^h7u#qo$1$E-H!#A*bKx08@U}zY$?rGhoZHAdwyDM6n}Q zwn}9mqkQ-D?j%u%F1Fm{T4eD{!h{D7{7-%Jh?~xuz0Mxq##MUbCd&kk>Gy6NiXCy3 z+ndRj-BmT}CK)P+4YcTL0}dd#zSwq_(l!}VK?9N7K(<}FL1{}MP34+4V?~}jej!Ls zj>hUzS*tnv!K7S%DheRl(#Jzrscye{QmK195_tA#0szTl@K~94iB|xQGjA8sWT`N? z;6RCu7lH|~aT6}_N@VD?lbs(6|65gcsMt?n|EXv<7ZQsL0UG1+#2YdJ4B68$G=hxs zevRvFN(PHG^Sz~S^GqS5EeJ&u%wh`D18qR$mbeaOJCl3f2UVUSMu!6FDivk3Kld`f z;@trZHwwu(Nw(W09GAoGS}PfKqJ3x!FrC-M&TR}7XzF^^@iYg`v&G{zdKCCGNIURP zrBVKcXq-)T31@@i)HJ1}Ew&6b_9V?*IP^@K57xbyPO;|glbFeq^~%eCi3f9soJH60 zS}J4H%mzPw6j)rw?RevE1zE4~pUeCd^7#owN@76-UK3cYMw45Cn}>J#7lQFO98Fp+ zD&>Km=sY!5FWhfKu8HDZ(p8_Q@7HTLRWwbl{~_gg zfZaSX9$OitbDeyj`{IqYn!UXmV#8`!)x|OWcgRK1?{jW*-O4`o=f_qa!W#Mf7c0aH zZFIW59N^V3UqZvTRF}NQ_v~BJx6p}$#u&qwmz)`bY2EHV%w4s1X3`sA`AV3q^`Nw2 zfOPm2Fa^1Wspja#YbCl!ZeR&OYX}6`fy|n?#494p$Qj&tX0Mr?5Hf*e0O7H!L8*p7 z9#R6-6tP-jv!VY#re`%E8JpD2!Q0bXppnrjO}K;Q3~oUzIcS0ZSxE|Hp-7E|vEKCe z9N{mK%yjlw>UePvFywO{%Vdx$DFDlxve3Vb@frO?@D`2*yyk0k_TvON50!_^(%s^I z=qI6c@~Ol$xW6i|gk#>l2S>TWT!ug=2&qP5D)5#jlKrdVl`=O5w;8hb81sN#0{R6z zks!MQcdq=*)(+)SYTxBS7k|SHoihkcwqUNrD@&JQJFZGn)Y;7we)Ce=vQ_64#|F;l zN)NEN8Ah%UbKr_=;T*+(u`dcv?k8$jH<5cJ!WgB*@Zuu~QS-XJO3kg=$cA$GZ_=60 zHQMPxbuGw81l10<8iDw8KWu_T{0Mc4C{3;Q-BkjfB2u*hK%cN?-Dn#JBdp+h8w-e0 zev|9>OESX9i=U~c+e`b(qGGE=5KDI5j?k%%b-d7FnBSz6?ZtA{I2edovjfD6aE`*|+B)L?AIK__q* zp_A+Mn;FXPCLcnkn8uU3ea!rI*a4$7e4 zAM+k3SUR*{Dq6+!&BFlzn|C2WwS8bU%9(+itWjU#VvRjs+?v8m-SXo3>KVy^_Phb@ zEyr=UYKb`dc71F&$DS#<(6!$U4FRK-H`rc8Q@9CSOOJ!I`CSE$b7EIyiQ$v7Q~(VK z$m2pjmhex2EaTZo{(76&@5qiko~1O&5f42tI1F(5dgwR*157zHM4#B(nxI^Sq_sk~ zNOYf<+3Blt%{>cRL79x`E3hmbDqk*M!#j$t`Jz`|6{rQN#YKy3d0K$#kO_*r~ePfAzhH_2a#}D4QHl(R*dvUke;l6)YB?+RB+!QunY=Qw> zoT37YyzAT&BL-N&_x{GdlB%+Dn0T8D)R@@I4O)ewa-s97+iRZ;-ltyd0Qa!*vZ65K zr+`Yot@j(-udNBtSD*o2m;og}NaB9Fo{yQ^b1pfnGUDZnZ9i6zd*Dj~0SfebF&*uY zQ5GWdWLUdONS;i903jiPu0sY^0mEeGuU*OecjwVZ7yb{7(1FgQiG2j4>!h9(uAcU;*i7V43elhXaxx! zK8xU5et)0PtY^T|Iwq{Av+Mup9fFpFL^MJ{CI*%QfIZI5q08P3s@KIZfFn@w>U(A2 zv;#y@DqXY$=tY}L(c$e1p9ggZHCDo-U>SkXwJ07e!6$=wu9fg+)I11dGDMMx+nrh5 zdP5G9?WVSIAFcZiabEssociolqjum3+!+wo7_e?TFQA} zM%OgA5DT{G4ed8y`X%{#O;Nhzsn(~wRc#$U=l@65cYssf{{J6)%O2SyNwVXZ*{M)U zMcE@NM9AJd5-OvNsE3kdlHB@YzyEce>v}w{ob|as_x*mq z-mk&gYAgLl=rYYxyVHZ9Z-M3*K*UBu!YNw?vJ7nb+g(*wRT&)y>(bvljRYiqh6z@< z1XGNOXfE#LLl#jnzqGpL=1e|I^R?d@yT@N2-*udInsX0KFYj5j9kPq4xSKbtJ=1=J zs#lM6oelqTo!*{NnwYKFq4nA%yKSzu`*A#vATTE+61^x7vH2{ zADt$Z6zt3ckYJ*r%o#WZUmmMaSBN{c(p%g;cRXWV3x7?|qwqbco$Phm$jt4K=XGT~ z5Q8${ZZ1LIO^q**mxiKhLV3JYE3`IfB;O@G)86R3l`{Ru2<{)5&&EFd1|U`Kfv#9u zuuCe;HwscSgLCGKoRrJ=a|(cW@C9Fr1?mLj)jFTruPEHT7#QzJAW6U&CdsrI!t3n) z_qJ#II%Jx1axLCW*-QFyPL6gLmxya6Jx^HsFC#oUBTGl}AW7X#i?xH*^9HvXz&S<< zp$TnB+leMvsi~>zBC_8uRD2l%moYHXZ*#6DMJ_dNaBx1gc=K$?iP%c>f(&Q;O&1zf z?PtA-F5|g9I)Rx-A6A@HhNUEGCktFFjg*^5x_x~-8^eWPy-22Uq#8-+U{Tk@u0{0T zco}@hK_ba?jy~bENSzzEuue1dTkfhyg$jYJUFVe=t!i{cGwI?SsGxpQna@xfG);AC zP_sR%wz<|2TbH?0E&NKKC8GfTlyXEZDF&MqQxR}HquJ@I;n$G-7fLJ_0h6Ys@~yPa zk^Bg)nN+7@Vw%py!@cbG535~lJ4$*NV_cfS{eY!cj2Q2zML?3sN{_!@c z{-v`5{BbC(hTmLy5QF<}qb5APvw55UV-XJpH-&MB4%CCjXMAXd?VDbA4QSZL+np_U z@iF?Elta$ATsHL-SR!}Q6Qkz=;y-l>0je?DPmzA3{);xJr(}uJJk@ld%3sP~)L4JdE$fPz+#O-tkv zZRG_R!r+-*s7%zF_l)umc}z4k|D10hH%a~!-fMs1;+H7Fv^3mjU6Dti3G%&o51K`e zdAKEJha1ZlH7~tQ#53BHB`?;MI2oS?v>XUHl5CYs*t5HHi;7-vgEb{+V6tL8vJr0A z>Zw&zo@-B5+}22Me&R}nahuPYwDnsYd%X^JE;2%2i8Z>Dt`@Qpn4psb2*MZPqr}A= zc-k4|01OACwHE#}6UNm<&0MFZS?JMee%=F{dONBO1zIUo1EmJMnDuW4;b}XZ-0S2eOsXKne(e@I6R$G&z5e^z~j;t|2-f zqp0DX3M2%7BGS#L&0YZAhT78G`gHjWde#`uh8j5!wAZcQGL?tH0)snX3ms)z zt4kQac7O;b|49;ymo`mBx7#*h$Spo7Ov9Uyh(kWgCaacrfGRLL2oB^4jA{kH& z-jDhoy*=d7CViblMEjJ*xnHpWpXPjV3ZrTPL~7fKJfk)mrW_It*>f@Oa9R5LWteSV z*=q|9pNn&)YK*D7C-1)fn|@GFxWVRW4gHl`Tg;txYPB~hR<1t6r^^9F8?w*@;$8fDy}SSt>m4Rc~wq9c=+74Dw1F>H`Sf^Z|y!^ zUwmVA=r+mSjhd1QG!^u<%PDgb^~Mjb4|XlHx%@O|`N_$V8v~nbt*Dm5Co-Lp2|u!6 zPh|xkAYOef3;USxq#R7)Lo&aNmBWJzR4-A*C`rd^OeA5+Ng#V|Y5;&tN*EPGg z)f1%_{=iU)Y7m54FqGdrjYf&wz%xgTvsGkvi3JH&tLo>kCeRJsc=rvUE(^>k@IWSi zP~8crSQ~shFf9$I zEeF%H<0EzINie3hx^iut14Z$PiQI)k;Aty7>0zha`ND9n) zHQygDVQ2V!85{Ky=}?)(C06NM!xRt-q(31b&M+`Gx~eqGu4srf>*01IR7ldjOSf4gZ4dK z**%w_rP94v#$pqe5~yPr#;XPzA+a#%75Ak6 zE-en^bGGr-_5jk{Q$XbI1 z$(+Ep_%U>;e^|kq??gRl6NL~z8Ki>B=Oj{4lyGU(i>ZDTL}-W=Mm^1X^_ zC3?ife20kb%@Wp7w5uA&K2c8pvw{}p0SjYHUt`(gry`qN_RS)=W8Z8nFY2B3y6+j} zwZoKb5TSE(tYQQ-!+sJsBr$ z5{cLhuv4LJ<#wm7xJMOgBBKKP+!m~on6rUDA8dh>@X6vYO-Uh>FaJC6xn5+aJr%Gq zL;bBw|vuT6aR(ivdDSR9s=QtiI@jY|(|<-7T|zM;)ncW(CEj=yI&t2~7m>67iB z4W=X*9-3&N`*iUpb(?)(n=ieoX|L5|5s{;Zv{iab6%v4t12P9u;_?w|`jPFW%k{CQ zOR?wqLrF59_`v>jj{sL__If;|QH(D6*i;w7QQ-by`RTOho`iN)U)W-~DEnwGap?b*y8nP=Q%%QUtZ->6EKyd>@?}GbYZ9dePoaaTUi+aLeM;)m^ z4=rR~SGz@v0Yny%ypFv5bH`PMAOAMQTe%oTTBn+n1vh@RR6b=yV$|>Tx~eX8@)qi;!<2qfp42MVWvH>3#^Q{ZUAKg#;VpJ6k02nzIeYVn&A z)EUjCR~K|J*<8a1Hf%IW=Z%wIH&&-GpO3-*9QBG{cZrI)b;`*4F&psGfb(I2VLU`u zw)oVzO+!Kd#bZVlJgi%@NX7byB}_~<`{5tQh-63I^KU4TCox@FeiGVY4Z)QSiBEFoDRC`(d*{Z-t|WnoO^nnBlkh*dk!FpaLf~9D|NI zd`eR@1$Z<1H0|270P%cg_6e9R?y=u(i1Ft3&pXh{>G|mNUFwT{VTNO)>Q{PrM&_kw zZyw2_;ZBBP+(r$bQN`k4*6x+4*L1GaVWLt3SUZ4>`b%&vyn1mETEpvC1l%eCC3q|! zn;+j5Q$b=BxrjYye5VkU^95tBmeO^<16VJHO)0h6n`{=@<1C)b#eUpq9*EF34_@C; zijOrD0Y8|#wZe@h&f=THpCIt*)zj2&8R>8AH|<%u*{P`N)c}ilUzkjzk+;6GepC#1 zf@vbO{5yO&`3Llhi9>fpI=My<{_eWn_f~~yWO>kDl0jRQ(YTLJ@9y2xFM}QB#8ZL& zVcKt%YOPT*$~I0+)8hSJSygz_O*@CNEkE+*X)0ACO_(|c@xzWS$V653j4*HpJKfl2 z!AJUny!Xk*==Q%ZonxuwQ&j!F?!Qx0U%8pn3?wg|;g1-wpMT=r7>Vx)XgCeXA^scmm_TOjC<&tkW*i-r1BSWucT(JWfmxmTk~J#TnL8}@ zhoavBf+p9#TBkNQVnR#RW+-zIxhc8r+S8LQ=|EdK=!Bly6fc;s$F8jXh7dLeJ1YRBlrYi9D8kjS09;{r%HACM)SR>Lw8TMLD%lgK@NCI3Z(?AZ3(j3qhb-JtBv)C0=PKssJ}iSBvb zZo!L#EEj2((nOD9u+ZUvM7>s-vbp|Aa>TW(b^CvQ-bwfW*)DXm^5GUR!1a#EkZOqE zkTYn6*m^glJpZ%SowslK$mBd_XfO!eY!Y`DN&$1YoBLl}R_9|9^SuM01NR3hTI+{h z+=mIIr9ibmfk%!++@Q{2$3m$`gh8rKbgJ^PgK#x9kL$$Efr60hkEi7=8jw`{E3lc6 zaYAGnAkZR_=|vdkL>46Xp$flHi;J47Of7gwe9{>j_TmJ4;)jQCd+rDVCz7hRu!H+P zG?}n3Rf*fu>l)ce138AGcpyI)xC(XF_^DW2Er+wuZta>-HcT$hJ_HJKD6G+LUt6-X zFV;AtWjsbG=LFQCFYegNh&Cd}a4zj`41L=X`|?rY}GmRZG!mVOF~8&t9QV zT@w<0qXHT^VwhgARO%g?igsmWp;27PP(ve|>uju_eEmnBI^0n!X&|kG_T?XAO77AA z+X5`M03;PT<^NDjKOE-X%^n({R72$SBqmNg5JwfW2FKFKdf0u?g9?;R#~6Y;1J9(yu~*+Zh8II`zI-T4-b2wAmf4NkOP(%fK@U)9W%~LtWTP? z%*JNV_RJX+|Axbg15+i5-RD!n%xa{=VQ)8Da4a2y!H+>W$x#;_j$WcnS*lz#YO{y?o&7%{Mj4{$j0QJr@piu1U0y zz@QQMIBS#W^hCNKGtd5u$i~aHO&}@`AhNI4Xnce$fLCS@XL^U`{9CCdmP~8gVvxma zb&&Z)G!B)huNoi9SqFFp!}ytwgE05cJF`{zWi1daxE!eX`o5$mRuP1JHmLs zwHRo!PQ&GHXmZZaZs{-mV_gY}{xI&v?`vN%TS=H2od-=#sLNzX({HC6)mrt&8 z{0>tj`sg)kcw#46NH-T{e?_IpE1ZJESi(IWyOcgotsNZ6aG zDP|cfp&Ar8nH#hRkO<(P*tg26j6f7C$;nRbCO5qF!E&I4$GC&+3;1^oq;(30dp4R$ z*$i1W){1_r2zjC+m`F)o;na2vzd12*MRUCNwU5hiMEqiw7svy_QO7zpqzHfER_P97 zlT?S8u>se1y691h_cHGB2ZT8bJHmTlI|rU_q8@2Y2{TduJMJoq}|Xo&nR}cG>yAWmwEDA6%rr zyUh*sSC)%m*G&PEUN{t48=e*;V>W1?@o}`{z*0idr^1Vv)=$lGJDX3>i;w?!^`-?# z%JE>&kmy(Z1v$3FfJ*i@^naupDh`3*PC5a$+5~e)&<7AQmUSLs}1PIa3+v4QI9#qFL>F= z)3K7sK$ciwpu`Du&(vC~br|N1kIiP|9%QLEO<*Vo00hpEzgqqqkDctZnwru~zq0cm zg?R5n`xz#R@9owh?!9buTt2TbO53vWZO?qS=iJA9a&v{$IYnV|`E7cTq%~8M=!f5P z9ggnsDXSNHa9Vov)#1RqNyL3J*0?hElnnZ7$EHib)#yS$6L4#RCW5Ae5RVbn&shAe z6Ha=C!n4}7rZPErpv0U2HYX>r1pwuCQ5r*CySsA-`t^BREk0=-+n+~EHoXm;exB{}k7b&FyJ#RRAaZDwZF|rsJcM$s8&65{7pT`J9PK5cR*4l31khliN;_5nDGFb z)~NIrNg|{(ZYzmf_TTw#yb!V1`ke!7v2JS5!WkT|PrT#>v4 zL%#Rvv#j|=n-AJE)@9D&yJiroa zXfpQlp9;YtWeQ@givhtAC{xoiM}FxHykEf5yVtYb7JwDzW+u1o!az5& zD4TbbLgH=Uv5;m#FA63Ru|wd@#WuFk;Jy_(_2l>nWs@aToa~g5mc&-{-xVjmc)j|K zpSQ&=47pWf-A?5AgRmu*Xj40gZS7d(zy(kp{&vhej)9XJcgr!uK4Kkn5@?nWqku z2;j162AHaE#@LkgUQ3+OD)iBTTJ|7}?Q6lYa_Xz?eIy;imKdH%1XDY1WGUrRJ`wqM zT@`F28vtLrue z6P*k366Fx$rO_t`eGCM(BwE5K03KjhuvN1^3vWkTP`YGfKR_+(q(sWqmCG-UHN_tN zj>Z3~BRMdRvXfA!3sZKG1~Y?`Y)9KHum6e$dPo19r-GhjWUW^6%p)*a;IIU_Zi_1U zD)EC)*WAEkdK?T&I7YpG_(`d%(qVwK@PcJcAmtjY&5YAzE`}J#x1{A5kbL_>)dGvd z__Vvxa!x^AiSu0KgyQL`dIy?qr=D9h-Xm|hquNhP%iXN%J1NYh0rCrp2Ly#1w{aBr zOALPOos+WM1*k6pX#ic1&hfz-J(r+N-6s_>NB^-I&wa!?bB=dOhdh;1`2&u9<@epv zBW-ut%pTYmzgTn{fjU`2KmJOW%MwA3i0O@W?7~>=eFLqC0P3vffSn;FU|p zj+Kl|SR!vt846P%6ip}SHvv-CPnw<8IH~1SZ<?E1If?EjX8U-Q+2m!-^7BIP8R zHK+!>`(IhG{z*&!SLWnBVK(U5s@fD=5B;nwV(0>5M*?|97O=7=fzjOp67(6eDtEA< zx`Z%8V1Mhp>W8>taEI1@)gOwl1c!}=yR1hUVw)2>zGoWQt?#QP>WPak(eGc;!^~h^ z>KB11R^}`zDyke$Vm_FSSCTWJ30&SEl8Wou*v|4%Vgv)MuVr*PM(18%3bTB4`q0)c z+{$st#sfV!yn$=T>8lwf(v-BI_!^0-1sS+6j$zk2Wmh4aZ=u+@OTX85tVNeSAhlM3|#S@}|-EVw)shp47vs8IwBe*kQ zbXU=tPH6!bQ&*_%c~YAnblfF9lYY_K<&3k%IrFLF?1>z=M`Z+X5ia2w#h?g(I1>{N z2=kz4_B{EePJMMF&tE!JWs6qZ9;{GDzDo`kZMF6#+B`abQme`Y93r4LS^8Bcz?xd-&YTZu8DeW1nj;aD~W;MZ;TB+^v_EKqlx zlZf1Z?>vA%lSwdDNGrv5=adtXPA~ z1UH?Y@u+M9U+B6i5P)iyh5d4W=g>C34P}@4O+poEPsa20yA5wfEywS@Px9YJ15Qoy zEPGRcGe6%Ss50?wza{KZxCFO*r*t`ZjuwGa_g}_m9BSkVQ#R0&vm@)c708A0L<1F? z1|Wi-MspElq>MN*gjxdbj22b4f90-WhnD3we1K09F;^)3pis@4SKr4`C$$02w>JQ zkP;yl89G}C)>E<&F?64nak%(f8{2GI*I*Ig(yd>mH0<7D2AP*o>e+<>Br#@O!G_l@gD)!RdVw{#|IJDI=e@n z1nVsUEq8c{)-#4Sd&{9q=GQrbY*KWFBdQNy>CZ;)V8J6A(gxCTPa2 zs}}F4Ck4G{ou}54!#-!4WaYqB@qwZH?||c|C&DacKvT9z8jpvi3xr;(7C^;2tE1g9 zQu+gFwCvfQg?{FPG*m6?$(TX=o8PO_ckxg#YS*MNMj9E@l?-ZdPhdGLC}b6ux&5PR z7>va25Orc{=6fF)sy*o@4|N-f4PmR??b+0R$TAnH_zO&=ggOE};4*5h4U~#Ct*c0MDB{r%+E6JVL$>7rwBaA!!%a>bc}R|E4zPn zloXKN&(asM#>waOirN%^mE9Y2C_VP*v;gSBZC3ukVulPe_VoGfb8iM$A)AaB_H@nC zp(bD`rp?Ky-h<>{b|fCR*#3A~?X|aIxFT4v5RNVI{h2}tEGblfe(6>v)ToiU}L3$PCnh0pjn%RkDXBu1n$O5aO5rXdB6h&CHX*9e21#!kEA z(MlWl@idMXF0yIWc$US0k~LB9`(mcnxBDB5bi$wl1Y(NOC-CV43~f;OBen5GfZnpZ zKgn;aJN}YB#&xVV7P<+h>|I5$_>BnxYk1`^eXm|9`3m9-%=% z+>#2T&z4DLJLwSHnEj+|_;u*pFTl0cDnk!flTiH-P{SB-n*YC+f|ypc4eMUhwd0 z1S5d>#N&ZS2-g;T#ahuYh)~oQ^FslSD)ZX?-Q_93NKe%8pAo+f24fKOX=h4~A+(2R z(?zBlwwZDN>WLc9Tq(u@X{QJ*E>-dZAYK9j$NAGE!9aAWb{z&n8obCgyG_-}QfHZ2q}u%Idu4v^Xode(dG2#vfdh*h8ha?3GU zjdKe6(MiPYl!C72DHVx_KQ<>=ISt8&OW>t9$ff{@d}j=>{;tF&ODPZm-OZ(NYf^&? z+4pv)>BbvbcC9sxxDuBHew-(_-76ukRV%dESsD4exy)Vz#kYKz;aOTz4cIba6g@$n zGmk{Dydcwu84vYqXX95hhF2Zs7~;Ufpx}kj($7(KVhVZ?1Asjr3{({o zjE+Q*{Q^aBNkPvUDw0zn)>mk#>yDq5vQT?N@ip!6(hhWtw0%f*GLjkXAr|9e2|p{& z1z9+LfMJIxc4w9+v;D0&tuQA*#s5s7$RfTb@=9Z@WItW+E_L5qu%wy0;*0~k5DM9^ zW%n#BxlPX+orh6CQjCY_z!jt8bCuGY;>W!^^^HL-36uMFF??vI^Uu`6F3+GHiJP@J z_%^^XAnf@I6ZwlPOGIbrQqu}1S6lg7k$ecH4RoBjBt;&KJd07@GOzbI5ehV%;*A1< zZ8(}uzU?z?g#9p9GGP9UFbDIQ$A};X;hr>m`T_se&gC(GzTamcm3BdgB8-fz5p6;c zX>*|pB^&fy%~hzOdg;A&Vi4vIHWIP=z7hgY2UyN^+LJJHS`~gYmLcjZA)$DQnE9$R zlsrt1Aem|Qmw0=kF8o;h6MUXCmAUYPn8fl)IB3KkQF14NAL8H}W;VDobT)oKB2TFZ ziz@TnDY#i~)+q#&a-2g=?eKQ!&6;C%Iwox_V9h<{Mk}nyaFC}}=gb_jIs<;>*JSCl zM7_2NQ?Y3VJ|!>LVVAM1eM}Gs1UGLyzC>?Hpgy1s8J5&ZeL8n?>5t?0@wI#M1T3ML z|Fdr`B{1m0u_9L_b*nAhim@ZNVX|tgTd?@zC}y00#78=V3#VeUcsu$jV?cQt7LBniFZW4cmZ`H zh#lQ|y5hs&L{uAd^a_o3s_oPI=OJ1UAg@dK#J^PIXmpbbG=Hf@i>kBLDNL9mc4c8x zBqeWYI|gCvTkskOrrkC3D*J^e0ROT3LI&# zEzXs>j2zUyK*Dm?;6L_1u{=!osB9RK$ZQ-6FV0Plrr1shm(np7V_m4YaR8b5Z-Jd1 zU2x2uZ6^`oZ@)tqT@Mrq%U=HHx~Yc4v7Ln%S4)BSWLR1_Vhs)jpwZL*wV@+YxlMnW zFIzz!xQ5Pe-ls+@`}4v+z@uj(f3{#IH~|a8c;H0=K1VgB&)*dNQ~}S^=ru=Nr~c6T zs=TrJfs+^Cdjqm?qXMQ>kxS~D;s+zHKh8{OC+*M|EA%ro0&f&-iSCN6zp(vi8&p%7N3g3q(PW=uI0OLHy z`vNJao`H!;n*%rlk+9e7?-m>idG>wRUxl6G;U^3-_DU%1Y!;09I|oK z%8}L(Cq^d>)Yflti>Os?GYeSnhgKEwtx!&%%}A>x4036Qi!k`}-?0+HUk_UNA8&Qj zcrGW!y{vqgsZ3cFT*{4APUdwN%!;m$d_b6ZKV$tJ{`MX^HdSu0;m1(Y)JvqExs!h$ z;FW6kO(M)A0k61sfS*_*K%mE#(kR}xCN|d^2u#2rhmdo0#+IlQYYW# z+~|=TdGdjYTM{nj((LwN{&in|av%W2ZtuICS+8yYM%ZhJO5 zts0`nR1nn6yii3Fqmz(Ao$9;-=8oVN0cCwo0V%OJ^c`7egbv;V6RXtBf~N`z38#fa zQ|NyjtOz{w3mqNP)48`P~zsV7p2gJb3i$bF{l!Lp9IH8X| zHmtLe7C(f+?NxqcmW0leZq-fhHRJ!%4^%AmGiKH*2SD)TXOp~R!yxi<}Zs;-w=$Bdge(a($$jFn~;E;u5z7;Nk$#>H;pgFXKM~& z3YFRC7uFATvpca~j1qmBZDM0-md@92cJnRBMH5%TrB7X&zycYjQ$^suL%;_u!^=AT z*`nXtJbz{cGA!b0Tyn=QFF+LmUy5hVpSdE7tiQFP0w`CSBfvS@_76F zMFJ}wSh!T^AB=BQdibDSTW@%YPWaZfV_1o?d5YcEwbhKNq3qGoxWicbH!l~lFc5?R zDx-sWbK{na-F_yV`#3lL_JXV-_UHgNZxdJGAD?ae6d6CS+$pQI+tB$*+?Uwb#O{$D zD@0=$w}EaJpb4nlNhZKPr_OVvXN{_e*WN%uvzyZ|Stxt&#;I$A4^N8_XJ;&L6#M7j z{g$vcIL3SLHpn;X5ANW03mx`TlkKCrNH94C0NW?9ux<|z`yStRSy|XXzBf(A@4|sg zAtTNH%KYYPNU-@WDj;M@joW$ffLZ&?e4%m+G+9jT_Vir&RDy z4zWVW1EOmpyIR5yL5Zn@$e}Zcf2je?IAmdmB9#dT2n#wuJPw`!il^kZHf$#Z^A3}R z1V?>_*l?7qsXdh<3c%21b5B+RG&gW{pC&IWPzmvdt9SNy_KPNZTQ%cP2ggLNkxjW+ z+DVLLcjrH*KXr-3x(`Veu!eeOg$h3UKD`xcx!1G%_`*rmfTH&Jnb9Zk-RbaGD1^N^ zR>r1RPw$tV%Qx3QGR$5&pi|)2J#)=&Qi;%2ZwIk{l0+i-B;|Gz^uCI2(;(wS7d)7Ra`UArk?(EBW7TUOXAH zOaUE3fd(*tp>zoqstY$ZcQACZTwy(Q=*tBjiw}wlC5za(19q_217!3|R#KS$fHnv{ zqH;QkL;{cI%m4Hdd7MiEz04Xk^DpP9S~Ac5z|zBLKsht_zRG)OjvqcOvaM++U6~gs z9(n8Z>k#LP1>4FA1=Aze1Ad0PFI<4^hATT<81<&VWMO>Nl+BNK_5<=cCC>>3MvRFZ z)9{Xqr8eLM0x25Zu}8DOtq?egOlh&CZ{CyYUpzV*M|^vTmb!xWhqHjU8cbnVs^3TO z%ByXN0&QqCVt=o>yBMSb8!{?%QnwTT^`d(aKBZws3=!YG5#3YCBG7+FAbd zJw-1r@;Z18Wxv~;eLowsODB1w>R~t8?6M?3Jy&i_NI{&( zF)k6jZhWgv2s2wM-am6$4& z@Fhb*4G6B+u{DwVKNI#q!r{;W7V*IlX66c=hV2X(MAbu&dz0I>=jh74Hvy~&%#`Aw zErMwEKYb^=>H>m(+v?Hw(TUheJSiYoExT6j>f>0SWjS>qYmyCJt=0|b=v*+`<-LCB(oV@s81CetaN|#@^lW<3! zmgb{;PLjBOY_?=x-OPnbwOm4TbqL@veX)qna}E-9f~nhv2e!fUAipe}*Nk8818g5u zs&zU=f#6fu$5li>=JfU6)msrlsM*Z5ynNb;uyIu}0=yP)Z_?9JB+hGR+KJ zz10kCYi2tcPShEA1MxzYunHiF`^$}Y{F#scH?r9hfXO%;6%+_3z}~a$)|eUyE;E9= z_waeE;4eyqaFjq05KK0l4WJ$}qn}Op6hU~|o$)XC#_zCO^CLCmk${20Rc=i%U<&Z8xxj9VKu>V7E52b8!{`C z5qk@+H%vEsqyq$GAL)k=Kl>~8Aw3h^EZmVF>2$4-uUzspxUGNxY(^n7wiq^5p74Op zs<$if)@w;a;1cNfal3txnU!I`eG1x(r1mDglP*YW$9O>dGHPm~pV8x!r9L}@6DFkJ zXxh}|&oi+0GuJ<>PX}!UIF+$S z(gYZvcG`ur>A52?))>w|QNfF3E*oCMvm9`qtqK55aosU`@nMBeUORe+na2eSUMZy# zX8A&cAGw>gBy|AtZL$@Tq=5#t5Tx`ED=cmy%PM382|Y86Y zGKFucFIZrU0RtTf^8#TWP%|{{gF`c4BSdWTg3*NkMo#w>`-y!{G(3rJg{?}U^QAfQ6 z6ZpwPjphc}KFU-%JONW>6rUP~yY+!*{R-XXz_k-apc{Co}eg6mi2Vt+jFoxAB@+E>*zsm;c5K+}FLp0>OB(osvq zsx9km?jL^J4}?(1l+y$_eT#^!2rI5NEr!)#tU zZ`|hTgzNldCeax)4kx_P@p5&?ebciGd`l6aeH3P*QEnGXbnW!XJKRvVnDB(t3NQAH zJj7~2`MG&#vj-SQ0|!FyI$yRjCb&nb|1q)5+VBu`PyHUo9nJh|)j+oBoPEl5%`qq& z7lp?Pxj-rtUU|dv?m1(gN|(4{p%YBQQjOh~HO+t^v%=P65?AqtB8k|vUvo24@nI=! zZ(zV412)XKGB98QF;BqOR;y%P`>jBdc;yWf?Pt>d^CkOHFkWHV2l>wgZfs8;L7BGO zKfm*5rlA;u$GH>G0v!B%{S^W0T&*0ep$Go52+H!~f6)Htdj_Qql{B{jT|(b~rP#^D z!Uvg~1SiMC4&(;$AJ#PYYCHGG&)8v2N5IKTG#nm zw;f5BMmVsDFv-qg%QBnb5pwIoTs+ot+a;)};@e_uk@I1eR|}Q;l;-lsrQd$iX=P@M z(&>Qn9B$bPZLqF-s~&zEga>S~*0)8)DuC$DA#xYgHg=G&0S&(yFl%Z9jXmz@#d*{O zjzo}74e2Tib6f>|)LAIF4`VfsXG~F5Vc#iCG0IjT$`kU2 z$ZFp2dETyzz6tE~R7y4B@|{^ui#)Iw6=99PbH^*!bMmAa?zvbjt5L@d27T*!mh#pU4BDlH4(l z$OzUxW-yc|giig~8NJ_V;lt3iJ(s4}^lS^lgX27BAyH!Q(;K&Kd&`z>_#bqu-y4KM zBY!3jiek?6o}~JNawiCvpA<5e0z=WAyBo;2zj9zd7-f|bdIpb3ohg_1G#4+ zFJN9A$04@ypbW01(kaTkB!ickGOGJKmOgy+MJmK5*11_Ll=20cgRINO5k#6Pr=r5= z2sNO#o1VvOz+7S{UN{6(RluIZk}~WF-Y~41ZtY-Mi!0L#T~$v?vnmfBr?+~J^9i(c zgK2!pLeKgu5dZe;66Wrpqs^|Jjo@y8vIbu%Fj$P*nDdTz?ii3@*UTy(yIApm6v}$U z$~r=X@5sbQUdw2lNL`}8ItwjURuMd_U@XuGL!5#hgp@baW}0BzA1W(#6qx>Cc@!zD|)>2Z6Z{jwzrpfMAUlxPMlA z8t*?HdV%;B<={9Z%|W{YiC0W-R5wiiOzm&FAU~IKNIF}E@FTj`tQFrsYiSNs=0W>i zJ+UlJrM4hHMv2f*MX6bI1fuxB_~T%X=;T6acMHTPoig+J{-_)Zf5gL#ivWu7`K*6+ zGtyWH4{|#IWZ}{&$OCOccFJ>=@g{Mj+x(fTphRs#t8D^7;h>q1fmjZv`?rQEFB%nr z0MG_DAoRLLV8vEoQcMi?Y`PLDTHqp{ZK5rnsa|0MtsLmUwS<{MayY?LP7h#-J4Tm) zVvzXh8YtixW1FQCE`n4<-}rQWEu`G=AIWwBdPy%^r6}9a5G=k@f-u4daIIlKOB{}P z=0hwp*M(I#kJR}lTxfS4zI}C0i0ujf^Eyup3^yyB;6B8cbbYfE41qbC)2y=U^Yf>` zs;9h=X3#Uu>h<-ImAOg@+D2{5Bvc@@a_doZ<8%eG=r;JGW$FYNWXuO6a=8*%S-;z; zeI4`C+X;oZDli^rih)%~hJ%UU^(#=~*YD+C*Le8I>dF2q92b8|2LJ-~@)!HT`rxjX z`?oSMKLt2Ed}EM~jXk5g%Q=OGwalL4=Es3kWSHkkF+F>rzz7cvdLoK^`vh96a=8Gk zMLw9JP}AZoL!uXMvD=fqw3G3^10G~997WD$AMWRKjblLOo=^@`Y+R-xdBE7++(wyW zU^NPH7oyaaBN7Kcv3VFYBZa3Y{t^U|G9hZ)9h|cQ`uXVPLTyN5McoCUNaKiIiI^|X z;+3e;ZoV%7yZ?0Y1S?+Vx5(@{%gDNCfN8eM8}|76}S#f8D68Oq;s3x zZD8pN-P*^Mk(^>ec91!jUS;DJbB3HWF0!WP&QsmsW}ae#VA}wWNTNomp2c;>d#**| zN56`II;|tz7FREC&hqg?K4ilHBWpfO&k#DdqrIf9AbvXKwb8DQxrss*pmO z)Z_?g(J5QW8nUim0X9SS3x|%9i3S}(-UDj0SP5uP1?$VmcCNDJB*LWGv1kMmO@tDA zws~43>_MW(TwkmIR4!3p7^t>9eI?H263pDsdPU7r7V%gY#~^U(+>LhL{{u9IjML=s z3lR`*=Hxjm1HS+)={*c!XxH#-Sej1wfm@SpvDK4BESSrH+me4QO&L_bW)N+15QcB0 zvkJ%thOI=LV$kKv9e&U!Qgqy@(|IO3Q&PVQ^t8k-S=s2;*)GYC)MEW&KqVcHC5oiF zI2eZY81%GGWwVI^k38cdky zp)u%bd|0H&=^FWk3m1SmrCDJXQXo1~ z$pAT>g^~gVC;^#L6~zpo@i~H`AwH~Twwu%KJ#+W{{#Ta?sYl>@2CXRcrxTG}E(8q$ z%4N#h`;~V}O2pzb5+W@74pw3QPhmn$C`?dlEJ6OY%hMvnCAW#)Uj=fv6EK*BH% zjjYeW4)_P2*5UzE;*0Xfj4G>-mLx`4PEnu4MN8 zPh}xfFr@_ul^h3*2jG~2UvN^pU6J`iV<3NveXm$kE~Lp;u9N7<)s%&5RIHhsDaL!8 z9Hd`%IuLlH_%-*%!O8&Xuh$-nc$l}A4UwPT)?Y~qF2p;&^%kEs(+;-sdvhWfUoJ&Q zQX&jX!0~as%ZK~ka>V9?VU{FDp;KRr^dBhp;hcX_km%(^L!_i;b}$5LnMp($i`~2N zj>)NGb}jnQdF<~OAE?47+S^ZQPaS#wc|LJ#*+J9jYb@)$$H5rwNy%j?QmPi^_>ns} zI%K|yGH%c~zUd4`gTZws&0+Ex->xWVOXz{7>Vn8rCfDVkcmqp{9ODhfo&=6nHOapb zb(aZa5Se(TAEGEto(JY*XIOOl06KzUcW&v6Ng!MFIOv?>VT`+Zj0pDc=zRlWCTOQc zQ$A+bg1gwcTU6LTMcb;C2Vr)Z6ysgwjJQ8XzZkumCl4RR9G#&Gxmu=Bg!?&vZVVKJ z;Qbn*6m;C+4=KU}CnW$mCQiPV?9Wq`2fb?z7?Xy5-~DK+uvh zsAKjFTO4&0;s*NNY?r-hUC-$g#3aVWe&PM*a5<71%wxuB&k4Uq@+CoLWc&)5(F&YC z1k^nld)YGJIBK&|Vi8Jg*%)^3CY*Hk5N?P*Gbq5vD2k!o_rz~?rp zx^rxxz*Nh{*2d_k4P^`_!TT}0*bICuUD7Fs$)T=&Co)@%vY8b5O2L1Au&CatPOzf= zv%Z;T!fjVtaM^FFgHQ-u*%AE zx2O0o=TMoCu!T-NdgFfx`8NNH9Wyk2UCC$xe(i{h7jX1Qi?s>uA`0F+0TjIz_yc?R zeUo8**8ZQ_b8_oykdrwRLzphWjNvOIVZe~^?&jbm)DYa89B3LPEq}LJ(8K(s3U8~G z6r`$w)KR0b`sZmx5-eY;#INk{jqPhym?87t;tS)g7f9-WWUL9;!`}rX&PV_&+rGWK ziCU04sW<|0M}xr@vZ^3orO6mL_X~RHgP?u`q>c56lo9@Kz;F{qv;ZK7mh0SW4h$1xY6;WC+Qt&=po@h1@}S z$IUj3s1!$2K7A3{rQ(0D1`*+Euvp3s$RVdHc+#fW_(D9?*^newB!2Vn6~<1*$*#Nv zY!5m@Bo74%X>8yH(EzsNNB1k%7uo672oZ$FH6%FNA;~uIN^H?x17v z*8->iVM5bN#Faw)mpi8;cjAsb{`Z5j5k9Crd`gtGh)#Z}9+N;iqbCeDDo`L0yuxGC z!2@%@Md8D?0k~|@T{d$gqp~Jlm80F{M2Y91@A{PbuV%-JX3?1dV|=@wM^J5C&-T9Y-&VJh@MWdK(#`3cw}N|z zfTzQ|Gp21+7feL*q392fWxKE>zzpPC1(<~m-d8i>+dB9^zr=sefI%=|uzyV9E{F7F zLb@a%W8?4-;j>1w(|zdpS|Sch2As(JfeY4xxalYP0$7lxJ}9~fdblrJ5UU9E^XX|} z>WBsy`~T;f+E8e~Vlrt0Z6zC&)5}M4&^PPdKH&eA75Cv&sA!JHQS~>)_6|_w>8Fr%&fyk5?*ZtkiDZq?)*R=mWD_X#W`?m}E zqal$A62B89sIY(>p>mvF^Z9{FK%Z4eif4<^B!bG0XPtw&sT7Qnc*ALNyUuH~+mK z$KT?d^$iTN$CevQz&|YJ@3W7?3DpcpG3*dZNbCFe*+*R1L3ae5-*Yweq0z%62i*i` zf|(7mL@1_I^zZUIHGcBK`zy!biSMDUk&Or)fi6{x@+feu{73#PDaHRs)px*i-L~y3 z4J}C$LP?3c60&!Q%7|1-Rz^gTG7~B!WmIHkWTjFhNp>PzcF5k@A$$FgOFj4f{-4j= z=XoF1uiy83UDtUY$8jF#d5MUS?WK0)KR4TNQdaP%aA&?w1urGJ%v2@z@UkBmy(2M| zCb)Spn7r+$bj)O5X$bQY z;rO}%z;--7Gt$oASU#v%cp1;{+UX<3dhepDoB)2F*fo08;5tI`Xu$O6wUW1GsYO;b zrdSk^o0`eL@sM6BEcibwbDs=3pu3lE@u^F1ag&MoGj|COZ3F>F7$Z^1@WAEsulaX2 z#0qlM(LPOHkG@)Gfx0g2JAbkOzF8iv8;UiFa-4MQg%SM7Y^4rF#b$^Q_?s9A+{dXARLYi<$ zlR#c%@81wmVV+VsF!2ua%}C__dlqykUP7<-r$Nd;lC0@=kzx7z-syICa$i0EOnn5I z^@t(d4!<1867rTBOU<1>_QVx-nRaXyR)CSjJ-`0H3?)c16m2J1tm}c2Pu>0({v;`X zW+(m$qMo1_3beX>uvET~D02(NwWCp$pK)E^Chtx%>ifXD<;D$&U(?m$i*o+~nTsUl z`!kTvLGqeU{n=o9$OdC1-wylETm7WOHh-h`*KzS*aDA>2S+*kSe^<-y@^Vdvmkhc@ z6%&twP_ztK$kR?oj-t8bWjP|?ZSpx7*t@3%J**wLNVAt@s=cyF4X#G#u~k;%5_Ts9 zmIN|S`tSD7kY3&jk9;vmjuepoZG1dJBSuGdY+yNQh$W-Qu^Gt~9};B89v{Ue!+Mx| zel(?#ya<8`@wNXhlHDB=to?Ukkic#-7J&Fw7>O2*qb0kXfA7r@A)APYB+AEU&EoXS z(kOb9al-`6A+IP@{yXbB&%?`s0(eTCJRJPl1(tSgcb394=cs*OXDtdZhP4@Ktt>sLn-%hyU4hn1K1kI1wP)$J4w~cMvHlaDZp$AcJbT>aK%Sau;Ul)v zFo_E?2KZbBuZe-W>KMiCU)p($jEr!39!K-ub!=77LG4zeQf5 z%zarUQ7<>)2b);M1Kl=kFTQU|UVxjntN~Uaa%KiXBV?xi^&c*8wLpt2KELB-3Yu*( zIx+vrUd5l)u_wvDx%^qo`m|UNIx$QqKA&RFg4r^-MKC6v8K2m3`iY!*DB`0I ze?XLw_hvqR^yun@tl576cNJ}q%kXcAJNq_L!G{%pn|usar4s((JT-HyFS??Xme)7i z3@q*i0_;yKQLsezh(ww)IcsYH5I*)phkRf#4?W_tCI*`=7_*y-`)lKhN=xNpt1F?T zVZr|_hj88=e)K>8vp9tUXk77aC$|)d`Xqw97Yz%XNIKXXdKUcrvURxn=v6#bM|Co%4Thu4Hl(Hhrk6~Hc+%?-KOMbn0fi}G<#myEXUX6Y#p_fd(B(#>`XRC<9g{dY|^b}fB&GvTBVDYBYTu$eCDlPpPThDesMPoSW?9K2Bvxm4Qmn{7kts2MyIP9KXlm;pD8D z_&d#yDm-}jFi6T-65m_#-;)Zz|CCy4k_c?EgivnA6ZNiS#R?9@m&d0oqE@9u$J&%ZS&D2Pl7 zBDNx>a7j$1Bzwu>YTbnAUhK4=Nmm;9Fj(!B635n&>?KK;HJeV54R3}EG&Pn_>+)ws z&OE9(cmI;%^sn|?7*UB2VmYezcbzn__5cwl6Ld1kPdaw}z^R6cBpX{jin$`!+ugRm zAe%lDyRk2{)~pk zjlP%wTAzX-{>!xJe6~+hq?LK}b5S?*X0jFI-y>Q2vs(K$O~)@L$y>ruxqI<8yZ)Ss z4qF%tQIx`5igd^M>vJu`DG77KY1(RZG7&VRcDJ%6(Sas86h2t0L1*NXvbkr$A!%tb ztXH+ErF!8(B!qR_xr#n3W%H$EA4kX1_IllfI-!|X-E<3!Nf8Dgso*{7p3UY$d-v{r z|Lqlxf(Ldz`X{N}bDIDJJXyrZItk-QddT6U`<-nwGgBHTdwAz| z{1|3>vM|;e&^Tq2-TI@Y#*xE!U8astr3t2}58n9oDSJhPeAEhQ{EyMeJ1x%d%y-6* z4-ScJJ1Wvv;iNm&8eTATQn_fvSC1!f%rR-eF?+JrCHbWK9P6>4Z|Vj-zUoiklI$22 zCi8)!6nt7zcG|72j2~bmA;FuKG@?6YhgU`^jN|qtF0#Lv9Kj{YfWFdTfrp@RhyH9j z>?CUTbzW{Jm&kB5CMG6ga}1Jn6h*LbNK+Km&sQ@85^%e=qX=ZmxDLYHYv8 z;9KO{aOJF&%|eqNj~+jh?c7Mh)ue_ihL0OV; zl6Es}9IpNNVzR+HJ(9DgwzhqI=9DBduT473v$A_-mlRVoV|`vszKV-u7ZM7v8TwI) zqy0gr0Cy}9o>EAjZ8v$giHV8%_RY?uw5EhZdtVz7*%jw)9ODZsUa80a7bZLr86p|U z_8Ya+lId(!MP#TO*h}l7m()-QJ$>EHDXa8Xb4ncQY2n^93QtY=U)xkKeOEX@&rU-% zII-w3o@getbV%w+j=3;%{uM!k#B-;dBnO8C<7!W+sHphPHCz!?;;1%nPpCDO;H;}N zPe}qHd7qTDxpS>*Rwc%HX%PXE%JR0n7n57vK#XyD80q!I&zSENmz0P= zUm(e{~_IP&yTotwQBv1@N8aBv~H_XRbD}b@rfdL>Z+*9!nlO26RlZ} z)!J2CO3YIZ+YW1Sr*_do(281MbOLHc`mOTfY;|0PEY z$q$-ue<&|q*eUpIlb3u_-Q3cTf_UwDhuLFm(zAPo9vT`OS0|fka3}}MWjifYX(Am* zyng*4>6_?7P$m`m(;G&`a4olR|G+>|QBgbII~Kwr=Ho|yB&ndZ>kK3UN;=N+T>P>M zEjFoE0o89@W!T=F-B0ug?KG>49eExkI4?h*CByMp95_{g!=kdXZj92>oBiEK%e1sU zsiAawwDVO$f^p4DwYlW@{uU=gBct|FyP63-U$Z{e2TQ?am#~GTi+3J*rWrdaJ%x%s zj#%z>?o6GNXs=eIdoto-+MhjlQ&+w5_f3iRYtxTbr{12uw-Qcxan_~UiNMW0lbI|R zvSKG*`m(%p7_VR0WYO(K@9tk|qGY>HT3`Ds=|w}bMnFAuCy^%GfDC|p`Sz{Efjfzj zoaCAraf~`}&AkXfH#yMig)7!7V^8i>ky@~V!Y?p3KEC-#a`sS=aoEMG=;LTGH^;aJ z4yC|%jpGfN6}L3yyi|aG#)4O|Va3QG`M6fZ*Z=0`(_>S+dHhVTF@Uj z_qjgN;7=mLi|gR=B^kgJ8GsB$L~-CgvA8#PGbff_I=s_qnvF%w#<1wgUiS|-co^w9 z#FTt5b{|S%@mw$+a@QH!G^df@{Bjr>erv#q)tLZ&9N zaE6!8IlR29)n`K6XBF+5k0capJJNy^C)d)pCvT)4GOYrjS!!x(2HI=f=Wme)2*@bT z*F;0b0TAaFa7UD7oDrCfKF2TUOcO5wIzMQS<@XGUYrc~{;IDw zl@6*n{WHpkB9PZ$jjr>8z)02{H@uqPF20@8J0^*l>rcpg>B>%rwitCXT#}rvKU*mf zC|~Zd^-Om_b&~PPn-l$CF&LvDCO{{6_=)#1`iE|MJjXccM7yFc87U>xbR4_L&r~a< z7hR1OQ!>^2{CmT7pp>UH&9=x4hMUEwu1RTBwy_aZf1xDO)eE~`W>3QJ1Z_w8F7Re1~`bN|^l&oeSY_Q!v^3(PQDtD#HW#^jiWIWOAd z4(?f(vu2Kt5|CPAAc}FV_!TcX|4WwcXvC*afh2Z+*>LDryZcUZH{Y+Kec^h^n(N$i zOG8Aq-X@=^$5RNm@n^T~I)xzRC4iFDnq_O%`I)+bw|ihAiTLorkJlid2e?(>z{iFw zYpv#i`^R7%2!$N)Pt79A?D@p!X3>Oa%9VyE!>9gYl*_yNK)6l=vHon$mSUIp)B||v zN*!-xeyLnmWZLJ@j%+OJiB{UNvJqd>h+fO)tsVJA6R6o`XQL|+*> zmj&{=kbr|k@HjM7d88$?=-aoRj)#YO(AQpnbqRoi@)x9qUlp>O69JjLr55e77A7xn zXZ(_xcanPg^eN6HVK2Y?s)Kk9?r9DH0Zik{ugx!eT&@D}k-YZ7yg3+lA0Ozy&bM$m z1xDAh;LK|a|EjRV3g4Qj)o_nVUoat31qxN4Z?B-N(thv<&F$W_oLzSC>aViMLdV6e z@lCLO0!AwANor8qMI+p$4US{EM}*d?=V+7~W>4%}epBSYBK@hUs(wZYzj^LkDn9(| zn{!7jEqO6VYv;43eY74g+u!J9_vTyx3wU#Lj+jW=uf2oYq{Vq@VIh75Lgi)vIs>qp zdCa$@VA!}O`S2&41l-BEblhI)@?|cHt3N)hcn)>u7ItTdRn>6<6FnJtWtwBbi26H; z=tu$-A^|y-iv9;0M$$I>cik+}PPgfXR1|^Td-v{2R25lC8!m)bHVVXA3Y+wW+us5< zPMevUQos3i>C2E!l2HkLPiastLRj#>VE6Jaf}-kC?fHh#Sq=&uC~9q*RwA-{Z-M&G zbt@jw=(l3#mvL8buWvv=0ep0!!`GdJyj@3YzzHJ3U&(FrU&9|pdpH@{@&+Yj7a0aUlG2kDvJYYsW z5aZQ%`uh4}LT&S^&)JI;2@9`>APXBK8=)PSVhuHOAw(Yl-Fa|VEi||zX>q886oN77 zPFR|!IHVYegsNQn7IHk@W(fIvGdhc9eS0&Y+miGRA`CWoGrM1sA!~k2kwl-*cPN9z zY`G~w**kH$W^Zq}bI+Z%tfcX@9f^DewN7keLFbCL>XTANU4@T-kFZ&VB8i)T zDMcqMP1?V>k+~@3D{W-r=lmDBRY?L^eL+-Tuh|D)q=3c0To$gNtSh;uYB2G8Sy96WbaHo=eglf z#-krUG}mlo_zaF`d9FNsBf89)KNUEV^F@K5OG9*kPEB?dA&NvRLBzcQy<}walgHM5 z`ut2Jk|K!wh)vVt4iN1SJ$%>~K)@YaAsi$i;K$K}vnQ5pUsi=GMJS$5d3fHQ( z(-relR8(XH;=>6iBp=e2T=OC>W*}?~f=mI%!Zsi*+>v+{rOL;x4(se9pbz=Lva7@&R0GMTAi09h?@5xlE-24ii$|LA7lb=3PRu!>7oIV z5Sn*)XFYjBJMr;NdCHWhJUblIZv_tMWCVgwz9)=uVj$qVrYlYML1yY3U}C4vp3R45 zPz24>vEJ<0%}feYAq#aP{M~k>&R@TNJ!xAdm>fxPn6*yPyqJi1=g%|Z3zmjUg?qDz z4S)2Jk7`GbFdXgpO29DM7w%hj=D#X`#kE;#o+tC>#3==Bp}lw=5WIXtLo*v4Z@~Y6 z>jXZS);_>2vl}UnC|9lqcw{GW>iUfvjKbG;5y=3{?6AYc- zZw}b#m8?0GU=!wXTH%XbzoB~~C^hrU>({SIA+0{c{xHNHMMb@hFE94Y&x}9I$bSF+ ziO7u}3XkFH3l^DA?@+F-iafeO)avKwgDFj8aTbA$LgvgA>25<(KYXR0Cs}q1qwLy{ZS@xCXAr$tcqF$fcrxVZE{<*A#d~uuXYEeId{0IU zV^wqH2KrU%lW%!%w1E`92`F<$dnnnIM~@!m8W4c~vUKT->pCQJU#}fPWLJ>7G|6C+ z=pl2wxd^q*7>A<onJC$a&~K2>x*t$0ZfS zl2s(z=iFUgip=}q!2_#+8*)%kC28f8Uo@5+$}UALV?2CmW4ZYFMrg@M4_Yh>jTIHQ z3p#T~ri2(>pmg9g^T%BQG3tbMvqr9}Qope)j(T zdybR>ecp+z;pTMTprF(OgVJY0J^;Y}=+$LDli;!NbU54|v7epRgTyx+8xcgrYE2&+ z{c;lB@Qg(#r55ra7Jb7~KKZEmWb0lXrx~NWIc?@s35a>;4y^f%y`2j%RD{E&U~eyi zc}eY9;n~{Hb8w-^$5toeT1bKV`Geiv-6Y+(&HQ90}I_wV_JGpBS2>k_pc_N+A}yPcq5Q{i+}kq86GTAVe5 zj^na+^9!5?>}vvbLEO0m6f>zI)AOB_n4mk9@oOcyNo3o$Z6xRPPY-53e*8E`Y$PL> zU}Ve#LU=Im7j&{i}37JTaj8!pddq&i`0q&#&(QIR)DiP^Tv61lA9*0?fB zXDBNdATTH)Uf{%cV_b_y-LaO&gflljVpX7i%^)jv|I0IZAqLO@B-|z1D8hQtkNoUi zov*fH0?F2WDqTRDodB}kFfeyvR&mtPhxC8ax5C-8^eVAi!Q*uj_94_qM(+NRVsR3T z&K`+j^C#+nl9(3BiV8anD<)==jm*N2W@l%chr{`d))P*Txi5n&LVK^hPDt3tOdZ9! z9w-u`q{x*g1GK|G9})fz zLkA=p!vb#>!p%&Ljnh1tMQD#%B}pP-r#H8PNs$qwH>ZZnDk5a<;k!kUVPy4D*mQiK zaxfw7&&0FJE=7UINk^vNv@l8bSnB@XXO%rO*9Ev*Gg>Uvbd6HXX0Q#?|JvsmKNEte z$BY@dfn}Fo$HmzXCD*-7NYD^-6=2##&OZb^2*j}WeR+K*j<|O10mRhk+|YdL2@qTT z_JR<&bRm2Tc{?mJR_qZ?+`oT65bGMqik_YuFO>%7T;@ z=2Sf};k8!rxqBNVR;ymP@GSoXbvWJ+&iD!l;#Z3ui?&36X06MPfAxx>q|5Kw--;;O z#*Tue{RB@yk?1)TmCJ9BQdNEkI&6F$CdX6S+GUuJ{N-n5RJArONz!DQD10O)8NeW| zix+ItLkajm#2m$&x(07{1-1hC3UO~p-25%AMmJ&W`l6<$hGM?|_XF*1baZ+tji5Zp z-F7~6ZZd47#0+ME(a38MS@-PQr?9|?iy4K+`eMW1NxwGG#f=iKysLvfA$1X5Fnup#R&$4;Rlh|0PN%HBp8|YiFs{Uzn+5gpMUf*cDFzE zs^AU`#Mwat^`mnx_8LW_P3enHdxAeA=I$TYE)AY!;$SfI}Zq{VFLb znE}y7@*dZ*m0!>yD)SJ>Vsgba`d&J*i+a$I-;1~$nU)r8C;p943Y6D28y977=%e=9 z4qXJvx$ukIh8R>5*el>%E4jagy|B2r19u3vql^Xt{WUy>AuA}?z){EmQ7#+=LPtU+ zs*>kYDTAId&%7eBJPb^9Scw)D7ylRtCdrrrYF({i-2h25Xb&aT^rG_eUc_7tF+X4S z*S<^fvK?i&aEOiDKHtHSjH>QKlkfC`lfbpr&_BRD-JwfTzEnSk`(StT@$pULNoJsI zP_0^t-&7d79{=XeDq+s-C6#A_UWl$@F^eFNrO38#yz6yPT7EY^#TZ_`_PI(XxeFBX zSo4suL;oJZXJuRkD(BI;06lUbIl+Brw>zH`SF3PI}5-E?-#FN&hcCB#X6}%CtOrY@SmTb z-{ALlM+Clc*b-QV$EZD!J6@#-S+ta*$FdyDiAtjO33u2p9sOyU39}%zqA`vjiDSKX z{dVH<-mzg)&Wo(+d2Td)q|2V~MvoGTp#?y7*3vAo?sprm^Zs zrh35AE@EY2obw{`8sKi2^sd{@=Dl7*EywzXhbkf{A5r-`E4x#(oXd8SAb$;!q<&}- zLh|A>5eyxnJK9dD9kP*CUzHp7zV1TMPO4QD!CF}cy5x#6L+}(+p;=_%lk)OwB2@2> zja|P~9`<%oXR>4L z>$FTGZicaz@7d*}Dl1+n?|}MqqvG4+`OfymH&H2ZkPe1E4#9e+D%*J|d!Dab@~9d% z^W_a*nK}ugk-V?Kfk3%j@f6UtXF$)?ol!CsX_BYutjNsWr#v4ZbGj4k*RBD|5kd6}a+c_9-;Z@ZtrlVw&yXnQiyE35U zC@x_JCz{&TI}+_VBU^6^5WgQN2rG$w(O8Y z7mcEgjm?RanDX9E*l;>P!6M$g+2~tDx*!ryPH18J9Pyxvqg${4SdXqKz-N%))nF#t zeI(f32Krd+7fG`n`G%|ogW^@+2+KXRYGUTwxwQ&Ou)s@xCVqAg?F!WZXJMKsd9Mzv z+Y49)$m_wc=%NMU;UKoyU856}fdXdqM2A5+hlD$0YzwQO6?b$pZ>pU?Uj)K>zo3&s zDNH=*;h`+y?!eWnSDPejYH@ETi*Ww@dGb{$6i(aTN-GOG><+`qQ^4?LrH)h!j~ynN z{57Nk9+Qe)Yc{i%*1S@mfuY84_(_#YzWRs+hCE*T+??kS$@$>NaR*MB&KSjjQ<|D5 zu)~0UZG3yBm;qZcoYVj;hfqvBf5-Jy5@Qh#do#wrQPNhgK;2^TSwZ!{1YW)iV0RE) zIAK5?JbIsR%buEoB-4OOBx+Nrq*uUqA!;*dCtcu$b_<|ZQw7!Qj$a&!!;lA*i!g?^ z92YJwE^>oQFR2Qk3yy@;3+AbARaI4#uP02Nd>gGP;_gHVZ@D}~1Mpw~8AWdQ76Q@3 zz-6rhem{c-Ped4UK?b3{{WY(M_D2pR=({RtxWmfS{hvLeafGL> z^boMpz1bJD(4SAVwIX;I^NFN9)?Ly7wuxFsJ4DE6i`FbHr!AdCbp2xajKXsd}ZpkKFUc;nEA$NPtJYLP{SJ ze@gmQRQ_N{yd4=C$syYj>NRopx*))(z-2dM^X zV7X-NqYnlJ)Bx0U`q0)2k515NeiikUY-Dbt5LlBKc5g;cdUs$qa^!{s_X* z4G%7skisQT?$6avB>fevA11D@-N^9nq{X>`+PLjIp)M5f2!#Az4#x&CEl{pr;yLK>Gi3(2 zX>h6IP13i>h=@CIF5W@W;99eF9m`njfYV7L>*D1Qp|1L0;k%5#xZO0AR%V_%W41qz z$OTX_`0U5JF12}R$j12VfnHsPX4N=+W%b|Qq%!p{( zi|B8zAN=@h(vt0!%eypBZt0rT9P5msD!%K4Kxoe;3<|tWV-sOn0Wq|Ky8xSmsO%KxSu>XiOP{bL zRXNnHeq(dajQ1|}*dy^qs8XQ-m$f2N1YdS8P~-*&A;cD0PMbgCp~iuVEP=oi?M6Fv zH%X)V@*S-0Tj67NOVQ8_9A=2Hdz$yMoG#)m?bCGH%|iGj^3VBX(=)zP&@9GrL5Hk% z_KXS5dbexo&3sO&>)?~^@I3tOIbbijP5$2s+T%-Amk+oe3p--QFMM5h?X@XZG6{8uv5**>2?c(AP_I&-tJ zWK_dQOA68rGD#s;;Hb7X3t}{36a;%ffKu1ch?VTYhU~_#t7GDT{9RDjdr_jg!YAG$ z(3IKrM3Gq0W@cuPzhVZB1)nkUo6wNz@1t|&ybXOJT@yD`e?~q8;_**=3uOeVG6~U= zi_->}wN?yDS%8HSZg<^k6IY<0sE5Mis+C;U%EA178e-4xG_MqRd-h>&`{yAWA(4*R zI;9X{-n~3o(A{S6Lhk^bBWb85Od+m6zn4F17lP>=@{XlMF9a%%zn6w%4juCs?s#(x zcU}yFfIv&NYSLcI1|k-qRQlB=R2FI)ixQOKMZ;c5Pg?+R*AO$d5@mtHnI7s~7vJyr zcF5#u{Adj^srL0w++-B9Il$`UP5=)qD1xc{ccvGrO!$d5)c<6AT}~^23DG6W(v0R!pbtb@`&oS<_|LdhUSy)vH&F zKO+3RN41b#LIQ63m8hGD&Z#J2-pQ*kgyf%yUw*0+btFj;dB7mVb^^AmcWBX?7$-NQ zXX=!LRi`&*ArG@7yPuKLO}eC9ynvS?1y_x!UK6r#)=bY1*cr&qBJLGvCKMpGKp4Xu z16ONbZv!E=I-yYxFxFR}$WB~HcrOkU8NN>vY8|yN%%YzrVRnWD#TLgYZGcyKWHAxg zBx>hu-Wu1wKGR=s0tXAeepD9quF+s3OV{JdmkrywyD+(2)FfgMQd^+lRi;PNom0de z0pSAy^=hEF{h0Y2acx1%$u1fnVPa~$^3ml~>xg!msF!cVD3A$Ebh|eJV9LQ917Lm- zjI0e^@Q-oiqUu9LZh+Wgl_Z;E{t#p~ZwlfewK;sPO&SXf{B1rNk%$9QVYj^~c`5ik zkhiYQkj!pZ)7GraS$p88X^d+E$Wm@7@3R^sI$Mq<>h2Imn-AU>WY|dh1PF*k8^K)y zy8yW-79ny5kve2IZ~B@jlxExEC0Nr&c0*r(ZJ*VA>(YYmiA;b3gW^PV@bbd#L9yri z(leM7QBzF2xr8FiDb{i1)VR5&D)>OoLs@H1x7A2ly>bQd$P>H_`TTnqQ?BxHColNt zbcPZ**W;dhv*M2dIyXIFj=qj9aoFyc9=W7rA`5EB%M`HS*Db72L_$%W9b;bOa);pd z9GA5tXOy!^aVOi^InAmsGE7&Ej@T__H%q`mYhAoZG(iY)q|44h>tXfCR9uRYxV220 zQx9?T^c}VzGbE7he-b^~5T7g?WGnE~O4H>2k=#FrfQtA-Bafr@(1vAs5gqNiZU5Qm z^pFpaD};%M0rdfcWKA-SiB+~eTEa`d#rj|)2F&?si(2msDISwV{5UejmggS9vBOd;6^m)2qNybnF1`nFF+gQ&? zFI0PZ9&Dc%{L=rx`ABn8V>l)0o>wC1OufDztX zOEJVG0$u%DF<0z-zf=0X{4z_&k{8i8#I3PJFJHYnZok!yrlSJhKLT8dwT5sZ(6sB2 zd}7lNu;sA{8h>NPT73ab_Wjq;p+H5rpYjRsm#hM&eoWp13bai7NaYE*_6}UnATtUu zw8Htn7SNzyGCkgCzQnxz#fPhnlNuw=Ks&XK*c-dW#4apoEz1sv_S}s}k>+K1KtKSc z2gO9NS-cwMnW)N7jnn-EIyo=i^5@om9hMGt-ZT{bl&58@q<4MBiN2jTo11PUOqqcs zOr9XUHb5)M{v?c9fK5j5pC8cta34tS)TWPSW@f2K>wwUk4jX@K>g0tiMe)l{9PTc^ zxkZ$+#?Usq&xm;nN%vI?h+eEcdf9CT7S{;Fi8BqZ{j(`GDeY<<#oRTwhYq4A!DLaw zFa){i|A>1PGDUmhBSzyplbEjmP9d@rRT0VX%6InHd-+wSyU+E{&fX*d6Xnqh1Z-k~ zzk&hR&OzRPCjKqH`4a?MCEq#t6?h=03NQ>cr5;99HYhe}$6qyv80A;+RiBqc12l5% zw{8UYT(c)%YDBXYDmRqK)}ba6+`vDT8vN&FHN{zbvBA_V_LafqSygq4qRCu{n7vcy z$4Y7)?@_Z9ql8zxK+ZtCOUC|ylBcu;GY1e#2A3rdl8}J7O{x}Ar87`7PiLW2Ex5A( zLFB!jqJboc$G80^A^UOs4uwtQJWldjQSjw z)w7m-?R*3*+r~xY1s_rlHCi>~}-ErX=HXzsEUJKvTX2sh}4%jXqL~!Z0&(ZbIK7 zrwWb{PZXq;Cm9*yP{4fOU;2fEO-x9s9S`=CK0T9$_j@5)xZiY!6h&U}ikf$Hv`6%> zXLe;K4isiVFQlS)Rwj=mbVp8O0%cf5Xjz;G4s_wV1ot294%kKG*rj8j@fC>%3cI$< z5~n0+>;!8EoKKPrA#V{AJ-Qh%umW{%92%DN+Vp`Rr9}xIk(^C1 z1^|t&h_>?M=jVSpM><=Nc?RPhB&DKXEEQbMcEdwo8^^;rm`*cgw`ns6>Ya**GV`D+ z2ELbQU`#Rll_eFLU$)>N!NpQ{5!xYG%1OLvYTlH_GHB{{n5j)FFzEH`!tIla%ZK78 zq;<4k5(Q@YP{^$(5y!!bm>Xck!P48~X69;b70qL@mNM{vy?pa#4-FL$q7_lVPzs*> zv=pDy9S@N_((m&a2m#SJQTXB|vKBdSrfol>6gERUDe>d2!S$*t#iFq=*tHR%1Sgtn zfOGry?c=B}y$;=)9#@68hD z-AC;vgLWc^l$*9gJ*=q808z!D2uGTbkKwMdR;To|laP688i7}wPi&vCIR2gb9q?4M zXw~KzViU2ci`TB@?MU}lTIv?9c6e{rb}%%5T_2TKFPJx&W+|F-;KW-Z>O)x$s1RYm zXxi=|aspys1PELow<48gAcViQnL3A}{x&GlaFMo%`PZQHzhKZ+J$u%_>TQD-ij=U} z6B-c1yamjY$W7RsWGfNL0**UHZORm&aw2x)kiRG@0BeTr%#m0WU}-|{sbXq2`ca4r z(SE!q5q0D=o1x(A&M*bxi|K=FDc^HAsQIA{ZTi~>>JoYb4ZPJ-q>zx|E4~5Q=4E?d z3*aT;pA@t#uK*GFAN~BkfO@L>-Q!;Jvf+}BgaU)7pI{sU)4u=~_>{KltcFJ1+%lk^ z3=U>mglJHa>0pbza7~z*Zre6q=%s!Wk-h`lM@+GG`Z)*Sk;Fnod%yZ_noK4na0!6i zIICHZq}GJWT{hFAe3TkIbh$)9KwEo+)25R!_2e>r8oqh8z?0GUl{x_|j113C2=G7`tJj4rtm^Z=CJsArmmb2!ra%rLnG}{vBel767B}$2RT;O3%?xobO3od541hjO-nv zg`=e7o;J_kf=;AQNK3?00M6daCe;e7+GjxiqQb)GI8dC%Nk9GB>!%7#bQ9 z8}1n9x3V>N-{%iT*&NVi^VztkK!vfcnQPS{8T?gjF=59W|2#S(U{>W26MIJZAmUjd zaMAE61IQ@I;d|srz}9?-9yR8_Uf5;o5WT~b@$lrU@7S-Wzhp*13LPN6FMH^b)QTqJ z*Sn4pI|&MPT&S1G;DAuih`5MMaH{ zXjcS@K+T!@6b`Qolt@oj2~nd1AB59T&$(W8eBUUtcNGYftWy@i(_ypk2jCU_6_+4| zE|AM6CVH@0A}9GHc!oL6UWR8TWcWRr5N{KA6cqEV=rVcnUxJqpA6|*y%|bc%S)+Nn z7ttlFm4yV-E9wpT;M=Z-w{hiiSJ<^eT+l#VgoGE95++ejfKXA1B#sE^3)gYttHESu zoJ`iB3=5#^-mL{rDhq)GaMgO@8e86?1K1Iv(y#~a_YohXLJj@GG5|S3M0;%iFIEA6 zGMuaMN`m|9_oEup4#JUx{tHO(+@IR+;`G)d#;z-pOS;GxM5JQ0?Dz;Er_L zyAU%N1fa+t;}AtZ2D6yQq_wL{S@Q?o;^1-cKgpMj~5>x@)Xv!W}_lY0q zB_$Oc6q#)AA3GE2Di-u|yO8@5$(8JA|Rxc}s)k}D z=YzYICVSxZdh_N0a@EtGxY`Uy2&-BsmepVR_;2k|SZ4XC>?2c_z?$0txOxyz&FLd7 zIq9CL7_iADMR68iT;~N^A`Rjmt$ymhF3Bb2&7*9P10mlIi~@b*!YheK5H-8Qh=&3^ zIw(QGQ=?aZ@~WW(<(iOt)qQZuqc?D-A&Dl2ZLP-J@U%2h1T3zERylJO$%ED`P;ikv z1^o_kv`r;>F(LOrvy9W%F&&+~0f_Xa3H6FEa&hIEzjM%$5OU%q@vx;U~N=iqK4 z&Cw{gBY{3fP!cQCcsnzT?U#xJyu63V`RSnISYBFGM7)8H(5nbxYC`3O1PDM_- z4($I+#JCIN2dUPmuSwb6`eU_wgtTMnr8=!2qM~ZoO1~v1lsHPYP3}A?KX?vB;!VS% zPVw64t%+8)*_>=3%<7}wuYEMZ^P#2Y4R{L?ufF(SUq7z-$ha?>^)QWfqS2h__qUt` zoIoTZ?35%LlxIkZ?PBKHl?bkRplL=Df8Eo02~VImCzg}%xU{%Hm^;Dzkm6+SN7Y0O z@`njou###QqCq#XiG*SAmX4{{5PL{5mzYA4`26ty?`_`e!H&e!2@6aYsr!=0McJ7Q z0U^4F;U$rV{xF{-X2;igI@=tQcDc85;M?aWPIrIL?_cOnq}B(Swqjm&sDw$N%)Z zh?yiQQf*T0);+1^@xqF=Zk3aai$7bhR?g1FUu#j?d+1*t!~JREx$gGL53p`0;n=|K z5b~cG_V;mrwdKH4L!8|lC&NNuNDjIt>{(=pd=N_ji0~lwnb^^MuN|;y!v5e7?!WLT zMD~8GFFe24<0XJ@3^2$8pcyC zbd)vzk51F)Oo~1GnNPXfK70tLkOo`q`<}_JdrQovDm-It--G+DV>nO8k;|9$sQUfO z`Lq$FI|#l|Qg=qcRk0OumwR6=OzW{Mpa{RA?zZdKt~=~sagN`X+}eziRo;-zI8=5r zwZR~zku0CgP<6eTMT&w-!q0Q0e^B+vMjM~lLPH>;TN}gl+nI`=SJYTRuy_*lk<#^0R zkNVgsOt&5w5Qaah$^$H~;R0FiM(FWVSD6`FOJzw%VD3+-ws0RV_nqU>V*Vg2=5{36 z#B)k410xS>XdWuqZ1CK_V8%NT(sd!7LUnjhM>(2Aixf_E8G#Wvi>sySK&EB8$TG zbc;emX3z`b+B?G{T5{BuL6*Z&B3?O_z$$=)2U5Q#pyVq`6M$Gp0tWf9EL4@uEX67Y z?zwE*`H|=UIdsSve~=inVaiUl>N~a0Do3k3G&{|Ym|N?yw;W4m`f+xq zF6z0e-cT`hc~yR!-L~h0^nwoTXKLI(svC+?XMTTa-r5nYxZ%Rd-Q+ep?l-dstQ&30X-h@-eLWuvyaNF6ytx})^z2wN;Bn}HG-7uuQEHT zv+j6c|8gv$;PRC_XTqb*Tl>RKh9w`Sca$$)JacCv&{{=FG?s~GU%}~KP49;8(bk9~ ziiw%qDZtxcGFR8qx>XVX&?Fe@<6r2zfvnKlE@+uA;1HpMq=W~44~Pv^uZp^~5{EBl z&9DW>EHIvYTWNAG>y`*1`oS4eBEdId*r8>5cnOTHh0I8Y(7ae_IODPD?okSyo{5N|=`UyK8XekwzPEdvZu9vL8DWQ*hpo+~ zf~O4FQw&yTua^$h4mC=QgqKk;!-&m?Z$O7kjgjN2^G9xqGdt z5%OvO$9omjP3?RgCp^!3OHS_=;CKC4ogZ{A;M13sHL5&)gNDJHu~xLX-^|<2PHwHe z+MQJ!I1ywR^dow3hiB@mbyqlExL)pMepF9yt+stW$mZK(zms*&Th~rW&H7WiM8e^<`2!4_#y}Lm^+XibxB6o%n4( z^S0p6C-a^P&-j1in)~NO$f=Pj3dHm0=7%QUMpxWCpsB1M+9CacRpXPNlKTDwvr5+4 zLt$S7C#d~i(kIk8{NjF4Uj`(^O*%?RRJ`v56Z~B{Ghk zO7k;wxo|(vg+XS9=tnv_PS*~3syYJfLwNl+Sk94gEO|#BGpjiw1&l#jdXXOD!okR- zh&p3rDpItkr>CP%9U#?+rMUHV{t4(#B`7*fJ}a2TpAO`?xDqo4+fc)AjTLnj4k6t` z$d4lAx-QXBSZ3X@i#KQLgb!Ecsc|xF1Qk-HTYd7w%Bb)15ktc3-)J`spI5q*yo1Th zdnhah`0A5P%&I8vAEI+H%(0~@>wUj@=c?X56J7?GAZIirXi9m(q*3_$O^pl9t&fdN zUDkLpts!VlC2C*mkE6{N>bX}A%$E9b8##aQzNflw<~ujuHo_6wcKg`UWQe0O7D0y? zr~l8c{nrV1y?ti*yjni7c$^MFUdBMd4&m}*wxY-qJ=28gclyN?gIlb<&+3{5m z#ZIIo##>!14(yL+35&sjucY~LgaqH1mZy1s$4)RTBss9H3EgYN(iuqmqG^g!)N6%% z1O?jdb<{W@S-N+O@jKulZ(%nq+nfGZu6Hsfx;8UE_@sKemydt47yDx>++>bTBA4_* zs`C}vZqV#&&PPW$2L?U}DX7maIAMfT5!96u%FW`R$$uZ+uKnSpHI*f&eWng+aYmIV zm+Tv8phUq$k)zc@>fC)wzl~nL7UrvdDGaR3Z|b@rn)Fq`M_f^%-$g2__UDc|rj~|@ z;dL5qn>0G8YZ4vL+{XxyJJ<$(eC0kLO2q9`#?Q>wjaL7l?K!ldxyjFLT$`rK;0}^^ZNDcWV%U%4h+0#u^Xf2b?`^*V3(GRuSl*Xr z97ulMn|Ad1N!^DU$w?>b7Z7*;CIT2}sE?BPA1oy^xNL4-TZ_m=IHymrc_;hZYGL|^ zBOl9pH`7uHXopL^o7>ruu}Hz~lTfrUpdE(R!_E`Fjr(5@1UIp+AGuICykV7^-;46f znj4D($`ZW`i3z=%hTF~F>Gpb$G^T#7MA2R*qJlMhd(}nfe0AhUWc$>{$xQ8XqM4$A zgw}J^9p|B|{wR3uCd?mZ-@{0@6x?v-kkjiSDMCiOkyb)mro_X58DlFvj84PIW|Vct z@Y#>|xk-KinbWLS0yvq(XcWcO%H%gBnK4ebY@8)7)!l{aCE@iFYdpSHj_1Bhe? zhiwqw(D{#kZ`ymWHgZ~7y)X+f_hh=U+JBycBh+Q1!9zJ^UZLcigCEKn{I17`H*YB9 z*=F$FSg+QY(!=_(^5~Vpv->V3o%d%IE`-@cdKb+j=xfWcF3+iQX_8xpDshVRTLU?l z0h@k;uj>=ciNL5Uj^E37%3gXGF2<@+VJgtyi>7glp3eQ^*oZGk%H0cEAQ-LR7p>B-b4FUucC8u_FDZFlVm3A zrB9JDaM3(%fe=rZX0qsHU4?phnA3jv zgsNT2kmXS}8Y+E0CZ_CoF|&zT%D^2d`&vVuSDH|{e!++S_Hw3E(TSDjaYf>MezKX> zOuQ#vm-3kF6w{`_9qEsr_;`HnkTX!To{RTccW?K((tN+Gqn~7iX>;@iW3JSh@K4Vl z85C43{%EM2mS0y9?=w0*U;ozMuP#77ijCGQqSmygH{|*oZJo{OS^27Tp7Oco0&ZWn zlyVppe>}!7&(HMLly_t@#JBZOx;Z|LtNdb}L29NGA5->r8%HI6Cg|NmlOc$Z4^}cM z_mAd2rrIlJLpsvXK#j(0q+JvTkuHqT^+H_Zq(>1wh*dd#R>L<)8HKnvo3A?X@pJFr z-}rWeNFO?{^w6L}W;5J^J_>AWo83>}p?4Fxnh2&y9i)d0KrH}P>=)?=DwXysoN||8 z!`-6~IAnPGWfBA`B@Jlqm{4XNQ|=Vaf2E!MO#4RV<=V$nk`f-@ zmQmgKx?mBcLO5PSK{VOwS6T0PYS(7N*;zaVH7ekyc?K0Wzgx8B^(JlaY18FVXGPS>R)gsn~=cl4m7GW|)mDB8~Qcyz;}fr1mq{!rF@D8^%g& zSj^hNd0F6tpWF6|FVB(hkyk z(0I$;KWcD0a(JhQoSL7U9+S^k2(A4F@NeQeLCg<38+XP6)mVO}gKK1Wx8X_pCD((S z@i2N}Tmqy^(NOPmT1OJ<40xZ_bV~33@ZE&pRQe@0=e;lesa{^aR0_KpyySO3=EOKG z3EMqPxu$|!Jn}KY{M^Ay0@6w?x*ClPm-=x!JZ@{K(e?N5;9=UxutQciQ)Mj&d(O30 z7dktAcWS<+Ui~hRwp_q+kP{|_Y04v|m$t6S3^O?(&q){YwCt`Fjhsb=fgtXth{7jb ztJzNVI_GuHa}AdcX&xml4}&DlieXyKHL@4Kv{MPjrSfJdxi&27p+WXR(MyHc+Sf;H zglX9r*2lVbM_glY$$3OoR|ZdG)APEjwn;r+#&vG&48PO--r)zhTIJN2v9w?&y846l zn^(yODs$|9Ot`({5Ke`k9kXgpod$+^(NV`G6ltd@M5#qq=GBQZVf37QZCzD-LT;CH zvgSdixAajtkFJNm7*(cUx@44G zq&J%c^Y5FPqKj6#7;kw4C9|QAj%eBJ>h2!0>q^ z3Kl*UW&0+u`Zu#zers-}Hdui%Mr^S$tVy-U)XqAf#);Mv6a@}Yv$HwwO|BLF5V4W( ziv1@8bHOv*92xOc#u-~#m;y#x27?v%JB;`V_?Fy?;~hBsa`RR7fqEziA5#zhP<@dW z4zr|6r^`!pXCkeWxqweF;_%-=Fxj$W0m1{b0#K-MF|5l|^)p9~GdOX$Uec zJg+|T<#}D_#0*zy`H|S#QzK(ERBpVCzgNjzTz@@D%}=;Cnft+p8{CUhx zoqN8^wj`14;OI<5k3(Ki?%Z#Vg#IbI&m{TOxzgA!>$UIp(rkOA^b5FMBMo6{6l%ph3gzRb? zgwbhj#315kk=!iBt-fo&P;M?%+SUcF;+a4uHGs7MtbH982%rVscP3W8$~ zi08OKBerw>FLG9CxUS?Ff%pfWLIn3g%~^9Jqi!g~4#88Q4I0S1_eTLRCKzM@jm!m+ z<^akMuvgi9w8`ukMo{Dt%4{H{2$V1IBLq1XblSASYDS>$d9P+C3E_yhD|(EdNB$+X9M|MlKD!iX0^^R2B7GsmHzzxY2h#vZTv zRaZ2fKhLZ$*uix738-n82_6wp6ALh&+uN_S{JDMq9D&U8nh$)@z1{EKKufLh?Kgu> zNlsP5BrUk|^g0x_H$OhXI8ufvUJpB!I~|X;f6?4s6}4z<_lP8k zxk^2}*R|iFMWFhY7txX?aT@l9ZBVh5i6ois3=P^3$7%QI%TN1~g{TGBf<{b^Zb>*v zOQvoK2RzW=+TiwHTK>k1>TcDbVj{TZ;H+Y$W~0tcf148m}#@7jsIX6L-Jh8ZhnDRQ8grd!$bqg@!J0-V~}R z?{;f(XvirmEx*v~Fu1x;c$q%+0-6EN%&WWXG}cy@s!|9(B|!!qCwoNP;KiEq{+UC8 zM=v+qNmQ1|t|g40r&kW`nhz(@b7!$KQq|8$%L!p-N=wPBq=a*8*)K9rL=cWgmH@d? zS3b^~=gR{%CT?od=5S1KmwFSGmffLCM53_uO;#rI^gfTsqWLcRIG!R?FULxKdY+q3 z=H2!86e|!!-eQmKEvm$r;wJ0Glit5YJn0tC)@R$N^4>p5pAKy8y3UJbxi_1Ql19W* z{1B?BbPMgWno9hY5qa;asVPH3zp-r+E44foXN{b|buEBUnBEF|fTZ^d!JX{m4>5jP zN~4=MM{vMCLEs|6+l8>&gkYu88%?uj7&o{A*4jXK|Jv~Dw)bs$SN8?ADfa5{32Ys4 zY8kW+`?LQM<_bDNsf2zW=rzx&xuLR^2}VJL>A8CE1v$X?+F%&06}|zxrM>v`sBbE3 zZf@?W(Z6b6Rv@xO?(mRQnJw6!|L{oP3BzNGNkkv0m%rIuIyqfvlDJUpb*Z_1Ev;;L zv8O?_*Fl7seqlteD@xw`6<5Rcz1`xVxknDPdgEPiHt295E2 zes8ImprB`}x9=ygyN=7|Y95W)n}X1Va+&||xy(c!`E{gnKTH)}MhnQ7=Q zi4T?4UN4#@SnkMqI?Hi9H8(Y}HH%Xxf>ZU}k9r21f)t@ZDsO42MypY0SJ`jD4`e;% z{|#m3XFq;|3E~!naXCXqN;wX*CbanQ>K+}nhS2E9fTMo=MbWBKXkb#Fr>Pk|q&_bl z2*FzJfgc_jR;igZvd)Sd=Yj~-vo;zU4p?ry1$mVC36J}x&Bdki1y`*~he{z-kmc^I zv(yVO>#MWddX-95kK|_BAUre!x{e?r=gB~N7t(fAoPQsS*lI>z^@KYc!cs1iv}N$~ zVV>?6*V~4;#p23(5G{OmKLy;M(obuzYY-XBCvlAX-zdh91P?1D@WT zJ|o2`MrV8f&uMo&1AcHux)t3=uUtLr{PN`G(Ee*UypEXH1j73F)stnAKdLnE{3VbmKg$}(I{^Cy`3D{yh( z45Z@xXsh(tK87SI^GTkF){AdPcxcbTRnQTCO6Gn#mck!B;0XQn z_d5+!A+hKD_s%}qL@a2fpcpmdxedxad>N=Upm2@gquX!KFrP}^s5P<9(%`2rs;rOt zUT&2qb1A4I5(=Mg!>oFU$^!Ar=lrK~QU@;%jhT<9Wu{HH?$l?%VR6z;yQIlqXWptG zJW>LX<<*rgwBURcolNiTmyHGDp91ka}=pB8LQI0J*7#C@8wP)5md<`wWvOi;Lt zKVPrC-mD`SW#@Nc_mxt3+Ye?5C;8GRm6T$2dg!XsZUfs?3O10GcWa1ATJk3Ha)P6~9fyNB1}BON}eFx5(9aFM1qXT^7()=c^0O z`)DJ_LGh!WT!Z&R6ROSa;S-dncK0%z9SNlv{Lt{5KIC(bDFjeumMa9s&*iiY8c z?iW5EYe=3eX*$^Q+K;^TE#<4_!wOODAaA?a1__u2QaRstv6tGSOzIGDAfCZTs_PezhXXPd+ zvAnD=<_r^mN%D0zTOaD}y;e-r8|chP=)+h>hXcyzM=IQvjR5RH*DWOk4xEoX`KY^` zz*mPye1z~yex9bK?9I!bge*ia-Xwxd#K`;wBXq9lyfi3;nw(l%#hGiwZrFsAmW;)A zzZi6tNLy-zL9co?i;_v>rk-Ujy`FHhIrWbgE)c;?d_RB2zknot~m}HsKIS(dRO;S(%Tw3I^bDrly2bO zakT9py#5aQa9@}{6%YAYz>!%Ybk4Qm-gAv;ne&KoFp~fU0AalU={CWQ-luDyiQuvU zb0!5qQIJ|S)85|(W@QcbBCQe-WrEs?Fh)U;)nGNa^fne5mo-zjq`=CFM@&qD%88(H zX(Vu8JIt~7fa~K$&`i$N-O2w=zx!Q)Xl1PW+PV};y{?xp;?}7}wCOVWO-HOtvV%MM zjkaA4-ntuaCD>Fh0F^!2^C`T2(cEg*z`4OgRIPcXjx9)^5`D=?`-~12{}UJoV>>$o z5ZElg>4GdzaOP9r_#a6D^?!oeQc>zevkBd~`W0$5I+ZDTTmEND%iW-zqoa%* z9v?RDP^s`>6KDAedsC$Y0`G;Qq@c?slL;EjFq^?lqlTdHc9b!Ky9P3Iqhwv;9doBr zu0*6EhF_9K2d22K$t5z$IO}Z{=JzYtZI`?LOWP zs(bS&4yd8Mx+s!xpIKfFxEUqum`8lke5#KL;~88g8@=Zg`MRYt(elsfl<9t)1JnIcLZJJ82Bcn`K$fD z%S}&poT8d#r7m~h+&dyl@R1=a9q(%HP8eA!Z05mU$V*@1zS67g64-ULKkC8$*{j8p z-e&xqke?byilXF3+~;k`%Vox*NX34<;?#n*8KP63PJB4tz8cerYb9-MS>v<@&r5{Zwa z62Y3F4!d>0H-X@d!0|!T8eN{>r*+}eDoRQY=uL4HgAZm7P#&FQa+Fm2(+4fBUon#` z!{X=DW?87CovxRkuuZsA{-=-OIJ{Wo!3w&c{Q|GXzyqfWy!_m4O2;@4&a*Ss7W?v_ zL+Nf}2+Sbt1xREzw6RGTbDj4(iuiDsAV2_Kk!c)TJ%ftFpL`VI zO+h)GN7oH#FN#HFhgkC(k_m*r`*y zmt;3J#nF*Pkwx;s-+mBrg}V^ix~I}4#r=_O@c;e|lHYz4p^Zj@*!D$zFnq0k{DJ(4 zB5|MFdVp&Imm-Dmy&9E_NDwf(ZOt>8O_r{@G=P0~4()4h1mOseURIduWlll0(^tA0 znK`{Cu@FsH3)g0ZcmFY#Wi1anWF&@I)#1Cp5oyL;9Z%OlPPm5C<^G6xRm( zLi&~^0H+h;m+OR51z_(@{H1Ckcqc|CxEIwP`MK$jA>hlKIrH^3qI%WiPS`{La;t|w zyHtaKj@=P-`vkv}*GG3;!?q1j>}Vu`WhGBo8-H#9qE}c@O9fLNl!(8C*&i@8AWUn3 zq%WTm4B;(-I z#bc1K_LerJIEV~*4*@7R1ft2TfJ6w<_SdQ<(o0fIsV?qB{GSvuSd^=ZnhI%7Q59?l z_pCO$pRm6Q zvN30>$y4W&JJhycR-HlEAXHU4u=0Z;VPI&~H`$@3U>XJS0=R7K_d5i1v5EfxV3~H) zIEp)Ecke;!8w)ZvHqw)&H`72~dXX4aU0G&;M4ux779Lym-oE#N-TUHln}6W^O&maz zbuv+1Lg%2aWn5h|HZs#Odgw%(m#<|Lo}1Gig^c!(3TOI_-oZQLe2x8e!9Ie}isx>d zj#o{irljwt;fCC$E9@o>S8(;!-3zS;exd-X=yxQ{gvn7*zw8;_Rz>&f9?+F7*wd)`tnQ-7bOllo1X!ViCY`osCAfk8CwpvR)%LPMAsPk zvrB6`+GuK09*Pc)2(VDEG~T+K__Ml{Bq0h;RDj@CrvpI<(&$&UvHZmPqb-*};bTMD zCs2JtL6MxRrobu4o=8h6(Eedez-f7mBsMjeL3<@69BN~reiQPK5c|@ufH~*r!Irs& zfwp5JWdL%2oortx*n@!dOo{ren$f{Ese|J`P^$+%xcvxt=|oVfzYJi76!-_6(Fk@~ zAfpKrA7OI53In6me~y2x!Po(eFC@aG9D)4%^zhh=VAK_mDcE;?WO$fIP%s*FSzj#W zoZin)A2;7;a*}p-nz9-Oo$HIfjx!xRTqP2Li~;_YO_LUy&29PZ58^yTPi-RxE&dZ% z1|@rJ#2Dos+DveW7K$ET+0OejyPTr|atV+-5mb9_BF0IpNi%h04aq_8P6zj#|B@pQ z4t)ggOM)3X&G5dPK4F^#sIdsPsW65Dy50ZLdwst4Ul2}#cgJxDr_qB_l@)Mc^9LmM z3Me*qU;*K&*zfgdnZ(Y7{uMa9fmlHFkerbZXDjlvv7uVrJ4;{a{}_(xRMp2#Iel`k zlGlHVjE<39kl1Vn+*NPI!X;c8T*P_U6Jda2IuHe2rUZg&YNo?)t_tZWk$2EX=25hF z=nbw#}m0R3)KOcJWfYX32VZMj2oe52l+q8F;|q zcL9ryw=Pzc^RR0Lv&GiZ+z zu)olWO8aYH>PLL#Vz;?hSBKv=it4d_7GQ+%!T>Wg>-BoBbi*qg5Hf8eTNy}f04EBU zD`CVhdZCnx0YihDo$ zhV-?+9o608`X&9|O)rGbL6-%Bk;nF$94S?kuoLfk?JeM9FM51MP?12C%qDnnw+Iap z!~ha(Y=Z`ZwznS}W`VcQEDFROMrQMm=JAEEy2H&2l~sA4>POYuhMj^Bcm!`kGzUej zenmeQS^2`7RkNlKUQ8mc40XZQ+ay6BstAc0f$L0EFN{!ec(OzQXM{wD)GDTz_XS^7 zHxJOxP06+sDpTc}L2PSc9fqH1xi6PU*n`2tKl3K80EdU(ZwgP0mQK3^2|oCbv#ci^ zPldRRq1t(IfWn)q&I9nSQrugd*d!xe{o6~Dh2G=x)mQg;>UIK`=lJdJH&LN?+zY!O z*_PixD3 zp(0=Uv!+OOpv?1!2&QNS+4T2CaV+fl`MPMA=SrAMg<(wf|0O|4BmXN<%VD;gFnvt0 zb%E_E-+<&O?s)kil`?_mAb68ta;Bdz=Qlls)cCJ!EN6o8sWJBfE;0v6R^R&%pj87( z5*n_jW;2W54`cezhk~AkS{{)QRh|Qih4`qRkDgYHU+5_#5gqOxQ%$ya8m`bwWRO6h zxI&TiT%WM@-CaUZ-L_&}DR$biXM5iXwe?n{s*>mGC87ipVAG-8g!>){Pl;UJ2sWn>jym+p_kk{HEsH|CNNUd;s`QQ*FmL1b7W_lcu z(KPv09_kA~h-3=V9c0JL<0kZ~xAQ3XDEu49qv)M509UomvO?_(Af2J5P4qmTk3M0! z){;S5`qP^K@q}BGyDXGrb(W*8UGlg`B;QC?2;?d?+p3u~;K+cdt?$d7Jm!|OarDB! zA84wMqxdm2?C;oOdE<1A*dyB-lR`R;JsKrJ_HgWRHq0*<$|MC)sAp$wSxhzMS_6$`B6Qut&9Wwu*E} z(24b6h4#J03Ve|^_;zunXuyG*Kz1z7uW~mx5}Fmy1dsdkBCddsL7@A!G1z>AYY^;I z>eXN+j`+!v5CSbUG$Nu&nK!sK{TxK)x~OZ!CO(kou}FWyaNm63bi>l+a<_fflaHQ( zUd}K&Z?=NkhpR5bBNb#>(Uf-Xk;)bUp7F9 z3hHZ4t16}{^AXtg(@U5H0&QaE4t=Doy|>-vhXzkCeD?m(cG+5=+sZ)vYjL_F-MjW4 zmtel>#>QXT_f9}h%)7@8(2a5Xla@AS&>CL$RGe_5M6M_|w`JN-SPr)YcqS}8jb;P**m`j_Dd2p`ySyYCcT)9=Zvbk%R`i`6V_QcN(9d!iEP)g?F64oP>~b6|y{ zv_}59Qp_Opjf&~=B(btmIEVq>g)#*M8s#2r0A4}7W0G0YZLE`Z#91)K)sHK6H883c z6-ChCn@IyIE{8@sp>Z~aHXQ}w1>(@!!ym!VWsM1R{zV#A3!$smA%@|6hI2Rg4|^X34>On$mE#f6UD!9|C=*#Fs6Q}y ztVGMt6jy)~oH+nl$gmHc{sxYwN`H{xh5i0onE2Lsusf_)nBO+X)fyxgi-OxCPlkBt=BbXHc3rKK|*$_2~2`?^~8H~Jb zJq+uCzzrf|<)Ql-rGjaM+qNwEBg0WuJ$r`jyfTjgpI;&Z2pa61A3f8bcUKn)gIlzM zIDbl}2Lk8(n(r|=O<7LS&M{QKlxamnISYMi(NK^=weQ7Pj3H{_ntQuw>v^A-IWmc9 zevu<&{hd%()0y|WUrb*q7ir0_YnPH^BL_y~Xy2li_S$oFdTBcM=9UaB>*!QFtv zvM!hsr0@!yC5;lO_DEN)Vpn(i%w;Q6RpNMa!hq7>V( zpM_^zstoLB{T7xth_016OPrvh^Bcxz7Hlv26& zd$$+Y3?kxCD9bDj7d^#SHeculYTKKeS?QbQHh zPD0ad=r}fe!|AR&w1)2oR$OI0TNMfWJ!e_gv@?;T9r^!_F^3kP!Ez z4>8vp+NT1CXDDp(Vgd#dvRcPoU8km^U3ZD-jOSLGOcW6l6s4YCb?5Ae43-78f%D-zQTs?eFuS)8n9mK&$&gVzg zBjxVHqqP;vK~PRyNN{?YhPM%{F^QT~hYuEW!v3V2Hl&LaYFiw$VaGfbN!QVzOm@X> z3SuBp(eF@dZh6!%0PoL0iDZy{lD2||*d1@JJv5z}PK~Yv+P1qR&Mvj_r0o}oKkFb>!*1r(O zSE$F0M~`%^X75)VKMlS)aox%O?Mn0g40<%F+j9xB@EMGE4fzRDinGr_#8f^B;M~(? zc*IT8Fk_+>mHEA3-t-7ZPOED{)l{Eo=Mv?umDx1xU=pqAv0kI!&kG2e?=YKj3iLrR z!x$(W`9X;wJlF`(h6OACMd$L16qr!-v*7PYsOi{|`#+{N4LItcl0EPY1^;gAZg8hK z1(zEb4zzMj32-IAO4!<+0!9t!|Nh8(s7Lj0-QgDSm8%rmlkGp-1`VCBbRv>kgO`!{ zD;-&cubd~abbty?*BX~hKT!_{B*IfWULz4%2saNSV-p3=QKn$>LeD0YpT7K9wXK6g zKm!wO#?W-rXliTuZ2kLAStX$T2JJop&hedth7lxR%y(B%OU{B~An6hCPU4CiMvn9- z05}Q<_WRh)fg5@{-a#aC`5ADsL!Sq_+vqWEW$%o3bzjFSXDTSqeEGGbdiCMv2ClzA z6VowgKj9>a5E9iTZUdnY^o{Vg0BZ#G#8ii3s{(p&9AFzjRe+2wSy0!raqL>{`>-nx zg1Hw4m5PPG)diilok?3PR0V3!Kk%baagM&>a2y>t;tedX{h|(C8j-b|!yC7!+ zV++6rf!Z7jT*zk@x&rWwrIlqR@aA$aU?bVdq5#)YhEjRr1DZ2QP|kcBUq2EutQ82l z-bp;gA_XzD_CPg)G`RYsGwU>~8XQ5~0loOgE#1(+r+|V2_-JIs z%X-(cdvy=Hs2^~_t;tHdlS(ECnQ1-)a(qCy`<$f=fgggBi@=byWI`PamDBaGWZ-Q+ zaw^RXG^KdpZ6NIe{8l?(`ShvaD=s~CBHQQB;eG=60Jj_tf`(@xmFNmwxh+v^1Le9r)Dno?`QzW=1VkL@Xl-$B0T z+FxAb%W@9ofUn*m%oRD_7n9n`Tdwczh}ZmR5FKw~(%jJlGcwW^KP|sx&0oF#P_*`r z&B~3&XTtwT<|;S_7Rc3Yf_u{4OY;;mE?Mi}TDy44^Os1$gMk~4+F4%D`ccwyvD;m6 z(1R}ttURV(h?IL<1??+1UP^$4@&At4ktPt5Z7;#Pi?F%|_}u5v=tdL#jXGX-`PZth z{JXt7Mij_~|N9S@v(vP5@71raU)N<-qKu0p7==w^Vv5(|;xAsnq^y?>naT<^m}(oR zTzfj&V0g{>`u*Me&(@^>{B51FjefTFdtbpwqY}q8J$uY~9dCu}8h3kL^XvEloPY$2 zrN=Ga?QNf>*=1B%)XDhx*~>W$wrr3;F|qms`aU;!GSyT0-#^QpX%tcF32Dw^P?4%#Sv1Qtp_%9V9%M>i8^3BgSvlDy^0dlRPr9c`$Sv$>oD%sM}9gpSr`e z(r&vYDz@<~ogdxpix-;8(C`f1tuKW!-AnmzUCVZ}(m$x93 z@yE|2BD)t=((}5^^-}US$^7iM`aIVZmFtyQ=&9H==^Nz~m+QoTJev^K!Q=)lUyE3K zD_z`m|7pAiTD>QJfPtj9Zr$*JIMnxnhU5*I{BJdd(;N4yzg7A^ijtDjyYRAXD#we5 z<(3RqlSM04A+vscMep9u4%WFxrAfKuDtC5_1j1nGDpSDZJJ~k))6LAD_S~i zlX~1nXoh=Xe*llpUGbOL@6HP1#kg3y!TZ`d^SD+6@aAGWFl%Y0yZ@o+$kmCGLFf2~^55_^Oz>JXf zEkD9h#c4^M(yrI2kSd_HWc~FT^Nai918GW^)z^Y=!(WTn9sj#f-hSI075|KfER_*I)x31}u2RqYK68cXzeHECExcOv+r#I~lU|R7RlTw-xgMG$z3fA} zIQz~6SM#~-sb)wy8(qeU)u;URPf~~SeRJ3}RY%Ab6f{H_nz8>rNFx>DXB2dwyXQP9l=aLQ?KGQ2*l>PiZo&?V$gbG|smlY!T0FS|nfj7H`(*e}{;teA zCN8122AwlKsIWx0d>m$FqUbQz9`85yvvqaH_ckTj4N#48KaXImF)wS<@|p0ZtdNRc zd)b2s^-ND;Wf6MUSJqAi)_j}^w0jY&YIhsA0)J?0nz~%5N{36vPHO1&_! zLhhhCSnnC1JfPC{mEy&!Rd`H69H*nDYqe>=U&Q)Gdr!KDJaxUuZ%3Kbh%aiPYwxwb zjs*VeqG*d3>kA^qgtzSOZ#_YViT6wd$}OC3UZH(2_3uH$#>?H&?fqy zzO0WZ7amP7|NHCfh;;o1|FFeV7)%k-F?H9&(9xxI1@GG6&Ki{J4;1n{8u4pkOsg2k z@kqt1UNY%#{31F!H`K(#V|)_TDIKw7`>um`Ge%M)R@z_EOB7RB!XqyI4D3g^8mqA6 zzb&#fDqnuG&Xd7!mCxDrH>S)QG3ZBzS2^4|=qresEo0B{YA`G_DVNT={ME8_D*J6w zzcP`Sl?_d#AtNh!=&x3{oYq7g39+Mu(;r{|?!5dksz;q{q?tTA@ZbGnH@ZDyhmm@6 zHo0x@=N3{bh0{FG^&~kBvMy0bC^)X`uD*XNO!a=4IKezx-=NP<%yiS)gxng}FBLf< z|Bd~9tca5p6I`lU$7kvldCs20bM-G|evJavt^R^2E8! zqjGv;jnm=E>Pi^Mu?ix{`R@O^_v3Q&B3Q($rMUd7?H+NFcy`1^mp~DDhTs6v?DgXf z5lt=5_I_P2wfwhtcVGS3eGRv3Sg+W_J&I=VU87by@%{Ac)h`KY>3-lmfUG@TBhN(h z;k0MtflP#^v2gk+h%Gf0UMG=#uFqF0evSz@)Fq@`nErS4<_-;;c5K;Si1q#bpWRRSy+_%_q>Djoa5SxaKvUe@&;1IhdX=M@u6 zCC654uL-VL?}a6l?)5a@74*X*t6qle?~ML#l!|bos+Y}hbH*7D72Cb<^@vc3kUBrj zAZla%ofV6Bx02a+%{A2Q4ct3B**`d(iqtegM+7d}KaGgH)0{QfC_<+sqJ=8gRzNre z2}k3vC22SlLP(H!bW9hqE?uiYsaMRO0 zXj$2dFy$RUDF}G3Q>ydkbQpUWmq}w)c|wSml10B>Sudddk)E@@CLMs~V3%Pr&~;1^ zzq6X-d-^`zL_!cVE->pIg@c)i;vlL?Ul^uWH-j0pkUj0RL@sFb#SmBKl?WHh;!lh{ z-%tt#D}gF&S1!87T@zqfGCr@rlf27$*^I5viH=ecSux3%sdewqfXu?Di_v z(l5c4OtR4?sj(&w=!(RJ%0w1KXNi6h!XbM2@K0Rizr5En@MYZj9w4bNBFw&-hdT4lk!ewdp=bYaHF$k5xqer$=Zv$8{co;{Jf^ zBPvDjf*p;^S~^8niF1k_Et_LvWmEz}9>BrEqxa)iK*y!!TAd~mH?db%Tk>g;hs!*3 z)O9K=L@1)~HZ1h-SNpgMuT?#dLD>b6$7Su-`S!|Je|}AMNFB(({*we=rC%vAUG#NC z+?ViRDP19c*9FJ8%LLhM=&#vJ-Acj;pSpe~u)6kzQOu(ULxiQqq#OYgmt=0e!ABEn zyabuK7-%^20R(306(=~<0t5yohfaE0T+l~b$T&n2dabLe@hYEH*w;cIg+iqD*?zQ~~J zyC=8jV@g?5k1GF#j0f#}w;6>0@|TS#Pi217T?=x>_!M0a$+=+r`8(JRj`sAN0msK} zC9emz_&mP>*3g|h&z(Tj%KNo2W{)$G711l0f_eO4=$ZnWQMu}))KF{0Yhzu3VQ- zRO^9kYO2=VC~{;=y35<9n~M5i@0kPon))*0VPrf~L&>rRv+(Ji`{1sf&1yP{0Ek+6UvyN*9&&coJR_2` z0q^G-=tV`_bkDs>lHENZc}edl*LA%=lteAX60XjVykC=%BHQ{j^Fc9#yp+*S@Iohl zi$KN;X;`_-7x&C-DLzS~_aSj-%$ch~xZF|3ACC?xwKNyoojYZJ_K)9gQR?$!$Nt-j z2@Wzd@>OU##>_vgHdRw`6wY)N&MltYHt`8W-92_TCjWfl6Q0H`Fr+o{LkXK#qeLv` z+uCb|o$ZRuaV3O7*R=VqJ(wB012(ijim!H|v$I)YktPRWXWdQb>PNWih1eK^dpS5@ zrh)Z+s@Q`K9l$9cehcS1e78&kzgC2dK6+&Kq}UC{vvbA-xF3E+M0J_2 zta~i;{%EsL)#NWuy9!ga~j@0J}l8x_k zp84ija&~&jpT59N>RQj0e0fcLMcNE*F>>e6%l)F!U!47AdUt+|U4^|bYp`vWrWP#U z=#pwq?n^F)8C01&fr9B|zb1^zdf@*F#TwbiD0W)fZ(^(P6DKXPrM8R-c4%8CtyF;L zzW86Wi}7skkWGxyma&_{u zHjuW$v02((Ku;DW>re=8^zo;~o0K)h1nax#nMrufr&meX_N%2{ za8?uGj592g_G6FEE@OwT$yDKM;H=!U`I(~G9Za-JX{^e8oGKG3F+ZY8bg8!6S4sdO zV|YGNyQ<$_7PGJ|VKamglDrZrQaBtDEYcI&CJ<8A+f&sQti)89ar*Uhiwu72L6>S% z7pZ6B5|`~3M_?oXWHXt=yg=w-rJBtZz5#inyDKY|whu=0CRtZ6#HWK5# zh*WrsQs}Pt=juy4$_)!H;)t|(jr9X0X{!rbb}Lxw$?CQCJy&Wrxj)@+!7@O_ zoQ1M~Dg3uFTuivwt*yu&5;z9*eIB_`2)>Spne%RJcb+6y$r(WLS`VUwpPF~H1pj>! zAQ@F$URvIhx}i{*a;OkNm4BgMq54wa(Gn`&ZT^{u;`sX_kMDhU&v|o0TprT>OOi`q zKoeI~H6wD0>ma8r$x?%H#^+0tCT~1)jm!GLxVHD_PpL3i6-h9QzL>5pMi)nP+waDy z&J8`EWoQh)|Lc4wAMok*!ZT}vbMzNMr6usBUQ5)d=|e;q4zNN*?uX=SH;ob**ywb~ zC~xSQl$n;!XrDCwN5$2}1VBoSMB=6Dx{lc+qeh$-w()7)x!H&;=yn)9rcEgkl<81D706TPibKl}0 zpI8v$wKX;vk}9|he6}?5XvUhRqdssmQ62B}w;7+e%ou)2FD0*Ns#Y+E`n}MJjZywE z0Ud?3*Z~W3aY;RIP7thgkaZlmBQ2_~veuww{+$(5wm#a=llvKP$hMP$V|y zF=hzRm+!WWfrShGjk@ItbbBa^n2IRNt?=9=I-7yea9Rx-*wu};OEoMV}+aLM+^O3 z-O1g)5h7a9r;5xEG;91MQT3d6ow*;%izqNHOB;DPp{jy>SCrOTO}k>IRxpEFIRm-V zG8KBwCy9NyIsU|6M!)1Nz6@4 zGqX2pX6Hlx@OV_#k>s1FLHA8fH6*s{`C>&DmGf0Y_n)2YcH^Wh>}-c+s(~n(Tv!n$ zD&~>#2q!~W!~<&`z7W2_$axiPi@w5rE=rKa!;EzRHzO(W3Dvw;3&#jZ&Rt!j2zBmN z{?F+4x_7recd`;Te}p~UJk2CLE?J4UnDGIR5=gCxv=WIamW^-jtvi#LV$5sybTYbY z(JLQgy>6M38WhkAV7_!tosazb#nQ2W9Xrv5%8A0aHwuiWNxGtPZ@$R6xHWv4#UcE# zQOi$eP?Rn7se9QNRN3Q%VKQvUt!?-btJ0NZ4OLbNj`w1(?(E7YxPL5BPLqvH9in~ z@zTdZ>V%+7OM}i$swg1tW_a-K#03=9)jNkv|*Di@NPw|PG-oBfhD zTiUEhiKQS~swWD%8&QPNM!<2_E6rQKR+?;X{$$qZAzAoz3Ky82{T9H(Y%!dlO|*)P zUXf^ZMb=0RuHLiQA7cXk9)D^#kd*xBHvch(BD`$1uM7Zq4d{hZoNYlrK$NvQcERAyKIg-JqEjTKo*JI4g4?xHS=Omuq00<-jvPD z7%f}nouneF+BWm1*^5yPj*Ne{a}c|;OuTMwBs<#Orm{8~&uDF3+LMb+-ChXC?nXV! ze*1psASJn0Bv-pD(k+Ma4rJoM2c&^rvwM7HREkK6=X+|at{wQn60z8|Sdo?Xp1|CA z4I{I3tHMamb15VI5H+mv>OInIaqqX|j@w!6p6L%D*+b-%3m4Q^<8>3O$KBo_+xFj( zrFOK;D4mfXW$Kp`JvUH}v434k(R!6dQbKRT`27zAcy;!tV+FfTj`j!B+vgGIOuC~Lk)BC?}ZxzHroiceDtRP-vco)ivqpZ z>(93x93CSj2js6VL_&a)sTskj$%)&N0OOC#>p{=>mMGy zg2N4*k>A>f>SZ;r28A>Z2=@==8k(KA!9f5|RT1;pIQ_})(!5~8PfxDBf4lWZ_h+8F z^&&nBsxrB&`MaTlI+>z>l>R8vf3qWOOzqgJrywii%fAJLw}5Qt|{&=!A9NXF`tJ&$~iY)uiET}Fn>P&^a~u36kB15f~8Y}wCxQ3 z!SVd(;#p7ssmxcxyw7du7`kF&n0B1*tK=6%vm`_o)+w z{jBubtx+SNpjxk7tc7g2Q;PVXB{E#3g_LfM5_hit5yvYYz5Y556`lkbR*|9?NRKh{ z31qp~<0DDOx2)zbW_$Yst;bSunAgR6k#@I^2Li>B_}Cp=#g{31f{LVGq)UCKbzae) za*ub&JMR=BBCT`$t9US3Hp~h*d}y1#0>afXCti@rq^)uNpIX4QzOIh-k3Q!h?Y_|5 z;z&2kQKlbR#cFMzkSQ+CmOYK0uLoPPn}@uPKHMJ{xg(DI^*^b#wBg&Ke{vs_*+YF% z`&FVUd~v39#&bnZr80gH6OoX77j@&ib%Uh=>fK}hnJdfa-wvvR5c#OUQ|KiWCH+I@ zp8z}7R`ZR-bxku!!AJG%#ZbN2T+%>V`rddd(t-xNK|UBPm;Pd5gZ%0-N<{9NOSXB( zpsA={6ZQ;J@W=9glAr+48nq6GdoyuoLZs*qIF`$TpQ89jt;@8p=VFX~QohZ&?cG4O z&An;tKl*h<$M(^k_p7CM|w?v3sKoW1o%VUu7dvbN}Z0cCb04Mr1Yku(1G=%5}w|9CVf4 z_mi)O_n>uT)cWz-ob!a0Iok2d^lzU-9B^p0fWy?ndN-3Q^J|ds%=b0_-wsjvKd{C= z-52b$U`@CtKfP409@qmnmunpP%sQv(5%w&60LRu=SUhV18%*qG@cAK|WrQv4p=(cf z>t1G+;JDsz9-L6m8tWlHie@u;o0z5iNnAiGXm?EVrPJ!`YGr8lJ0W{|Y_elD+?qQg zq^OwTApEUZns56#ZZSQpW(r^)dirK3ILE!(FsAzvbp^g+L;#*x@Q*q>8hbTjrOpoe zq1AfVc@q4qh6AqSDazwSKmEP&b5k_73v+B?=+WrMt`XxIDYEZMYG%X|ua4{`>D}vN zHf9zM9On_9MBRm#@bF_3#Y+YWxvtY8Lj1fPc+M5^p;YKx9V#oNK~I=fureB6Ys!V$JNK&@e?YbRP(KLXa5vnl0_iWXn@@<_n91=Q$d3$+H_tuYH`kDTcB~4A|E~2}9%3HBj$4kVf978*5bvD@R;q8L7Q^9z}~^W}6Qha_I|c zky8u_=K<8Y9j@@s0XC3o(C?zN3DBU3D(Vo;-m~;rvT1cTUkOSM^3MBKeEG4HJHfx+ z-I*qO)sW4+ePt>6`JqZzjETVno+itJiO1|hb_}4eLcyZ=O6EEnXmF&d%*^UU{Op%D zKU?>o&$Jt$y87{3(%+a6a044H;*{Z$ zk!On)=tP?!|0y50gR%6;n}$2clt(Fy|y3?U+2l8U61N+aFK(A}Ox zJ2X;K3Ic-C(j^TdAdZwnBi-G@Z;$uBcfI%hum7yYI%jb_$jtLR-@W&z_S;;xh-uFV z(*(<*^-M2y^+p$jskTs<3GpN6UGrzR8(YYHTJsoUL!>(_p1sbyS~xBsD;_Hu!$eOY z+w?8Y+BC*ts`ygD(rtQTLTcw~A2@>gWzfUM*Q7PPe|zJ?h#yl1!W29?CFY zh`8JG@TFt^Lg7QUTg{(QQOG_ZYPk@RgjQ6UKo5w1h0&|qNl=#ILU;MRysev;8r3~G zD}f`Mxv!Fd4m%b(nq3zv2BL=0S`Z9&ed?9v^a& zc5gPfLb#*kw;S+MHVi>M_IwQQb8$X^EwnIPhz$)bu!dm;_52${!Nct%;eswpQDU*i zR#CWn0hp=lH>%fVycKCQPhWCpvQpq4QCCT^4Yd+^^jG5((qF9Whdw{qHbzx9JbKhtPY?pV^Ui4V(g~>#`;iB%)N@tfr~~&BD-k_Z4f|xTl{9 zmihRI&-nYo)x@N)_&V&~RGFO%?KZSyfZ+nu_M%RjjW|c*=HXc%?gS%a9ifOvv(gb; zeeN*#W(}Y)bRmxrrL8HPQ=IOpJ;jzcyIi@Hx*{3d(Ps@r$>^&?z3SySPL&4V{50+Nb+4xP;%_Q9DhuW(Xk6<@5N-=6_y(RDbZo zqQ_d$ISs+Om;F{w-$vNs>;R}^in`bjzb=VKd<7cPDJu-bk0@m4M2OXVb$1>AiJ(?;SpJ9`)yrek`%>|G$QQ6Nr zvTlX7WmuQjZr*oL>2M{^u;8I><-1o*tFO6YDN0X^E_zas87v7ruWrDD;gzdjT%wUl zla@{6=H`_J3Pazm_6|S)63gDwPD?MGs4l21Z|q3dI}0>Wsp@@G?QC7^OKvs-;a1u$ z85dIZ(*e+m(;|OaB`hsDu?~`N>|Iz$pZ4b8Wr()8tPQ7R2|I5wyK^sn4W5x^ zg4V7;pva~@(nnzX2B|`eq3jiZTe}c+$c%<*(z7z<++SZD5ayS)Gl(0#g()rz)kS#Ocn+@^_W8!B)1uHx`~N9^fYI{KHWY%!7Q@aMH`j(#e@C|(_0 zffiFbJ{nnD_TmR!*atz8ju>{t+Nk)OK|0j%wRjs+0cZBEY#qC3`8N5s3^;ZejLxY) zy<|mIJ`K0N_u%TXT7*W8f+z)yLEKcyz5xQ4=DdN|qYZyz=MvK1BIcmya5Hmwk)UaT z`Uf9L@ugh%F!LN)KGc38fQfbfXGu;PfhvEa#xQ%ooK!BxUW#*y;oz z=s<&LC1YCNU1)a@lNG6TSP|s9c}3q*Zko=FJ7jj|RT~yndbN3S9Y5@>;Tx>pVDW8w z*CC^ej#UWQBagsfFtvMiYcr6BIstJovtVhu4^qgTalc~x-9E@Ye$eAK1c5R8jn{p@ zaEjPyEz525 z7zag+alLM{1L8s^H%7H-gH<;_X6*;NDF);VA6|ugQ6~?S8!{|V11KA_a<#)Oh5%5b zKdOfFrLCgC?(6$~QRC=t;Ul!lyTLD}`Fm>b4?++t$q_zX>FhPNv|$;b%iMXnu!|ox z|4n&j-uVcCVlwRux+KM{c^O5}iD9VsIYvU>L6Ny+l~>8p%Z-Q(g0|^f!V>(+N#os& zY^S96YqP0<1+4B624B^MPoK#^%}qDKyavs#AZhA~h+3b@Qw5b2(9eA#^q%HsN$z}8 zH9^yTTqW!+hmZ3^^H>4GvfE`NC}GE}?j~`V2F4BH;r-K1Q)B27Lfs`lWamoWb`5;+ zse`8u_Y%-+-Kron>^H@kfJ+epHdr`hFAtN!F&{$QMmnY=IYg#ffMeOJ(=CqG&m+Kn8FL6>#)!BEvYVUS0*np`zbJVOnk&%knVRGih zIr$hlCKJmyi!0=W+A)^pUU1vptX{2|dU$CmRiCb%T<-^ZbtKg6T?bG3J;WgTX&*i3 zn;Hp3bXfb%E3z*u-AE+a4W8y14-3S3qC+VA^$Pft?#*7JzOZGsq0QD$1qTTnOfqsN zLR7qn@T#rd=T!$4gI;1C{EA)7v9Hd3Uv}=yCIQ}p&4s?YC(atE;HdQg+hp?^S>anb z1D3JMZJo&JFs0aM>Ii|v#Oz91rOeSK#`kUsT0d{2=F^7QCxx#iIg>~~jWSEp;`?SO zgxB=wb^KTXOz#5j3@N6hBbzj%f%;}ds|7Ma9$I29C3K);>V~@%JJoJY#_3ZEy*7gC z0$!?cn0WuH6^9{|?s(!Ns(jcS;j`W6J#mh10jUQ2gMFC)=ugnfL0F0demV$qzo2l+>u+>?fwfPD|Ck%TBoi z>!W4Rr)VLUiAMYop!13meiFEOJh&&nx?--mxzMjg^ZFH|MKs<38a+UQN}pM zBo3jZ*k;mqX1S7M#goG z=2LIva)x#YF_`tkGCD+t>mJtN9M%C;Lvl>U$SJ%?X>@^}-slZQ#9f9Ce)SD0`y{}k z)OdQ}qTh=AzA3c`Vk6M~+1gFWM}J?rF6CUl5+(2P4^lD@-b5G+b6(zIf%y%GU#i1= z;Tj1^x$J?uK~;|7&@|;>S!l1Giwcn@v|@{2nQ$jwxIHqtU>b%WirL>;vBh=2U)FTj6%mP2nP5TKmwNPA#2PhY>=U$|Ku=@WLeTGzf*NwF~k z9f5Y?-u-kX!ZYR1SZ<*~@d14-erQ~-`klPOfl<|dvqGC~6|LOnAlfKOj0Um|m@!j% zDHm#xg$M#<_#&JlFXL^nB%clqF#k}FK_JRZo-?-ioh~eK7qOV7<3F6?ehNTgHd|Ob z>FHa4+8-TRv1*G-Wa^cXt&I^syAHKygd*gKUnb1(U32%r2MTUNV-xKy-m4@y+EP)! za}O3pp2h192cFxBsj1I^m}myTY*P>z>h(|K=Z9k~bWJSlu&p);&kb{WPZIqfj^|%o zYtiHT6EDvR!60&f{;j`$9C3g4o%0o%U#(Jl=K#2R-DTDgJ0QJXQ(X|!)#|h#_fZKB z7U)XHJ+!}BIOrv4RLHWvCmwL|^nbUed~YyW(M;3zyT%J8;6f)D=%3~iJ-iz1yb03FktS93%cJ?o+Qh3#4+AFE1k1e7iyS8T@{ za5#G@n1h9=Vnbx_czK<9i&=u!=WxvBO}713xv-dSyEG=?ZC!?1)-TJ@0x;g@mrl{p zeoy2alwx-hU971t%{^ak!>C|HZ6^~b{1?|l7{Un2rs;QpK}n)GQ4u*7Zi_P)9{AsOMA|B#24 z5p}7eGUS$E5lyfLEIOJA(O-oDy#nMD=!^~?z^UF!mG(cp9_nK2c0}-#%4Rz%D5C=^ zHk3)C{tHH&t|AHYBm*uwsT|O4xwDu2C>vkGEbN>JZ7LV44*}eV1NxeiM#(1dt9}!UjVTIfz7I z9Hdvu99!m!Q)28QfiCsWWZi}r01(1A>Kbo!QLIkD@hd3+3NwZLVT$ga0yHvor~5sL z^WnKGdKO%WzJ=>>{C6Oy=RHGkhQHAZ1aKyg83xe)kZN$Mk^!-G@?+p&f!?i;_3Wd! z)-I=`)U|Vg7ICONMudaxIyljAAgNc{B*k zPhx!KAWS{SkPI(Y=|W^7OGJb;B{g^V`3>>GWzY0wzrCc13OR9&l0L$Xe>3 zl5saIRVymSqS*dsmlrBPn>GRMU zDd>_xu5QSB-ma@$z9myD9$M(iO3+g=XLqV`_}kPUr$0p>+labAJHu}LhIt(BYdqSy z|7)A3BjVe@1gEiJ7HSe#BD_c(ts`VUHU@BYI3*!{1X z^-XtjQktS=X{++u|F`Iy=0*r!j`J}^vlP3ZPOPGrt9EnmsD7R;C8z1+6>`d)T)V9m zxapwjV$Lzpc=;~pPYVr|`=7sA9Wfm$3YP^YBLJ0B%k&Wr94n{Vl2T7NWggWOpv{J% z!yWP53QsvYDNRZ?xJ@Ep$tNOIpdgU?R^lc!lkooniveu}$RCFVDsVWrq4I86H2E0X zMUz+Uv4nj^^wmDZdY=HfkSA$!%k1Bnj-d?HChSw7ex`mQvv|!mHBr*^XRRFShOPRb zwO536z$3NYjL5miAUM$4@_L~c5bMU4(CCVIp?_S2Rwy0Kj8Zx%9HqVzD$OVq0aI&R z%V*2-K*r(E>_M5Klq)OG>|6lIgcb>Yu~4m_4QaZ}q1kz9FH>c%TXmG&C=6WZBz!pJ zGJjnL1`K%PNv0!f3pQ>?I2{m=|LCk-8&hVP2Jn$jDGNglLOIau5p-n7g4%UOpHA%k z!CANbVZaNB-13kf4DR?&C*L+*-)|mWRX)x2CQ&Zt3Q1bWuT}@dv(TILy?QBOvS&cj zT0C_56+1A1P@-^kG%5ks4HMFtk<6qLZ)*A$NeHO}_`-oJ3)abR8yUI03b{3i_8pp9 z5Ho;Ps&VOIS|UYtHpuHwtP`LHI_p{+N3O`W$aa^QzzisD@s zZUa>f1I`oETbPoY9zG>oGs5L2Xj&5`Ar(E!l7;hc@}AiVgn;hyBBP8xXC6`>Ex;tuf)JC)p_99?KZ*T zin#k~SypP#Q>^~ynHaPI8Wq7dB2tVOP8y?}d1Oi%8S*q==dqS7??}%T)!L2#W(n;c25qPB{8VE9 z%Q>~KEduZDOgd?GQ-e=REew3W^kRxxM{ytnH57Vt1O2MrxU0m0<{~9=PO4&y*81w{ zC$|-}vX1)G%bEWnmJ&u1)Kr*A9f~zsY|mUI!qcw?a;ge`7IK0 zgN-Q^XV?OQQ7hlT245LDCMs?y+lq6gtgqO2&9i`LH_MJPd-sfmL;ys#-__zIIl3jY zBCLkP#EhuEI7;>I4Ut8xn~FZ>=1f_MK}IXLfW#$VSw$%99vWx3AJcp}b3^Kr*~g`J z@2(OgLnagP!kDECOv~=%W-n|BTCy16n3mT}NFs@o?<5|G-Hxy?5_X`P)+qQe90;zKdD9eoKYlq@-ei)OVl*1Jsm)MhiVN#oxf(Lh5JUOJg+}tt zB3hk6M(fg-%#HJ1*zd(nrg zfm#+K*MJ!NLw7BT2uUNHKtb_1vwspI>89W+g*QvMeX#d=!yJRut$4*A+l4Gv6Z z@L;n=vE3!Nzm;FE^}Rg4ZnbbIfK)jomGH}^&|9V)(c>$r_gUokfF9qsK-+JUIm6Xp7v&& zLB>jU0>zC`sUn+Ttsz5E?p*#~aGHVQl$|8I52lM9xRXI@wi*!cXNZvSw2KL6AXZ!YddO+lXAAK{!1*TvunD$xo zqJ#B$eCa347WhpZtQ2Vco>5)bVYA~|!3M&A5eS-5cMB^2QO@zVdV?q9P!*%DA3fjW z3&X!(gM|N*BIm($d~Kq3tH(<(qz%^8c-Hke(?0?wIcmQCd8v*aAj9sXpFpziual|7 zK()c>OTEGoZwAU6;Ee!6lv3kv8rFg-AekA=q#(8baO_~u37TI9s zUM>wCFD7X2JHb9RPJK0)iyPX@`xm0Or5++fjm4XruZ#Vb=q+JCtfS?Cq3Y5N;-MI1 zA8Ie)W0h{!*y7vV%W(Jq(M@qH^__@_#Nm;@v=DtwOv!gX^-JD=@ad;yh{CQ4EHwc* zS32I4Y#%jsoENtr(E^{#N zW`L|a=Wu+{x&U9pZZg> z%_Oo{44EA~Gdj(Y&*{QhoW0Hg8WF*mk#DBWKpP&V!XnnEw|za>g{G?n3_N5dR>jx# z1|v)AkJ7wOmMjGVsYke+9h1n)%#*7emA07}b$fv|#yYUh@=7%c6OlWlqp%O7&4L(< z=`;;^n!l-OYhYO~2Z5CE$&6fbYw z7R}tSa}!$Wy2(5}usLejh({m$nd203%@WD_ir=k{4}Wv|9T3SP9L@uviM4A;#lt@M zrD^vnMq*U_4lythlVSKgCjP#-4?}eab{w~>F6M|A`C3OvCu{)m&_P+h9{At1!qUG? zY3}MppxZLKd0}g73k9Aa{1E=p6KTM8LH}vr5Z4u|@CyJ$tBD3p-g2!+>5@ivd2n>irF+dX4JJNj0-pv3SfroHzIz2m z0hXVukr~7?nc(!y!7P~!Hx<_mDWTp_z=LLq46_>k&3gy5a^*Ie9g>kD21{aebN1hN1)L!w*kElX8{^F&8=5xSFzlxtGRgZ z2Z#}IFF_jQZXDD5{$@m*6k`ybUCQH@4?mw*7TAXjKDXhho@W@)v9`H(v}{p16aU;E z(KlFwyFhJ!SW?QG&~?0BVz6F^Nq{N}H>08=+OrgHzX;EdkAM)PxQbc1UKdB%v|v*) z-iXIVu5%u6gX)^C{)f-i{k6EqZN;Jqu33EmzX7Z%-(~`c%~YC}O+Y}bDA^CuJx_l% z(KdJESC5EdkcfHJs)XgOnMjG%@pfGS{NEBYXFsS$uA5Nq7Gm`DEjf2D4bV6nmMv>S z$}Jg+qLaK-x9eRcgIA2xo5~}8!VmP7D9z3@ zZqTG|`g^UL@KP`@$%-pF%IKKWkL2WZt3r7?nx*n$tpJe6)_5*>2u5H~J_OVGCgN*> z<(oFVUUqCNZI4S#tQ!nsc#RMGmcUU?J&~)z^4OZWze1T5Ss~6 zc^Rk8x9F(_v1Cg#(3&-(Bhm_g-k~V@<|)=NT`Z6Ml$pyGC!-*BRpeS<(PXP6k6TCubtFx{Sv-RQFVxIPJD$?zo^HIKk7t%8LIs()*@B|~Ak2_G z^c{#DIHo|p4c=<=nlA8rB=-Ju*Hdr0@6wECEch@Dg}{Qu{GPD!n}Z&3eEkx0{@A5H@CNVYDye#R?fKGLQ;HNSQ6|p1H_`1zB+{DH4xt|zi z0BT_JS-37^`DSV*FfQ`0I`w-}2YWD={Tij+(MTSqBuuN$CWcf51wQgf(A}|8lzh_q zz!RsSYF+MQBt#L;g$Rcy{{#Ooz--`lF=9j-zh*QtRdJWVJs+RAYW=AVhPd{Tl=)+b zTA*RpQ=^cU)|TA#Tkn~VvZvi1Zh99fV0jhZEnDgW(zR#$e4hZ3TOhT z<t2>V(K(7k&Xr{sP3Sd5P}z>#5&6Za}MZO zo|R2@6Ppx(CtatVPl@0HW4C$xvhXU>N0D-V2opR5+X=X;E$1y>%@ZALUh1J9E!_D|y@$ z1mW4wa_wO;-c|==WDzU?b^>i*Dde2+e|=}bTUJ1r!UA?3x0Uuv@;x7-}BDzgvbOi77g5hbKj}I z1QxSP*I?UyqN9FUx`fIyet5;88%Gq8G`n>_Ab_QB1-);f9q*lB1|n({+xT zWuEdePis@{SkZ^G?61<@@K7lgW#VLk&^N}k0Qs`c78 zr!lE3{$K-gm{NLaT!mD`50H>lEVc$f;1l)MwU-6_fdnf;957c&KgCl96{QT&F`Eoa zB3?5Qb~(r~8w(XwZuxmG<5}Wv4NW0au>`KMdII_n)Sexb-WTCeK=xS@HX?4UqOkO2 z&!Bl7ZH$V_2G1DOOJ}woHSzAtg*n2hA;gjOKe>~8egnM%29fdpm1>SoDaFcEGeUKB zu!utIHr|LJq?QbH-H>Sh$eCm=o%@h@M@CT|!&Ky}kny5B`kI z>%G%3aF{IRSIj~HvpM=+IbNZp)6e)9@NfTq)%HBm+#$eI6KDvUhADW+2ETH({jc2TzbqDUXiPV{%jMoL1JB8j>>tNJ2Ll9QGJfEk(LO*=2rZtCU> z%CnH-?LXN8U2gTyWaD853zfM7?ir(ID4zg|3n5HonnYn4xAzux`5KdpA+;D;PZlk3)FAioB`9$48T@5#Ur6ARU7R;9VaOHRMe{Rycl|| zgF%pQeCH$YDmeq9lHjGw2?y>Bi1Hv>xaw(dpIH~dpYamwiQ&8~%OLrQ^B((m6ncId z;se~;dJ?RGAfM{vg6Yq^Mgf!)eTxJn<5&M((Pz12q!;b=covWm(@$~*Tg&2JfQbiOLb5f(hN)%aFVP`&v{%~U+7JX~ z5^mkqpTOHHLT4gPh6PGX6^53Yy}0;G1B%`q`3&Uc8P0CB3)pE!yn5ehf( z!jIRJcr*&UN|X4hxwxGc@3{vrva;BY8Z4Y;A29IqsXyqp+x+{wx1o9@9(r1@p280< z?R0*kDxw?b*WBAh_WKKc%l=#GN<%Vq0Sb@4I^L{^UlskwylCHiRrF}N^dl<1fjpct z)5d@!$jF_h#s~shvaNu$r*!!Ia_REJr|M?#yGu$-an!uD5&7_{34lB^5onT9ZPRs` zrAsk8qh0-;3YG5at!}1lubQAi;zB&_$OaM&T66E+()29M!rLWb*h-WpkwbgA4@Jc7 z{?$o=O+R6Yw^vk&KK4Cas4!`txx3(AEfph*qrx17joT4@f8qAw^=#$pA$8S4{PWy_ z7m_XIHh_D%O>0o&#jS9u3zmM+T@l)NuR4Unbi1#)u9@O#$#)m&9>kl$R=Ubv1zmfP z6+`mJz8`KQOJaI5nD+PaR?o~y&2rlXyRj%!hRJ-p5YRcz2z{z*BZ>b;X2tiyS(t_Z zXTP)tj!Z}#kQ=(eVcq-XAlbTbZZrKfEbzd?wra*4oUvfT$QpaEbH|~h@nXp?^?)HZ zzhbI;sUbSj$COi}cD>gAwE_tc_I;~|wj)5EnQ2?*F$@^80Z9*jXX4W<0ZrWX2P>rB z4~s@^H#Q=dJ2lpZ-dGn@1%X_{Y#2N$JPQH#om1d9D4%{C%;)QH(5vmW;-cj?>f$jc zQf}(MGu9zZnF7y_MXMw^w!G6ka9=N9BZ@$8@VKt%ru7)zm6O8UJju*^=lx#JrTauQ zUlR+EVrcYS%9mx1HQF1MXi@b$xr---@HB*Fo}@4_BQbjOJKd?&M|$Ck%25`BlBDH*&UoqO)^lEP2*(-oAPTx;XAQeeLeDREN-zCuSsj4HP36L%w-ZUuW zy>WKl_~%aT-mTJm=yd%y0+Sg04u8YOt#HxrD<31W+P`DVf#S zjCE?(JCjlBxu20ZoNV=;+qWx*Dzcv5POBuRSH;4w4l-d-mpbSsyvUr{BL%WZc(i_A zES~)IONV4xgxN66Wl(le0p4!Dg%YMhX%j*blxz1q!>!ANb4_Ek_>SHDuAPVh_*M3> zVSH{IqhbL}`R@QyWkaZ7S!!$>z-6GHu(k}=@AAKDWwP8a|sHbOuV2;Evm(xlCtGrTDcLlbOzswYTGP4^HXj{>!#i9hgNsN zMFa*Dab&DKJ^(@Exeb4h$Q-KrSVK&Ur|;48i<|1 z<~#S;_hIu!tjapnjzFTG6LDyui{7c`4(t!f?s|6`g-YuV&VcvA2qt@D+g2jWS55YO zB&k!x{wZ@P7{%ptNZ@UG`bW>3(?){i>!4s2q%e?sl^PbUzycI8SRO$BuiKuLNJ-NJ zUbe?&l1LKfWckyCBpk6mLWL%z?bH`;olFF%dSPUMRAO=PCAFoxjdkKjSo~%J`d~!U zi;wv-71$Ykjg*x*BnHyHHIG&V4?wtd_WI$R@R4$OQ?CQu;ts!~WefgWy^{0qr!qpi zV&kJcpD)vNu1WFD>5cQclM{Mj#d^#1N=(MkG;XvYX^B(|_*vDdzbTr@w`I0u7yY-RFbWlNkPrTo^mG80v6~_fBjfE8DQr}^fG_^Kv}%F z*B4ngulG_>SRP!x6M^J+-3LX+8~HU2x-I$6zJ+OWq_}-ENA^@Mlit%d;Zr%)fj2GY zbBIjSYip$*vAH9JEHVo01x_X3CBYl^rE%Tb(jTyRlswj78(sxHy ztKbI4VO?scUftdQUQh#7cxZJLyA{Y7XGr5i~oFJUO^}SwF5*V2X_g+97oHrCgZ0=8YB`Gok%scy4z0D*xnAZKJ#POsC zf%sNZZxM7QH@KY(aNX+OtKT1SDUqtH4}*J5Xydo42h3ATy^gJ%exWv4AVIgEz=Nc8 zDI_#;Tr9OW+v;--j5p@lC4eupl`(^}xm0k>owVSvyB|+DESIja5C#X9ZilkOJM9~# zo1weM+g9Ah0IJ)KVEPiu;tdfE!We@DoATlKRJViMLG&XF5<+tbXbJ$DWX=|9;i3wk z-##RByaOY3^1+v*@;MO}L@#7V^j>ruC4bpUX4+MLFxuTD^|z#K>352%)p>Df^`b(3ihgB}7g{ zU~TtW=UeMOSh3|wg+UP3&?Sded8v*tn5ECI`XsiQaL%zG1^zSdqT7+$DXc}sE zu8SUg+Zu6g7xP}|GN303T_0?~#hFClz##eyZC%KlmN-v;Rqr~e{PLB1YbYE-Ot{B% z)yd84oX_0I5O}~&B6Aio$dnv$w8V>u_%Qh5H_=}yn)B(de`EH?|K#83xef=?xDNeo z%F_w31f?$|bCj->V;#ZIztI=z`x^c|A~*NW<17-qn-TD&)&F?_yYXQjczI?0Mz~#3 z7e`G|Cv)yOFBH$qP*l5*0g2A=y}fADeGR5a-=D-T@Swe+&E>3ZKQ1xqCd*zq3z!Ew3L z{@DTS+LYLu$8~~)ljWS_!@&j^wn3T*+#cDMHmDu|L0V?AuD}Mn&T)2~eucyB+7~^y z(ar@6_D4@!sD{eJ-vwLl1>>Cb6Q92wu*_uTu~Koy%k~#`Lj$fJrcwYiO-E_qOnHo z!EtwgiW4r-ds2=*s^a-Q5(u~J8={P%X{S$DUiWwEna=cDdH8X7yKp*ascBIxPp1TM9f13Drcb9cQPs$Sn@aBa0pk##64=^dMp+qREs^4(&C<1oOg_jdEt4Ul z3>f--{~nVRr7egSS3`P!fdhc~R!;xIWM$D4E6`PDTCqX>4}2Wn>%mZz!J2a7@1*Ut zusCvjG`@vhD%`R71uE=eqxQWmy7c{*?#3n9i<-Lz z0#sL5>*s98N=IEs9G6y7?2jwXxG%|2Z(-66=&KX3dL_9+700V?xGiNN9M*fA(F+a` z7+L~fq`H2TuE6flm6~>vD2w>qN>Y49s*K~nM`PlrXcphO^R8GQ72{%7&>;c=GqU7* z0R$@3odGD}1wYVgu~38?>QH2}w781U@C~;*>d8=`K5w*%+Mk|`_^pYd8m5ncp>p=} z=@C0(2SK64nIhCM4?{F7$k&~E^k-{Sj!wi4j=msRhb*>NFhukspD_u5YJRyGyPtKG|;EAV-puS{#<@I+g?z8#B4Z>A7p~N3BZ7JeN$c z*TB2SSX)B67iMO$8OP}-Uzy2!%XQ4-&(RNc1U$KyVgE2GXz@#XMNgOp{W|D_*W)W{ zfz4VfXt15xX)v+gsJ2M?d#M_dAAC14>7}g{&w-o-{irSo$?6qu0*qp7+5ms)@M@n` z#hHzy5zFcDr(}I+!~j?oxzUvVQwBm=X{d zf=H4r)u=GiJK%{NsZVDhJHaqMnJBy<7|h)cb-hK?m5P)6$%bTTp`G7r0D%M=T4M$1j^_1!SvJrITj{5YND`yG+rA!ga>)H2 z`5Z$m|GgJ?2foVO47@L^M5ikBMuG+ z7xpw<2XH%`Ec>|YJ(F+!uQAYGB}E6Ee3+1S{dM5~src6wNW3^e1^Hh)xFSpIdlSN2 ztEwo6%$M>3j|W4wDDJl^ZtK?y2UzJq^ar3m+z5fs?M}orkEbYq|FGgo`gA&kU9Cj# ztyMDSD@U%3ozT)&p89gsZL_4fPq|sTTs&TlPZsK1;u%}Yq%%9B8SlmGoRHvr)g>@D zHwRhcTou?t;LE3z-+OsF-LUZbnsR<$n=@#Y=acoFGr;(TmuAfXPw${dsT~hMWeTw~c*PlvE4;U-n%zci+`s`~ zBf^IjldSF$Y&f_FDs-ZLVS|O+uwj+JBizFz5kAi(Hm| zPGXmR29C0}8(m1@CX@9zrB(VHIcSZcNmuA;Pzzd5_Oau{bfb3~)3WQG_ zGxgTMps?`yIWp#svVTYSbbM_OuVE5`Rm|D|ncZKloF1~uN!8f+iKi6KoYJJ99?#)p=kT%) zcgPez`2RwusiZBc0Q%Ij;q~HL)y{V`bsbvSmIQDX_rulI&G@KbzI8xM-qa6MqP-#u zIb3k@pf?p@I#iqM!mID#`~aGWUP*f6JVbr>K$M~MvO`5=b}rylfFYuuN^t{h%~3zk z&;yBE3ExUtZz{z6ZYMeCy@*Mlj2|C=oWnTfBRFJuP9-r4GEf;Kjv0b!n2m!6#|_Vk zZ_s(2v+rb*XyQ?toz);H`n70X1Vs(W8MMbDf#c5c*;6C%^Ep%R^;KT<7A@KYv5N^Dx_pANs zL0Ux8P%}dT*e6Ba4uwMex5J395@MTNscBjbqkBVO$^^-YHwY=GD+G0WbFK;B7X;B(hP+=U`AUV0iC+=B?ajY1+7(5ly4m zbry+g2R&p|$SW^{+=?PpGT8D!MX}-Dk8b-43nWY}x{DYNZ0A3__%vD+~Oytw2mQEhIN4u2nmShV|V#*pvef(mPX< zuKzf5xUGbJysCOY*bYh*7jN8jpX@16>fSF|(!EJu+5sm2?#U$#U)ADZgJirZ0B$ZD z+PALkw>h*&KfXL?pMy0A-By^0ZNq4K&FBkYTs^&{y$AB*E`yf2Rg%KKYHziM0Ds3G zoY=TIZgaP94pi7mu~V2pps-Q8k3=V(Z9{BDtr?J6j(pIrsua;&lZANzJXF&gQbGYt zgz8U0~BK|D!sAJHZz|YX%1y`4vGL?e@JAU z`@cwXee(0l_`m|qduZZ*{Ir>jDj|f=wq)(gv8JK+fP^$(Jgi43fkQTJA6cBC1~u(R z?)kio??%8>0FX&0y<208-|4WW4Hi$xbi!+9bf@}4#yys+IbuWQhrUN6LjmK5+-*XZ zw3gy_@2f9iBK`J_T>;`?l>|n^Xe#`!e=Mw`AXyK}`e>p}0j_K?^$LLg0@f^_st->Q&mDeGhqdlxe80u(UvO zmd3~8<4_jR$w4IPO4-0QInizPd4x5*MXmC)dISBzes$SyLub z9*ManfHfr*_~lOeEEHtqPn_>j3&(k4++M0s2Ap#(CUxf{(fbZJDukauD}xNgv;#+) zN7tXSe{xPs^BcRbYmci!RS%m3gn89oH$mg0QZ87oWO4IE-XN#mMODpch@SjZJ>G4j zCKv)R;Qmwymq`L!w8oruBO7 zI^nUSa{-1TNtY6Q+eE%_Qx%Y@P|nWf@?$|JtR_uQA~3&m5!lI2i)Rpp2~nRA4XMlw zD;p|d+sNAilxjzIV6#xdyC?++n22Tq&9JU;CqxIazOQTXsC}PkQJ}_znwMg1(xZx$ z{-DM-RIbEBJRLV-ClU@$5I-?|ub80gNC<%4b?p*ULJ%|xN}MdH_*95}fxP2pfV$gZ zhfeq37il~2V2_3YEFTwg`7jjN`47djx?Z2V8gR|KXCp#whH_rEQ>ZYX4 zmUQ`bw%Pd*f%RKE9yCi3=$YnW)RVpi5MK_A46}l0t#yd8durfrg|Y`_+7#VMK!sg7 zAuKn!=yr5N^W;!d*L^ZD#baRDd#`5PjO*rlGn&)8b8A4W`n3UW@v*@0{Jl>VxC@^W zpEvgU9;yQBeBX8S;GFkHmqFH)G!rb}f{90dwMIqi0_?c%;q=I_+dDu~+wPm|q|$nMkUfxoC%9nLP{Tnb^TC|Q__C4vkOW=;3P2OsA8DyE;K@=9ZQT=?u&olhVerfgx z;~`L|TL6>~0`WBoH@2&(tTCJbH z4(X)gqQ#<960=hZzAePTeze*#kb8FJ!nuOOokF3_51b{Y=0N~WS5`(xnZY;&v8n*O zW<e2{OkyZlsJe`*C%&eTL@ zz!xAy1?xLv*a3;f)$5O7ODGE=`wCnfz}o3_Q@mpDTfGMEFywUKsjI=U9l$$6_Ee`! z``|T)8B(_i5`w*U$Gq$~8jc+WMx9kpkKi2I>Jf>=$IICZeZxPico;`r(EiVJdKwCY z#8!&yFDr<0!`r?ZWRU+!aJYTjQ2pNxY#xl7e^i&6@=|4H+NZCTO(2bnCRmZ@Z}u)& z>Afti2=;3dZhG5r7UmdsM)_JPt_yueM7O(0V1SYokj`ae_!JH7F07z2md^b;t#TZv zs!SqQ3wZyvhU1<7Ph|r-!0}`|T5TpF`$F(p3UCk@%$4fXt<;Qf>6ee%)|@)hz%Bzu zO#?SZsO5J6|ALkifc#i*ag{6`>Fe#4%X1BepnAYLHBE9Ic{@t1Hmr#c0QEYVeQ#E1 z$IGG!yXr*%>)k51b|v3gjZJ^<$Eb*u`A%Wwe@3N0Ev5wMmb1$#Nh&;+>e!LAn#Pj&_^0JzH<08iNF z!C8spzAO;_+5N#A$QzFN0SDoQNYqNVbfosw3EfbVDbHVD@UVBcpuWmSOmnN5`QuAp z;mNPV5uni2wudxzuB1Tm?p%bk4uY{jSOVrQ2vc*APAQodx+2>&gI^f2Q6n+Z6K!xz zl5B@_#lPE_PBL&yC^9!wDI`GdF5T~Cz)UIO7pT%r+c5p7VCncUL32HQ{7_(c`Rlbg z=!s$sPBt&&{mw<0i(~DFOSOMDi~F62NRM-q_KDu_zVx;G`l~U>d>mz>MZdMWEVmw4 zNE2SLB)WIJRqJ*{bFw6H&_Y^?`(+KLVM^lU`X{;5l?=~n)i|6mYj!gV28UA$Q{T@# z%jJW}O)Gp9U`z;6`?By-Z8Qv&lFM!=GU>?^k(DNZwOi9*+qSdtBj(sYCUJadX8@_n zEHoaibJs*t;8MSPVhAG7rF#{InC`I)Us5Z^Asj0kL%m)c>38=2;oyfBW1T?Jduc1xbo_ms^o6TIG+} zh#yIwrG%^&itr}|f+zovtM`D$`tSe8iAo|w$WAs{3F$&+k-b6`mpv-7SE%e&WEOJC z-g_0YE<(sA#AVMUE9>`scYp8mzt8`ibWZ2vV_et!HJ*>T62}Ja!p=(W`RhMx!#_!_ z$Z{xT4539*is6^6<%c`Wz6vokFo+uB{qdB2o5G*}sX-u!2CGdUNFg3^(t!f38<;Vn zeAuQDEhzkov0!9@4B_eXZAch02tnCE0EaLn6C2OtL1Xg{{Fc6#)R_^b+pmVX;^qrx zV6I%iF`tM(+H0Z_hp%V=@kDyt`u`@;|N9H?ZsE$2o&>^X>b5hV?A<{;WGOy721rf? z|EFKFm1DMfz8zjupxp5WdJP9q@B)&)3CNNFDGd_F;`jtaRa@cHe^lTRT?>VUKf zxRoG)suCRa3fj7X=@i$sJ%Gw%4g6vv0>Of$pNuUWWABS7KPHz5^?IH~N*0!F%UJF(k?LLBm$E(e^t#a7cmE z|J~F25b5Jhis&VgjS;ldkkLv+cJAU6W2#4HeMB*RepZhXtUNu>sDc_X2K5JvbubRf znGq?me)SLOg9sGJz=2roaVGblQeB(D-Blx@PkpweZ{q&+yD!vLLR0k94-;!?clp(i zw{1XPrfPpyt3$l67~seHFc-OGNVb&E84gSDZEW>xZCMxQEg5C(rnkKWaU7T)T1qYK`c| z=q7>DUmWvhMtc3vhHnSb#gr$TAyH+(&|Fb@!5u%d` z4&t%}rH!g$Q%z5S_p_}y)d&XluMuJsU4Vvl(cPQ!!&Srn2y$K!&Xq2ZQ$N88!iskn zX@WY0ay}3kCFMd)S+C9|=+v0M4#z3Oumw6PJM;z9cbEXdEC42X{mN#DW0yviVsuW+ z&mKi?jK{<^bDi;JKw1K=9pwb{$gd0mpOEoK12l}kzB+)KXRDd{ z_%*VC0i6B!0nx~d>`^t-f%M{V#oL4R4ZjdmJMsajT!rl!BxsLlF9;cgkr~DvL&Cn!;7JRR5Drj#wYkxs|RI^}q$hiTb2tXRZTSFrG3MUT6I(Wgnr|&(QOJ_cy%`2*Z3n@En z_F!xqc|1(^yRz!(Iro`gEo}dS?OutCAu(8v3?bV?#zBa774dFb#`A5qwkhVlObe2K zRle|(Fy3C`3-!~f7sn2-&Rb88*(E7L5`{g0zH*a$ZDMFc>Jax=5IdHc3k(ccN3*0^ zD8h;{Iu;VCbARcd&cm=_*`e|V==?w})#_XQ9pV|v$@UT<;y33?J06Jr!MXERa0C_t zM+RUJFnKl6dtem+) zo9nB(`(!rRxloXR?jE#KS}ni6F=2Y=w)ls5Xe{(0L;2wvy{e}CjX0DFxHW@0`=|mS z>?4y$zj7_CEWkb#N$fs0A&PFf668&+pt-jk-_UL@JY8#71Rp00D^t#8-AWdv-aEZu zZhN4ja^8SC@&MkBe|p&Hp3Ll_cZdvlAPFhe0s^jVqS?_H-p1VQgu>3Jg*7SajMdCgDar{*I%VX^(24kNG<#mj z5dAQ&JlQR6n0bGtL?efilpgfWv%)|q$8E3-ZzD+ySE-=b;SMAZXk9{6I2I8nQsLF8 zXtk){E1e=JSh)h^HY~t2u5t-u$WsK={-Oq-IapYo>m&hKZtk1)eSdT!3kJI$+$NTJ z0ZBxL57W=vLuLq03%up?4l@SqG)zx8_-J-9Mxs~|OMvv_E66qW=Tz8}s!quB^Y|u? zBbi@VxcR&NYR!gkHtqjJZxHG0{@)my66LdMh726Y`G|B6#HdRp>hNr<`#u`4JbKh? zm$Vbws!bLo7u~@|*$*9@5WCH$#EXk$KW7x*(IT&{>l=7iKWmg?D$IS|aJqQZUI$oFkWwpM`R)E3Iaa3hN&A>! zEEMSk)UeuBa1?n>thg^rspGa5Qk#eQPv|NoLnn9LQQxH0y(9#Vqz|@cv!q^Tt#45= zVAFQl8(uG0TT;+%bW2KTNm5N-}W1pO_@a;z=fTEwE_ymKx_e1BLd+Iu5nX~6WNN;6&+O+fJV zs}EM4FC6+k^lf=J<;x##x2rnE;qAFby{6v!;6VEeO9FWYyeEdM_-4_uO9T(Xy;mhInUjrDn zEjevnvMvfO3>96EQTBjG?d6;5C_uxFB)HpB zf@n;_SB1i2FGjeEk*xa6goxK~goWLq;#zP5k&Q4@bEyhAZ6g;ErvBlN1*v$XypE<}ul3eaIJ;8x$F(_7(f))% z_Fr`dk`R)W>`Di4@9;ONYf8~<|NeY*F}LPZ`rbU5_i&phQu^5apMYceenS$HvIgW_ z$gM+1w*fUA6rQub`OtUKRHI9HiB!?vFpu)OYV$16u>Bln3K^= zSc+S5t?d(pn+dJT1xj7QnwMe~tFYg_L*TH*@N`|;!qX&Y+>;8$6ze^m_Dr#^v899I z&z!Y>)M*jt!E`hooM0jRh+m_iihK4u7`DGUIpG10!=W9-l5pApABv?^ltm(LsDe@M z4e5iSSS^3Otg~Dn8coc>0?s{OwrGmiAo2*EF%<5)E;=q5H{Us}kMj$t@lsTz*)pIj zc=E?hZg>+#VYGX;*~Q;7OU$H5a`)#;=9@2p@=6^)o?3(Mpp;*3Q3^NET%j*=bItp? zFFJ-znzX=oSR%X2K9m2-{|Uqy3zmZ14l}o2J9|ARyMt~|sWys7r&+-aYEhFafXy2s;v5}!vJ zXnK2$ITD1!x)lD-TEa&LpiAx61W&rXwj9)MA<&`@SVzPIP``1F*4ty*( z`SsFEc5l;M`Cm@7NzK>?t*_d4uCSBHN_E<*t`4TC8DH=k%=6fK7uoE@`26Hx3hLoT zw%^oIM74$(C^}zhb{9@FYA^YuXH}vV`mi4|0c;CGE2+b0c<5`M}8XX#CyQc(Uf;X4T zGLH$^-?fcnp__sp8Pt%K!*sHTb%NXAv%Tq{r;b0MOjQzz7>9UfIt0mf=vNi%V2p=V z;mclsI47_reX=Eec(@?F;FNm6@cChs8>BYhIw=3os)4^NKTPgkflt46A7pGWQR4C* z9q|tZZsyZ@x8*Ik?;S~*Kdl~Gl@!>XSz0KW)V3%^9GcVNJ?Hl1M5&JJqoXsa&2H%EtY_e3*N7A<{dXlk$5G z+o0z=wA8&WDqPWnWGNYz7lW?iZDO&(>0HEvmhUq5FJAQuXC_=sYBT)Y>L6k%UV84( zFF-lOoB!p4M|LPg9nY58Xm!Yo%+yFT2qj8TJ-bZxp>JK$)I6`B6W;ul!YDiw7mL$g z_LOBRGeVToMEVc&a0gP*E;zh(TN9)-;OjhOX=H*n?TcKueFa%5qA1F`79}A9d9Oao zhSpS^<6#jCQLjjZ>r-;Z@fEVom$z46UpV!(m7%t2iVwoq*Q2a=|IVaL4^^T2jI!s zG|c9K(+|SMhwRHbWiKmG)HzE|(M)@d{7gBWMTxbrsaGi*U)c1mx~1;^J!a&nBr2_+ z{uK>>qgDp!e?E>gM=P=aLZx=-h~RFh%4r6>;uY!8BgM3GWRIj%y*avV3p&hs6x;Gb zmB|UiIklAo7?LMhY7^2%mxS(jmFPW%zcuZgHFRUatuakxDGmt}ikq<10gdW9${W;sKo)5wM@7@K1_K@=&eA?l(sRD15TexYS?fI+w zTAL+uEa|*JOHo#IU`_Lw8@Ifvs16%BW5Ws1&b#*U9?TWvUcbziNX=Ybt~UWGK^A9+#jlJMqxjd+9`?Z zXoRAcrJ|PQWggD=4yUYAJUD>Srvczg7=RM2+m6kD&lk4Gu_O`fN`J!y={1i4&5nBB z&;ymh&5XwhDGO__GdJB)9I?uBc>C{Z{TW{2TqUPCDatTs*3cR9pa<<2Ib`|J&#b{c zXOJVBC~yh$p@%8#wpGQDd5IQlvI;pN=7z-hVrEH7ep4^}z~N01z{LHwV-6%^?vyZCq^>0^CJ*zeBWaA3&afassMaS z2>cQlAiQk}H{&Zr4w4i@U^+o(Uap)`g>3@@yY#y*nhjp(-ZW%+F`&Yu1bW6@1;0Ae zdX>+kA^iNH>Dn}xg{NFNFRM z-zAhNzURNkke3U@>6>7K0lI4?!{b1xGxltOr4fXK_qtj6}`|h^=A{)5HgCc2KXys61$MF>nURKZ4p??f4<~y~J9Sez$--a)j24r-p z7=~L1T!>CVbtXjnLp-IHf(R|N<8=<6nS)!prXUF;5ggozL6*%hKF;O?7cuc`2qeo9 z4aV~E%%Nb+Cx+!WTCN9I6M|y#`y{(_%WM~g4q+HiTg8d;Kx_w6N!BZK7Vr*=JNC{r8LnU1XSAn^CldPA5zaEzFX*s+r; z?WcEQ2oAn=;X>aMsfLDr0%cunh2Tf}F*8_Zdvqa^--ua@1!&|~MB-Hg%<}5=Gea<3 zRL_v7Eub(a*ZkMlpr84L3)S-k41!l*Y*ky7d>Z;N zxMoHxac_%#wPd8j_;IDCpVpOoAUq*DB{ahrAdd>Pp@n^q;m)V{+Dz!^;G%H~Up@CK z)mZX}@A~7cbK zL{fhJj&C^S_wM8%t9;BD>MY3Dr%87y#$(O_YGhbOdHQu=;q_ANTjJ5;M1!_U?A~*~ zQ+?UCJ-ap)b>KL!c@2ZS_E)Q%p>dKsqMQ2R9nO&B_i)8FT~#7BSdlZV(iYF3mUJBy zHaV0`0YR~YN(;nJYPs#e#J6(PO2NlQQ9>D=7LBXj#oyrp^6&UVT>y0xkJ3u0q|Vn zs{0bwRg_rh(tK4_Z~?N2x~$+>gEJ&BQ;2r7ZvhfoOx5MJJ(tD#?$dW&dWM)% zV$i6RXyrC&oc?oW<_77ZUxsqXu>;lV?~yny9&`M5%jgl@x&EBy@7{?W0~Q%$1}C?eD-LJHF=To4Z%;WDPUlVDVR&(pN9y`9PIl{TS{Ld27P z7vu2T%#jX8_gBg8xA6ahIZRu32uVUBtSyuw|FCiffAw|m-ZUx&?o-FxY92ojod?1M zqdgR|5C>~H2W7TfD!I@!$ zg-*SCEJ70?;|gLR+yb$8!1Zt9W>%?j{~brZr?UM!Gti%A*#)MNB*iyT-fTS{-<>xV%SVdcmK}h<4s>yyO1Ww%{Ds+usKBp*JBL!%^Y3 z0sm~fg_cS)xl5(oW%;_%-Wz=VCNT5D<*a~q>FT5jt!T{`N69!I3&1^h+?5Lk z;n+tfN))b`CY}LMBnJoO6$bfKi4qrJ@1PxXPq|Z*Kp?+otDfmzua2^WYn>`<>l755@ypl9pp%(-d z1u8rtE1_YWqFwPhHzfR;U;|9BDAl33HYeYOHV+&2DA7VU2a?kMwJg{?yEKW*VUA4O z1pUYZu+<45Xn?eN)IM?p`d1saq@<3~%X3`Hi2r@G;hQ@AdEI4&U)b=$104KQI?BD} z>VE@18zD03%Y&)EdRjRHa<27oGJKaSg8voT^AM5$uUDKXf2fyDf$A9_6z+1*$Y>VF zf9H@toE^RT(H|RV%Q82eCFVd`ynTUL^Rq|4yH`*f(tX+Qz*mx+%f$s!8)e;B?>p^W zsO0?^_N3x0dKrUF9*dVM(B8KLCH|&2#4H3@(2)nmy-bV93~4K{p!=vDQXG)?%5-G) z=3ls)Wah$JfVJ^l{FIw$^e~gUKAaYrF?|wf5P9&I5lzn`iU>C zC#iXmNse6MiQPNJHz&bZHXNAyzZ)qqF1 zL-RQoNq>Z~VudV+YKvq|2wFL$8Hx%5&2l7r!d$82!)`7Y@aM?C--Iz19c0#o6AFVp zbcG^&r_~`>4tztg|E}c>7(y)rYG0oV@5v<-UrAD7HV$p2Oi>jWU(6=b~})wi_0dF1y@=ni)(senpxvC2z(4k&tUFgl05rbUoCyg7wsa zy@XQ1Xy9#;H&(c!Yn?5QAh`)swcE~aeb64#w#R2$iJe)5((U#bafT_Mc)6a@UH%sU|Z#TAm$2!1;)SPot>TDaAPuu+@1wFFu_7U zJ9D>X3}RS8v{P_qbf`aX>bt|-9Y{*AZ(x8#zW$5OO$Fi|2aC#rflGAaow1x4}0Ok;w6! z{54Erj(sqGDsxBSJi+&qJ{X}W=aq7YBUWApnj5r8c8~r(CqL~NSa|1Fa8SDA!0j%p zejVii!986r0Ap*Z0~$c-1h*LIeq<`+qA6`Lqh4D_Y}-}RY^kbYiusscYZryBk*(&y zaw4BgR5Nh_xM0PSw3Yn92jQIk*5)oksS5TFs2|V6#r}CA*UtBg==r`*N^}Zv>`hcEY~BGsLw=z+wK;zb=0V?Ldh)lz6-fMVwGt-6aEJsC z*$KsGj4qib1=2F6gvS7CGgr1J!=m_zz;2tmLmp*&hQ3yArn3x|t62Y1THQ?T$o z^B`#cu0$+Ls%jbF&u~rVPQ65fTBHN;g!by%!HjIwFCo?b#$6wDoB}*RTRtMXr{_{i zrs21)u@BYD>iPa@UNQXY+|xzX84? zBk#t6M!+&ia%f%^GD9jm3wuGIKyN%OgySZGObE7;5mg34^laG+YlUx3qMah(A*FJ{ zC?)k|CaM_&KQ?52r@*9V>m2#xj!Bcd!c6I->(MGL@{5%kmclL?H!cltT^^;JSN6E* z$V+VWAem{$y~#-E4h#&*p~*#og=L0zaL(f+EZU7RV4-z%*<;jt?RPN?*Y9@)Omp`= zG3wd9JZ1*@oS(`l=PYlitkgSJ%S6@zf}v&A!`q+q%Fb8_Q^2>UAVh#s zNEhF-5Zb95B7Zy&G^#}+aO|H)EzH%eT@Kuz--c=7cDdA9xp!|QEm08S!VmYJFw+N> zA-dW?)wi`@4G=y4gDsrVi0$bibh_T@32{ zWuGa|n2TR5;Sg~wgz@XQVRC|nqUwe-59p1iAKK~XQ(abP82gM5n3`u^r?RSu(S18r zY998lrZ3$qZ+TP}NdUlYQ@|8)<*D02bs^SBNFBD580Fw!Recy~@+*IhxS5~3p6&%dUm)E>U-Lmf;RHL*cL!GOqNkQtUk_9|-nocd2BYP!c~-N{Du9g?_?f;IiFy*ht#)1<;KMVme(+T9J_k&cRre5d2}K{W8{EuXJ?0q^WH;k18RmM>VL0b z&JFGVu3%j1-ZPXPmSuh;j)xHLV+U&~(wgoBI7iU4GX>=4>L{N^^;l(is`CmH-T85b z5O9Z>4#;Ytj7Ws(5oU6O^Y04Lgk??UGbVIru6n7|ye4Q_ANI!KF(4U|mjWsVGP&Ch;((`({-4qGTjFj)ctnrlb5Md^JD zUU_+x0pkKAJ{O4Q(%XK|7zueEWokpcBy~Wc3`@#N{$mOH0@MLX3z`}>z%VCv7;NDw z5HDTY3F@1)hQ@aiEHHxBe4ByA-0R{`s+(yPcO4aLqtcWHu%|yu;v0qH}nfbw{&y^5-b!* z7eVo3kw_9abA*R!bXykxc+nA{ZlO-LUp_FWJGC0odeU=+(uvkdsu+Vu{EZqr7tJ$3 z6+K^4q&sXlR*Z5$+o4HiISo(3h>Pa26II3bA0e5VFrE?|*@sxQq^ZO+T`*E~4ApOQBF-S)PGYHYklhV-g*Vp}UoQyfIQVYAa^|@_(PPfKBePQbHD>2W znCiPYi0vbrzppbpsfyF*+bOo8ONFkn_a5_!sLc>{!k?kMg+FV0eE1B;L-)~Y- z;q>vW?FmFMR22DKi^K&tTHYF`xD|mIU`lplb|YVOv?{-e&80OBlO2#a{mVPbSy(1Z z4DdYIjll-VYe&k{>Ro+Rtc{Om<T6V} z#WVZs$j{ZS5%~8(!x#vpfauBL8!B>G7s)eM4P7%tiIM~Gm{JB;QeCSzz-~~h5!oiz6Caj8K==0iO`x`mWHcGvv+qJ|A3&JnO)Sn>M9w2YDV=(v!ymA-G53cwmx%MkVg$eqio$R;*s404QmqYtC&Gqp_h z8|kl7OW@w`YYkL8W+}wl$}TA|GuM53EBSOR9yDEqzu)l@Q6-F%AuraUNOOUkizA@G zR)VXzvr?ejKw=^WWQM(g&FGLP(bctf`3q}=ub(Ae32Jzc{23#fQzZC-6ng3ky_kQ3 zM3Zt~tFhCLQZ9F2m_tX(imLM3#|Ye~H;+E2bxw$<4p<*ML+3X#9NbBZ@SD22`Qd8K zwUq#y2I=6OV6XtqvuQvIR=+Wkz|7^WYhfMLVWEY#s$x})vihzRH%!;UCsZn#sss*w z>+(8qb`H37WN7Qhg;*qNnk1{TBjIeH6*7##QwVVnopz+~%gqHA_AOlybdHc0mVv%> z$hmhhdeIoQWB8KxMuXO4azc619Tp;}5QcYZxj>D|VK|AAh$Tkp`_cu>!Uc`5Rv=-R zzWe)^>uDbLKHH-ppWO_@hK0SeW_GH$AUW&9NViWf(IxE-$14s0s?Pw9dIiccq^b$@ zrH57%_tX1+g2ihV6UN_E%7-{5zDXU}e_#5e7j@DmR{sStL#?+ffyJhx#>t0^D7pvvWRL0^%{UNlk7x`gA+*MXZMMdC9*%y3&$;%~;GH|MlV+nxj z1rM%VW$yU2q2-^ROmni4v9^r56zRS%f9d-HQ|MsnCP3u`XiD(IXUiM+H(76}tqDAB=n?&wWgSk_BArw}If#->kdYYm#} zw$*pE%Fcjeq6!1hv+(o+4mdV}U$|fvg(ZBI=Ih(u;&``(59sdOt9yeYq##kCFw2CS z^9K!LtVLcfU2?zSX9d=#}--jD;W(pr>L+HwOjS0-+ObYl9LDi#iV`GdR(l* zkF;xr1Ex&pumt`LuK>|bfRv~)@3-F(hek{YxZ|yc zu+tg{Nc`XPpB5><9Vvl&e!{BIr_h z=Sx&xf&dC$b-1j*Q~M4R0?C8ocX}%yFQWh0krU=0OKD=FfX z1~`lpctOgf1?0#LiTuH&zHh^${PUyhi>)=RnKs3y^<4vw3Rev^_4D5XzL1dV&m7G0 z3`G4g%9vbYh$w>d^qJ7Tym2^_!bX^)V3z*Ue&LRH7xQDYg0H+lW@P}YZHogx}k?8G?e+z5+hJxU|8T!AA;mim@=ehr#` zjM+Yl6D{Dfm!eDmm_*JBlr9yM#M1ZJ!JrPIGaA3VnXkV`K}Ii}GAjKa#K>uM@CFPS z`8uh(5Ak57n14qPcnZZ|Ne+eT>2u`ic_1K5T6;1;^)Tl~yQS7%y*jHU0#L4a0QuS$ zV^jU$!795t`waZ+Vn~AfhPO0b)+BSJpmVbP4~*8=m#qAlgtfJGphFjOCN9yayprZ4 zlp(MDLGPXtwc^PZZS&-0?!>Hj4dpkb<~#Q7LIn7Iy&(~V;rMvtBn33c`$&H74@WHH z?k&Of-2-n}Tb5uwo4)qc<2>L4Gyt1{OV2IOe^cQ3=K168M<=m0H8-C_1g;~vyqtl5hrv$>)MO@4)f65TnK#!$?H3{84s!S&4f3SppxK-;jYQ2 z{lZYihmNzB|E4%gSF2Vx&Y60N^!uPErK&Cdg_+K(a=L>Khfibt zAhMP=e3dJldVya4z{kqJw9tn2eSMZn&FGd(y5fd%fs9c6IV@?IWKm3x95e;@x|GCF z6H)cCofe||l+XikR9qf9-$nSxn+H!rhHp?`w{bb>ia$Ti%?0g>%!I(mhkBIM)%DYV z_+XOcDe||5Hj-1X*(phl(b@Q_1{jd9>k=F>LZk%EFkySazl2C7iN~-pIoADo=s#{%VhPDCSG{CefuY&1L8lm| zD+r~6aMU#EOdx+-UeQnzC_D>oG%3X>QiaR0fBrUSAMx*4hoydzHxfA_p!$7N=-I8< zVCb4p-yP|7$=wFc$?ML{-W%8Gwxk$FSz9iwkMgFdyh?KxcV$94po6i$yi+SYW3;Yd z+fy|7=Y>d11U$v*pbKxmj8tZWs9` zS3#}p-gNfP)X};d6>-49_Ge`Cu0b|?__zuA5Bt4AQZ47f@AjDKr2Faf7{tU1=CDMu zPt&AGNSE1NW#g3q~0+|`nDZb|0 znz7lB*F2A}^^HIZK6{g+MKq_^VehLouvNr>T3#N;nR6qF<`g==c%=pwvicm9*!2qYK zx%n~gfd~%1a77<=>eer(8|7H4rs2yhOjS*igCOcb;=9~sMPm$US_Fy)Vc!rhq(75_ zHvfappjxv?sd(By&4hzVud6gcB}+d}Ng|Q_z_&Ry(O_@P+AWcHRgK-QC3C3JS#&<& z3jO;1)840?+^^=s5j#{cz38YaXtoe}^hGz^;j;`G+A$Bcpaa4l$OPs{d6? zC<5I|`UHEbswv%!Rr>2rka*wLgxSSiyh-YERDf5CX*?-?_Mol(SozYI!=zJ8Qn;K3 zK)xK2r?@o!b2zAdKcn_L)axHj?LyIJih$43J)k(81~l&=1qo%v#l?eWoH6Gh?VTSg zMVP?ti!2KxpY6f%EgR$%@U@6U{k|(|Khg+6x0-kl55|Unw{S|miBXfV;Y>~Z@5u3; zfD_^2KR*72dRiBim7ESxX<`k8APhkDrfC-&MzZ$qAw0dD=p{0ayklML|XTzMvskE`!)PPG6>Ldip0m5a}r1RnWlm+ z3?S(!zvTm&lG@lSu_}V4*zB*8?=H4Pj-$F=X_|%{ilke%Q54mxnVJsgi>cP#j33&; zvx70N*2Y>0yTGL!u+~c{8z9J5t^f&=HqpCJ`}TC5!Zxm+Y&ao^8hWi7=Gf8L&O{ky z0A=U!?ZhMR)s06rsr0-HOwVoQvXCZ%_ioettAjbv?{HB8G)x-{ZdmYu0bc}>csu}0 z!%$3je(n_LWjY4F?E0JvmhDok3tQOlV_MJ=zA{!lZFQZ6h<+5qoySA$zY(^}LoV}- z)ezC8jeSx>y@iw4zB+Tf%fL6lkqa5>7 z2n2yO?^gnifB927mdjD;2k3#yG&}1g4#kL|h;fi^0W}=@g1NFAA9T;cq{|TCh5@-2 zkioIJM2-a23r#PJEed#uWys~t`Xr~YWTy~CH56tqIiIz!mvhe;Gtb~~CACWeDxb)A zr`2xdEPl$0BCd#rH^Un3mzmQQ|2tog9v0+U2mibI;{^n%*WIjX|iWgf&w0#}{a2Zt$;mJxQIzuFHmFCerB;`KljhvTT@`wf3DAGs<_i4h|nRsq6oR ztv;T;vkv38(pVs4N&ko=gR9qonf}C|J36}MdoyvDB3zz45mj@mJS)cXL7D+(z~o8t zP7y&;#-3fc^ahcR6cX7qzZn3^&$WtqtvOjG6R(edP_Yi2OJKPJJ!70{a9ajO{&B~q zd1ztS=8u$x{w2~7154V%8<+*EG8HO?Pm09gI0C^o3j+blg@JgKBqDb_Y)tIP?H`!5{Q1LAy&NE`NNKL@+V* zOa+Fk{MKjdJN9|~S7Vgn^aqL{ufmeb{M)GkBSlagine~U?AdL09Gux++zA5F2NjiV zcck_H8T77Du;y^Tpz}TU8F;^h0==!%d3YXH->G?h4;Pr;+Zn*XTXCj-1JqnF@GK2L z=qOlxApnuE9uCgSSjXn0?iAX46V`|rm!Z<>5|!eP_>Zdd07DrMgG0c$9}ZctJ7OBm zB@{;ay2yaDK%D_20QmVsT;;?Jjmrf!hHj{eES$dyCn5CoR2ZI@WZ05QR0Bp55xa@5 z&8L?wgzhhHemzYQJoJE37=ZDL=Hj7k6qr%sEPi@0ZDpV$ao<<}H0@qcX$FNhM?W_V zvmjOmfK7{soDkxI$Q1mwuFxFu@#}>q)PA%5++jO%45OG9f3V{^;S7UW<`+277WSvD zF3!Y-Hj|_n7m(vCE(+0S;`)kr0UnQ1ZbjZQvI$@oU%r9W46#Z$zk7 zhulpb@Ky;_AO7yd7l-zYJw&?&D`ha*+w%X$1Yq7%5WXsBec-Y;ET9ZCFz@f+ z3a7lL7Mp`B*5Ot-@Ye1il=BKNw1I63i0%~+6~;S+eJMd32OtN0Aq%)VY%v9s^17Eo zYe?V~0qzX=*rzN_tKh6^?xj^{0AT~XN#Yl;Vt6x9K*3F#u#MHuvEt+VZpC1Qos}Pb(+^0Vz&;nF9T2({J^rD zv!-voROFFmiI%KxtJ)-BD6*p6Cgx?J%&rn6+0Z6TEc5C4>`zol^OcqGVcrdc{I}U} zi5F`l7xvAyI$R?9R1CvSm?G`Z$h`}=*=J&$R|N(WRtGY%`OFwc1h*bM6N=Fw!A8WN zuW=Atr}aWzfowF|vFSa_0B7;&TBqwR0ZpFZ*Av?<3tfUDFj@`Y&`~(oY#7ktT)S@NT=5- zw;+I@zh?I0{QSJ@&eDGm3N}e=1gC|AqWc{Lj!%0``gmh~yqYLe8h;ptJUCvkeD1re zsF$l69U>0l<62kf18xJf4-X@i!nr#v2S8(a9%&9xoyHiq#&n3-cDM0SDqO-(wh4A@ z7V2y%4HtZSzUfy_+AUgmhh7I`;rV)d&x7Ju4zW`P17NRwJ3N|Z)Gc=xz>xzMpSL}D zf()wdA1OgjTfc#;W|~7Y$qmD`B2$d^Vw3?ia`UvJKmY@8Kis^R%wK>|%J5U{ML2(~ zU78x{spNxm?a)CE89F8~B`ghu80Kbp$2kjsvHcu^3EnbJ61Yw~3RfktcYz7RQfS26 zKtv4@(9?sw{$oGAe8G}p-My!RO=PE_xK#!em7`0ghIU%*78<$=;dsM3XrUOmSL>}@ z!vjDRZcOmmbz;Fyvql`3jQs>@M**9Kn_i;$Oobj5Uo(qkqJ4ruzrCAMRH3hAn#HI8%_ia^;hRq84U=810*sozFW)-L2$d%=fL7Nr^FC35) z&X(m^zxR-ePZOh^spbc=4bWrn8n6pOGtuutMJJ1Cz}f=W9(*Tf=8`vKvbmM zqA*U|CqZ_~5&xA%5`v1oW8~{+1DhEkK1}6VV!_|4Myb0?J-brD`@v&>@5NsHr)XDVpP7*jI2N>ZI*Z>Y z?&4>dM=+heW>`iL+;#}cP+Ojq1#Req+zC2dwaTTqbV1HtE zU$`~Z0r7W8)m>KVbBU@S&(33+y!e(~iG~sLZUZTju=r^I>;1j4;d;dyqj{J+C=gi} z(th(F6R+fd7tqHw+2bM$Sasvzp~$p3HC|-%2}YS`peQSudVT=KT*Ibxo&(v*74nS3 za@xAK5u$KqZ|>bBKvJ2Ys2Yw6azZ#-Q9Wj`FC{DY(g!=X#^f|RFTd{oNIPlNj>}R797_?&W>soiokRSE*tRBK*c0F zV+=6cR|T3B4wsxR!;H_evotTv-3DeYZ6Oj#&xmXN=7J2<2U2mSy~}^}^Qr)j1s)5e z?WiXKUXcN#^`?GHe^OZN7$T=Rsuy~If;)Pn+9d+{44Qk;cLqik1}x>Y_vZk~6hNc_ ze+Tpyn!vzz1w~k|3N1?d#fjv@Dks(`2{qa+fa#ps{*8jA1@~$=bU~;;d}x#-n)8-; z&p+w~>sWp$ttkT``Ktp*t1s(^MsFrff-PvlVVJbW6p|1?U7wo5sTe|E%M`K4A87Ul zIB=nUHtMpw>`?!qED=jWUwNTJo=<|aQ}8Z3WqGa7`(n+pF+SS09fjgtnvt5cMU3$DYD=PaYT_Z=U_{rP=jDdt(0R2w#lc%06qh*9yYCT zuB>ke3Mg}Ma0fe24%A{K?p6205>8L}o{hU#SZepu2G`*F41<3QT91k=YV}-z5eF(> z&v`lk6D(Q`>W4M5z8jbNdzZ-W%$6-#uO9w3tlp^B0*e1)#4k_Yv7%2J{J}lYf}EmK zgWGwe#W(BG0>t>jv2CsTEn$&kMB;cs`p{t#0IGlS+@RQXfcd-N*1YsFPTCzrB;P;@ z56;iSo|8ke_(#ug3V8N~ReGjM?|mT?ho(v-*fc>c>xXl{syE2s7g_<;z2I@--(gdP zXiDDu9`7R~yZuP9KZ1iXcrK=SL$=Or0C?HIN&U}qQEb?dgF)t4^kyXG>DKXEBG3E6 z9=7A5`jOglxB zRX*4y5lw1_pY+Yh2c)~o+ z6hpQFaw7mpKf@iAt2hHY5$Lpd58wNbFrvmENZQ2~>Acg=>~I$bDRG!$ z5!mzpMXH093XVf(9vL_eUzmCcXG|=?D3!ibmYWNrn+nA+Jd9KBTVnXRK{*HaWJnrt z0Bb&Aw0T-dYxll^%}LW_V!%&1=XP0tvx_9EC+y*UN5=q2W*fmN?Mbx4=A@ycE8^RH zUV!@R>9{;n5>+S$>s{XgEV;u%d%96bEjX*8wpPT*m{7jP{z#14Hamt|a=f~Hd)i8kIck1PVLhBG#dn34a4-tF3 z6MZ~0b#gT20grvprTaz{M`XX6r(YwJJMsY$xriPZE?E1V8ygT%hl9(TrH4l%B0WK1 zUeKaU^gMut@^}2#@oqyZa?wL>W~Bv@rwL(s{z~@ogwOlWpYd3GI`CgT zZnhWTwr}X&yV>J$0rKW{ku>4`@A38WKL&zZ`6y2D^6aZ969$Ed*Vxlx9UUd3T9t3w zau3n1xg13KTR-K&Bi8&`K4cB`r;r2-`g#!UfuzJ+6hLi~JLsM#Sz_cedKoQ%C_zxd z>L0M|fLHaO^N6Nj1d7l>KS^2(7i-lleJ&axZvAL=?di?{pH?q$ssR`|EGJDO?|d&% zzb-(bz~TV4SjN6EL?=7*;D=tUbbq;6@+|SKb(5>-Kh~S*PCVFEYJCEC5?~X}4dvK^ zJAz)7y6_V)Z-K2#lfX@!?s}qHy2EG0Kt}icXBmU2`9kMuqhWT-L@@7zf`2K*adJsV zmN2!jbHEYSHmEi-ZA?4GmD1Uh#$YCBRX*20058S5iM&)ylL(2ku~{KYo8U@PpkyM5 z$o|||IM*%*DfhbYJ-7>e|8v95E0>M% zNDa}Rw$1fr6jWpLIhj+pDgGVz@^17Yq{NE9nWH}^t4;mxU9^GpA%IX3N~LGVt{G1e zI4&^p(!tG1a!d_?(*XEk2QgjNizuJ_k?ubn>t{lbRLR#KQ`!o>mW~bAxQ9Vh@YZvc zYT3JOFkJKP6GToINoU!udiNU#Vjj%8>xV5~->bkIrp;qC;6l+dvZlUAb(pD7yt#D4 z4+n|3?q%!hK6Gf9Sshn;`(%{ln1hq2j>WJe6XO-#WxlK!7i7q=-{1UcgVa7r=A@np zSGc}T-b3G?i8EoEtCF$;1Ib`pIPr$sC!Eix6b><-Qt5SxKj;=_q_I6B(I{5I^6_WO zKRhXGFiGY7BT91qAfYG@Epoq^HN@NoD)lNt{K#rW5TWPOsEmHLr8Ce2F}uR(L)E-X#&1dg>2m3+MLNZ}UUtDHFh;#Udd zD;)?z_(xmOkXw+XC(B?hYGl|3`fL6FGm-CuWGc}wnI>3hAhkiEi|CY1Vbi@H`s_rC15IKTEm{C5Vsrw|qHGS`O<*&&T zi)PW1Oq-JAW>y@{G!Xgc2#1Ql!s%-QoI)0u3ENg^S4SkJ9_{0x7c@YN7;tm4uHmZ!iI*BFA@J zFl6trQ!yYaYhP2v{`krVVHs%3+)BB7k72v!=g6YxH`>{#cL&gY3Kb8%$`Ia^rEc=T z#CW7PP%eahziD!&T;#?&>76HQrFLH2!_>$0)xhTRdpGftQd>^OF)@hE9Q;)+K%+um zF`<`L^35EWB6gIew~6b#+Ah@G6(Y;}TyB@wP30pU7J&!1TOjm;mks7a<`l+}!&dioJ2Fe+oTVT(B4wI8t%orX zQ5e&*rBn_0Sp^`qRR1c4|Kf#V$)m1!OW@aO0q^5;zUK(-WD0SjkUw-}!&B>~fE62U z4>lz??lcbER*_0Q47z4ebRwgRADHU0yuR8*osL)b`!?-wHI5?d8jr2?X>HrRF4eK> zbb}RB1HFzz`6?>=`NsyknsvZWXzRxU`G@K*uf)MM@-Kf4Qk8p=9E<|-bg73Fax?kn zyWP^O)3TgY`(oc#i!wB6|M+=6*0Z|y;Ro>+s4Cps-BJ`>%tvE?eD`t(^sxUD^pJ3% zKK!DOfju<30E~g%SN)~(HG+RQrF25(jX(wJ4bY{fUVbUs?dAs| z2JIjazDOIwO2+F)99E9yCltK=z`z`a;clk~@j`AI*+48qdzA$oR}^l%z6#}t^3GTu zrm&YBvX-H-Hx+LtwI0~y`?xAnrV!hocb${+tQVl!cIed8f)5U`KI*(Yn7m=^Mmq0C zpUZmR&TNlAb)P7B9Z4{=J^99~&kp6XZNQw?bXXKIUu-A2N49ytpe& zr^%i4<&t*0d>}!KVf#K~^%an6P`z=Dmhyhr#vjwn(cwQNYE^e7@(@F#KlS~E4zdMrhmEWaLq z0p4#5r0$1cRt`agS-IqLH-zxplTHL!y* z53)o0+S+$XQ-+90XV3pktGVI!(hMb*{I#km<RZ85bqn!fd(B`6RCgh5az1=@ayP$Fa37@o$)(YejT8Be(1Mb$>C%k%issvH&_FY zDXDBdmHceYzTkI#=b_8`-c6$*DAHqB);gv1AejTe?EwcQ$n9Ilj&zBs0mR~PouWRq z>swbBCq1f8loM+|t@M|M3xxtpzGW*3kFahv#%eLpbf1b!{z7;Iu+BR?Ejzv0pedSg zQN(crzhmt2{_4X&`Fb)9iWMR^?!A*66|44HSL(wKAJf(O79O5=Yz=^61MmePcASoG zYMIf5>iEK4>T@b8AZ=iB2?r8fPCPkfd7UF0y!3~Xm{YCTd{&_09SDG2UZr<(P4(^8 zrQS)QdH!w%Qjso4R%f74gUQxfJBN-<#Bna_yew zbcr4K72Vj?(o0KyC&O<_eR9por!n4dGc?Ndm&iaGzdfdhbJp{0(3lfxDSsjF=`wm< z^Z>m}C5)4`B|_W^l*pBMesPDmR?Dznd+DZBA3B7&s>nvUTcf3HV~UI8 ziei|wz2?yPT|q}^&d&KI7VNNSVn34wMx-nt3U@wMwRgfEQ6sE_h&SvMG=AAUa0}Z& z#>g7R5!>v+S#AhHE|IT52w^&6vVUKb4+^?R260PJ|Ep+oFJ@jufrboG;8YNIzQ z$?NFsIfd;3I zGM;-%kE-r8xyNG}Iv}q?9{`1rVum#!Wx{!0z+mm~Q+*e`+~T(EE?!wu@cVuD^h@@Y z$M5vnq38Pa;N*y?k87m6UjMZ>4~Awzgu}QqGH-xe9B)ntMBC@8(`9L2(b%A&!?W4D z0$lqnA<(G(VOolJP{{nNsbMUyTgq>V9cU8$lzJv`i;x)=)3!__f?49NfHKm)O2Q zy{yxsfmekSBU(R=e0VpoM#B9$Lx`9|Jg~J~mUET*?@~lLIk3^8A<6-rvw!71!I#bT~w|kYR&~qn6k6&{ctljA$U8a!Mdc-{gja zn6?13KWdEK41;UipB1d9QC8VUEp@D&#fJe+qxd?ou1|2ui|M(4%xWe4s6f4~lWfeX%|SwIToA< z1l=XJZqw|CBQytwZAUz{zL3O*$dkwmwI-)%2{jjC386_4YYCIZhBdn^>4GAa-YFBo za65?cl-?BnN%YhSG}{hYT{0Y!fv+YoaWO9@A1vr@PRU@xEEHQ@A76-qT`_;?C$d`( zO8mt&HGalhiM$EznG`jmN#2DXx)?~k-Bb+T(xej)<7Fmg%2CQnq*Z@Q4(8?mS|z*c z1Xh3kb!%x55^Sv&@a0Jq6Z>^gvEwAmw2hBr#U_ZpV0;7c_^UpEH=y%$+(@e$4Wb_M znBEY%hcOoj+><>7RjrD|&hz{lRXN^7sW3z3IW1wE8$;8?1hKek9332Zaz)(gJ1 z=X6Bfo=r)}X)!M3)1`ziCY{+f;VROQ@8mVta*0zq=>FSGGyrpL8t*X-Xgry3a2?z) zsicOl;8&xfDKuwy>9`zI3l@*m(Ffh-dmm{+{Ym#cxYGvThK(DNsB^TRJrq$^d4p4H zYy({{0}N-lTXTL?i=cOs{AeJyCs?-T&t$_x7by4qSo4y-Cx~^jcG4B(+en{+Q8kTz zeQT!TTElOC8k#ZwGEvXvB0)UFWR`(3=Kd7E2$*Yd-&pi8FQ)4#DX2zd{>Kp?mnG ziZmO{OA%~)zj5G7 z&CC*r9(2c0D%C=L%(H49n4vK%mgv5IRQ}SC0|ov5$x*dk8V&6FMRoz}MXHz6-s`at zGP)4|EeJcBJ*GLOZ8hIGx*)ToTUl+=$>0C#rCS5roYRN%CTId+{81sETWxfH4;n1H z>B81ZGSQg&vI)nPTmx?>);-P;LVdc7FyO$jLno+WCRiV;i+=Aw==w{HtX4k4*e$U% zhuCND*e4l>F~Au*ZilAwDKaJt04VN`wqefLB#u*J6d3Bb0zT zF(-Hw$W{3HcOSG1Ruk$}ZRn&x*)KjwpD5baCxsSyYL4%P;2fxnX47P<(ARlR2d%-+ z-D6NF;RsmwPMKpS%P{D9q`U6r=aHob0@uWtPiv92i)5}ntu|&K%wqmP7U=AD6-Jr` z#I)W;g|0i6)%qKA9@ojt#|oCIj|vhe|FfUht}g#N^rr>Amt9L>XrKEnBr@BdH2ran z#K-lNW?$#U&#qG)w7FHKY++xZ?dAT;^BA9Q>lQk-$WI1h%cS0l82zXZf++AO0W0*Z zmH8mEt_y}0K~ z*l}dUd?uP6CV^gT3g14gs-9fMJDKs_-~tikJEC+)HshFhi{y){*S(WO+@z+%LDNc| zslnU&*oyb{SKikL_sodTx&UoMYNlSI{VIDQg`eg==Egf;P)hnN45V-T!5L6KpYEaCLjn+n``?~Yp1NeTaAiW6o( zFGoMY<>yo43qLdCxxRcD@$EvXuzNbDL=s2?zfftbL9#uHCF*?fR*!HgU&P0oOuo=x zjXiQ|IM$=OFKNCjv>R_NB1~UJWi1_zeZv;-nM3jE_qf$>GQU&yAUfAVIlc0Z14de1 z?-ZT>Jk7h#{2h_0MBZNUn2($0IhQ8!x6SdfFt)|cPl4N4)!fSiKnx+LRM9HykG3ULbP3KwU zR&?#s(+M^kRuWvCkLHfQ;Wj5k$A#%`LU>#jZsiCxYczj*?jwK(PTip||3kBCPM&co z#y@KQ5OK0=2|d>F5pjXk(EUYv`3b*fXXG@hlXVcv;{#=Cvtx5BkZ z&|*(qpa($k%4Ccm>%6+`xO%1T?H*1j(dp-ToixDPdtKct--?N`LhuH|mnyXgXxphM zxtA194Pg?FR@ye#b!ZK;d^galG@acH|Eyfr6U3NeyaM{iC=4JoHUg&N3gCSn;a3$S zdpg($ZCmocT0uUX(WSs8&VNI2Npy4#7~RSbp`a7a-YHRhL5#ppVe;VjAE#Q;%(`Ry z>|>p$J61i+;Ta#Ik^WLe-%D#ESGzl`;-w)Yv=>fDoW3m(f|BI6x~uCz*Eq|&@o+ilQJ z0<0|*M0l!)>Q*Bm8u{j)ZzNyan0{VIJMK(G-K$4ix`B&6<sPX6gkO|N9@F^mu(B@n zG!XthqEhmmDfa72Mhb;od}BJt>dta^XPRl1+nA<@63!K=ze!f_hzh%VwtZ2e%Dz6S zu{!m%K}+96>a>|qtK}9VO)`!4pkf=mc2a7p^gHG6zef@-d4anA5MRHJhY)CXJgtxH ze7=u6gpZs_5?G1=J(0;?6)<13kI9yN(NJC7@ zMp)`5Th}DxGF%SDXZu}fyS;gFp_+Dt5aXSvT%CU?Wl8$5Nem91DLhqM!&kxMmZM+7 zc`RP?x~NsSMj{pD3zXsMG5Ws0VZYV%520JV4U*>D zRKnxc;~-vjy4I@tN$9S=HG5Yo95_e6USAXhCQM+LDsMQzl+~Zk9BJq$d!Vqbi6?o0 z)?{1lj?Hkk&EKprg$G4jpDyrRy-TzmRNSqVDJ~~D3*07WRv*99wUL*Ge?|;bB~i^K zW8x@IQ%yX(n4W1f+?mZ)%;7Q(dl))jF<47+T7B}0*S3pW)f;~KG%vD%*1p4jT*5ZA zfo)^2 zrH4AJx9(;9dvWt~N4(!mi!d6M6H`XRlu`nyrLqIDquV50ra6WS8}c$Nc_7r&T~Ew+>$PHQWRAIiRu#aba)2z__^ zAZ4%SjPn9*RQ9}yW7p31A1d+o^h0)44unZ)^VRJ2;oi(j1A*^@6o)Qvq0t0iCohtd zWKUO^{f4IWpUn29vN$=1#`Po_lQnMsT_U+)Zuz*zdA`AUr0im@ADJL$a-JJGI?efi zg@XP6k5l!qOWP(nN@C(2ii*$|Gi{Vnm{=%TH%DY5UN}`|%;jmdxR^AH{ZaBIdj)m0 z&E0Lq?#3Nd`f-%WyJuoX`@>Op(;?c`d*27keK0wisnIZDU~I&>tw7-LhT7eMXC9r| zAKsh;O49y>=kx>XQ%bg2%0Ea=deq7cP|n*ijwwZMe)m;A^E0fSn6!(I4&QV%u{82L zgnn8D*8#MUl}?+!{qZaFg)jOM&B*_W(lT3u+!hs;Y60(0L{IjKXGTvt*x;B}r@y6y zJ&vLfIl%Cmi?&qS(u{Y6dUki?h;7-&)Puh)7+egd6<9CKoD7!DpxYJ2?!jR7f?j*R zKUFo&HS)&RTY)`-)r><)41?=cRjS3}j)-nu-}H6@-4~;jZKc;=fK1g|ztigjgoN6$ zv=UuExL>f@!<(sydc>jM;qwi<)nYT_l-8A8tU`5)oG79-7g>Ew^A(}aol+;SG$g-0 zupVj*z2j^8)Vi)@RW+FeMaLP13pHzGwlWtvvaWl^=6~>z(UEl-mX^U?!XSVb>a1QT zl~%`i7*i(~eD4{Sm_sG^x)sgxuu`cxxo>~u#i|~D^yRwBu)zguH2askH6`q8F7~d} z-|TeI_do*V#{<)`*3sn{Kf-x z+SaW8n<5^c7W7qQ4gYv0S^adOj_OuM4iBI(Ap^XsTUVh4?^BEt;G8<_lVbX<)TU?hsgeJwYCzE1PtcZf&sQwLF|zF(Jn4e^AvnEJ=Dmm}&YwyHuKd zU-!qq)i;ZsE>%p|Gmt*z$q@r>XRN6GI8vZpDY-LIwEhh~r4#&tGcF0lI} zJQbQ2H_@LNXB#GD6y+(HOql8bM^DI;RgY825J3I5K zH5v*w)p4++6kA>{P%R*j7TRmbk=%}VHEDhuEiD)v(PS`OTd9~nKdE)Ai2mXSVB;es zK|B_4aqes1dMs?qdDotXQ{5Rg=3u`+p@|vcb*O#@rWT)K{Vi!jd7GvW2aMYpYI{DKbt)JOrcZrS+}+?w3akQ7@&+Qw9|0ld~6z; z@^R~*8F_DmA)m}+qc%&vc!XROKOq=Vd#HGL8nCT&lc z@u*Wop+Te;QFOv?-YKr`3dH$1$3k5!mpZ9dVHmT(Q1q8)W7U+tOog1*h$mHOEQWtf zlVrq9!mn$lts*I+^Hvez%WhLj=&b{}Pj5H>QdvrS&&uJ`>B$GyY0z%swO7=ricmAt z4r2Y4FCG^5E2TIyVI+xSE1YACF zF<+C`W`BsR!QH(un~%k|r4W=S=6+dSeXiE#zw8VHCkp@S68_*~iXP*Yh0Hd!s~g_0 z_gaQ=(KjV&!h>ylI6u^VWN6&Wv$}p&aZ*W+&D9OYn;MrF>eOd`HU8apm-kr4qvk&% zCd>>~l_PC;M^c#=+m?yCbz2N|uQcAou%bWlT-&vw=vPKpve`1@Og1r8G556zvW#VC z#O73G4J3XjGAVMj_Wa1J7kDLG0lO`m_dQL`uDX7!d`#|5!>TU+v9Yl$El`?BGuUBP zk;8q1YLh897njM1$dya#Qh5zdlaO;Ro%kgB(bRWCs=>Zz-7gM@I*T#4caG4XYQklY zh{jMv1%nmhX8S9xPja^!Ii`iPe|hG)bR@7O!EB_-x8%Lag4ht}m7R(?IVZZX-}hEl zi!XUhtxII-^;X>Wv&nnsYUCGBMrm-`jWu}&L^|mZSML9|FiM&;k|$7=xl|Lkbw$o)*Xb3 zB;$NAcPwP+1IVp>OdB%>Qk!j?$~CvmHzS{y&|I8pu;_TW{Z^+t+hAeR|0VZRkN@_i zJ35kU$=-rjmO~@oGULjZF3-NvflzTDf(?NF+}e%U(b+Tk>z6KBlLi0=yj;%~Ey4&{ z`jt$dXt>1pSW2!oZz%X^)f(c&Nd7iirF~&CtqG^ODcyV@85O}nB-MNqhS2SBf0clh z53fL5uXPYW-Aw2 zwm^0=WI)KF(Wc$WrZbTA=$C$p&y_|dfOs!x+=jUmnyNAD(H;03ae+bcP)4*)hF&K< z#<#vcjLnpn`=JI_*kz7&LX}o`M}%@mXvmUVkD|RW$e0aeebC=e9bD72#LQp4^xNjG zr}j|a`***E$T~HMzatj%1|*}0bxdu#y@ZkccX{+$t$8e4~;*0hB8 zq3tBkxvkk<|ImLoJpN4PudCS|OpuU!OC4I7qXl5yeF7L*8jJKEH^#3ZId3T-@3}Bf zHgW>rxd4~-bF$g2`_77ujsL&Tlm9ma2fotMY=eT>KD>pHlYejfWbd=QxdThXMM(H< zA&y&Bt~r@#wYUI4G?>h-Ds#|-AG>g$C`KRc{eK>%JsG}6&HB?;6;#%zCJVEQ|30$= zRP!bN_!77Y(<2=6)WPI@k3>pb0CVkzVTBbG)G`NO6+5aK)$h$m(Ur`+c= zjgj9<;KF6a-aDtE9b}`4uK(ccr2o6`=D!#A_qX|2za@)&y{m&%|JZ=7hZ$iu>g707 zk~fjmBJMbk1-B2AgE~J6#F#(<_}0gFaUA~|PR+?fHwd0?z7_~f8iFR$I{QKD0 z1l&C090;7KM_0AFEMayfU4_>Yteh7vq!o3UZzyPNQdII|hKx{0CYMi_lveQK{N07- zdLMaKf5L7iTMP-C033yo`SBs4CF}HK_QLY#r`R#-88+Xq&xH7jp9Xo?`dt>d0SjFD zFd2R(*d#d^eZ|VerT+bO^yyz%9M`K;Vs0&Kz{luKmcXP#0?CNn-SI1;6Zl+sp9jrPv7waKwdFZbmxE;v^G)`~O?XI~c`Ev-dan01;D zjx?kVzG^&H)i#+aSv58|R!*6pwQBL1S!iv?w_HWrxcqCzA!65_1OMlIkkA zZo4Zvjn^i_od2W-1klH2kILQ#wVaP(!&Fw@?9s_q8?8`0oFo9MiP{A&12ds!ic7xS zkJ-(WJNG%^7Yh3nV1%Ot?mDN47ka!e%!_v{^~HbsE9yaPbgO4k$j#LMJv!&+e;h!H zDWl{!)9exHN^y!h8FM1%Y)RCbh?Njt32DK1$|%}zet4o|S$kf$oim?u=)4mT!`0?V z)x7Tmf~h9_jNbLFgTe}1VbJuRMLhBr8pmvg@~VQu;`-ud$DEvi7Tgprq|EmmqP}P4 zoEPtWHmcS#t94+COX6np(U;}d`LIk~Kz`O=D`|M{?roES+55WzD49-_o4+twXzo9& zQg@i^{BtMk!TA7%C^A`Id*vzp{B+g417BcEc#2ujZ)-bANN%MQxmCP=5w!V%DR1>W zMW8dyv7mMhPM2LdE#`@{IbNZay zba38YS)xnU@Y^{rZ!tAR@{mX3An(w8ckH}4^*#1;At{DKRSl^=tzzTOyS1DvDT7}; zg{+?x6`j(Yh`u-yeKBX<9kSMlh*~=SSi$nci@)+}-uz@2(dwHW%3D;BxNv;Q%li$M z!%iwL>xUF|OHf3XuAY=JgqU}Q$)@~p+D*_dEUHg!ABhyoarfRSB=J7V!hM@X^Og)2gX9TUEjT?H8WvJvsv-G zDGP8E3!L*4cBI8db~=%*8e`-#NJM@g7q>K#MC72e!8mGv*Uh#R>S|Kw;2VWVF-As= z%G9|V~te=JOzZa7H!WGIlVVIv`xBcSZ$E#|*2-BOGE zsRSjXY6RGU9}!i&ks>P^($<;2o;&*y{M`BmtF(-Ur6hCda=uS5%={KdMr6VUTO9iQ z^=I);v!8Dxd^)i|yBlk}1MZF#d5$P=#g)v*qO!G6s^ft*d0{ zc=z3vsyI>vL$tFi4ddD(q@i&XiGa=ch2ox%9 zN!o)*moe4pIgL;J3?j~VtCIQ2w7g${YtW9piA~E(+m)fd>bYX0@Kv^ujQ?GprE}@! z7kInP!{@xB7O*|ClLB|8ZT8%({< z^e&$+68Iq(`17Q5&3~s%re+1}-+G|2y6fHSyaahq-qQDNPz^K>+pKNIb~s4m~Tz?Ce(c{~S1G>xIP{Gjd$t&4fcsm`%)B_hMC>&8bzou(wcSgWa5iC&6!ZAJ#JT3aBYuNn#Bld-UMqoa>$T$F?J)9Bb^WS*YTh)IS}f8-!mIV?Ud*m%^d z^!jqKsbk~!)^mNd(r6p;u`!--zB6w@QnQsmNHRybkyK>e0wFkx;4#@^@alnK5bYn{ z1*WC@%OhiwUefy(h+Nn9oSstzkp3P=;a)mO^AlOP-}Ik;XVYDh#;e7?FSeQK zkW-z?Me~uO?G^N7dSW{oB*k z1e@)vfk1dOpMUG4o@?Nd6ASo7wXL=&iX6Mf~3o>mK!H@%4QrTmjMRn|9YWMwR2`3>g` zIr$O{f^g1Xsl|@gr{x9^CM&4~i;9ZYI1W-uCT+c6ule1@vM5YA9Hc!*0O5@eOiTIe zO!*{+oxI6g183I*8o5u%HV@*UVoDUEY7u&tIv!+~)LHJFCm2%i-Xmf3-@!#)zqAAA zHAX`ovToO!TTHTPH&>}2Vkyzmzq!%D{ocKoFWuah4|?-ge5+9p6yeyJT0R-C-ERGz z5zi@J z{#D5@$fQgC4xH`3sTUr-jq}pS_m`9IB&wFh=S(tomH0+k4}1?8MD}6&c3>3qjHM(~ zkd<9$Zf;)I9`fewGmhpbiy>v@6G}q!9Ux;BJrZ5z;(q32Z|}P z^PKtQVC&iL%8;fMqwNwB5)Do?DpnDcGsx* zP@8WFtk8SPt);NIFuz0i=F9B0HvSJEK73Ve@Zl2YR(kr!p`o`coJPmS;=fyq96abA z9=?}RzyRN7l%i$_Nxdhj72Ero)3}{x#$Vy*OG!(24i4UKZFv0n@o{PC)jmEx_#@59 zOkZExldc;N2ns$eER?#vIB!$z%Mzb$%EKUR-np`^t!?G%)tVERRGKE$C}+OC=hZ15 z1}iV&_Pl+j&5-84l1E=JQSVvUVLjAV(OKkqIsN7y`0YV#6vpZ;=Ej_$9^p1OJaf(W>-F-dhm(z)@Ft(~-2CzJ zPiT0$!Y)xu^1!BMW|s4qd>6QP@1E`4NQsEmpKtV(gY4`dD=K31Gh15JL!B4wmM>p! z(D-4`rcIlsrlxL!bd=yUYmH5rCtWsRY5E;kmfO!WEV?-?E;@s1D~7bkvbW}!%W|DN zcnHgMJH}VgihaDGC|-okW{Q3FvF0{5Y{J6N;R03#S4g{V+{GYj$xan>F+TF_GcH?u z`^~#{$)7t%2X^5v*hge~dU`%@oPP76)_vv{G&}enJY2uMzHy2#Dgmd#X`nITK0fJT zq_oz=ldgO^#a`>2xzu8xzJ1F++26F{W0>V3b&#vtWk9SXa{j?cJ8%k3EMD43h>4%@)MLv?3*th@Hr$u*ldZ>C~D ze7NXZg7Vw&kfSznn;BdmJ~SWxaeU)0(L#I^NT`ZH3dI(ty1oPp#pm;uU%A-b-FgzrQN9G z(6sGX1;eug@OsnSTpM=7B;oiR_$0;S?nr#zLfO|>Cr3s`V2m_k0E@0)zdnqcJ$se& z@3Wx`UmjQsljcy`GVrdetE;rMl(>&NadB-nT*hl$6#+dUv*`=B%5U9&36K1BvUaX* zZrf;my;WbmF0Kf+N2U8}xX^tI`B5$D=3icj_$x(7ADo3#qvG6lHl->fTXY7`BHVd# zp`j&r?!~e;CyIuq!@!YWc6f`IlJsa`!%UL5vrhyM8kuC-jp>C)L@c|m(3N)=R*|4n zveACLN1KwMt3Y1mbWLsT<&0Ygh;RK>79e%{v;{=75^YDW1qqpH_TD~pG4A?EXA$wa zxLzI{i{D=Y_puCb{_At8%~j4D;t2>T$;hltNJxMO8dVJ|GhyTB_s2yB;UH>_OqjN2 z@*351zIEZEY0DvHs)9j?m@-9ymRhj;t9^0l%T22%6nwpxjx)v%SA{MLf z;C~z@TDL2YJ=~r3qVC3<4H z_O$0o;@S3-LS6Shm@A(NN_V6ddXg4h@YH;|eRWnVlT?`ST}xAF#YY zUY*rAPoj+8zY|4VVe-q)$jhtLAI2kwFHTHMjO$b5V?Eh9Im<5PI%Xuc z=_Q%>eBCoc8B$bJV+;X12d)$@01mjUZ+rVbbEB#0>4UMg1M(c+59z{xzgJ7~!0H*7 zuV2r(Y}kop73^wjI>u+72a-*&jcY*yq+liE~fWcQPT3XtcmX_g> zkv*cKK@dm@K~_|qZm#o^(|DKg&D}VJ0f#Pfba86szIgte!NbGjQkv;5M3+y^&A}wt z&d*ZDP|Imp&yCvYrJ1!Wz_k-+c`l|7H@qu0^-ov_9ItBhQv{PMzvkN<$K+WRUYyCX zpAfd}p+g9cc=_^9ee4x}i{EmoX6-&vQQcN%4sRUxo6>koM>xKGnaO3Mxb!fTtGts# zRySr?u=%sfhvPU27}a|nF{-}_z|}u6uryJ(+!pc3G^_oPiQrC_+f#e)-o1;nreZYn zfcUeQFL&d(=$G&($LHtgbJ#rF^e-X%^N56tw}o%^xU0s*(uKFU*-!jYv13-#b`A{< zm3z)}ckLEdWxFtFBEL9(e|2@5nU<~wjJ5HD_KE`s4ggB)`1w;RRxKq?R6$N|1FhIF zl|^@DQ9;3deD?_6(3V}hoLU5a67ovPL z(sI5dz$vP#Zu4e568$*g4RMNbCWu(FXB>ZgT!u?HRTV0ZX$*aX`uCUL%ub&GgVlKEQA<~UGpk)z?~7v4YA z)m0E;)3)~G&r*LG8JUi>_S;=>z(2pey8z#n6*3gauYc*o)nyKp!J>)^Z(qp;23lI* zKE^KC*w$^^RQqg7`cjZ-w%PX`j#La&ICJK+{L#VKM=ewdi`u(nd4-4B zh~+AG);ub_k)nEZcw%Jao%Jy1LX&CQHowl_BF+n9B>gGBwB)1W20lOD_g*VcbFBwT zlisEzHoTX?BQf_H-d~PT-nxC;wi+Baq-4C?xYXVJ{cGhj?>UDqf@Qe^kwGF|?`rm~ zU$TQa6ipz}q9HzjNW;iskJHHFXvVGOPIazVjn}Ne{UIS{VGuG^PpM3J{rbVb-0Op0 zMbNqsQ6&47ZkzT+i#@H-+sDmKd-`-q-^SpW@WjP-2)(+)%7(E^cnr09^L#qh>ic)$ zy~XT_Sxldl4X<=!KUoVLAemo0E9MOr@Y(qJ&4E1jfk6Uz|ixJ z*vGWH6kZ5F?yZgDM~)!v!*sdhh-cCACsw=(Ya*&YBMle1`GaZ=E%SbqS|-h@+5(L6 zne8}}wtwD5_21eUP~A7duwnJ&K=Wy&heqR<{yLwzRv7~%CIRWmiGwv|Sv!%#bJp3P zaAQjBw#%^X{#L2)xV^~Qoprmdp7 zB6g$uk+1v${&W4$w|6ymKUdG;eyK>}3SOaX+EJ$&0r>2ww{NL)G^>zcbI^E|i(}oC zT(F?`zSBM*kYc`ZvQj|>Is$wy;p(U5qobpD5_jMF`T4OJrNu(PtS!4tM}vc(f6LoI zg(C=7Y=9iZfSm~3_~w>s(##Q0zjZ6shigUaq`etWUAwkdL_`F2(=Z_6>OAK~BOfi4 zDG27%7-eb9v^s=;9bnowFyNh*CIoj|=4xqHKeQ`1H<#*#nB94j*dVF@cXpPtRwQQE zkrYaKvvY8Sn|Bo#78R)+6?i)N4W+e`qht1FIE!V+znL|k0lepXgh!#PEX=?CygSM1^%`bc3_5p-l93hIkm zPV`AlUy#okG{l8y#1F{g00>+6GoXM(oyE~to^|Zan>Te+B7gJ15u3lR=dt*SE_`Q< zsx<0vZw?NQz3l9tO~)9uD-6 zCsvPA*hQYxgSA_xVEg747FjYEZ#l4Ya;{AOQx$>gV62=wk7^jFq@%mL1J|&Jsv!P=;$pGW@L$(pIe{ ziRk7dDu#|D!3R!IYjfjFqQrUR=XZ^=6rD!Q9zA(dh)Vs*qx6?AkF9mrLKPNt#K^jC z-gnp0Vx$!TMPQt7*>VzRW(r+w4rT6jvoR_$+wiiDTQdVtT4+xam-*cas*o)dI#T7q#dM>@O$gpHeQ9~ZJtn52Ci6zL9 z%2NjrWwabeR^eS;`*>&B{{8!Tbc#2KTK%ys;>qAZ&qYGiy8jGzBG7AbhY3yFOQ{UK9NBiT@5%FUcCDFKBDd#kQ%d}ZB|G15 zY`lbguo51nG1X+Jbi`xW0XeBaZ%`K}wxf9H5m2y}o;BlBzQ6>M(lqIu(YzJ&=D z{O=#1?mGfpz@u7!al?697K!&_ZGiNsa07)UB^Gm2=J4UAcUP<#rcmO?8Lp%nYoi)h z+33A&H7&CoQj9<$vrs;vQDimGMJ!m6Zt4R110=Nk5t1?3HWewXxYm0k5iCPD`!jOb zfUw@9*8M+zJ=0=6dNddz5=TTDd2eq^2ET%W0@WNa?dNi=KrAZt!<>o%*D56GRgl|b zQw)VGaKU@l`||u#gs;T9k1s?l-d>2>2xooq#Q8gjz~m+Ans)6}xqNj4`if$gsxP%(*mAvOWxUlrssGaNEcpgHY9e&9X5=sG&W7{`?dkgDn96cm{@{<}_c!+DHP0qLanI zcbyt>nuKq}`AsY=Rq%b~fS~u^9XZT%;~V=+lPmz z`gMK%G30K__Fkh;fBrIg74Hgk#aC>U+Le~&H*)mrIMY;L!X(3h-8_2qi0UJJNr01D znyFKhaNfeqAu5zFuMq$4AVC}*DPb8SnGL41Mo6201bhO?0AfN3#4KdW)Lj+U4XGhg zrU3AI^z^AT5Zi#%CK%a8p9!qh;xX?mEU)}jRJ7_2?yfJFRz+7ZcvD^XQ-<2B5E_st<^JKlWnq!fWDM6c+`hQ*3{@!K z_0M!`y&jUcQ)qH(sme#0Qu-#qMkxPU!kIS-bm7vC#)6ij8s3gPr%16d*ZM-({2mU% zD9Ak385a8L8X9)cIXlS7`4h@z2e`O=2yljKizGiqk8CJ|t&YGSH!zDm#}TnuoTpg) z{JaU}dlfdL_PS3gMe@# ze(drpdtD#|PX=Nl04Q-@PJEDJ)Bs?(WTdNP$5Cs}JDW#mrl!>EGwp$A_P1ngxzm>8 zaM9LeYPjRQdWPSHD1{Lyvvv0Ld=3?N;7IlrWY{p?Qyr0?U53!qg`^#J7+!54>^Svd zba7LTrsB10ow|};1d$B%W+3uZYJ z+fcZbjiEj>pBmPNAM>cY@r%G!IHqFg*}(WB^QY`{a`VUB+^9pHCNBpTAa_M7((-ga zDakkzPrpe(rxoU)v=2STZP?pw!W_f*G&u_IKm%tq33)lId{7~@u}&d9AOH{0tKC4G zeI;hxYHH3SPJMyMXdz+y;2REkX(u(II$sm!IJy?5R9If_2^zQC)RZkEM)AV@tcd+M zI{}{}q`m#7-4`Px*51YMT@wU?-H}6OSixB z%TrfRZxpFulI=+6q1-mOOJx45)Tv|2V{GMFFFc&A!bf$%94>}7#9p~L9mvecSO#k% zu5j(zwc0oVvedbFo9jx%km#aaH!YHD)u;NWE~W$rg_@SOg^wyS794{=0Zy9_CT@7k?5NSi#Kfwy-}!OcOB|Hg4f z@5ub9z5#m1vko`*NXdWDH*h>i{zj-4hl$uDIe{G=9RkLUj}BglJg25sHZ{^kvUh-) zBwHqFuH~O^y@?sJ7cN|w#!0m8eehocEeJjNZ}( zz~gTgB|JNPrI=Q7?v8YX_VABSDxzV4TSvf~G_Mk;e9d!qMxIfU>onV#=PVg4WTpoU zC>y*XD%l@MVb8+OOUui(yoXO$e<7{s|H=T>cU1%H8$Hdl7dz`n48@I*?A4RV^a8 z`aOUCHn}N|3wiQO&f%OOL^>+OX=z|>eGTz!2xK_4=vIA1^}LK!D3@~VQxdfRdc*3Z zPMsoEg%7iM*;}QE4{EWzli!2! z`vAc@3nk^{9~P4WTIqs9m_!Asb~%9sB%3rpMKVBg(oH*d*j1RL-_VqxpHXF4TtS=wHE&O#uoliD!MLY}i! zvhr43L1Im+{r6-;)RL?3;PsCpZ%9OQ4q#8rgMw$bd-59^wx_7cYicr+eiNYSY$%XX z$v(8O3UIAvQ+V+b1{7c_Wbk!2Mn1ReRlX=i5)d$fDg!;Og_r1o5=D#mSJHgQo(f8v zgD*Mc#P^USAB>F=qzSdJWw!jT3@ML^&y@u@SK3sLdHy&3fX_$_4Z*ztja5z5CEK>Ifs^-srhY)efUsae75W{ zW5Hm<=G4O|qknk|wY|je&r;}fl~GJOO!jNzTl(8_f`LxaF)*9~WR2YB5J7_g3JCQ^ z7sj4L>4L*w`11Ju#$>}S?aqthC@z?RQ56;zvTKhbS3RSs7zJnM#dyR6kV#&?aA05{ ziT!13PT@xoEGgQN`5#_7hxCb4Gy6HJI6r^?`*k-?#W!5YyX`bH(N}^J$SXKFII$gN z67{-u*HDs>d>_3wlI3kt6g#e`$GT72>*IC4j^DqF5n+k$#m<4=xJ$Qz&|g{%~en=;%n}z!tC34nJru4Gn6chsntTc+G2`q_hy=8u8SNfp>YV zYB|n7f#0WQ`r-#}h5SP_{NEanZhT31LR~4%wDq}wAtWk!k2M9eky3Ttb(o&wjT;=m zRwhN|PoYJR19~i0m0Ve`<+pgPu+=n-jq{fpA|g3h9@5bx@AZOKw-*qEKKp`MS<+C; zf4DvnWgLN_)KYF7JiPZIU8fp4<>dIF#xE=`-i+1&5HC`YqI?g>`AIfv7!0Bh!CQW| z3bp$bD*v+>Xq8<{;BlO~ z2^t3B4Vbm((n|fF4BL7B8P3)DlotY9#n8}_6 z_oY-t7SNYJf%b9MsC3P^pweBvvn4{Ue*#sj#P}UisavBzivQh)+j>N>oaDb*I;?8v%mC zp?A~EUl_9`Am9d^cSdGCg^Owx7_gP^^r2r$z1k%NEIm*^S8P8mHFZAT=ZA?TARJ3e z?ndDwH}?XiL;Fb(GNI5-*4EZ!elOe1^bb3+EeOEaNu_-Mc}SCHWW&LU-f<%o$Tx4^ zV6$viH63=r3<7wE|Dx=9ZqbhoS!aRgVJtGte)=Sw@zLGd-M+sh1tlL(uuSswxQMo( zL`1fYrJ2d{`JYj1S1HtL=v2u|e-akrv6{%SWE^h1P6m zt_@JJp>tHGS|<{<|G)uIIOnx{&KFv^ZdL!XVfXIc*xRQcFK?f_{#&^#F3s*n!^C^E z$H%LlPH-GK^2Dg>>6Wcq@5IEIWTog*DL>Xu;sBY8E=1k!aA*yOJs<`$ykO9zm%M!I z3|puq4{fL8N+EWBWUM(j)jFKQI+llf|gPB%X!{eZ}_);Jn8y>{R3y~ z&9g#kjEjpR(tLbqPM<#Qw80mJSRRaOP_YY{U^=sWh(}}Uo{|FN`2ON7_G`X7<)6D; zIDW3;n|;=&x>ThQGji7M;X3Xy%N~>VyrtQW_Ctw*IMPLY)lY|u>^N}NL~=>{&}@uZ z_k;QZr}tfHYzo?n6kBJ6lr=8ClhhB>bKmG8Ep@JMMFzw~J;MdZ6J^ z{r$^}juE%AVTHz_b@F|9HA{O!BAKrLd&HYJU(-j)2I&PlkA?yQA+fpU3*bn$($U$p zq$R3zfgm+On_t-m=y(eq-AM-r0q`{hIK>l*mb|h+WbSN_uPmmdHe|tnL2V4io`uNo&jRFJ5ej-xW7z7cqEIW!WiF z=;*3s?o~kZ-+)hE$Wcn03Z45pJWQr{jGRA}aHg7np=A5uB{T5cbl;(%a>*q(Qp3m~ zLUsSq?h^`EuDnq>B?b@zOCfp3C^Un1AgXCu#WdLtTD2zb05pVy^(aPe{J_-IRPK3= zgDcG^nJ#v+{iAjI>lYf;lUjCU6gIZAqTT?S-367|o9O5nunq>947Snwo3*p{Ifj)| z2y3`Ad5AFbUSA_N$H?i4ml>&UmO{s5UD^^ML~A}wJyyq1zM8JTVaKfxOWd~}oXnw_ zu-S&6YRm6?Gp)MX;vmhFIc7%DtcmkZPD1Ezs+XNae{P(R3MdwG%G)j3U5mX5UKr#d zR=t@b_lFagAaH<;sUHv`bRn9qmPekM_7vtF`Afta_a8ni9QrN-2;wU$0*jc7(uN03 zO-*Jv)A7EEEQ>UKBO|6iuKkhEm-)YMz1?Sp^Sl52c~d*4QXG2bkJc&a=sXM#W~>V4 zjRD&klEfZ%sC~^zsh#D<|^HG$=9b8N?co@VY%%^$h&nXhK;W< z`8CvwFTI{{?vJ(AC_h*E#kERDwx&Vm^U;7ceb>nVPhkMuu zcx0!rOYh&i_X-WVw6wGmn)w93py1#N^g3joMtzVuIX*Tf3rGFQigtWJ!0Z$24tWo~Zc6eX~AkZcnylr&X{Uh9QL{ z^$P_~R}0)z;w|0XuJdk}><+>AwgrMi1}uVnRiN#~%(VsMJo;f5ypaOIV6NK*Iduv|aR$UNqdPd%U>45$qY&ePLVKh-!= zS)?~=vvi8>zL*~Z!?Ft7h8!tvA_lME8C<>~WR;&!f9ET{S?=hlk~#MWU5(9IGHXtT z^9QNOvI&KHt)JZWiH{|wSw)GBN#SFc^eqc66>38+Ro0m8lkY9LO1`kmXyHzbd&Wu3 zQ%yfE2s_%9T{+S*s7Ga6a3P^(?YeaxMyMBGgNrrqg&tsIyAATn24^%olZ4$@sv9?M z05{ym%Bny(GQe9KfKQZ_4aN^dp(7h$5p@_rQN*cd1H5Y%xELPKwSZ)m|NH%K9{8`& zT~;tWaNt0JcR7S7q>ILhI`GQ2v?6Xxt+sXJ16pNEi4@hJPcz!(-Z1W#inJ@P;Fs2V zspq~uN&8WP)K&b%p`;{v^)(&d7ur*|^ly<0=l7B{T#s7T5X(YwudeId^|scqL){np zT+B{WBJj{QnQ$OYVZOMBN zupc>9HwK@WpPp7O?So8%LoVc`i;FNmvBoam zk#o2CQK&=IzrGz`eW755p;Q|m-i7&SN1aLS)t@`Uy4nsTzFDqeiDf8!8KrukWOk@ZbcvV1fQhhrZvYIx=Df`$nlki`{Cpf<7L z&?8lq_5lm+@9T^3d{3}_sQB2##Fm~LSY5j&TJdN%k>*skVF4(qs=o1jkK=tKi?ILH zlnn$v){Q~b|0`tuFJ)FBRB3<*^N9lUQ&W5P?UQoVyHZ(hoY5aW`~K8~jzQO(rK_>3 z=1aP-gB%yjRjD5ioA&lR{(48Z+1C7Z;nR?J^jR`eyJGm7IX9Wm7@|zwN|4g}xbtPs z{a?vR=MQyjM9c_R*-!^$F=5|6W6b*)D1UpJMd)y|{e4xxKE1Q=?>8qo#1!aG-lfYq znx%a;XS0D0^H$U^>6S2zv=91HeSCbj(&kvS`BCn6C*K0?Q?jVu<;$17Y;xU!H+Xt^ zHDsDyz)KlvirGxaLO#AJ4&Ata-5cKK^3{Khf3vjd`1rVGTYWnSn&xKJ!Nrc!x27%E zR=?;}Z|8Zta8T5j#*f+g$ITi;9m`s_Dmul~vy67Kb+56pqh~LaD@T-G*nQ8(V>8~a zZPQ|36mKZ6Ae;5JIi2SL>hAYY;Dag$!Wx^lKg-~|=7fM;!Pc4Dgh?L#lN%1W3AshiPXO=mvGC3^1O*^NSpSAsuq-L~@XTg#BJW%G*@ z;lFg{F;`c3V%+RznzVny*VO0(#)@RVx8#Z~mlU{T=$FsC`4NpCJTSQ4ct!QV<=MEe$@vx+n6vpT%Fx5XsTYMlT zst1$;)QM|X3XuxN_svuQx{_(JA*Bk0iCJgnp~=a~W77{-aDUQ)*dG7i#tbS4k%UPM zGg^W;!!7iqn8vqn->&ppRw~A+{M5voLaPj|j|U}aW2vT1FJ;s2Bzw|dnBN-T)hOtb zYm^Cph`mkAV*enMw0XlR)!1og-_@0+de#-|zXpbtoF4OtDUzT#9ep@mwm+>iaiQQ$ z zY!J$k05DfM@j@1tzKE#^^uoM8zq3R~cYzZwCRhFsF^_P3w|K90` zB=+wLRBY1hJhZ+O^daO6XLLX&)xt^xBqc&64g-WHi(IhUjsrv1>t02;oi);x?BQVF zuW+dQF1`S*`GJi?I_a`or|W;)FF*9UTsIIhw}_9tvACmDv)iF)`?K_hc*g}{!PUj4 zhxWd=Sxs+le%z~f*&_PVU;;TCgk-ywK9pI^`XZ>`u0` zvojz6^4PPy|K(wE@fSFA56QZ_ivcMI{j&Onf<}b73n+_9d;H_KkTa3WJGd~qf_dTU zOT~(h_pai)`@Xc4YQu)S`x^fs&XpH^{eL?*D(!98g-|&TU_nBm6Y0eZmIjA8gPw~j zM8v8YeVIg;c%d$89lmF6@cvc36YUgOi{=fh#O7`{+gW~G4s0>qDB@tj&vHvniiJ6d zHlyXm_dkP0-UxAFrrYG!T-H? zaKZoj?AIJa0#0^z=@zEI0s*D8Lq0o}A-grTv-)O@h>rSTsI~*0;&loIZQciR9bcbv zaM^Eu`B6u4=JDz$0}8#ndlmiI{gO*OC_WyGRq9orhAx_8Z$ps><$e_2r6|Eje|iGGg$ z&qjYu5N$Up0Uss_FgZcoDA^5_C_EXggL%Qh8c~J}^z>??L*U>7U*$m+4atno4vlzk zuLWncj7+}e>B-^@GytK1n{Qn2k*>2;_~y-8;35sCh2vvEt-8NjEQJg+Is(feAal8q zv?3<{t(Q+*DY55bQEbbo%*#s?g0ST2rD!l&oqOMVpm}SZa?&8)QH2ooHMg8FkOT>@3wHSG{hwD{b0E#fG;{ zFCMa9XEZCv%`vRE^uC{qi}FAG{!k%{$H!yg`*<7f<1F@vkT9P6L{pCQO@}$^xbLlJ zL~h75+@LV3MaSos)>+-`L|dli7V&yauEa{Nb*tFis>bUnB^MO9dFi`+^k=KIM_VFOQC(Zh!O#EL({n9YAI^u*f`Zf{sPi!PZ`>*L z5YL-g$kJt7iq5^RivT)ZU0o~fre|mB=lfHPGoF=}O0xUE6%I?P9GFQ#xa|wv4esRN zKm>W{*nMVh-u4Nep-@XwAL#8R`ULHy`Fz&nqN4rd;}2WR-}aBG?W!m#=}V_W+YLt! zkSomAs+>XTfs~&Zd`{;T%~S+Syz8`J@bSVIA}|=UZ{@(9JFC+(GJ=LRI)`NHcy(d# zvO4I}`pp5|+Upi`oOvZ39eJg8QI5gM!C&0Sc1H()?*HjX)(bHEqL%me_pc6ATQuIG z&=)#As=H1Czz}NMqTq6TecyQQyK?POYTE)nrj01R9Z5bVQK(Gw!1`3p_{JwDblI3#`7e9U zbl22OJiu|S4*dT#pis0ZOKH#$0u9yb%)VNMBipPwb_9>>6_pSg}XjJRce<~EE6mB+RU@msGnE846QN7M?1*F z1Dj9?+Ah@2z+pz-;ACD-&UgV;C%pff8u`WJS}iFWzkb@W3x&#ySR>sdCFOgT^FJw4 zslvbV4<`<6nfmXF--roh2jgqv6cBwun~@b5`1((?lljAfV`$+g?@hRevBzzLuh0}($lr`?K?pu;Fc zn)2u;PqqR91o-Q;<9p+G*R9MxV?qA@N@jdejlG~Qu>p7<*UUEK(mf(9)J;UV${EBg13|Bv07i zXpWx-25yJ41ZxtA$CJ=dS>WjyP=dKjN=l#z;AERByFC?E_Ze%#$*HI%XU)kt&gGng zn-h3xb!&_>8t?pW9PEEE5nrgz>Tu~>R)ViuP@6o*Hk|H|J4S%bj}p^K9(~<5hP@gA zN>ao9ZJ|m*I>_gMfB-4{&j4)$o2TCGlDm*lW1JwM#B&p#kdF`Oxz3$CSKQID1HlZT zp*sO25io^7g5Vi+es9)TX;ts-+`j@M5E^V%ia3{V)d@kX&FBrDB{Y~HZK{8}&Ai3S zdcuU}Xa5A#L-nKsY}QBTboMsH&ymLT)Wo@6!^GC&;e2a-P;%hK!aJ)5MD6r9_EB&bT0dPuh+TFt?FL3bCO5~1>0-}0sB(R_rIl`luSnrlhOmC%o_XpJ%(Vk{`Yb#w#G^)}^WfP|MX5BzT{xhlqeDeS^$Jat#>Tti;^NkGQTGxKT36_{P`AI^ zm(h`Ucrd7KUI)ISpy8+46_1f((??vU^g*qZK|mtnzQ6|`AOXVV!o`Tz z$^wu!JD zNIHO_M>!!ABgE=g7vDW)@%1TdVUT+pd$N1A+(_Z;bDzY^J>GNJ0fPq!x90AA`J1|8 zsj-LrU0xVO`Sgn^C@}j21yZVr+dyGLJ^xDGEN-|_QsiZTh=omCTV%7!tF`5~j>J5t z>n{%q4D9{-^-7M+fb99G>4^&C47JWl5q|zHr0#sU3S0`3p`7$uawhQ~J$h8cVWgfu z!3!X?ZiIF&yk4NU{NQhZ4saS&);i}?_qysJX{rH5sRaieN5Fl^yvVz-g;9)jX#B1- zS4P`XZYC&RprVM{yYMACHufG7nK@7D=v|*vLnBv~Ydhcq7F- zcuyK#jiQm?5gX<+(w)s1{_7;>O@Hg^vV!n%{YPvRAZYb7x7%g&F~E;0l_DfIcjefi zCO|Qz1c7GftD8=`_ne^^WKwEn=<0}IxOLdZ@uF5|MjAnj(u*Wi|J0gKZbQcYSarl0R$!pNI|}ri!CbuSqOTNuf9S2s!=2 zXhI4&hx#N5-Id;Kns@JiC?cn?AhYl1&SH8-My`ptDy^#wPiUOBjI??$6#C+8dMOOZ zTwYcO=FP^+Pr<}}I*d#7PGsvY#p`?6hs0`wISewi%zAs2chq_N9`$llj(8m|tLpV& z2%TJ-!M=qQ*Czy6R&76#r+y41Kth`6a{*g|J7mC znAOV%$iq%&)t$=^zBHbXG4)Jj9N$ym_&>fi5yu$BZd|g^Z^S6RBtquk8t!gl=b4T= zk>W{rbN7X4IktWKJmBD*f-B*OS>t|aCG8UwW$$(A+1T)XFPx>_c0l0@EL9HU?T_Jb z@PnuFDE5dSi-;tIrb!a=|2?38-9M?2YHst?zA*BDZ^7cyaU0-7-ERiJS(=YNWkK|A zK;XL<<>5y3g)2S#vmhHrOW=4Va56m!EkH7kg(&ahr>%hdFaeSZ`wv@_XVQ`k6DuD_0kF zFee4-J{hULnLMv5sV<)cFAooopl_QhcWhh_35q<1KWIAM9%fQdR6GFjHl|&=?c1ec z9){IPg%#?H-q!B5Y|>dQB6glYnO`zqa(|ps^y@)Og2pz0k9b&Awp*N?or$gl%J0(^ ze!GZ)0x19*M1BO}@^X@d`2Y8q`uk-J-PAg8bpsS=qA!v=4Nqygjry!<^DLAHaPd(f zOm+3Zo23a&%QJxBUaJJD`IZe|UH>ZLZ1B764sY$K5>&bu4`ESyg2YG8Nu;mJ? zZe(oqd8E=7C4zy!Y=~vX_fW3LLf@!TYkH=bs#D9N9s|>XuUA)FE{IC63Pr#2`g!Fr zk>CGSfWJ&kZQzS{jvVHpp z$hqUhp1R6mH2MV*!82joy+kh$i{ss;i;Vr&Ojn}}~#XS($uVUWyp_%!W;7Co!D9**Tn_rG%bWl_K# ztEAkK-N?z@l_utIK;MMF;?C6=O* z(!*UyBYE$lQP~4}B3^=B2Oi!JCr)BsUVw9gxJrnS7OWTM@s$VB?)BdpN9C)=Nq;7? z!OEVwAU8KKYa4g_{TkzvmI&JLU+Kqw*D{n}1}QPY@*tpf+riLFnvsnI$v@wpA3IgV zPY=Q+pj@hB^PYtLte6afC8##*meH|gjqJgsUHHSu-h_SV>KU)+6m9C2LV%V4Arx5B zl0}9Jv$tUX?Q1-b!#;`JQBTOaIG9D3!v%3B=4a3Hn}*-MeS_2-O`ReX6Id#Jkg4H6 z*bvKWg7<=Wng?< z4jT({$M?jWCQVP=_;nJ$9|A8W6<7jv?#1geE4V`crIY^+S!&F{BYGz%^(N9NfU4e8 z{31JBtdAx-QZ3jk#q9K5eyvS->q_ySb@aPWrP!{sbQy)UQtP9@2b|<86(VvwGCl^L z2A<2Be+}I+CkyYF5bp*R=`}yokI~m{pDgkArF)cIDMWg>S5B|9FAPxsj%L{P0eqj9 z_l322Ih1>#`*A+8A<{E8d8WSII0FfMW{TGhE4e(XJo8iV9T91*b;DNxi-L zQRykUxth;T5bXoXALnfw6ym(RJoF zb(nz${$2H#|Cs%SyZ>NB(&{drn$R)}gPaM7MOKB3j7)#=i#?BdDSZYg4v?4Bg*Wp(DR zF}L1#C19@!Z4Xh967XMr;XQj^N%D!o&%FYzc+EFJ>5!9#dcILnh$m+W^3%YxV}NG2UmO{PLD_@C*XcG|kEx?%Dz@gSa-3+;kU-R>YYt=N@EmNTozBkeKoR=ddElck+lW;bGHt5SS51hoKqhaL1 z9)I)Xv6T)KO2T$u@9D;z{Lgoj(lvY244RvD5Sj>u*6^q~t@kLgvaSS0*YxVt8Wxty zc&8+xqtCnp$etWJkllG0i%Qx{d&9$7-(diITZdY5N?lOqZ5?B!uEx|?`8k4XUX|Ry zDE{TX-T=Y&UL`*1xO`OEoTs>MpJYev<)e=cgcYQI)&jy-l^kwbcf3C21wXxrzt(fJ z;NO5_646Wp!IhL87z_0Gw<)uM-{%%&+y|?0u@V(vd^~B0f>f!zohQ5F#A-i&R4VPA zF@oiw(r|;97v0Tc*ov4YTiWX$`CzyDG3vS8hRzC*wdv=4L1I(Dcu;t?b954>h_kG& zYfVf{h+tS#lL;!aqLvsVXJ~te(7YSL$@=5_mzQi(-o4}F)tHcpos9HSM8g6p*hrE@ zi~0GwkFS;ApM*ED1Y*Gj= zJC(I%Qsybjt<_1iFI4n3&{qH${_;{pCeRL&?yX$-3~19QMY+ZY@WLfrbMX=f#V&p=WFof zGkY3cW8; z#J@z0b7ygF?DsIKV(GH;ta;@=pbKfcg4Wlwt`azPlZ zzp$D3c`i{;kIu(phSc8nRsxnI@(K#HcwXS?l;xqUh)a`6yIC4Ws5&t5_lOMf$A~f! zEFkv-DHH5sVmfimtYR*F_~BD2ZT8?8*je;2$P(y~o!tx5=^>)n$c0!L+#37AgCm^> z;J^OMw`TpuUMvzpp=F#TOvsBMjvw88hgFsFelM^6T1Rw~z+@y(oeJN|l%UE9I-f}D zDO~JWe(d6Vx?zB_32@a->C=NaK0*Y6@G@{qCaK#)Ci&yq+ZlHNTVk`-M==xaz6D^4 zgx*9peIwq*Sh)S5r;SFvfR=W)RZor{=EPYDxietvBTf($(K;B%NEipqZic`>aFX32 zsUvn@&w(ZEfE$)1ZUD`KykQ#s@GLSF`CvC*zmdr%f%jYdjeq$~bqmvvY({!l0o%L& ztLJ=EDxZTWCF;RG^`wql4Vn)EBG-H~i*s>xY+Q(G>HDPVPO5^O%%TPfVg05+)mLm_eIMU(jsg`0(uu>$tHMOYW1)Q={V3;0GX66k+Qa4~|^KS;5t zB^W3Cv6kI8mx!y?{EB73rw`R<&JWo&*jKlg)33Gn$g zLhk&COWw1!O26j8cUMG|XlDJm*qRvP17I301rhbaI5uZJm$5+~7%S1>|0fEA4qDf^ zF1!Z6*mWf0Zr}vD9*HCgkj1E;s97S*`&RnYsZ|sZQz~cA&d+0Gcwu2eRe8}!`}w<- zsfTQRq6g5*Fn;{vg>Iko@7!ErSQV>5qM3H*rlm{3cG0Hi<_p{96cRb6Kkj=(frzb> zn3X1a$~(g0$8s5YT(LOi;k;@^l`a0e_v@?DD@W4Ix_F*@9b-d@&gQPl=LVJcV6(&m z1u*n{hi|2D8`L5|@TK|RT@ny{VjH&z*G^sC{mI;Q6#gu`CuTo3_PON|$RRp@&GV@KVg&nN$|WB_RpHSg`L4oz$H^HL-=p zSz4gyJu!8hy{Z*71GNICis<@4FNKLa+Gn%-OzaZ?z3Lc zI~9CIY^MY;VtoJ+kkPS}lGeK$;-nV^SyDJt$%WGHG~TZKjq(XeoPcTq#})DR zLC*}v$rbm?LCY_POahEUvrv~3jupzEVmLG&=%u{-q~Rh0&okE62l$>B&=jASmnV%1 zj2^VNh~=CdoUoSZIfjvig@7^Qh{b+*ChEvZo~5PGU+&?2bY=trf)8cu!354n4;~oT zEla^@<863A(I8DzB*?kp~T)ARAFzjY%e-Q`qb0}8K zF9rV-ZtPh>!s4SP2k`MV+*l5Vz88SJ7<-rBxCf z2qe{MAktb+&uY~Q=f^Qj$KkW;Qj9T677t+4$J#b@f}-xavkl)A^z9Fx8Pt?~FIxLG zuHSU>CM5;?Q!zUk#et~J`rb~Ve+`-jx8<+KG{TB7*-`w}a$y1-$lowa79YxsM^lVgH{LAV0T zgP`HyP_S43+CTtEjvP3wl11N2>Fd9m$%T}+a)6w*IIPlPV1k?ME^rGNLW$i6PU!T@ zWe+a0#^NA}Bu-_>7d)VnY)Ugc3pDRNtcng=jBhNywInc}`14~M+Sso!IQ3$>IDq86 z%*;$?(cfbx866AwFT~_RF~I5`l44Sa^zt;vVZGNE1@`g2yY-{m(o+4R7xWCHBDF?W z`Zc;Jk7|dz4($NPhvE%w*KVV)YX;)!nTH1lI|jD?)4;665OP9MA9T!hydzvxLLwZy z5Pv1VeXo|Amq*l%U;tcyG@M<;R#%)tG(kGzGF7DWR5_2>BRY)wc z9Um8>wwDvubb?L(-alqBaJ13ZL3vw$=MFgLWup4S`uJ|J7Ol4B8>beWQk=f8{;bo0?* zBru+<{Vt+rtCSqTLv8!mE+`l?joRIgvzeIraAY9Uq47UY%9Shofm9D*;)Fr)IhIdT zqSn#9d-s|)MQ?)HhYo_F=6JQL$zh!1IX-h^ZD`)nt2^zSh0K@quEZWd64Kz(2NbeK z)!anS?WSN9!?Em%93}JY2>V+pH;*-lXiEjGHy=vk<9}`4@x9xpHPpo=)z7h@I#zY6 z`e8 z>G){J(Q*Zo>jQ9pzz!JOD7y5nsi^l;#cohW%9^TR>EE}MJv(vCYTZtUvF_y3v|9efM@}eXPUh- zZZn|K=6+b%bbBUjEXUEf^R%grt;%%G)@{zt_PxF5X*yhmCMSoAkIBld{hhFZirS=Z zau>l0$ixFvWk6V1H4Z299E;d^1aPpzEq(>jaZMa??>n)6he<#a zx(^_kQ`{}ig+zm@RWY}OYU`z!!@CT25miSevGOM%{>YT z*hXBGYbbkaG>*Qz{PFEi;>ZzoDlF<1K5&4NmzS3UB9A!T-{tW^|7z30f2k{yFir_D zulns>4!arhR$F-@&~O+6p(NpHaHCz$(ng>ouAb08@+@|2;;dXn5sE^DdIFka5yu-e ztNbSTp2qc9{6+Mu&E$j!?~L7L5|L2ZAvQU1I6dL zCPFP1ZajOuHc+PKFF*V9%wA*O}8L7}O5pn{8MvZUXl`$7)=I1$h zc@@dnjYdcL>c8S*zE>a_gVoi}%sw2p$@7|w1-NpDWM|+q-Ow8Evdb8AFFDnMpre`} zy{BN$rpcMLU8N$9$1&`=s%G%TYeVU-{yuVA=!an*@k=M_UGt!)`j41Yb*O@L93^r` zD>=guBKz7Hs1xliP~t5g7dV6BwXHp!k!L}GV>JPEuP5+8_U+kI>}PTa&F!^upin>! zf&Y#(GlNflfv$j@_}FFd#ecp#8|0CIw=7kG08drb2~jdyH7;q8G7=zkNhcL@#ctQl_YfG zuxneGW_9UZ6@KHF*^PxhN{I6j>Cp!t?=UC@GOxkLG7{fa7bdbTw5A-_+Fc@PlrtTw zt49}Y$G;vD>;4@3{Zeq&&YZ?0M^^LJ(KKx~GB&d3VBeIaF6CmnPe+$$=}&t$*V9+U z6qC%tF#am^4h_j5iU6&7CIT-i+aT!T53M26sI0QWm5w&|`X|o=c;g!&Hp?8%&@k0W z(G$ne)ehTTRz-fGOq66w8&Xt|(NjfVcdR+V*{hi8WQuJP9aC@$&HRX6$Fkn9V(Q!c zw^HndVGrP1TP}Go)BViu=(~YxQ`Af6AITD4IA$cC@4!zKJf(a;ewq-|Y z-r-Q)ig4L~YmBOXaXc6hYn9$596~q;Vqo1d&C({* zJ}~=aQDcB)gd{|Ki~3MaZMEnl;{6*F^JQK?J?x#dwq z|KwQB(owb`o6gC~zQ3GigV+W0NB9AFiHU9T`EzZ#^atH4F z`JI7JTwB{bnEwR&9HENuZ&;wb0hnsh)E4uQ($bAogLF7Md(;Igh zR;{i@H$3_e6~(m<#}{DOM}WCTPzs5yLC9tl!`VHstQ#xTUKGl9=kG^o6VyoiI zD8t}MmJ?2a1YJjzcwkMa&N_H0fu%#1wvs1_>id5QQEJ}{Dk`R5io~&HQgC4-6+gjx ze|~>-_wHHWbr}fTq@=uSVGXli|L*raLFGuPAaQsr&FyW3zklWQ9-J&KhY!4TBh4$OA^$tJB?R;u;fvTe*%?S2e4xFX@f|I_=<21V&2Crc^5(y63>f+=~{;% zkqeNBV#IT$0wl&NI)h%B>#Z>wwO5w3J)wx-K6JYunyPqlJq$R+x(*R7NkK#iU@vkn zNg*KF?FhNR`Kk}UYNquURx~m&*&x?8Wj-z8GIl2gXy)bTT@pyYIjox>NSJoO@g#1V zgumnW6N#HXUn+_Wg3!*`+D;2rV55I0fH#Dh7{XSiUjHdU?el=jjOpXD)LBR|_r|+6 zQU6Z*Qt0tic?*j;D|+y&L38NQQ3{nU`RdcI%Dze*%}tHHcP~1ELhiZ1hFkdZ+ZY+0 zYwVlH<{-QQTXhgtPC{^(@{rg+&sa|gFFs2R;GmjCHp(GubId!hI}NGXtgJCv44wG9 zxEV+uMxn;}=J{W&Z6BLBhhSA;K9Vr7&aRSTeAI!kVytB6HZBCPS7dE0bsnxP| zQc?b*NV*o*{NYkc|JF%?fp6-qck&J2c#EIq4y((UnR@jg>5<}ot&;Ug&-F~rt_qYi zxNu$FEASykRJlr!(d59{K}Tphk%2=jXx`pO&~Q^gvq<8ya3haC@*&_(A#V}sgrvg; zK}X&@2{ANd_Cjt1cI5DAPySf{(2zH%aPYuJIE-DcT|=(=t#qfdLA0NrK7NFA$pci) zpar0HGcz*;7Lm{u3LH2P?C*yanx0FYyeouYNh9KkM8G2BjE#e1{VlZo_RtZA+9iS5 zq#*QQTbNsJH@vGKrNbizx>LPEiXgl#$t!YPdEf>_vlsLI6glfW1vKl_Iq>F(dfwZ z8ye%M1egWY6MTglxGCCGjVR~VjM@qfvREY>HBsgvE%L)QMLu8|B={7Ams`!4qm$Da z#6-aM;&AgHcAd-Y`cv`0urkte0}~*MOT;|obq;L6YQW=t1JxJFRSkIhzXv~?veQsb$$n1D zE%240z1_LHbfF};E~wy8Yqft5eSvPZuGMSh>w;(dxihaC8(CTMz_FA+B=YO`@1u~6 zoq5D64*@I~w|)#V9NgR_VzU?79NOBXMs%^n4H8nam4AV*2;^i0@+jqOJ=?PL)pA7nWRX8^{7=#h{D7j z@0o%d6e3XnfkQwUZCyEM)tUM2dBilbfk;N-JwLx8jP7LErPs5DjhP@0xTzotN}^VQj~Hr77wq4wUvrpv zHE#dag+)QZ@52H1rXt6{nV$w@e&WI+$NOHjG>R2&K>A|Km{5~w{yO~A_7)K%zmZvy zyym&h2qR@6B451A26Ujw<;ZvxEnSheP!&A>FP(wPHzM(KJISTOr9VLu;ylPsT?B#V zf^=JGg;#PV(gs!(L4SE5l0Y3?(+)Wx*c`r#mcrMZFAryL5}UTU)M|OGDbm;brN*-n zmZXyNm$E0GUS|-~RcoH+!Bw|6{$yN^)v@|?=I9Lx^TKYPwdpi%V~P&UuxfBk3; zP$cA4sK88Y5gyZFNrnp~l|My|CYoiGaJT+99yqQC&=t~i z<#RmMcegKct;>N(}e34>Qao84h)O2WT)%Ko#a$L}H7Z2G9~60BDED zG!2xlDe~-QWZ?U~eECJN%L3U;TUl5(2CCWq{{0)t$0`GD+LPv=RgpC&2kMGsS%qMH zL;F-^bbC4qsw@(=L$U z?aB$Z`%)VnKH#pvv$(iUfQF`QE~T>1_o6e>Z60VxMZ3Ca3C?!Pca1a`31$_FcVt`e zUcGoa-SxLsf4~zFm#@N7-`|Fk{H)6#?j_X5T%XQbc~@lAJl*pLS%Wgos-|L)QR32S zdJ;khZFRD!;&UV_8)IS(1BwisZx+i}Z+27T7~4b~!e74BmxUaE zTmKl}2wU8=tKbnhWbGc8ehLgg#&dc*CzFNDUE@u+sxl-Ge(kztyj??Q;C)54ND}*re0IczU>eeJ8&5zu3x!P3Y!D8T z@N1)oh_t$ZB?G$_SkNft`mK$7=oEQ*ohEy#XzC=QEbV`MICQNn(9z}4!NdI@r-j8n zf1ZjP^t!Zb_bW>)!N%bAmotS7l_{a-#dq*>DTj#is>XGT1a-sHG6+|g1U@@p^8 zFO}MtC+hC2o)FDE7#ic^@Rx?(9~*HxCGxdfX!^@TK0+bW(<%*zb1z3(+)qp7b{sQi z92$|AFc~~o7}Tl9$gL=8z^uRJ}4>z471?V;h~24fRK!#uq1~ zEYhU_(g7vcozKVljaQH{#=mL3l$Uxl5-t$U(mBVZK{9IA)MK2L$@zvjVckdHASXwj zmuBVau5w@IzDYZSqVY9n{RS?zDY_Qf*WFke`*k@@dxM!~Ypwf4^wovd`H0Nsa|yBO z@1$-fc1g!LCD}UK8vhyEC4Jg+AhwO(3%3U3LJb3p|O+w=$^Tzev z-QD9(n^#W14->K^3Pze8H&meUg>>U*@mJiMW(bX~u&{6{?$apn>9!! zqEkkU++Sw#@J#XT2+U)I=dA4Vs(41`HF&v(LZO69=i;JKOwkUj>c^hIL~`x37j>WE zef4y^#ElR0mYm0TYlPIBzS5P}G7Zx(HJo#zkqGDfF&0`uQ!UL2 z8sGC*55&Xlb)i$bO#pK^Fdhm`+~EPo#<9U%eCVdrYAN^^jjKDZZicXY5ZK^rG#AI_ zJS;Fo5)2tv5v{5(bR}{yie5PxW-qVP0-H9i(*GPi;VH0j@4Hw2AB_!wm@Rkb2IV`N z8y-BQym|n|XzSx1{ZE7>dBj%Q~#u8m6MaJ z7OZRQ=y(PcI#lRg=E@|$;JhM}OD-)bIc4_6r0y6OtNYX`Sa6Wzr4LTaX{QF&^mhm; zDhRm-M*m}V^_8ANS43#GCs+8BlWO{Vbc1Q&_gN%JYTGlLT)CnIFn|ImOKK-egDK|I zmo=%6pIs25w9VuGUDMq)JF|H|cc(r_YVre~2BQk(j&@z0T{OCbJzRrSH=J#B$ zL5pqm=9Xvzx*VT-kCA&pdIARk4Kx%uTT`^ATv9|9zq|;~EzC4tA}`*)eG99VyYH?# z;|zX-(zTe@u0jb=#d*{EHJNJ+!8afU?k+kA1fZN#PP?7EAT63@v97rG?Qb)d##>)A z4mntMJ?So7-D=f+M*PI3jqFi42v!vrd7Pb^P4X=W<>G3!d?jCXZuD1+K}Gl0r<*KtKywzP7(fjx1a9>%!BH@@qMc8dsBD zNiti}o3%SyvVC~h1|nmJkR=le32rdjA%-i{Dp|Y*S29o+oS}I;0DDAiPguE9D=pgn z9iTeXs-O5}awtW{G7+o9FWGO(SY$#9y zHh;vCe$BuHC3l!yw%tOparb8(Dq7m2{;>DQq!uZ2a9M--I1}k2p~nFtfo<=G&RyNG zBL4|4eWCt)DNeUQhy*h^6@qow=VNjK{{)sbY5I#j4-TH%#wb}=1a1f;a|)`Gp@mN_ z`Oe=cPF_31pzafNVMEOQZ_R#9CMxaDQ+&&FO9u}gR^faY@TBf?w&v8#53issM`b)7KPD=)O}B1etW#@fr@KV7{H_fHVpOt2gq~g;>O{p(9S1am~-m14=*mQ($H78ZfRrC$;7) zrW;XBZxY=N2SR+Eppq;B?*JSHlQBdvMj6rW5khy;h<@31WII9~@qm&}f?i2Zk+^kVGDy3Pe^I>|1G z{}w`pdU|@=vL3AzW4fUKN=3Ak;VnY%KoUjLF=x^Aw|?Y@=F5^~;=+MqfcpW7?uXPL zvK|0F{fuV$f@>*yZ59gy&|)svB zkJ(hbCbB{a-58`7?U`2@A^H3Ib!ZC)vFHf4N{NEYo9R@)yi6Qt$#*yyYH=Ls==Awd zfy};Z7Z&RgtB-MT(Y^d|>2vRFApDVdQ9{LP$7V!6V4?oh)aL=O6rkLSq+u% zDsqBg&`^MQA0y~xJZwRy!$@MS!pE$JnWY59jzm`=L>EF-Z!DKB+qXMhs69pcln?I| zF@AvI&Cr!JyoHTMu&h^yRpIabP66u=6Sp39i9iRIgqq*w6FMQmE6XM4O6g*;0~S-h|xuOZr+lo z)oWA*)?1i{5sZa96fju1LSaL0P18)jvH_F?2M)jjqgf%hYuAWh3C;^?$TX3a1cP`% zcB}-vgphJv#<33U8On-sXj^a|g?#q2jU=g&KUu{W4SIC84)|?x>*XuiwEDb6fDEQ2 z0jiBk+1x@*d`vPQt1F|WLQReS+C&H4Vq?wy{R z>V+c71b4ZSBuoN0Vz}x0W@%lG%+wOR$CD8GK0EG=a61PBlwqeG1sEp)4$cBBt&dgS z2dfPN^@It-&%?vxByz?v?4E4zvtf?aME#bA<+b(6lW#3a+j{iE>9arQnVf)r)^9@l z&{(T=k6%Kgpb$r{=@@K{^i+)d9_o0_5~?_la=aMuD{03z5TnJagv= zXM+CM1!1W!20IKFZ2)a=u^Ipi8bm^k zfz34n(|@wS9|;%F;#sx&V3)vxl)3`!Y*3N&(;Q--)Y`C z|IMuY;Nd)`nb{+UT^9f3J&BCYEm*NCu3ocd!fCQ)8vN*LqolBm^mHrJhd4%wVgwXG zIl`iMAOWAKT2WNndKNic2;K)V0sMf&IM)n;^=l@znm-~s<^~iNbZTpgcUl+TJ`=fk z(S6UJF?ZUP#pzM8ox4i$`s7puo{g}E}Dsdw{Q6A}&^$pwBHYfN5L_$ty z1zhr|!`O8I9JT21KUf|)g3wiB=s|;?{Cf$yJLGO2yK#G^K1u3GuV4lVxZ$zFR6s>r zTYDg=oM^QPtw}DGgN%*n2H?c|0MUFoP|^@SgVoUXfeZG4VY3MWsnFJNbt+|hP&Qg1 zoMwmX&PF!y;U)@FqcL5P8y!e`KT=+BurEHX?m9@MYfHz8;|SXE&aCH0A-yDe1PIqa z4weNdem-@r{L#j|1KzrKxm|Kga|?JqOD&v2Z0og6mH+PFrq?ZN`dPK7$GQ|UnB2^3 z8+X!G=J>4HKr>$~)x8$+;@By~kILzAC@vl~5qJ4)!HQf> zF3}~zzz*E0Wdh#@y&XE8-gB%l>18-ydzo%8*jOsCD>h=*(Z$f1SUZ zzw=)p@|@^F(6J&6C3JqnB*;RdV4MM-G7Ve>_fkp)^XP`a4G2{*kw%elY7$Hh%`5I4 z1c?-ZUypd+zOgYMKw|%=u`3UUa_{@M7QGdvdWtORJf+7$SwaaZNrhChWs5{)DN1D+ zS|o)Wm94UrnCx4~I(3|yLP=r{L(-5CB_<7-c|N~7@B8=b($zJ&Zr%6&+rGPRARL0OwGShnNvlXfyFD9@a_940>#gD4!0Z?v;hcQVP_ZUovy?mZFCs(C28~>Yp#5F+c>=k)~H-`d+tHM6TO`jV1xTkI{CAEHdo2uw7^bx&y7>T_jgcCF;*u0)H93#XRqXelA2^0t~6I&a)7mrvhhG>)E zX*Aqx!Bi!R_5`_yAlwK&DNlC3e)P6&kR)bxcgv$ueCo6A&RZXs^?v9Z!(?6Dd0(AA z@)45i>w0^O>Qo@8O*E&`+nNk-LUq#wW)@^s7s~ufUeL%xp$mM5x8uSn17H&M@Q2Vk z2oID1ogwT=)`?TWjtmP36YoUc0x$+?gdnNKjW-XRM3Gj>6oj0M(BVL@W9hOD8#eS6 zfh`7Pxy=u|6)UqWvm4?=>nH1-s$xgiE}L(ivoxobRu_|_3th6ZhK*MGEM=c`>zpE6 z6_W_3b~-d)THDJvu`IJJiR>`~FoOF^kHxG(7gg0B*f_x8*ra7oOfV56hXGCo96hYq zK(~)E$v6aH1}tP_(6HTfdc+KcCq61P$9B1mX8be<6(r-q13wk9J_#PDk!%1Hy&3!! zS{F}EIMvOHlF^Mn-5s2W5)>hgXf8;3;L!?i3#lDtZrU}v{Kc|+tq)WR!!BCud!u2s zsJZa0>SlT@GFW1%;m?vgo!0E}iB|g!UO=Wq4;}#mQ3aGT2+@Q5TrZYkCXv4AngCse z6dge=DS;Oo)SFITe>sU7m(~^762!s;2SF7%9@g4DATuZpiGfYmX&l?66b`=X5-F)? z_Qz_Tka9RG<=uh~xT7^f8sec)Fa?2*HA+fXS{TM)?7VkHKe0MhnxYL_4$1On7$?LQBMwKN&IlCcKBQw zDb#C_Om9B(BEW`|spJ}OpjyyTGv4&~T6u_acpLu37)EJ| z!gy7|*x2akV>WvtyhFEb^{oTw;x-s`O#^U@zL*o8JDj{zcF$4)_nZd)%s=s4a1juPJz90geQP~-}n1O z*R3oxoj!F{C1Bx6LlPSsT0J}`h)Zz;#@-xM2~|!%rpgC&tPlifgQN2Pn7vz&0;)9c za0gs~ew`7_lSzz;LfkM>tB+F-X8}{?0%1w=1BA1$F=??J&~j(t(Uv(p;YCJ7I76^L z7)27|I78^IpwK9W)>Z@tN?nQ6IXxl^l*Qd{$C*-EAGUTp@IF_b!039p3WOghloHRM zzkSU9sUKbYb&3G`2-xd}Qhi=4@o{W&ad5H5%mYVl%APq&_Y68p_y*reoB2-x(GBMDh_?sajo7|i zuz3Mn0bPfyoO=-VlF%vE^JK&qW{jOUIUu!t?5Si(*&bofESa{(BTbnF5Gi$q=>Vi1ba1tg8QQ5+#ZeSw7dL=hncs9+J&6dEETFm{whokw8b$PLnm2o%B? zo6Ofh9WFpl84NXl67WC;8>T;|Sd0jm`hmar=}ij^5pfG4pdF$~07ylT{Dg=W>;Z2b zt9_1pC+9!HaH5C<-4PL(gyIbTX^Vp7WjX(mt*jjHlz5Z~>SbsL03!24Ib_E#H}mrN z`ZaWK;7l!I^uI|VFl$WFzXiZ&%cPJ~l`5f=J?ko$nLn5^lArWLeNE$E<~#51x{(xl z#aTQ5l6T^}tNiuroQ(cRt4i2-*+?&O_QC@*SNxb6Ij=ZDCt^WvAI)HBdNH1p8a9g0eeCQbq&4%W`;5sUk-z-SoC(CgW9)R*Sm(YnF9Vo z9`Bn6>O@M*A}4z;MNS^ezt`NnPflto5}g>q_KK8?FwqS7+Pe$NIK~KNDp8AnRSpgq z9igaQiTmz31;;~zA@nFMBhvyX#W?bzAym3sQQa8F-vg+8KHgZT_5cP#`Ve2CA!#^? zef#8>?EXC!4s95uI}nZ_-ZXYFTn0zv4o^n=V|RB7`*V4tu=^VD5jSt&egmLffA!)8 z6h$5P(tkRaifjyJ_+?oR{{8h0^~$r%r`x|Il%n#2Fo?VkuY$Gp75TdZHwipruKpPQ zL;I@G3eH#T-rjrHp5LoYcF17~9`2LT=+bg6E-2UoB1ALXbk4Uw?ef48CcR=s8+z~Q z*i-D-1qa!PyT_4q-e)oou3EKwY&jcs}#q`a{By5Ql_tw zD}9#}$DNLacdLl33}fmA6xkm3w5DSBRY^p1H=3+oELSOwF$n^H3K7ocu7`fx%DYZ+ z&L{U9BskIqJ_kZWJ_#4P0t@Wv;juF{MS1Al(!t6{8;`yocz^=;eelNy;Z{)@%rEw3 zPM`7tW2(o?T1^))ms0QnSF`*Q-NY#BY~WT8G3qy5H1Fb!JL+ejWziYxCAL&rkekMG zaWS!vNVcWrt*bfyymfuLUfP@88c9(aNoEYybg$&ibUQ@gw?+%) z@la~^U8Tz;sK)i7nd%MuXGUlkx>6rQkXO7yLs8d%#FgPa{9SH+)VWjeM=$}}trg@p zhU#ow>UjQ@6lM7Rv!;8d`dLL7=A9iWhtC0KR_3aHL5uuiJ5JN^+%!25v)gdwN}deB zMU>I}&^VDT5%s5~*SAP*u{@K)!jMa-0|bm3O~OgIY?&XNcXkc&=go7fu`8(ldPVGl zeGslpl>gyAm-_9;Y3T$L1$XJ05h|RrxSd~8c2OTs?a}AmAksi``m_fv2{Y2pUA{aQ zzJ8`1SSKh+bqzF#P}CO5-*A%>0y9@mS(2yN%Hl*%F{=4EOtyFv6J7wjz@@dI7Ntil z!L#$0ELlRsX%J`52mrvu{2-aEm11|)sjx5GTxYma`~D<1z8hI}EB2z_S$-$6gL(jk z-%)XbL5vTHA5U_v-pb<4%*>O>YQ)9GK|u17mY4rLu7KM~o9{QVkTSey@jLKy@tL9$ zo?%ZZ2{}wk1YftELilcu!1yDEEFAr7hZZ%N4V6?V~po7393 z@!@>~NxTUO@y_X)c$d1-bE8F!l6*Mi-hTr@pYfE%(%QT^AF^A62>n}7o{f9q9=CJ# z`R?d3el>nGzF|GGZHOQ26z`T*UT%Yakob@bee_Iq-j}l0gqxf+zh z@UOr?%X}+8p~76dA|}UH<>i1>3H?!8oNFnb#@NCt;?qWRiJSjfJogEAba}w8n<*&^ zt_)QctC+^S{T^qcwf6a8jr%OFAIO#`>fKA~4{P8@*oUq>IW#ylgo@S8#Kc#KCN_;- zz^G7;F;$_I#QDA~5u<%IeOc2o>C{<|_F8%sfw1;-&!n(5ZS(VBFW0nHKcOMZRwWzX zdlld0*^IztY~_uy3l%h8J>5WT+;ljkrmRr?`C)(KgH@~u7FT2Ui4~ilf7_hnUs0|2 zB*3w6jGx_9=!BbG-|>}SJ@2C7UZsg?no^tv1oaF%kxV}u)5~ZX$o^91&vZ%+jdFL( z58bdr$KME=Z<9!a0tQ|yEREbg+*z?7qmv3VW&J&=YIumf^8ZX!vQ9MO8KP9974$aX z&ne0&yCvFC%By!fuRIij{~E6=|7gyJd?))Vt}^FGdu_-c)xdHWUU4ri6axHc%%1Gk0 zkXjigH0M64te8zPw_}E!wU1^-9S1s)q3ZThuPJBVHHF!q59Q0Y3>F0oEE+RCMc3cl z-69tMNt8^nwLhoWRqp-0iXM=f!PuEFS|V%pWGR(0TEqS&(le=7%aQ&3&to!f&UcPu zIY~pIBMhm{y~7$=$~O5P&dysd8g31DnbnZo>N`|g`dU)^scN;&@+~{h79e&q)j z=H*^0wem&5-H(p*2Dkb1YfA!mM4r-A)6lpZ8@s4RFgb)AFJ>In)%y&_4n$-SgR|n* zYYV$P_`$VDhuAQOR!?8+U45|JNB7KW8-(EWa+?ytsjIOj8HaCX(rC0BJ6F4VCk2XI zD#w}NgnrzndZKHhTjx6hDQ$5&>_Ijq$GaW4W6vDXnJ}Y5A5$~gGkzrFG`kG`AJ!W^=VW zv~_)LbxYRhHZ5GG+}+$Cn%Y{X)zMZKm?EinxOqg~{e*xfQY55X^CgT+rj1?Uu5lOX zxrs$xs;Xe-8kZ-bwhGH4XS?{>7O9wKS(-rhCD*LQvclm5bNR2QNQ-=;*EF_?zKD|8 zUV84jy!Bek(QQTk6}kQudJI~6ZA!`tOhaepsVL8z8oT#ngXp8xXxFLzEAq)Auo1CHFT+E>#WQm2-^d7?u&JUNx=S?_oIl{)X+k(T~SU#s5} z8s@hxi($nz^bLv@hr1}o?o?b6W8*Vas>PWMXejOtbSgY1W!yRul8)|p;b`8t?$sq> z55ir-^7p>-o_(QEJ5*=Ny59Rjw?)=pi~0Xd6iJ+Yl*-M*M-|EMIszQ=er8>d8RgQ|+Wtfp)7;_%~S&7NK9)y~ClTf+k*9v(jOix)2XA(O(VnY7YV{xcHHU=wwo7I z?tZ)M80(xmBdK&Ykc@@xKYnFo>ivqf!JPijUtZ4FG5@zGisSKP`9ArNU$ZpzxS0QV zH?|B@37<7y9ITA|}}X`{lb6 zH~LMZUlkRRxw*Ny&S$@?{azMGMswMLOCD6{(@#8M2obdg&-&5!vG~N`FSyo1MQ~L0Iwc(pvuin1Ji&e%=yyrsevh-`y zSvUm^R+PrSBP6c8w6vsC?J9s7TWe)*J=pydo0Rln)Nx`>?008cw8!!X{Hn(*ijN(U+$eDpzXvDOY^f+s@O7 z+AzT9TPqtnHmV%9x&E9Vi&S#8KGx=o1|L)Ox63nQJ$DQbzic0IeXOeL{`D*D<45t?&Aq~DCWAt1zUozM?R>qU6p6=; z(+;mHD`_ast9^>Uc_sAyds9=Udtwa1ikY%(vGvQtRo%V4<(;NyRm9ErZph1TJj%TC zo%nxOxA=>GYmtM)6(1!3KFjPZrJ#J(xHhI{GKrC>?#?hFOj zo$-)j??d+?`&LGsvWGp1HddUJjgFmCA^R(J%r!MN>ql?jp3`@qdxP=){oB0xa))-*wTO#T0S8BF1a&u2f zNlAVF@X zzt7+h0JH(hqc2pJ_M{DSKBYm%Ek zZ{zp0>@Qc5J|1?a#>nJrPfo?(Y(cvUUhLgftes00ePxgHy&kW2p+UY5iBbCd_d=Rl zTC(qccs=r4gS}XHFwKYdgow@1lg=yq)}JRexu}Hbh|6t;!}j*}aB6D}3YD&2#lf)f z@sTDPdS7Pp-Z8#!ZcZXD!W6wCs4`*6qC#%qy=#`ITZ%VW{-_C-S6!#6qobq!pAON< z5s#JT^J$*`@I5lCs^~}eN48gIGozVW{tm*OX2O35p<&4d2NM<^zU`6RY!VxAoRpNb z)9G^*V8l<5a4l$PXo&ARZspwDmv&9WqDIqS67Yvx{HSoqLvb(G?wCZ-2|xU7T3C1F zUJV=bYV93u?cm#LmrqC^jFZ~`dhm|%yYJ3j+nS)Ws?X||Tug^5?JGCFtFE=s+m6jh zAMeJrQd3i#Zp?O99_=j@+l|XjO;0m!NG&WZ2$;4I<=w3yf#N7syFa|L{HNb*Q!9?+ z^_w?d<2B@=ffIsJt&^+nH8C6~FJHd=xwDhGB+j2h1^(rUJlp5bpJnXr*AlN)x*i{H zZx7V@`(q0U360I?7PbUY8$9{-OK-O(`}VtK!8Mcg`pEnq~(l_SySD9(=OhlEYHcg=6%o5FnaFCI~>e4RaN8Cfz|J- ze8wZyZu`VXRl_zshP4cQH5=q2kAC5*#BB}M<`#S;gjFn_>alzo{pa?mk911}ZNQ*y z-MuU4q@|fHQr+GjI9gbozjCm8RUbZwA=l36yN@6BmKR36Tvr?L1ss20fBNhhn}h^? z{J9(ct&_3QYx{>;`T5O^kB5po7wTX>EUm1rs;QAAtDVOfwMH>!Wo5}ID4gKp=GM^E z474ivVBYm1j>Gt1XN{Aa`|6Dw#9Ull{N`Vu%eL~uN3!b88?cq;=Hi;_gVm!`WaO7@ zS7g#mXgyd?EWO)yld|F1D^#AXNQY;=bEWzZNl9sGcYpsABmSMB8MH0OoBqKH83A2) z6pw%0lpbEOZ%^b)G~_d=rty=(#Bs>Rad42I)b}~u+*ynEsqXxvdhMFs-W&|%(Qc;? zlP0|u7Zn;M^kW4rzTslzGd1+xrW0VY8g7UV2il1dVbCs5?|C@xQuXS`+?0zFTP&+Z z7Ac1cIclF_VT8%fy}FgFji&_cC*&z9DL*HWZ*4W;Gmp)^OMB*XWp^rGz;nkAldbzl zS68>c+D*{&3bTRFp|hnB-L651f8W9h%)tb8U47{J)F-IDS64HbnVD5?_TDYF85Xb9 zzX%`6dZ_ZWv$OO12k5;QdCr_VS`CpNgUt{<@ewr>SQN!Z4cJc3&aYm-#yxT3#7-0Q zk#DhSYvbhF=Qip%j`6nhX`j-L4=i>MlsWwO_T1t7-dN5h!v<*ZT-fTF=g{8f-_4Zl zx!$%j7C?%(*}3GBSyDpIeWxI>!#rTXT9JjUzQnA3iX)0ok5-J55Vcw9n`u`V%gV~^ z#299mmYQiTlKte_umT!e({gO(jM+GN*>(5{8e{TgP#JB;H0flmnj%yi;o zo_Cl-MU@_yvBwLL_2yL}zxZK_d~xUl#bAYPYqa+UI|1wJcHJ>!v5s|GpBNNWEX+c0 zzQnA{+c$5Fzdjexsc{!Be>50uRakR|D}j(H8sF#WPa5>KYm%%Ip@KtwtRiliqDP<|Mub z)0$pW!-yAvRWahu4F&kUUVgG?*B$#;wjzgjdPE()H|KkFs$6(oRz`?Jny^&j?4}d$ zOc);TQ6s!UOfND9vv3|j6Kk3Tlq*UqD%Pay695I}Jv<~9T_n`R#&Pu>JA~~%=w}rc zwzl)tMWm)ORd4;W+JW^-!isx_R}Z(wfHb}>5Nd@if!|D;50lLNE&`R;&rx8;x|Nul zzoke_U&BfB+Db~MT!oqcjCJtr*)!S0{Xc2PL@zsnMMqq5rl+T~a&qJVqkRAVeM*2) zr^kphKR@5$ECEXyWgv&j&SGgNzxRPFN4&~z&GFH8-TG!mH>aA!k`RJ25-y9r-Ikwz zm03-{;YZMJq4htuBfr<4Q!+GcjBnlHk}b5Y?U>fh6W_7MHh$b}l|EV{jbOM;o~abw zl6JIlerjrpg*uzh7it&|sid!+SVYh%YOY^6GN+x6N;0}<(_aTK6xK22YUj(rQonsuDkz+Gva>|N^w4;y!;G|HLZiA;~F$U3fNL&P_f7e zgV5Z6nNfp1LK1>gSiMd-;=TU}6O&6XxxZMtUU>lRtf8#WAUu{`Y2x=wTR@l%09mH& z5oW^}txvTDJQKk90udm_@HmR;G@!t7)eUq2U*W(R%%M#jK@LSl56q}qV*p_SFz z?o3){Z7tK9D+9CVDwatxjhghKQ}yxDK6Jc4zdM>rJFkSJVHN{aTx{M!_p`5W(*M-O z{*{_gn8RG-f4PgSvZ`v6;n6zv&)#0PGvUU7|N1{VFQiZIU0cDauh|~q#y9lS%-*j} z!!O9Mt>~0KX29@U_ZI~`%xs^gEBWf7t4mu@wJe^eS4P;}+-xVFqEm9e4PO531_$|i z?YD1gLmn$NWODVBYh&K;SCswC|D~2-nxwNoMJwoImK!gE@jufx&zOjHTdPotH-h^kAW{ zFp&bYcO~Ahjl#>ob4_74iFW5;bB?}AnS*U5yx9cxh@uBS&bb-7xQJjrtDMx!*Y@V) z<~~I)VvW$K{Y2>bFDpI=YpsCssW4-2U)nUnYKua}MS6dr)2>JB&Uo)#+n@%c`uU#R zI=hqDh)%9Q$H@E-^ptms$;r)KTdrDR6BVVFBVc$uWbX>VnbNPrue2ZGsmnCU${edN zM-T9>q>(;&@St)wBRse|WWc%whztOhvv(N8$W)*4$Ub;*;rF+c@YK{)*60V19x0fa zoyFAKRFJyY@jYQAz@_xsndb8em0|G(C`H+JpZr~l3mt%DJQgZVG&nqo0XJ1uuL2uN zc|39fKOg_%W4mw%&i}-Oe5Q&EJhnhn@u62+);4o;EkkDXK|h z!;eRsp`Wb6dZdPL!36wOI4cDaC8u#(JvlQKBe5}_z^cNg!t(D{Osu!_F<)}a z?5wn(#BfUgfo?GV?5<60qB&-d0wZGG?`tWr*|g)v^?~|;SvsNWbX6q!~ZK;WMp=Kj%6BCD$<1DlLDwze`pexwsmA+fDH6oC-}=&641UL-J!Je^`}QmGBHFJHc_ zLbo@8;-H?b=?cqp+)m7Hd-(ArOp7RV1~l+o&nUYdmRZ`^95o(Mo;ib!aakS;h5AU< zV5Ck`(D(Bv!)hhQx5FF$r5H?D4`@_!6wASGa4_~T4{K45*|XXc-+ zghWILF=%imq9m4M_C-xT5wNPnk;$QsL-Ntw#ibm|X{cw)LP3?*pZjB%@Lh z2wZOe`(tYrmC&v^CpiLE@BP8iu#F9;-5O}BI?8bz0K!jT1bmJ>-M(%q#BtceXoSIP zQh2Et&BW`y7H;?icGtGoPEJmmpCLe{!vP8?W%WG7lCd#O17f0gW$PHk zrcV*Hg18vyq^w`fOFf6G-TvTW)_{pg>J+|IVcY#%K-38^Yz!pfTDjVx z)z#HrQEN~_Jk`^v(I%8kAk()FLBiMHU$ePD=xDBZidl=$Bi#tbQ*o`f?SA8m_qGI& zZV4`QBcx#k^BnrX0{pVjXklhH-b`)y)FO>JPoB+hCe?!kSVSvGJ*OiTVXy*V)9J&s zK9};?x}sWq3@|m;VgM(*WH})r0Z^^O2X_Ew7xkZPk9d&EJnYz=ZcikAq3eTGk()ky z6qf`Ai>>-DqN1Rfd756-<`Y!6)u2O!H1T=;kZ=Kl1oT;^pU(ldye}`8dvgnLp&ZBr z!SpuuKtwizdT`hO{5eg8_ewjryQk+(*3c?+`$^C%LZC?f?sypsWf5QyLpU`b9%jgE zdl=R53(<@*d!CX^qzHWm2jii3hl(wt$Bhe{Y)RKi9yS4D4hCdE5Bi|1-LP|?adQ|I zLI`X1IHc*oMlh>;3x$)wk{Y1@$0Q|PzAwE8llj5R3l##O#3n%Tp9`4c9e7LoH>B(D z`J}&5O`HI%{0X!jtk*e&)+5!lum@$#&Ci`vOMY_dVkHJplIwU7_11@}6KD?U{(}+V z0@SprS`zvB^XC|K-A;2%D*@|P)zOFc=|<+`9VHMI@qfI#DLr;+GeqxF(x;~vZlm6N zJz4Cu4=h&Eq zYoH_{B^5zO*m3NIw~x*C-rmY4i*5y})UAG@`zE|KY~n^#&p@y_Iu#w=Y$CSyi2^Bi znC$v}5br_&#+h`y5C)J+5F|$?huWUnI|`YHl2>1mA=T!Nf!hhbx`Rt!%>^;_mO@M$ zCHNj5b8iyuU>p$9hO*1-4a_YgJ>96wzU{t4UD}+?zDL1XmH&q6P%?u(gJF@+cKr+0 zP7|{Q%5ad&y0~m%;kVVPLG1#j3GhwAWcwxu`Bl93?Xw_lf#|^NeYjv~YG~M;=J4f$ z>(TCv#+^GAoeP0-tVko7dmU1vkg4(d-8*hBa?g99MZI|WQUExBCp^<|{}6s~?b5%1 zM1Di~(#sB=I&Udx-a$YtdwaFtmz5!?2uw@-d&j65PDi>p;*;mEIr}gr;J_|>{ zm$@5A^{+)S%l*Sr2z`YA*5AJD7uIb3V($lkAj6hg`(Ga<<0=>EsemTLfDf$w050@j z!*e&_{e6x1vnS3rn%v_|5;# z6Jiq=ifiKl?h118#9C$b?x8o*wLp^w|73Q4Uhd{iQt$vyhd0N>#$uJ}MMgvrLCI<_ zUH|dphlZYB7;pp4J9oH`7XzR8WhjsErWe1!UBP%FB7D zq@_h)xc%1X*ZT+0MIO;1t!i=68n75Elw*)3L-IP%iWtTHcU;83YFJnK zi_R9Xj=(>@=q!C!X(`1^QQH>Kd7?UAikbqbbkIGj<%KB(X3z`( z6;)^kV3w8-9>{8IKa+fEn6KSFSaI)ClDrUIf@tYLM446J1OUM%VE)qbY^U#ASg-@c z)zZ=eKqG=Yiyvwd#2~LIecVM~*YL@|MMO!4lk(Qrp_Tc0P=WpW7J^zQWkKdujHG+h z4;42Sh7f=+=$J`@J<|b-e_r11vJss*`mXQLABz@rDN(Kf@pmS=#G6H~X z+~9u#rtBAMl_*+8ul{w8GYUhS#_JI;a{FU$Jy3E2lx{_!ZBF749EpVR<@N(%Q71ta` zYip&!I*Cn5iOkE(*RlJ5!f^V<)9CNeq1rgyWL=3+SUfI!~+hoz7Q8j z4)YB@t`^he#KZ)ja8w*egSAML5r2ASCXQ8KL355)Zr9*oIN(ljJ&nK(6`Ou~o8Ng} z6SOkV?0vMey|n&alzVZ3Wc+lIptMy~KPb}!SFYZq7)c{dA*8LV<4xUQ=z>5dBq|C5 z(UEH66lstg|H}@*D(k(}L=}4ZMFZBLIf70E>i{_wDk>*ow5R(3EJH6xc0^cM*i=4W zmQPkLq31mgm0OzU;RPV;Ks5mU_rD}y1=yT`8vyVEvIqg)_d}S>?b{T;e*NNi->|e4 zdJ*ukZg8muJj!C{1#RnJ?^Oz>y1csuFP#)*lEvkD3n-QD;41Zk86V7C+$Tto#2_8rs;rpe`H8w5jv zkT#*s<}js<2Jy|C6I4`G_zd-=cmW6iA>$K?^mogxWnm1|uI%1N)bzQSJsa>hF~IiT z9W`eW1YrNSwO|1&gpo%x0h3o`jB{mgmK`m4T3TB4u|SwXW9PPh|9Ywazxe8ilQ<*n#cA3-m@Z38OPRZO>4S;s&)Gr^Go0k~ta=2(D6`Lj z0Q#6tE-sBG#jreBKokcZpLTC?ycnDDie5ZSbec?c2On29(`^&7TL0ePT z*N-UuCVt8y**;jGmW`Wx!D^WmVTpx*vBW4Bz-!RU;34vo7%>g-6{S2u#f6>E$-^TL z8qm2LF9HWjElj^9Uj_wwuW_Me(tgMD3XpktfPz2|1nH@0#b6|q74Y_0!Oldbir?73 zI+9ip4|J^%W87JzxlMPZY`Ow!MwJ-G!&Q&q{2}3ckka&3_R!V?4AVP!iw)tN9t9VN{=?4o~5Mz0z-;h28e%Jd^D?-*7We&3NX0k}dV*5$OYA|qILES0#+IIoAbpQT+ zaD9=W1F|F{zWkss1Q?L})(*f=8F8ColsrE8|SUgkTXgAjGM5cXwf$;9*!l zHz#(u@k0RCBx;mWQir|;s;wlffyvhE;~~rwA7)aUkU#D!8TIw+b;No5?mcD%HFXh( zaANyx5?Ty9U_?Z~`eFw@hH@6fqC>up-!6x=pMc8N9rN#sQc3}(`0rT4&=i@rlE6}d z*Z|Z|^p(+EB2fw24|H4TK#%LN+!EV^NTmP3h6NL;7yvpb?YmTW zuD3VSqCj1Qm!+qt`wEskNaIlPA6yC^wrPE7TN{cL0I0{5YDH4fP*ogGK`HuJc zD11#In|60~`L?tuBDVq76nqO#t@%1=hj%M%n!#r(c9^~fy%_~az>sbPZ#61jO?>W@ z#kOi(AZiAOdz(m4L0b=T(5=M*#5>3{~&2yHD579*el z8U@Nhd3i|z*$js;fe~2OG}7LZP)UQK4YL-S3egoi&Zzx0B_Tk8x)>}-*xBj%`NWSN zJpwU}20$RIEl<4Ks1xX}U~)HrnvcXPBUmucFFT%t59E*F7A$-D2M^}F3&%Izkz$ON zII;$YY((K}aU|Qf(4vWe>Q8dp7A9fV&h#I)=fw+}4wNQv!ivq>&-~p}uy`OC1C$*U z#t0B8n2>e#GyArNx;p=qGr08dfKkOD8ovWDi%}Sb6nZbJ*|3Zu;fApB&&8TY@eEbcO{?yD-pFAh|GUA-zvq3t7H)pOq~OFv5|_x(a@kTpUL- zly{`*^V?5+Vx@3L3;JcyKf@Y$|57MqW{g2eM1u+@5o&`ZA*juu*NQFVm8xrL1tljl z00btYRbHH(tp^*n*mKvxdbIXlts>OTZ<2qmqc8@_5>8pczGUzlA!pnNE{W}jLCvi0 zXtNNZEKpMIAp|lBnW9hcAN<(z@P>RJtN=)mM4nG|#fJ5RA{bCMi(S_yyv`k>wiMg^ zzi2eW_H>F`2&#ZkM^A!@gb8oPPHaB`s0C1VGN@~R0XyufrwIwK^75=dZ+@zPpjJ=r zok^rtK`u=HA~ozuP}BuHwyaV2MHK-&Q$XFoseEwVRpn)J$V1wv{h@2bomE{0%U6!z zFTsS=VC7Vtsye`q&Oa2PxDkr%2sa{Ed1tkWR0GBd0M;Z*E={$?lNwsUa9h^>T@pgR z@P^GM8jB0d%ZeQ2u|8@XH0>I{=k;>KlqRQphWCf0VahWkJrri=7$)An&jK-Wnh zdg0H9n^#7efk1u#@dJO!*2YFrP3>FyVG|Tcp~3S&DLXnk@P9Hv=VN1|U@_Z*Tf0lj;W&DsGu`){#?Vh)xQU}An6B2GTO zR&d}29(?CO?E;<*$q4DO<2sfM_VY~9nUKdol<5>-wJ${I2v9wMV51#CDr2zF_j$K% zO9v2eaMO(*8q!a;Ydl#AtJ@y_pLll|g>@O35R=NtyakX4lMXRNnVh^7gF@zwT41}K zGdLIoIV71dzD?sYVhl_hdYRqnd3o3E3n53-Y@G-OiIA}Hg{TVvZy!9!HZfFCXvRux z;^yO1LW(TozB7CYu^7XXJZIXy(4XDhKA4QE zoA!2f6^)t#tAWtTRN5&*LP|q5a|?^i@^bB?4{zW8e&T%*H8gpA?pS3cG7Ad}TY`xT zm>*g$YNLqVh=`Ev`B1Y^5kV0>Uv53%4?rRl+9@L=BN9;@9R)#~4*}j~3~~iZa>2`i zLPX1uhR!YkCO|z21Y05$ID%w@019-VeYo`gNASUlVj8HuD9zF#I{FF93h*+NzC)W2 zbzN|tNK9|Ss{5-XIQ0)$i+q~22fhNp1qKs}9Mo_0yCjns1H`9o0KEhF2l@?9vnEW8SdUK|a8|_OJFToO^kJlCS|R+^wVzbKC&g z=Gx(^$K2i!RCRE`$G|_pyYJ2b4Nq)8d&~fEr($qAZOKK#r1<&4A4pqJ_A&1`+S?Z& z1Sz|_f6NGzg-J*zic3n$RyGF)_RKFp=WF!g#n?zMEOLA^*ml2vUZVDbp zgc9=D(NIrQRf^?D<-oB#UWGl2JcQZhw5N}#VG04q6!&Z13 zgtwA?j&@GbTuB__&BDRs~y zfckb>3W2ZNf`~0-)S3kVjE2z;<19(VGh;Nldb~?}@o)VlZjw8}%nHeBGl}AzlWUMI zW>{SVU+%Q)G26w&Q*j)OUB!C&F6wOcM-VJq@{L|HE!KgNuKW2R|D@3 zEUm3S0XatjL1^_qdwLlEbb<_c;Vk({m?yWTNAG@wf)$1s3RoxLFBe1FWA`RYg7U!6 zpMg;PVKPua4rChuAMwXLu;(B+0>(C&Xxw$n`Z@}W2b2{)HID2rf`duNwZQ&JlJ%+f#^5hMX3X$ZCzbO89ck`F0x zd-HSyAi~wx4_gULMaUkjgU$&20ki}o_^inLLgs+Kzd!r1KHDDN{M=jvq|ca9Zp+fv z7R7Qe)*Uz??sWCyl4uObQ>uJ5fxt;kphTiUp}M2bnpRb3hKD+1`(G~V#mb5ci7-fY zjd=dKQ4L1Q-{=Sn#5bYm!*Y`aMeN-5=e}!`t)}oP9E@s%}Iu4QzynoA|3>(rInjC(MhYTA_;qs=R+rBkIgoE-C{d{|{9Y&taxfe~0516VNhSpuba6@xiFho>d_S7YF(-4CxVU zHe_2Ob>rv5GB(aMuz6ud!F$aB)eNa#{}tDTnh2})b)}2p+zO;gOxmCELMAl;JKu0) zQ+Birgm1K%LDAby#W93rGEgwGhh=)jX7}&2NJ%k*T7m!m&AWHZr7HMwPxqu|WJExR zpWHXtegym*w$PddTdXgK3aIMqUw3LO+P4pDyYIZR8e4$Kw+_!a=26kyD-gC!y3XWe zD0t}-WoNy5r^4vb_wTnZ%f89UA?)et$;!)<*U~~+aCpGt1fhUx8^rq6T~9!&4x7b( zJ}%DAs2M{@Q$dVDTXNg0-4LX(ugRAwAokB>-tpq>`BsM6I76lHO3Yr6FN{7(agHd* zQbFFS2$T>5_qnqNn}t3KkWqx81x#e359TeD$B`8uYS>%`Zqrog2`G7pG8`yJ2NY2R z(llEyl@S5Jk3wKi=yfgm;ms?%9NtjunpU>iLB;;?;RE7Lh;j1hy;s)weIQd=psjuU zHQALd^qT>WVwnin^32+aG15SF!`rX=NAm?=6@0`_qCGUEQ#7@=Bt8?-pWAUUnsm(- zJ7M~4m`KIr1%8M6L;nVr-ojB#Zd2sPhtuL)is6jYXB9qwT|7=3a+SmiXpZ{5_prIx zz=zD4I8w}JgP{2+C8fslQJ&WO%V)wfOZ%R5H@%>2rQ>m#x^MC7y|S*b@m(cS#rHqi zP3sfi#99*R1)N~NOHbK4e_e!fYC65L$j1eCj`gFJ#{TgR<(n%viB@+5>^|05+GJ_y zXSN#2I^Ih#F=R{nv_-<(5^1T8JDtMB%ofY}F!!lx-$=fam-A%@$G!I07883XOJW)c z{at;%#fw>*GGINHPJ7s3065=pr` zuZ9Reg8RTj!9PKrG5q330$?IY5Fl;;I)xk$&W~kGvv-_Nzq1#<1r4Gtfncv3TE^uh5Z>6Or0$IXevR4RAEBS`$6*bW z;W-m;Q3fr%xf5p_`VZib%b$lTHnqCxc2=B57rd6}#Y8-!$EnM7TtP0K5rS`91 z<;~3AFl6xoxKmIF%E*x)bxBY75cPU%%cUAlDIf&`Jm?GOr8l&4_eQpBLs0Y!HmD?5 z9M+R3BxOs36_bAWO!Y`11LJDQbutpsjM$ei_k*ww5k08gUv2`uLj&~IL7OUK>754B znD}@>&ZKZ?4I~Vx!CS^itW;C@985BUPzc`y#tx+=4i+t>CB}|V3hFUtm^G@N(H_ddE5B4mwRpaWnhNWmEs!!ldCgO!N>eAA3 z^6|!PH&dK+a#|3`$=6tGesn9YtHQHqnZ-)YhRCk3*o^becT zFffSfX=pwty;18>z8N+AOSX?+ml2(*a8vR5u=SebRgJ1NXF5Y<^DDsteruWUI2fmg zky!A*c?dWZT0MWH_WDgWrK}yMi|%VyHqRXnGEDX-|JZzwdrix7?(TG4 z*TSZ}oDlm(6VpQvB!6W4lU++c@}hp?Yntmxq4y=EhX@DWeaDNRMVVr z`w+IjwA^jF_~cA;vB*36w%394gwxbtB_3OKYZ~T1pJNVf=^D0boE^UrW}5FJii?}` zCBd|!CKLbB4d>5vG4-+6lKDvqXSZiNI4|<3Z_s1PVM)ka;atMg)U?~jUfsj7%+cEN zcpQx-oM0Ak#g;WtUwj*3G|=rrQ6vaZ70B1vOs->Cvrgg%dwq3^ckWQ5Gk{+OJICJt zP=hR?S1FuRko@yi5Rqnpk8o-Zol(mztffoTcQ^^BDyCUBMP)A~F$m#Ekr5by8Z*1N zC=c%gjc~eSWxHM0REGBRH*nW;5U2;G?_(E@b;18*g|d6A_PZ1fBpJ2o(ceVOhUrhe&bWw89pmR{^fEu0kq(;^^FWUGuZcP}*@1*ssLj*gyy zPzAdbbaO;?C69I{vvPAKzTY^@CcX&W1+vRWFeDSAb-TnsEY|{<4={CmJ%#ET8pWfI zlWSt&1KN*zZ5z*he*<`Rr|qV^$jY$yos#lk4r8xgl0f9h|P}-8?%P${q5U`Y{SE<_5d9ozh7Kt6IYcgn4E>F z!gNQ~qDhZrXl^(ks*uWImZZer3-@xh5GbFRRC+JO`DKBg7dZ!-&n{SmK6YB@{;r*WI1py87E)L87snlShR5#e5EBU;l8X((QY~m+&}P0~hF*qYQ z6@GrJwP}nb_$I~uF{NAk&+uknzVa>_9GO{aSBIm9=A+wZ6*taOZPYOBSd8{2-UkvM z112>vQ1E{d*Mmmp5GOO*yX^>xRzgBT#L{0s$;;#U;N(5pPqg{5P#oJ!SdIWFE)d!S z0GcgLq|!v-sKF7kA@BVqbZ{-&i2}aD6CEDBs^KWm=tp-T{u2l~8WL{cEI{;-$0Ui=u{zgdImsfMp()7SzL&rz zMjzP>6H5A`Y~>WLadB1mEkj=X+^gdRADzw-S>;`IaXCH=BrSElO{##S^!&kvkftRq4tfaEdENG+y0py|i^?y%9v!8hQ|@q(vQThKjd{!C z%@^t#+OMC}uAB-SvvY%QlfOeQK^awl-#PtJq04NC%al3bu1cRIUsp*#o0OAHdh=9% zk20I1B8Y^!wz#zF?pnzxJP;tn$DIoz^T;6-;<~b85OCs6bI|9`1=1GPIGK07zRz0d z6fOmJ%mqKC%*vVd5owC$+%T6wJ4TC0pNZN36~TN?JJn%tk~L@VR3?j$(WWnrJF&J< zA9p+%b_0K&n&^v#9I8jXNqL;#i3&ElKAg8V8FRELmVkj4v5eR_M^Cu!NJWJuS&Lz? zSxJA5pVA#Rlx(el4X2CcJ))V|Uo23b6bF!bG1^ugyJ6lr&8XdieyyI_I~uYn=o~OQ z0+Z@GaS{o|$N*zJt_#B5LZbh?6|HqQ%|D92Eb8aq1KKO za;SWd35C6?f7*zdCTrAVu+PJ*$9d~KFYgVK*Q5uIMnzs(n@d17@EDtn4vMel_L#uo zzDnq0-NVC?9~@^UP;|Gex7X$MOGSAnc~ER1Z7{Y}G1>z1DRhWX2>edIuqtR+94Ou1 zw@rt<2*{o2KyNiVgTu?)43d;RoOdv83_KZ9Obu0Bi5l4hUaH^}Aw-1o;~1oDNgoa{ zR_?xCh%G&Dxx5LXY&Z3+HYHtqpRsOgg2vyg>BR-tQfc`xvp2NiaPd#AH`oQmIt=Dj z6a&!4+P?Xl%hZ?IN^P{%_?I6U5#W_Fc5L>9$<*~)`kj*|J&R*5UFGGybI2-0q$AEk z86{)!{#AL-_0waK=i_F(Uv^dZ@%$M}WpUNDe!uQ$Az9Zne8=q-1I`TTN69O={>IIv3#S5&oGZI- z8D?hL8JCE&1_u7ZQn?r^Pd@BULHF?KNr74!GVKt0USi>}YXla%JkPLLBzV-VXRb=qNH%GP9 z>Bo(o<;Qj5Z{NOUAC_km4@4XR>GvSF@tnJv8bNPceVQ69S%I4C*r&UX;4Itwii$}n zx6B5v<0oh$6|cV(Ar3#o+XyQXqC7R7jx`2+y@iI+K9G||s}Mw3WGtawhupaMIl$m} zD0bOh(S^(cgwVa~m;v~p<9Si47x%Ha+Erh*bt*{4`VP;@DhAU5@7wG-#4;$9#^er1 zwtVDr!-LqdVmMB*tA45(Q)7 z|5gu1pOf#xt*3hU!6d(~#RcBG_ffV7ulk7PtIK&sp}uy*)u6`HP&0-4v@V!G0eqox z_g+*r*32o^F01z$I`PqsSf<90-Nv0fx_{~9>O89-UuR3bp8nBu>=J>%#w>3D7ZXf_ zO$}{X!k0DDV78*m!(r=&w3w!tO0avX{GsW^v6rLM1q^PSB z%T~X{`3`>#gV@rPs-zOn#k3&Wm4_V7*mKlSjVlvsAcin$H%q zyuLU72fte={h=HbN{hnNX3aripF;27YTO#uBeKfTiiquF4*Vfi`8}X*J`j#&PNU&^ zOj1w{GJsLJgh2HNgAr2M6rDMMbC9jzxT6y?$ajF-F8-bqBRk!Frvj|912`P@ z3DzjY^szAL{0Q7Q5ecd}zAg?qCUmy{2V}Ysb%Qfl4WLWeK{8@ILhzeWQP`8(O>uGY zRp|T^@ER1OQoB4&iBuM3ByHU#z5pjAQH=!)i?13k5&%y530Ze=hS3D_VIjr%P)A3v zBPV0;OtB6RI(}aS<`}RFl!`+4TQ~@xeIjp4Gol}2Cxn+c81iM^M2FGlr{>cyhSzs- zSYhq_)PB}B{VRx!UpM86e5U0v1rZ5u6G31j=l7|m=`~_KpPIWCataS(OV$4xzcN@ z2F=p5SN({5`|+r=j87vouTp(YSHTntpm8T2ahX4yVl*>1~}_99f{ODn&E(z{(PsR~@4%?h_pQgFn88PnAH{ z7|Zz{@08+u5zc4VXYZz($g2hiJz0Ah&zT#Ib#(F+$hVKwSEXaGHyV{^WY=CRHnc5gznZU&dknPCNs;>@SNqhMk7TS& zVfPCzQfZbQXwB-{w}d!BKH4V>ZGj)}!-D&+HGKXHxmi!3G9}dfO&tGDD3Ma)6u(|P zRmH_*vv|2^zhx`p4Xg*22&ZYKOt}Jm8?3L|8~d^Cq~*LAqoJJI-oYkNGO^6cR$qMV zaY&$K8aC!5bS9*2LLK)QtN!;o3!kXcn(kS06;MI}jKt#f14lg*a!OY>>C3Gnps4N} zNCY~45Zy8XqD!l$I~=8svAiG{?*fBQdeLJ zhfQIzP%VD@qRxG7_f#yGm%%+<#3$}sm^D_Ne;|8;i11o_JHF;qHphA%%fvzaH0gBZ zGcm6Tc$JvRZHE=z zssfo%DF`x<hEJiVv4cLL6;;lq6mDBFU%KHP4Ek_U+T&(e>< zvE&eg!dA#{7C-FaieQvD1EEEJ@=!Q=M0dJdR1R!Sn-9}xklh6#YvfH9)b2e5-aqm8 zq9w?C?0DA7&X`$POonk6)Wgv{h*Yh{rybxPuN)sr|4*_GE|S9kdl5yHOd%4GnGw@s zoD{JgiTLkprT(Vo+&3TUd#*J@ueb9!R6ck16NSosi%oowe3C>4Ol0$9*Gp?k_WX<1`w)0t;5F@^7-mWMt!NlPB~LW0RR}i^#v~9 zR%#iZSQ*Jw8Yg0wGEW&cB_pRU}V@c^H4c*>R zlK37t>mC$#$!t*WFL&45ev!TGM zUwb#a9Jo@6pEQmCaQ1ad%q`iy?`)9_4AJMwVaN!AGA#=VwYRETkv@0d;==y$Nd8q- z)BK5krW)GN*H5|frN@#4bs3d8C};-W5{)3$j_0%YI6j^8# zwak1j$>&7B{V9B@XT8=)3>!L9Fx|A+_UzZJ+yWVM#rb}8YZffd{&G`%6r4uEGISXg zXwJqJgK)*s)jzwQ$fNsvBMVsbHWl;`=Jfk9_lXZK)M6i&WZMgFgX8V623L=dMvnu? zuu=7eG|J_x2~RBfARY0S9)WWN84#3Qg-WfUp>Y~rPXW10x6i)baC93k1>W*p1;et39vFhMAi{raRr3(_jD~B(HvK%yjVdR zdY^n?K=*Tdo1C`|*XJBTU5yXtyOOV;>juG!ElGaC_xp?#o?wZfU`#@iU!%X|Y=?;E zQz}*oZi-;3g&2$b8~9A79_BgGj3cit)0%(F1BsKvyf`?zw8J>;PDy=+(_CStmkzM! z1^f$5-JFsJ(r_B$D=z-62CeWWDbB73j_&F)i2*h#v>-Rwtz4F-2_w2jiNS=lpJEMU zb7ck(L|9Mbo<;`eN`atWtTM+m?X(<$wbhEBUP{!*`6DhYD4=ls^RpQqsLP-Ub8gMu z5D8=#_^HViJ(}NRs%aJAU>J?nLv=PR?l8L?BOzXvy*Z1|!Lt*x}q68vz2k{r$#j1}^G~KuOe1b7#gi6Ncm#5Ei z>;Zs{NqiYOA*FI5`_fVN_syoe^He>=AKfFfIC-@tuVRqU0Fv&6H((k#NtTcEECzE+ z^)@s%pT~3VxDYyLi;%(n z#e}CyA=;8m(%)~e>|I0E7DapEkPt*W;dZcAh?LpAhC`!mUD(+CHD+ zM2_+v85wNo{s1_fG&=L)j-i(@By7?B35T0~NC5ML`&=M@y9U>KFt!*#gwX2{W$>IL zP&@?c{u-p<87hWd=*w+KqaZ)YekQtUd4uJ`+1QGAxZWeFV|AU(^09AaQr)bJt7764 zf}qq~uFvYZ3#;pt3c*uKt8~#pbuDddXLs>r8@_9VrzlI?ZmZxb0vo>lwb8}RlZ=by zjOu5z(IB$^o}Tc<#p_QKHaeYp>3BGKzkinPw-mywXpNR<3oVIbQy>$Gy`E8XH+sC| z3o_AOzk9>V5SjU*fY88)#W`Px0qC|v=FR$T{;!=-a?)vKKy17e3C zuHSbP;-7PK>m^tZ9&k1P+I%J@`+M(2LjoT3Ymgb*Oqs*a8@(4J3zOga>cYG_Fb^Id z=k2c00OZg$i8)Kc*B~Wat?= zU%Jp>{TR-3SC_XsTi_d--*LT+Kjj_#fWL5-Ze7d4;mhytANwlQO%My%~>+_{Yftwu)@E2aB|-YIvP&|`VFlOUq| z4>2gb!GGCwLPbVktHBNz*c{wRh15rR%9^(F^A8k0d~GREV+7@o`p&c>vP@rb-mZMZclr`6UdS4U>b2~K5xyqkd2Z*Rn*mqSHTA8 zlR#q%Sy|hjC+2qNFPuZJML3vW&|krVN`~}+W~)S#=A=)Gibo(M(bd%rK)nkb`6#y_ zh>!|Nn=`>I4^a}VV6TvvuLc1m$g-wjoV)@BfI^6-fY(HFYaK(wFqA9+{Ki?7Y&7aW zF25p|XIxMQ#3W+|;s)voxex)`06b?nP#h>9N`);cXkal2Bj#0X!1lJV*l+$If0%in z7~tyX{CzR+&SxNBDD#;a(<}CBxMB4&?^fw+{9T8_^YuH{>P$; zWKmY|1@z%M9O217^gOPW)jd|>rfmA0eHym{MBgHWEUpmxE8qWw6a14z1k||HCEpOM z=M_mmkeRZ;M507a5)zyK2HQ{Y1$+Sx28};FC`80B<%TTvZoG=J{}>y8K8So?#E3OW zC(jHR5DF3{h#cU=x6{0ZKN!MF#Vs-?7k)E7xz}r5{=i-;om3kT6RQ=`ODfdv#q^BvZnrWbDiNdS2YQ#zY_;Z-Aur_e3Aef|{s z{nWPL;AA5BTJ|*6@ArxoFn~nT=eq!h7K<7doFJQ96Wjaa8;3N6lrZ-M=+G~2J6k^Z zs4F;rP$o9AjxTSMQ`= zNL~N=t2t_}*Ab|J3`Y>a773{%fQpShfQJ|C1j?X!^@aS;Nqm`oOz>48@*PzBYY_qt zK_LF928BD^833dN_dJ3aCjYTD6+wDM$W@SJQi0D2Tv{+Lf}#oKj%HL=9!A(Z(2sa7 z<)gSn@K|6J_T{QNOQe971yr7(GQ${$II4@_9b#&_f{Y+w0wG~39qvGxMJXvMa($Wh zWuT%30I&X@AW-mBz}LG&Vwlby(fvK||5H$d3uzzRUe6IA-P+!cJPPXogfoMQY>3Tj zPwp5)fw2kF$(M1C-^E4lk37q+57ZUpSU{S9SVVMK`eh<@{p|MJ z>|&Dhi_RXwyWfj9FyL1i^XV4vvrb@Lt|j7=%PJWbNk$A#Y}PMqWR@1)N}G66ERIIF z&vTVscSvr9;49&JsNRTUqaUXGYnPX=_j5K{@&5Wg7Wd`e?~mMor0>~FS>_9B=#nEO zR#THN>j>J6V{3YB2j?%4fR&3xmdi;}Op#6FoBI?ITsmeZ0Z@%FTf=!TjC*2$HEdeT zD9j}4WfJ<=b35t$BM(FcVn#GPA4azB2iysApT(rWqzm)vz-$mX77}cvVFNXPk2{m# z7B<2xO0jSBfGj3z#0VZa?jLNBBF(+qzAiBkda@2Vd;>U z!2MXxz52B7$!`YTyYKlD*LLvDzw}=zE|nuat&)_s+1wf@3#Kj^-{It)Y~x5?e2MA# zrXkXuErXsNUfEre5Hny9bBdZs1>RL`TDJU-{|VqLp7H0Pb)sowfTk9@tS6c$?dJ5C zNgF3OqzuRpqa$0W=6m13?f*<(Q<}2vFUc8v5=todXOn3oyK&X;S~unR#D9$$KdH4+1?g zS8LBs0go|+gceAb+(2ngV1j}gLwaP01Wc?14Ss3}V2B6gWt+Jbpm5!Q*_ITpp`ts> zp%*a#x)B!UPJFNkUIizolH)CeU{if5^HSQz9Vi;?4g=UT_;Dc@|(#lhE7Y#w636!@xc82<4&-|DF%MDd_V4 z{Lu$kgaD`EH}&+7I=_-%&*Lo4!Se!#JrE@yb9q#P#V>3qQo8nykJmP>V&NdLl2Xv< zfO0M-Nnn?Oy#43R^S*GxqSYDxW< zwC@wHwsydT`HHBVlTz*Viq`bpEL)l?_p>t~^fDY2 zDf(T@YeE|>6!6Xfd^Ps^rH_x1Kl-A0vL3U}kIS@`cQo@X1RWpv+U30!(+0BY>P z)b!<-p9q~ie}nVlc5lDystre`j6)kU&81P%Fz#QqvX;y)FvuVTfp$ z*v)*Wiftx){urRd(8fudyia;KXCtOE&%7dIhu#?RfV~JHN?Psj6T9zE`CsnHWI@SD zc@hVU^j&&(R^-kIXIOsZi8`m~h-pwg1^U6TNFk%5hLg9rq!<|G5Dqm#Lq?qsA3(Ip zE|c{shmuy+i&FIP*%|b0v{g5CU04~z>gfo86(jQ#d`EB^0(zOeU8(d}MyJMJk+d(V zXa5I((!L>%Ed61{ZEPw7_ru#)s;61T$9eQMw$e4qzQo2vxM?CIK?-0PQ`5 zjN`}#i?Yzc^NYNTqF@G&c3(Fm2mo){IFt^`ZHK!R`7lsYI7(1kM>wfEK48H21R3jl znHWk-KfS94-Bm$B0ef#$M)T~e+orn5)j&)&bwZBovC)vCiU9PhD3enZq-LO>M(k#a z^B=W4;7=K-VJRi+V~hdU^VkiZpul3Oi_7q+ zLa!Js(gPk-C_gMLoH&&G*w|{6D#UTn^O3%!eK@{xKqI%TDvSrc(${7N56KazCA)MsmJu!x zk=?adM${*3y@mFx_bbFq>Uxh11CNIv0}obq9Ru!;A3`&K&Ho z4p^A-Ia~R$P!K!MYuqBS)#JZ#uc+efjhmNxQ=#7Gdy@dY2zyp-+&D!DnoE6ym-Vv_ z?NbANc+mp>y}=9>LDDgvQNM0|uxR!MhD?xNnS)kk#UH~-` z#tsv=Dy$5L90u9@`I#3W1jfEtZm>ijZd&npnxymRL6i|krGEHq zX<^ZsdUmc&&eKmY$TW+F-Dq*fniCcKW@b(`J!dYJOV1}O@x#$27!AMX7@y`HCgeha z@;39Qsb<2MbknCa#k+fqP6B?Hym3syMs_GfH>l3C?#K0nDQLACQL-#bb%q9z76lP> zh)$6peI;^gLtI)2qV8e%9-?^C2eUlNJ(~c%rr5mG9Jp6?vy?O79zfv`A3ypVM{Q&d<`KnfOYfG0uX z4GuX4H@C9mQ!v_~`G|Qo;ZN5rczuGKEFj2)*a1}d$o2y$@6T8}T8b{+Uvpa%Kv|Hb z`5rz*?*gJ9z9W}BWPoQC6xgNDJ#R2a$H>1p00`Je8xzVGb8;QP_OQAq39eQ40HnBY zi2=n3YmMjK!gLDLiQfPS+Y`;^ZY;>Z_@dwD{0;z%)FA;VjG$*tIC@X~e*= z_meWZ*#a~)UA=ekOT~S(;{vcnXsmuSTn2Fcf^?3EV(Xkmcz0G__z+^^FW2NNa zb+a9Vgyd8u^9oS3HnurR50rx^O;2SB06_!p6v!ipI?Um{0m>m7xgkZNuw6ST5ahS7 zY~sp)9yAF*Kj{PgkBK;f0ce&QLgXp#q}o+1S5>c8liTF1q-QYy!^Cs}&I`wsJCx;q z0UAyR;qYRM$m%qZahQl-Wq9;s#+Nf5D2H<}QN;!>eX>=7OO$x?t)3V{y7F?K-M6`j1SC&T4iMNs@#|Uut@Os zn5x~tg{;!g2wR{463J5KfR*HBXg-I zF_8!~B5{r^QU(SMVJUfhvbYt=N@i?|w)gXgN8Q;l0i9?byfOcx_iyokj`i}CFk8KR zMVt9ZmIgbk2jaUw(Oa|P3|8&fl@X;>BlnNt^*r1!_M|E05{V;*!m8p*vfe-_>)dWF+Le-c_o(x35K4mET+ zp-D-y46Z=h&%n=|qhFGP*Ukf06qqBOi1SGIczwIY17Mi>StEH5T z<-GLsBR8+SAhjG2xuTRxztT7Y{^mWV0vRN*=?W97i>3xvn?E<00&z=US$*y|4*v#X zpeHL*^aFob%6RFc0Lmphf4rv7@jq%9x-BWZh)n9*o>gVZSm3Qe5Hd2OW*|XNj#%k+ z+-IscF_Vua#ccSaRpin#(^HMG7j1cH>dqWE#JLaHL?lLofz*6SLkBl2fX@oH_al!z zwsE0lMN?SvQZ9Tcl($`~+7%`UQ?hVklQMZOulbE}ofhl&n>_FmC~3JSu!YH4e_^|s z&qR%SN#H!=0}*R2b*LZXyT%d(1drY#*)|CMKj!l&a4-NgStd4&dJt~PSAl0hze1sE z?(|?kI|Hu$r%hfi`pqdBhDm126WHuiCL8cxPHt|}A&DO~XN z9%vC=Kj3%SJl0t^%e|geoQH$@H)N&)U7FSe0)Sb$%=~t!r6&pU{FHH1Gf*E#bD>ct2Vy7!G<1a526XvI_TJ2H$t>mAW%X zpn}L$bUC09r~k|UfbWxbl1ng6t%1v zXlu@e?a}L2!_l|o36O#J#24gv9F$=Kk}Z(k4Vf>^xOB$;C|U(Eq6vJ4ScQFF^hU=s z(7KZ3-Yx_@r3s}VLz?EvkZP)OTz32T|c-?oP=A|EfbQfi(aWo=JZ zE)_JOykyj{!t)MemGA-Zn$t6u>`t;eqxXuk%(Y_#>BCtlBeP0bMQq1xiWF|fM3A_M zlRWqjYzQ;=1O6_LPu6283F^jiEcY_#8okraTLIY`^gk8WK6|HsPQ zy|39nmdJLjY|UJ-ex<#w;K2F1wi>{}A^cbW$I?$20TGEmv3bw(E;d*rz3ax)O1srj zCd3^{@WM&K%pUtD{(ZSNA5#btXg-WJwlK{td;2X1sXq*W9dg`6?2bgH;V2zwv`4_i zlaQg_?y{uW`coe+Jjlmi84~|O-_t8|cVuCgm7FxIRIcKsd|?;SD_U%%+P1_kGR7al(RSjevex}Do#n;^3t zrCR1sh*t$s^ov^(k9ZELu%?e(z{c>~tdtclv%d_w`gim_^1AZO&>%Bg6p0EScp!=IsHMV96NAMx z>^O0lUH|;)9^b{&x!WVt0hc-q3DdjHRBieW-`+DZ*#n2iIBmIi=F^?@m>gpek1u4x z`DlFipv!nH{cqQGWiJ7IfOfk6kJ?Id^6*~Y4|LRv8n#QM0}Bs#YiHBmw>>FDAt4W= z)@xBh$p1x!38YS*W7WuOaTgx+P>(~Dne3A%ml0oT43Z|5%7?{}-@|~ChahYPS?eK$ z%K}md6;4O53SyX=naRwbG~`3%7{qJgLC_i+%U~*e+ByXVITBcD2N(*fafn!l_>w8e z<81-SDq2Uu82+Fgph+;@U7Pkox>vB%z%-U7x3+R3@oNGM$b~5Gybjh*%w_S%7Q$e&%sxK0s?HKyCAHnZ)}<({ON`B2WjQ5PU8O4MS)q z^XWQ1N}xx_)Z*geZql|qxN)=~-wWBCfaXO^E@YxWtg0d~y{I?939Mu7YzLmW(07OT ze?dkl#tEt#Y&|H%d)f7XH_e$LSJUYWFlvv6o*vZc%4@{x46)~^^_&!`{$LECkPAp53MrM{udQOyJBVF@@ zH^k(!PYllTp0MH|apKiHLWrbzx^svH4jvmO^6?c3{-l>Ww|mlc;C1zSsB?G$cgZTQ z_RFpS;kp31-cQQ4f8=xyK_y6KKv70taeKX1#Xi_yRzdS3Wg7=*JzhMreb%xR!YQMo z#R7^EkYs|=SlQI>>m)-}Dc;}Pw_5`?K=ZO{N@c`g3mTi5?@}?zsfBHwfl#d{{@32* zk*dZCPZyKZzERK;T*wvHxoe20>6cBH|0joq1`8s|* zh}%}zR1op~-q8^$tUGM#*0aSMC!fM!({nqW%fd#>5WMUnyFLc~0I-F8Dlpn`dtbQN z{QVKhr*E%y?;7T)&O^8Hx!=TQfH&YfkO?b0?KJ!t<4`9}C$&dX7~?RXwIF0uEeiQ2_w+PX!5oRsg#-U>p%6yJ?u`4?Q4xtC7!QB2f18XE8xu`zLI!$c4HWok{GxrrpU0d^6 z^MXhLFsfdJn^FgI7$K4dsnqcQeuGxs90G74lOS==1jxdFG2g!x$R?0QWU>hsdPO2I z7ks-A#RH_ z#{&cJ%d^SlL9vl>)#y37{OdeYFlhyG1OYrBI5_-4#hw;$RG?30d>*{H32?Caz*SCp z0+2;K@0uh(l~wIev4biG9X60< z9Yx?L$d_VQIC&?li68OfV;vo)6J$Zb)n8la$nUzkMBFhJ_KpJv=hXh9ySbt=m(=8KKZm+YvbyfkE9%8f?^zU;&(HVrHvgJ+CL7p`VoD8d zwg~qz%2HZh?TUb}gT-s0^-yAqr4G=!v+)yGfuie^2*H~MsrgE<3r-AgF%rizoN49D zfqwyfWW*0;y)L^Z0-go0!#%j{;L(H(lq>2SaTe_z{7oVtCOBAYlZ{o?04;Wx?U&GZ z9E1@g7movCprue^>+%ywm*HOKj~0CWgpEm?fK{*4QKd|7!@GI%Y1jL!VU|K5rGNLE zflr>nK?XjDN*84chV}Eo^%O<#1aFs9$-1$T<% zrMYM$!XSUB=?DTm$U%IQRlf3ds1a5>P_IFtN?6&L?~(tJQ@Ip;RFiO!(RJDALq`*V z5BIt1^)N_NZ5*=B;_eEXknSGQwoBNVn(Kr$^Clh5*6W{o=6 zvDdS2v{mX9++bh{zn9asNDf6qE>BR^&R*^HY_mqW{Vmd3(}~9$P*Bsb(Fm^nWte`O zP5y}`yTgWtGgpD)rSMP+@S+4ExiaJVJ#U_SXbb95%ESE3#u&y!Npr{%r&g486HHij zazDO}Kj!&uOZ&(4Xm8fRqY)@Zim+XLzZL#yc-l8#6T}`ee>BZf3abCa0kkFS7rY4Y zYslMwC48t|Rq9zXFLI~Rw1GX^x~zjEI<{ek#;(u|09+^m?&iOX4#fF&nWD8dcV)OK zGPOl1X~0<<_^cLw#*Opv=oi&wskon<^u-2|~vl;77oxp1P=E0*y?lqM-c1D>E^l{t|Q7u+4c@Z4oZAXbVX8~ecydub{U>GN$+nh zb-kfUx7CXag3n_@Ro{8(rI~D1wKT}Lo_iGf?jP5&#`8d)On=V+FQhZ{!Oxl&CJjqZ zOtTEOSJhqu2v7^MS6l=rB>AKwegHXh0n5h;ZG5pesJ{t-tajR#F=v@z1DY1O?4rc3m-9OO7JL%<< z1h%xGfmv#BFr+{pZzk+e0m{)3LL$!{u3^IkzYO9SI)HlX?aCq`fgz&4Ufx`gKFYb| zTXBFKXta*!%oZ7tM;9O0gi9-h8zj9PtCdUq;Lqp5zqz;5RMNe`=o|?~+rS&#|Me$k zpm^a8a9sF)l4gPgXg(CAe3S|*b+|$xds@uW3rQWDI+dr!$I^qyC&3;}4Fg8_eU(Ut zi~Qx|SQ{njCmH~osg0h2NMX6ehbdY`dKFp}8W77ZiQX;A{e&BtKk? z)GfbWUr{i29k7}NXiN|1m)q*_!Sg8>aDNa-Dqo#Gfeo2^9%&qElkj~EINacpS3vF# zHfo}O_2xT^5ek_3hs}y~ztf6IX;1<&QGZ-mT%^2h{Pel^&+B3znMgyF)YT&p5Y|*+ zQUw5V03>sC|3j?HqDJiKVADY|a^&&;Xm4+VeyIuIa1OTER^QI6u;`#I?jl-N76niQ zC3rlj!0mTTm*(~vokJ-81!98)Rw4dE!Se}R)Z+h!d2X~?HVj6BP#~N(x(h@wpF^x3tui;ANja4=(<|zdjA7xsH90Z_u|$8?dbiqS z9K?S{L}=xkw+2x~jEKz}3L9O3`#v11tPx-TimE}L1M!>gd{u8Cr7il%r#KZC2QKUz zgYvcLuC0zfDLI=+Hr)Ha8T1HwuZ7$dH8urz)ie$xs~)CjY8&TIXn^+h_S8A!A=rTX z%i(D!cM(??2dy@N8Wg@IO%bM=2vfQla82eHV!r=-gJq_o#y^PYd^-0=w>jT{)}IIK zunVxJqR^%(3%6)f-G}hW$Gzxyc_{H80gfW<9GYKPppZ`22G*DGjizD6v(R~)nH^8q zP4-U}6V$zc=oqw0u`kxHm-ULEjV~kuGi@J$nJ%{T_9uM{5XFTz!cRnfwa?bpc2Df; z>dD?1vk<*E%)@9C7?Zvko*l%=O~U#u%8cK?d*Vr7p+v~=BWUGApf#G4~rs->z)wS%jTiq}b6DffHiHY2q6kR@9yl?Oti8WEJS z0mD9ocUF#Vk)kCiO3=cfiQxH&v~$x9sKIORX>m9q88Hs_j6rNAd@2DEv?jq_>_&hF zMSu$J9yeQI0O<2NEEP-)&O^utxRf8eUI&;MQKPCHy6@UTx>o`@aB_GL`wq2kP?FI8 zv(hiMfq!@(n-yY^*xbG+uK_y<=h}r8E%w zh-+A2sJR6Ta;rU%#eFGktmwdBUd?@ojjJ(_buskzl?lB9mbWjs2~>q9t;lik=wJVx z2>MZae`e5QZEvroVDQ84C%XsA^p#%jGPbEbbIXGhBWu37N6u26dj}ShsrPel!GZ(K z2C}z{c#q!KPkP@}WM{=WmL}o4IW2T)E~R}#h25NEDx~K3xdt3A5?!7LS5nn&_QX=e zzhYvzt{pmp#kUH-AbV+EkkO*6!Rb;Cp4zSP9Z#Jf)+s(iK2?6=GK{S0Y5^2@+%s42 zxp$?JVWdivX#rc_!__;A&#yyOw1;*q8_BS^&+X*_vFl2A(e~z%8 z+;G2Vh?7qm;lLgGSiE9)S+iZfq#aIgw$v9ALsm)<tbVP`Ya}-(xJ^Tzn;M5zR>&?|={kw91WDY+N83>h{|WriV6r}?XbP5gp0Bo6 z`_&`0@nF$kNs-X z$30}}x@DbNdOumm_v9EgV>B;PtRG?YFFA=PPVBhr{(QgU^oBP10^Wqf?;X=vnsX#F z-#Z@kxY%C^4#{5ncsqRkXnBhvIP5_pbjMGBJ%4Cu<4jNO_)xJvqrqA<@k;Vf9f3Ja z4A|UP=-$M;Zd-f$pUyerX!l0;^)PL3ENY_Xn>ync1w055oF%ze(b@C{PB*l#)x9gi z)VU~R$HaVRtQQPp{!uG?NA9ppZsqv+PvSRfREfH|JtwY!t>a*PCp-B~CN?JEF?%&V45nulnZvG*ugX~ygVxgNQ|CL~ zgSWO2hurBsdljjQQ=epVTe!LP9e4;OjRR;f=Fw7CRyODD*K)xP5Jk_!)CJc0GbF`8a9`t?eHD%+MiuRAE^T)zDeIk zp0yNl_CxU&w5+|^*6hgWxXJ7o_jH}?j2wyVziqxM6fHFKJK5>jgKtu}b%K5tj`*e8 z+X7;~$D(Ke($DQ7UUj;pQ`ooBTlS5n(`VIb!?t$U7S?jOx8EIgEDBM|(BU2{n|2f% z_TKun4;$UW1iUAR(Z{k8e7zd;wuuXk@6K(%ooBV}DWXrY3cW^_*3Hg#%=U-pq(eMw zqQGQzbmlPGNLamFTG({^+a}G&oWl!|D&!s%g*LCDT4k|3Kt*Em1YAn=6%FYD~f>>LiXlf^7rr`4|aDrO_$3<7gaRYc-Q2 z^q<{|zL=9O_G-2L;Ogaw6p8u%%NL{ciL@S2ij`~*6R-x&;9FixY=r~R;PXQczG6_LWZhdeBK>qSo!9ZI^lk!vCG^lxnO9|arf;?-b1HYrl;GIIoa`7 zz4A-{Mo4qT1>WlHN%$^EQarX8SCilsX_zolIbpLzVyN3zQ}wHKugA~(^cq+3p3K>n zOj~a+Ej)WK+fxOAE8vJ#jd88gJY7gX4YMp6So4hTjc9{e{OJdV!-v{lgV^G``Sw1! zPRpQhbQ|>nk4i%6jWvB9mvovRetT|yDoi^cRbpjXKD!>rr)_gMLbp$~)mp27rWOJE zv$N7dpB2oJA?%s?QoqwVztJ7>20`HW$iLe?`CL|}Lcltm=y#BqdLKROJuW}}PWt@B z3TcRNh3L!gI#S5DdW{{8mU=}hXU%C&9Q+QE{_@qHKhbHW%18{I642isKzVCg58Z^a zFZJkiNtN76>NJ+vgEXV0*7}W#`LsxkPKcY;j&YftEZ!E_f6=+; z0x>WZJY2W4PW-m6`)yVC)fiXICfl~=myO;4uYzfuDn~2S)~g694k9>X@*DvAk3;{$ z($*G%8!>L*wo<2o3Mptz-P(9B;1gCojw`?dnC#;xPsCb$J`6jyUxpmm7o$|Slcp)FKRAVv zv`|^lb1pbxEfirTsS8asW>Bq-&ME#`E><+?Yx0BY^v#`naktxf?s(E>IXtV3FJjnd z_d8ALsFi#1P`N|2c`$R{uUA%v)1xEVRGs#QL0^s}#oWeetE}TRl1E%N%(~Cbi3y9Z zhMrV#NUX3W%#A+j-})M!voJj@H`jVC5=vAeJ3=|+X_{0+Ro|j-uJxPiQD5|4s}of$ zEeF{CCqp0GW%*UGh%vk;_+eaC2NO9|E&NQwAx}saVo!Px8RBVPrI?)GbLNa4QMT4R z<_aP=7v+R1>{u=f&3X8)Gt4$P>^)(V@t6ENQ2{i;)5AYHgZh`;AG*S8EaZJ@6x4_x z_L=oxzFwth_67Tnbb8>|BPsJ_&WbtXhKrk2@~!tQ-`wWpeR58tn=SHMN$=`2@+!^m zkM5Gn6X+U54!g|WTfsira8}k@yQZ3GEzB=rTlIvI*4b2tPG(FVzj*{ zsT32eii?WmMu8_bTQ>K(eE@gC0zu}RH*d)1`vtyQbGVH=w`Wl3 z*)w=ryK}d8X0T;#FFx95`F?@V>2l}kPmhx+<}*>&pcqKa>8&WtYChLqZ2W{J%GYN* zi`ivGCRNL%5UC4f)>6VVRpA~tR>qVk@K|+9UwZ5XU5O?#yHQ5Bbb778egTWilV?Bx zIGXwm4UM{v+L{%lNc$v%Dgm}TwyvFwuEnBOV(jZVY^XsUekOFdwB1bGq>j7szSZEi ziP9)+7Nbg=>pVL8%aeYxY~2QJhAixFtC$SuZN-;f4>)oee(pZ9WU7J@qawAa8)V)q zL-iK@CKT`=VZa|=w!a*BtPJVAYMk*=`<%Hv=KBpdGF;Q0!ahmeVo4EY3(pagFVB25 z{bp}9Hj9~N=T5-SSrzGZ_8`VDUY3OeE+^LaIopD$RNxDi#e3$0$p0)0T1uvh)y%;~ z>X!aQw0_Z8FU5BhEOeILPlTCU>n61eM6}z4L zO0AS!8i%{9F8`Ur5sIHyqWs=pp_NY4YWdCgvO#y-LJe)j(c^}+5*n&zsOIAvwXW@K z(*;+LEJdY-aXuf94~3rvb_ETW46%ey(t2vIm9}~}DSltaE*^4DZxPn~v-(u);wfc= zVCv52zQqa+=Hg5!rSjY_q=P?=I^9jKx0bDrI6b*bG@;I}nAK`lejwjQ?v^g)@RM0Rp_w+p?KTH&8VX9CYK*BmbRC_4!U=2rz&9Ma!Mb9A*kd-@7ux95@bcDnFZZO?S4e9L+TdDI$e`Q94k`Pb#x3~ zK{oE(jt;jXCH8VL=uKRmr_q7b0_P zUR+b72?~IJmnf{`7&sEXoSdx&6-X9i%&!9f#p7=llhWzQ*xAUAN8QN`B=d4@xJv@P zAezplhg#DNd=ZR)&FT-ZFIi#vL#*PQ4AC$CDg7}w`q7I7JjCYRg1?j^Fd{xCYpqIq zjZ>}=7U8aLSk4YUppe&E6KOLoSyJqwvbbI4MxT~q>ulUL6-lb_V5`9uBiKB~>lV+M zh>4pK&GsF0&1z##+&rS}Ao6+#wl3#X`s|0div=u(xJtw*@KymUfja|QF!e8Xd*#n5 zGUtYCb1pDI;H!zmqz`{A++MNsP-D*L(;Vs@R>XrJ-cG$P{=!MCiYS%b@K&$I&fZEla|R!ZlJphs*=s_9MM_x-&XTj9)ohfIYEI?S4DZjq!@kMQL}Q|BEXj1_cP zyT|@~uRs-G{(ATD&Oq(R{Wj<>Z02`+*)oq_ui|^?YG{#6yC773HErSXs*t=h zecHBRof`HX3qx9TOZD)}>5^*l|0<}GYWj2U_rg=fyH~Dki^VQqld!dYL4NnxY?YUn zN33ehb?{oA(A~TM{)VBYY^%a}C;TUgZ+~0ivhgO@;L&bzl5$NKlpGbj4AB}()h~L4 z$I^Ihjij0Cwvl2qP0DnD!RpA9ijEpa``q?QBYSd-!le=Ax$6A?zIk*OvSi&1M$-7cnpKx+816(aCAK#%?T8mXvA2(Jvz*Lv#g~N$=7UBq z=?U>U5Gkukr*k27W+(5TXl3?37t9#gRvC*r-hwC4lG*|#Sv^Tp=seg3s--p-UaL`7?wpq$cg9$ede5{B6zaX(21Y2!{+15Y{ zi$0E`J1>sM$xc}+fDrq(@)Rm~!a;R(_iI_~<7?_5;qmBjN>cz{p#b0c#7aHGK~w{k z>4`)I4QhRjGj*^r(BHGG4}RHmVzi#^GegjV1#zglZejPF3C+*;*B(w3f%eIqS4aYf ztA>mt#F(Q#*kTnzC%Em~9#(J0)?s^Ja}SGlRsV}CByBy6>Solm#zL*=7;4qE~t^5XS8_tg%W8Tf3Hepu0OJ@7G`&z!PUeE>e>rF>}>J-kQ0VZm|ltzRStb-PvU+bgY({%w5;)zGA0g{b$hWbJJ|Y z+>bE!mfuBUUvNUJa51pROOwhXVz0YN@=}g+2Vd{LF0}!BK9l-kwQA!; zKgCif+@Q+X!BlN7@wj?=&^M!k#i>Uo~6N8D=S(7B~7Mc^04D z(X+6$*EOSSwQ4jzh%Zj0l5fC%*SL)de?=PJr}6!wy~H*X>s=GuhGIU))wd-HOtPU* zaEwH5lHg%a?j5y_Sd*eY2~$P#Uo#X*<}@XYeBwKJFPJ-$1gV9Tsy7dWMK z;ykU`LYMXNkHTXjID)7nls2Q!_M_>%RMEiZf$=fthE}??=D+H_C#d>kM#uvIH${Yw z*?Xn%wNC>ib+LAxxKanBtI<<4QC zzb82Yf`jGnJ;Q6p3Io{o{_3_zqN)u&RKZW_E1L1b;v`+?0`DYT7H6U%wRTGSJooAG znxG^_2o)EWpU0?j2aOvN5+sswllm^}K~rp^hlc1ziaI^ zeEu%MsQpf;d9tN=qH0x(*wa&6r@wP*(KLW{K+VO~Jm?FvuysA3c;_-n^Yr$pIc_)1 zq;BaZ*Cq4*%y0McjLyYrbmV^_jeftc-*Q1@Raa5``QWKb=Fc-gIwT{}ET)o9rS_qgNmn~m zYAx4RxySiKvJW6uVLc==Eb$8|s>PF(>oCR&51wq@+AcdvxzGRGB@i&1@vU097#HJMU_W zJCx(kLov0^tMb6l&^uZfo)gW8419qAot5+x@oqFJ)RsUIFJeA&iRp>qQZ0iVieV$c!E~R;`vNlgv{xbOp4PepdWUeYP4I?xggo>7orR+A2xDP<=6hE2fUeY$DMeKAeT)J(b)#HWH2gOWo{5y3-RC~n zW9&F$q7i_fqXXWdKnUeiV<%Caou*cd=GeW~QTmZP-qBR$9HTYGY7DXtPS@pQ2t z4~P&;1BI*+mC~0gJQNaZTX4jX%YQb!TsK!znEyx|NRJP$*Dx?D?Z)3Wu*AQ}`dls` zPf(eMk=_^K&KIu8H<=Po6Rg^tMSzDjzb*E-r@i24z-kw;RmoAOgdm0bOW!Y%WrA2H?%ASA6tjf z-*ebNwJ5(Q@D)@Rrj+NSbnQ*Smt|wZ<|5S@n-{phAKxtjNhwITi;R9Yd;Vqp(?$B8Na|-CyTMsWE^BulJ=$3J$C z4K!PdB8!(i+zut{Gv3;oH3%1tEl)(GpSe}fN%;jJwm|j0UjN7lFbV$}Gk$OA37eP9 z&O8|z8OIm2{lt;RuzJa4t+KztQ->S}6T~Cm-6flsO!vgAr!K5HocQ?p?ayAX4MWI| zsM9P-TpI`a|DX(PQIE}O+~@<~j)L6ai~oI@we(h!JJfg-czoL|pC!UwkC+RO+QRf- z5B`x3W>AYLoOUYFcOelL5t|&{NqCN$33nLCgHBF-j{Boe=ucPIPDy=h&oD=p4j`g_ zf2sD=RuTxJ@8D=Zy*t-GJgitXf`@^yQy6LYiRVgPtNyhlv{&9Tp?$Wk4W%nkex{4?Dv}CgQZPwwVfNRm_%5iM)x$eE z58VY@dgs5SaLwx@L7>loLa8<xkviL*vzGUajnO6b~Eys(6r#z-dgMDm_q*WcZhW~b=crV{)ZUAQttV3JR zxm`mfv*#oe0^uI$G;@cB+o^VL+Hn#!Gm+qx8G{3>d3X1k^mYc>#3wk^o0w=3ZJ@T7ZV6KOic_bM72y>`Lzq8MF|Hk(*g^!s22whV^C;G%CZ4aV?idlx^ zDeF_3p&d#%`yPvjsftc7_pLesKPq^cctVGK!v+Uv+wXHuglpMW89i2?O$y7UF_@x0 z2x?Hp>?YSusb+;f&ydr(L9ybsLL!CE&w)wXXqp(ciz(L7AsLy_zUg7BpCA`D zX{k`AdnHdh;Ow}*{LGFJp7d&y>UrVYGf%zm~F)2s+y?Z*|(R&SV&qr$K-ptaD~#AcQK zz80uNKs4}$y|}L(ci-C-z_p(japAMQ#dFWV>0UYJgW?ydzj&&d$TYg;6K8aJ>6Y(n zbW{BO;6xan!wQ6b3sJ{wLRL|mJxq+dJH_pcLW$JQhL~K@JrjljT|)?Iv5t8 zf(BsTi4B(c<}@*KC_T^qkDE`?L8(VDts_Q@Q;=R$=hEywdQQ?opT_&<`r#hdrZr53 z?BCUkpLJQzzNWHNYI5eU6V#Z#dI={GlRGCE2yOVR^S$@$_B7(^HWe7ha>R!coR;@I zV%08^o6vK>!T7T2_0p@%t9`)_GkdInHc$1y_GjX<;BOPXRaicIUoiO|KX3m>##Qxu zZ$vg1INlwQ%s8G8&m0FU+5X=T($7Prmd*c7Czu8XPnY|zRD;mG;XczAy?(C2^2!Q? z6PPS#NZG(L(qV}8aG#_~-}i4i!;h#vcvB1ZlY{gr;5-AZ4C#i~klK6*HX|LKti32$ z0)zA?{l#x?r%anko!)pd8&on7F$$M5Yx`z$+x8J|dolb2;QzkUD!w@B`;A=2g=ZsU zCl^eQC+Ql5HLdSLk}86uk=I6t8rQlrtqHI%!=DeX_7(Zv9 z0amR~A&OU*KT*J)u|aSj5XW#dW1!<3>3*u=YtcUQaiKr|!hygJ1l8UkbF6@7q2WMZ z?SB)E!$C*>8v4^g+SV_7Un-1scA3NbtyYv!Uhw-)B<~TE#0 zp?mjJ=vO}C6{d*${~7d&aCVDX8iIy_e<0uDi#CD$&yX zpp3E~U``|Bx(S+&pl6uNWk*^UMeju@!i`8dJMEa_@cjQ50Ikw+*qZkSysITCy2ND!g` zSN&1GwBy1wZqr$pCEkdj&ZP&bM_>4e^O=?iioW*$^1p-AGdbaOA?m|ddb08knG_97 zj5c>_A76hx>8a4Ot7^>-k_}-^SFdVN;paU?UEOIp~19YOAVo@vH{G$+SgC^gHp zs4pb*(QAR66^R%Af7q0u9||PJCR?_y;5l6i4!afIWI%jXA#nx}Z{7@ZHE0A|v+D>I z{K;QNh{11d;c=Ab`}ve;nqcU9?ukST3PIkbAVoHBORr3&hxVYE^c-T?o)R}ul6MkJc^#n0^-UPaXn*^NtViDfV}% zz2T|QBXoK8nWEfrGPSsMXfGn@NkcIFeQTp@PoWX{I#+|XS$At*XN^@IhdljbH^1>j z@@VgA_M)1aNU)UCpw&)_x{Wr%^8ATaYk`1ZK6h-3Z=YHJar+x33{JdVc4ZqN-MGCk zb2fK2J_p42KTQeHX}Rm6Dw0?W&Vw$-6~@h-(3?zJ!wJ!~;<}NxSB;NQgWqd=8<})9 z1Ogsy>d3J^sU+Y@0obPRaf=@#!bGFNV}p(hy157Fr8i9iIN`Gdbw_Kv>8_CX|6}V< zps8-#H*S2Jhs@(vW$ZeV|}?{%H$aeR*R8le$s`nTHkPv=f;C13B>6e;u(M@SR2 zf@OzMy4u>m+dRgp^V%`!u;n%aR!OrDbt(g17)H7%f>#8_hvUXpr$Nj8x(#j4MYgoN zi3LX$_N4QwG{{%<4}|wkw|Xh)TJ7Me+9G&=)l}mzmm}#}kn$Nzl|h{c_B-Gce^NGT zI33c!hF9)dvV&Hiy0UM7L4S$SxeDew#xoKtZM7%eDih+ausjLnG~&*#cOgreY@dh zSH}kf`cj#CVpk(wZTe6B9Qpa{UbUixpsT``5e5k~5}&MS|G33zt7Yzy<*d#-mKRkx z)jv?K=Ctgh*5#Jsb4bwI+B=9tbvEw;b@EP2TzM0pT+8x9YOxO|x2wzmlgv zZnmM4V6h*+Y{~jTe73;+rM5W_k6+-^mahSqy-`237!*$`pL)4jKq$72S3T}xm3^$C zLPliQgnkyMVDkq>lOMaNHuZ8@JH`jeM_NmIJL}#%ECh++=P3e&xER`}n6yR)2wyHitWExs5ZP+nI}QVMd|gXBWqL z&qhu6b=i7ZuKU&Y4<@*7?$Y|vp0RAnFLj&*dp+e^Ofyh&kPudGchBU#sMN1MM= z7EgSUl6(?rvG&H$M8PKdlsnlz*v^A|qiu^va@cLmx+fJ=Je-(z6SezTrJJ$ypyXx$ z@OwFE;>d6$jHvUYuG>NcC`!iZ5ilNGQF~JSD~XKf;S;|Yc&?eEXx;wz(}BOVlsB5H z|Bg7I@cPBMo6~AxaG3-id<>Q&#@d8v;sR~kJ7X}Inv9SAHviy5W@Z2SaNCZ*xZ_9o zRLJ1q`t1QIg=H!3**NQ|Ao71Tv9KN+8{2$H<8sJK+xb6+K7w%+e{AJS-Y)?`^+2yX zmYuT4J9C@A;kKZ0&>=iL`ppN{-X0h|zjpUuOt*XVP0d-4Y}1=T;ObMep8O~tCO6>R z<#}<2`#(oG?b%;^!0~~3{(VB?8bV$r=IRC(cHP5dtxkce3Rn-4dtX3x8i8O z3#$9-SWJ>02H&Nr58H>wA4K5W-IX~!D;#I7GFE@4=e|!xcXMVNyZ&!i^++F?aL+C5 zNAGiQA~+~#$&k4<5xT1Px=))e|KLdc`2C7c#S@e zk_%K-%y~KTQ#glc?MToGchQJ-|8$R+UW|72j<9U8M&T$2{rHwI>B(NQ3D$4dM4XJQ z6tj4vBemzpIDYQqYz`N>ruEHJ>E7VXSc?@azPi)RCs`|b{vE!+{KqFbF~gMLPh_iU z4Z*tK&fx9U_aa5f*~5B)(5boY?%CkGG1p#g)M+L!mwBfcK_0B|kH$lvWG4+?#y&Rw zoMmtm{w_edS6O4Zg`A&r4JrD#-HnN}uk>?CvG_r1HcBMSG;TQhV=QZ=rBhYUOiv+; zd}R2y6=nGqr#KS#-t1X<-}hwwaHN=3-QSi)s+30YAKU$|FAAQ+#hVNS=;ELZS4q@q zt^yRE$#?FY(Vwz7dHW{943T+Y`S_z`=O|Dr6&7;S`eAhduQbf> zkE5qa2%#-ii7H@ot({h^npRCq26WLJAI^Ru z9M0Bz$RBHPLQa%?qzK zx>z3J{fFYvH2d7V2itQwcNiVN9!aj`m(Wq5cm!%PR5Ucnbv8iYdzml<@}8b zs}$$l$g=tvnmyfKe=iH;&hstuI-)Gngb}WPOyNUr{69|%*E@pm-Zj?fYV&YGAkzU6kyGEmF}V*N3DiXLRZG!5u@K<#8DT@;bQZ z)(*wg4{NF^7r|~HMn9ui`7*TcD38l;5mRoqOLC(uE&Z&a(@em&R`cC)vwK6=Xp5B) zKDyg!`ijIR?FRN9jq=)^Cfu4Y~Qb$u@6htyJbJ* zeLUeR5|&CvGXL*+lh>&YMXW4py@v44#mKmR3j9d@<$T3a*1)||DD_Egr?#K3(Wq0AJZ zR6{-Nq%D`eA$fZ4>!G~UVHFmgc_{&xFxPa`$%nX6qi6R83WhMej86e1!6U>i+B%N? zJ&iX7{}->o(HTd)Bg~&eFqKEz2Z;_RlHeE`MKXWmiavgo*U6J(;PjX2QX2ph<&Mui zS~R^s=a8dDq{7dISf#2*%O+Ni3rk)06qwkWKr#=h5oW=H)) zr{iSK=bD5iTTQ!Arwv-e1@l_O{Txhj97zwIXS;;_56bZ?3Lapm)9-qZoXsCew?oA40B)pzE{qV~tRpY?hKh-Y&B;?X2l$qA=m((=a+bu#a^-o9D8;rE zvt#Am19s26q?)IH<{$J)`k1R2Jv^a{%B;|nBaRy_q>K4U=eW|T+R0taBR^2f~ASYBXhxwAUH8uvkWGYwO zbLSEyqBf6??wKmIlqhm{bTzPsu#`QEh0K1JVRETX-G2r@n;lJ~7$7aOLWY~>>RIE1+QOyJj81IAo zp=k&z`yqo6C~eZkjXqCwDffBWof!(ieeFR2Qsd5GSQbSwASY?cQ>=fDcA3P;ZJ|@f z6!|sRn}Y8zAg?Pt8@P#*8(J`K{9!DU)v+!$g0dLJ5kOI7?avHhj4{E|vOtEJM?b>V zCJ5(NDn>hwQodeTsOzT)VuX+4&rsg)OAOTikw4F>632a0LvpKMDVpS}yN|b2s=aJw za8I!22{qLd7uI#qcgTB*tC^=n5^HZi-PTSxxRXd=U9+%!N@U3}_Ke(@dYCu1GW8 zta#06uawjLuSb(gPw8U(VmFBjGK*(bmX%prMwQ=^`SND6irVvYWc9H-UTt^J5QWjR z;HwjgH7rhxQY4D;u@;f%Nl)s8HQ!I@x%3#p5Q40q<0vlhpl#SO=#mu|8$8XGW_QjL z-?SULw0r!$-tL<}+a=Oe%`-o%%CuxR9UGxa9#4#ESHIf95$7z)Z6;knbuWV+^1;4e z$;K|?ly3ZP^MEuwqu74@PzrH(7H@l^+f6MNgZ3L-oxxxI z3fE>dgl8TE_kX+fZ%?+9Q z3K@C`)Idz#`b9{0gER{#YaEU?xAbIBB?eQXqgvKI`)%ZJ%i28wiXWOb|Lu|#AmlQK zIFbxVJUR_s<11FYM^hP4V0zf1D^)0H#3tBY%$X{b6HwwOQ17VI6ENdtd94PKf%rD@ zdyKx8i)A@%zA6SXM4#H}`UtDMsJ?OE3%mqdO}1`#_g9j>ZjCXsFjBr{LbTkLZ?Efk9g>CQ(m6v*8xHz zD@_dZ6h&Y%umNLkFoN~r?rYaKU7-I9AcsI6ogbV#cE@S~qH&9)iPpa`l1s z`fDNZUqg~yOf`ZO4LzYD@%N9T-0D|%=#jVzN>N#p9iCY7zpHy3v_*K|9k znYP$dk7NB9$;*_cxJ$B-72#5OX+P;|uw^fQ80C($Kh&&v%EKbO#>30KY5Ruz!?EQl5c$Y@C4W5W}6sE zIsEaS1M%%F74dU%_9AjhW$lSAT18h4(9o@q5Ze&CyLsb(Z06k zu6|=2{E^}NMq%c7yEtcRU9t?KXZ1#+T{kWM+(gBt@ej$USlc_XCQ?xp$I5~Dg<`*- zZ5QvTTGVXwr7Y8VT%O87$#%>!a&%Y_UYr-f^X_}HR1-wr^f8>Gm)*8tZfIx#f*|Z$ z8D~IQ(%;fRHareO zTe>5Wa(VT7zfR%!`agbrMq!gQCU$jct9YL0<7T1fWRBrrj(PV!Zk;536ZiMkeoEme z07YJ3E6?z@MWka6sk*PW_1|7bEYH(oNmF+8fWXX#jJ5qOb7^M+36<_d`E7+(vZOoI zBu7U59vC4)+QBMU29)NGZgbJqRmytipb=BzVM@2+uBd+(V^q`(gTL z;XQHlPFK3>y5t~v>Yn>>dJ8<%z7JgLF$}GVX-?OYRvWCIl7xA4oRd8K6-%@lr!Jm8 zbM@km))kW+<7_$Hxf_Kj%L*q`oFehSMPq3p&OClY*x;lm-pjr_w{DnV zd;joz%VqVkJYG~@xgpgPA5?WrO8f}5^G{Tg@7I@Wo(~fV&W#5dVPy4=MRZn+p+~&m zn(Z^YozRy^d*jbf|p0oHF`PMu-xI+m4 ziqBy_2;UIg!0L{|wCZmKPn_;6?&qG>V&{mw6a2has_8)xdqwAwOe#&OE@KU{-tiS9xC%8m19 zP>S*GUfjMrigLX8gq>8`ND-|k-%ns5@9?yvrsS`LycH%&cM@0PajVz!A$Ypm|MO%0 zz`0P+&M{o&6ZEe7dd#7lrlau0NP#o`=jA>Z+bN9W#(CB2h&?zf5=ST*!G#CsS0Aj$ zBHOfUNCry{R)Lq znhS4KcMIn~UdyPGtEizSD{BQDZRkw+W@$}2{AtY`#5_}u@0BV{bLCyIrj6>RtutYU zSu2(4BRhRqfPztI;29L3qrdEWc*iNLBo2CK6i{ec+7pK&^Rqi}&>_-WL8_&)G225x zKH5EpEwwEQ@=~ck!^~USAw@?{>c+0Px^j|uS)u1ai&vZm0pvXmU1FKZVw|Z?3Ub5? zH``0ND_+c5@>r6C)C^@f_9!?f-R|IH>P^2!#A;}0YaXY1A?NfP55C-!(x%JKJojg_ ze85VFW#^h>8vQ3otr%Oh#3EZpUadXLRf?VyXF$c6=_Bky9)lq|KK zmL?*Tk6Unrs_OO)?(Rmr$`=#;nVx#j?i?9% zmz0)5ZyhP#xbj=*iU@nO{77AXHt3)1#u_yb(BR7kq~axu1W$ePn$|e&pkU>>V>Xv# z*d$1t|2k$vO$lPOS1g{^yxXi#C(Q@RWGF28$Xtq3p*S%qjlatppMLrBE815-2EGmH^DA z7#&VXiz%nx;Ym8$1ilSZ;i8vLhQpa#>U|`BX0UQf?&(K6bHytIz0*{U?A`kJm%S96 zN!RnUjnya8c-4covP=tyoXuoi3a04QSmL;k1oGB7W`l%+!s9PMm8sKt`z`&KT~+|p4^+TgwiQ2psK<`i}pwkD9$({bgV20 zdo=hWPG%F6OJpXhU$b&wV2c7Ppkg0K$`utzb<(&Z!7)!Br{YpN6Z7OGk;9N1b=+h< zn|`n0!my%=zd$o*yZ1tlA&EzC%c;~T8;g$PJrj3j-m-Gf3NzU6JZNX}lUoOz+TNDJ z&OgW?$m`rQ+p0D;<09p#c9ll7{Z8(&UjLVpbl@ZsOC$K(+SQd-)vTLU6ypLk{~DSz(|NB7?T>&p^j|J;?y{%UOt zcVyA-pyx=fj`8cL+RrA<=X|kOcl?=2I{cqDy>4f^Xlkk!amSXzxQjoGGmL=Ep}SAz z-6J2VqdKe1LYI4`Lafq}KcQ;l|K{6tQV!qD;X#Hb({lJ%zT?}^6DcgF&bk=PEJjOdIp!(tHpMcb z#P8PwH)q=yGj{*Vpqvv6l%(H@yP>~(@&d)qARO^<#I3?C@0f+**m=D3nLW*+>;@r8n9GIq*Bn=-Cqh}`Ms%2{Um zdPWfst>3z(AB$V#DEq#c#kOLb%b{%4k?0>KnqbiZ>GZhx`m1kXbV$u7#+QCZRJ)1I ziu>R0le_rsGTl)@unW!iBQpF$h~5*v!@1R6Syz+;?4y1jH&qZ=CPBZB z^tu~O6N}<}2^h{n?zG~KCa_35g;YLaq6)k!qBmeST$TY=LQ(tv^rHTw##x8{`F5#P@){bxDO{r^q`!8XT`kilPPwX)2%e7QF` zDV`ZGmSse85=HyjA!*+`)FC|5YDb?-@*jK0M74AzRFYfC&>HSwU|uV)`X-$9UZ*U; zwffBEN4@fVep~*i6;l$K$vh$3_~7*p=6aS?&n@5fiT0lgz=_eED1p}t_7|wC2#&}B zJ6rBgS>ttQ9wcBCi|3X&w9HTNBmZPC~ZrLf;7 z!DsirhBonyY~!B-hd5ufHWL#u4q>@mF&qx}V{@MtoBcbHEe$8AU$yMpwaNS0l;5y3 zt=G1S1B}ge1`G*7v~|gAUIJm^J~AVo`}Z(553I47dI=>ysdvmDaE0n7`dn7EG;XZ1 zV$)&7ps?Le(^ljlklE9YX`IptcZ<`% zV@HS(vT3xb$-7GA>uCth!Z%;Li>CbKP9d zAw98)fqoV@dhpn7PYk2*acW4ZlP8Zapc8`(KC%#WuiCLI_RoAifRHela z^l481;Ge##Uzc`7unkjS##@#gzf)7e*L?OaRz0$@+wq>I+<5*gl3OZj#*V7|Df;hM zE={j(WqSr@64KT{OyzOqYZ6382ZiS{6GO3=9=oyEn18%>gW%BUdNuTN0wKLTzh~UH zwX9=O?4YqLi%f?~K?qOMOJg?8FPnZYuG1oupLK(hwK=yxr0HLwS~Fht zujr014fdB0jC(Ebl9u`yR?CXg$uRs&ix{Jrj1Xtkw_SZL&z%F>_Bh${!k%KdHi{gQ*Wd|)-(L0oUMM+)}s$zeJi~sUg8ltJ5>NPzW4n7+Cl7dx7xHuv) zox#V;(at@Pc)S8Flp;~hcdZOu38bsa3(WA=g@YWjW1VHKg^cs~&9>qe(UaV54?F>7 zVUiH|*9n^+)|_oS%TRKs-@xIkCY4$|MX7bZys0QXYh$K#{gs*ZeLKk4sxS1>uk@M= zqg+MD-q`g`c@v>&PP^^oD-IU5K?GnVooBo(pkOFnGddU`bTp40zjE~T=wCb#`pW$2 zlAbW_%a$^w^rnJpOnj6_29NSrdB}A(TTPy-KyzAxOiru;W!rgRXGGI+HaaHOpaAO8NG~>3&L1!<;QyvB|BM zFNrDKEeqibKUyb;?!HIatu9i{;|t<~%muyGt7+THt^%VKzRMpG&OZM9!)*lyuZ4%b z<7uxX?KQuKl*jKA+i+PV->Nfelwq&**nCsenj3ptibc&fjB)v1$KztDuPhu>ehVGf zFB2|c(Z$(OZQ~J&#G?C?Z6Ci@0*OW_1l^w0HUBrvoeJhGR%JJ-)OQqI>dNnRrXmdf zBtMya>X7VCnz4Am{_K#+ajrmBSv*)gE0n>tUq9C@DvY*r%6H(@(v|c|dXQLJFf_qu z`BvD(jSH-U!|T+bnR}6shmZoX`H{R^gW_4w+6quL4qxdiRvemnZ?@R+jy0+q(q@sl zhKdqgr7o-Lk{g>;o;Lm z4MSWyn1qFv#Ys5>@)b5dDPR%kCi=@S?leoiX!W4f&;c+m%n9=j7Kn&+&vC&OP0 z($vo#+uFu?A-Xw9;_ZvG@cLf5()rnFOA1C-Oq))#5jt134+VJ1zobLQtau$Yhtu)u zuQ?Bd<#cww+Ht5!`>M7K(cgRzYX55T2psXSysnyW=uGUa>|(R)Ziosdj9p}JH-F^@$t`>{#Z$^=kWN0h0|Kosq0Uo=>u=) zGsal3r-}2lrDdC!0t_#VG`V&8L*$8@UEuHPOJBBj)8Yg!cX%L-?mn69Krm2M zj+E!s2|;T$6eIwgPUtRUK9S~HQCiaC0!iW{%g!3tXoP|9NAkv@odlyhJ|GBorssCF zj=}}4;~SnC((U(Ei5UJy_EOO93&;}Hx@IgY+PT)XnGV9vXC1!oy+r$11~J-JFSpr8 z#@M|QOW?#7-n&&_D<3)J)+U@f%m6e;aR12is@Y4SXCF{`LmMql!=)NyuT?dobppk8 zWP*qxUEgUy)gKxR2+|bBZdml32Utbf-)MuY$V&NG`Yu~j)tPXi*n#8FB5p%Xcpu94 z%Zs_*(E*4P-v+XZ{7t7119#iRGUM+M54l*kd^}*bsCL!lqhyY$Yig!sBAO?2;N z1Zv;K41^f=_X*y~Rd02tVa5IPQA&VI0w2%R?}mm_Ene&C3Y8v8F}^>%A^7J)w&ol~ zyI~X`Rb0xk_p)F{9iyUa_&7bTYogUhBSxPw-OzReMcc3Peu=cuJ(CT>sr<1X;=)P! zCppGPvY@U<^vOgY^d#~5)W<6)bmu6Ef#!ql-*)K zfR(xz=LSESvOOUq{2J98=7yxHiKHB#5n&5VOdmp2h0pxWBg zx9SMTI<7}OT!XF~7IimKuKjy`@jhk3(OH@Ks;tK>lmQW`80~U-eU+~7x|hCf|G$7p z@MGckp~6@WNPs{T`7UrI?K6WjPwEdtk>rxSo1Hk~mQIxM&Zd_?6z|>T8}Fh)(c@5x zL2iHUl`>_gqja7qH$0*L>5UOurQiX3{SR|~SbV*iL2J)Vs4E#k4rB57>7#Skml9wA z(y-<8hATYh0`lzpkQRHoGkXgb5b#Q1iCwCg(ro+Ho~DtgKqf10h0hvI zZZ3L2aNuO$eT@u4`X1N1_kPD#06R=#NrXu9$O6WiCAwwBR7|6jtSDW4?!~4P4ztY2 ztiUtp5HB)+X*|5x>%$tmD#DM`dcE6MAv~vPswXLUn&~|pY-Q9kys+A?t!DpOf7D&c5<_Gv^0Kp!4wy3gs7hPKvIls)ij0xBJebJ;TvqvH#4=O~z_0speWVd0h?Gam-aylGa!us;ps(cp6-l@nxRJ zC_x#%?^nomS38(3_Un>(A`7SXwiE@Hv4@49Q)TsaRI|~6blTOd$Re!-Xw+wnqu^ls z$*fV^?OaH=)K2rRSiNfobc-KuHJ^x#z*Yt<-YD&rS7PA?HEZzGXX}-cn2ydOH*H z2Hify4-WDBlr|WdUAsQn(X&M2#BzM8^YGXOKKSmCaV~gva}oi#Ex6E=Rm@%nCV<||%+}&3O^$m+ciVe2uJ> zuD?X^5Jc(_wUgLeb^~s7=Pu@fKQyS&h*nqDlnuTxvK%la0p!@exVxE_l1#PWsw%|R1~6c-d!=Yg)zsUu zU(>_`-qjnVDISm5mW}j9TV(Y^w(hOkI+K<9qXwNu)ot9Fy_>@PY3QRai5rFNjL-C- zrBJ~^j=Glxz7A>`uxu;Mwy1rRL$K-EzP-|vZ09Zcie3~qNJPX$^n7)yi?!9{OK_(h zFk!yvLjjE165xy9lkaeyNwZ`PHEl7dnN)7VY3!ET8u*wcuWJLUmakQNrN&Hk$g->6 zgF%(&x^hZl{4SuOwY>I@d-%_noqZ`7w|L1ohBAPJxJxVV}?FAFh+Aen+g`D z?Qd4u3YrDc=c>X;enV_@nNGrMF&v&m`lq9!B_M7QbxkSRG}5BPT&Duf@0ZDo+sq6z zU+0Q%KULZ4Z!L^*t|&Rdv>;=QAD^~Q>ue!#P@^jwiQslTz^Gl;g|=GaDxx&3Bo0FF zcgK)eL-c=DqX#(3 zF)vba0TyX@Q0$#d#%25)>{A;vs;12|#z}t{Z|kw3v{K4zc2LotjbG`Sr+GP{T>CXJ zx6v`jVTDSSzt7_*$s*}2zC@O1hd8Ju$Ca`#$0$gQ^ou`1$^|6?#io)dTy;@Js9Rt4 z+MX;hm}X50FL4b%@ekKU_e{XJyUu`Xu{wC)=4#Q8f?G{ z`~NQP{>P138N0)wqHVw=$RUc_0JsJBym*ne5bm>563CRDIp+Hs&3_bEbm{x2<%Ru9 zQ%ENArWUj@$?#VxC>`xXJf5TS*3kGl8_knMDJ&7RtDS%APv*BVn|?lBgRW`1S-TX=v$89p9N*t*yyV}DA(@JPZa#SS#cZ=r zB;0z>EvGiL-_ff5>N;_SP^)yKD?x$Ew#WZgDZNg3-3y$Jjqwc5exAo0w*?~*gC>c%GM?kO3 zYOc}ctTIj)OOkyfC9XxuG@Gr@FZ$mi4W<8Yk+#guDG>69PSOy9 zd}-k$`-^|kJ4I7F(l?=hBSUHBu@*9h$5&65#X(LjhzUBCE!8P}zY-{J@D&E?Fl@9h zV&rwSLw0vmy*{pw3^U@eEmpLLEql7>7TaCT;|+8=YoKjm6NemWA{$Y?ia0@sXtaY` z*1NH{S%UY2+uhO$jOo7W-e_&bxY!E19U(;0gP~Dy-X!uzYa74{3rvD zi!-MldFc^!c`Os@TCau2L*tY|3NMZe>Ba!%`>_k%RTkli3A6P{=M}NiRt<2$@=5s} zjNuF^D}&>Dl-{I+rUN3~)cDG~f|XOmlIm^JY<4B7t})9AcXGVrWj>>Yu}Io=nEfnL zTgU+qmWvbTqtc3ZNo>HUi_zSYTmb!|9dj@fLxz85b+S6rSqhtxpHZ`qG64J?_HCeV zfgzwSXwDZ2YZRHf(V!WQ;JAtqY6pf{VNX}}iYIOFEB_e34_yrdksG88p!aE510p79 zUQbBx@VBurj*%F@$I987>-`5rs72x~s43zFqTsc_IpdF#uJtrEW<$N?9cce-1EJr0 z30`Jl&jhLs0);-*t6S$dGfW@0`~$!XpC7|<=5DqnfE?hUmw20!eCFe>@zOZ6Dz_SsPh9`tetK62;mbCYZfV|d~p*6G<2&@ zpgTwYFO5b*yDdn7yf5h;{UJn;<2Qmia_R8N%&q&pNj@s z&NsJpeSf?>x@psAYVlFsi%+nCUfp(f;Me@5yAJg)Shq|ym@+nNPHY5j^g4jN`Qg|4 z2sn{Q?_bG0A#{Hg=omPD*;aWNW_Es_n_zQsrgs^ZwK)~pM#`6=%b|3v@{wQ&V!10Z zEd#h8FMWGS2KEN`<=M$FvWnPqX4x3|r9n+Eiw*99cmQ=Z(|ClLI&4q^2)Ul;2-zkR z@do6S$rntz?y(U%3amQuLVnwxKm7fvwkLH>1|>Ll>=MUai#x7#Lc}xtbLyuxz-yOu z--#=TKm?{nF$iZ_vl8wo#F=Yt$m@!0c8|1HVfiQi?w?%|>;4|0g)DAEl-rJ*>4gcD zj6}6rzdO z=cF)xpC^r{)#J`27&IV5BuFPy)>kjGx4&|08Ic=vD6@*~o3`227z2S) zJ?Va=>9A_SWIff8Vwy?I>q%woZx4T83JJx)K8%jBGZI3hjpJjag%cZ2WCYzB{Gt?zPJ2TSufeZ!me{~DQsF%gQe%`7oC|?_gAxD zx#G1fzvBT>ZE$j=bUbdk=c})1G->hPjb3N5=Yi(W-`t{P(6cte2Ego}W?-)8D=Dh1 z47=?5F#4Z&&kF&|Tl|HgVCz4>UcsJA{co-@%SBg#r#>v0{G#)2DCEfiWxx%T!Ya9i z-_X%-S$;Q_BQ)_jrFvU{w&q}>d_TS*M8;a`RA%1BVLAz+B5_-3!LEo^2}?C3eLaAE6LVN3KEz# zp*A+hbs2f{8dmmo&xkK{IHIRvKk7&b0U*kOXQc7A-hOu!!@Q${i^KP>DE1=i(e`|H zdQlDa-{NUb&~4bc<`C_qFzdMnn+`ANr#B$3G~`%t_Wx9Ykh-_FY$07(jyxF zf4l1;1*je@qDF68h5zfd)=h-}rmzXTajzAX`)d0`wSyf8BIo=sd*9(Z!b$kl?^6@J zy6>hL>t6QOR=N`Ji2loL2iRo!;@M_*r^5rwBnxA@@_ire4Z_=KvN-z3X)zN6BedV{ zqr>)BLzx1_u=)x`_iWftLxBTd0?UK)$Ogs0)S$xPz6hnO#@Mr993bf5!EFAV2xSfX z2Zf9C=kl{@wX2mYiW#MCwR^VNiYEL^;!rT9xsdi*UXAFOKZVN~_poK@$b zaz%`bNrLYJ$hw!30x{*Gb~n2~$`#_03qfYe#Uv_MDiW$2E+_zFJ;91;B1ju`3j6zb z9w%E~LE;!gmcz5gKVKF&bVBMUs9n>CNU_|B( zn{dq~-8^0bHBYrb-*5My){@R+@h#gmmPH0DhGu@QiQ^s-38ibmWVGQn){fU7WJ z=2PmL75mTA%}U+KCPFob!5feY<`g#G(%7$)Cr_GFiqSfk?f{|1UxGh_*Pfgyp%8Jo zPL+GtjjjLm0D=DuS+N#n-_wI@1wZkE7kb3Uk}+z58%^k+_8V-ihR7iDU4Z0i@`Y>P zFL}3gmRNoaoY_VF_z{P(bjGgQY1hN|gHpNq3#S}iR`3C9Y^;$R-e7}1?3dR@v;*jq z@R=vPG(JKd@YzxIMo~U`RR4rQd=x5)-eY;LBKVb)p1*o2cxktTWtLH{k?j7 zb*(I-qo@wQr*c?M6G@&a{ZqXPl=q|0@BU3tp+rb#Q?ZVbBFWl&0F*TQ(6sP*Q(48l zv-C%XirtYj@=oT;UxO^*M7qkHpe>0k(xQC60wn17v5xuXFX~F~VaqM`#Ne>ktTcZd zKHXrRkHRr*z={J%gcABC_^+}#0a9~ zhkoctiYRwSnE^pJ@V&NF&E-9SBv3@(ECwEF2>uHU1AD=H|<@bhY<`E6Qlj{lc+75SfaRcKXR^vLFNfOi%z zFMP`&l`*b=aY$WPMV=l#Xu?D93^@?WA=(Wz1beF@*Q`V!{G>Rz6Qv;V^NF44eTQ_1 zpX0+n(F`Lt zPQK3<>HThQwv!PK;tVk;kKNBkUxn*v%hakjXAcg4S+M}Uz3*5N`p_xPCSzWDMgMJe z-j~rFKhAO@@r8XE&FL$jrQEDpo~XT&v>oCVwajH0Gtrg6xWczwncaq07IaAJ@u4U*3S>p#{>ry$S$gzNTvWa% z`#X`$1ZB7;DEli^d56|=DR}Q%TMrk0V{>JkML#6DJu+kfCPU!X~K2)Ke4Z$siEOY6n1^#`ubS|Bm`=$(Sz=< zMsOe@Y|WW!fI1_3ln4U@ej3E)QFmq=%Zu?MiJu85*TkNeI;Y*6I1Iub`}gD`4R$}% zQhn7-WL;p>*nNB8mZhyG5T{SUnBD8C9J^DrFU_n9M!1K1LEo3e1L zk22OGjB|aBz?DB)a57^(b~)#DZbrWb7;0MB)sl|PesSM3`$fJaX(izU?>dbsl__<7 z1!8@>8V@78`$G3h@ME@=anX+N|KF$kMbn#9gvDPU21+*3BYunX>XUMu~@W zpE+(jZDgj+s4XP`NwmoJ1tvPd#ID8J7~VEd13MNK*I^axo2@44X0o3IwU3!w;EJkn z!juIUj7XfiaE7qAm$-`g3SaD2d>KO!)WKjz_}SnPk17D((f@Q>Yy%_$SN*qg`qI?9 ziSrVZ%ecUbgNXszI0*UHVd258LmQi`og-q>v$N?&iae{ENJ)hMa74`1*^;1j4>wELeJZo0`OFKRbOiIs_&^yF%T|4wm9=l{>u<#R-KS z?mCR`OlJ6wnCk>&zq&3OMRu6t(`Sh`j=PQ_=>lo(=-o5ErwV&aAw}+$_FkkA$AVfs zy;tq|diBpeBISmMQv{FeS0v?Xk+EeUvjU@rq{OLVe!DY47sAA?(pb>zxeY_ra*J89 zDOz%cwl9&~IA|gCSHI>wW^76dqBNir^RcCqzsSp5;vt@8^nUqy5^Chx7i$z-rGMC6n&rny0~_Im>+1>n&h7MG&il$b3bQ-l_v6m6fi1+R zNW;9lzWCm;jG=)wawuI)U^mZH)(Iuf2Nz^|>Y~wQ?j=S4d6Au3arN{2oDSxeX4-ke z0B_XAl1&b0)g4`aKE`KXV;dtne`11ih8t7u@A3Kib_;FP^fNt*rFv_DGV^CGEN^{G z&;QELVZ-tXMdU@W+a4zgQs0Z21qDVa^0YkbBNXID-d3_6zgn@0m?9;^TLs72Gm{!v zYf*=&z?s88PmgiX(o)>#_JS>Yjxw&*`M=@(v16#BT*0m9;11UGa?y>f62cciTxGxd zyu-IoSX~?8&{RXmNqXhF`}S*gn7*@E*z3$6`YaSSHHK2LkoRO&#vFp(N5Hc{83U`9 zO~N5$bMSeiJdynDu5j+zeV=>Z5posvfY0cyD#&5#K|xwaIJY5+zU-G@PKd9_%I_wK z1@@j(7N!%U#RMArK3}Kq(t@4|^j!G(IjwI<$zR~c{=Qk`$Yoc<_kA-VQA?ZjYl9-cIdzbdRz2L}j8|#O4+kqp ztTW~Z8O|Jy5lL3M-ihRT6W)~j@;|J;fLujpSM52=JV%Hr;RjJYa2Rs{6}F&?fYfaF zvc3YlB8qK@&y$K8;-jK}w1#bq>lJhR4cMp^pK{dsQ{y8>8v}UDR1TLcfwjjCOaB|~ zx_V6u^HjJNWTb#Jl-9}q%o@jV?^~H0QWEgO%i9WB#~b`{h!Km72=D16+>7}OWvCCc zRJ^K>>`TT4cHOYc_R*nbk1`dJuAPS8P&F0ZG18PEfk}vvXw|Rs!Vy{3kS)%=ChYTR z0NSy60x5CV1LFDDb2Oq5((z)TxEVdz4Lhvi2i#SnP zwmqF#tbe9-tz&7m(js3(nL^I;;gYj`|yQpGZ?O5QtR#II|u4eVSfhKz0dJ zM-^5z`9+`C1)^S(_#S+ddG>2N-!ur#nc+>xlrLKYx@?HZn2-LO^{te_KrLa=PN zNT_$f-~;u!X0@)E&N7epXVwNnFhed%8wGaI`CC0wNzoh zA8He;i%*sogCCb+v(h?R%~aLkgR%?$EM_ zTjj+u96j8)fNsOioFg^XzJx$lnY});kj))+%7zbAUU6Cs+%YaDjEqA=48ohGt!FV2 z^C_nM_HG_e(T0VK&Ow~W2zCQm3-~FRH>3Y8Km@Xa`Vq_AffE4JMfoaQ&mOVheJ746 zwy@T+}^KwGHFp)4y(V&9AOa{vcsY3hD8n zUS*oLRAx9D-&dP&W;TD##Y8burRyeAeKLeJ8W6?ohNump-@<6Qv>}e3zDK2UaHc+) zpHGYeP`}&f@QVu2JW&0mtC~m;>JpPLCb!vj`6yqNAl_r8$6JIF6oJ?mt>r`JVp*rx z+HUnak1jOr;G-m&Fj*K%I~xp|1R6AcZ}&}X|7?w=0_#nT$g-y=;0Fl^$FOU;P0ksL zIwf~oqJxj0wkh-Q(FB8{%0O0Z<#qk-RV*D6EIztGCBP29#NldeE}>*U=u8z3H@M#XXF^0%|E?2N|t@aSryo_Oc#j|g1-bNBXGMD{kBON;;acH*WB zn0*Nk)dO=mB6~u%=Jp3BHB4-@$CDey1Q{fqu7zVkQ(dT}%sNTLpn-XzUopoOab&Q8scTRCnrOU} zg}bywz56q^*=UZS$66SLX|kCqV+zxqGHb3c8e8ngMWEBByg!hCf~za;pT)Tztv3)X zim2E`>YKrgt%mC&4u?aJr~BeJ%ak%Z*Avjvh`p9WOE4 z1ZfyY#S5LiJ54XO{rUgH(^o)M)pgNgBi%?#cXum|ba!`m$5lF{yQM_ByQI6jrMtV| z=KJ4!XAEVCmwONAoW1s1bI!G9K0|`xJHXDx^wa1ca)OzgK=>%6GLH}ScgX}1U^Z`n z#Ltw(FNvg7sR31h0e5aIS~)`S^FJWUc^iPLW%k;6&=UT6!qh*STrYqgPKwQg za)8OPg(oUPe`mh8dYL={z-z^A@>QKgS>UA##+mtzZ1bKyy&7ia;wV>g3O@H3q64@C zDE>OLkY6UVaKQmio*`|n_>-~%n9TKByk^0PgZA?$d_ZqbB>glb3K;!n$x02!m@;57 z5{D`R_Ac_;eQ>dsHlYy9k2DU5 zacw7WK*gy-4g0b*Csm&wo2sjw2t4FlbUyoOUQLM<5*X9-b7*<@wmd|}o;^xbn?JW; zBTTG}Q6S>~Hq+<9UoX^oqymO~z#{4hz9Ez%smaL#!OX zp%4F6zy%}%CF>)sTtR*UKv+L)Z-ATKO32Xc1MC){rPcHlKzp%Hn> zGex`+n7jrWxqE645T^kC35F7ZC%AM5fG&Vw{9Zk-v0QO5do4s$p{&P`D`oYH75hiQ zF470oNsP4%(2gj?W8SkL2(}EeNF}G@%{J}Am*O-B2B^ZAJ*+g{^if}?`xOuqF%rYc zQf}HEYs2P+&&gfndUjU_z)fgYC5v&>5(4!gP@L-iMVzQAo7q_nS$77YHKMFe&{ar* z9s`I9XU*HmK&dh6*fDhl1%;i~LLAbQ1)#;ALEy{mUm$YZvZzMCcNNd%5sqN zQLF+6fIU*=fR+b|9@zc>ufJWtVD{fX{Xqz@-CSZ?D9aW!3ZN6VTy(Pu~K{ z4FJa=DFYB7U7BA32*JRF24Ndm&iJAR)a`9xG*KI1{Z~ZxzvKS#5~ux;02PpSov;d; z0NLUkr=`Gf8MwERe9=vyR}YZcqpWZ8{hCWbPFnu@_c>vLb6_QSQ|}o_!7Tl zw>QAa>2hG}&F0G#_?ujvJ5t~)!e*<1oi2m2h3oytXmC`l$5LriUibs53U1Tigg1e@N%aM6M0iPx41^j zd4KQrIJ7;siLdMGn)vkOjf?OB*in%}!B>Y2EheKJ4YTU#WgiKumO89FH^)4_47IUo zzQkCyO*CW*CJ~22Kwx5F=^PzJ!o$PsT(@INg4=aee@?kg?K1>+1b5qtRF~|6Wsu3% z%4abDdTD71q+s*&^G{nk44)?9&zOx44-fB55(X}1JW{c>^WVRJPryzV^Raf}F@oCr z+l@zi5rnM0edYdIC!ZUY(P(SMy}1OE{#a|R2Anjb#5@ht+?%Dd$G$|9V21@f)gEon z?O(N;bUvF189aylJFtmM8Zri-XWO_`r%4Ac0$svB@jIjEy{nON8yG<-U2vRWby~7b zj7>KnVJERY3=hZP%NTup-;fy8naGdSZd@5n9oxGBd-357Eq^yzyWJoe^JDWXg|0IQe8W{S7yt1T6ky!O1u|0^QR1Nnt2_XPG5!O!eE3ZsS*vzLLwU`M|tZD zS%be-i%^XEXZSKFs-qjteq0u1=!m9DBdBjtS*=kEm) zB}XHxJc^sC2YwiKz@ekCLyw$c<^ z$xvq`@tiW%L7LPVbK8udPP6kmvy3=4;`n|fI7hRB^#%+pqGYj&LkmGU^sHbwK`$vA zPc>z+999EJwQ}CjHkk^=s^cu9UY2~!P8#~^FXed9o$Cl?-0V-n9M&gV5S?6SVt*h1)l_lqtc( z*KqLgz}W3Vw*FZ>Rhk^fsw(P94H`Wr-$`nAVnVRl3*IYT%1&dOkGQ+LTix$1Ywq`= z>C^e-tncc@-H+ER&*fK-Pxr^$FMrx$>2vb(0#RjnqJT4U-mJ;Tdp>v;Z1|STVw?Zw zYoFiF0(VPbe=3wMw|D6-!5h)j(e4jRi4mG=*Nv5G+x`*CwRz}5VZK>6xsvR)V+6sM zYuh`G*n!P>%(%F3!kEMlHf2YbwhLCZUWUA-Gld?%z_|?(XTSUk$*SHO&tDGqb<=aPjUZ?Z^qxzc>ILOgV^H6<|WX{&{~>Qgd?g z!tmOdD8XiuN%~l;&eK$>Q{|lBga<2*WgXx3@-5P%m+!M{Uc?WR46k_5D~8wvMnizM zUl{-P=RM#AB&ehWd`HBg3x>xa1Kc;!pD&^@8oV@jH}tQ@)lx`&~o#$yCi5u=EOxoC3UxITkw^*@Rh zj`#C+90!Ky!c4cX(+&?EZF8V-IevqF`steB?Fa5#tAMaB8}wdQqvPWiEvGR)do7UMJyS2 zS=WNaW7^9^7w8+#hKLc0(f5mfcIgZZ*^4{1OX_|xt2D|j5? z)d=GyeW4%;5kg#oqA8ea$|o*PgPeg$SE6KD90zRfzj-%t;FHGkDbfzD-ZpqCnmd6Q zR8hoK+A4ai!x5jQUdYTdxu74Ekn64_t`t8-zfD$Wpc)}^2vg4gTWzUt01c{)*^CKKT-%|&T)twyIQ*S^ zL-|NZBvM#jn^5Q3ZvWP=$CjQci!*{ncz8B!7z1AouLl*yS$Bp;)G3{sIx+;zFLmHk#mLU|{ z06JOwM>bU^f{6>K4W~v~r7?R%L;H6JY&g4`+L$+D@NVj(qteB6yO+Qi*5zt;8|vvI zaOJ#nJ!0R@1Rf3!jt^ce+OK2vU-R{{!Fm|`Q%>W#jl>u5YlrzuJbI2c{op3 zaO-crRV1Z8ooy{`;PndVXNME3!?h!Zr*(uk>vRYs7OIrK|Mgi|H3kwu`?{y*;x?{4 z1!+!u-m+M~7m*pUGRJ|Q=Jysen*t800*+CX&2@YO0d5k`eZ+A!Yk-JBtKRu=dw%NP?C`Uh zT}RO<`>@Ee$~Vc94fYG4#%c+5g=KSX*s@-zx0L5e#b7U!zi&%XF`j%r+8ART=20BS z9u4`RyNgn3yF%^E+_OaS{!h`8(#nZ8=ccsEX;03!^XqR8d*P5j^22p=ySg@^NUOmb zBsH^3-(18rG@PBC^OP|bpFeyM^!37TsTC;N-*l<{LluH;tz}YB;jywW&$Fm(O8!7qSK64 zhNBAHQv@R?XCH{!^*3vwocl$U7xuy|uC50>4Qa5mE!Whv_eI0JLps@zf1Q`dZ$HQM zA%%892jti$Gz?SofZN zYnKo_Zzt&;1{r&HCup*XUOcu^ITnv;Asp!~wpp zqM{$b-L2CsX&ozgiNGKRn?wQoU|{T)Fc39l+-Jx{qtgORjUj2^^Fzza%Q^39v>IZ8 z-CVRm1aJJyUIxj_ZvP98f6VVbUoE-l*nv^KFob?Vc}zs(u;S)HePRO8gnk7D`iHKn6d^=sT@_*` z=M(qcU9*akAeV`vkM~0o_!(#*AQv0I4qhNaMwCT3pg-3yCr72CUL(4eAAgREXmO}) z!OF;oj_NP3=C@9HdYu?T2BOIHLi(-)NBm@ugC`Z42)P06y}tovxw)gf7<_%Mrjq}s z4D<8>MHWJmd_up`ts2ezkn;RQSw*GEy>Zc+D9P>4`O_d-G$r~_Qsykp{NS;^KfRlw zw3pOV$w6e}L`lQJ`uRjuQw`w5{HfHFev)+V7%Dz%7EghXM%i=hN_xsKpb{Y`7BFW? zUPqGv^CjJK9NtCsVRp@*mdQ>{XU9@2l^gTWV8gq)FJ^%qVFdJGn#xa-ig!k1F``R^ zXl9a_JBj-l40(?Xwjv3{QYnFx65>)pq823+Zl?RFq|{|dLb$iDxzBuQFmVkaPkl!w zDn%vqdI53pNyrk$oxQF?vDDS^kK=PqpVC6AQ&lomr;jVkck880&WX~7VAZDQwXf&X zev$~)%cG*M*2yXQO1PSe@Y5n+~_TIlXh5P2&8ouJiDC=GQ6Zj{)nB`1Bj^p?vBzP5KkL=xfs~HZC5w{oAZh{!cx{+ z{7orM3X3PJn0`4d8Pp;3NVUa0!i(j1)Bad1231M|JB6V-pj^)^)4z z?Z7n85Kt)E-EJm?8ZzA?UVHs3NWK{$orfs3d_%R8}mij4ov zm0q|6skEyE74c^(vHTQr^FwAW9s0r{KMmB_!UFX$=Qb`-Bk)y(LHsr8$ieWXy_!WJkz~Ef8CtPe;=)@D#K>*(v^1!edwM2QNf%6CPTK%EJq@1 zluI{hedt1LDrniL$eDi%BGHA5RJj>@IJ7C^yTB_k-q8H~ zET)OcHYf|?#pO;8*%@??FtEnWJzKMeqnNl`!%PYq(1nb%J;N4-wCc_$6j_0>947SLJhw=A+>T|yC}%F9OjTi>B{ z3PgQXZrbnSIjm9AL}_UXGSEZcTiqSy2_nY&f_gC}^c}O5EM*H6yM##S#^-#zpTS1d zou@ysmgXw)sLDjlHs{D}2MiiR-@#%884bjHZJhp>LQ(=W1xknpV^pDzc^&2{s#*Ckit*}eL4&ep?E|!2wEE2F>r!3~ z9?X`U+@kT`U>rUkk)mFnY56DTH8#7I`N}Q>1Aa5gh&X_c&%YmbP$=>Gtpa!hvm*Uo=Him^RQ4FT{vH9@~WS@G{_f$OAKo5?;Aw4 z3^YXEi%3%&8_XAv-C5A!-U-q}ijb}~{$)96yOoTBwJV7NBD@q*vZ9Z6_cn!Du?k)H z^sFKUCVjg>rKWjzsO;N~K$rz|*@cGZ3H34{0rYN(Y(17(ZQMJ3m+_$b<~Sn&+vu z$7eUtk18x^_UJ9HG!s-9kN0^k4srW{*bk|+y9AkDv=rsK=<#>&REN7;-9^dGS%eEJ zLV}W`(|u=?+j-~B&CT=8V!G?v$FY?a`Xc4RS;+Ha*OZFRqfBtg|3VumOG4O}T?I8A z$AtA)yspu;OH~pj$;v_g9zvesX+LapAK_d7G5O``gX5dxNRz(1yK{XEV;46%clUOJ z5|$Iou~((RIB@3Hj=&*&yRKk|CT#SF|4(2TrT_!F)cB!A&%RkePRhsqrLnEt?4iO0 zkTErI&x;$`{|PDBCMuN?CY-snuSbufS{P^n1@qr&;Uc2B&;8n1tSXoXL7iU$7n zJRDm5J?DuW)8{vHn9O(l)=C@E3ZMN}KRDbOaZVukDe~DS+=ENyOSBPi9a@?D@kK&u zG!p&&y$6y219G?lTgt~f?#m}73#aH^-19qMTD^6XZpa_{pt&OdUYqjj4T8bDX}btacH2zv2y;0^Iq#hHf%o2GW1e_)WwL+jlNg;x5^YM3M9+oH=%(_8@a zTVGrg;X_z}q-aW@Dbdh_lfW1`Gbcz^vHM}0c~03bQC{y6M{tnp0?;uz|G z_M8_dXZe=+wF|)my8!m0h%W%Z0N27onN*&B;_iTil zTOZWiykcSRH|^0ODjxuxVy)Ad9{sv178^jN<8wq`A>@gl)BBlb3v=P1-!-V0T#E7d zcNk~$&$<^&i=aR&jS^hM(8rpwpc$$Wj&7zn&=$U6lw-RVJ?r+-!&|#oPLX}+SUa#g z+X@NTh81oNT=Q~hxo*#$@_z0Optf*yuG+sGh@+ZP$E*CfXwClS&6}S;e-3iCywoz} zeUdlVbDX36)Bq57Ua(>Z!1q7`y$N{gr$)%}BOLlE^K7@0Ozc6vc)k%MQvu*Dc@-EQ z9%s6g4=sOf@55;in^PKmY2Wm$R+~{K>c?&J@LUOot6qF)hSPQ1BIFa}|BK*dvgTlV z8;3sjuRebT#op$x_2WvL-kAIw^c-l!pWYF^0rcGl6Vsnx$9YV9qo#BX4SNlpjG}XP zELpVn%PwrI&ezv!KS`fChlFt`hsHE-KdREJCt4!v_nFP=YKx0pPOWXAoO;e;=OMn_mD>&HF@5M{-uA6XKyeNoDQ@MabB%Jo}VvyM#fBIW2Cu)`5$*>7D0?qEmqccUG;~!!o$9D(NZ>8 zzQe$F{ z%II|)I7ZSBQK8{S#(UqdwPW3$Ns}DtmZK81-Gh%I&-ba$^Ja^VQxiP5o#zLDBmz}1 zz9yL84}oOyZQgjcaGDW!dq_P#_gsd(Y*AbHKNaOD>yhK z6GoVRrR-ie{_biuy?W(~Jhi)9tpi~oeM73pjCW|?8jeGFqtgk=fiWg;2`LYYdw7dY zh6!rPL*InvMP6n8fDsSg^s`*h(&j*d!?-xP-Nl$_<@3`$GZRz0AR=#PA9=>lmFLe>tsDu3mglIl_l)*NI%DUcG0)n@$J?N*{)spG2&N z7$XFxlsIWW^{drwIV`I@sxkF~9Wl`3Rx_95Q@4X15h~||TuhipYO@MJ73{hwJ@koF ziw#|>LRXu>qPwFvq7*QJSR8Mg`{YBI{R91Ln|@NF8CrL zh2?^9bn8nf*6A1(N}&K?peF=Hf8)&$0t(>5tnAv1>{ zX_lukqi1-~!b=%}{JShMRUS*BZN<34P3mdnKZByI1H5>haJGzGUl z)euiVGqWYEj5a>Uqa6YJik1lVdwD)Z+ZXaAPvp`8@nYq?!tCN)X*Qj=F&|T=1ejt3 z1XcrNaAoHgcs5{26_u8eByG;kRl0c;T_mpaig;;w?*23i-4js{lmBWwAkZe=OG0s62_mP--#KWIo?(| z1d}pfyloKs?cl0e63L}Nc4R@_1|3x#45*{HCE{sT zR|5&G=dPALj?2foaiYY!RU4gYb#B@*p(3ZSYm2!)ez)==uRUOv^9 z!#={hCCkt`vG5Gf+*S8stv3-{W zxwPWA33kVR_ybGkUtWurufo#h*EHE4O2c-~y)FzXMF->0R?QD-8#=OneqDwph3V^; z_CNYF;tvNHB}y`fmOZCEBU>U=jX3WI?}N2}LkBM?r?tCl-7~a$aW^+jirFp{#Bdt- zvCa-k3I1WjK&%j4Q8B3=`*&2WlQ&XH2^EOv{K=;2^gVsk8QQkzb7ODCL+$T9gY8%a zX}#yi>mL2`lW__9p(Mz6`>$1=%7&n3XLaW9mv$5~I@e+qtV254%SORDHto{)e}|)P zTPI1uwCErlHsn$sl_+bv&vbrL&053%SDNg1#p5ey9z3b++ltnH5;CRmXKx3u>zNOv z%bpEwt2z$)8Ti`5KlUNYMy;s0Zv&G{Gjh0iT&3^=7doW_S!TH4KR%r;P)XF5F_raZ zUEc<4?9t4%xBG-8x4tq*Z^LUXEPqbGb-YTve6(pO3#onM%->MIC2v@F)&xWT+$p7D za?TUqN_BpRibDQW=7z6xH=f|S+J`O~FjKbjgIXWLB}tr$vG-pAHs4m9`{-@96P>kTdwG-@)O9{e>O zmXwOL8N2)AsPh%69+=T!-VJl@)37bt3TnG;qQ}$g3W`60nwm@nJrOCpF{GuV9go6;x5mC&mM^n7YVriPp%pDJtr z7aJ(BZ0NtVsQzBs@7^pU7H5K2n}$1=_qEIvOPL7+@Xn?q19C9nZS$<+UunrHiIBC! zAqLn=ilWOZ##xq#rSwRVMCq?KJ;fUY7>K(+qyb7;;0IlMIa>3Nqj;{%1|g!fiF=yw zfRtUSs;%e3l{S$@EGf!t zJFZ^L0opA1(JY+F-1jYPh^LBRz-Rx7E`7OSfb1`cj8gsjZ(8P~b7LalYVvQGZ1$2Y zr5%EIp;R&lSP9rFiUtf>GO?PAF#xZIpMaV-F>J`v*6!v7stb-(XZPP-4D+m{0G?=b zhkehOl03#@?DzIoA1idqXTsW~YyRWb<%hma)*l0w$U@C@QPgPV>a-kq&R!+HPju!l z`hsPYE`S6@_HS$lfqp3<^NC}Nz)_RSpC&a_)BAIs9rg@%XQe-&l8L37zxE}f29-_2 zsSfm4lJqQow9hCqj69QtVJakFDMrJl%t+C_?%}s!=V=TwD0ae!0;gz{3w6 z`u&&gim|1&>H{oWHO~Y<_1aafXJ;6A%A`XaqD*?9Jw(iXHb4_=#VIB3?wgBTB33p zH7K5ZJD|GbNnwiih|k7+cDr*$VK3rH_S$-kjr4c}rAT=Gx7Q_!qNN)%UNi4HGgH~k z@xe1So~@;9gR5ER`1jFY{hBogxYcr}iFI`@FUdIWnbbBtScqs~bA5|ilGo4jHEOtO zE9Sx#+m+tAiw1V%eYLE>Wch#NE-wEr!U(#IE~{htIouI?id`fI>Xysz9Bbgy1u|R| zngveWqH_$6_WDDw&c^tOpCpPN8t}P{N$C-;?hTt z24R4M!a57({JP?^bsSWEOv&RQnwH%%#X{ElXKf7eJ~xCWazcNz4X;-9Yy_Qr(IA);kGCDY>VJ+`34U{Pze zlFjwrIdv}z?p3vMqkzB1v#LR!Ag<$0%6=@mO)n25qLY%}N(2!q(%k2sj{Wo~!~{^} zc(aBvQWOc)52u)=4)^QZ<>piU5vheT+yw>lr8CcV!f$*bKe(SShwz+sQK&S~q2s60XK8oLePhOnzDKcs((928ph@ z%Zu2Am{RHo)ZgDfhzD5JhcyeT&F-u(nl3LdFP__tm}lq404R`B z0Cq`CM&`RCS20A|=o~uJO?$-k3G*Aeejm%BHX@zLxqng-1zu7${%F}!7FbNyHG= zuI7(lt)Zqwxp~YKN(B@lztj_;FK_av_kc*~Vc(BA0o`9S({Il5w|40Wqr1%Doi4>~a_+|>uO^Qsm_8l~Uw5D&E#l_a7@CyKEW*Yjxr=|8!Tl7?rMQ~B<%&`iVo zi$02{5^1fSSIKg`JI98Hu8Cde>coP8rD^ay?2N&>6MO|CE8gKUXI5aoYV?-`8B8_s z?c(Jsy&GR81Gvk;$c^{)0t%{x)G$Cw1O`1_?x1CU!Q9o<8)l6bGqU46nB0R0AVt?V z|JAnTc&-Af-Ytn_S;F*o^J=6Emnd4a)4Ei5=*NJfNP5#tkPG5jsfT<7FapJBZZJCx zsn%bH6jLTCfFdjz6YJ0re!pTv3|5Q>7u2XwgfV64qL^6nQ_{~-0+@e^n@Rj^=-Ko; zdCCcYrbxLKnv3ueE62w5_T+s7iaw$l|CuHwGY}WxU24dLP{mJXTa%2d??9Tpr2wQ| z>FWB+Xt;k0jKLtHrm{|m(XaQpKt_ITOc7rQF4op%AW(jwg%~YLb2~RmVMd#7qvq1@ zm{288Zj+(NfXcw2QVlEZXALwPC=|*a+n_2fpM_A(zHNrAni&IGNj`CsVR`|yVsfR>OuV}9=)@*p_O#7v?4o=H40$Ww}8AXVi z6Luh7Q&sN4T)qynEuDT}QPtbk@rA|aEelVRh1Rq6bI2e)yZ^Q}VsUjw{Reo(y$E{3?UQHyP(h!v@<nQus{+YW*wB3~l3?jueY@N5##n74ys+-otrc z$Za80OrhZw$Nbmhhb`*gCdbi8uWJ@sSz)F-W~S`iB`qsJZS|1EO(McCJ9q8(TUYNu z&>x(R?S@+Me(-{~R+n*LbNJL=0gR@-GFJZkp7izc5v+fuh-@c!l9qK=4)E1$@JWHC zg${T0>NC4gtz3cngwG_n2+=ke0LN$mSwLt?cIjGGn`4p?)AMJMss%9PfHzPL+PUC>FK>{)AqKuzZ7UfbO&f_H*U8xdRnC4@&l0TzBMhFE!YH}3uAmmwA8Mfe?dpk0H1M7PxO^Qq7@4hpjox4*!`3(P;Lc2z8xfh2JcxuS6qi1&bfTMA09U;yMuVg_@91oT#c5bXEec&2F$y7>BdamnN?Pn*U5iMC0{Oy)h#Dq4Mvm8~-IXJ}wa`~HQ!Jtt$SfsG zWDQO6431nm0>H;I9FEcX>E0$Y27C}G!q3oOeekxdv>!~MQj)C}LlAsaH}?-`i=ab$ zHT18=-4Z2YLD(daOdkLk0B+@>%j4VLr^ju#WQvkB!b)U2C@km)`YFvt1fPwclg1rYTd$s`BSI z^k@SBM*p@62%aKX@v5D7OAhlRnBQEr=zzi~&OFUrDr;eeFD;}EfVOafOC`10*VM`2 z4ph5tiK3eMJW2Z@ZJY^%uVlm@F-B%ODRwP?d~sp+`JVMh$bb@PAb<1jHG-SsSmOaC zaGeTs@#`>|VY?@~I-yZ5>V+|V7!)c67gvrkVkY_1<`9M0>_1f5F~}H0$Jxp=e(YfT ziJb{%VPHbet?iLvNy;?2OO`ga2TzAydEyJu@V?<9hzQIPq0_lfMZ=`a6T~$CuQ*D+ zp@D%4=iK@f&TB)g=%Vo8w&XNp*Zz^*w#$M1%wJ1G2Zo2w?o}h4XQ&Ta=D-+s@aA%q zP5J%bYX-gFC9yZ__a3KRxib^HiFq?>Al4~bp(BK*cx`KaA1dPYSP@$kt95v#t?@lN zmS|%cT8TpQ^yvtAadx_pM+M7*7&LVJWl;FblM(I z+k^X;MLfQX_MVxC#hEVDt8r}K?8-xJ%Ujuyz@WD6c-M&7AJffpEibj@bp3nq`7n1x zOAFsH&$;BX=lM`Q>J?*!{pPn9eT=vL(J{y4#l*H{>%jq2Af(s~7qW53#f@*dzNMpl zd$lRG4q+-fYomsDPQ1E-?6C#oAy$;{S-pc~BppQt@BHaMQ^CP~av~=~LEk9 zX{GTaNLzn;{jps{Om+2|Ukodib-kJA(s?RXCVuo%nU2DdqK60v;6UVzTm>Zto}iAK zDNs{nXSeIN+@CvUj9#)}1$x`sAk0@KPj>z>-UrhPFi;0ILmzR=F1tLpXC&r;UPGu@j~&z(MbSJeV9^f}B!Z2Lkuc81}jdc2xG$gUQm%L+8pA8|y3VX!X@( zLTv6iyEfhZ-h!B;#)#1+#hZP78MPNi z3eU){mRdZ_PK>+rVgohjBm5bYfpjIBHOHdqJuh`V=XPobK*9T>NhJWBN>tki>}$-`!!SsI-%L?~n9TT11Tr5?PIkeCPXW zNf%Kz0eb`1D0+^k!?zhNg!F*TA{1FmNl7_Edaa7v5%B^fYgq*iGwwvimkZ|A3|Qi$ zm>2u{kzzbU4{h4u^Z|nfF#%jsIJh*;2&R;;MaZ%*-ZkbclfvY}WTsmWZW-e#K?6N4 z)*d1g8+ymCt!}MGxrWt7h|$t#c&XBR^I~bKXogq2n(6)%0oOq4{+Rhig5Zk;E!dH& z+?Ie=pG@WR&T)=r{&=Jj)f9F}LB&f#xdNuaBV~0l^T$w+ZrQDuEr=W2U%aa}#bS2g zXY*>Q^gxoz**%5_C(LB?S*S&J+z-PV6Od(3Y6gXWs4Tk4c28Z_oLO}=XL8>ICJb~L zbel7M%O;$dLH%o-X5onWTDdsQ@buvo(1}f~%zfq3vE@?bu}1DNzoU_;*d;GU}zezml#G<$-vQ-1{?q>3fa zCt~5+4<-$%9qwsk?V&k^#Dg{q>GH%<2@ZU}+;(c`d}#J&_D0o%1z&P<@8RLc zmC0*yMdg#78yODgT`6~*ToE6a=voDpeoG7v>=ZQ*vf^I8et_YIYhIAg9}e{jjQB$Y zTkwrX+RoLxoz(953hbq2te_aw+|$T=Al>B>Hsiz?Mc`*bS63V^)(S5TUZq}V6M(K= za-fLkXAFI22F_F1clY=CGy94g3_3vOqfx`}>Xi2PyMN|o)h*K5w|?Z`U;nWr`$)Z2B3?|#hkm>Pt3Qq zoO}20Ei%Ko*RHjVIYh=9t9LdBdJnR-Db$;7mdD~qwWW@18`!f9e?5mVZ{tdHtiFw> z)#!c^JOfT^KeQ^APyh$1AE`>Z=yu6gIxu) z98rVUArORGV!#3w0$U@1@SGo>ezgjhK_#~}WiN5#&9`KC`MP!ASk{`+lV|s_V1&L4 zT#}~J_&hlw#gmj2+7dq-c7Dlf;OWg zvx;y!9vY!4t&^iaJ1ervZ2!o>3a`kjcYlL!6Cw#;vNk9y{Seb9RIHt+n7=y{v4w8a zrB}aEzjAb` z%b1b`-Qxu0R#C3l5SI@CKX7~VwO|T+szC@SCxCL{OD$rkFph?>5*Aj(Kj-?LR92IU zEF_5>#*{(0t9OaeJ$77&7mhA9iwk#kFtiOFUv!|r?chlw+?mNj7{q|it$a7%Y>lR* z+b$&9m;lh6EGkE5S7d?6!HvrnAo{nzMG^EWM#cpjmRzaQ={-Le3V7~-nqbI;_VwR) zn3{j{^U^~)jyf$XPPS5P>_^+~9X8YWSHBO9m@_`Leyac3R#WW8?}g}r_*j;k2DJ7u z#bMZh!K`;tXV_b(UNfCU#_ z@B1sLrntIys2FSqe2HW25`z{i;lDh*ps4Hv0=tJd%lunsCJ?{>t3R>9Iyn(3{K?b0#+isy&tarnpjySFk@+P!iT zPm(EG`_~m0UQO8d%pYiP(D60Slk#D(2Ewl=7BKa1&o*Bi%pS;ZmX5u9=1ek%&{Z_s z>YR&`Q|+l$x%WhaLoM}Fd#xzz`B>!>#$16YHxk02`KpcBhR!aG(5lQ zT+>%y+4jVd07|)q^8AV!zU4&ABy^RJDJvSTu*gkD?>hTXH7~D96A4gZVc)(^3B4jK zJ{Vq^;W(oqr?WZd;d2!>p3;B;(LRUUPz2g_EuO0bN4-p4&oz7NRgVfq(^VqIkZwSn z4Hb{0RT?T}hlpeHZh@#tS#NFt-Z?4m=$AI~;!P;kecHdj3wo>fE6;q~2W4-_A-ixs zX%enLw)YkY~}hQE_EL9-pWy<~d~aMM_UVg?ay4;df(u|<2tvZDOB zDWk&fI&-mQ&k!h3?$|O6BAMCTLNt`|N0JvO+;6*|1n03^aJ@m#coVTxrUEUb*RJcLkdL7t8pU zxx)GI;D8S90{HA#0P>(W2fvHt&5*zVpQQCc79HS$0Q~;1vPKi?`5`^?d6#XG#;(Y&G@FF5`{oAJlAwdA&k-RL#uLv9 zpcWuE{@5cA7O+P9oNs)|uf|GWMY4#6T}yGhNNQ93qmtdSrBwXgfsM zGP`=${2`Lk9*OF-7(jn!C`^hgq0gA~@m)i;gOsrxn!nql6tP2#Dv=5eqfJ`bDU`z4 zTc83c!U)~+TH5RYXsEd9>p)c|iQX0e|CC)@h<@rqh`0Jl0_E*ZPrtO&hcQ4)5)hbL zg6#L7pfZ|%AFy@yPq!v`&2s!uGL4DN?7#-$8fgTmwF0Nl1mV8KCRKHVU+L1oj$zei z!?&EuIxPfT1nw#IG|R6zwZq|TUJqYGJsB{<8EroI8j}3zul=i58BW` z3+a$`wtDdd5RNs3s}jc#jxMZ&&U+MzoZX9RWC_E>8G?56F(S0Veb^e!=U-ac!LiPG zFoadmuQQ>9E;#Lmy_9<0r5K^lyNx7evZ@GvDkU1R;DI6ZO`)v6wWcl}#F}NSQjV zg$*=zzF_rvs1c1*1JZ?v;1%HCK%Td%>a(oNUM>{SqFd|D=Wliz02E@X1*21Q()pU1 zbmAbGON%B4F|yF2E5& z$J#{87LJ_2vjty(;9qVp!HzRROi9V6LmT~SHR}u)z)PD_8e(|gL%zs`I2`w?0a*a; zU9*&dSk41N)VPNwI#2p`N2@{Yk7FnXZO^z=maYJ;4=2`p2mom?t-U@Qob2K#I<%a* zM{f?B(Vt;Zat~d4KTP&d9S*d6*12rZu$>Zwe<`bSAb?u_w(LV?$OelmoFBEZw4tLq zSsuDP&1*5D58i!rcTye|q}M=RwBf)@9$i#yynZAI1#}MQ%4^D+$^;tk!>6uw(maxZ zEP_+_X~a&=BzFxN9v_cC10-603kAjWA@Zq#J>EjCCJum^gF5lvv(}E#>CqP4-XDX3 z!Zr^ZSYAjnNLcQCbx$A$Xc+d^q{*t2XbmBPdTFdQAnHC4Ljo}Bb;f7HW)3BRCREX2#6 zpAZPd!bH|%jsn`AkzYFQTmKJF=K+p|AGLoe*<@!&c2@RYiR_V`6|z@CHklzLQOMpa zo9vLiv&r6j@AW@^@B99{uBWS19(p+6^S#gKzRwj8puXmERbA@~{Q(VW0E67LSE@We zpctofM03YVXfs||;nZ=FVLKVS{82sk3=rzu+tB*b#(VPL6n>(!ohK)>DtbBJpwMsf z?CM#{V#SZkg*|)Nko(x*5YQ8`m?iJpLFWH!mp?T&inq4Xvs3AO+5-gE zp=u?)s_31PionE`j4WWc>u zQI=Af!2lByT}9;UUT63@QcL!rgfOnrtMxPqbbXR35iICqkfLNzoGm!Uecn|Gd;b&$`%G z0l-!^8cj7VCCQ2}1|eA#UyOhWY}&xE85FIZ859iPDk|8=6)xdc7xZiUPKK4*Spg4O zb+H6ve=NUE=q1{FUtwJQoft%thTcm+c>xUuu$CqA;XPl=g7Dnz?)YeGaZcQ<;_-Z! ze>#a5Ipz{4yGc~h9iV8Je?;W;45Ndyi9KFnP=D`@#0OiW91Kt$^gh9SA6m>ZcVRA* zSZH4>3(5V>angMVBdPnB2G<=e-!<4KI9B)+)?q$Z$sS&ZrWyuvz%JLp4C|?=0rP_6 zSozR{0UEeP%$a|Iil-Pv7}!vgXB=)WsiKw|%vIND+w+S;f#bnnYB_MT{=h=6@`#+K z@Rnl=2CX~p1?SKVnY*?I7xwk_Bf#H;r$j)!FpY|Z!vU@@z-0+1+&9lbgi`}flF79{ za7-z^dAN?P%B1;r%!bAI%L_9c33fa36j`pwR%@O)XMUhf%3RAfF6=UHEpyp}7U&SG z;O6IryGGV1mX*kqtec|(He%}&+;9s0-PT|0lqE|w2_4HSE z>T~gNc--=ML3IB2T;9%fXKUml@0m?0ey9DfIG%!03oe}xggMcfS{}?t6OgE4L5Fb{a4s~p{%@(^SvwvYemsI zORS%w&9%O=iGrB@m}Y|8#~XFK4k7n%e=05}4;T8B^H8?yYkP}ltn}hVT%5?WNGk-B zt61fw)opL0JHavCTx0gJ%NqbFCp#Dm-8Ej4_u;sreA7v5!k@Gj6 zrEOej7b$R}f=V+9zN8j9f|FE0yY9A|NfbuzTRK+HVps7X(2hAGT24z#`vGA%mwOT3 zi;>b4DFf#A)2;;;M*qJ#W;wm`+O9j%zg0AlDLr=CwyXefgC8P(W1~;)M?ec)>z^(a z&a-j~e!<+bYsg8I`@nb49;2`=BM4zsC^ZcRU>rE%(69 zq0niHp?n`Lblp>uoX^fROxENKk)F2z=r>`I`aXho3^8_FtZ4tF(IHYh%&CNJ|FI&+ z>fR|G&p+w%M^z_qnPUCNN^eC^eR#j-l_a)PP9wM@z(rB5Qls@#wP3gylm=M>VBpHh zI1{@KpT4pm+jP-Y%rrv1ecQvE(lhEc>%_)s3V>B5exCyCJ52#G2_>oWD`h89Pj2CZ zUzwN-PC4RwU%(h*evs7NKe+Wcnx(%^aUvLE4cfNSwSLZ8ov1RauWg!JY-jK9kfc#eo(vk@0{iKF(fDruV-g$*&L7BE zdRpD4Ut-Cwd$FK;eNkgV z^5D6La6903->Kt8z!vrtH2%38uboPGbC`yyRB1Wk<@>W#YZgsbNz{!^Hq>yqHR{=U{r(2w-0s)tEXq1qBjlp zTD34m2OH?`*A`|g*LLfkjJ#ijIM3uj6fwfPnRBaR!l#HoMXfrgd4&{#%=Nm_7>~%X zwMpdJ%3WWAacS*D&+i4#=+~fEUYCu0rRf=57ucI-rlVE__@`m%`0jlITSzXrnPc|m z0*Wec*af0f#x1fE%&n}9LEXjH*IPNr64d$?9H1uoX6Y3g)YfSO$9@9qpKnKN^B$0K z4GQK*tB3dwyy3t^tr*voL*8xM(L9|f)OB<8sZ6x!pE4VHes3+i^GGz$VNqLgLO=o3 z53}vByUyRotcgegC5wz?@)h+cY!c5BK*iv?=>I$mR2tbHCQ*d0fmxn#JmM&CqPbzQ z?=aRgx$>g}1J<14wO$R_!E^9lk&)hIbwYp;oy;u|uT&cki+QZSspQ)mXVKiV8syoK zcT8kQ3iQ_^1(bQyF|OAX!E=}_&|HT(x$gEkRZPxEnH5zjdTrYVm+y3Kx=WO9S<2W? zd8?mwvi&2opY_UbxIX_XOY3#joRzKF(!f!k^zaMKRTssx`}e&=BhbVz;I+l4?HRMH zbm6z`81?~9+AK2v=idz{zH@%tx>s7U?mvhg7clBzX`c$xWVCPPDLq&1LO68iS;>nR z>hepQF{8Qhkh?nDnk?STy2RAL3sNJW(r{tp%C_f36a2OAwo0} zwBrrys->9ARcX=rpEcX(zXDtSuUQst0sy@Sab5pKUX`uJcFg4W*=ifm)q;!+wu8ff%kT0!k_bV~fe-pA_Xj_} zp$fFR399!~gH zvg@H@ka49BbM@$v8Z#;#o~O7dPhMD#7FtbD$l&YIux=pk!W5qhJ&b$Y34E2M>TN6y zOc?|7VF;O}^;0=dhA$yb+65%l$UT$~tVyiZ2nsHcYY+nUR}cSD2S>_^|V#M zh5$J0`Q?e@1xnk?ceTR{9T=viNRV>#OD~b}(x--t7bz*{<_BW|=2%nn1pNDJYbb~zsc=#i>2!Z(D(5#Ac~-7r#uQ`H zs4O~=9>WA9M59ujBegqWfiZMO*M@cn0hOSf^}0Q zQ?yce*`M&1fM;VVTZ5^rveFow@UL=O*Y3MFSN0ma`!U55!6U({m@LL zVoB3JoHdrbAy#sPniz2zo26SkYa@Rp;*H**1DOwo3kL2lw|@a)hvue9Y6{ty+o~_KV}7GH{t_Jw z5>F8j8%cvnwI`~uAD^v^Gr1tzTJe_XEDQuE^>69?_6<)kFSW2DfBxd%uldmLJsQrd zf~9YmznM|cyo#S4!le9T^fz&f&7ZBsvSP1l7FDZ*DZI@!SuZFwNE3pQeVD9Q(oc^pdrR9A*>9G=>F zqPOJFPCr?flp;9huU}ij))BdgJR#tj@N%^w-v75(!=zXl-0x9JA!QIVdoq(>1T=6UuQl3h zb)-bVd-Vj{+aKL8% zYf|;oSF>j|w1n2yueUDHUhNnU_r1!t?@?kRa|wAyM3wKVxx5ra;^IknEaJRv=hJ5Q z6EDup6hv>U(++jCCo`Y<_i8iQQ>p&*Y7DMyIVxn)Led~9dpt}2Cn2BaqwyD>Cg}+5 zr+jDGC*TDGF-j1&F<0$*)~7pWDj1(8`lr zM0?|KMh!|Z^u)PL4|30Wo72&$GQ+Oe`}2aS;nvI(8nA8A{FW=l_4*a_a!mWZF3pHi zbNvR}>_qXn-bfK+hCU)WsEXhe0ZUmu*8kwKQ9$eR6xL+u-GQm*-uv=}jotODPqZA4 zPGwI82;NMKp@deQJ&edInW$0>@`#WpT`@~zU;<5A+?&W|&?wT8PDwgmnwKv^vGumt zjcmZXpUN{pf&RNFF-n9?4rg`!QA~G9j9{J8YGG98DLvKGP4vG{} z{A8`%M^6e=o`89LxE=p01>x4Up+tlic~5vM0d6?0S@X=q6z2MzVS#k%tLWwFRN13o zkjqeJUa&Vm5vx))z81?XGIqX^rlV$#8&#rC$Y{>%vB}UF#XOr3&d7)cMFYkHPh$;x zw6q$&`S(kbakh03$O0~X-#wm@t89n}aPem>ZGB7o*2R1GDaUWQNGK-IxNb z45iaj3XJw^f`UaaZ?vuZoqY7`nLR0O<>;;6^*O9I&D?#E=OLC_9b)s$;J%^FwYL2C zhr6~&=B*bsqh5XsBQ;2+GMtEo-9q7Q>utsdu(}OKd(;p9)t{>omhXh5@*IA=D`CE3 z%{>kZG8_8mxmWH}M8r+r;Kh6GDhFD~7bJiD`0F)cCcYOv1YN5Q9xIfHH*J66uuSak4J6i>{#y!4i zP;JU#%Ub^`h~`bQwLh%r~R333_IP#Dr*g)7Z6{FThKow( zZ^I&siQ(?aX|`GR=t%Q7o?oF>B7FC>hb!XQ-;OnPX{}F&($n9>7dLgWTtF-`cet(x zZG)@TlafIxcb_fyhNe3Q-i^bQWd8Xhhx>S=4SvP*UA6dEcP9#$>D)1{5rSGm$4!<% z!N4o@$hPB*&;PEV5iW9Wev5Cf9|?dTnh_prZBNPmKk4?CXUyUq z#U6>}sHF~=!z>+)|M$#@x9+sk8H2#5gE1JO8eq=R?As_JY;Y-0aqdwp$rWqybWu?X z?_RYBOUi=T_Ym5>YCW|L+`138K<0$k7ZzT2b6Dn2?zarqa+{j5>|V;6tN%w zlCBzAGqu*%8_IF{wL0=CX=^}f4W{QTt8`sHv|%P|9^D>(*88nfojrElEp$>o-b5W6#RiG75WBZr%#d#7etwpcAO{9(%HUIIszCTl3dU>hu)d|tpB z*+BpUH*>Bs*N?4(F1oD3eODN_hhAmro1eGl5po;wL4!R8)Mt+(w@)q2cexe=2}2rc zh(inre7P51j~Hja=-<94y9j}C$F5=5o~W+yeV^fpIIQMd-e<5$9?eIohJI5X%cXF% zYpLN^G-aZDeZ7Ke=jGm29iMGoe3i|x3>lvXj-u7eZs(PC&N0tjcP)A6#MpCM*1U+R z`*HIha(^sbNAB2*GF}*s^_h&?=RN$ z_Wynr!Q%WUxX%K5$VK1HPR#yr6}?J8cWlsANKOu8=hD6?ci|0F457D+=3C0@;nz9VBu<)8l`;S35RoI{rQHEq=1>+`J?&MLHHN92qWJL^3sPZtHD<`w8Qjjc{Z@)B zPH==MuUSAss@PnpaN65qnO+PZ&f3#G#jUHIso{rn7QMmUtR@B@5gF6Ke{!Ow;E9rv z-u=3y1)nn_>Kt_n(wnDI&$p1OQW+qw`$3r0+2x98UxfFias6Qi+raqQz}nD%4eG7J z8}`F>Irj1zJZ)JUr2i`-FodKk=M1kryh~3Vcc-+pE>^R4;4=_ST6S}vZNEQ>Vam=0 zrc-7G4Gf$j%N7y)E=+RL@s%xju7Q#QL0xmns(FKyhc6*j8ovt)9qWME;1dnh!#CeH_i z)`V?0ok{o90>O$UDcfatUHLHK6XJvY_jBKSbM$|zsSJciv&9sBFZ;2<*=pZ^?}1*u zvueg73Gzu|hnmaUz7XGcyDN!%+Z4i32`a-op(;?P!=(IDv=6EwmEhTx z@F2xveT?!+ZZ58ZN%qM4VbCR{?`~oV+9Zw;aa`uj<9jHW+9_&3|E^g{ zt9)I<$GJmN(yhiEOv7z{@M}4p28tj<$OP^Yb#%uD4l4Ygf~E`ajYmt!0Pk74152xu z&kN>`rWBFeJPE!bB_7S+uv6e|4g5Rc6Ushy#O<-y^VrFKMJp&oy><5%27!S?EeCgf z?fQiZ*6(&nf5=k^VG=_D-~@e!Ass{^BQ zEpsjtn6EJ>iuQAi=1I|M^A#HGgg)?H%4_>BI+lk@YxMN;x~d7|f0|ZZ)}SfdERDRY zh0Y{INOD1PcuA)yVA0x7<>GQFR9~`(CnU1>@|gG>Rj>EV?W(QSyGi+i2dYwg=f4FQ zXN?>0>dKdTor?+|$%!=VRzep+K_d7KgadG$Eu<7v7M=7*6n(c|+;~0xY38@-hqSY1 z_Zo-wK)z8!**u^BA_cg(cfB`J~ z8cLdv^0=?dtidHE6oDlziU6pReDsVd(ECCH$B*?s?@FAWfc_i6i_2TwHj91u&W+od z<<6*)L?Un(yh*N@<(-@_i57}n4pC5JCmt4ea%6^pfn=7SSxHo^w%XU)^9Ky?hB=NQ zeg_+m4omN}CT8!hi|M-Nrm&+3#CfBFaZu*(jfvY0>upX0q!922d1l1NCMzlZ(~(x0 zT}(2k+A}+t<#qztEajWDG(ib!^w_UX3TZ)?HLu`Zoq)#;<6NeqPBnj2%=nL#&%Hfg zNT|f!KffKd9ZmDJihLbEdQrnl?FIIYV(d4HaAU3U@(7S%Wn48d2e`~3)NuAZN&Rhg zk9B!g*s(3%95>T@J)I@12HUW|PB+m`0+*)wJY)8+o;;cP3gmGpPjS)an^+rD*ZBxm zO_wE{U^cJ!I^ES8Z(6K4GHPdkza)FtiZY8mle5Z*gs@J~7$#aczr|I$LKJ6pF-l>@ zUM2B*hRtFN<<5z|iWvyhZJsk7sPq&{1_qR1#oeBXZBfDwle&evJghJ4%IKX#rHSr* zC3#!tG59-jv;Tzqn&j#+K-rTe5j~sp00bLy-3WPWU%f&5ax7TaJl&_szFx8^H_Sd|YQ{f3g0On9WiJ8B6rip0#?F+W)~n@W*&u zOwL@N+;99pVZb}~R>6e{s9^fwns;vf|9LQAVO_`aK9q;R(Y^d|T2A=YR6Dv3!#?yg zguNT1bt`$`rO?5Q?~&y(py^*~=3&8TCzJ&b7Z9Jg!S}F5PLroKAPT{f`(@7XLlg_C zY2Er4c1pZe5~yvCUQYd=7U4=Al4^N2<6a$1_ZQ_}iWskxOln?Qf(uh9M2amuB~vmh z)+wvB;k@VuU;kctE75coI79+mLrRViLkt z9!Iet(7~R8AxqpA2X#)PK91^44p)C4+nIm=mLpv(C#+^5`&ANFCrRQS9-Ria4uUsJ zaH#aOBwXk_QQ1t~DlmI@Lv1aRr6&>Vax zOW=#G5e9n?s9!Na#3WqRt={qFu|1B$xi8_xQALea69)Lmd(De7lynQ12%5i;r2xVS zfFYo&PAS`P#6HBf%0cVi=q}_SccR8$ZMr9kM+~kIHPD7F`8V$b-j`q*l@f~1(8?|I zFJ;pNRn{?5)|){L{3fFKzpXi*3J}M?3u}2vj5w8$UcwLg3h-Hfmp7fLqX3l`JZ2Hy zRQK{p5}IsF=XTXm}2Or}2~%`5!UPA}?n8=&qaB z9J_BDSnyhm+G|Y!&p?rM$>vOYDdK%SD;BQfK|9rpdU{C7u zK1dG5@xu9|RctWcDwHfVDomkVqdKDmI&A8GeO}PKH`vnPUO)U2Wif+&hi=CtXqwaI zsRJ=G#tmXZwKHC+Oa7Lsjv{;_lAxZ>^T8}4I~ z3X37^LKT;J2wuBZj!!!E?{sJ5YNxyBPO)Gv6NZq?v>RU-bM`kvk59PAM(H2Z+~Ie= z^GSuL&7Hh&H0oGN7gn}T-FvdpgicpT=E^w?&H_k-CjEyk-!x2*5yql93*?eIpBW$Q=9T(j=UT%=^@&gv4np!0Hnz&8#RkLU}jUfb~*Kp8^i z?9!ncr!A};=&vgx2vykDS9HAlv}`cn(P7FJgG4U)w)?MD)%=w(%C{L?i!PIrqV`Hs zgXvZAY1w}v%w4zc=Eo62Unp-*+cts1VAd{KTzH5EO~&mHzZ-7Ycafs}5fZ_#l5*|- zhjh61L!r9OvHu8|uqcJ0|9Og9D~FBs4}}ypDGLU;0-I>3ypJJj zpqTI!P;-M=mAam1DQeBJRQUuTJt<9PB-nY(CUVK*;RCBJf%0Qo953{)PRA4dvE2# zZ~z%vRHIX_KW|$kbS?s;_8{0Qw|<-&qIv?}A8aERqQ+F>RMP&VmP>8%po@|5W+BZc6hzhVv zpZj{%C2>NOj2cZx?e!zNsNjwlzJ|$op)K-4w-FU^0nuU~8ge+zhx)YBbm%k z(m7orHOJqW(d|+MveL-6;><1qMmW3M7{f20`R>038gVfi;^w3BWOx4 zAXIq+E3WR&`W@c=RWOy`E7Ifh9{tu$T)jhMm&MRB?zv00ulikZpXmMKEZlP{S z&%`oqQ}}Qprd|Y8U%8s&6o!^DW_$eG%H2};t1yRcA_}Ak9>7Q;Sg%7aL&+=1CrU5m^vX`lgR)&7m9lk*-^O@VwRSlL+qk z##{>tyW)f}aTarI=%g^weB6l?h&7TmFPL~)lSK%zWbdADqf)dsYhhK59i4GYSE|67cdGoMpU59i_Oy~+=GUH0S-@V&yKy!SangFGE?1m=GHrxA* zDmbjNQnm1R;G{NVI?}B?fMf7}-qx-nB?F5chg5Uyib1gWZ8Y~z=cePwgm|`mhttux z`2Y{u-CZZA@aqa53iwlK(j%4fHa)Tvu)M8i{Lw0}b2}Te`1ek@MCX;oOzv4n;{ z7`ve@;4$sosGHgc2ZX&ZqUN0i&Uq-${&ZBY>^=A7v`G}pLvr%A!My?B1tq*d1mJK@ zV1;JggG9=F}IZ*5iDgE%DZXio_?>|ccMPE$qum7n}8Jtj48V2HceFNPO-gw9BP}aM%y~76Ls^w-bS;iCw)4Q+rF$`J{=4PnP>skdj?~+w3 zqepEVxuFI~`~!x^0~ViMyWIb83B#?L<{c~c=y$Gq-u2t(vuRW9|EU!sgkVEgxXl6e zI!vQ{%FezU@&1%PR@9rquKq0Wx=J+}pXDwI0jz}K0|UzzPm82+Eg<_fB!dG~ zOzad^Zj+L;)I_DL56Be#C$86LE4PocLUh*O8%G>MLfxjXtXrN+QwEQg3rEMLliEkc zlG*5EL(6H;{M=57D8nHDnr(>QQRC?|G{AO}0e78_ME%{Nb=)cm?E@W+M_gTx^a_SM zmgL+f{LHDtMi*ea07SK{VMF-g!6DM!v3$y_k54qA2*jPZnn{zw0-e~7lw4mx7Szg1EoB!V zexvW5-NIQfG~TLbHB#r}l+jkcw*&*AUnps<;jy80V{5Q>y)u;|lJ5vRz<@-={$Kx`)pVb*3JGb$Th z=I%1IQ>Ju>tT-t=01R=!y1coo%Af)2%P%r#x-?YKeRU=}h`LaITl1)VqnW6+g*QSt zhK_*0WmvMehT^>L2TKK`f!cchNE875AmR5zOo)VfG(RIbg!8QbQ6+}%T*eF?1RgN{ z3E5^PHl-;u!PX`7gKE^avQ?BS3F4Zm32G+5LWof;jQm}gK!$fi3mqT*V=b=8966F2 z?#R2d09l|vwHnWPw;R0!CT(lNU@=~>JzuxRE1MPy_svGg0dKFEw3Tze_{VTN!k`pg zN%9r}Yd%@DEO^%?>hz^i22|VT?8`2Djss)f#KPOkbux4cdY_6hhNM#;6Vn-_q^xvy*qc&w! z@*AE?bcS)=V#%N#T7{}y9R0!MW3~b1M64KRWovQkmOpb9@j!ib=n;9Fe|Ak$=c{lj zy{mg#$-KAK(iER!Vo1Z(D?{9MBC#*ysW}aNv;LF|R)R;xCntac^?(c+^c> z?OBayUl_t(m^&__PS|$zb}Rlzc7q#29BvfuSXZARk-}cv6Tqm)s$So)OR-Cz&uY=h z6_zF&n~9+>H3-~rIfusgAt{*?*HnI~G~^b1t2GH22pB-3!5z}R;tS@IpOQEnQ2E6} zN&syB$Fo(H`SCuSIYjXt9hoUnhxNU?s*4VgcAMH)8mxb8x!H$e@ae<#ocG(n8xe?` z@u9m^H*s0j=`j?Ud`He@>}@3RW~I5zomC)re|ZqoT9I#84LK@LUBpG*^uo`&+rZ$} zLNvOs+0M^*tHcu|$v=pMyxu`n+S;FOu1F=+X_t}@2w41sIiyoGX-@lHipit&TWWGTmA$UcHD-msVdlS6y|Pvm)!OCE)DC5hFRYu zVvX?)bgU5V#tSx{u?;QcTM)!2Dq$?sDFahx_nTZUnSF_0xGf%>0pBP-qa4N)>sqJ0 zcln8Mdge|`Xk+sQUy zlIKyJIYqVAf6Gk3-vt^Pg8i!+K9fdTA<}CC2tbW(Cer)RvpIBfOVNNIqfzM?}>}|0g5p5|Xl*Hi10?Jc^I(#wExb_gmNVI{i^m zUJ{NpJe84Gd_(g%?IO)|r@`UbkJ+qAJnvm z5104)WCL_`N;xA$H}}i$-E`pfcCLwLHQ@B{)U$+eO>M3QZ^d}XR8SX7E)4@Vw#Tw% z!5LS{6y;P>vwlJU;Q-bSD{Le798HCYzh>S!7lc!8OsNhm5qRjZnsT(4LL zFjU9MfJTmp2pfR?iO3SwVhCKuHcatm)A=6}fe{90GK3L>rEgICcC1GG7d(qaeX_9O zNCu{;nQaU$%HAJg)nX7?t*Dho;j%xT;wT9fi(}F z*`bFhis|J>%W@6@oZ*+mz;j>fP4Jr=Q{gG3L!p=Eh-zMf&@6|* zrT*#jiEi6w?IQS1+;PGQ;@dzoDdx~>N57d4gxN}nC&ow0?5Av zPB8e-Aegk3vO<*C%&kcJ;UH7^`1&b)GLSjP{GHZ0XP9Kw`fC;uGUKnVuI~AXySclsZEmK6W98>h)=Mu@=b8lA zgard}H6{RHAh-tp!PO0?_poKOJYDnZD#N&SMF#ebwO5VzpG|y(qyQ9zWKV1+iTUx} zLi5TaK-G}%PO}_tEwpH!KDt5(V!aeD(OHSyqbzwhqwOdVcqkCuiKbTG$dmFin-Phn z?D{gPl;h%|_{lkhUt*d|o$L3}<#!5yfPe+~D~CK^hSh5yRDES*uG443%LWACF$k=X zdt+d~m@2BODJw*IM(*B+b|I?nh91qh~iXT_e;G8l*Ufa1FT=xmz?u;k|9W6fWW|eFD#=eUa$PW|(T1QXv)c zDZBP6U;Y^`*}Dq#P=cQ|$(`HMlf=6JbfsxC)$~1U!E0{R);Fn)R13+TSM`gtAR;Xn z*rn7|R_RxtrIOT8E+yN22>A1L`K+M>i^uX!^jz3TM;Jowk-VS0y!~oy&(T8M`|=^g zx&s#LvIJ+5l6&4~HkjYq{#?@wvd78`xfv%ME8-E7GLNU0Z$aGBl0T>O(znr1kGl5T zs68%~sJr&*r;z+^p51CQ5Rqxyd*B9P0mVEw%&>dq!(+lLoSNqs3J#lYytmocs?nwY z(V7~i&p21jC>AG{f*dyglJ1Sg4PfvdQFg>#mVz7vQ)d~6BbgZI?dHHKGtu8iaR=w0-gI&CxBYP(Wj^wuOVX`|Y=Qw8l_?RY2Id2YM{A@9Fi#jPG13aNk( z|7Z58^u>11P$;n2m>vI+DryLzpa?6zuabJ!LsXiMCTS{J{vpe|( z&F3%O+l7+O@dU6&B=TYCGVf!2*d{3KU)enO=)r1A+Ovbx-lhFN3s1p%i4Y+1A(XBD z`|YEkawcGKZmWu>xIb%=**K{PSI*D&6DTSHcz zq6too;4&|fXiIV!50vY-SzdW>>X|vJ50B`r2~!D+Nf11Epz;7mSrP|_*@qlGLIsI{+CMulksOjG(-FHqBc^W8r`8(!}pz+(Ee!ueY{U?)?Ql&|=zVhXe z)Wtvd5XJPpeMrm<5}21<6CI_{TO_4lcKNnS;(uk02>+Cv|49m;h*ZI{E*N*-h>?{w zaNeqph(D3ON{4O40wrdlecAEl;JxlNLV~R&sLG;prvh%QnkuX z=sNFkboVgoKCkBI`kwzaGDhNU@lzU3MQ$&1moW%oV1|GUN z4w8(|U0bwb$zRjIem*cP=4YVgXLz%7bjC%H&LFqAAN&l8XJ=t55#@ z?B|_ppO>ZG>SG?xze=tko zrT$Ve@+tEJOY2K(6MH@5W6iZa%Jl>WSd0#mTKypeeEY4oe_6 zUK~|r%~5mETh-Boj39ROrM{80fSry4HrGu^NcRRK2X%;mnJ}x{>4Bkn(z&2zkf^s`9)ow7V}9Y*ru-qbmAp8k{Kmoiy{SEOcw_-1{HwA9p50$+^g zKTbGTNt}Hwq)ir8NM0Q?>^@(Y6DqG}4AJFL_Mlc-V+_Y%5~@3VJ|z%jK0^}dCX6pP z@KhZ8SfS6i`pO_#aG6TKnmu(Un?xuX9K9!C5 zGzNi~_SaaQJ9tRy_pyFrD@!=s6F$?o->NbH5m9{W4DZ-RYFX=*znrL6mf ztbBMf3pyg#X}L>ob$iqxb$`P9k&`OhxFp+c0)+y*aO!xKH<5Aju6-Ci(=)w zvvG1l38KIMb{TI(fBiwI+#wVGh4guUozn16Z|&43elN>+R`}(s4;1!-an$%Gl?nu< zqjG)mFzMEdj-^GXMtB1Srenn5>s?JvO)|l5TS8@Y4~w+z?U$VZ_NLQP`zl@BjHD+6 z0tMI^tvo4bwAh2CTQMq@*1PU=#iXYFgyZHDX>nqn2hu8%lKt74#;wnWxN#=gkeBJc zpiX^ZK8J^|{c)rodXr!@_a}?{t$s7`N>NM@h=g-*SnKeHmC1x@eN0OBT5yV@g#RDz z{>;L`OGh>QSR2Joc_rGw*Noi!!O9zVW^$9K2S>M@UuG#LtRAin4y8VqV=DaeGAuuk z)u3L!=7c;wHW#FVAQ>IZXE^zrY|L&Br0lZ z__VEU!>x*iC$jWtwhmT@bRi)j>6xzD_~ZnPZtQm-6->+b#NDg=#2-$d*DNE%rmSW{ z)6)7!Syz`imigF#_C{r+g@Txvk?O`cEsf!^a2W3V!V9Y+Wovni=aQ1jmJT)^?;fak z4C7i{3!wOlaig(tcXZG|AuJRXy2{u-Sqg%t>A8)sLIQ8SsaNQ2c$_ygq+dR(w4a%&{=+slOU-Jj$3?#^ZWtM zN9r`{=f)@q=>xG);xqrCBaV|RhJ=Sx2QD%2K7n#A$}g-BA38dJrAEyY2D_xMV%G@I z;7K_K2Q3%I1v?gwRJ?2}B-berqBZjvgt-GsNXjg@CK(>HM7!KEDVbobQ$f>)m%gX- zf$+x5LKGY2v>}xRoxcq-ii8%JrYh71Kc6ax`nJxO_wETXa^1I_=2A*Enms`Nxa8u% z{pg0J!hrtf&SweZ}{f}CQdro{Ed zU5qMW5m^~o8u14&GI&*;r5?-XhP}b?>9kTCT5HuvZSwD#qYmv6XZiUssl$jqOi{_g zB4uT*P%KHr(;U9K9GAO^|E@Wz>FUaZY<38XhyE5+PRpXAqH?+H8cvj35yi^W1v|*X zcfsD|caAq`+%^+mtKaV^zN)ZJaXna$9Jb7Ox6(TR3+Yx{hzu?V7e$-dO*mrBk2l91 zA!$gp#(8TMG>0_Xsp`l9^Yq)*!nwJ**>H$C9&c(hF*B$C`I8qUex^B~d}r1T`;HjC zE)khXUBahnRkz|R0jS3>yGxPq%c|!o53Or5B=N;-q8Avcs&S^`uFACNe%F{gwR8yWf2R~?=+i2{x^a1_ z%dhw&^srF8;7y;BvmrTxtJbpG>q&+D*~f5t9$D_+Cm-%fP5&-$88sbIVu=dJ-IlsS z`sG(vHQPy_xC%8GH_7Gk?`8eFD)(6yr;sy<%NagN**nTPDhc9@pKH~UM|hdD>x8bcs+B1)8BdQC4{ z-XI(&Fr(e@^T=I1e))VY`2BqF9$xw8!{>5zr>l)~Z%uen9j_7O-t(~Y@f9be{PIUx zcd)}a6HdOn@o@bqd!XW$X`$@3dqQ#n`d!@pq&FXVdXC^AQ=ZA1|8w@QltZji^uaTn zNSv>u%wIGV_f;Z-jAYasgFfG6y)XUxkHOb=omeQQyIY>=NkJ+xrOTR|l-8X`pRAA? zW|a2!_Ndkn*9(g`{VNWQE}RbE?KPWVXa6}2(ov+jLspRM5$jT9u@x$b{Ab zR`j^D>_k|EVw!kCWm?30j0xyIG2EfgCj+mf!6g-V9SyN@x2t0u-E(y<@a@23-9!y zE1Mr0CH9)%Z+7^?bG$u;PL1oGCKZN=ptYs=>z_V?SeWXQm{Id`^Y_N?GUU`vq;_?f zRXWHKqu627YLiJwWPL$oQ6q+dvS$9Gzr}K%zSv*16h_2AN0{ZKcr%Q~r%UmW#n~Lw zw7A<&{O8EC%n@!rTXbd~WNR!Ugx}^B;$~$Zw}w;7RPPk_QT|PQgu;;XRXjVvJer5k zw6YDoU;r_ZNLiN=86g{!ZuL{lXP#UF618@SAx)=`VaWXSOQ=w1d2 zzZE9qyf0QXS5Gm}6N;~YmEGt3r=PS~J2dAIfI&tiB~|Gw{UzWHXHF+lFQ=bpWv^~73Ry;e70Y!25}_egn>gr$x- zERx8~me>m7Kep9*z|UNr7_{a7?)y7rHs)DRki8^_^cl4T?i5{X7AvAR&S6i-%3BAk zoA`s!Sd7XzgJqSCEl`LMiMZh75u@Hk29sCPf}XmQH;Y)C5qj-wV5sBz$XQe* zk1>vWEt;YaS1UsCbmHRWkY9xI7k2_ZX()DDo1YMo75DcFrSJOBUJdcZtK?{!p1*5li~H5ydN4%^mJy$91N}Si+p~a zUFsm;iwvnNa$)e+KP7E%c#vw-;RG)t@rREoRix+^Mza_8kVz)^$LL)JF9dn9^bQY_ zcb#lpx~cGGa$F+8Lr7DrE0E(*`Q75RT$}+rp4q%Xx|fHMY!;-hVv|0$~h@9yU7_sch&1f1q*o|k)$b6wq)a1Odb*So{Ngv-8rxUa&_Kdc88 zL=XHl^VJvlF0P@``=`exc&NCr>gM@8!tk$w!LzSO8>PQQFMOS~$Kz6>NkDt9C)>^& zOhM)Si#|(S2bjWs!&LF3;{#%I@; zEbnA_^;?WY4omN8%wDser{AhQX4)Oa3G*?-{_(@^_*pryjcqsjEQslS8;#m1<7LFV zKt9pa?2D6YRPXM6`u5Wu5*c4L^qSEvHEP${e2SVTa)i7#^9?@~{N*B6fASQUmq)P! zjr6@<1RozT^9l-JJv==2rV7K_YJ^R+LvvY0aMU#+aVHb5A>Uo46Mf}-u{fiSyRGht zD}H|^+u_w3*VFt*a+#z4S3O!H!mKjg(CSxgF3ZXbnCWgrUX#Jj*Ls2B@!8oiQ-w0m z3L`|)|EsJ&20M0dfN_Ix*pK-_S#_|Y(0?B09^4*HvOL=~vUyj#pb#hW#5XiNY&er1 z4=NKJWOz^^hbht(($uP#6tq;IOIiBFZpr0OB`qHn6YKGo5~WCAD)XdQCnu*Erz_ZeBNFe1BcGhEKRHMuyKmMgst8mjd$-%9ef{QA|oOS%F1LNF)+x5NjUKJ zTJ4JepwH5gV1&7>=n6?m!FL~`W^Tv#+Z2C6y!gnb$4JW`LB#;(-C1?iyK*vBCxdBj z2>n>_=F9JeHo&WFPEKULBAUyp0tM}Cg;*9) zjoiiZlD8!j%Ux$HJT#bUk~KhyAMsJgiO+y?SGAgR3Vr-S&b~o?rpv~uLn!%6x@g4N z*|tLmBOP66LE*v4n+$oGQMnfdOK@tcj`_Oz`HPDdP*zkF><+$vwraHY!vtMrHoQN) zSl)2E4m=f8wK`xCN|Ln5Df6=9S=(4+jzzOXjJ=LL?_W1jUPsIumGHa?$`=9X6d4nn zT4u4Qq$w{SSsJX{3=5c0h_1?!F0QIe&~E7H0Y&#O`6FH4!d_jS)Q<$Zy}pm+=YLe> z5~xvlelbQH=5xMWA^G}kjpBE(^ z)kbNY5$8NIi(R1AD_4`jWL7q|U{lY5(!s)CPF+>OaKi&thWXw8>4LnCk zTX)$DIBtbw>FOCzjlxB5s8nd?Za6LpyJ>%Q>8>ZOpe2WvKObpalu(CC0}+1Hk^j@@oaWoZf9>0|7}BP~CfHyU1Q z^=7Zo?RzOrk3Tbxs97nR9HgJIVI$z;n&Yi1O#E(3PBb}KD_62!5Q|Zo2x&U|rqqQRg2{rB}Ak}`ip7Ip-5t=|v7In#5>OJh$IJuHDtB5Ft>=Mz+os4|sXaTL6O30;9(-BpL z8%f?D##Kgkx*$*;(@+%i#)@WI)wulmunO^LMcX=fK>?mYywFQC8ZbwXJQDcpnG~I_b6?yh-d(WT7G(= zf9EW5W+xwPHbB6iZ4hICtz4A@s!j42g5-z=f*-S8mMoU&c;|NyvoyR>^N;GXA`>}s zkN0aT*Q=k&m2w1irhBH0sr|q%Uv+83{JAv@7ABM+H2=#F7bh>D!8&GkG?`nEE8=$D z!VsW<)Ch|2f|`$~~`0!_m$ ztTMF>qh1=Ag`i?H#RmW5E&g^v{L259)k~Gkyy$2x)IxvSp`@yz*AwM=X}(X z2<6ka1D^v9F0Pa%Bd|G0fwz#M%WF^dJBcYf;kmwc4>*l0y&( z{#1;do%HRyOvjSGWuT@GIzK-z15)$l0H#R1ZuQfg03d|PiHYLlS-hpZYu-sL`fZ|! zK>`0x)>I_IFpBqQN;Lp>G=mX$P|F->EDv)nP+GYbI6e zur(3~DyI^;Bs#T9&1%VgBfyJKx-KxT{{BiDjvPF}xP!h#p@x#RKykL=~m zR$P*L`9teSm@Zs}bPSQfH6kRC0&A=R-X9D##Kz9>#%(G$r$Ild3n-uta^~Ar7xAnZ z_l}IMqSscaUIx)kQUC&pK~w56TUe`!k2FVGJ~EG4KFVs?_1jxE@3b$3g**pbGId-q z(XD$!HwzCI@bpZ{)h?RIX4x;WN88+(M%3NJ-^#~Wi;t^!%;;=6buc<(nGLpLjr|1z z3Jd>)s?9q#o1X|XRvh;g`w=`B48j62HZTYVY}Ez8yeJ+^jY&?LjD1NX^6Hyc ziZRuz@|Bh-yD?Htua%UdGkuy$6=|Py8FgJaiJ-Amarubx;+aq>AQN2S;9^U>P5z)> z`lJ8d$2FyY{)wJEzS-NcCbQLI%zK3$8Smn!V@!V*UW!K5C*D$137Bb1;`yPkF8tM? zqB3^Je8cP!4FF8>*9%)(Gfb;e#N9`yA##!a-XNe5hZGqIScuE^kY&2bB8tP23&t;Ci_%apZ$yF?;-oOA zHBn8~UtIRp@m1&rlC1QpO2Vt9c^NZsOLSl?H%I3q8HN|mbP{%0t1=JKBVMqQnw9@~ z`JZdR(<-SkZk7{k`_W>t^hKiUY-;Y~>{SoJQg3MfuFS|x{}tNmvpz|&nbh>Tz0a%{ z7pj%e0wJ=Rrt127?I`=nZ@Do|oh9z`2l-5K+wI7P;BWPupS;F7N4CMbqWzS2x#6%iBq?ekjwj6|LUHX3ZTJobKtmnVNnsy!BTT2&4ZquF%r@54g972n=i_H zyAk`?*a?>|%Lv;$RZZn|vWqgR@XbmmGH`m)r5Qo;-i_b*RGP3+~#rw z1>OCE;(tPEXZ)zC%f?p(T`(%+jec9a+tHE~K8rrZAJ#=)pXs_07j%=~{C4PT>QVML>q+d2q zonhCl0b6Y%B)(U7JZczF>*9u0Oiu2F5cq+!wI4(}c2e~iumCMFBr7gO>P|8-u4#CC zML$&_g$m@}*)lb1DwN^!+T==Ty*o6r@XFb|v~-$0^YfwfzbS|*kzX-7k3Fsj_m9M)PEX=n2{HU}h~T)b6BPhl6mh3BzAa2Bv!a0D5WkXY*U3n1|J>1S z+QMIE)<{=?f-rIcwJ$_ivp|Aj$FwBr*Fj#1$1ebQa=^YpA^N0Y8Z@pB7%UF97%Xdi zA1V$Zwpb1XnD6^IM%oQaUy|3_gryt$ZK%a4fOG#QM+;Ikp9Re?=o~47QiCQ37uY3I zf|#!Be6jzImSx_*pOaK$pDi3Aoh|TimY0vFj~bUFv>XLjr~i|&S^B$8vV5-Z-HW58 z<}A?0QqQcAp`$ZfE-qFJ!yvWTo6HAgRT&_dKXXM? zfC*AepX;3{;Ib#0{q}Q7036aZ9Ht=D1MOWh1oX}?6bKg0)(g`BQGtIv1$E><_*~(- z?oEmrtGd%F=L##a_S{er#z+e|tU~}}yaRM`@8R}>CT&cv{GB>j%B$d%wb{SythoU z8v=5eT;vL{?!eRzK;$Bq=}g|*i|z7L^>!ih;D3Fj>Ma(e>#KqqNuDoOU8?S>&siVS zT@f+(0_H&LwA@tiEl#e6m5r)niSzu{JM(3L)a=P=?n&Bz!rcUR=s)AiIb~chWOK6= z-e>yA17Kt5>)eNuIIUC0a)VE!c&gI|O212@{xr{0H6w8=0kSbBQV2X-HF&A`q^W7b zE%k$B{_r0?R#u5Fjf-jX&Iu$i=m603TK2Ei#9gR$HEx-%?4M|l-up;MO0Hf!iP6s6 z0t(rree%T;!E9(v6lM2C$D2>-e7UY=jwB^Bp$Nb-+QE61Aqva4#Ppn=S-PUUdSRHF zyvlqZ4@_k2sTS*&fN?go+ecqHT~}-woq)GKT0x%vCv?rNeKiBX&hs$;0yv z#$dTjB~K0BoJw|T!S5w#Z89=tT3n-q{?H8|^Lm|AU=mIsV)48LIU}(+Jv^h@_jN*2 z@MUG-rcTUkO(SAI`S2=FlUFLKAh@us4EcKv?aW+?AnEdV^$8@Ca*1mTb*=NJPiFpx zg!J4ctMxRZM|M&9n_DDTJ_4yMx$6wQN;qabu|-0aga=Qe(TnDNsJf5caXHU40R<*h z7DJzfiO!`3kMxQ1IztE5#hn;3x*c0`?T%ha*xFaG9dtOv(+;cPoCbLc5-2W=BB-an zz#&zbz58P!&=8T|qcSp;ARdZ{5mySp`KojQL6?72MnYy-AsQC2#&@2Xg*?3USZa^g z6IUp)TfTGOxT4vfaAz4IgF{2u6BmhH$%}%X1>^cl%=iA6zm17j4v^Su?fXYuem7-m zCijY}G-u0zvMxDyB?a(D(l>9sGNI-faOGd8+Nb%7K<&4R$h0!A-($0dXO%c=ha>Zp zTVfU9GfcY6wUn(Sw=fy^;;OsllaH}g1U&wW8{=@|PoIQin>&>kVydlSF99Fa_aYH+ z3CAE4lucv(KnGIGcu^tY;mmlbU`09BvN1759{1eap}@nMHtUvGR766-=V-aLbcD<+ z+R$SsrS}4WD5|Q4h6LEq@mLKNz5e~B3=tOtA%TJx+vRCceJcW89Yk>Ufi}ryx55Vq z(%_j&Otdb;AJ~J1FyeetP%* zy%h8)ZD?J8+;yQ$h4QlW{rmUHY54LC&xDdvQVA5uo6o@PZs44z`F$WA6+;kO{N;x< zU@N4TD{sC5a!HxCOuYG#tjdI*3Iv%7BujcJdKs0{Z{B1JItGS^OUIEZDWo#zFWa5` zhp;Xkvi59~x6Yqv$rlVd-HLr5kLe4C@Ti22&)oNpKChKtS!*fKY!wz1Sb&Sw+2?_VxjWWBE&RPO7*iqZ|%0n`!-01VdmS!rtoV zwc?|Sxy35z#FsG}tm%^G3|J{_8m4#FFX_zaMHQTPL(^O<_ znkRdc644U^1aJ zXs^ZIAIIBt&jHL$9-}lVwEL9GcCACIVd^C*snQ2u?i^_!E_FeB#H19bMh=j+h;%ok z$esc`>(BJ}_tuu*ljU6g{UDTEd!yG<#P9q^HS1q?nVg;K5QPYmH}CU^NWUWHm<7P`pksmJWkrcbueqo;0>!p!w#hW60ITg$tl(02Fee5M8cOs;&bL z`{r{D1nX5AIGzR3(P$muz~sH-&XW?avZ!4&9@ zR_gY=dYR(yiw@Q_bSvQe^phsltQ<6?x`5J69mu)+VCesU`wX<20DBJR3x&&*0v7l_ zzbF_{(ypo)jPJsQKsUweUW=;EVx(lsUMjw>WgXGkZ~DlNZo{5K(669pT4<__ zA=IowE*i6m0#a}mtxhE97EicGYCaM|SzTfvt1!qDYf+lC`iicfzDo%G5A|P@*kgl?A@Qp2$8dRT4k!lVfM$ z5;y1`82*h+bxQcEb+-4H)+_@{+lIR~XAFo~!0GzEVe}H^3`s0#+|O~t#GZNwG9-8S z6MyF%cbXYlclLi6#=cCX+KYu@Pu9Sa3#Q1(NhK7ED93iuTO$ytk15v7E?{U^gJ<`h zWNn0n^5gC2oD>-K_sEy)3XaKhT#2xTlOQJ%D0;bhg2 zF4nmO>hDiyk6SFPRtkD-bEtLi3v-W&aEQNLcQ~tmof#!XTjl0Ujqc3+(CGZE6|ltz ziv4*iY z&UnqevUI2{Z&G?9ys~*9;*NZshot8#;r0HNrN%hYwnG!nGHRvKg8<0RW@Zv4j_nMx zDSl%W5y`dd*e_XoC_&lsd)4{*qy$*|YR^xP`|Z0qfa>QJDqC7I9xc`fx&2)f1NArx zyW!u(dg`nFnJ|!+l>}C`DI0+-2)9POqTgHZ51MUv{^a?1wXhFjN;Cl0N8-p!p)9=3 zaSLQ}g$qD3>F|Vah3!d+AVyjjvF>17!>$XDAmDI*6mGrLh>4 zY1JCKfXNT4VqyW3IvKkj(4FV#cXyVRl@%B~0!>Jjl`$-}Ihx2WL!~?fRJ{0sjXco# zn+EN*vRZDjm2H2f1EKuy-@e^E21-i)FSfX~CJuNqJ79||4g=_I4jZtVwT9?aUx6sz z???i8JrDwXmF^Yg<;4Nbsk5FC-iyYQ%NhD38ls5MIff47FK-P$ak0*a!I6R|$Qs(G%uOJvlr2m|3=aexa)mJBWzQ62!2#p|bM8DFSq-6TMYV%WG9@ zm0GNAt9Bq}!oV#s>Q3~e+iDlnAB41(qGj0sErFWd_23;m0Um9NlcOO)YKVB@$W8-O zH0k}^NvsepEkY8VQeyNka-%xy6BtPunV|6ajEomzqNg`}7-Y4Wh>hsXWBKYOq03uC z$&4DOfItO(B)EBmc{f|;aJ)&oy}kXA&);?ZyNA!Kp5O@*bZ}sWgM;&6ehWR8269RF zTF{Uj;=kp=FzbmN=pQE?BP}U8Xx~zAeQn3kEX~IJn%}o_H3C!GhneyI;bJ1nZ)9YI znvrq8!uGeb{Rc-u*Z6W5kbvPSMb+(o!D;?e+}wQr{b3xr2pNO|8Fp5H`T>THk1h-q zuk#h8&T_^a3p+6>DSyri24ddGZ(2j{b2(+C2LVwUzSyu)6kN>a1leGZP3ddRna&B*uyRCZFu`sgt~ zY5u)N7S(BfDeMP<^f{3LfAia%=jhE3&9@*m=+Gh9_VQIvMtJ^&U=%Q6U;sybFZ<;8 zQYMz99}wI?!*0)fwL=?e&?4h9=MF z{-68(QER>VI1$JJKS;@dAngJYD0?wT`1}*7WeUI}4)P?Z(y=5FK!bj^Gbo%2U+w&C zvN;&Bw@^#Ixw(mjiyQL0+iNRHMYtbS)$2~}PjE6Yd?Xkk+_FBIFR{1OeCSs+A%dD} zhD}V2O7*pg8dH3?o%zo%IHDgtv*fVua@pCD^$r{Y@O->H3*N{OhGsjNdb8J%z7hVq z`a1(YI02HWpM8$2`f`t39qrn>cW8=+`<&zJj6-BQ@HT02FsyZK_f~ZZ?8@rWhb&*^ z9s}6zKceHl<9w}HxNdIIA~wI!uhGuzqRYLsLqLk zmx~s{O-<`_%a4yzDf1P3(kSbJ6C&E~iqCX(UvKclubGM&_FE%*oSU4guXD(gV}$(+ zJM)mv01A#?OYjd5|7)Y4#ZQa16(5ku7RCnxH~Sqp&EurLu-|1-^wM{IuMV6Ng|P?U7RS0}7Jyo+7u>-o5J zxE>1mM_tG$U2 zdLD1~g?MA_O$MEuh5s4dmrye>F})m{!*S;8&g$P^Mf-VdLczLI=TrZI z7JGT1@tHj4T~)W#p;1yr+0GE>hs91^fonSSxS_3Xz8-Gvr^DLbVQ~CArpF6>^=Hq3 zku4bP4bVz{(;4=1_!%-^r4w|#+G(==EdgYCqSKFei%y&jjp|jnUpTX(imMmQa^$dL_r)#u-lYCk&J zB>5pW*2@3rH2ocpnYza0i-Z6>eSiD>nlE+$yx`f4ipT~St%KCzdKe1oP?v+*7r^lU z_TJB+!$l0}NFc%IaMFEk3@%K&-gt=ou13K1_RdE*^z44pf-7CngX8XUKM1520kB}U zfj-~bj)loW8O&&~;Q$S)_gBfN=~oqz$6$aC_;PY`1pH1#+MlkESKIbWvL;Jaa!)`1 z^TP&a3~*;cW@q1l*QUPN2qQlQeI$io1-pYy{38d#bq=tP{Godb*gh-(kWJ6boUJ%d zd<3~KHUhLno@`}$a4a(_*j~YDreu9!GkuDcK;0r5*m2qbN%qrQ@48<$do!h&FJHZy z0@5?HZjLn$St@im9b1;ijE%wcCR2_WoyC_3c7h~yKEd(Mc4 zickpHkT!t!Ishv17*Ps;u=@cnj*!mpTw;OdP|e$NhJ{g>a_@oaNQM3Nbg%Q}?2R4I%C8JM6{3ED`N zbCKZFkjn<-qQHKw=i9_YTG4YuJ(gz@E)oVQ#y&kiKVe8n2o`)fJKV?_L#NZ=s&av}aPQwEgfpQ<*o8W~*gfT}|E3_MsyB_RnYvU_(k%lKggb59q zH$t4aC{wpscW8IKJvXcW+h8`PW>^^f+cYKq572Y>oF}@|?`D{7ezlN-zMer);3TA{ zGo+4AOoWN&#}{E^zkK;JJ}Zk}E-EfAOc%^zNyXXqfRV3;AsN}|luM%B_5-;RFo!<` z1hXlbnB+NX?c$PWUE90c-f6zsKLjTbic3ii)fyuG`gJvUpF#|#!C^3u^zUMdmdt_! zENYcIcpn~t3CKq91|ZO;s~wvj32?3fd&UJ zA2wZ^vWl91rEI&Wadx1fz%RGhhpAipQy>E%+r0X`(`?tgbh@#pBC+)C+qb?PAEdlV z0t&6ejjP-IX(HMS6E*?|FsfbdpxMp`4m>8r#_;E0}^h`Te>9*>H2#`~EpPPrV zS}>Z5bsZdHJOpgjSL%OGUMdG^hKX7u7oT0SZAh~Bn5=OgkI3H82#5@{N#jc!eL zMA4Wj{g(gH%_U0H2|C)AR#GvtKHyIO>qTkSePdU^`ur0q&7k9VHPdF@>1)=X$LKwu z2t-a5-dLVoqcD8gLfXQd=kJW`Pet%0S=N147PwJ>5P!O*V69%$cU}sVh66$MkKFRT z`04)sLt9-Ax(HRzqevxgqXQg$iD!lh6xSW_Kb{uhvcJfWS-?Qp%H01NV*8hb zjwFo3EGogayg^Ak$7uAw{oqdfNUBZV2a=)=%m5R~1^6+|5W;4>YMu;J(j3xn6I>n- z98nh82Ex!ey-)dqoGDNHDo%Xrh}^4g`@%!xc>v}B9~U?al?!B zryiur(i_!lovS&m5BVmQPue2XO)-ZEJXuN)Juj%E{qWbkFVLYQVxx(9F`(m5A^xuj zez20kw{L07-ZqBrXzSey{Hvr7+BiIUk6$?QMFCUvu!o_<5jN`UK?3xt5UiR#jogA; zq#_OpIv{t@=f#9E^jxcDCdVi~pQo9$%gkN{yGh!pi13mW!1~EzWD#phvw|jHf zxGpEZNvEc!+U}1#>n*0JKe2gOr61NA;C+LNEvn-$Dn*?^odeJU5s; zzIFs?{k$QxPzUqBfNU;g;sTmY25kduz7Os$`_l+$19^3ISO8q0#R8}(P{3~VNSpe{ z6%hh94D6Y}?N$^|69>l~Og1Vxox)keE3<=tcdCpU64lmqVimj*TVMpSej66V;b7Jh z)bGLZnQ=l-@Fh=tq`0^#@r*$O?#X-Q+XhLzjC;(^_bd}_!m#JChs~%1@YSqR&($mpx0!~;k z)t68>;sAPu6LR5w|VttkqGdL-#dK~`fR9fezdfX=mS`F zt$ZrEfRo~vk%tHGdZC!#@UdHg@Co8UfY(O3QWX|VmM9TNlkmSBOJf}Y#Ss(+*q<+0 zd*C5;%t#qbe9}3)puk`Q>UjOiV8pAlSq+s(riAAo&JzPWXMtw?7MJFGE}l4C)-3Er zZNRo?1DIv=yNhscZZ1434*w!bK|XJqQ0jcmqNRUzU6+-WRmREby!M~pGaQ_1^<>}v zzk(`VO8#r+u>!B_<3z9R?rWI>-F)o0>i`(Ip$D&%qt`!&2JL%sXH1NZ9oG9`I{~uoWieBNK0Q4>+2Lvp-DJ$n%)m@7N<``2u<2>_Xfh!p z-rw$pH)uWw<)wAbyUGkH@;Bc3WGNuNCj^3pdoBVtD{a`eTTP+RD zl*p*4DNv&I+LHLeqYe0n$0sFG|15;SjD61u8TJuV*W3^Y+3N6EN5Ff#ojb5y87P$3 z|NVRei^6{kPFwGc$i^2J=Vy|({#|mBZ-^}`AdTTBOZVzi9dGC59qS*s{1*JI}GIF^Wn@#52j~Gh)F^Z+)zE@3w}AKiHrp$n%a*jH+)Da z)-hOBzWhAz9yM$(nta)Ah30RiCI_NU|!SR8u215 z|KT^9(I-dQy&HV~VA!J#d}%Zm>R7=T{PMEF&`t^wr2aTK20oW4eLIE=R+d?E(U zPr9(+*ZVpkrnDqbmigyHwtnlLhoXmq{GKCool7A8VactLDtxb>JJ})^t3_9x#TWdy zX?JSQ89_S4T!?Ki(_-t>@5__*FTD_cJbVkXJZZ;1ovXzT6?*3IT=89dKk~V|&xp;R zp{xN96kjaD&kxRBpwU;m)sEEj`uNSLo{lg{7`I3Oeo?S}$|D6z<=O~nHu<{HR~EJN znC@7QnOrktQHE95a_;)0^C5mPz7iwLN_w-wj59S2Jt&J^?i|C)S2V;Hv*o)m_C3Gb zUxI=HDr{Wb+vjOWpK-krSX+P}$jp=|oh@0{<`)%7{#f(+ije)&AGGKgL(>sN$I!@QnvX#R=<#-?h@Yw@784v`F1EL84G@Z-8|L$vJU{-+IA{n;| z*dh)_7#vts5;it}-;?@(Z4+j*UE<=hnnecNWzfRm&9?PMup2wMHUW-C z!KSR37);OelUH(D8rW(3?OII$pv%h1^@pMp*K|yf4qu@*{)!F`hKFteffPo4{sHc% z8)AtZ0F%19NWkDhIw~rv_>`2G4c6}9+o(hIHaN7PtE|tm2c-Oth|)>34yyKC8oy%fAy?b~G>kN8Vff7pl-buS!^F@S`=uW|J9Aa zacrxIR_`Hsxo&RjdH;_D$bpdJVgIqTDB9bZ(h1RKb=2Z{22?)L`s)|fsCwAyJ%;?!q1!!aL2K{wR>+W{ zeV-c0zfAi9?+yhelTt_s4)jnEaBZ_M{#8aJe&PTLHK`$3NV>33N73I9k{c z={Tl#-Sd@G_s2=%H=CS)u`ap^A)d zH6A$8?t^bEYM?)mH1dIs4GJJhP=wxHpTq&rZ{Wx+$ZQ~o?c>9vkq#b98tR(gvk(Xb zB*e#q4-^Sn3Q?MQgMkctZm%`m z=6%b7f`S4qUqOE9x!UAbXqFHH-t1QkG~vPtqSP?=HD}l$(sNb{(LRoI?J|s7BOx;x0dUGzyPB6(O@SF`IHR6R+l#cNr2}NPUyAq zaWRtQgdc>j=oVYgN_yk)msm)yFXr`NMWi1DIh29T<8>cu6+wajM)%GBR|8|>lz%>a zpY$W-@1L+#9-~T{Pz9oR17wJ&z~R7%}j4^`GykUH<% z^GYH}`mErv@Rs;(``xw7z9k;+uJ);5-5(@!o95s5h_rD&m5%^2WG>%nB|E0yXCQfL z$&IFijyy*v`k<&ViIa)dF}PC?c$>__Fg1hTQBQXh$m`zP+%U@%kG%34UNSHhLflCm zrKD>Ikr-0qw+=%L!Sg$k>Z2E3r({=z1R}!B{C-h08fR;!>YExg_s!%9Kc6zppJGbH zLy{q*724lda!>IZBUT0rDry?NyMPitq1TTLB4dFjlrK_Lkvir276w*M+Yf7T1CUac zE{%t1SQimz-hsA4TiXw974^ec!5t^@uSb_aRE5)hGlHJY{&yF6>ySRf9p0PU^MQ5v z{l?+VG!v7nPHnAQMCKkVx!+$D;2fr!l+T6v^X~~!705!Sh?!t|fb7hYIsY-rhHG27 zJYPG*79UJ?85lnJ+>go(`lhgmiK7-5wfC;>;#;1y`q|LTf20QhU|R7UN#b+U#mL#+ zxW50`tXuD}A->{mjVUQTM8VFE4G^de81APJr`KnzafdX{L#Tbz)o98%jyGoHQK)8U zpgOBJ>W6h{yS1aCql?^lsn^<$AQ?PZ^bYNA#I4RIJ@4K=cw%<|I9FmgXvKW?zXeyE z6i!!$hlVzS0E-(#z6JCPC{`#D5fRW800a>bfjb=ozCG<@$k{HiV!$j>cEEk|J35HK zvIGNl0JXAy^%4~Vf_iacL5P&V4FRA_z~?~0V%Qa~kQTV{{_W>uwAzLHd-RaA)}Spu zm&18nbx8MaVIn)8* zyY}7hfditEvth-edHqtoqh2>9@5tF={VoMW)Sm6*%~)+@B*eA%xg?&N^sSrZ$B!H2 zOvL{$?t>4#n>4hvUBQW93Wfnyiy*KhTzX24{l zI%d>gxD9`yTcmOhlkS_%XXP9x7`#=ycT zW@=glj*7dhijNx^2$PTy>ALrQlrfJ{kJ>~b{9=dMb9Yc#X^0QaX~K*7FC5~|I5|04 zjatFf#<)u;swgU=B?K%SjJrQX(9_z2S%c#Fs{Xqk7Bv(DdaXTT>i>XO9kg~5rdS*_ zurKW>g$_KnROTciWnka}Tn9mD#W+)NR=gMkQF?CGn9 zxn&$1E*`3yg*Xz75dQ`8_%d?~G&&ru?;Du2Cw5+4z{pL-lJXB-u+0_|lYsNTy};LR zez?Mj-C93$ud`nh7Cak#6&@8O;q5I5z~U4LMg8WDk7KqRXmDv`A)tk1>*@YP2H+G} zMNk@FrQ6Iz62}$&%=i8KcQ)@E2S?5wFbyq8w;H1z&m(69SmI~EFo!;$_4V~o!=OYt z8zrb8k~?vom4mNQq6ls(4FIIi2JS~dC4~o4DkpHjK@-c<@x}uk(CUCu@R5rs{~E`{ zV?eQ4ME^-Qhdh;N`l>f=lClt*EOj+m{8IjYLN(O-Wzyz#SysA_A61A20KFoz1}1U?gJgg}*g zus=c{aT^;dBR>C{)q{6GTXt;C$40Sdn$$)XYP?wfkS;{>{_dJHh6Fk*66F z?bM+i6#A#zGpiNvMy@795F+u_f)$KDL;_on$m8WJ;6Kyc%ND%)I>!1elYA5zbx0Zv zoYer6kyuJ$VPVOAw&-3YR}2S`Q4gBgAI#*ohN31=CLd@GZ(XGip=$x8(4d19fOlO0 znyJB6-VaDx5crWpmHx}a1x&$-fvu~J()K7yt%`))+_)NpPAUK{z-dV5J^4f;;)*w- zk~{Df_pr`sz`Skc3iY%RpGB-@I5-WN>K?8IxypR+ZagwjBstDCV4-AnWiyW!X#cfVc*tN$=C23izYi zN9nFxo{DWN2SLoFISjp^MPY$#43Gg4f56@5R%08!dwbG@R5(Nz$_+vZ?w)&%#u+_6 z_pp$P?aU8X<~`ajI?R67X)IXqVpN|+Ztu&DwDH{1h`Hsf-GBJ@ekh2?0t2o>8Po4C zVA&tRsnci~^cm`??E(Z1pqo*=W?DY>2(+p|LloS5ak~h~nv8uHSG&JW>#0Gzw#%zbMo5A1uY4eqa|)rZ%`cp%@Pd$0122y>lC`*AkQD)pdxgY z>~4<;i+0>zOCF94fDNQfir=HWu z+F@m%>`E=;gixUPHcYVk!V4>R)GULEAb20pp^{C^78_(e&y(p345@p-(DZbE!N}9Y zrc9pmLM|%43BZW!y1MNttQThOTYNxHi=mnUH^O5}e(G%xhF9{db#uiavF-_0LqxhuNXS7rpFZtL)ffL#Ub z97Y@fdWXNyA5r)AuePoL4%vL*8^D3i-y1uQzT&gh4vZ>seaSir+5P$_Qw_*?z*B>R zVVTv?-xrMWsgTn)I75L)Hgadh!@=w?NTlyORktr_PN^+yg3&>M@btbE!50bqp7|0h zfdKV>Z3wh!1%sQcfCee)+Q-|ug7M$-w>&}UK1@?QjfC$O8#1{NB-d2_Cnvb5rE>Lg zJWkjFoiBT9nSfwUaNaNh1LBDgH!gxCw8KhTJ<;VIJygMHc&g7<^RR{49T2^(33RRz zKt9ywzBvBZdt3?gi?MZB0+zJ^5zr4D1x3Z~+@! zMfmprm^urns?#pq9~uM!Y3T-~yFoxyP?1oi8$lY8Zjdekr9)bzySqe6L`u35q@<

+ + + + + + diff --git a/docs/layouts/shortcodes/generated/common_state_backends_section.html b/docs/layouts/shortcodes/generated/common_state_backends_section.html index 84592cec94a04..383bb165c7a23 100644 --- a/docs/layouts/shortcodes/generated/common_state_backends_section.html +++ b/docs/layouts/shortcodes/generated/common_state_backends_section.html @@ -44,6 +44,12 @@ + + + + + + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java index b95a1d97d883a..d17d12592d1a6 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java @@ -109,6 +109,19 @@ public class CheckpointingOptions { .defaultValue(1) .withDescription("The maximum number of completed checkpoints to retain."); + /* Option whether to clean individual checkpoint's operatorstates in parallel. If enabled, + * operator states are discarded in parallel using the ExecutorService passed to the cleaner. + * This speeds up checkpoints cleaning, but adds load to the IO. + */ + @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS) + public static final ConfigOption CLEANER_PARALLEL_MODE = + ConfigOptions.key("state.checkpoint.cleaner.parallel-mode") + .booleanType() + .defaultValue(true) + .withDescription( + "Option whether to discard a checkpoint's states in parallel using" + + " the ExecutorService passed into the cleaner"); + /** @deprecated Checkpoints are always asynchronous. */ @Deprecated public static final ConfigOption ASYNC_SNAPSHOTS = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java index b8972875e2f4f..5fa6b08ff0f30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java @@ -17,6 +17,11 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + /** A checkpoint, pending or completed. */ public interface Checkpoint { DiscardObject NOOP_DISCARD_OBJECT = () -> {}; @@ -33,5 +38,9 @@ public interface Checkpoint { /** Extra interface for discarding the checkpoint. */ interface DiscardObject { void discard() throws Exception; + + default CompletableFuture discardAsync(Executor ioExecutor) { + return FutureUtils.runAsync(this::discard, ioExecutor); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java index b67dfefac74f5..6d1bacaa83a66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.util.AutoCloseableAsync; -import org.apache.flink.util.function.RunnableWithException; +import org.apache.flink.util.concurrent.FutureUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +48,7 @@ public class CheckpointsCleaner implements Serializable, AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(CheckpointsCleaner.class); private static final long serialVersionUID = 2545865801947537790L; + private final boolean parallelMode; private final Object lock = new Object(); @GuardedBy("lock") @@ -60,6 +62,14 @@ public class CheckpointsCleaner implements Serializable, AutoCloseableAsync { @GuardedBy("lock") private final List subsumedCheckpoints = new ArrayList<>(); + public CheckpointsCleaner() { + this.parallelMode = CheckpointingOptions.CLEANER_PARALLEL_MODE.defaultValue(); + } + + public CheckpointsCleaner(boolean parallelMode) { + this.parallelMode = parallelMode; + } + int getNumberOfCheckpointsToClean() { synchronized (lock) { return numberOfCheckpointsToClean; @@ -71,10 +81,35 @@ public void cleanCheckpoint( boolean shouldDiscard, Runnable postCleanAction, Executor executor) { - Checkpoint.DiscardObject discardObject = - shouldDiscard ? checkpoint.markAsDiscarded() : Checkpoint.NOOP_DISCARD_OBJECT; + LOG.debug( + "Clean checkpoint {} parallel-mode={} shouldDiscard={}", + checkpoint.getCheckpointID(), + parallelMode, + shouldDiscard); + if (shouldDiscard) { + incrementNumberOfCheckpointsToClean(); + + Checkpoint.DiscardObject discardObject = checkpoint.markAsDiscarded(); + CompletableFuture discardFuture = + parallelMode + ? discardObject.discardAsync(executor) + : FutureUtils.runAsync(discardObject::discard, executor); + discardFuture.handle( + (Object outerIgnored, Throwable outerThrowable) -> { + if (outerThrowable != null) { + LOG.warn( + "Could not properly discard completed checkpoint {}.", + checkpoint.getCheckpointID(), + outerThrowable); + } - cleanup(checkpoint, discardObject::discard, postCleanAction, executor); + decrementNumberOfCheckpointsToClean(); + postCleanAction.run(); + return null; + }); + } else { + executor.execute(postCleanAction); + } } /** @@ -123,30 +158,7 @@ public void cleanSubsumedCheckpoints( public void cleanCheckpointOnFailedStoring( CompletedCheckpoint completedCheckpoint, Executor executor) { - Checkpoint.DiscardObject discardObject = completedCheckpoint.markAsDiscarded(); - cleanup(completedCheckpoint, discardObject::discard, () -> {}, executor); - } - - private void cleanup( - Checkpoint checkpoint, - RunnableWithException cleanupAction, - Runnable postCleanupAction, - Executor executor) { - incrementNumberOfCheckpointsToClean(); - executor.execute( - () -> { - try { - cleanupAction.run(); - } catch (Exception e) { - LOG.warn( - "Could not properly discard completed checkpoint {}.", - checkpoint.getCheckpointID(), - e); - } finally { - decrementNumberOfCheckpointsToClean(); - postCleanupAction.run(); - } - }); + cleanCheckpoint(completedCheckpoint, true, () -> {}, executor); } private void incrementNumberOfCheckpointsToClean() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index daee8986f10f6..3925f3ca2fe3a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -25,9 +25,12 @@ import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.FutureUtils.ConjunctFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +46,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -327,7 +333,6 @@ public String toString() { /** Implementation of {@link org.apache.flink.runtime.checkpoint.Checkpoint.DiscardObject}. */ @NotThreadSafe public class CompletedCheckpointDiscardObject implements DiscardObject { - @Override public void discard() throws Exception { LOG.trace("Executing discard procedure for {}.", this); @@ -371,5 +376,34 @@ public void discard() throws Exception { private boolean isMarkedAsDiscarded() { return completedCheckpointStats == null || completedCheckpointStats.isDiscarded(); } + + @Override + public CompletableFuture discardAsync(Executor ioExecutor) { + checkState( + isMarkedAsDiscarded(), + "Checkpoint should be marked as discarded before discard."); + + List discardables = + operatorStates.values().stream() + .flatMap(op -> op.getDiscardables().stream()) + .collect(Collectors.toList()); + discardables.add(metadataHandle); + + ConjunctFuture discardStates = + FutureUtils.completeAll( + discardables.stream() + .map( + item -> + FutureUtils.runAsync( + item::discardState, ioExecutor)) + .collect(Collectors.toList())); + + return FutureUtils.runAfterwards( + discardStates, + () -> { + operatorStates.clear(); + storageLocation.disposeStorageLocation(); + }); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java index 462968fe160e9..2b6c4e06197c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CompositeStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Preconditions; @@ -29,8 +30,10 @@ import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkState; @@ -165,6 +168,18 @@ public OperatorState copyAndDiscardInFlightData() { return newState; } + public List getDiscardables() { + List toDispose = + operatorSubtaskStates.values().stream() + .flatMap(op -> op.getDiscardables().stream()) + .collect(Collectors.toList()); + + if (coordinatorState != null) { + toDispose.add(coordinatorState); + } + return toDispose; + } + @Override public void discardState() throws Exception { for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java index 8b0e2846e60d6..4fede9683e5bb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java @@ -194,22 +194,27 @@ public InflightDataRescalingDescriptor getOutputRescalingDescriptor() { return outputRescalingDescriptor; } + public List getDiscardables() { + List toDispose = + new ArrayList<>( + managedOperatorState.size() + + rawOperatorState.size() + + managedKeyedState.size() + + rawKeyedState.size() + + inputChannelState.size() + + resultSubpartitionState.size()); + toDispose.addAll(managedOperatorState); + toDispose.addAll(rawOperatorState); + toDispose.addAll(managedKeyedState); + toDispose.addAll(rawKeyedState); + toDispose.addAll(collectUniqueDelegates(inputChannelState, resultSubpartitionState)); + return toDispose; + } + @Override public void discardState() { try { - List toDispose = - new ArrayList<>( - managedOperatorState.size() - + rawOperatorState.size() - + managedKeyedState.size() - + rawKeyedState.size() - + inputChannelState.size() - + resultSubpartitionState.size()); - toDispose.addAll(managedOperatorState); - toDispose.addAll(rawOperatorState); - toDispose.addAll(managedKeyedState); - toDispose.addAll(rawKeyedState); - toDispose.addAll(collectUniqueDelegates(inputChannelState, resultSubpartitionState)); + List toDispose = getDiscardables(); StateUtil.bestEffortDiscardAllStateObjects(toDispose); } catch (Exception e) { LOG.warn("Error while discarding operator states.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 9ec66bb95fe62..2f2cc6f6f6736 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -29,11 +29,14 @@ import org.apache.flink.runtime.state.CheckpointMetadataOutputStream; import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.FutureUtils.ConjunctFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +56,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -652,5 +656,39 @@ public void discard() { operatorStates.clear(); } } + + @Override + public CompletableFuture discardAsync(Executor ioExecutor) { + synchronized (lock) { + if (discarded) { + Preconditions.checkState( + disposed, "Checkpoint should be disposed before being discarded"); + } else { + discarded = true; + } + } + List discardables = + operatorStates.values().stream() + .flatMap(op -> op.getDiscardables().stream()) + .collect(Collectors.toList()); + + ConjunctFuture discardStates = + FutureUtils.completeAll( + discardables.stream() + .map( + item -> + FutureUtils.runAsync( + item::discardState, ioExecutor)) + .collect(Collectors.toList())); + + return FutureUtils.runAfterwards( + discardStates, + () -> { + operatorStates.clear(); + if (targetLocation != null) { + targetLocation.disposeOnFailure(); + } + }); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java index e657c8d740012..a734294ad9dee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; @@ -88,7 +89,10 @@ public CheckpointResourcesCleanupRunner( this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); this.initializationTimestamp = initializationTimestamp; - this.checkpointsCleaner = new CheckpointsCleaner(); + this.checkpointsCleaner = + new CheckpointsCleaner( + jobManagerConfiguration.getBoolean( + CheckpointingOptions.CLEANER_PARALLEL_MODE)); this.resultFuture = new CompletableFuture<>(); this.cleanupFuture = resultFuture.thenCompose(ignored -> runCleanupAsync()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java index 25c623e18461e..a3b043eb987e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.failure.FailureEnricher; @@ -122,6 +123,11 @@ public SchedulerNG createInstance( shuffleMaster, partitionTracker); + final CheckpointsCleaner checkpointsCleaner = + new CheckpointsCleaner( + jobMasterConfiguration.getBoolean( + CheckpointingOptions.CLEANER_PARALLEL_MODE)); + return new DefaultScheduler( log, jobGraph, @@ -130,7 +136,7 @@ public SchedulerNG createInstance( schedulerComponents.getStartUpAction(), new ScheduledExecutorServiceAdapter(futureExecutor), userCodeLoader, - new CheckpointsCleaner(), + checkpointsCleaner, checkpointRecoveryFactory, jobManagerJobMetricGroup, schedulerComponents.getSchedulingStrategyFactory(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 51ad43605c781..8873b938f1a25 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -170,7 +170,6 @@ void testFailingCompletedCheckpointStoreAdd() throws Exception { assertThat(pendingCheckpoint.isDisposed()).isTrue(); // make sure that the subtask state has been discarded after we could not complete it. - verify(operatorSubtaskState).discardState(); verify(operatorSubtaskState.getManagedOperatorState().iterator().next()).discardState(); verify(operatorSubtaskState.getRawOperatorState().iterator().next()).discardState(); verify(operatorSubtaskState.getManagedKeyedState().iterator().next()).discardState(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index cb0e17bea488a..49a22a4d435be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -60,6 +61,7 @@ import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.PhysicalStateHandleID; import org.apache.flink.runtime.state.PlaceholderStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryImpl; @@ -1520,9 +1522,12 @@ void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { TaskStateSnapshot taskOperatorSubtaskStates12 = spy(new TaskStateSnapshot()); TaskStateSnapshot taskOperatorSubtaskStates13 = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState11 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState12 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState13 = mock(OperatorSubtaskState.class); + OperatorSubtaskStateMock subtaskState11mock = new OperatorSubtaskStateMock(); + OperatorSubtaskStateMock subtaskState12mock = new OperatorSubtaskStateMock(); + OperatorSubtaskStateMock subtaskState13mock = new OperatorSubtaskStateMock(); + OperatorSubtaskState subtaskState11 = subtaskState11mock.getSubtaskState(); + OperatorSubtaskState subtaskState12 = subtaskState12mock.getSubtaskState(); + OperatorSubtaskState subtaskState13 = subtaskState13mock.getSubtaskState(); taskOperatorSubtaskStates11.putSubtaskStateByOperatorID(opID1, subtaskState11); taskOperatorSubtaskStates12.putSubtaskStateByOperatorID(opID2, subtaskState12); taskOperatorSubtaskStates13.putSubtaskStateByOperatorID(opID3, subtaskState13); @@ -1561,9 +1566,12 @@ void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { TaskStateSnapshot taskOperatorSubtaskStates22 = spy(new TaskStateSnapshot()); TaskStateSnapshot taskOperatorSubtaskStates23 = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState21 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState22 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState23 = mock(OperatorSubtaskState.class); + OperatorSubtaskStateMock subtaskState21mock = new OperatorSubtaskStateMock(); + OperatorSubtaskStateMock subtaskState22mock = new OperatorSubtaskStateMock(); + OperatorSubtaskStateMock subtaskState23mock = new OperatorSubtaskStateMock(); + OperatorSubtaskState subtaskState21 = subtaskState21mock.getSubtaskState(); + OperatorSubtaskState subtaskState22 = subtaskState22mock.getSubtaskState(); + OperatorSubtaskState subtaskState23 = subtaskState23mock.getSubtaskState(); taskOperatorSubtaskStates21.putSubtaskStateByOperatorID(opID1, subtaskState21); taskOperatorSubtaskStates22.putSubtaskStateByOperatorID(opID2, subtaskState22); @@ -1625,13 +1633,13 @@ void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { assertThat(checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()).isOne(); // validate that all received subtask states in the first checkpoint have been discarded - verify(subtaskState11, times(1)).discardState(); - verify(subtaskState12, times(1)).discardState(); + subtaskState11mock.verifyDiscard(); + subtaskState12mock.verifyDiscard(); // validate that all subtask states in the second checkpoint are not discarded - verify(subtaskState21, never()).discardState(); - verify(subtaskState22, never()).discardState(); - verify(subtaskState23, never()).discardState(); + subtaskState21mock.verifyNotDiscard(); + subtaskState22mock.verifyNotDiscard(); + subtaskState23mock.verifyNotDiscard(); // validate the committed checkpoints List scs = checkpointCoordinator.getSuccessfulCheckpoints(); @@ -1656,15 +1664,15 @@ void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { new CheckpointMetrics(), taskOperatorSubtaskStates13), TASK_MANAGER_LOCATION_INFO); - verify(subtaskState13, times(1)).discardState(); + subtaskState13mock.verifyDiscard(); checkpointCoordinator.shutdown(); completedCheckpointStore.shutdown(JobStatus.FINISHED, new CheckpointsCleaner()); // validate that the states in the second checkpoint have been discarded - verify(subtaskState21, times(1)).discardState(); - verify(subtaskState22, times(1)).discardState(); - verify(subtaskState23, times(1)).discardState(); + subtaskState21mock.verifyDiscard(); + subtaskState22mock.verifyDiscard(); + subtaskState23mock.verifyDiscard(); } @Test @@ -1708,7 +1716,8 @@ void testCheckpointTimeoutIsolated() throws Exception { OperatorID opID1 = vertex1.getJobVertex().getOperatorIDs().get(0).getGeneratedOperatorID(); TaskStateSnapshot taskOperatorSubtaskStates1 = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState1 = mock(OperatorSubtaskState.class); + OperatorSubtaskStateMock operatorSubtaskStateMock = new OperatorSubtaskStateMock(); + OperatorSubtaskState subtaskState1 = operatorSubtaskStateMock.getSubtaskState(); taskOperatorSubtaskStates1.putSubtaskStateByOperatorID(opID1, subtaskState1); checkpointCoordinator.receiveAcknowledgeMessage( @@ -1727,9 +1736,7 @@ void testCheckpointTimeoutIsolated() throws Exception { .isTrue(); assertThat(checkpointCoordinator.getNumberOfPendingCheckpoints()).isZero(); assertThat(checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()).isZero(); - - // validate that the received states have been discarded - verify(subtaskState1, times(1)).discardState(); + operatorSubtaskStateMock.verifyDiscard(); // no confirm message must have been sent for (ExecutionVertex vertex : Arrays.asList(vertex1, vertex2)) { @@ -1852,7 +1859,8 @@ void testStateCleanupForLateOrUnknownMessages() throws Exception { vertex1.getJobVertex().getOperatorIDs().get(0).getGeneratedOperatorID(); TaskStateSnapshot taskOperatorSubtaskStatesTrigger = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskStateTrigger = mock(OperatorSubtaskState.class); + OperatorSubtaskStateMock subtaskStateMock = new OperatorSubtaskStateMock(); + OperatorSubtaskState subtaskStateTrigger = subtaskStateMock.getSubtaskState(); taskOperatorSubtaskStatesTrigger.putSubtaskStateByOperatorID( opIDtrigger, subtaskStateTrigger); @@ -1867,7 +1875,7 @@ void testStateCleanupForLateOrUnknownMessages() throws Exception { TASK_MANAGER_LOCATION_INFO); // verify that the subtask state has not been discarded - verify(subtaskStateTrigger, never()).discardState(); + subtaskStateMock.verifyNotDiscard(); TaskStateSnapshot unknownSubtaskState = mock(TaskStateSnapshot.class); @@ -1914,7 +1922,7 @@ void testStateCleanupForLateOrUnknownMessages() throws Exception { verify(triggerSubtaskState, never()).discardState(); // let the checkpoint fail at the first ack vertex - reset(subtaskStateTrigger); + subtaskStateMock.reset(); checkpointCoordinator.receiveDeclineMessage( new DeclineCheckpoint( graph.getJobID(), @@ -1926,7 +1934,7 @@ void testStateCleanupForLateOrUnknownMessages() throws Exception { assertThat(pendingCheckpoint.isDisposed()).isTrue(); // check that we've cleaned up the already acknowledged state - verify(subtaskStateTrigger, times(1)).discardState(); + subtaskStateMock.verifyDiscard(); TaskStateSnapshot ackSubtaskState = mock(TaskStateSnapshot.class); @@ -4303,4 +4311,88 @@ private void ackCheckpoint( .build()))), "test"); } + + static class OperatorSubtaskStateMock { + OperatorSubtaskState subtaskState; + TestingOperatorStateHandle managedOpHandle; + TestingOperatorStateHandle rawOpHandle; + + OperatorSubtaskStateMock() { + this.managedOpHandle = new TestingOperatorStateHandle(); + this.rawOpHandle = new TestingOperatorStateHandle(); + this.subtaskState = + OperatorSubtaskState.builder() + .setManagedOperatorState(managedOpHandle) + .setRawOperatorState(rawOpHandle) + .build(); + } + + public OperatorSubtaskState getSubtaskState() { + return this.subtaskState; + } + + public void reset() { + managedOpHandle.reset(); + rawOpHandle.reset(); + } + + public void verifyDiscard() { + assert (managedOpHandle.isDiscarded() && rawOpHandle.discarded); + } + + public void verifyNotDiscard() { + assert (!managedOpHandle.isDiscarded() && !rawOpHandle.isDiscarded()); + } + + private static class TestingOperatorStateHandle implements OperatorStateHandle { + + private static final long serialVersionUID = 983594934287613083L; + + boolean discarded; + + @Override + public Map getStateNameToPartitionOffsets() { + return Collections.emptyMap(); + } + + @Override + public FSDataInputStream openInputStream() throws IOException { + throw new IOException("Cannot open input streams in testing implementation."); + } + + @Override + public PhysicalStateHandleID getStreamStateHandleID() { + throw new RuntimeException("Cannot return ID in testing implementation."); + } + + @Override + public Optional asBytesIfInMemory() { + return Optional.empty(); + } + + @Override + public StreamStateHandle getDelegateStateHandle() { + return null; + } + + @Override + public void discardState() throws Exception { + assertThat(discarded).isFalse(); + discarded = true; + } + + @Override + public long getStateSize() { + return 0L; + } + + public void reset() { + discarded = false; + } + + public boolean isDiscarded() { + return discarded; + } + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index ad07d34211772..03d8d5ced590b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -47,12 +47,15 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.InputChannelStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; @@ -74,6 +77,7 @@ import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -88,6 +92,8 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; +import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewInputChannelStateHandle; +import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewResultSubpartitionStateHandle; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.spy; @@ -425,6 +431,55 @@ public static KeyGroupsStateHandle generateKeyGroupState( return new KeyGroupsStateHandle(keyGroupRangeOffsets, allSerializedStatesHandle); } + public static Tuple2, OperatorSubtaskState> + generateSampleOperatorSubtaskState() throws IOException { + JobVertexID jobVertexID = new JobVertexID(); + int index = 0; + Random random = new Random(); + OperatorStateHandle managedOpHandle = + generatePartitionableStateHandle(jobVertexID, index, 2, 8, false); + OperatorStateHandle rawOpHandle = + generatePartitionableStateHandle(jobVertexID, index, 2, 8, true); + KeyedStateHandle managedKeyedHandle = + generateKeyGroupState(jobVertexID, new KeyGroupRange(0, random.nextInt(12)), false); + KeyedStateHandle rawKeyedHandle = + generateKeyGroupState(jobVertexID, new KeyGroupRange(0, random.nextInt(10)), true); + InputChannelStateHandle inputChannelStateHandle = + createNewInputChannelStateHandle(3, random); + ResultSubpartitionStateHandle resultSubpartitionStateHandle = + createNewResultSubpartitionStateHandle(3, random); + OperatorSubtaskState operatorSubtaskState = + OperatorSubtaskState.builder() + .setManagedOperatorState(managedOpHandle) + .setRawOperatorState(rawOpHandle) + .setManagedKeyedState(managedKeyedHandle) + .setRawKeyedState(rawKeyedHandle) + .setInputChannelState( + StateObjectCollection.singleton(inputChannelStateHandle)) + .setResultSubpartitionState( + StateObjectCollection.singleton(resultSubpartitionStateHandle)) + .setInputRescalingDescriptor( + InflightDataRescalingDescriptorUtil.rescalingDescriptor( + new int[1], + new RescaleMappings[0], + Collections.singleton(1))) + .setOutputRescalingDescriptor( + InflightDataRescalingDescriptorUtil.rescalingDescriptor( + new int[1], + new RescaleMappings[0], + Collections.singleton(2))) + .build(); + return new Tuple2<>( + Arrays.asList( + managedOpHandle, + rawOpHandle, + managedKeyedHandle, + rawKeyedHandle, + inputChannelStateHandle, + resultSubpartitionStateHandle), + operatorSubtaskState); + } + public static TaskStateSnapshot createSnapshotWithUnionListState( File stateFile, OperatorID operatorId, boolean isTaskFinished) throws IOException { TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(1, isTaskFinished); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java index 06e6ff4146e75..376f251a3d796 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java @@ -32,6 +32,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -272,7 +273,7 @@ private void verifyCheckpoint(CompletedCheckpoint expected, CompletedCheckpoint /** * A test {@link CompletedCheckpoint}. We want to verify that the correct class loader is used * when discarding. Spying on a regular {@link CompletedCheckpoint} instance with Mockito - * doesn't work, because it it breaks serializability. + * doesn't work, because it breaks serializability. */ protected static class TestCompletedCheckpoint extends CompletedCheckpoint { @@ -327,8 +328,12 @@ public boolean awaitDiscard(long timeout) throws InterruptedException { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } TestCompletedCheckpoint that = (TestCompletedCheckpoint) o; @@ -346,6 +351,15 @@ public class TestCompletedCheckpointDiscardObject extends CompletedCheckpointDis @Override public void discard() throws Exception { super.discard(); + updateDiscards(); + } + + @Override + public CompletableFuture discardAsync(Executor executor) { + return super.discardAsync(executor).thenRun(this::updateDiscards); + } + + private void updateDiscards() { if (!isDiscarded) { isDiscarded = true; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FullyFinishedOperatorStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FullyFinishedOperatorStateTest.java index 68144ef630fb7..3dd2ef9e43614 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FullyFinishedOperatorStateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FullyFinishedOperatorStateTest.java @@ -18,11 +18,20 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.junit.jupiter.api.Test; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; + +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateSampleOperatorSubtaskState; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -49,4 +58,30 @@ void testFullyFinishedOperatorState() { .as("Should not be able to put new subtask states for a fully finished state") .isInstanceOf(UnsupportedOperationException.class); } + + @Test + void testGetDiscardables() throws IOException { + Tuple2, OperatorSubtaskState> opSubtaskStates1 = + generateSampleOperatorSubtaskState(); + Tuple2, OperatorSubtaskState> opSubtaskStates2 = + generateSampleOperatorSubtaskState(); + + OperatorState operatorState = new OperatorState(new OperatorID(), 2, 256); + operatorState.putState(0, opSubtaskStates1.f1); + operatorState.putState(1, opSubtaskStates2.f1); + ByteStreamStateHandle coordinatorState = + new ByteStreamStateHandle("test", new byte[] {1, 2, 3, 4}); + operatorState.setCoordinatorState(coordinatorState); + HashSet discardables = new HashSet<>(); + discardables.addAll(opSubtaskStates1.f0.subList(0, 4)); + discardables.add(((InputChannelStateHandle) opSubtaskStates1.f0.get(4)).getDelegate()); + discardables.add( + ((ResultSubpartitionStateHandle) opSubtaskStates1.f0.get(5)).getDelegate()); + discardables.addAll(opSubtaskStates2.f0.subList(0, 4)); + discardables.add(((InputChannelStateHandle) opSubtaskStates2.f0.get(4)).getDelegate()); + discardables.add( + ((ResultSubpartitionStateHandle) opSubtaskStates2.f0.get(5)).getDelegate()); + discardables.add(coordinatorState); + assertThat(new HashSet<>(operatorState.getDiscardables())).isEqualTo(discardables); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskStateTest.java index 51074a49e7c03..bf746a3497251 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskStateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskStateTest.java @@ -17,28 +17,26 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.state.InputChannelStateHandle; -import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.junit.jupiter.api.Test; import java.io.IOException; -import java.util.Collections; -import java.util.Random; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.apache.commons.lang3.builder.EqualsBuilder.reflectionEquals; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateKeyGroupState; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generatePartitionableStateHandle; -import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewInputChannelStateHandle; -import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewResultSubpartitionStateHandle; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateSampleOperatorSubtaskState; import static org.assertj.core.api.Assertions.assertThat; /** {@link OperatorSubtaskState} test. */ @@ -65,37 +63,7 @@ void testDiscardDuplicatedDelegatesOnce() { @Test void testToBuilderCorrectness() throws IOException { // given: Initialized operator subtask state. - JobVertexID jobVertexID = new JobVertexID(); - int index = 0; - Random random = new Random(); - - OperatorSubtaskState operatorSubtaskState = - OperatorSubtaskState.builder() - .setManagedOperatorState( - generatePartitionableStateHandle(jobVertexID, index, 2, 8, false)) - .setRawOperatorState( - generatePartitionableStateHandle(jobVertexID, index, 2, 8, true)) - .setManagedKeyedState( - generateKeyGroupState(jobVertexID, new KeyGroupRange(0, 11), false)) - .setRawKeyedState( - generateKeyGroupState(jobVertexID, new KeyGroupRange(0, 9), true)) - .setInputChannelState( - StateObjectCollection.singleton( - createNewInputChannelStateHandle(3, random))) - .setResultSubpartitionState( - StateObjectCollection.singleton( - createNewResultSubpartitionStateHandle(3, random))) - .setInputRescalingDescriptor( - InflightDataRescalingDescriptorUtil.rescalingDescriptor( - new int[1], - new RescaleMappings[0], - Collections.singleton(1))) - .setOutputRescalingDescriptor( - InflightDataRescalingDescriptorUtil.rescalingDescriptor( - new int[1], - new RescaleMappings[0], - Collections.singleton(2))) - .build(); + OperatorSubtaskState operatorSubtaskState = generateSampleOperatorSubtaskState().f1; // when: Copy the operator subtask state. OperatorSubtaskState operatorSubtaskStateCopy = operatorSubtaskState.toBuilder().build(); @@ -104,6 +72,24 @@ void testToBuilderCorrectness() throws IOException { assertThat(reflectionEquals(operatorSubtaskState, operatorSubtaskStateCopy)).isTrue(); } + @Test + void testGetDiscardables() throws IOException { + Tuple2, OperatorSubtaskState> opStates = + generateSampleOperatorSubtaskState(); + List states = opStates.f0; + OperatorSubtaskState operatorSubtaskState = opStates.f1; + List discardables = + Arrays.asList( + states.get(0), + states.get(1), + states.get(2), + states.get(3), + ((InputChannelStateHandle) states.get(4)).getDelegate(), + ((ResultSubpartitionStateHandle) states.get(5)).getDelegate()); + assertThat(new HashSet<>(operatorSubtaskState.getDiscardables())) + .isEqualTo(new HashSet<>(discardables)); + } + private ResultSubpartitionStateHandle buildSubpartitionHandle( StreamStateHandle delegate, int subPartitionIdx1) { return new ResultSubpartitionStateHandle( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 9d3e14bccc3c5..e530edef9b15d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -24,6 +24,7 @@ import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.OperatorIDPair; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest.OperatorSubtaskStateMock; import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.StringSerializer; import org.apache.flink.runtime.checkpoint.PendingCheckpoint.TaskAcknowledgeResult; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; @@ -35,7 +36,6 @@ import org.apache.flink.runtime.operators.coordination.OperatorInfo; import org.apache.flink.runtime.operators.coordination.TestingOperatorInfo; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; -import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.TestingStreamStateHandle; import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; @@ -44,7 +44,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import org.mockito.Mockito; import javax.annotation.Nullable; @@ -66,11 +65,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -230,8 +225,10 @@ void testAbortDiscardsState() throws Exception { false, CheckpointType.CHECKPOINT, false, false, false, false, false, false); QueueExecutor executor = new QueueExecutor(); - OperatorState state = mock(OperatorState.class); - doNothing().when(state).registerSharedStates(any(SharedStateRegistry.class), eq(0L)); + OperatorState state = new OperatorState(new OperatorID(), 1, 256); + OperatorSubtaskStateMock subtaskStateMock = new OperatorSubtaskStateMock(); + OperatorSubtaskState subtaskState = subtaskStateMock.getSubtaskState(); + state.putState(0, subtaskState); // Abort declined PendingCheckpoint pending = createPendingCheckpoint(props, executor); @@ -240,10 +237,10 @@ void testAbortDiscardsState() throws Exception { abort(pending, CheckpointFailureReason.CHECKPOINT_DECLINED); // execute asynchronous discard operation executor.runQueuedCommands(); - verify(state, times(1)).discardState(); + subtaskStateMock.verifyDiscard(); // Abort error - Mockito.reset(state); + subtaskStateMock.reset(); pending = createPendingCheckpoint(props, executor); setTaskState(pending, state); @@ -251,10 +248,10 @@ void testAbortDiscardsState() throws Exception { abort(pending, CheckpointFailureReason.CHECKPOINT_DECLINED); // execute asynchronous discard operation executor.runQueuedCommands(); - verify(state, times(1)).discardState(); + subtaskStateMock.verifyDiscard(); // Abort expired - Mockito.reset(state); + subtaskStateMock.reset(); pending = createPendingCheckpoint(props, executor); setTaskState(pending, state); @@ -262,10 +259,10 @@ void testAbortDiscardsState() throws Exception { abort(pending, CheckpointFailureReason.CHECKPOINT_EXPIRED); // execute asynchronous discard operation executor.runQueuedCommands(); - verify(state, times(1)).discardState(); + subtaskStateMock.verifyDiscard(); // Abort subsumed - Mockito.reset(state); + subtaskStateMock.reset(); pending = createPendingCheckpoint(props, executor); setTaskState(pending, state); @@ -273,7 +270,7 @@ void testAbortDiscardsState() throws Exception { abort(pending, CheckpointFailureReason.CHECKPOINT_SUBSUMED); // execute asynchronous discard operation executor.runQueuedCommands(); - verify(state, times(1)).discardState(); + subtaskStateMock.verifyDiscard(); } /** diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java index 86da4b2a66fd1..747fbae4fcac6 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.util; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.configuration.StateChangelogOptions; @@ -118,6 +119,7 @@ private static void randomizeConfiguration(MiniCluster miniCluster, Configuratio Duration.ofSeconds(0), Duration.ofMillis(100), Duration.ofSeconds(2)); + randomize(conf, CheckpointingOptions.CLEANER_PARALLEL_MODE, true, false); } // randomize ITTests for enabling state change log From 3ff225c5f993282d6dfc7726fc08cc00058d9a7f Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 30 Jun 2023 18:29:55 +0200 Subject: [PATCH 027/104] [FLINK-21883][scheduler] Implement cooldown period for adaptive scheduler --- .../docs/deployment/elastic_scaling.md | 5 + .../generated/all_jobmanager_section.html | 12 ++ .../generated/expert_scheduling_section.html | 12 ++ .../generated/job_manager_configuration.html | 12 ++ .../configuration/JobManagerOptions.java | 26 +++ .../scheduler/adaptive/AdaptiveScheduler.java | 21 ++- .../runtime/scheduler/adaptive/Executing.java | 101 +++++++++- ...rceMinimalIncreaseRescalingController.java | 4 +- ...eParallelismChangeRescalingController.java | 41 +++++ .../scheduler/adaptive/ExecutingTest.java | 173 ++++++++++++++++-- ...allelismChangeRescalingControllerTest.java | 61 ++++++ 11 files changed, 434 insertions(+), 34 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java diff --git a/docs/content/docs/deployment/elastic_scaling.md b/docs/content/docs/deployment/elastic_scaling.md index bb38d7549bbde..f342f05ffb9c3 100644 --- a/docs/content/docs/deployment/elastic_scaling.md +++ b/docs/content/docs/deployment/elastic_scaling.md @@ -174,6 +174,11 @@ With Reactive Mode enabled, the [`jobmanager.adaptive-scheduler.resource-stabili In scenarios where TaskManagers are not connecting at the same time, but slowly one after another, this behavior leads to a job restart whenever a TaskManager connects. Increase this configuration value if you want to wait for the resources to stabilize before scheduling the job. Additionally, one can configure [`jobmanager.adaptive-scheduler.min-parallelism-increase`]({{< ref "docs/deployment/config">}}#jobmanager-adaptive-scheduler-min-parallelism-increase): This configuration option specifics the minimum amount of additional, aggregate parallelism increase before triggering a scale-up. For example if you have a job with a source (parallelism=2) and a sink (parallelism=2), the aggregate parallelism is 4. By default, the configuration key is set to 1, so any increase in the aggregate parallelism will trigger a restart. +One can force scaling operations to happen by setting [`jobmanager.adaptive-scheduler.scaling-interval.max`]({{< ref "docs/deployment/config">}}#jobmanager-adaptive-scheduler-scaling-interval-max). It is disabled by default. If set, then when new resources are added to the cluster, a rescale is scheduled after [`jobmanager.adaptive-scheduler.scaling-interval.max`]({{< ref "docs/deployment/config">}}#jobmanager-adaptive-scheduler-scaling-interval-max) even if [`jobmanager.adaptive-scheduler.min-parallelism-increase`]({{< ref "docs/deployment/config">}}#jobmanager-adaptive-scheduler-min-parallelism-increase) is not satisfied. + +To avoid too frequent scaling operations, one can configure [`jobmanager.adaptive-scheduler.scaling-interval.min`]({{< ref "docs/deployment/config">}}#jobmanager-adaptive-scheduler-scaling-interval-min) to set the minimum time between 2 scaling operations. The default is 30s. + + #### Recommendations - **Configure periodic checkpointing for stateful jobs**: Reactive mode restores from the latest completed checkpoint on a rescale event. If no periodic checkpointing is enabled, your program will lose its state. Checkpointing also configures a **restart strategy**. Reactive Mode will respect the configured restarting strategy: If no restarting strategy is configured, reactive mode will fail your job, instead of scaling it. diff --git a/docs/layouts/shortcodes/generated/all_jobmanager_section.html b/docs/layouts/shortcodes/generated/all_jobmanager_section.html index 5bcb175141750..79b0e7f804b66 100644 --- a/docs/layouts/shortcodes/generated/all_jobmanager_section.html +++ b/docs/layouts/shortcodes/generated/all_jobmanager_section.html @@ -26,6 +26,18 @@ + + + + + + + + + + + + diff --git a/docs/layouts/shortcodes/generated/expert_scheduling_section.html b/docs/layouts/shortcodes/generated/expert_scheduling_section.html index b5a3163a32198..2c07a95372fd7 100644 --- a/docs/layouts/shortcodes/generated/expert_scheduling_section.html +++ b/docs/layouts/shortcodes/generated/expert_scheduling_section.html @@ -86,6 +86,18 @@ + + + + + + + + + + + + diff --git a/docs/layouts/shortcodes/generated/job_manager_configuration.html b/docs/layouts/shortcodes/generated/job_manager_configuration.html index 711d09020b7ed..d07705d7018c2 100644 --- a/docs/layouts/shortcodes/generated/job_manager_configuration.html +++ b/docs/layouts/shortcodes/generated/job_manager_configuration.html @@ -26,6 +26,18 @@ + + + + + + + + + + + + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 1da99be996670..0378e380923f2 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -507,6 +507,32 @@ public enum SchedulerType { .withDescription( "Configure the minimum increase in parallelism for a job to scale up."); + @Documentation.Section({ + Documentation.Sections.EXPERT_SCHEDULING, + Documentation.Sections.ALL_JOB_MANAGER + }) + public static final ConfigOption SCHEDULER_SCALING_INTERVAL_MIN = + key("jobmanager.adaptive-scheduler.scaling-interval.min") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + // rescaling and let the user increase the value for high workloads + .withDescription("Determines the minimum time between scaling operations."); + + @Documentation.Section({ + Documentation.Sections.EXPERT_SCHEDULING, + Documentation.Sections.ALL_JOB_MANAGER + }) + public static final ConfigOption SCHEDULER_SCALING_INTERVAL_MAX = + key("jobmanager.adaptive-scheduler.scaling-interval.max") + .durationType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Determines the maximum interval time after which a scaling operation is forced even if the %s aren't met. The scaling operation will be ignored when the resource hasn't changed. This option is disabled by default.", + code(MIN_PARALLELISM_INCREASE.key())) + .build()); + @Documentation.Section({ Documentation.Sections.EXPERT_SCHEDULING, Documentation.Sections.ALL_JOB_MANAGER diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index 4ee22c9584840..67615ea79cc29 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -108,6 +108,7 @@ import org.apache.flink.runtime.scheduler.adaptive.allocator.SlotAllocator; import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism; import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.EnforceMinimalIncreaseRescalingController; +import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.EnforceParallelismChangeRescalingController; import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.RescalingController; import org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntry; import org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry; @@ -204,6 +205,8 @@ public class AdaptiveScheduler private final RescalingController rescalingController; + private final RescalingController forceRescalingController; + private final Duration initialResourceAllocationTimeout; private final Duration resourceStabilizationTimeout; @@ -294,6 +297,8 @@ public AdaptiveScheduler( this.rescalingController = new EnforceMinimalIncreaseRescalingController(configuration); + this.forceRescalingController = new EnforceParallelismChangeRescalingController(); + this.initialResourceAllocationTimeout = initialResourceAllocationTimeout; this.resourceStabilizationTimeout = resourceStabilizationTimeout; @@ -1162,16 +1167,24 @@ private ExecutionGraph createExecutionGraphAndRestoreState( LOG); } + /** + * In regular mode, rescale the job if added resource meets {@link + * JobManagerOptions#MIN_PARALLELISM_INCREASE}. In force mode rescale if the parallelism has + * changed. + */ @Override - public boolean shouldRescale(ExecutionGraph executionGraph) { + public boolean shouldRescale(ExecutionGraph executionGraph, boolean forceRescale) { final Optional maybeNewParallelism = slotAllocator.determineParallelism( jobInformation, declarativeSlotPool.getAllSlotsInformation()); return maybeNewParallelism .filter( - vertexParallelism -> - rescalingController.shouldRescale( - getCurrentParallelism(executionGraph), vertexParallelism)) + vertexParallelism -> { + RescalingController rescalingControllerToUse = + forceRescale ? forceRescalingController : rescalingController; + return rescalingControllerToUse.shouldRescale( + getCurrentParallelism(executionGraph), vertexParallelism); + }) .isPresent(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java index 9cdb9afc7f286..10f872bbbd578 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.scheduler.adaptive; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.checkpoint.CheckpointScheduling; @@ -39,6 +41,7 @@ import javax.annotation.Nullable; import java.time.Duration; +import java.time.Instant; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledFuture; @@ -47,6 +50,11 @@ class Executing extends StateWithExecutionGraph implements ResourceListener { private final Context context; + private final Instant lastRescale; + // only one schedule at the time + private boolean rescaleScheduled = false; + private final Duration scalingIntervalMin; + @Nullable private final Duration scalingIntervalMax; Executing( ExecutionGraph executionGraph, @@ -55,7 +63,10 @@ class Executing extends StateWithExecutionGraph implements ResourceListener { Logger logger, Context context, ClassLoader userCodeClassLoader, - List failureCollection) { + List failureCollection, + Duration scalingIntervalMin, + @Nullable Duration scalingIntervalMax, + Instant lastRescale) { super( context, executionGraph, @@ -67,11 +78,29 @@ class Executing extends StateWithExecutionGraph implements ResourceListener { this.context = context; Preconditions.checkState( executionGraph.getState() == JobStatus.RUNNING, "Assuming running execution graph"); + this.scalingIntervalMin = scalingIntervalMin; + this.scalingIntervalMax = scalingIntervalMax; + // Executing is recreated with each restart (when we rescale) + // we consider the first execution of the pipeline as a rescale event + this.lastRescale = lastRescale; + Preconditions.checkState( + !scalingIntervalMin.isNegative(), + "%s must be positive integer or 0", + JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key()); + if (scalingIntervalMax != null) { + Preconditions.checkState( + scalingIntervalMax.compareTo(scalingIntervalMin) > 0, + "%s(%d) must be greater than %s(%d)", + JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(), + scalingIntervalMax, + JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key(), + scalingIntervalMin); + } deploy(); // check if new resources have come available in the meantime - context.runIfState(this, this::maybeRescale, Duration.ZERO); + rescaleWhenCooldownPeriodIsOver(); } @Override @@ -124,23 +153,74 @@ private void handleDeploymentFailure(ExecutionVertex executionVertex, JobExcepti @Override public void onNewResourcesAvailable() { - maybeRescale(); + rescaleWhenCooldownPeriodIsOver(); } @Override public void onNewResourceRequirements() { - maybeRescale(); + rescaleWhenCooldownPeriodIsOver(); } + /** Force rescaling as long as the target parallelism is different from the current one. */ + private void forceRescale() { + if (context.shouldRescale(getExecutionGraph(), true)) { + getLogger() + .info( + "Added resources are still there after {} time({}), force a rescale.", + JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(), + scalingIntervalMax); + context.goToRestarting( + getExecutionGraph(), + getExecutionGraphHandler(), + getOperatorCoordinatorHandler(), + Duration.ofMillis(0L), + getFailures()); + } + } + + /** + * Rescale the job if {@link Context#shouldRescale} is true. Otherwise, force a rescale using + * {@link Executing#forceRescale()} after {@link + * JobManagerOptions#SCHEDULER_SCALING_INTERVAL_MAX}. + */ private void maybeRescale() { - if (context.shouldRescale(getExecutionGraph())) { - getLogger().info("Can change the parallelism of job. Restarting job."); + rescaleScheduled = false; + if (context.shouldRescale(getExecutionGraph(), false)) { + getLogger().info("Can change the parallelism of the job. Restarting the job."); context.goToRestarting( getExecutionGraph(), getExecutionGraphHandler(), getOperatorCoordinatorHandler(), Duration.ofMillis(0L), getFailures()); + } else if (scalingIntervalMax != null) { + getLogger() + .info( + "The longer the pipeline runs, the more the (small) resource gain is worth the restarting time. " + + "Last resource added does not meet {}, force a rescale after {} time({}) if the resource is still there.", + JobManagerOptions.MIN_PARALLELISM_INCREASE, + JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(), + scalingIntervalMax); + if (timeSinceLastRescale().compareTo(scalingIntervalMax) > 0) { + forceRescale(); + } else { + // schedule a force rescale in JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX time + context.runIfState(this, this::forceRescale, scalingIntervalMax); + } + } + } + + private Duration timeSinceLastRescale() { + return Duration.between(lastRescale, Instant.now()); + } + + private void rescaleWhenCooldownPeriodIsOver() { + if (timeSinceLastRescale().compareTo(scalingIntervalMin) > 0) { + maybeRescale(); + } else if (!rescaleScheduled) { + rescaleScheduled = true; + // schedule maybeRescale resetting the cooldown period + context.runIfState(this, this::maybeRescale, scalingIntervalMin); } } @@ -196,9 +276,10 @@ interface Context * Asks if we should rescale the currently executing job. * * @param executionGraph executionGraph for making the scaling decision. + * @param forceRescale should we force rescaling * @return true, if we should rescale */ - boolean shouldRescale(ExecutionGraph executionGraph); + boolean shouldRescale(ExecutionGraph executionGraph, boolean forceRescale); /** * Runs the given action after a delay if the state at this time equals the expected state. @@ -244,6 +325,7 @@ public Class getStateClass() { } public Executing getState() { + final Configuration jobConfiguration = executionGraph.getJobConfiguration(); return new Executing( executionGraph, executionGraphHandler, @@ -251,7 +333,10 @@ public Executing getState() { log, context, userCodeClassLoader, - failureCollection); + failureCollection, + jobConfiguration.get(JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN), + jobConfiguration.get(JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX), + Instant.now()); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java index b02907acb7fd6..edf82d9a42fbd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java @@ -24,8 +24,8 @@ import static org.apache.flink.configuration.JobManagerOptions.MIN_PARALLELISM_INCREASE; /** - * Simple scaling policy for a reactive mode. The user can configure a minimum cumulative - * parallelism increase to allow a scale up. + * Simple scaling policy. The user can configure a minimum cumulative parallelism increase to allow + * a scale up. */ public class EnforceMinimalIncreaseRescalingController implements RescalingController { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java new file mode 100644 index 0000000000000..94c11cf0250b3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.scheduler.adaptive.scalingpolicy; + +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism; + +/** + * Simple scaling policy. It just checks that the new parallelism is different (either increase or + * decrease) from the current parallelism. + */ +public class EnforceParallelismChangeRescalingController implements RescalingController { + @Override + public boolean shouldRescale( + VertexParallelism currentParallelism, VertexParallelism newParallelism) { + for (JobVertexID vertex : currentParallelism.getVertices()) { + int parallelismChange = + newParallelism.getParallelism(vertex) + - currentParallelism.getParallelism(vertex); + if (parallelismChange != 0) { + return true; + } + } + return false; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java index f89cd0bb54a6c..73ab5c7922ce0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java @@ -82,6 +82,7 @@ import javax.annotation.Nullable; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -146,7 +147,10 @@ public void testNoDeploymentCallOnEnterWhenVertexRunning() throws Exception { log, ctx, ClassLoader.getSystemClassLoader(), - new ArrayList<>()); + new ArrayList<>(), + Duration.ZERO, + null, + Instant.now()); assertThat(mockExecutionVertex.isDeployCalled(), is(false)); } } @@ -164,7 +168,10 @@ public void testIllegalStateExceptionOnNotRunningExecutionGraph() throws Excepti log, ctx, ClassLoader.getSystemClassLoader(), - new ArrayList<>()); + new ArrayList<>(), + Duration.ZERO, + null, + Instant.now()); } } @@ -252,28 +259,121 @@ public void testTransitionToFinishedOnSuspend() throws Exception { } @Test - public void testNotifyNewResourcesAvailableWithCanScaleUpTransitionsToRestarting() + public void testNotifyNewResourcesAvailableBeforeCooldownIsOverScheduledStateChange() throws Exception { try (MockExecutingContext ctx = new MockExecutingContext()) { - Executing exec = new ExecutingStateBuilder().build(ctx); + // do not wait too long in the test + final Duration scalingIntervalMin = Duration.ofSeconds(1L); + final ExecutingStateBuilder executingStateBuilder = + new ExecutingStateBuilder().setScalingIntervalMin(scalingIntervalMin); + Executing exec = executingStateBuilder.build(ctx); + // => rescale + ctx.setCanScaleUp(true); + // scheduled rescale should restart the job after cooldown + ctx.setExpectRestarting( + restartingArguments -> { + assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(true)); + }); + exec.onNewResourcesAvailable(); + } + } + @Test + public void testNotifyNewResourcesAvailableAfterCooldownIsOverStateChange() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + final ExecutingStateBuilder executingStateBuilder = + new ExecutingStateBuilder() + .setScalingIntervalMin(Duration.ofSeconds(20L)) + .setLastRescale(Instant.now().minus(Duration.ofSeconds(30L))); + Executing exec = executingStateBuilder.build(ctx); + // => rescale + ctx.setCanScaleUp(true); + // immediate rescale ctx.setExpectRestarting( restartingArguments -> { - // expect immediate restart on scale up assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(false)); }); - ctx.setCanScaleUp(() -> true); exec.onNewResourcesAvailable(); } } @Test - public void testNotifyNewResourcesAvailableWithNoResourcesAndNoStateChange() throws Exception { + public void testNotifyNewResourcesAvailableWithCanScaleUpWithoutForceTransitionsToRestarting() + throws Exception { try (MockExecutingContext ctx = new MockExecutingContext()) { Executing exec = new ExecutingStateBuilder().build(ctx); - ctx.setCanScaleUp(() -> false); + + ctx.setExpectRestarting( + // immediate rescale + restartingArguments -> { + assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(false)); + }); + ctx.setCanScaleUp(true); // => rescale + exec.onNewResourcesAvailable(); + } + } + + @Test + public void testNotifyNewResourcesAvailableWithCantScaleUpWithoutForceAndCantScaleUpWithForce() + throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + Executing exec = + new ExecutingStateBuilder() + .setScalingIntervalMax(Duration.ofSeconds(1L)) + .build(ctx); + // => schedule force rescale but resource lost on timeout => no rescale + ctx.setCanScaleUp(false, false); exec.onNewResourcesAvailable(); ctx.assertNoStateTransition(); + assertThat(ctx.actionWasScheduled, is(true)); + } + } + + @Test + public void + testNotifyNewResourcesAvailableWithCantScaleUpWithoutForceAndCanScaleUpWithForceScheduled() + throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + final ExecutingStateBuilder executingStateBuilder = + new ExecutingStateBuilder() + .setScalingIntervalMin(Duration.ofSeconds(20L)) + .setScalingIntervalMax(Duration.ofSeconds(30L)) + .setLastRescale(Instant.now().minus(Duration.ofSeconds(25L))); + Executing exec = executingStateBuilder.build(ctx); + // => schedule force rescale and resource still there after timeout => rescale + ctx.setCanScaleUp(false, true); + // rescale after scaling-interval.max + ctx.setExpectRestarting( + restartingArguments -> { + assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(true)); + }); + exec.onNewResourcesAvailable(); + } + } + + @Test + public void + testNotifyNewResourcesAvailableWithCantScaleUpWithoutForceAndCanScaleUpWithForceImmediate() + throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + final ExecutingStateBuilder executingStateBuilder = + new ExecutingStateBuilder() + .setScalingIntervalMin(Duration.ofSeconds(20L)) + .setScalingIntervalMax(Duration.ofSeconds(30L)) + .setLastRescale(Instant.now().minus(Duration.ofSeconds(70L))); + Executing exec = executingStateBuilder.build(ctx); + // => immediate force rescale and resource still there after timeout => rescale + ctx.setCanScaleUp(false, true); + ctx.setExpectRestarting( + restartingArguments -> { + assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(false)); + }); + exec.onNewResourcesAvailable(); } } @@ -468,15 +568,15 @@ public void testStateDoesNotExposeGloballyTerminalExecutionGraph() throws Except @Test public void testExecutingChecksForNewResourcesWhenBeingCreated() throws Exception { - try (MockExecutingContext context = new MockExecutingContext()) { - context.setCanScaleUp(() -> true); - context.setExpectRestarting( - restartingArguments -> { - // expect immediate restart on scale up + try (MockExecutingContext ctx = new MockExecutingContext()) { + ctx.setCanScaleUp(true); + ctx.setExpectRestarting( + restartingArguments -> { // immediate rescale assertThat(restartingArguments.getBackoffTime(), is(Duration.ZERO)); + assertThat(ctx.actionWasScheduled, is(false)); }); - final Executing executing = new ExecutingStateBuilder().build(context); + new ExecutingStateBuilder().build(ctx); } } @@ -497,6 +597,9 @@ private final class ExecutingStateBuilder { TestingDefaultExecutionGraphBuilder.newBuilder() .build(EXECUTOR_RESOURCE.getExecutor()); private OperatorCoordinatorHandler operatorCoordinatorHandler; + private Duration scalingIntervalMin = Duration.ZERO; + @Nullable private Duration scalingIntervalMax; + private Instant lastRescale = Instant.now(); private ExecutingStateBuilder() throws JobException, JobExecutionException { operatorCoordinatorHandler = new TestingOperatorCoordinatorHandler(); @@ -513,6 +616,21 @@ public ExecutingStateBuilder setOperatorCoordinatorHandler( return this; } + public ExecutingStateBuilder setScalingIntervalMin(Duration scalingIntervalMin) { + this.scalingIntervalMin = scalingIntervalMin; + return this; + } + + public ExecutingStateBuilder setScalingIntervalMax(Duration scalingIntervalMax) { + this.scalingIntervalMax = scalingIntervalMax; + return this; + } + + public ExecutingStateBuilder setLastRescale(Instant lastRescale) { + this.lastRescale = lastRescale; + return this; + } + private Executing build(MockExecutingContext ctx) { executionGraph.transitionToRunning(); @@ -523,7 +641,10 @@ private Executing build(MockExecutingContext ctx) { log, ctx, ClassLoader.getSystemClassLoader(), - new ArrayList<>()); + new ArrayList<>(), + scalingIntervalMin, + scalingIntervalMax, + lastRescale); } } @@ -544,7 +665,9 @@ private static class MockExecutingContext extends MockStateWithExecutionGraphCon new StateValidator<>("cancelling"); private Function howToHandleFailure; - private Supplier canScaleUp = () -> false; + private boolean canScaleUpWithoutForce = false; + private boolean canScaleUpWithForce = false; + private boolean actionWasScheduled = false; private StateValidator stopWithSavepointValidator = new StateValidator<>("stopWithSavepoint"); private CompletableFuture mockedStopWithSavepointOperationFuture = @@ -570,8 +693,13 @@ public void setHowToHandleFailure(Function function) { this.howToHandleFailure = function; } - public void setCanScaleUp(Supplier supplier) { - this.canScaleUp = supplier; + public void setCanScaleUp(boolean canScaleUpWithoutForce, boolean canScaleUpWithForce) { + this.canScaleUpWithoutForce = canScaleUpWithoutForce; + this.canScaleUpWithForce = canScaleUpWithForce; + } + + public void setCanScaleUp(boolean canScaleUpWithoutForce) { + this.canScaleUpWithoutForce = canScaleUpWithoutForce; } // --------- Interface Implementations ------- // @@ -594,8 +722,12 @@ public FailureResult howToHandleFailure(Throwable failure) { } @Override - public boolean shouldRescale(ExecutionGraph executionGraph) { - return canScaleUp.get(); + public boolean shouldRescale(ExecutionGraph executionGraph, boolean forceRescale) { + if (forceRescale) { + return canScaleUpWithForce; + } else { + return canScaleUpWithoutForce; + } } @Override @@ -651,6 +783,7 @@ public CompletableFuture goToStopWithSavepoint( @Override public ScheduledFuture runIfState(State expectedState, Runnable action, Duration delay) { + actionWasScheduled = !delay.isZero(); if (!hadStateTransition) { action.run(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java new file mode 100644 index 0000000000000..0d980702af3bd --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.scheduler.adaptive.scalingpolicy; + +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the {@link RescalingController}. */ +class EnforceParallelismChangeRescalingControllerTest { + + private static final JobVertexID jobVertexId = new JobVertexID(); + + @Test + void testScaleUp() { + final RescalingController rescalingController = + new EnforceParallelismChangeRescalingController(); + assertThat(rescalingController.shouldRescale(forParallelism(1), forParallelism(2))) + .isTrue(); + } + + @Test + void testAlwaysScaleDown() { + final RescalingController rescalingController = + new EnforceParallelismChangeRescalingController(); + assertThat(rescalingController.shouldRescale(forParallelism(2), forParallelism(1))) + .isTrue(); + } + + @Test + void testNoScaleOnSameParallelism() { + final RescalingController rescalingController = + new EnforceParallelismChangeRescalingController(); + assertThat(rescalingController.shouldRescale(forParallelism(2), forParallelism(2))) + .isFalse(); + } + + private static VertexParallelism forParallelism(int parallelism) { + return new VertexParallelism(Collections.singletonMap(jobVertexId, parallelism)); + } +} From 0ccd95ef48bcd7246f8c88c9aa7b69ffa268c865 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Thu, 26 Oct 2023 06:02:52 -0500 Subject: [PATCH 028/104] [FLINK-32850][flink-runtime][JUnit5 Migration] The io.disk package of flink-runtime module (#23572) --- .../disk/BatchShuffleReadBufferPoolTest.java | 210 ++++--- .../runtime/io/disk/ChannelViewsTest.java | 136 ++--- .../io/disk/FileChannelManagerImplTest.java | 127 +++-- .../io/disk/FileChannelStreamsITCase.java | 516 ++++++++---------- .../io/disk/FileChannelStreamsTest.java | 24 +- .../SeekableFileChannelInputViewTest.java | 98 ++-- .../runtime/io/disk/SpillingBufferTest.java | 151 ++--- .../AsynchronousBufferFileWriterTest.java | 161 +++--- .../AsynchronousFileIOChannelTest.java | 282 +++++----- ...BufferFileWriterFileSegmentReaderTest.java | 50 +- .../iomanager/BufferFileWriterReaderTest.java | 48 +- .../io/disk/iomanager/IOManagerAsyncTest.java | 437 +++++++-------- .../io/disk/iomanager/IOManagerITCase.java | 73 +-- .../io/disk/iomanager/IOManagerTest.java | 46 +- 14 files changed, 1073 insertions(+), 1286 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java index 116fb5473bc4b..5e64751855a02 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java @@ -18,71 +18,70 @@ package org.apache.flink.runtime.io.disk; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.testutils.CheckedThread; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link BatchShuffleReadBufferPool}. */ -public class BatchShuffleReadBufferPoolTest { +@Timeout(value = 60) +class BatchShuffleReadBufferPoolTest { - @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS); - - @Test(expected = IllegalArgumentException.class) - public void testIllegalTotalBytes() { - createBufferPool(0, 1024); + @Test + void testIllegalTotalBytes() { + assertThatThrownBy(() -> createBufferPool(0, 1024)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testIllegalBufferSize() { - createBufferPool(32 * 1024 * 1024, 0); + @Test + void testIllegalBufferSize() { + assertThatThrownBy(() -> createBufferPool(32 * 1024 * 1024, 0)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testLargeTotalBytes() { + void testLargeTotalBytes() { BatchShuffleReadBufferPool bufferPool = createBufferPool(Long.MAX_VALUE, 1024); - assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers()); + assertThat(bufferPool.getNumTotalBuffers()).isEqualTo(Integer.MAX_VALUE); bufferPool.destroy(); } - @Test(expected = IllegalArgumentException.class) - public void testTotalBytesSmallerThanBufferSize() { - createBufferPool(4096, 32 * 1024); + @Test + void testTotalBytesSmallerThanBufferSize() { + assertThatThrownBy(() -> createBufferPool(4096, 32 * 1024)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testBufferCalculation() { + void testBufferCalculation() { long totalBytes = 32 * 1024 * 1024; for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize += 1024) { BatchShuffleReadBufferPool bufferPool = createBufferPool(totalBytes, bufferSize); - assertEquals(totalBytes, bufferPool.getTotalBytes()); - assertEquals(totalBytes / bufferSize, bufferPool.getNumTotalBuffers()); - assertTrue(bufferPool.getNumBuffersPerRequest() <= bufferPool.getNumTotalBuffers()); - assertTrue(bufferPool.getNumBuffersPerRequest() > 0); + assertThat(bufferPool.getTotalBytes()).isEqualTo(totalBytes); + assertThat(bufferPool.getNumTotalBuffers()).isEqualTo(totalBytes / bufferSize); + assertThat(bufferPool.getNumBuffersPerRequest()) + .isLessThanOrEqualTo(bufferPool.getNumTotalBuffers()); + assertThat(bufferPool.getNumBuffersPerRequest()).isGreaterThan(0); } } @Test - public void testRequestBuffers() throws Exception { + void testRequestBuffers() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = new ArrayList<>(); try { buffers.addAll(bufferPool.requestBuffers()); - assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size()); + assertThat(buffers).hasSize(bufferPool.getNumBuffersPerRequest()); } finally { bufferPool.recycle(buffers); bufferPool.destroy(); @@ -90,46 +89,45 @@ public void testRequestBuffers() throws Exception { } @Test - public void testRecycle() throws Exception { + void testRecycle() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.recycle(buffers); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); } @Test - public void testBufferOperationTimestampUpdated() throws Exception { + void testBufferOperationTimestampUpdated() throws Exception { BatchShuffleReadBufferPool bufferPool = new BatchShuffleReadBufferPool(1024, 1024); long oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); List buffers = bufferPool.requestBuffers(); - assertEquals(1, buffers.size()); + assertThat(buffers).hasSize(1); // The timestamp is updated when requesting buffers successfully - assertTrue(bufferPool.getLastBufferOperationTimestamp() > oldTimestamp); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isGreaterThan(oldTimestamp); oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); bufferPool.recycle(buffers); // The timestamp is updated when recycling buffers - assertTrue(bufferPool.getLastBufferOperationTimestamp() > oldTimestamp); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isGreaterThan(oldTimestamp); buffers = bufferPool.requestBuffers(); oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); - assertEquals(0, bufferPool.requestBuffers().size()); + assertThat(bufferPool.requestBuffers()).isEmpty(); // The timestamp is not updated when requesting buffers is failed - assertEquals(oldTimestamp, bufferPool.getLastBufferOperationTimestamp()); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isEqualTo(oldTimestamp); bufferPool.recycle(buffers); bufferPool.destroy(); } @Test - public void testBufferFulfilledByRecycledBuffers() throws Exception { + void testBufferFulfilledByRecycledBuffers() throws Exception { int numRequestThreads = 2; - AtomicReference exception = new AtomicReference<>(); BatchShuffleReadBufferPool bufferPool = createBufferPool(); Map> buffers = new ConcurrentHashMap<>(); @@ -139,24 +137,22 @@ public void testBufferFulfilledByRecycledBuffers() throws Exception { owners[i] = new Object(); buffers.put(owners[i], bufferPool.requestBuffers()); } - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isZero(); - Thread[] requestThreads = new Thread[numRequestThreads]; + CheckedThread[] requestThreads = new CheckedThread[numRequestThreads]; for (int i = 0; i < numRequestThreads; ++i) { requestThreads[i] = - new Thread( - () -> { - try { - Object owner = new Object(); - List allocated = null; - while (allocated == null || allocated.isEmpty()) { - allocated = bufferPool.requestBuffers(); - } - buffers.put(owner, allocated); - } catch (Throwable throwable) { - exception.set(throwable); - } - }); + new CheckedThread() { + @Override + public void go() throws Exception { + Object owner = new Object(); + List allocated = null; + while (allocated == null || allocated.isEmpty()) { + allocated = bufferPool.requestBuffers(); + } + buffers.put(owner, allocated); + } + }; requestThreads[i].start(); } @@ -168,125 +164,115 @@ public void testBufferFulfilledByRecycledBuffers() throws Exception { // bulk recycle bufferPool.recycle(buffers.remove(owners[1])); - for (Thread requestThread : requestThreads) { - requestThread.join(); + for (CheckedThread requestThread : requestThreads) { + requestThread.sync(); } - assertNull(exception.get()); - assertEquals(0, bufferPool.getAvailableBuffers()); - assertEquals(8, buffers.size()); + assertThat(bufferPool.getAvailableBuffers()).isZero(); + assertThat(buffers).hasSize(8); } finally { for (Object owner : buffers.keySet()) { bufferPool.recycle(buffers.remove(owner)); } - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); bufferPool.destroy(); } } @Test - public void testMultipleThreadRequestAndRecycle() throws Exception { + void testMultipleThreadRequestAndRecycle() throws Exception { int numRequestThreads = 10; - AtomicReference exception = new AtomicReference<>(); BatchShuffleReadBufferPool bufferPool = createBufferPool(); try { - Thread[] requestThreads = new Thread[numRequestThreads]; + CheckedThread[] requestThreads = new CheckedThread[numRequestThreads]; for (int i = 0; i < numRequestThreads; ++i) { requestThreads[i] = - new Thread( - () -> { - try { - for (int j = 0; j < 100; ++j) { - List buffers = - bufferPool.requestBuffers(); - Thread.sleep(10); - if (j % 2 == 0) { - bufferPool.recycle(buffers); - } else { - for (MemorySegment segment : buffers) { - bufferPool.recycle(segment); - } - } + new CheckedThread() { + @Override + public void go() throws Exception { + for (int j = 0; j < 100; ++j) { + List buffers = bufferPool.requestBuffers(); + Thread.sleep(10); + if (j % 2 == 0) { + bufferPool.recycle(buffers); + } else { + for (MemorySegment segment : buffers) { + bufferPool.recycle(segment); } - } catch (Throwable throwable) { - exception.set(throwable); } - }); + } + } + }; requestThreads[i].start(); } - for (Thread requestThread : requestThreads) { - requestThread.join(); + for (CheckedThread requestThread : requestThreads) { + requestThread.sync(); } - assertNull(exception.get()); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); } finally { bufferPool.destroy(); } } @Test - public void testDestroy() throws Exception { + void testDestroy() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.recycle(buffers); - assertFalse(bufferPool.isDestroyed()); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.isDestroyed()).isFalse(); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); buffers = bufferPool.requestBuffers(); - assertEquals( - bufferPool.getNumTotalBuffers() - buffers.size(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()) + .isEqualTo(bufferPool.getNumTotalBuffers() - buffers.size()); bufferPool.destroy(); - assertTrue(bufferPool.isDestroyed()); - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.isDestroyed()).isTrue(); + assertThat(bufferPool.getAvailableBuffers()).isZero(); } - @Test(expected = IllegalStateException.class) - public void testRequestBuffersAfterDestroyed() throws Exception { + @Test + void testRequestBuffersAfterDestroyed() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); bufferPool.requestBuffers(); bufferPool.destroy(); - bufferPool.requestBuffers(); + assertThatThrownBy(bufferPool::requestBuffers).isInstanceOf(IllegalStateException.class); } @Test - public void testRecycleAfterDestroyed() throws Exception { + void testRecycleAfterDestroyed() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.destroy(); bufferPool.recycle(buffers); - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isZero(); } @Test - public void testDestroyWhileBlockingRequest() throws Exception { - AtomicReference exception = new AtomicReference<>(); + void testDestroyWhileBlockingRequest() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); - Thread requestThread = - new Thread( - () -> { - try { - while (true) { - bufferPool.requestBuffers(); - } - } catch (Throwable throwable) { - exception.set(throwable); - } - }); + CheckedThread requestThread = + new CheckedThread() { + @Override + public void go() throws Exception { + while (true) { + bufferPool.requestBuffers(); + } + } + }; requestThread.start(); Thread.sleep(1000); bufferPool.destroy(); - requestThread.join(); - assertTrue(exception.get() instanceof IllegalStateException); + assertThatThrownBy(requestThread::sync).isInstanceOf(IllegalStateException.class); } private BatchShuffleReadBufferPool createBufferPool(long totalBytes, int bufferSize) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java index 40257f22463b3..b76c5eb41a76b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java @@ -36,16 +36,18 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + /** */ -public class ChannelViewsTest { +class ChannelViewsTest { private static final long SEED = 649180756312423613L; private static final int KEY_MAX = Integer.MAX_VALUE; @@ -72,8 +74,8 @@ public class ChannelViewsTest { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { this.memoryManager = MemoryManagerBuilder.newBuilder() .setMemorySize(MEMORY_SIZE) @@ -82,14 +84,15 @@ public void beforeTest() { this.ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { this.ioManager.close(); if (memoryManager != null) { - Assert.assertTrue( - "Memory leak: not all segments have been returned to the memory manager.", - this.memoryManager.verifyEmpty()); + assertThat(this.memoryManager.verifyEmpty()) + .withFailMessage( + "Memory leak: not all segments have been returned to the memory manager.") + .isTrue(); this.memoryManager.shutdown(); this.memoryManager = null; } @@ -98,7 +101,7 @@ public void afterTest() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadSmallRecords() throws Exception { + void testWriteReadSmallRecords() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -134,16 +137,7 @@ public void testWriteReadSmallRecords() throws Exception { for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(rec); serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertReadRecordMatchRegenerated(readRec, rec); } this.memoryManager.release(inView.close()); @@ -151,7 +145,7 @@ public void testWriteReadSmallRecords() throws Exception { } @Test - public void testWriteAndReadLongRecords() throws Exception { + void testWriteAndReadLongRecords() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LONG_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -188,13 +182,7 @@ public void testWriteAndReadLongRecords() throws Exception { for (int i = 0; i < NUM_PAIRS_LONG; i++) { generator.next(rec); serializer.deserialize(readRec, inView); - final int k1 = rec.f0; - final String v1 = rec.f1; - final int k2 = readRec.f0; - final String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertReadRecordMatchRegenerated(readRec, rec); } this.memoryManager.release(inView.close()); @@ -202,7 +190,7 @@ public void testWriteAndReadLongRecords() throws Exception { } @Test - public void testReadTooMany() throws Exception { + void testReadTooMany() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -235,33 +223,24 @@ public void testReadTooMany() throws Exception { generator.reset(); // read and re-generate all records and compare them - try { - final Tuple2 readRec = new Tuple2<>(); - for (int i = 0; i < NUM_PAIRS_SHORT + 1; i++) { - generator.next(rec); - serializer.deserialize(readRec, inView); - final int k1 = rec.f0; - final String v1 = rec.f1; - final int k2 = readRec.f0; - final String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); - } - Assert.fail("Expected an EOFException which did not occur."); - } catch (EOFException eofex) { - // expected - } catch (Throwable t) { - // unexpected - Assert.fail("Unexpected Exception: " + t.getMessage()); + final Tuple2 readRec = new Tuple2<>(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(rec); + serializer.deserialize(readRec, inView); + assertReadRecordMatchRegenerated(readRec, rec); } + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Expected an EOFException which did not occur.") + .isInstanceOf(EOFException.class); + this.memoryManager.release(inView.close()); reader.deleteChannel(); } @Test - public void testReadWithoutKnownBlockCount() throws Exception { + void testReadWithoutKnownBlockCount() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -297,16 +276,7 @@ public void testReadWithoutKnownBlockCount() throws Exception { for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(rec); serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertReadRecordMatchRegenerated(readRec, rec); } this.memoryManager.release(inView.close()); @@ -314,7 +284,7 @@ public void testReadWithoutKnownBlockCount() throws Exception { } @Test - public void testWriteReadOneBufferOnly() throws Exception { + void testWriteReadOneBufferOnly() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -350,16 +320,7 @@ public void testWriteReadOneBufferOnly() throws Exception { for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(rec); serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertReadRecordMatchRegenerated(readRec, rec); } this.memoryManager.release(inView.close()); @@ -367,7 +328,7 @@ public void testWriteReadOneBufferOnly() throws Exception { } @Test - public void testWriteReadNotAll() throws Exception { + void testWriteReadNotAll() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -404,19 +365,26 @@ public void testWriteReadNotAll() throws Exception { for (int i = 0; i < NUM_PAIRS_SHORT / 2; i++) { generator.next(rec); serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertReadRecordMatchRegenerated(readRec, rec); } this.memoryManager.release(inView.close()); reader.deleteChannel(); } + + private static void assertReadRecordMatchRegenerated( + Tuple2 readRec, Tuple2 rec) { + int k1 = rec.f0; + String v1 = rec.f1; + + int k2 = readRec.f0; + String v2 = readRec.f1; + + assertThat(k2) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(k1); + assertThat(v2) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(v1); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java index dc96ea654e2dc..17ab746d41a30 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java @@ -21,26 +21,25 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.testutils.TestJvmProcess; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.nio.file.Path; import java.time.Duration; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; /** Tests the logic of {@link FileChannelManagerImpl}. */ -public class FileChannelManagerImplTest extends TestLogger { +class FileChannelManagerImplTest { private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class); private static final String DIR_NAME_PREFIX = "manager-test"; @@ -54,63 +53,68 @@ public class FileChannelManagerImplTest extends TestLogger { private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10); - @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir private Path temporaryFolder; @Test - public void testFairness() throws Exception { - String directory1 = temporaryFolder.newFolder().getAbsoluteFile().getAbsolutePath(); - String directory2 = temporaryFolder.newFolder().getAbsoluteFile().getAbsolutePath(); - FileChannelManager fileChannelManager = - new FileChannelManagerImpl(new String[] {directory1, directory2}, "test"); - - int numChannelIDs = 100000; - AtomicInteger counter1 = new AtomicInteger(); - AtomicInteger counter2 = new AtomicInteger(); - - int numThreads = 10; - Thread[] threads = new Thread[numThreads]; - for (int i = 0; i < numThreads; ++i) { - threads[i] = - new Thread( - () -> { - for (int j = 0; j < numChannelIDs; ++j) { - FileIOChannel.ID channelID = fileChannelManager.createChannel(); - if (channelID.getPath().startsWith(directory1)) { - counter1.incrementAndGet(); - } else { - counter2.incrementAndGet(); + void testFairness() throws Exception { + String directory1 = + TempDirUtils.newFolder(temporaryFolder).getAbsoluteFile().getAbsolutePath(); + String directory2 = + TempDirUtils.newFolder(temporaryFolder).getAbsoluteFile().getAbsolutePath(); + + try (FileChannelManager fileChannelManager = + new FileChannelManagerImpl(new String[] {directory1, directory2}, "test")) { + int numChannelIDs = 100000; + AtomicInteger counter1 = new AtomicInteger(); + AtomicInteger counter2 = new AtomicInteger(); + + int numThreads = 10; + Thread[] threads = new Thread[numThreads]; + for (int i = 0; i < numThreads; ++i) { + threads[i] = + new Thread( + () -> { + for (int j = 0; j < numChannelIDs; ++j) { + FileIOChannel.ID channelID = + fileChannelManager.createChannel(); + if (channelID.getPath().startsWith(directory1)) { + counter1.incrementAndGet(); + } else { + counter2.incrementAndGet(); + } } - } - }); - threads[i].start(); - } + }); + threads[i].start(); + } - for (int i = 0; i < numThreads; ++i) { - threads[i].join(); - } + for (int i = 0; i < numThreads; ++i) { + threads[i].join(); + } - assertEquals(counter1.get(), counter2.get()); + assertThat(counter2).hasValue(counter1.get()); + } } @Test - public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception { + void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception { testDirectoriesCleanupOnKill(false); } @Test - public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception { + void testDirectoriesCleanupOnKillWithCallerHook() throws Exception { testDirectoriesCleanupOnKill(true); } private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception { - assumeTrue( - OperatingSystem.isLinux() - || OperatingSystem.isFreeBSD() - || OperatingSystem.isSolaris() - || OperatingSystem.isMac()); - - File fileChannelDir = temporaryFolder.newFolder(); - File signalDir = temporaryFolder.newFolder(); + assumeThat( + OperatingSystem.isLinux() + || OperatingSystem.isFreeBSD() + || OperatingSystem.isSolaris() + || OperatingSystem.isMac()) + .isTrue(); + + File fileChannelDir = TempDirUtils.newFolder(temporaryFolder); + File signalDir = TempDirUtils.newFolder(temporaryFolder); File signalFile = new File(signalDir.getAbsolutePath(), SIGNAL_FILE_FOR_KILLING); FileChannelManagerTestProcess fileChannelManagerTestProcess = @@ -130,24 +134,27 @@ private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exceptio Runtime.getRuntime() .exec("kill " + fileChannelManagerTestProcess.getProcessId()); kill.waitFor(); - assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue()); + assertThat(kill.exitValue()) + .withFailMessage("Failed to send SIG_TERM to process") + .isZero(); Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) { Thread.sleep(100); } - assertFalse( - "The file channel manager test process does not terminate in time, its output is: \n" - + fileChannelManagerTestProcess.getProcessOutput(), - fileChannelManagerTestProcess.isAlive()); + assertThat(fileChannelManagerTestProcess.isAlive()) + .withFailMessage( + "The file channel manager test process does not terminate in time, its output is: \n%s", + fileChannelManagerTestProcess.getProcessOutput()) + .isFalse(); // Checks if the directories are cleared. - assertFalse( - "The file channel manager test process does not remove the tmp shuffle directories after termination, " - + "its output is \n" - + fileChannelManagerTestProcess.getProcessOutput(), - fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)); + assertThat(fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) + .withFailMessage( + "The file channel manager test process does not remove the tmp shuffle directories after termination, its output is \n%s", + fileChannelManagerTestProcess.getProcessOutput()) + .isFalse(); } finally { fileChannelManagerTestProcess.destroy(); } @@ -189,7 +196,7 @@ public String getEntryPointClassName() { } /** The entry point class to test the file channel manager cleanup with shutdown hook. */ - public static class FileChannelManagerCleanupRunner { + private static class FileChannelManagerCleanupRunner { public static void main(String[] args) throws Exception { boolean callerHasHook = Boolean.parseBoolean(args[0]); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java index d6c24918f6bea..1290220ed3e32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java @@ -31,18 +31,18 @@ import org.apache.flink.runtime.operators.testutils.PairGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.PairGenerator.Pair; import org.apache.flink.runtime.operators.testutils.PairGenerator.ValueMode; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class FileChannelStreamsITCase extends TestLogger { +class FileChannelStreamsITCase { private static final long SEED = 649180756312423613L; @@ -66,8 +66,8 @@ public class FileChannelStreamsITCase extends TestLogger { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memManager = MemoryManagerBuilder.newBuilder() .setMemorySize(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE) @@ -76,295 +76,261 @@ public void beforeTest() { ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); - assertTrue("The memory has not been properly released", memManager.verifyEmpty()); + assertThat(memManager.verifyEmpty()) + .withFailMessage("The memory has not been properly released") + .isTrue(); } // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadSmallRecords() { - try { - List memory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final PairGenerator generator = - new PairGenerator( - SEED, - KEY_MAX, - VALUE_SHORT_LENGTH, - KeyMode.RANDOM, - ValueMode.RANDOM_LENGTH); - final FileIOChannel.ID channel = ioManager.createChannel(); - - // create the writer output view - final BlockChannelWriter writer = - ioManager.createBlockChannelWriter(channel); - final FileChannelOutputView outView = - new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); - - // write a number of pairs - Pair pair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - pair.write(outView); - } - outView.close(); - - // create the reader input view - List readMemory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final BlockChannelReader reader = - ioManager.createBlockChannelReader(channel); - final FileChannelInputView inView = - new FileChannelInputView( - reader, memManager, readMemory, outView.getBytesInLatestSegment()); - generator.reset(); - - // read and re-generate all records and compare them - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); - } - - inView.close(); - reader.deleteChannel(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + void testWriteReadSmallRecords() throws Exception { + List memory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final PairGenerator generator = + new PairGenerator( + SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final FileIOChannel.ID channel = ioManager.createChannel(); + + // create the writer output view + final BlockChannelWriter writer = + ioManager.createBlockChannelWriter(channel); + final FileChannelOutputView outView = + new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); + + // write a number of pairs + Pair pair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + pair.write(outView); } + outView.close(); + + // create the reader input view + List readMemory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final BlockChannelReader reader = + ioManager.createBlockChannelReader(channel); + final FileChannelInputView inView = + new FileChannelInputView( + reader, memManager, readMemory, outView.getBytesInLatestSegment()); + generator.reset(); + + // read and re-generate all records and compare them + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); + } + + inView.close(); + reader.deleteChannel(); } @Test - public void testWriteAndReadLongRecords() { - try { - final List memory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final PairGenerator generator = - new PairGenerator( - SEED, - KEY_MAX, - VALUE_LONG_LENGTH, - KeyMode.RANDOM, - ValueMode.RANDOM_LENGTH); - final FileIOChannel.ID channel = this.ioManager.createChannel(); - - // create the writer output view - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channel); - final FileChannelOutputView outView = - new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); - - // write a number of pairs - Pair pair = new Pair(); - for (int i = 0; i < NUM_PAIRS_LONG; i++) { - generator.next(pair); - pair.write(outView); - } - outView.close(); - - // create the reader input view - List readMemory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final BlockChannelReader reader = - ioManager.createBlockChannelReader(channel); - final FileChannelInputView inView = - new FileChannelInputView( - reader, memManager, readMemory, outView.getBytesInLatestSegment()); - generator.reset(); - - // read and re-generate all records and compare them - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_LONG; i++) { - generator.next(pair); - readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); - } - - inView.close(); - reader.deleteChannel(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + void testWriteAndReadLongRecords() throws Exception { + final List memory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final PairGenerator generator = + new PairGenerator( + SEED, KEY_MAX, VALUE_LONG_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final FileIOChannel.ID channel = this.ioManager.createChannel(); + + // create the writer output view + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channel); + final FileChannelOutputView outView = + new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); + + // write a number of pairs + Pair pair = new Pair(); + for (int i = 0; i < NUM_PAIRS_LONG; i++) { + generator.next(pair); + pair.write(outView); + } + outView.close(); + + // create the reader input view + List readMemory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final BlockChannelReader reader = + ioManager.createBlockChannelReader(channel); + final FileChannelInputView inView = + new FileChannelInputView( + reader, memManager, readMemory, outView.getBytesInLatestSegment()); + generator.reset(); + + // read and re-generate all records and compare them + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_LONG; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } + + inView.close(); + reader.deleteChannel(); } @Test - public void testReadTooMany() { - try { - final List memory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final PairGenerator generator = - new PairGenerator( - SEED, - KEY_MAX, - VALUE_SHORT_LENGTH, - KeyMode.RANDOM, - ValueMode.RANDOM_LENGTH); - final FileIOChannel.ID channel = this.ioManager.createChannel(); - - // create the writer output view - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channel); - final FileChannelOutputView outView = - new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); - - // write a number of pairs - Pair pair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - pair.write(outView); - } - outView.close(); - - // create the reader input view - List readMemory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final BlockChannelReader reader = - ioManager.createBlockChannelReader(channel); - final FileChannelInputView inView = - new FileChannelInputView( - reader, memManager, readMemory, outView.getBytesInLatestSegment()); - generator.reset(); - - // read and re-generate all records and compare them - try { - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT + 1; i++) { - generator.next(pair); - readPair.read(inView); - assertEquals( - "The re-generated and the read record do not match.", pair, readPair); - } - fail("Expected an EOFException which did not occur."); - } catch (EOFException eofex) { - // expected - } - - inView.close(); - reader.deleteChannel(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + void testReadTooMany() throws Exception { + final List memory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final PairGenerator generator = + new PairGenerator( + SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final FileIOChannel.ID channel = this.ioManager.createChannel(); + + // create the writer output view + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channel); + final FileChannelOutputView outView = + new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); + + // write a number of pairs + Pair pair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + pair.write(outView); + } + outView.close(); + + // create the reader input view + List readMemory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final BlockChannelReader reader = + ioManager.createBlockChannelReader(channel); + final FileChannelInputView inView = + new FileChannelInputView( + reader, memManager, readMemory, outView.getBytesInLatestSegment()); + generator.reset(); + + // read and re-generate all records and compare them + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } + + generator.next(pair); + assertThatThrownBy(() -> readPair.read(inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + + inView.close(); + reader.deleteChannel(); } @Test - public void testWriteReadOneBufferOnly() { - try { - final List memory = memManager.allocatePages(new DummyInvokable(), 1); - - final PairGenerator generator = - new PairGenerator( - SEED, - KEY_MAX, - VALUE_SHORT_LENGTH, - KeyMode.RANDOM, - ValueMode.RANDOM_LENGTH); - final FileIOChannel.ID channel = this.ioManager.createChannel(); - - // create the writer output view - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channel); - final FileChannelOutputView outView = - new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); - - // write a number of pairs - Pair pair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - pair.write(outView); - } - outView.close(); - - // create the reader input view - List readMemory = memManager.allocatePages(new DummyInvokable(), 1); - - final BlockChannelReader reader = - ioManager.createBlockChannelReader(channel); - final FileChannelInputView inView = - new FileChannelInputView( - reader, memManager, readMemory, outView.getBytesInLatestSegment()); - generator.reset(); - - // read and re-generate all records and compare them - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); - } - - inView.close(); - reader.deleteChannel(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + void testWriteReadOneBufferOnly() throws Exception { + final List memory = memManager.allocatePages(new DummyInvokable(), 1); + + final PairGenerator generator = + new PairGenerator( + SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final FileIOChannel.ID channel = this.ioManager.createChannel(); + + // create the writer output view + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channel); + final FileChannelOutputView outView = + new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); + + // write a number of pairs + Pair pair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + pair.write(outView); + } + outView.close(); + + // create the reader input view + List readMemory = memManager.allocatePages(new DummyInvokable(), 1); + + final BlockChannelReader reader = + ioManager.createBlockChannelReader(channel); + final FileChannelInputView inView = + new FileChannelInputView( + reader, memManager, readMemory, outView.getBytesInLatestSegment()); + generator.reset(); + + // read and re-generate all records and compare them + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } + + inView.close(); + reader.deleteChannel(); } @Test - public void testWriteReadNotAll() { - try { - final List memory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final PairGenerator generator = - new PairGenerator( - SEED, - KEY_MAX, - VALUE_SHORT_LENGTH, - KeyMode.RANDOM, - ValueMode.RANDOM_LENGTH); - final FileIOChannel.ID channel = this.ioManager.createChannel(); - - // create the writer output view - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channel); - final FileChannelOutputView outView = - new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); - - // write a number of pairs - Pair pair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT; i++) { - generator.next(pair); - pair.write(outView); - } - outView.close(); - - // create the reader input view - List readMemory = - memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); - - final BlockChannelReader reader = - ioManager.createBlockChannelReader(channel); - final FileChannelInputView inView = - new FileChannelInputView( - reader, memManager, readMemory, outView.getBytesInLatestSegment()); - generator.reset(); - - // read and re-generate all records and compare them - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT / 2; i++) { - generator.next(pair); - readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); - } - - inView.close(); - reader.deleteChannel(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + void testWriteReadNotAll() throws Exception { + final List memory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final PairGenerator generator = + new PairGenerator( + SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final FileIOChannel.ID channel = this.ioManager.createChannel(); + + // create the writer output view + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channel); + final FileChannelOutputView outView = + new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE); + + // write a number of pairs + Pair pair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + pair.write(outView); } + outView.close(); + + // create the reader input view + List readMemory = + memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); + + final BlockChannelReader reader = + ioManager.createBlockChannelReader(channel); + final FileChannelInputView inView = + new FileChannelInputView( + reader, memManager, readMemory, outView.getBytesInLatestSegment()); + generator.reset(); + + // read and re-generate all records and compare them + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT / 2; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); + } + + inView.close(); + reader.deleteChannel(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java index 1a0838345e51d..79621341e90e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java @@ -29,19 +29,19 @@ import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.StringValue; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.FileWriter; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; -public class FileChannelStreamsTest { +class FileChannelStreamsTest { @Test - public void testCloseAndDeleteOutputView() { + void testCloseAndDeleteOutputView() throws Exception { try (IOManager ioManager = new IOManagerAsync()) { MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); @@ -56,22 +56,19 @@ public void testCloseAndDeleteOutputView() { // close for the first time, make sure all memory returns out.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); // close again, should not cause an exception out.close(); // delete, make sure file is removed out.closeAndDelete(); - assertFalse(new File(channel.getPath()).exists()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + assertThat(new File(channel.getPath())).doesNotExist(); } } @Test - public void testCloseAndDeleteInputView() { + void testCloseAndDeleteInputView() throws Exception { try (IOManager ioManager = new IOManagerAsync()) { MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); @@ -92,17 +89,14 @@ public void testCloseAndDeleteInputView() { // close for the first time, make sure all memory returns in.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); // close again, should not cause an exception in.close(); // delete, make sure file is removed in.closeAndDelete(); - assertFalse(new File(channel.getPath()).exists()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + assertThat(new File(channel.getPath())).doesNotExist(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java index 36b5912c5e46a..ebbf2850329c5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java @@ -27,18 +27,19 @@ import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class SeekableFileChannelInputViewTest { +class SeekableFileChannelInputViewTest { @Test - public void testSeek() { + void testSeek() throws Exception { final int PAGE_SIZE = 16 * 1024; final int NUM_RECORDS = 120000; // integers across 7.x pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes) @@ -64,7 +65,7 @@ public void testSeek() { } // close for the first time, make sure all memory returns out.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); memMan.allocatePages(new DummyInvokable(), memory, 4); SeekableFileChannelInputView in = @@ -73,97 +74,76 @@ public void testSeek() { // read first, complete for (int i = 0; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the middle of the 3rd page int i = 2 * PAGE_SIZE + PAGE_SIZE / 4; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the end i = 120000 - 4; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the beginning i = 0; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to after a page i = PAGE_SIZE; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to after a page i = 3 * PAGE_SIZE; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); - } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { + assertThat(in.readInt()).isEqualTo(i); } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the end i = NUM_RECORDS; in.seek(i); - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek out of bounds - try { - in.seek(-10); - fail("should throw an exception"); - } catch (IllegalArgumentException ignored) { - } - try { - in.seek(NUM_RECORDS + 1); - fail("should throw an exception"); - } catch (IllegalArgumentException ignored) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + assertThatThrownBy(() -> in.seek(-10)) + .withFailMessage("should throw an exception") + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> in.seek(NUM_RECORDS + 1)) + .withFailMessage("should throw an exception") + .isInstanceOf(IllegalArgumentException.class); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java index 9db101888982e..e600da1c864ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java @@ -33,15 +33,17 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.ArrayList; -public class SpillingBufferTest { +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +class SpillingBufferTest { private static final long SEED = 649180756312423613L; @@ -65,20 +67,21 @@ public class SpillingBufferTest { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); if (memoryManager != null) { - Assert.assertTrue( - "Memory leak: not all segments have been returned to the memory manager.", - memoryManager.verifyEmpty()); + assertThat(memoryManager.verifyEmpty()) + .withFailMessage( + "Memory leak: not all segments have been returned to the memory manager.") + .isTrue(); memoryManager.shutdown(); memoryManager = null; } @@ -87,7 +90,7 @@ public void afterTest() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadInMemory() throws Exception { + void testWriteReadInMemory() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -126,9 +129,9 @@ public void testWriteReadInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } // re-notifyNonEmpty the data @@ -146,9 +149,9 @@ public void testWriteReadInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -156,7 +159,7 @@ public void testWriteReadInMemory() throws Exception { } @Test - public void testWriteReadTooMuchInMemory() throws Exception { + void testWriteReadTooMuchInMemory() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -185,34 +188,34 @@ public void testWriteReadTooMuchInMemory() throws Exception { // notifyNonEmpty and re-generate all records and compare them final Tuple2 readRec = new Tuple2<>(); - try { - for (int i = 0; i < NUM_PAIRS_INMEM + 1; i++) { - generator.next(rec); - serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); - } - Assert.fail("Read too much, expected EOFException."); - } catch (EOFException eofex) { - // expected + for (int i = 0; i < NUM_PAIRS_INMEM; i++) { + generator.next(rec); + serializer.deserialize(readRec, inView); + + int k1 = rec.f0; + String v1 = rec.f1; + + int k2 = readRec.f0; + String v2 = readRec.f1; + + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + // re-notifyNonEmpty the data - inView = outView.flip(); + DataInputView nextInView = outView.flip(); generator.reset(); // notifyNonEmpty and re-generate all records and compare them for (int i = 0; i < NUM_PAIRS_INMEM; i++) { generator.next(rec); - serializer.deserialize(readRec, inView); + serializer.deserialize(readRec, nextInView); int k1 = rec.f0; String v1 = rec.f1; @@ -220,9 +223,9 @@ public void testWriteReadTooMuchInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -232,7 +235,7 @@ public void testWriteReadTooMuchInMemory() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadExternal() throws Exception { + void testWriteReadExternal() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -271,9 +274,9 @@ public void testWriteReadExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } // re-notifyNonEmpty the data @@ -291,9 +294,9 @@ public void testWriteReadExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -301,7 +304,7 @@ public void testWriteReadExternal() throws Exception { } @Test - public void testWriteReadTooMuchExternal() throws Exception { + void testWriteReadTooMuchExternal() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -330,34 +333,34 @@ public void testWriteReadTooMuchExternal() throws Exception { // notifyNonEmpty and re-generate all records and compare them final Tuple2 readRec = new Tuple2<>(); - try { - for (int i = 0; i < NUM_PAIRS_EXTERNAL + 1; i++) { - generator.next(rec); - serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); - } - Assert.fail("Read too much, expected EOFException."); - } catch (EOFException eofex) { - // expected + for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) { + generator.next(rec); + serializer.deserialize(readRec, inView); + + int k1 = rec.f0; + String v1 = rec.f1; + + int k2 = readRec.f0; + String v2 = readRec.f1; + + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + // re-notifyNonEmpty the data - inView = outView.flip(); + DataInputView nextInView = outView.flip(); generator.reset(); // notifyNonEmpty and re-generate all records and compare them for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) { generator.next(rec); - serializer.deserialize(readRec, inView); + serializer.deserialize(readRec, nextInView); int k1 = rec.f0; String v1 = rec.f1; @@ -365,9 +368,9 @@ public void testWriteReadTooMuchExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java index 17d191f406801..1bb9e4336a73d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -19,34 +19,29 @@ package org.apache.flink.runtime.io.disk.iomanager; import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.util.TestNotificationListener; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Mockito.mock; /** Tests for {@link AsynchronousBufferFileWriter}. */ -public class AsynchronousBufferFileWriterTest { - @Rule public ExpectedException exception = ExpectedException.none(); +class AsynchronousBufferFileWriterTest { private static final IOManager ioManager = new IOManagerAsync(); @@ -54,139 +49,118 @@ public class AsynchronousBufferFileWriterTest { private AsynchronousBufferFileWriter writer; - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUp() throws IOException { + @BeforeEach + void setUp() throws IOException { writer = new AsynchronousBufferFileWriter( ioManager.createChannel(), new RequestQueue()); } @Test - public void testAddAndHandleRequest() throws Exception { + void testAddAndHandleRequest() throws Exception { addRequest(); - assertEquals( - "Didn't increment number of outstanding requests.", - 1, - writer.getNumberOfOutstandingRequests()); + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Didn't increment number of outstanding requests.") + .isOne(); handleRequest(); - assertEquals( - "Didn't decrement number of outstanding requests.", - 0, - writer.getNumberOfOutstandingRequests()); + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Didn't decrement number of outstanding requests.") + .isZero(); } @Test - public void testAddWithFailingWriter() throws Exception { + void testAddWithFailingWriter() throws Exception { AsynchronousBufferFileWriter writer = new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue<>()); writer.close(); - exception.expect(IOException.class); - Buffer buffer = new NetworkBuffer( MemorySegmentFactory.allocateUnpooledSegment(4096), FreeingBufferRecycler.INSTANCE); - try { - writer.writeBlock(buffer); - } finally { - if (!buffer.isRecycled()) { - buffer.recycleBuffer(); - Assert.fail("buffer not recycled"); - } - assertEquals( - "Shouln't increment number of outstanding requests.", - 0, - writer.getNumberOfOutstandingRequests()); + + assertThatThrownBy(() -> writer.writeBlock(buffer)).isInstanceOf(IOException.class); + + if (!buffer.isRecycled()) { + buffer.recycleBuffer(); + fail("buffer not recycled"); } + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Shouldn't increment number of outstanding requests.") + .isZero(); } @Test - public void testSubscribe() throws Exception { + void testSubscribe() throws Exception { final TestNotificationListener listener = new TestNotificationListener(); // Unsuccessful subscription, because no outstanding requests - assertFalse( - "Allowed to subscribe w/o any outstanding requests.", - writer.registerAllRequestsProcessedListener(listener)); + assertThat(writer.registerAllRequestsProcessedListener(listener)) + .withFailMessage("Allowed to subscribe w/o any outstanding requests.") + .isFalse(); // Successful subscription addRequest(); - assertTrue( - "Didn't allow to subscribe.", - writer.registerAllRequestsProcessedListener(listener)); + assertThat(writer.registerAllRequestsProcessedListener(listener)) + .withFailMessage("Didn't allow to subscribe.") + .isTrue(); // Test notification handleRequest(); - assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage("Listener was not notified.") + .isOne(); } @Test - public void testSubscribeAndClose() throws IOException, InterruptedException { + void testSubscribeAndClose() throws Exception { final TestNotificationListener listener = new TestNotificationListener(); - final AtomicReference error = new AtomicReference(); - - final CountDownLatch sync = new CountDownLatch(1); - addRequest(); addRequest(); writer.registerAllRequestsProcessedListener(listener); - final Thread asyncCloseThread = - new Thread( - new Runnable() { - @Override - public void run() { - try { - writer.close(); - } catch (Throwable t) { - error.set(t); - } finally { - sync.countDown(); - } - } - }); + final CheckedThread asyncCloseThread = + new CheckedThread() { + @Override + public void go() throws Exception { + writer.close(); + } + }; asyncCloseThread.start(); handleRequest(); handleRequest(); - sync.await(); + asyncCloseThread.sync(); - assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage("Listener was not notified.") + .isOne(); } @Test - public void testConcurrentSubscribeAndHandleRequest() throws Exception { + void testConcurrentSubscribeAndHandleRequest() throws Exception { final ExecutorService executor = Executors.newFixedThreadPool(2); final TestNotificationListener listener = new TestNotificationListener(); final Callable subscriber = - new Callable() { - @Override - public Boolean call() throws Exception { - return writer.registerAllRequestsProcessedListener(listener); - } - }; + () -> writer.registerAllRequestsProcessedListener(listener); final Callable requestHandler = - new Callable() { - @Override - public Void call() throws Exception { - handleRequest(); - return null; - } + () -> { + handleRequest(); + return null; }; try { @@ -201,22 +175,13 @@ public Void call() throws Exception { handleRequestFuture.get(); - try { - if (subscribeFuture.get()) { - assertEquals( - "Race: Successfully subscribed, but was never notified.", - 1, - listener.getNumberOfNotifications()); - } else { - assertEquals( - "Race: Never subscribed successfully, but was notified.", - 0, - listener.getNumberOfNotifications()); - } - } catch (Throwable t) { - System.out.println(i); - Assert.fail(t.getMessage()); - } + boolean subscribed = subscribeFuture.get(); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage( + subscribed + ? "Race: Successfully subscribed, but was never notified." + : "Race: Never subscribed successfully, but was notified.") + .isEqualTo(subscribed ? 1 : 0); } } finally { executor.shutdownNow(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java index af85fc4073cec..a461a358d34c4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java @@ -23,8 +23,10 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.util.TestNotificationListener; +import org.apache.flink.testutils.executor.TestExecutorExtension; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,27 +43,28 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class AsynchronousFileIOChannelTest { +class AsynchronousFileIOChannelTest { private static final Logger LOG = LoggerFactory.getLogger(AsynchronousFileIOChannelTest.class); + @RegisterExtension + private static final TestExecutorExtension EXECUTOR_EXTENSION = + new TestExecutorExtension<>(Executors::newCachedThreadPool); + @Test - public void testAllRequestsProcessedListenerNotification() throws Exception { + void testAllRequestsProcessedListenerNotification() throws Exception { // -- Config ---------------------------------------------------------- final int numberOfRuns = 10; final int numberOfRequests = 100; // -- Setup ----------------------------------------------------------- - final ExecutorService executor = Executors.newFixedThreadPool(3); - final Random random = new Random(); - final RequestQueue requestQueue = new RequestQueue(); + final RequestQueue requestQueue = new RequestQueue<>(); final RequestDoneCallback ioChannelCallback = new NoOpCallback<>(); @@ -85,81 +88,72 @@ public void testAllRequestsProcessedListenerNotification() throws Exception { // Add requests task Callable addRequestsTask = - new Callable() { - @Override - public Void call() throws Exception { - for (int i = 0; i < numberOfRuns; i++) { - LOG.debug("Starting run {}.", i + 1); - - for (int j = 0; j < numberOfRequests; j++) { - ioChannel.addRequest(request); - } - - LOG.debug( - "Added all ({}) requests of run {}.", - numberOfRequests, - i + 1); + () -> { + for (int i = 0; i < numberOfRuns; i++) { + LOG.debug("Starting run {}.", i + 1); - int sleep = random.nextInt(10); - LOG.debug("Sleeping for {} ms before next run.", sleep); - - Thread.sleep(sleep); + for (int j = 0; j < numberOfRequests; j++) { + ioChannel.addRequest(request); } - LOG.debug("Done. Closing channel."); - ioChannel.close(); + LOG.debug( + "Added all ({}) requests of run {}.", + numberOfRequests, + i + 1); - sync.countDown(); + int sleep = random.nextInt(10); + LOG.debug("Sleeping for {} ms before next run.", sleep); - return null; + Thread.sleep(sleep); } + + LOG.debug("Done. Closing channel."); + ioChannel.close(); + + sync.countDown(); + + return null; }; // Process requests task Callable processRequestsTask = - new Callable() { - @Override - public Void call() throws Exception { - int total = numberOfRequests * numberOfRuns; - for (int i = 0; i < total; i++) { - requestQueue.take(); - - ioChannel.handleProcessedBuffer(buffer, null); - } + () -> { + int total = numberOfRequests * numberOfRuns; + for (int i = 0; i < total; i++) { + requestQueue.take(); - LOG.debug("Processed all ({}) requests.", numberOfRequests); + ioChannel.handleProcessedBuffer(buffer, null); + } - sync.countDown(); + LOG.debug("Processed all ({}) requests.", numberOfRequests); - return null; - } + sync.countDown(); + + return null; }; // Listener Callable registerListenerTask = - new Callable() { - @Override - public Void call() throws Exception { - while (true) { - int current = listener.getNumberOfNotifications(); - - if (ioChannel.registerAllRequestsProcessedListener(listener)) { - listener.waitForNotification(current); - } else if (ioChannel.isClosed()) { - break; - } + () -> { + while (true) { + int current = listener.getNumberOfNotifications(); + + if (ioChannel.registerAllRequestsProcessedListener(listener)) { + listener.waitForNotification(current); + } else if (ioChannel.isClosed()) { + break; } + } - LOG.debug("Stopping listener. Channel closed."); + LOG.debug("Stopping listener. Channel closed."); - sync.countDown(); + sync.countDown(); - return null; - } + return null; }; // Run tasks in random order - final List> tasks = new LinkedList>(); + final List> tasks = new LinkedList<>(); tasks.add(addRequestsTask); tasks.add(processRequestsTask); tasks.add(registerListenerTask); @@ -167,33 +161,29 @@ public Void call() throws Exception { Collections.shuffle(tasks); for (Callable task : tasks) { - executor.submit(task); + EXECUTOR_EXTENSION.getExecutor().submit(task); } - if (!sync.await(2, TimeUnit.MINUTES)) { - fail( - "Test failed due to a timeout. This indicates a deadlock due to the way" - + "that listeners are registered/notified in the asynchronous file I/O" - + "channel."); - } + assertThat(sync.await(2, TimeUnit.MINUTES)) + .withFailMessage( + "Test failed due to a timeout. This indicates a deadlock due to the way" + + "that listeners are registered/notified in the asynchronous file I/O" + + "channel.") + .isTrue(); listener.reset(); } - } finally { - executor.shutdown(); } } @Test - public void testClosedButAddRequestAndRegisterListenerRace() throws Exception { + void testClosedButAddRequestAndRegisterListenerRace() throws Exception { // -- Config ---------------------------------------------------------- final int numberOfRuns = 1024; // -- Setup ----------------------------------------------------------- - final ExecutorService executor = Executors.newFixedThreadPool(2); - - final RequestQueue requestQueue = new RequestQueue(); + final RequestQueue requestQueue = new RequestQueue<>(); @SuppressWarnings("unchecked") final RequestDoneCallback ioChannelCallback = new NoOpCallback<>(); @@ -216,61 +206,53 @@ public void testClosedButAddRequestAndRegisterListenerRace() throws Exception { // Add request task Callable addRequestTask = - new Callable() { - @Override - public Void call() throws Exception { - try { - ioChannel.addRequest(request); - } catch (Throwable expected) { - } finally { - sync.countDown(); - } - - return null; + () -> { + try { + ioChannel.addRequest(request); + } catch (Throwable expected) { + } finally { + sync.countDown(); } + + return null; }; // Listener Callable registerListenerTask = - new Callable() { - @Override - public Void call() throws Exception { - try { - while (true) { - int current = listener.getNumberOfNotifications(); - - if (ioChannel.registerAllRequestsProcessedListener( - listener)) { - listener.waitForNotification(current); - } else if (ioChannel.isClosed()) { - break; - } + () -> { + try { + while (true) { + int current = listener.getNumberOfNotifications(); + + if (ioChannel.registerAllRequestsProcessedListener(listener)) { + listener.waitForNotification(current); + } else if (ioChannel.isClosed()) { + break; } - } finally { - sync.countDown(); } - - return null; + } finally { + sync.countDown(); } + + return null; }; + ExecutorService executor = EXECUTOR_EXTENSION.getExecutor(); executor.submit(addRequestTask); executor.submit(registerListenerTask); - if (!sync.await(2, TimeUnit.MINUTES)) { - fail( - "Test failed due to a timeout. This indicates a deadlock due to the way" - + "that listeners are registered/notified in the asynchronous file I/O" - + "channel."); - } + assertThat(sync.await(2, TimeUnit.MINUTES)) + .withFailMessage( + "Test failed due to a timeout. This indicates a deadlock due to the way" + + "that listeners are registered/notified in the asynchronous file I/O" + + "channel.") + .isTrue(); } - } finally { - executor.shutdown(); } } @Test - public void testClosingWaits() { + void testClosingWaits() throws Exception { try (final IOManagerAsync ioMan = new IOManagerAsync()) { final int NUM_BLOCKS = 100; @@ -308,19 +290,16 @@ public void requestFailed(MemorySegment buffer, IOException e) { writer.close(); - assertEquals(NUM_BLOCKS, callbackCounter.get()); - assertFalse(exceptionOccurred.get()); + assertThat(callbackCounter).hasValue(NUM_BLOCKS); + assertThat(exceptionOccurred).isFalse(); } finally { writer.closeAndDelete(); } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } } @Test - public void testExceptionForwardsToClose() throws Exception { + void testExceptionForwardsToClose() throws Exception { try (IOManagerAsync ioMan = new IOManagerAsync()) { testExceptionForwardsToClose(ioMan, 100, 1); testExceptionForwardsToClose(ioMan, 100, 50); @@ -329,51 +308,42 @@ public void testExceptionForwardsToClose() throws Exception { } private void testExceptionForwardsToClose( - IOManagerAsync ioMan, final int numBlocks, final int failingBlock) { - try { - MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024); - FileIOChannel.ID channelId = ioMan.createChannel(); + IOManagerAsync ioMan, final int numBlocks, final int failingBlock) throws IOException { + MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024); + FileIOChannel.ID channelId = ioMan.createChannel(); - BlockChannelWriterWithCallback writer = - new AsynchronousBlockWriterWithCallback( - channelId, - ioMan.getWriteRequestQueue(channelId), - new NoOpCallback<>()) { + BlockChannelWriterWithCallback writer = + new AsynchronousBlockWriterWithCallback( + channelId, ioMan.getWriteRequestQueue(channelId), new NoOpCallback<>()) { - private int numBlocks; + private int numBlocks; - @Override - public void writeBlock(MemorySegment segment) throws IOException { - numBlocks++; - - if (numBlocks == failingBlock) { - this.requestsNotReturned.incrementAndGet(); - this.requestQueue.add(new FailingWriteRequest(this, segment)); - } else { - super.writeBlock(segment); - } - } - }; + @Override + public void writeBlock(MemorySegment segment) throws IOException { + numBlocks++; - try { - for (int i = 0; i < numBlocks; i++) { - writer.writeBlock(seg); - } + if (numBlocks == failingBlock) { + this.requestsNotReturned.incrementAndGet(); + this.requestQueue.add(new FailingWriteRequest(this, segment)); + } else { + super.writeBlock(segment); + } + } + }; + + assertThatThrownBy( + () -> { + try { + for (int i = 0; i < numBlocks; i++) { + writer.writeBlock(seg); + } - writer.close(); - fail("did not forward exception"); - } catch (IOException e) { - // expected - } finally { - try { - writer.closeAndDelete(); - } catch (Throwable ignored) { - } - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + writer.close(); + } finally { + writer.closeAndDelete(); + } + }) + .isInstanceOf(IOException.class); } private static class NoOpCallback implements RequestDoneCallback { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java index 064700c9aa596..bec845a8cea5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java @@ -26,10 +26,10 @@ import org.apache.flink.runtime.util.event.NotificationListener; import org.apache.flink.util.IOUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -39,12 +39,10 @@ import static org.apache.flink.runtime.io.disk.iomanager.BufferFileWriterReaderTest.fillBufferWithAscendingNumbers; import static org.apache.flink.runtime.io.disk.iomanager.BufferFileWriterReaderTest.verifyBufferFilledWithAscendingNumbers; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class BufferFileWriterFileSegmentReaderTest { +class BufferFileWriterFileSegmentReaderTest { private static final int BUFFER_SIZE = 32 * 1024; @@ -60,13 +58,13 @@ public class BufferFileWriterFileSegmentReaderTest { private LinkedBlockingQueue returnedFileSegments = new LinkedBlockingQueue<>(); - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUpWriterAndReader() { + @BeforeEach + void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); try { @@ -82,18 +80,18 @@ public void setUpWriterAndReader() { } } - @After - public void tearDownWriterAndReader() { + @AfterEach + void tearDownWriterAndReader() { if (writer != null) { if (!writer.isClosed()) { - IOUtils.closeQuietly(() -> writer.close()); + IOUtils.closeQuietly(writer::close); } writer.deleteChannel(); } if (reader != null) { if (!reader.isClosed()) { - IOUtils.closeQuietly(() -> reader.close()); + IOUtils.closeQuietly(reader::close); } reader.deleteChannel(); } @@ -102,7 +100,7 @@ public void tearDownWriterAndReader() { } @Test - public void testWriteRead() throws IOException, InterruptedException { + void testWriteRead() throws IOException, InterruptedException { int numBuffers = 1024; int currentNumber = 0; @@ -124,28 +122,24 @@ public void testWriteRead() throws IOException, InterruptedException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.read(); } // Wait for all requests to be finished final CountDownLatch sync = new CountDownLatch(1); - final NotificationListener listener = - new NotificationListener() { - @Override - public void onNotification() { - sync.countDown(); - } - }; + final NotificationListener listener = sync::countDown; if (reader.registerAllRequestsProcessedListener(listener)) { sync.await(); } - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedFileSegments.size()); + assertThat(returnedFileSegments) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); currentNumber = 0; FileSegment fileSegment; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java index e97109b4c755c..89b9658ef670d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java @@ -25,22 +25,20 @@ import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Random; import java.util.concurrent.LinkedBlockingQueue; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class BufferFileWriterReaderTest { +class BufferFileWriterReaderTest { private static final int BUFFER_SIZE = 32 * 1024; @@ -56,13 +54,13 @@ public class BufferFileWriterReaderTest { private LinkedBlockingQueue returnedBuffers = new LinkedBlockingQueue<>(); - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUpWriterAndReader() { + @BeforeEach + void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); try { @@ -83,8 +81,8 @@ public void setUpWriterAndReader() { } } - @After - public void tearDownWriterAndReader() { + @AfterEach + void tearDownWriterAndReader() { if (writer != null) { writer.deleteChannel(); } @@ -97,7 +95,7 @@ public void tearDownWriterAndReader() { } @Test - public void testWriteRead() throws IOException { + void testWriteRead() throws IOException { int numBuffers = 1024; int currentNumber = 0; @@ -119,16 +117,18 @@ public void testWriteRead() throws IOException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.readInto(createBuffer()); } reader.close(); - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + assertThat(returnedBuffers) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); currentNumber = 0; Buffer buffer; @@ -139,7 +139,7 @@ public void testWriteRead() throws IOException { } @Test - public void testWriteSkipRead() throws IOException { + void testWriteSkipRead() throws IOException { int numBuffers = 1024; int currentNumber = 0; @@ -165,16 +165,18 @@ public void testWriteSkipRead() throws IOException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.readInto(createBuffer()); } reader.close(); - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + assertThat(returnedBuffers) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); // Start number after skipped buffers... currentNumber = (BUFFER_SIZE / 4) * toSkip; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java index c274a607c2779..c077da5439355 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java @@ -21,19 +21,18 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; -public class IOManagerAsyncTest { +class IOManagerAsyncTest { private IOManagerAsync ioManager; @@ -41,13 +40,13 @@ public class IOManagerAsyncTest { // Setup & Shutdown // ------------------------------------------------------------------------ - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { this.ioManager.close(); } @@ -56,305 +55,273 @@ public void afterTest() throws Exception { // ------------------------------------------------------------------------ @Test - public void channelReadWriteOneSegment() { + void channelReadWriteOneSegment() throws Exception { final int NUM_IOS = 1111; - try { - final FileIOChannel.ID channelID = this.ioManager.createChannel(); - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channelID); + final FileIOChannel.ID channelID = this.ioManager.createChannel(); + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channelID); - MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024); + MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024); - for (int i = 0; i < NUM_IOS; i++) { - for (int pos = 0; pos < memSeg.size(); pos += 4) { - memSeg.putInt(pos, i); - } - - writer.writeBlock(memSeg); - memSeg = writer.getNextReturnedBlock(); + for (int i = 0; i < NUM_IOS; i++) { + for (int pos = 0; pos < memSeg.size(); pos += 4) { + memSeg.putInt(pos, i); } - writer.close(); + writer.writeBlock(memSeg); + memSeg = writer.getNextReturnedBlock(); + } - final BlockChannelReader reader = - this.ioManager.createBlockChannelReader(channelID); - for (int i = 0; i < NUM_IOS; i++) { - reader.readBlock(memSeg); - memSeg = reader.getNextReturnedBlock(); + writer.close(); - for (int pos = 0; pos < memSeg.size(); pos += 4) { - if (memSeg.getInt(pos) != i) { - fail("Read memory segment contains invalid data."); - } - } - } + final BlockChannelReader reader = + this.ioManager.createBlockChannelReader(channelID); + for (int i = 0; i < NUM_IOS; i++) { + reader.readBlock(memSeg); + memSeg = reader.getNextReturnedBlock(); - reader.closeAndDelete(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test encountered an exception: " + ex.getMessage()); + for (int pos = 0; pos < memSeg.size(); pos += 4) { + assertThat(memSeg.getInt(pos)) + .withFailMessage("Read memory segment contains invalid data.") + .isEqualTo(i); + } } + + reader.closeAndDelete(); } @Test - public void channelReadWriteMultipleSegments() { + void channelReadWriteMultipleSegments() throws Exception { final int NUM_IOS = 1111; final int NUM_SEGS = 16; - try { - final List memSegs = new ArrayList(); - for (int i = 0; i < NUM_SEGS; i++) { - memSegs.add(MemorySegmentFactory.allocateUnpooledSegment(32 * 1024)); - } - - final FileIOChannel.ID channelID = this.ioManager.createChannel(); - final BlockChannelWriter writer = - this.ioManager.createBlockChannelWriter(channelID); + final List memSegs = new ArrayList(); + for (int i = 0; i < NUM_SEGS; i++) { + memSegs.add(MemorySegmentFactory.allocateUnpooledSegment(32 * 1024)); + } - for (int i = 0; i < NUM_IOS; i++) { - final MemorySegment memSeg = - memSegs.isEmpty() - ? writer.getNextReturnedBlock() - : memSegs.remove(memSegs.size() - 1); + final FileIOChannel.ID channelID = this.ioManager.createChannel(); + final BlockChannelWriter writer = + this.ioManager.createBlockChannelWriter(channelID); - for (int pos = 0; pos < memSeg.size(); pos += 4) { - memSeg.putInt(pos, i); - } + for (int i = 0; i < NUM_IOS; i++) { + final MemorySegment memSeg = + memSegs.isEmpty() + ? writer.getNextReturnedBlock() + : memSegs.remove(memSegs.size() - 1); - writer.writeBlock(memSeg); + for (int pos = 0; pos < memSeg.size(); pos += 4) { + memSeg.putInt(pos, i); } - writer.close(); - // get back the memory - while (memSegs.size() < NUM_SEGS) { - memSegs.add(writer.getNextReturnedBlock()); - } + writer.writeBlock(memSeg); + } + writer.close(); - final BlockChannelReader reader = - this.ioManager.createBlockChannelReader(channelID); - while (!memSegs.isEmpty()) { - reader.readBlock(memSegs.remove(0)); - } + // get back the memory + while (memSegs.size() < NUM_SEGS) { + memSegs.add(writer.getNextReturnedBlock()); + } - for (int i = 0; i < NUM_IOS; i++) { - final MemorySegment memSeg = reader.getNextReturnedBlock(); + final BlockChannelReader reader = + this.ioManager.createBlockChannelReader(channelID); + while (!memSegs.isEmpty()) { + reader.readBlock(memSegs.remove(0)); + } - for (int pos = 0; pos < memSeg.size(); pos += 4) { - if (memSeg.getInt(pos) != i) { - fail("Read memory segment contains invalid data."); - } - } - reader.readBlock(memSeg); + for (int i = 0; i < NUM_IOS; i++) { + final MemorySegment memSeg = reader.getNextReturnedBlock(); + + for (int pos = 0; pos < memSeg.size(); pos += 4) { + assertThat(memSeg.getInt(pos)) + .withFailMessage("Read memory segment contains invalid data.") + .isEqualTo(i); } + reader.readBlock(memSeg); + } - reader.closeAndDelete(); + reader.closeAndDelete(); - // get back the memory - while (memSegs.size() < NUM_SEGS) { - memSegs.add(reader.getNextReturnedBlock()); - } - } catch (Exception ex) { - ex.printStackTrace(); - fail("TEst encountered an exception: " + ex.getMessage()); + // get back the memory + while (memSegs.size() < NUM_SEGS) { + memSegs.add(reader.getNextReturnedBlock()); } } @Test - public void testExceptionPropagationReader() { - try { - // use atomic boolean as a boolean reference - final AtomicBoolean handlerCalled = new AtomicBoolean(); - final AtomicBoolean exceptionForwarded = new AtomicBoolean(); - - ReadRequest req = - new ReadRequest() { - - @Override - public void requestDone(IOException ioex) { - if (ioex instanceof TestIOException) { - exceptionForwarded.set(true); - } - - synchronized (handlerCalled) { - handlerCalled.set(true); - handlerCalled.notifyAll(); - } + void testExceptionPropagationReader() throws Exception { + // use atomic boolean as a boolean reference + final AtomicBoolean handlerCalled = new AtomicBoolean(); + final AtomicBoolean exceptionForwarded = new AtomicBoolean(); + + ReadRequest req = + new ReadRequest() { + + @Override + public void requestDone(IOException ioex) { + if (ioex instanceof TestIOException) { + exceptionForwarded.set(true); } - @Override - public void read() throws IOException { - throw new TestIOException(); + synchronized (handlerCalled) { + handlerCalled.set(true); + handlerCalled.notifyAll(); } - }; + } + + @Override + public void read() throws IOException { + throw new TestIOException(); + } + }; - // test the read queue - RequestQueue rq = ioManager.getReadRequestQueue(ioManager.createChannel()); - rq.add(req); + // test the read queue + RequestQueue rq = ioManager.getReadRequestQueue(ioManager.createChannel()); + rq.add(req); - // wait until the asynchronous request has been handled - synchronized (handlerCalled) { - while (!handlerCalled.get()) { - handlerCalled.wait(); - } + // wait until the asynchronous request has been handled + synchronized (handlerCalled) { + while (!handlerCalled.get()) { + handlerCalled.wait(); } - - assertTrue(exceptionForwarded.get()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } + + assertThat(exceptionForwarded).isTrue(); } @Test - public void testExceptionPropagationWriter() { - try { - // use atomic boolean as a boolean reference - final AtomicBoolean handlerCalled = new AtomicBoolean(); - final AtomicBoolean exceptionForwarded = new AtomicBoolean(); - - WriteRequest req = - new WriteRequest() { - - @Override - public void requestDone(IOException ioex) { - if (ioex instanceof TestIOException) { - exceptionForwarded.set(true); - } - - synchronized (handlerCalled) { - handlerCalled.set(true); - handlerCalled.notifyAll(); - } + void testExceptionPropagationWriter() throws Exception { + // use atomic boolean as a boolean reference + final AtomicBoolean handlerCalled = new AtomicBoolean(); + final AtomicBoolean exceptionForwarded = new AtomicBoolean(); + + WriteRequest req = + new WriteRequest() { + + @Override + public void requestDone(IOException ioex) { + if (ioex instanceof TestIOException) { + exceptionForwarded.set(true); } - @Override - public void write() throws IOException { - throw new TestIOException(); + synchronized (handlerCalled) { + handlerCalled.set(true); + handlerCalled.notifyAll(); } - }; - - // test the read queue - RequestQueue rq = - ioManager.getWriteRequestQueue(ioManager.createChannel()); - rq.add(req); - - // wait until the asynchronous request has been handled - synchronized (handlerCalled) { - while (!handlerCalled.get()) { - handlerCalled.wait(); - } - } + } - assertTrue(exceptionForwarded.get()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + @Override + public void write() throws IOException { + throw new TestIOException(); + } + }; + + // test the read queue + RequestQueue rq = ioManager.getWriteRequestQueue(ioManager.createChannel()); + rq.add(req); + + // wait until the asynchronous request has been handled + synchronized (handlerCalled) { + while (!handlerCalled.get()) { + handlerCalled.wait(); + } } + + assertThat(exceptionForwarded).isTrue(); } @Test - public void testExceptionInCallbackRead() { - try { - final AtomicBoolean handlerCalled = new AtomicBoolean(); - - ReadRequest regularRequest = - new ReadRequest() { - - @Override - public void requestDone(IOException ioex) { - synchronized (handlerCalled) { - handlerCalled.set(true); - handlerCalled.notifyAll(); - } + void testExceptionInCallbackRead() throws Exception { + final AtomicBoolean handlerCalled = new AtomicBoolean(); + + ReadRequest regularRequest = + new ReadRequest() { + + @Override + public void requestDone(IOException ioex) { + synchronized (handlerCalled) { + handlerCalled.set(true); + handlerCalled.notifyAll(); } + } - @Override - public void read() {} - }; + @Override + public void read() {} + }; - ReadRequest exceptionThrower = - new ReadRequest() { + ReadRequest exceptionThrower = + new ReadRequest() { - @Override - public void requestDone(IOException ioex) { - throw new RuntimeException(); - } + @Override + public void requestDone(IOException ioex) { + throw new RuntimeException(); + } - @Override - public void read() {} - }; + @Override + public void read() {} + }; - RequestQueue rq = ioManager.getReadRequestQueue(ioManager.createChannel()); + RequestQueue rq = ioManager.getReadRequestQueue(ioManager.createChannel()); - // queue first an exception thrower, then a regular request. - // we check that the regular request gets successfully handled - rq.add(exceptionThrower); - rq.add(regularRequest); + // queue first an exception thrower, then a regular request. + // we check that the regular request gets successfully handled + rq.add(exceptionThrower); + rq.add(regularRequest); - synchronized (handlerCalled) { - while (!handlerCalled.get()) { - handlerCalled.wait(); - } + synchronized (handlerCalled) { + while (!handlerCalled.get()) { + handlerCalled.wait(); } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } } @Test - public void testExceptionInCallbackWrite() { - try { - final AtomicBoolean handlerCalled = new AtomicBoolean(); - - WriteRequest regularRequest = - new WriteRequest() { - - @Override - public void requestDone(IOException ioex) { - synchronized (handlerCalled) { - handlerCalled.set(true); - handlerCalled.notifyAll(); - } + void testExceptionInCallbackWrite() throws Exception { + final AtomicBoolean handlerCalled = new AtomicBoolean(); + + WriteRequest regularRequest = + new WriteRequest() { + + @Override + public void requestDone(IOException ioex) { + synchronized (handlerCalled) { + handlerCalled.set(true); + handlerCalled.notifyAll(); } + } - @Override - public void write() {} - }; + @Override + public void write() {} + }; - WriteRequest exceptionThrower = - new WriteRequest() { + WriteRequest exceptionThrower = + new WriteRequest() { - @Override - public void requestDone(IOException ioex) { - throw new RuntimeException(); - } + @Override + public void requestDone(IOException ioex) { + throw new RuntimeException(); + } - @Override - public void write() {} - }; + @Override + public void write() {} + }; - RequestQueue rq = - ioManager.getWriteRequestQueue(ioManager.createChannel()); + RequestQueue rq = ioManager.getWriteRequestQueue(ioManager.createChannel()); - // queue first an exception thrower, then a regular request. - // we check that the regular request gets successfully handled - rq.add(exceptionThrower); - rq.add(regularRequest); + // queue first an exception thrower, then a regular request. + // we check that the regular request gets successfully handled + rq.add(exceptionThrower); + rq.add(regularRequest); - synchronized (handlerCalled) { - while (!handlerCalled.get()) { - handlerCalled.wait(); - } + synchronized (handlerCalled) { + while (!handlerCalled.get()) { + handlerCalled.wait(); } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } } - final class TestIOException extends IOException { + private static final class TestIOException extends IOException { private static final long serialVersionUID = -814705441998024472L; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index 17ea323fa7ea2..ee7cff8f5ead8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -26,12 +26,10 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.io.File; @@ -39,8 +37,10 @@ import java.util.List; import java.util.Random; +import static org.assertj.core.api.Assertions.assertThat; + /** Integration test case for the I/O manager. */ -public class IOManagerITCase extends TestLogger { +class IOManagerITCase { private static final long SEED = 649180756312423613L; @@ -56,19 +56,19 @@ public class IOManagerITCase extends TestLogger { private MemoryManager memoryManager; - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); - Assert.assertTrue( - "Not all memory was returned to the memory manager in the test.", - memoryManager.verifyEmpty()); + assertThat(memoryManager.verifyEmpty()) + .withFailMessage("Not all memory was returned to the memory manager in the test.") + .isTrue(); memoryManager.shutdown(); memoryManager = null; } @@ -82,7 +82,7 @@ public void afterTest() throws Exception { */ @Test @SuppressWarnings("unchecked") - public void parallelChannelsTest() throws Exception { + void parallelChannelsTest() throws Exception { final Random rnd = new Random(SEED); final AbstractInvokable memOwner = new DummyInvokable(); @@ -141,27 +141,22 @@ public void parallelChannelsTest() throws Exception { try { while (true) { val.read(in); - int intValue = 0; - try { - intValue = Integer.parseInt(val.value); - } catch (NumberFormatException nfex) { - Assert.fail( - "Invalid value read from reader. Valid decimal number expected."); - } - Assert.assertEquals( - "Written and read values do not match during sequential read.", - nextVal, - intValue); + int intValue = Integer.parseInt(val.value); + + assertThat(intValue) + .withFailMessage( + "Written and read values do not match during sequential read.") + .isEqualTo(nextVal); nextVal++; } } catch (EOFException eofex) { // expected } - Assert.assertEquals( - "NUmber of written numbers differs from number of read numbers.", - writingCounters[i], - nextVal); + assertThat(nextVal) + .withFailMessage( + "NUmber of written numbers differs from number of read numbers.") + .isEqualTo(writingCounters[i]); this.memoryManager.release(in.close()); } @@ -185,19 +180,11 @@ public void parallelChannelsTest() throws Exception { if (ins[channel] != null) { try { val.read(ins[channel]); - int intValue; - try { - intValue = Integer.parseInt(val.value); - } catch (NumberFormatException nfex) { - Assert.fail( - "Invalid value read from reader. Valid decimal number expected."); - return; - } - - Assert.assertEquals( - "Written and read values do not match.", - readingCounters[channel]++, - intValue); + int intValue = Integer.parseInt(val.value); + + assertThat(intValue) + .withFailMessage("Written and read values do not match.") + .isEqualTo(readingCounters[channel]++); break; } catch (EOFException eofex) { @@ -222,7 +209,7 @@ public void parallelChannelsTest() throws Exception { // check that files are deleted for (int i = 0; i < NUM_CHANNELS; i++) { File f = new File(ids[i].getPath()); - Assert.assertFalse("Channel file has not been deleted.", f.exists()); + assertThat(f).withFailMessage("Channel file has not been deleted.").doesNotExist(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java index ce8a50cb30305..67a65d1f24ae0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java @@ -22,27 +22,20 @@ import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; -import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; -public class IOManagerTest { - - @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); +class IOManagerTest { @Test - public void channelEnumerator() throws Exception { - File tempPath = temporaryFolder.newFolder(); - + void channelEnumerator(@TempDir File tempPath) throws Exception { String[] tempDirs = new String[] { new File(tempPath, "a").getAbsolutePath(), @@ -59,24 +52,29 @@ public void channelEnumerator() throws Exception { for (int i = 0; i < 3 * tempDirs.length; i++) { FileIOChannel.ID id = enumerator.next(); - File path = id.getPathFile(); + File pathFile = id.getPathFile(); + Files.createFile(pathFile.toPath()); - assertTrue("Channel IDs must name an absolute path.", path.isAbsolute()); - assertFalse("Channel IDs must name a file, not a directory.", path.isDirectory()); + assertThat(pathFile) + .withFailMessage("Channel IDs must name an absolute path.") + .isAbsolute(); + assertThat(pathFile) + .withFailMessage("Channel IDs must name a file, not a directory.") + .isFile(); - assertTrue( - "Path is not in the temp directory.", - tempPath.equals(path.getParentFile().getParentFile().getParentFile())); + assertThat(pathFile.getParentFile().getParentFile().getParentFile()) + .withFailMessage("Path is not in the temp directory.") + .isEqualTo(tempPath); for (int k = 0; k < tempDirs.length; k++) { - if (path.getParentFile().getParent().equals(tempDirs[k])) { + if (pathFile.getParentFile().getParent().equals(tempDirs[k])) { counters[k]++; } } } for (int k = 0; k < tempDirs.length; k++) { - assertEquals(3, counters[k]); + assertThat(counters[k]).isEqualTo(3); } } } @@ -108,19 +106,19 @@ public BlockChannelReader createBlockChannelReader( } @Override - public BufferFileWriter createBufferFileWriter(ID channelID) throws IOException { + public BufferFileWriter createBufferFileWriter(ID channelID) { throw new UnsupportedOperationException(); } @Override public BufferFileReader createBufferFileReader( - ID channelID, RequestDoneCallback callback) throws IOException { + ID channelID, RequestDoneCallback callback) { throw new UnsupportedOperationException(); } @Override public BufferFileSegmentReader createBufferFileSegmentReader( - ID channelID, RequestDoneCallback callback) throws IOException { + ID channelID, RequestDoneCallback callback) { throw new UnsupportedOperationException(); } From 347e4ca6c265334a35969d1c8358ff5a9f066e92 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 25 Oct 2023 16:17:27 +0200 Subject: [PATCH 029/104] [FLINK-25809][table-api-java] Add table test program infrastructure --- .../test/program/ConfigOptionTestStep.java | 43 ++ .../table/test/program/FunctionTestStep.java | 75 +++ .../table/test/program/SinkTestStep.java | 55 ++ .../table/test/program/SourceTestStep.java | 52 ++ .../flink/table/test/program/SqlTestStep.java | 46 ++ .../test/program/StatementSetTestStep.java | 46 ++ .../table/test/program/TableTestProgram.java | 472 ++++++++++++++++++ .../test/program/TableTestProgramRunner.java | 112 +++++ .../table/test/program/TableTestStep.java | 73 +++ .../flink/table/test/program/TestStep.java | 54 ++ .../program/TableTestProgramRunnerTest.java | 182 +++++++ 11 files changed, 1210 insertions(+) create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/ConfigOptionTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FunctionTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SqlTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/StatementSetTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunner.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TestStep.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/ConfigOptionTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/ConfigOptionTestStep.java new file mode 100644 index 0000000000000..7f66122f1cd99 --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/ConfigOptionTestStep.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.api.TableEnvironment; + +/** Test step for setting a {@link ConfigOption}. */ +public final class ConfigOptionTestStep implements TestStep { + + public final ConfigOption option; + public final T value; + + ConfigOptionTestStep(ConfigOption option, T value) { + this.option = option; + this.value = value; + } + + @Override + public TestKind getKind() { + return TestKind.CONFIG; + } + + public void apply(TableEnvironment env) { + env.getConfig().set(option, value); + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FunctionTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FunctionTestStep.java new file mode 100644 index 0000000000000..ad377bae4cda5 --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FunctionTestStep.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.functions.UserDefinedFunction; + +/** Test step for registering a (temporary) (system or catalog) function. */ +public final class FunctionTestStep implements TestStep { + + /** Whether function should be temporary or not. */ + enum FunctionPersistence { + TEMPORARY, + PERSISTENT + } + + /** Whether function should be persisted in a catalog or not. */ + enum FunctionBehavior { + SYSTEM, + CATALOG + } + + public final FunctionPersistence persistence; + public final FunctionBehavior behavior; + public final String name; + public final Class function; + + FunctionTestStep( + FunctionPersistence persistence, + FunctionBehavior behavior, + String name, + Class function) { + this.persistence = persistence; + this.behavior = behavior; + this.name = name; + this.function = function; + } + + @Override + public TestKind getKind() { + return TestKind.FUNCTION; + } + + public void apply(TableEnvironment env) { + if (behavior == FunctionBehavior.SYSTEM) { + if (persistence == FunctionPersistence.TEMPORARY) { + env.createTemporarySystemFunction(name, function); + } else { + throw new UnsupportedOperationException("System functions must be temporary."); + } + } else { + if (persistence == FunctionPersistence.TEMPORARY) { + env.createTemporaryFunction(name, function); + } else { + env.createFunction(name, function); + } + } + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java new file mode 100644 index 0000000000000..42bfbb9da87cf --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.types.Row; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; + +/** Test step for creating a table sink. */ +public final class SinkTestStep extends TableTestStep { + + public final @Nullable Predicate> expectedBeforeRestore; + public final @Nullable Predicate> expectedAfterRestore; + + SinkTestStep( + String name, + List schemaComponents, + List partitionKeys, + Map options, + @Nullable Predicate> expectedBeforeRestore, + @Nullable Predicate> expectedAfterRestore) { + super(name, schemaComponents, partitionKeys, options); + this.expectedBeforeRestore = expectedBeforeRestore; + this.expectedAfterRestore = expectedAfterRestore; + } + + @Override + public TestKind getKind() { + return expectedBeforeRestore == null + ? TestKind.SINK_WITHOUT_DATA + : expectedAfterRestore == null + ? TestKind.SINK_WITH_DATA + : TestKind.SINK_WITH_RESTORE_DATA; + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java new file mode 100644 index 0000000000000..eec3b1677b010 --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; + +/** Test step for creating a table source. */ +public final class SourceTestStep extends TableTestStep { + + public final List dataBeforeRestore; + public final List dataAfterRestore; + + SourceTestStep( + String name, + List schemaComponents, + List partitionKeys, + Map options, + List dataBeforeRestore, + List dataAfterRestore) { + super(name, schemaComponents, partitionKeys, options); + this.dataBeforeRestore = dataBeforeRestore; + this.dataAfterRestore = dataAfterRestore; + } + + @Override + public TestKind getKind() { + return dataBeforeRestore.isEmpty() + ? TestKind.SOURCE_WITHOUT_DATA + : dataAfterRestore.isEmpty() + ? TestKind.SOURCE_WITH_DATA + : TestKind.SOURCE_WITH_RESTORE_DATA; + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SqlTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SqlTestStep.java new file mode 100644 index 0000000000000..c6809dc16cfba --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SqlTestStep.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; + +/** + * Test step for execution SQL. + * + *

Note: Not every runner supports generic SQL statements. Sometimes the runner would like to + * enrich properties e.g. of a CREATE TABLE. Use this step with caution. + */ +public final class SqlTestStep implements TestStep { + + public final String sql; + + SqlTestStep(String sql) { + this.sql = sql; + } + + @Override + public TestKind getKind() { + return TestKind.SQL; + } + + public TableResult apply(TableEnvironment env) { + return env.executeSql(sql); + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/StatementSetTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/StatementSetTestStep.java new file mode 100644 index 0000000000000..98cf44ecc5ae1 --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/StatementSetTestStep.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; + +import java.util.List; + +/** Test step for creating a statement set. */ +public final class StatementSetTestStep implements TestStep { + + public final List statements; + + StatementSetTestStep(List statements) { + this.statements = statements; + } + + @Override + public TestKind getKind() { + return TestKind.STATEMENT_SET; + } + + public TableResult apply(TableEnvironment env) { + final StatementSet statementSet = env.createStatementSet(); + statements.forEach(statementSet::addInsertSql); + return statementSet.execute(); + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java new file mode 100644 index 0000000000000..1b194b39ddd28 --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -0,0 +1,472 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.table.test.program; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.test.program.FunctionTestStep.FunctionBehavior; +import org.apache.flink.table.test.program.FunctionTestStep.FunctionPersistence; +import org.apache.flink.table.test.program.TestStep.TestKind; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.collections.CollectionUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * A generic declaration of a table program for testing. + * + *

A test program defines the basic test pipeline (from source to sink) and required artifacts + * such as table sources and sinks, configuration options, and user-defined functions. Because some + * programs need to create artifacts in a certain order, a program consists of individual {@link + * TestStep}s for setting up the test and the actual running of the test. + * + *

Tests programs are intended to reduce code duplication and test the same SQL statement though + * different layers of the stack. Different {@link TableTestProgramRunner}s can share the same + * program and enrich it with custom implementation and assertions. + * + *

For example, a SQL query such as {@code SELECT * FROM (VALUES (1), (2), (3))} can be declared + * once and can be shared among different tests for integration testing, optimizer plan testing, + * compiled plan testing, transformation testing, and others. + * + *

A typical implementation looks like: + * + *

{@code
+ * // Define the behavior and configuration of an operation.
+ * public class CalcTestPrograms {
+ *     public static final TableTestProgram CALC_SIMPLE = TableTestProgram.of("calc-simple") ...;
+ *     public static final TableTestProgram CALC_COMPLEX = TableTestProgram.of("calc-complex") ...;
+ * }
+ *
+ * // Define a test base for example for plan testing
+ * public abstract class PlanTestBase implements TableTestProgramRunner {
+ *     // The test base declares what kind of steps it can apply.
+ *     public Set supportedSetupSteps() { return EnumSet.of(SOURCE_WITH_DATA, SINK_WITH_DATA); }
+ *     public Set supportedRunSteps() { return EnumSet.of(SQL); }
+ *
+ *     // Leave the list of programs up to the concrete test
+ *     public abstract List programs();
+ *
+ *     @ParameterizedTest
+ *     @MethodSource("supportedPrograms")
+ *     public void test(TableTestProgram program) {
+ *         TableEnvironment env = ...;
+ *         program.getSetupSourceTestSteps().forEach(s -> s.apply(env));
+ *         program.getSetupSinkTestSteps().forEach(s -> s.apply(env));
+ *         assertThat(program.getRunSqlTestStep().apply(env)).contains(...);
+ *     }
+ * }
+ *
+ * // Run the test base for a category of test programs.
+ * public class CalcPlanTest extends PlanTestBase {
+ *     public List programs() = { return Arrays.asList(CALC_SIMPLE, CALC_COMPLEX); }
+ * }
+ * }
+ */ +public class TableTestProgram { + + /** Identifier of the test program (e.g. for naming generated files). */ + public final String id; + + /** Description for internal documentation. */ + public final String description; + + /** Steps to be executed for setting up an environment. */ + public final List setupSteps; + + /** Steps to be executed for running the actual test. */ + public final List runSteps; + + private TableTestProgram( + String id, String description, List setupSteps, List runSteps) { + this.id = id; + this.description = description; + this.setupSteps = setupSteps; + this.runSteps = runSteps; + } + + /** + * Entrypoint for a {@link TableTestProgram} that forces an identifier and description of the + * test program. + * + *

The identifier is necessary to (ideally globally) identify the test program in outputs. + * For example, a runner for plan tests can create directories and use the name as file names. + * + *

The description should give more context and should start with a verb and "s" suffix. + * + *

For example: + * + *

    + *
  • TableTestProgram.of("join-outer", "tests outer joins") + *
  • TableTestProgram.of("rank-x-enabled", "validates a rank with config flag 'x' set") + *
  • TableTestProgram.of("calc-with-projection", "verifies FLINK-12345 is fixed due to + * missing row projection") + *
+ */ + public static Builder of(String id, String description) { + return new Builder(id, description); + } + + /** Convenience method to avoid casting. It assumes that the order of steps is not important. */ + public List getSetupSourceTestSteps() { + final EnumSet sourceKinds = + EnumSet.of( + TestKind.SOURCE_WITHOUT_DATA, + TestKind.SOURCE_WITH_DATA, + TestKind.SOURCE_WITH_RESTORE_DATA); + return setupSteps.stream() + .filter(s -> sourceKinds.contains(s.getKind())) + .map(SourceTestStep.class::cast) + .collect(Collectors.toList()); + } + + /** Convenience method to avoid casting. It assumes that the order of steps is not important. */ + public List getSetupSinkTestSteps() { + final EnumSet sinkKinds = + EnumSet.of( + TestKind.SINK_WITHOUT_DATA, + TestKind.SINK_WITH_DATA, + TestKind.SINK_WITH_RESTORE_DATA); + return setupSteps.stream() + .filter(s -> sinkKinds.contains(s.getKind())) + .map(SinkTestStep.class::cast) + .collect(Collectors.toList()); + } + + /** Convenience method to avoid casting. It assumes that the order of steps is not important. */ + public List> getSetupConfigOptionTestSteps() { + return setupSteps.stream() + .filter(s -> s.getKind() == TestKind.CONFIG) + .map(s -> (ConfigOptionTestStep) s) + .collect(Collectors.toList()); + } + + /** Convenience method to avoid casting. It assumes that the order of steps is not important. */ + public List getSetupFunctionTestSteps() { + return setupSteps.stream() + .filter(s -> s.getKind() == TestKind.FUNCTION) + .map(FunctionTestStep.class::cast) + .collect(Collectors.toList()); + } + + /** + * Convenience method to avoid boilerplate code. It assumes that only a single SQL statement is + * tested. + */ + public SqlTestStep getRunSqlTestStep() { + Preconditions.checkArgument( + runSteps.size() == 1 && runSteps.get(0).getKind() == TestKind.SQL, + "Single SQL step expected."); + return (SqlTestStep) runSteps.get(0); + } + + /** Builder pattern for {@link TableTestProgram}. */ + public static class Builder { + + private final String id; + private final String description; + private final List setupSteps = new ArrayList<>(); + private final List runSteps = new ArrayList<>(); + + private Builder(String id, String description) { + this.id = id; + this.description = description; + } + + /** + * Setup step for execution SQL. + * + *

Note: Not every runner supports generic SQL statements. Sometimes the runner would + * like to enrich properties e.g. of a CREATE TABLE. Use this step with caution. + */ + public Builder setupSql(String sql) { + this.setupSteps.add(new SqlTestStep(sql)); + return this; + } + + /** Setup step for setting a {@link ConfigOption}. */ + public Builder setupConfig(ConfigOption option, T value) { + this.setupSteps.add(new ConfigOptionTestStep<>(option, value)); + return this; + } + + /** Setup step for registering a temporary system function. */ + public Builder setupTemporarySystemFunction( + String name, Class function) { + this.setupSteps.add( + new FunctionTestStep( + FunctionPersistence.TEMPORARY, + FunctionBehavior.SYSTEM, + name, + function)); + return this; + } + + /** Setup step for registering a temporary catalog function. */ + public Builder setupTemporaryCatalogFunction( + String name, Class function) { + this.setupSteps.add( + new FunctionTestStep( + FunctionPersistence.TEMPORARY, + FunctionBehavior.CATALOG, + name, + function)); + return this; + } + + /** Setup step for registering a catalog function. */ + public Builder setupCatalogFunction( + String name, Class function) { + this.setupSteps.add( + new FunctionTestStep( + FunctionPersistence.PERSISTENT, + FunctionBehavior.CATALOG, + name, + function)); + return this; + } + + /** Setup step for building a table source. */ + public SourceBuilder setupTableSource(String name) { + return new SourceBuilder(name, setupSteps, this); + } + + /** Setup step for building a table sink. */ + public SinkBuilder setupTableSink(String name) { + return new SinkBuilder(name, setupSteps, this); + } + + /** Run step for executing SQL. */ + public Builder runSql(String sql) { + this.runSteps.add(new SqlTestStep(sql)); + return this; + } + + /** Run step for executing a statement set. */ + public StatementSetBuilder runStatementSet() { + return new StatementSetBuilder(runSteps, this); + } + + public TableTestProgram build() { + return new TableTestProgram(id, description, setupSteps, runSteps); + } + } + + /** Builder pattern for {@link SourceTestStep} and {@link SinkTestStep}. */ + @SuppressWarnings("unchecked") + private static class TableBuilder> { + + protected final String name; + protected final List targetSteps; + protected final Builder rootBuilder; + + protected final List schemaComponents = new ArrayList<>(); + protected final List partitionKeys = new ArrayList<>(); + protected final Map options = new HashMap<>(); + + private TableBuilder(String name, List targetSteps, Builder rootBuilder) { + this.name = name; + this.targetSteps = targetSteps; + this.rootBuilder = rootBuilder; + } + + /** + * Define the schema like you would in SQL e.g. "my_col INT", "PRIMARY KEY (uid) NOT + * ENFORCED", or "WATERMARK FOR ts AS ts". + */ + public SpecificBuilder withSchema(String... schemaComponents) { + this.schemaComponents.addAll(Arrays.asList(schemaComponents)); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder withOptions(Map options) { + this.options.putAll(options); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder withOption(String key, String value) { + this.options.put(key, value); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder withOption(ConfigOption option, String value) { + this.options.put(option.key(), ConfigurationUtils.convertValue(value, String.class)); + return (SpecificBuilder) this; + } + + public SpecificBuilder withPartitionKeys(String... partitionKeys) { + this.partitionKeys.addAll(Arrays.asList(partitionKeys)); + return (SpecificBuilder) this; + } + } + + /** Builder pattern for {@link SourceTestStep}. */ + public static class SourceBuilder extends TableBuilder { + + private final List dataBeforeRestore = new ArrayList<>(); + private final List dataAfterRestore = new ArrayList<>(); + + private SourceBuilder(String name, List targetSteps, Builder rootBuilder) { + super(name, targetSteps, rootBuilder); + } + + public SourceBuilder withValues(Row... data) { + return withValuesBeforeRestore(data); + } + + public SourceBuilder withValuesBeforeRestore(Row... data) { + this.dataBeforeRestore.addAll(Arrays.asList(data)); + return this; + } + + public SourceBuilder withValuesAfterRestore(Row... data) { + this.dataAfterRestore.addAll(Arrays.asList(data)); + return this; + } + + public Builder complete() { + targetSteps.add( + new SourceTestStep( + name, + schemaComponents, + partitionKeys, + options, + dataBeforeRestore, + dataAfterRestore)); + return rootBuilder; + } + } + + /** Builder pattern for {@link SinkTestStep}. */ + public static class SinkBuilder extends TableBuilder { + + private Predicate> expectedBeforeRestore; + private Predicate> expectedAfterRestore; + + private SinkBuilder(String name, List targetSteps, Builder rootBuilder) { + super(name, targetSteps, rootBuilder); + } + + public SinkBuilder withExpectedValues(Row... expectedRows) { + return withValuesBeforeRestore(expectedRows); + } + + public SinkBuilder withExpectedValues(String... expectedRows) { + return withValuesBeforeRestore(expectedRows); + } + + public SinkBuilder withValuesBeforeRestore(Row... expectedRows) { + this.expectedBeforeRestore = equalIgnoringOrder(expectedRows); + return this; + } + + public SinkBuilder withValuesBeforeRestore(String... expectedRows) { + this.expectedBeforeRestore = equalIgnoringOrder(expectedRows); + return this; + } + + public SinkBuilder withValuesAfterRestore(Row... expectedRows) { + this.expectedAfterRestore = equalIgnoringOrder(expectedRows); + return this; + } + + public SinkBuilder withValuesAfterRestore(String... expectedRows) { + this.expectedAfterRestore = equalIgnoringOrder(expectedRows); + return this; + } + + private static Predicate> equalIgnoringOrder(Row... expectedRows) { + return (actualRows) -> { + if (actualRows.size() != expectedRows.length) { + return false; + } + return CollectionUtils.isEqualCollection(actualRows, Arrays.asList(expectedRows)); + }; + } + + private static Predicate> equalIgnoringOrder(String... expectedRows) { + return (actualRows) -> { + if (actualRows.size() != expectedRows.length) { + return false; + } + final List actualRowsString = + actualRows.stream().map(Row::toString).collect(Collectors.toList()); + return CollectionUtils.isEqualCollection( + actualRowsString, Arrays.asList(expectedRows)); + }; + } + + public Builder complete() { + targetSteps.add( + new SinkTestStep( + name, + schemaComponents, + partitionKeys, + options, + expectedBeforeRestore, + expectedAfterRestore)); + return rootBuilder; + } + } + + /** Builder pattern for {@link StatementSetTestStep}. */ + public static class StatementSetBuilder { + + private final List targetSteps; + private final Builder rootBuilder; + private final List statements = new ArrayList<>(); + + private StatementSetBuilder(List targetSteps, Builder rootBuilder) { + this.targetSteps = targetSteps; + this.rootBuilder = rootBuilder; + } + + public StatementSetBuilder withSql(String sql) { + this.statements.add(sql); + return this; + } + + public Builder complete() { + this.targetSteps.add(new StatementSetTestStep(statements)); + return rootBuilder; + } + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunner.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunner.java new file mode 100644 index 0000000000000..cd874739135ee --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunner.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.table.test.program; + +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Interface for test bases that want to run lists of {@link TableTestProgram}s. + * + *

NOTE: See {@link TableTestProgram} for a full example. + * + *

Use {@link #supportedPrograms()} for assertions (usually in test bases), and {@link + * #programs()} for program lists (usually in final tests). + */ +public interface TableTestProgramRunner { + + /** + * List of {@link TableTestProgram}s that this runner should run. + * + *

Usually, this list should reference some test programs stored in static variables that can + * be shared across runners. + */ + List programs(); + + /** + * Runners should call this method to get started. + * + *

Compared to {@link #programs()}, this method will perform some pre-checks. + */ + default List supportedPrograms() { + final List programs = programs(); + + final List ids = programs.stream().map(p -> p.id).collect(Collectors.toList()); + final List duplicates = + ids.stream() + .filter(id -> Collections.frequency(ids, id) > 1) + .distinct() + .collect(Collectors.toList()); + if (!duplicates.isEmpty()) { + throw new IllegalArgumentException("Duplicate test program id found: " + duplicates); + } + + final Set setupSteps = supportedSetupSteps(); + final Set runSteps = supportedRunSteps(); + + programs.forEach( + p -> { + p.setupSteps.stream() + .map(TestStep::getKind) + .filter(k -> !setupSteps.contains(k)) + .findFirst() + .ifPresent( + k -> { + throw new UnsupportedOperationException( + "Test runner does not support setup step: " + k); + }); + p.runSteps.stream() + .map(TestStep::getKind) + .filter(k -> !runSteps.contains(k)) + .findFirst() + .ifPresent( + k -> { + throw new UnsupportedOperationException( + "Test runner does not support run step: " + k); + }); + }); + + return programs; + } + + /** + * Lists setup steps that are supported by this runner. + * + *

E.g. some runners might not want to run generic {@link TestStep.TestKind#SQL} because they + * want to enrich CREATE TABLE statements. + * + *

This also ensures that runners don't need to be updated when a new kind of step is added, + * or steps get silently dropped. + */ + EnumSet supportedSetupSteps(); + + /** + * Lists run steps that are supported by this runner. + * + *

E.g. some runners might not want to run generic {@link TestStep.TestKind#SQL} because they + * want to enrich CREATE TABLE statements. + * + *

This also ensures that runners don't need to be updated when a new kind of step is added, + * or steps get silently dropped. + */ + EnumSet supportedRunSteps(); +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java new file mode 100644 index 0000000000000..bff700d9a159b --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** Abstract class for {@link SourceTestStep} and {@link SinkTestStep}. */ +public abstract class TableTestStep implements TestStep { + + public final String name; + public final List schemaComponents; + public final List partitionKeys; + public final Map options; + + TableTestStep( + String name, + List schemaComponents, + List partitionKeys, + Map options) { + this.name = name; + this.schemaComponents = schemaComponents; + this.partitionKeys = partitionKeys; + this.options = options; + } + + public TableResult apply(TableEnvironment env) { + return apply(env, Collections.emptyMap()); + } + + public TableResult apply(TableEnvironment env, Map extraOptions) { + final Map allOptions = new HashMap<>(options); + allOptions.putAll(extraOptions); + + final String partitionedBy = + partitionKeys.isEmpty() + ? "" + : "PARTITIONED BY (" + String.join(", ", partitionKeys) + ")\n"; + final String createTable = + String.format( + "CREATE TABLE %s (\n%s)\n%sWITH (\n%s)", + name, + String.join(",\n", schemaComponents), + partitionedBy, + allOptions.entrySet().stream() + .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",\n"))); + + return env.executeSql(createTable); + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TestStep.java new file mode 100644 index 0000000000000..67105b68d53db --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TestStep.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.test.program; + +/** + * Test step that makes up a {@link TableTestProgram}. + * + *

It describes a task that should be executed either before running the actual test or as the + * main ingredient of the test. + * + *

Some steps provide {@code apply()} methods for convenience. But in the end, the {@link + * TableTestProgramRunner} decides whether to call them or not. + * + *

Not every {@link TableTestProgramRunner} might support every {@link TestKind}. + */ +public interface TestStep { + + /** + * Enum to identify important properties of a {@link TestStep}. + * + *

Used in {@link TableTestProgramRunner#supportedSetupSteps()} and {@link + * TableTestProgramRunner#supportedRunSteps()}. + */ + enum TestKind { + SQL, + STATEMENT_SET, + CONFIG, + FUNCTION, + SOURCE_WITHOUT_DATA, + SOURCE_WITH_DATA, + SOURCE_WITH_RESTORE_DATA, + SINK_WITHOUT_DATA, + SINK_WITH_DATA, + SINK_WITH_RESTORE_DATA, + } + + TestKind getKind(); +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java new file mode 100644 index 0000000000000..e7b58aaa86439 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.table.test.program; + +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.test.program.TestStep.TestKind; +import org.apache.flink.table.utils.UserDefinedFunctions; + +import org.junit.jupiter.api.Test; + +import java.time.ZoneId; +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link TableTestProgram} and {@link TableTestProgramRunner}. */ +public class TableTestProgramRunnerTest { + + private static final String ID = "id"; + private static final String DESCRIPTION = "description"; + + @Test + void testConfigStep() { + final TableTestProgram program = + TableTestProgram.of(ID, DESCRIPTION) + .setupConfig(TableConfigOptions.LOCAL_TIME_ZONE, "GMT+3") + .build(); + + assertThat(program.setupSteps).hasSize(1); + + final TableEnvironment env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + program.getSetupConfigOptionTestSteps().forEach(s -> s.apply(env)); + assertThat(env.getConfig().getLocalTimeZone()).isEqualTo(ZoneId.of("GMT+3")); + } + + @Test + void testFunctionStep() { + final TableTestProgram program = + TableTestProgram.of(ID, DESCRIPTION) + .setupTemporarySystemFunction( + "tmp_sys", UserDefinedFunctions.ScalarUDF.class) + .setupTemporaryCatalogFunction( + "tmp_cat", UserDefinedFunctions.ScalarUDF.class) + .setupCatalogFunction("cat", UserDefinedFunctions.ScalarUDF.class) + .build(); + + assertThat(program.setupSteps).hasSize(3); + + final TableEnvironment env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + program.getSetupFunctionTestSteps().forEach(s -> s.apply(env)); + + assertThat(env.listUserDefinedFunctions()).contains("tmp_sys", "tmp_cat", "cat"); + } + + @Test + void testSqlStep() { + final TableTestProgram program = + TableTestProgram.of(ID, DESCRIPTION) + .setupSql("CREATE TABLE MyTable1 (i INT) WITH ('connector' = 'datagen')") + .runSql("CREATE TABLE MyTable2 (i INT) WITH ('connector' = 'datagen')") + .build(); + + assertThat(program.setupSteps).hasSize(1); + assertThat(program.runSteps).hasSize(1); + + final TableEnvironment env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + program.setupSteps.stream().map(SqlTestStep.class::cast).forEach(s -> s.apply(env)); + program.runSteps.stream().map(SqlTestStep.class::cast).forEach(s -> s.apply(env)); + + assertThat(env.listTables()).contains("MyTable1", "MyTable2"); + } + + @Test + @SuppressWarnings("resource") + void testTableStep() { + final TableTestProgram program = + TableTestProgram.of(ID, DESCRIPTION) + .setupTableSource("MyTableSource") + .withSchema("i INT") + .withOption("connector", "datagen") + .complete() + .setupTableSource("MyTableSink") + .withSchema("i INT") + .withOption("connector", "blackhole") + .complete() + .build(); + + assertThat(program.setupSteps).hasSize(2); + + final TableEnvironment env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + program.getSetupSourceTestSteps() + .forEach(s -> s.apply(env, Collections.singletonMap("number-of-rows", "3"))); + program.getSetupSinkTestSteps().forEach(s -> s.apply(env)); + + assertThat(env.executeSql("SHOW CREATE TABLE MyTableSource").collect().next().getField(0)) + .isEqualTo( + "CREATE TABLE `default_catalog`.`default_database`.`MyTableSource` (\n" + + " `i` INT\n" + + ") WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'number-of-rows' = '3'\n" + + ")\n"); + assertThat(env.executeSql("SHOW CREATE TABLE MyTableSink").collect().next().getField(0)) + .isEqualTo( + "CREATE TABLE `default_catalog`.`default_database`.`MyTableSink` (\n" + + " `i` INT\n" + + ") WITH (\n" + + " 'connector' = 'blackhole',\n" + + " 'number-of-rows' = '3'\n" + + ")\n"); + } + + @Test + void testRunnerValidationDuplicate() { + final TableTestProgram program1 = + TableTestProgram.of(ID, DESCRIPTION).runSql("SELECT 1").build(); + + final TableTestProgram program2 = + TableTestProgram.of(ID, DESCRIPTION).runSql("SELECT 1").build(); + + final LimitedTableTestProgramRunner runner = new LimitedTableTestProgramRunner(); + runner.programs = Arrays.asList(program1, program2); + + assertThatThrownBy(runner::supportedPrograms) + .hasMessageContaining("Duplicate test program id found: [id]"); + } + + @Test + void testRunnerValidationUnsupported() { + final LimitedTableTestProgramRunner runner = new LimitedTableTestProgramRunner(); + + final TableTestProgram program = + TableTestProgram.of(ID, DESCRIPTION).setupSql("SELECT 1").build(); + + runner.programs = Collections.singletonList(program); + + assertThatThrownBy(runner::supportedPrograms) + .hasMessageContaining("Test runner does not support setup step: SQL"); + } + + private static class LimitedTableTestProgramRunner implements TableTestProgramRunner { + + List programs; + + @Override + public List programs() { + return programs; + } + + @Override + public EnumSet supportedSetupSteps() { + return EnumSet.of(TestKind.SOURCE_WITH_DATA); + } + + @Override + public EnumSet supportedRunSteps() { + return EnumSet.of(TestKind.SQL); + } + } +} From 0388b760fc66975c70f797ad07f2e073738a7171 Mon Sep 17 00:00:00 2001 From: 1996fanrui <1996fanrui@gmail.com> Date: Thu, 26 Oct 2023 11:28:48 +0800 Subject: [PATCH 030/104] [FLINK-33316][runtime] Using SERIALIZED_UDF_CLASS instead of SERIALIZED_UDF_CLASS_NAME --- .../flink/streaming/api/graph/StreamConfig.java | 15 ++++++++++----- .../streaming/runtime/tasks/OperatorChain.java | 16 +++++++++++++--- 2 files changed, 23 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 2fb5b81d4a506..79a5e904ea98e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -83,7 +83,7 @@ public class StreamConfig implements Serializable { * Introduce serializedUdfClassName to avoid unnecessarily heavy {@link * #getStreamOperatorFactory}. */ - public static final String SERIALIZED_UDF_CLASS_NAME = "serializedUdfClassName"; + public static final String SERIALIZED_UDF_CLASS = "serializedUdfClass"; private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs"; private static final String NUMBER_OF_NETWORK_INPUTS = "numberOfNetworkInputs"; @@ -374,7 +374,7 @@ public void setStreamOperator(StreamOperator operator) { public void setStreamOperatorFactory(StreamOperatorFactory factory) { if (factory != null) { toBeSerializedConfigObjects.put(SERIALIZED_UDF, factory); - config.setString(SERIALIZED_UDF_CLASS_NAME, factory.getClass().getName()); + toBeSerializedConfigObjects.put(SERIALIZED_UDF_CLASS, factory.getClass()); } } @@ -406,8 +406,13 @@ public > T getStreamOperatorFactory(ClassLoad } } - public String getStreamOperatorFactoryClassName() { - return config.getString(SERIALIZED_UDF_CLASS_NAME, null); + public > Class getStreamOperatorFactoryClass( + ClassLoader cl) { + try { + return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZED_UDF_CLASS, cl); + } catch (Exception e) { + throw new StreamTaskException("Could not instantiate serialized udf class.", e); + } } public void setIterationId(String iterationId) { @@ -768,7 +773,7 @@ public String toString() { try { builder.append("\nOperator: ") - .append(getStreamOperatorFactory(cl).getClass().getSimpleName()); + .append(getStreamOperatorFactoryClass(cl).getSimpleName()); } catch (Exception e) { builder.append("\nOperator: Missing"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 1afee0f75b1f3..2a7a8dc3c1b54 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -641,7 +641,10 @@ private Map createChainedSources( @Nullable private Counter getOperatorRecordsOutCounter( StreamTask containingTask, StreamConfig operatorConfig) { - String streamOperatorFactoryClassName = operatorConfig.getStreamOperatorFactoryClassName(); + ClassLoader userCodeClassloader = containingTask.getUserCodeClassLoader(); + Class> streamOperatorFactoryClass = + operatorConfig.getStreamOperatorFactoryClass(userCodeClassloader); + // Do not use the numRecordsOut counter on output if this operator is SinkWriterOperator. // // Metric "numRecordsOut" is defined as the total number of records written to the @@ -649,8 +652,15 @@ private Counter getOperatorRecordsOutCounter( // number of records sent to downstream operators, which is number of Committable batches // sent to SinkCommitter. So we skip registering this metric on output and leave this metric // to sink writer implementations to report. - if (SinkWriterOperatorFactory.class.getName().equals(streamOperatorFactoryClassName)) { - return null; + try { + Class sinkWriterFactoryClass = + userCodeClassloader.loadClass(SinkWriterOperatorFactory.class.getName()); + if (sinkWriterFactoryClass.isAssignableFrom(streamOperatorFactoryClass)) { + return null; + } + } catch (ClassNotFoundException e) { + throw new StreamTaskException( + "Could not load SinkWriterOperatorFactory class from userCodeClassloader.", e); } InternalOperatorMetricGroup operatorMetricGroup = From f31770fcf5769052f1ac32a6529de979eaf339a4 Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Fri, 22 Sep 2023 14:35:19 -0400 Subject: [PATCH 031/104] [FLINK-33128] Add converter.open() method on TestValuesRuntimeFunctions This closes #23453. --- .../factories/TestValuesRuntimeFunctions.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java index a381d573c41d5..3ab136d451f42 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.connector.RuntimeConverter; import org.apache.flink.table.connector.sink.DynamicTableSink.DataStructureConverter; import org.apache.flink.table.connector.source.LookupTableSource; import org.apache.flink.table.data.GenericRowData; @@ -631,11 +632,11 @@ protected TestValuesLookupFunction( public void open(FunctionContext context) throws Exception { RESOURCE_COUNTER.incrementAndGet(); isOpenCalled = true; + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); if (projectable) { - projection = - generatedProjection.newInstance( - Thread.currentThread().getContextClassLoader()); + projection = generatedProjection.newInstance(classLoader); } + converter.open(RuntimeConverter.Context.create(classLoader)); rowSerializer = InternalSerializers.create(producedRowType); indexDataByKey(); } @@ -725,11 +726,11 @@ protected AsyncTestValueLookupFunction( @Override public void open(FunctionContext context) throws Exception { RESOURCE_COUNTER.incrementAndGet(); + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); if (projectable) { - projection = - generatedProjection.newInstance( - Thread.currentThread().getContextClassLoader()); + projection = generatedProjection.newInstance(classLoader); } + converter.open(RuntimeConverter.Context.create(classLoader)); rowSerializer = InternalSerializers.create(producedRowType); isOpenCalled = true; // generate unordered result for async lookup From f68ae194a6fda868c4d3ff3e3027cfc519c9017a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 27 Oct 2023 10:23:42 +0200 Subject: [PATCH 032/104] [FLINK-33369] Use Java 17 docker image for e2e tests on Java 17 --- flink-end-to-end-tests/test-scripts/common_docker.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh index 43c387c1fd8cd..e38f20359babf 100644 --- a/flink-end-to-end-tests/test-scripts/common_docker.sh +++ b/flink-end-to-end-tests/test-scripts/common_docker.sh @@ -54,6 +54,9 @@ function build_image() { if [[ ${PROFILE} == *"jdk11"* ]]; then java_version=11 fi + if [[ ${PROFILE} == *"jdk17"* ]]; then + java_version=17 + fi cd flink-docker ./add-custom.sh -u ${file_server_address}:9999/flink.tgz -n ${image_name} -j ${java_version} From 838fe2fafd3f19b99aaa879c5fdc680b85571684 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 24 Oct 2023 11:33:34 +0800 Subject: [PATCH 033/104] [FLINK-27432][runtime] Replace Time with Duration in TaskSlotTable --- .../TaskManagerConfiguration.java | 10 +++---- .../taskexecutor/slot/TaskSlotTable.java | 8 +++--- .../taskexecutor/slot/TaskSlotTableImpl.java | 13 ++++----- ...TaskExecutorOperatorEventHandlingTest.java | 3 ++- .../TaskExecutorSubmissionTest.java | 27 ++++++++++--------- .../taskexecutor/TaskExecutorTest.java | 4 +-- .../slot/TaskSlotTableImplTest.java | 8 +++--- .../slot/TestingTaskSlotTable.java | 8 +++--- .../slot/ThreadSafeTaskSlotTable.java | 7 +++-- 9 files changed, 44 insertions(+), 44 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index c3c76220c1050..ca7aaabe813c6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -53,7 +52,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { private final Duration rpcTimeout; - private final Time slotTimeout; + private final Duration slotTimeout; // null indicates an infinite duration @Nullable private final Duration maxRegistrationDuration; @@ -80,7 +79,7 @@ public TaskManagerConfiguration( ResourceProfile totalResourceProfile, String[] tmpDirectories, Duration rpcTimeout, - Time slotTimeout, + Duration slotTimeout, @Nullable Duration maxRegistrationDuration, Configuration configuration, boolean exitJvmOnOutOfMemory, @@ -125,7 +124,7 @@ public Duration getRpcTimeout() { return rpcTimeout; } - public Time getSlotTimeout() { + public Duration getSlotTimeout() { return slotTimeout; } @@ -199,8 +198,7 @@ public static TaskManagerConfiguration fromConfiguration( LOG.debug("Messages have a max timeout of " + rpcTimeout); - final Time slotTimeout = - Time.milliseconds(configuration.get(TaskManagerOptions.SLOT_TIMEOUT).toMillis()); + final Duration slotTimeout = configuration.get(TaskManagerOptions.SLOT_TIMEOUT); Duration finiteRegistrationDuration; try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java index 045ffebc7a592..9594c5f009236 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -32,6 +31,7 @@ import javax.annotation.Nullable; +import java.time.Duration; import java.util.Iterator; import java.util.Set; import java.util.UUID; @@ -96,7 +96,7 @@ public interface TaskSlotTable * @return True if the task slot could be allocated; otherwise false */ @VisibleForTesting - boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout); + boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Duration slotTimeout); /** * Allocate the slot with the given index for the given job and allocation id. If negative index @@ -116,7 +116,7 @@ boolean allocateSlot( JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, - Time slotTimeout); + Duration slotTimeout); /** * Marks the slot under the given allocation id as active. If the slot could not be found, then @@ -137,7 +137,7 @@ boolean allocateSlot( * @throws SlotNotFoundException if the slot could not be found for the given allocation id * @return True if the slot could be marked inactive */ - boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) + boolean markSlotInactive(AllocationID allocationId, Duration slotTimeout) throws SlotNotFoundException; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java index d8ee8e0546339..1099edad8a6ef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -42,6 +41,7 @@ import javax.annotation.Nullable; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -54,6 +54,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; /** Default implementation of {@link TaskSlotTable}. */ @@ -274,7 +275,7 @@ public SlotReport createSlotReport(ResourceID resourceId) { @VisibleForTesting @Override public boolean allocateSlot( - int index, JobID jobId, AllocationID allocationId, Time slotTimeout) { + int index, JobID jobId, AllocationID allocationId, Duration slotTimeout) { return allocateSlot(index, jobId, allocationId, defaultSlotResourceProfile, slotTimeout); } @@ -284,7 +285,7 @@ public boolean allocateSlot( JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, - Time slotTimeout) { + Duration slotTimeout) { checkRunning(); Preconditions.checkArgument(requestedIndex < numberSlots); @@ -334,7 +335,7 @@ public boolean allocateSlot( allocatedSlots.put(allocationId, taskSlot); // register a timeout for this slot since it's in state allocated - timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit()); + timerService.registerTimeout(allocationId, slotTimeout.toMillis(), TimeUnit.MILLISECONDS); // add this slot to the set of job slots Set slots = slotsPerJob.get(jobId); @@ -398,7 +399,7 @@ private boolean markExistingSlotActive(TaskSlot taskSlot) { } @Override - public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) + public boolean markSlotInactive(AllocationID allocationId, Duration slotTimeout) throws SlotNotFoundException { checkStarted(); @@ -408,7 +409,7 @@ public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) if (taskSlot.markInactive()) { // register a timeout to free the slot timerService.registerTimeout( - allocationId, slotTimeout.getSize(), slotTimeout.getUnit()); + allocationId, slotTimeout.toMillis(), TimeUnit.MILLISECONDS); return true; } else { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java index 5fd5bef8e405a..b3d71a4a3c5ca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java @@ -179,7 +179,8 @@ private TaskSubmissionTestEnvironment createExecutorWithRunningTask( .build()) .build(EXECUTOR_EXTENSION.getExecutor()); - env.getTaskSlotTable().allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + env.getTaskSlotTable() + .allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); final TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); tmGateway.submitTask(tdd, env.getJobMasterId(), Time.seconds(10)).get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index 002e680c6f7c3..3fb4c2d3b3a5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -70,6 +70,7 @@ import java.io.IOException; import java.net.URL; +import java.time.Duration; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -124,7 +125,7 @@ void testTaskSubmission() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); @@ -153,7 +154,7 @@ void testSubmitTaskFailure() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); assertThatFuture(tmGateway.submitTask(tdd, env.getJobMasterId(), timeout)) .eventuallyFailsWith(ExecutionException.class) @@ -189,11 +190,11 @@ void testTaskSubmissionAndCancelling() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd1, env.getJobMasterId(), timeout).get(); task1RunningFuture.get(); - taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd2, env.getJobMasterId(), timeout).get(); task2RunningFuture.get(); @@ -248,11 +249,11 @@ void testGateChannelEdgeMismatch() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd1, env.getJobMasterId(), timeout).get(); task1RunningFuture.get(); - taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd2, env.getJobMasterId(), timeout).get(); task2RunningFuture.get(); @@ -308,11 +309,11 @@ void testRunJobWithForwardChannel() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd1, jobMasterId, timeout).get(); task1RunningFuture.get(); - taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd2, jobMasterId, timeout).get(); task2RunningFuture.get(); @@ -387,11 +388,11 @@ void testCancellingDependentAndStateUpdateFails() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd1.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd1, jobMasterId, timeout).get(); task1RunningFuture.get(); - taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(1, jobId, tdd2.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd2, jobMasterId, timeout).get(); task2RunningFuture.get(); @@ -443,7 +444,7 @@ void testRemotePartitionNotFound() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); @@ -478,7 +479,7 @@ void testUpdateTaskInputPartitionsFailure() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); @@ -536,7 +537,7 @@ void testLocalPartitionNotFound() throws Exception { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); - taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); + taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Duration.ofSeconds(60)); tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 6719c33a81c42..25ae5986d567f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -2942,7 +2942,7 @@ private AllocateSlotNotifyingTaskSlotTable(OneShotLatch allocateSlotLatch) { @Override public boolean allocateSlot( - int index, JobID jobId, AllocationID allocationId, Time slotTimeout) { + int index, JobID jobId, AllocationID allocationId, Duration slotTimeout) { final boolean result = super.allocateSlot(index, jobId, allocationId, slotTimeout); allocateSlotLatch.trigger(); @@ -2955,7 +2955,7 @@ public boolean allocateSlot( JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, - Time slotTimeout) { + Duration slotTimeout) { final boolean result = super.allocateSlot(index, jobId, allocationId, resourceProfile, slotTimeout); allocateSlotLatch.trigger(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java index 7e61b06ded82c..6cd166cc5ace8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.taskexecutor.slot; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -38,6 +37,7 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; @@ -57,7 +57,7 @@ class TaskSlotTableImplTest { private static final TestExecutorExtension EXECUTOR_EXTENSION = TestingUtils.defaultExecutorExtension(); - private static final Time SLOT_TIMEOUT = Time.seconds(100L); + private static final Duration SLOT_TIMEOUT = Duration.ofSeconds(100L); /** Tests that one can can mark allocated slots as active. */ @Test @@ -455,7 +455,7 @@ void testAllocatedSlotTimeout() throws Exception { final AllocationID allocationId = new AllocationID(); assertThat( taskSlotTable.allocateSlot( - 0, new JobID(), allocationId, Time.milliseconds(1L))) + 0, new JobID(), allocationId, Duration.ofMillis(1L))) .isTrue(); assertThatFuture(timeoutFuture).eventuallySucceeds().isEqualTo(allocationId); } @@ -495,7 +495,7 @@ private void runDeactivateSlotTimeoutTest( final JobID jobId = new JobID(); assertThat( taskSlotTable.allocateSlot( - 0, jobId, allocationId, Time.milliseconds(timeout))) + 0, jobId, allocationId, Duration.ofMillis(timeout))) .isTrue(); assertThat(taskSlotTableAction.apply(taskSlotTable, jobId, allocationId)).isTrue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TestingTaskSlotTable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TestingTaskSlotTable.java index 2dbda2f898be1..5408e8b437321 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TestingTaskSlotTable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TestingTaskSlotTable.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.taskexecutor.slot; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -31,6 +30,7 @@ import javax.annotation.Nullable; +import java.time.Duration; import java.util.Collections; import java.util.Iterator; import java.util.Set; @@ -99,7 +99,7 @@ public SlotReport createSlotReport(ResourceID resourceId) { @Override public boolean allocateSlot( - int index, JobID jobId, AllocationID allocationId, Time slotTimeout) { + int index, JobID jobId, AllocationID allocationId, Duration slotTimeout) { return allocateSlotSupplier.get(); } @@ -109,7 +109,7 @@ public boolean allocateSlot( JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, - Time slotTimeout) { + Duration slotTimeout) { return allocateSlotSupplier.get(); } @@ -119,7 +119,7 @@ public boolean markSlotActive(AllocationID allocationId) { } @Override - public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) { + public boolean markSlotInactive(AllocationID allocationId, Duration slotTimeout) { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/ThreadSafeTaskSlotTable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/ThreadSafeTaskSlotTable.java index 3623662af8fcc..32df96931e957 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/ThreadSafeTaskSlotTable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/ThreadSafeTaskSlotTable.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.taskexecutor.slot; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -99,7 +98,7 @@ public SlotReport createSlotReport(ResourceID resourceId) { @Override public boolean allocateSlot( - int index, JobID jobId, AllocationID allocationId, Time slotTimeout) { + int index, JobID jobId, AllocationID allocationId, Duration slotTimeout) { return callAsync(() -> taskSlotTable.allocateSlot(index, jobId, allocationId, slotTimeout)); } @@ -109,7 +108,7 @@ public boolean allocateSlot( JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, - Time slotTimeout) { + Duration slotTimeout) { return callAsync( () -> taskSlotTable.allocateSlot( @@ -122,7 +121,7 @@ public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundExce } @Override - public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) + public boolean markSlotInactive(AllocationID allocationId, Duration slotTimeout) throws SlotNotFoundException { return callAsync(() -> taskSlotTable.markSlotInactive(allocationId, slotTimeout)); } From e914eb7fc3f31286ed7e33cc93e7ffbca785b731 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 26 Oct 2023 12:53:07 +0200 Subject: [PATCH 034/104] [FLINK-33371] Make TestValues sinks return results as Rows --- .../factories/TestValuesRuntimeFunctions.java | 197 ++++++++++-------- .../factories/TestValuesTableFactory.java | 37 +++- .../planner/runtime/batch/sql/RTASITCase.java | 6 +- .../sql/join/AdaptiveHashJoinITCase.java | 2 +- .../ChangelogSourceJsonPlanITCase.java | 4 +- ...figureOperatorLevelStateTtlJsonITCase.java | 4 +- .../jsonplan/CorrelateJsonPlanITCase.java | 12 +- .../jsonplan/DeduplicationJsonPlanITCase.java | 2 +- .../stream/jsonplan/ExpandJsonPlanITCase.java | 2 +- .../GroupAggregateJsonPlanITCase.java | 8 +- .../GroupWindowAggregateJsonITCase.java | 6 +- .../IncrementalAggregateJsonPlanITCase.java | 2 +- .../jsonplan/IntervalJoinJsonPlanITCase.java | 4 +- .../stream/jsonplan/JoinJsonPlanITCase.java | 10 +- .../stream/jsonplan/LimitJsonPlanITCase.java | 2 +- .../jsonplan/LookupJoinJsonPlanITCase.java | 6 +- .../MatchRecognizeJsonPlanITCase.java | 4 +- .../jsonplan/OverAggregateJsonPlanITCase.java | 6 +- .../stream/jsonplan/RankJsonPlanITCase.java | 4 +- .../stream/jsonplan/SargJsonPlanITCase.java | 2 +- .../jsonplan/SortLimitJsonPlanITCase.java | 2 +- .../jsonplan/TableSinkJsonPlanITCase.java | 2 +- .../jsonplan/TemporalJoinJsonPlanITCase.java | 4 +- .../jsonplan/TemporalSortJsonITCase.java | 4 +- .../stream/jsonplan/UnionJsonPlanITCase.java | 2 +- .../stream/jsonplan/ValuesJsonPlanITCase.java | 2 +- .../jsonplan/WindowAggregateJsonITCase.java | 8 +- .../jsonplan/WindowDeduplicateJsonITCase.java | 2 +- .../stream/jsonplan/WindowJoinJsonITCase.java | 2 +- .../WindowTableFunctionJsonITCase.java | 2 +- .../stream/sql/DataStreamJavaITCase.java | 4 +- .../runtime/stream/sql/RTASITCase.java | 6 +- .../runtime/batch/sql/CodeSplitITCase.scala | 2 +- .../runtime/batch/table/TableSinkITCase.scala | 26 +-- .../stream/sql/ChangelogSourceITCase.scala | 10 +- .../stream/sql/DeduplicateITCase.scala | 12 +- .../runtime/stream/sql/TableSinkITCase.scala | 20 +- .../stream/sql/TemporalJoinITCase.scala | 30 +-- .../stream/table/TableSinkITCase.scala | 74 +++---- 39 files changed, 285 insertions(+), 249 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java index 3ab136d451f42..f582531f7bc53 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputView; @@ -47,13 +46,14 @@ import org.apache.flink.table.functions.LookupFunction; import org.apache.flink.table.runtime.generated.GeneratedProjection; import org.apache.flink.table.runtime.generated.Projection; +import org.apache.flink.table.runtime.typeutils.ExternalSerializer; import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.flink.types.RowUtils; -import org.apache.flink.util.StringUtils; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -71,6 +71,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.stream.Collectors; import static org.apache.flink.table.planner.factories.TestValuesTableFactory.RESOURCE_COUNTER; import static org.apache.flink.util.Preconditions.checkArgument; @@ -83,29 +84,40 @@ final class TestValuesRuntimeFunctions { static final Object LOCK = TestValuesTableFactory.class; // [table_name, [task_id, List[value]]] - private static final Map>> globalRawResult = new HashMap<>(); + private static final Map>> globalRawResult = new HashMap<>(); // [table_name, [task_id, Map[key, value]]] - private static final Map>> globalUpsertResult = + private static final Map>> globalUpsertResult = new HashMap<>(); // [table_name, [task_id, List[value]]] - private static final Map>> globalRetractResult = - new HashMap<>(); + private static final Map>> globalRetractResult = new HashMap<>(); // [table_name, [watermark]] private static final Map> watermarkHistory = new HashMap<>(); - static List getRawResults(String tableName) { - List result = new ArrayList<>(); + static List getRawResultsAsStrings(String tableName) { + return getRawResults(tableName).stream() + .map(TestValuesRuntimeFunctions::rowToString) + .collect(Collectors.toList()); + } + + static List getRawResults(String tableName) { synchronized (LOCK) { if (globalRawResult.containsKey(tableName)) { - globalRawResult.get(tableName).values().forEach(result::addAll); + return globalRawResult.get(tableName).values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); } } - return result; + return Collections.emptyList(); } /** Returns raw results if there was only one table with results, throws otherwise. */ - static List getOnlyRawResults() { - List result = new ArrayList<>(); + static List getOnlyRawResultsAsStrings() { + return getOnlyRawResults().stream() + .map(TestValuesRuntimeFunctions::rowToString) + .collect(Collectors.toList()); + } + + static List getOnlyRawResults() { synchronized (LOCK) { if (globalRawResult.size() != 1) { throw new IllegalStateException( @@ -113,9 +125,10 @@ static List getOnlyRawResults() { + globalRawResult.size()); } - globalRawResult.values().iterator().next().values().forEach(result::addAll); + return globalRawResult.values().iterator().next().values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); } - return result; } static List getWatermarks(String tableName) { @@ -128,23 +141,25 @@ static List getWatermarks(String tableName) { } } - static List getResults(String tableName) { - List result = new ArrayList<>(); + static List getResultsAsStrings(String tableName) { + return getResults(tableName).stream().map(Row::toString).collect(Collectors.toList()); + } + + static List getResults(String tableName) { synchronized (LOCK) { if (globalUpsertResult.containsKey(tableName)) { - globalUpsertResult - .get(tableName) - .values() - .forEach(map -> result.addAll(map.values())); + return globalUpsertResult.get(tableName).values().stream() + .flatMap(map -> map.values().stream()) + .collect(Collectors.toList()); } else if (globalRetractResult.containsKey(tableName)) { - globalRetractResult.get(tableName).values().forEach(result::addAll); + return globalRetractResult.get(tableName).values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()); } else if (globalRawResult.containsKey(tableName)) { - getRawResults(tableName).stream() - .map(s -> s.substring(3, s.length() - 1)) // removes the +I(...) wrapper - .forEach(result::add); + return getRawResults(tableName); } } - return result; + return Collections.emptyList(); } static void clearResults() { @@ -156,6 +171,14 @@ static void clearResults() { } } + private static String rowToString(Row row) { + if (RowUtils.USE_LEGACY_TO_STRING) { + return String.format("%s(%s)", row.getKind().shortString(), row); + } else { + return row.toString(); + } + } + // ------------------------------------------------------------------------------------------ // Source Function implementations // ------------------------------------------------------------------------------------------ @@ -179,7 +202,7 @@ public static class FromElementSourceFunctionWithWatermark implements SourceFunc private volatile boolean isRunning = true; - private String tableName; + private final String tableName; public FromElementSourceFunctionWithWatermark( String tableName, @@ -284,22 +307,29 @@ private abstract static class AbstractExactlyOnceSink extends RichSinkFunction rawResultState; + protected transient List localRawResult; - protected transient ListState rawResultState; - protected transient List localRawResult; - - protected AbstractExactlyOnceSink(String tableName) { + protected AbstractExactlyOnceSink( + String tableName, DataType consumedDataType, DataStructureConverter converter) { this.tableName = tableName; + this.consumedDataType = consumedDataType; + this.converter = converter; } @Override public void initializeState(FunctionInitializationContext context) throws Exception { this.rawResultState = context.getOperatorStateStore() - .getListState(new ListStateDescriptor<>("sink-results", Types.STRING)); + .getListState( + new ListStateDescriptor<>( + "sink-results", + ExternalSerializer.of(consumedDataType))); this.localRawResult = new ArrayList<>(); if (context.isRestored()) { - for (String value : rawResultState.get()) { + for (Row value : rawResultState.get()) { localRawResult.add(value); } } @@ -322,22 +352,20 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { static class AppendingSinkFunction extends AbstractExactlyOnceSink { private static final long serialVersionUID = 1L; - private final DataStructureConverter converter; private final int rowtimeIndex; protected AppendingSinkFunction( - String tableName, DataStructureConverter converter, int rowtimeIndex) { - super(tableName); - this.converter = converter; + String tableName, + DataType consumedDataType, + DataStructureConverter converter, + int rowtimeIndex) { + super(tableName, consumedDataType, converter); this.rowtimeIndex = rowtimeIndex; } @Override public void invoke(RowData value, Context context) throws Exception { - RowKind kind = value.getRowKind(); if (value.getRowKind() == RowKind.INSERT) { - Row row = (Row) converter.toExternal(value); - assertThat(row).isNotNull(); if (rowtimeIndex >= 0) { // currently, rowtime attribute always uses 3 precision TimestampData rowtime = value.getTimestamp(rowtimeIndex, 3); @@ -348,7 +376,7 @@ public void invoke(RowData value, Context context) throws Exception { } } synchronized (LOCK) { - localRawResult.add(kind.shortString() + "(" + row.toString() + ")"); + localRawResult.add((Row) converter.toExternal(value)); } } else { throw new RuntimeException( @@ -363,25 +391,24 @@ public void invoke(RowData value, Context context) throws Exception { */ static class KeyedUpsertingSinkFunction extends AbstractExactlyOnceSink { private static final long serialVersionUID = 1L; - private final DataStructureConverter converter; private final int[] keyIndices; private final int[] targetColumnIndices; private final int expectedSize; private final int totalColumns; // [key, value] map result - private transient Map localUpsertResult; + private transient Map localUpsertResult; private transient int receivedNum; protected KeyedUpsertingSinkFunction( String tableName, + DataType consumedDataType, DataStructureConverter converter, int[] keyIndices, int[] targetColumnIndices, int expectedSize, int totalColumns) { - super(tableName); - this.converter = converter; + super(tableName, consumedDataType, converter); this.keyIndices = keyIndices; this.targetColumnIndices = targetColumnIndices; this.expectedSize = expectedSize; @@ -409,29 +436,27 @@ public void invoke(RowData value, Context context) throws Exception { assertThat(row).isNotNull(); synchronized (LOCK) { - if (RowUtils.USE_LEGACY_TO_STRING) { - localRawResult.add(kind.shortString() + "(" + row + ")"); - } else { - localRawResult.add(row.toString()); - } + localRawResult.add(row); - row.setKind(RowKind.INSERT); Row key = Row.project(row, keyIndices); + key.setKind(RowKind.INSERT); + + final Row upsertRow = Row.copy(row); + upsertRow.setKind(RowKind.INSERT); if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) { if (targetColumnIndices.length > 0) { // perform partial insert - localUpsertResult.put( - key.toString(), - updateRowValue( - localUpsertResult.get(key.toString()), - row, - targetColumnIndices)); + localUpsertResult.compute( + key, + (entryKey, currentValue) -> + updateRowValue( + currentValue, upsertRow, targetColumnIndices)); } else { - localUpsertResult.put(key.toString(), row.toString()); + localUpsertResult.put(key, upsertRow); } } else { - String oldValue = localUpsertResult.remove(key.toString()); + Row oldValue = localUpsertResult.remove(key); if (oldValue == null) { throw new RuntimeException( "Tried to delete a value that wasn't inserted first. " @@ -447,27 +472,17 @@ public void invoke(RowData value, Context context) throws Exception { } } - private String updateRowValue(String old, Row newRow, int[] targetColumnIndices) { - if (StringUtils.isNullOrWhitespaceOnly(old)) { + private Row updateRowValue(Row old, Row newRow, int[] targetColumnIndices) { + if (old == null) { // no old value, just return current - return newRow.toString(); + return newRow; } else { - String[] oldCols = - org.apache.commons.lang3.StringUtils.splitByWholeSeparatorPreserveAllTokens( - old, ", "); - assert oldCols.length == totalColumns; + assert old.getArity() == totalColumns; // exist old value, simply simulate an update - for (int i = 0; i < targetColumnIndices.length; i++) { - int idx = targetColumnIndices[i]; - if (idx == 0) { - oldCols[idx] = String.format("+I[%s", newRow.getField(idx)); - } else if (idx == totalColumns - 1) { - oldCols[idx] = String.format("%s]", newRow.getField(idx)); - } else { - oldCols[idx] = (String) newRow.getField(idx); - } + for (int idx : targetColumnIndices) { + old.setField(idx, newRow.getField(idx)); } - return String.join(", ", oldCols); + return old; } } } @@ -475,14 +490,12 @@ private String updateRowValue(String old, Row newRow, int[] targetColumnIndices) static class RetractingSinkFunction extends AbstractExactlyOnceSink { private static final long serialVersionUID = 1L; - private final DataStructureConverter converter; - - protected transient ListState retractResultState; - protected transient List localRetractResult; + protected transient ListState retractResultState; + protected transient List localRetractResult; - protected RetractingSinkFunction(String tableName, DataStructureConverter converter) { - super(tableName); - this.converter = converter; + protected RetractingSinkFunction( + String tableName, DataType consumedDataType, DataStructureConverter converter) { + super(tableName, consumedDataType, converter); } @Override @@ -492,11 +505,12 @@ public void initializeState(FunctionInitializationContext context) throws Except context.getOperatorStateStore() .getListState( new ListStateDescriptor<>( - "sink-retract-results", Types.STRING)); + "sink-retract-results", + ExternalSerializer.of(consumedDataType))); this.localRetractResult = new ArrayList<>(); if (context.isRestored()) { - for (String value : retractResultState.get()) { + for (Row value : retractResultState.get()) { localRetractResult.add(value); } } @@ -524,13 +538,13 @@ public void invoke(RowData value, Context context) throws Exception { Row row = (Row) converter.toExternal(value); assertThat(row).isNotNull(); synchronized (LOCK) { - localRawResult.add(kind.shortString() + "(" + row.toString() + ")"); + localRawResult.add(row); + final Row retractRow = Row.copy(row); + retractRow.setKind(RowKind.INSERT); if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) { - row.setKind(RowKind.INSERT); - localRetractResult.add(row.toString()); + localRetractResult.add(retractRow); } else { - row.setKind(RowKind.INSERT); - boolean contains = localRetractResult.remove(row.toString()); + boolean contains = localRetractResult.remove(retractRow); if (!contains) { throw new RuntimeException( "Tried to retract a value that wasn't inserted first. " @@ -547,7 +561,7 @@ static class AppendingOutputFormat extends RichOutputFormat { private final String tableName; private final DataStructureConverter converter; - protected transient List localRawResult; + protected transient List localRawResult; protected AppendingOutputFormat(String tableName, DataStructureConverter converter) { this.tableName = tableName; @@ -571,12 +585,11 @@ public void open(int taskNumber, int numTasks) throws IOException { @Override public void writeRecord(RowData value) throws IOException { - RowKind kind = value.getRowKind(); if (value.getRowKind() == RowKind.INSERT) { Row row = (Row) converter.toExternal(value); assertThat(row).isNotNull(); synchronized (LOCK) { - localRawResult.add(kind.shortString() + "(" + row.toString() + ")"); + localRawResult.add(row); } } else { throw new RuntimeException( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index 8c5f34445c97e..c4da3ec230158 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -224,7 +224,16 @@ public static String registerRowData(Seq data) { * * @param tableName the table name of the registered table sink. */ - public static List getRawResults(String tableName) { + public static List getRawResultsAsStrings(String tableName) { + return TestValuesRuntimeFunctions.getRawResultsAsStrings(tableName); + } + + /** + * Returns received raw results of the registered table sink. + * + * @param tableName the table name of the registered table sink. + */ + public static List getRawResults(String tableName) { return TestValuesRuntimeFunctions.getRawResults(tableName); } @@ -234,8 +243,17 @@ public static List getRawResults(String tableName) { * *

The raw results are encoded with {@link RowKind}. */ - public static List getOnlyRawResults() { - return TestValuesRuntimeFunctions.getOnlyRawResults(); + public static List getOnlyRawResultsAsStrings() { + return TestValuesRuntimeFunctions.getOnlyRawResultsAsStrings(); + } + + /** + * Returns materialized (final) results of the registered table sink. + * + * @param tableName the table name of the registered table sink. + */ + public static List getResultsAsStrings(String tableName) { + return TestValuesRuntimeFunctions.getResultsAsStrings(tableName); } /** @@ -243,7 +261,7 @@ public static List getOnlyRawResults() { * * @param tableName the table name of the registered table sink. */ - public static List getResults(String tableName) { + public static List getResults(String tableName) { return TestValuesRuntimeFunctions.getResults(tableName); } @@ -1936,7 +1954,7 @@ public Optional getParallelism() { @Override public SinkFunction createSinkFunction() { return new AppendingSinkFunction( - tableName, converter, rowtimeIndex); + tableName, consumedDataType, converter, rowtimeIndex); } }; case "OutputFormat": @@ -1960,7 +1978,10 @@ public DataStreamSink consumeDataStream( DataStreamSink sink = dataStream.addSink( new AppendingSinkFunction( - tableName, converter, rowtimeIndex)); + tableName, + consumedDataType, + converter, + rowtimeIndex)); providerContext.generateUid("sink-function").ifPresent(sink::uid); return sink; } @@ -1989,6 +2010,7 @@ public Optional getParallelism() { sinkFunction = new KeyedUpsertingSinkFunction( tableName, + consumedDataType, converter, primaryKeyIndices, Arrays.stream(targetColumns).mapToInt(a -> a[0]).toArray(), @@ -2000,7 +2022,8 @@ public Optional getParallelism() { "Retracting Sink doesn't support '" + SINK_EXPECTED_MESSAGES_NUM.key() + "' yet."); - sinkFunction = new RetractingSinkFunction(tableName, converter); + sinkFunction = + new RetractingSinkFunction(tableName, consumedDataType, converter); } return SinkFunctionProvider.of(sinkFunction, this.parallelism); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/RTASITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/RTASITCase.java index 3f35086e44394..ec077566ccf02 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/RTASITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/RTASITCase.java @@ -65,7 +65,7 @@ void testReplaceTableAS() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("target").toString()) .isEqualTo("[+I[1, 1, Hi], +I[2, 2, Hello], +I[3, 2, Hello world]]"); // verify the table after replacing @@ -96,7 +96,7 @@ void testCreateOrReplaceTableAS() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("target").toString()) .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]"); // verify the table after replacing @@ -116,7 +116,7 @@ void testCreateOrReplaceTableASWithTableNotExist() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("not_exist_target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("not_exist_target").toString()) .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]"); // verify the table after replacing diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/join/AdaptiveHashJoinITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/join/AdaptiveHashJoinITCase.java index 2e7bd7e05e2c8..62670e2307993 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/join/AdaptiveHashJoinITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/join/AdaptiveHashJoinITCase.java @@ -168,7 +168,7 @@ public void testBuildRightStringKeyAdaptiveHashJoin() throws Exception { private void asserResult(String sinkTableName, int resultSize) { // Due to concern OOM and record value is same, here just assert result size - List result = TestValuesTableFactory.getResults(sinkTableName); + List result = TestValuesTableFactory.getResultsAsStrings(sinkTableName); assertThat(result.size()).isEqualTo(resultSize); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ChangelogSourceJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ChangelogSourceJsonPlanITCase.java index 5ceb0bd040cea..a6b78346f7c42 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ChangelogSourceJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ChangelogSourceJsonPlanITCase.java @@ -52,7 +52,7 @@ void testChangelogSource() throws Exception { "+I[user1, Tom, tom123@gmail.com, 8.10, 16.20]", "+I[user3, Bailey, bailey@qq.com, 9.99, 19.98]", "+I[user4, Tina, tina@gmail.com, 11.30, 22.60]"); - assertResult(expected, TestValuesTableFactory.getResults("user_sink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("user_sink")); } @Test @@ -74,7 +74,7 @@ void testToUpsertSource() throws Exception { "+I[user1, Tom, tom123@gmail.com, 8.10, 16.20]", "+I[user3, Bailey, bailey@qq.com, 9.99, 19.98]", "+I[user4, Tina, tina@gmail.com, 11.30, 22.60]"); - assertResult(expected, TestValuesTableFactory.getResults("user_sink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("user_sink")); } // ------------------------------------------------------------------------------------------ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ConfigureOperatorLevelStateTtlJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ConfigureOperatorLevelStateTtlJsonITCase.java index fb879bf10c5fa..2f47d76f62ce8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ConfigureOperatorLevelStateTtlJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ConfigureOperatorLevelStateTtlJsonITCase.java @@ -113,7 +113,7 @@ void testDifferentStateTtlForDifferentOneInputOperator() throws Exception { "+I[Jerry, 1, 2, 99.9]", "+I[Olivia, 2, 4, 1100.0]", "+I[Michael, 1, 3, 599.9]"); - assertResult(expected, TestValuesTableFactory.getResults("OrdersStats")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("OrdersStats")); } @Test @@ -187,7 +187,7 @@ void testDifferentStateTtlForSameTwoInputStreamOperator() throws Exception { List expected = Arrays.asList( "+I[1, 1000002, TRUCK]", "+I[1, 1000004, RAIL]", "+I[1, 1000005, AIR]"); - assertResult(expected, TestValuesTableFactory.getResults("OrdersShipInfo")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("OrdersShipInfo")); } private static Map getProperties( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java index a90c3cd86cbfa..a6e4fe6198acb 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java @@ -49,7 +49,7 @@ void testSystemFuncByObject() throws ExecutionException, InterruptedException { "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v)"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]", "+I[1,1,hi, hi]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -61,7 +61,7 @@ void testSystemFuncByClass() throws ExecutionException, InterruptedException { "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v)"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]", "+I[1,1,hi, hi]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -73,7 +73,7 @@ void testTemporaryFuncByObject() throws ExecutionException, InterruptedException "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v)"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]", "+I[1,1,hi, hi]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -85,7 +85,7 @@ void testTemporaryFuncByClass() throws ExecutionException, InterruptedException "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v)"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]", "+I[1,1,hi, hi]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -99,7 +99,7 @@ void testFilter() throws ExecutionException, InterruptedException { + "where try_cast(v as int) > 0"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -114,6 +114,6 @@ void testUnnest() throws ExecutionException, InterruptedException { "INSERT INTO MySink SELECT name, nested FROM MyNestedTable CROSS JOIN UNNEST(arr) AS t (nested)"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[Bob, 1]", "+I[Bob, 2]", "+I[Bob, 3]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/DeduplicationJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/DeduplicationJsonPlanITCase.java index afb9f5e2d5c45..191ec3a6997f3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/DeduplicationJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/DeduplicationJsonPlanITCase.java @@ -74,6 +74,6 @@ void testDeduplication() throws Exception { assertResult( Arrays.asList("+I[1, terry, pen, 1000]", "+I[4, bob, apple, 4000]"), - TestValuesTableFactory.getRawResults("MySink")); + TestValuesTableFactory.getRawResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ExpandJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ExpandJsonPlanITCase.java index 1226dac39ca6c..80109626f7be7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ExpandJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ExpandJsonPlanITCase.java @@ -58,7 +58,7 @@ void testExpand() throws Exception { + "from MyTable group by b") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult(Arrays.asList("+I[1, 1, Hi]", "+I[2, 2, Hello world]"), result); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupAggregateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupAggregateJsonPlanITCase.java index 367253eea761b..9b2f9b01f7a6c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupAggregateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupAggregateJsonPlanITCase.java @@ -89,7 +89,7 @@ void testSimpleAggCallsWithGroupBy() throws Exception { + "from MyTable group by b") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult(Arrays.asList("+I[1, 1, null, Hi]", "+I[2, 2, 2.0, Hello]"), result); } @@ -124,7 +124,7 @@ void testDistinctAggCalls() throws Exception { + "from MyTable group by e") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[1, 1, 4, 12, 32, 6.0, 5]", @@ -164,7 +164,7 @@ void testUserDefinedAggCallsWithoutMerge() throws Exception { + "from MyTable group by e") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList("+I[1, 77, 0, 1]", "+I[2, 120, 0, 2]", "+I[3, 58, 0, 3]"), result); } @@ -194,7 +194,7 @@ void testUserDefinedAggCallsWithMerge() throws Exception { + "from MyTable group by e") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[1, 1, Hallo Welt wie|Hallo|GHI|EFG|DEF]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java index f461640b3e0d2..81efe95a642d4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java @@ -81,7 +81,7 @@ void testEventTimeTumbleWindow() throws Exception { + "GROUP BY name, TUMBLE(rowtime, INTERVAL '5' SECOND)") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", @@ -104,7 +104,7 @@ void testEventTimeHopWindow() throws Exception { + "GROUP BY name, HOP(rowtime, INTERVAL '5' SECOND, INTERVAL '10' SECOND)") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 1]", @@ -132,7 +132,7 @@ void testEventTimeSessionWindow() throws Exception { + "GROUP BY name, Session(rowtime, INTERVAL '3' SECOND)") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 1]", "+I[a, 4]", "+I[b, 1]", "+I[b, 1]", "+I[b, 2]", "+I[null, 1]"), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IncrementalAggregateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IncrementalAggregateJsonPlanITCase.java index 8888bca40349f..6f72a3930a8ef 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IncrementalAggregateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IncrementalAggregateJsonPlanITCase.java @@ -72,7 +72,7 @@ void testIncrementalAggregate() throws IOException, ExecutionException, Interrup + "from MyTable group by b") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult(Arrays.asList("+I[1, 1]", "+I[2, 2]"), result); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IntervalJoinJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IntervalJoinJsonPlanITCase.java index 94a1e41b2acbc..d4569058eaac4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IntervalJoinJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/IntervalJoinJsonPlanITCase.java @@ -65,7 +65,7 @@ void testProcessTimeInnerJoin() throws Exception { "+I[1, HiHi, Hi6]", "+I[1, HiHi, Hi8]", "+I[2, HeHe, Hi5]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -112,6 +112,6 @@ void testRowTimeInnerJoin() throws Exception { "+I[1, HiHi, Hi2]", "+I[1, HiHi, Hi3]", "+I[2, HeHe, Hi5]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/JoinJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/JoinJsonPlanITCase.java index a722226e044f4..539cfc401e823 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/JoinJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/JoinJsonPlanITCase.java @@ -121,7 +121,7 @@ void testIsNullInnerJoinWithNullCond() throws Exception { "+I[1, HiHi, Hi8]", "+I[2, HeHe, Hi5]", "+I[null, HeHe, Hi9]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -132,7 +132,7 @@ void testJoin() throws Exception { List expected = Arrays.asList( "+I[Hello world, Hallo Welt]", "+I[Hello, Hallo Welt]", "+I[Hi, Hallo]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -141,7 +141,7 @@ void testInnerJoin() throws Exception { compileSqlAndExecutePlan("insert into MySink \n" + "SELECT a1, b1 FROM A JOIN B ON a1 = b1") .await(); List expected = Arrays.asList("+I[1, 1]", "+I[2, 2]", "+I[2, 2]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -152,7 +152,7 @@ void testJoinWithFilter() throws Exception { + "SELECT a3, b4 FROM A, B where a2 = b2 and a2 < 2") .await(); List expected = Arrays.asList("+I[Hi, Hallo]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -163,6 +163,6 @@ void testInnerJoinWithDuplicateKey() throws Exception { + "SELECT a1, b1, b3 FROM A JOIN B ON a1 = b1 AND a1 = b3") .await(); List expected = Arrays.asList("+I[2, 2, 2]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java index 6507481d6389b..1de8f66bd56e7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java @@ -45,6 +45,6 @@ void testLimit() throws ExecutionException, InterruptedException, IOException { compileSqlAndExecutePlan(sql).await(); List expected = Arrays.asList("+I[2, a, 6]", "+I[4, b, 8]", "+I[6, c, 10]"); - assertResult(expected, TestValuesTableFactory.getResults("result")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LookupJoinJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LookupJoinJsonPlanITCase.java index 71e76e0e2189d..9e4ac5fa35484 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LookupJoinJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LookupJoinJsonPlanITCase.java @@ -79,7 +79,7 @@ void testJoinLookupTable() throws Exception { "+I[1, 12, Julian, Julian]", "+I[2, 15, Hello, Jark]", "+I[3, 15, Fabian, Fabian]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -91,7 +91,7 @@ void testJoinLookupTableWithPushDown() throws Exception { .await(); List expected = Arrays.asList("+I[2, 15, Hello, Jark]", "+I[3, 15, Fabian, Fabian]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -108,6 +108,6 @@ void testLeftJoinLookupTableWithPreFilter() throws Exception { "+I[3, 15, Fabian, null]", "+I[8, 11, Hello world, null]", "+I[9, 12, Hello world!, null]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java index ca64b2126dfa3..a5cb80e3709d0 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java @@ -66,7 +66,7 @@ void testSimpleMatch() throws Exception { compileSqlAndExecutePlan(sql).await(); List expected = Collections.singletonList("+I[6, 7, 8]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -104,6 +104,6 @@ void testComplexMatch() throws Exception { compileSqlAndExecutePlan(sql).await(); List expected = Collections.singletonList("+I[19, 13, null]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/OverAggregateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/OverAggregateJsonPlanITCase.java index 55fc7a0a8d2e8..c42c961936de3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/OverAggregateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/OverAggregateJsonPlanITCase.java @@ -75,7 +75,7 @@ void testProcTimeBoundedPartitionedRowsOver() "+I[5, 33, 10]", "+I[5, 46, 10]", "+I[5, 60, 10]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -120,7 +120,7 @@ void testProcTimeUnboundedNonPartitionedRangeOver() "+I[Hello, 4, null]", "+I[Hello, 5, null]", "+I[Hello, 6, null]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -197,6 +197,6 @@ void testRowTimeBoundedPartitionedRangeOver() "+I[Hello World, 18, 1, 1, 7]", "+I[Hello World, 8, 2, 2, 15]", "+I[Hello World, 20, 1, 1, 20]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/RankJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/RankJsonPlanITCase.java index 1b866271787cf..b6ce7733ebfc1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/RankJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/RankJsonPlanITCase.java @@ -48,7 +48,7 @@ void testRank() throws ExecutionException, InterruptedException, IOException { compileSqlAndExecutePlan(sql).await(); List expected = Arrays.asList("+I[1, a, 1]", "+I[3, b, 1]", "+I[5, c, 1]"); - assertResult(expected, TestValuesTableFactory.getResults("result")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result")); } @Test @@ -70,6 +70,6 @@ void testFirstN() throws ExecutionException, InterruptedException, IOException { List expected = Arrays.asList( "+I[book, 1, 1]", "+I[book, 2, 2]", "+I[fruit, 4, 1]", "+I[fruit, 3, 2]"); - assertResult(expected, TestValuesTableFactory.getResults("result1")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result1")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SargJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SargJsonPlanITCase.java index bb40b5ab18133..e1b2558e4fccb 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SargJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SargJsonPlanITCase.java @@ -41,6 +41,6 @@ void testSarg() throws ExecutionException, InterruptedException { + "FROM MyTable WHERE a = 1 OR a = 2 OR a IS NULL"; compileSqlAndExecutePlan(sql).await(); List expected = Arrays.asList("+I[1]", "+I[2]", "+I[null]"); - assertResult(expected, TestValuesTableFactory.getResults("result")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SortLimitJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SortLimitJsonPlanITCase.java index 7473057ffc424..cd80e6a07d58b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SortLimitJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/SortLimitJsonPlanITCase.java @@ -45,6 +45,6 @@ void testSortLimit() throws ExecutionException, InterruptedException, IOExceptio compileSqlAndExecutePlan(sql).await(); List expected = Arrays.asList("+I[1, a, 5]", "+I[2, a, 6]", "+I[3, b, 7]"); - assertResult(expected, TestValuesTableFactory.getResults("result")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TableSinkJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TableSinkJsonPlanITCase.java index 73577d0775b59..0ba61358fbfb5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TableSinkJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TableSinkJsonPlanITCase.java @@ -68,7 +68,7 @@ void testWritingMetadata() throws Exception { compileSqlAndExecutePlan("insert into MySink select * from MyTable").await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList("+I[1, 1, hi]", "+I[2, 1, hello]", "+I[3, 2, hello world]"), result); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalJoinJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalJoinJsonPlanITCase.java index 56ebcf559476a..ca1ed35f05004 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalJoinJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalJoinJsonPlanITCase.java @@ -89,7 +89,7 @@ void testJoinTemporalFunction() throws Exception { + "WHERE o.currency = r.currency ") .await(); List expected = Arrays.asList("+I[102]", "+I[228]", "+I[348]", "+I[50]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -102,6 +102,6 @@ void testTemporalTableJoin() throws Exception { + "ON o.currency = r.currency ") .await(); List expected = Arrays.asList("+I[102]", "+I[228]", "+I[348]", "+I[50]"); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java index a97a90bf00330..fe623816f1a9c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java @@ -48,7 +48,7 @@ void testSortProcessingTime() throws Exception { assertResult( Arrays.asList("+I[1]", "+I[2]", "+I[3]"), - TestValuesTableFactory.getResults("MySink")); + TestValuesTableFactory.getResultsAsStrings("MySink")); } @Test @@ -78,7 +78,7 @@ void testSortRowTime() throws Exception { "insert into MySink SELECT `int` FROM MyTable order by rowtime, `double`") .await(); - assertThat(TestValuesTableFactory.getResults("MySink")) + assertThat(TestValuesTableFactory.getResultsAsStrings("MySink")) .isEqualTo( Arrays.asList( "+I[1]", "+I[2]", "+I[2]", "+I[5]", "+I[6]", "+I[3]", "+I[3]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/UnionJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/UnionJsonPlanITCase.java index 1d7096a851865..2145dd0179ae8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/UnionJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/UnionJsonPlanITCase.java @@ -61,6 +61,6 @@ void testUnion() throws Exception { "+I[5, c, 9]", "+I[3, b, 7]" // a=3 need to be doubled ); - assertResult(expected, TestValuesTableFactory.getResults("MySink")); + assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ValuesJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ValuesJsonPlanITCase.java index e71546fbec65d..420f895aed520 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ValuesJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ValuesJsonPlanITCase.java @@ -36,7 +36,7 @@ void testValues() throws Exception { "INSERT INTO MySink SELECT * from (VALUES (1, 2, 'Hi'), (3, 4, 'Hello'))") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult(Arrays.asList("+I[1, 2, Hi]", "+I[3, 4, Hello]"), result); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java index dc07965637c7a..f636ee8bb55aa 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java @@ -103,7 +103,7 @@ void testEventTimeTumbleWindow() throws Exception { + "GROUP BY name, window_start, window_end") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", @@ -127,7 +127,7 @@ void testEventTimeHopWindow() throws Exception { + "GROUP BY name, window_start, window_end") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 1]", @@ -160,7 +160,7 @@ void testEventTimeCumulateWindow() throws Exception { + "GROUP BY name, window_start, window_end") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 4]", @@ -200,7 +200,7 @@ void testDistinctSplitEnabled() throws Exception { + "GROUP BY name, window_start, window_end") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[a, 5.0, 3]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowDeduplicateJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowDeduplicateJsonITCase.java index 75eb714c9df6c..ef2cd5d58d904 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowDeduplicateJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowDeduplicateJsonITCase.java @@ -96,7 +96,7 @@ void testEventTimeTumbleWindow() throws Exception { + "WHERE rownum <= 1") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[2020-10-10 00:00:04, 5, 5.0, 5.0, 5.55, null, a, 2020-10-10 00:00:04.000, 2020-10-10T00:00, 2020-10-10T00:00:05, 2020-10-10T00:00:04.999]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowJoinJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowJoinJsonITCase.java index c4e84d96b32f8..d0405ca07b11c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowJoinJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowJoinJsonITCase.java @@ -119,7 +119,7 @@ void testEventTimeTumbleWindow() throws Exception { + "ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.name = R.name") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 2, 2]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowTableFunctionJsonITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowTableFunctionJsonITCase.java index 1ccf0e03da1ef..b816fdc189a53 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowTableFunctionJsonITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowTableFunctionJsonITCase.java @@ -90,7 +90,7 @@ void testEventTimeTumbleWindow() throws Exception { + "FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND))") .await(); - List result = TestValuesTableFactory.getResults("MySink"); + List result = TestValuesTableFactory.getResultsAsStrings("MySink"); assertResult( Arrays.asList( "+I[2020-10-10 00:00:01, 1, 1.0, 1.0, 1.11, Hi, a, 2020-10-10 00:00:01.000, 2020-10-10T00:00, 2020-10-10T00:00:05, 2020-10-10T00:00:04.999]", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java index 71e05066d09b8..9827e37b85e24 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java @@ -735,10 +735,10 @@ public void testAttachAsDataStream() throws Exception { // submits all source-to-sink pipelines testResult(env.fromElements(3, 4, 5), 3, 4, 5); - assertThat(TestValuesTableFactory.getResults("OutputTable1")) + assertThat(TestValuesTableFactory.getResultsAsStrings("OutputTable1")) .containsExactlyInAnyOrder("+I[1, a]", "+I[2, b]"); - assertThat(TestValuesTableFactory.getResults("OutputTable2")) + assertThat(TestValuesTableFactory.getResultsAsStrings("OutputTable2")) .containsExactlyInAnyOrder("+I[1]", "+I[2]", "+I[3]"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java index ba1b2b6513523..a673e87a0cda1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java @@ -66,7 +66,7 @@ void testReplaceTableAS() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("target").toString()) .isEqualTo("[+I[1, 1, Hi], +I[2, 2, Hello], +I[3, 2, Hello world]]"); // verify the table after replacing @@ -97,7 +97,7 @@ void testCreateOrReplaceTableAS() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("target").toString()) .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]"); // verify the table after replacing @@ -117,7 +117,7 @@ void testCreateOrReplaceTableASWithTableNotExist() throws Exception { .await(); // verify written rows - assertThat(TestValuesTableFactory.getResults("not_exist_target").toString()) + assertThat(TestValuesTableFactory.getResultsAsStrings("not_exist_target").toString()) .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]"); // verify the table after replacing diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CodeSplitITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CodeSplitITCase.scala index 74384b4cf0df0..ca99525befa81 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CodeSplitITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CodeSplitITCase.scala @@ -117,7 +117,7 @@ class CodeSplitITCase extends BatchTestBase { for (i <- 0 until 100) { expected.add(s"+I[${Range(0, 100).map(_ => s"$i").mkString(", ")}]") } - assertThatIterable(TestValuesTableFactory.getResults("test_many_values")) + assertThatIterable(TestValuesTableFactory.getResultsAsStrings("test_many_values")) .containsExactlyElementsOf(expected) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala index e4f757103b26f..b788535a22c3d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala @@ -58,7 +58,7 @@ class TableSinkITCase extends BatchTestBase { .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5))) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") val expected = Seq("12345,55,12345") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -84,7 +84,7 @@ class TableSinkITCase extends BatchTestBase { .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5))) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") val expected = Seq("12345,55,12345") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -110,7 +110,7 @@ class TableSinkITCase extends BatchTestBase { .select('a, 'b.sum()) table.executeInsert("testSink").await() - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") val expected = List("1,0.1", "2,0.4", "3,1.0", "4,2.2", "5,3.9") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -135,7 +135,7 @@ class TableSinkITCase extends BatchTestBase { .select('a, 'b.sum()) table.executeInsert("testSink").await() - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") val expected = List("1,0.1", "2,0.4", "3,1.0", "4,2.2", "5,3.9") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -167,7 +167,7 @@ class TableSinkITCase extends BatchTestBase { .await() val expected = List("1,2021,1,0.1", "2,2021,1,0.4", "3,2021,1,1.0", "4,2021,1,2.2", "5,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -193,7 +193,7 @@ class TableSinkITCase extends BatchTestBase { |""".stripMargin) .await() val expected = List("null,0.1", "null,0.4", "null,1.0", "null,2.2", "null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -228,7 +228,7 @@ class TableSinkITCase extends BatchTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -259,7 +259,7 @@ class TableSinkITCase extends BatchTestBase { .await() val expected = List("1,2021,1,0.1", "2,2021,1,0.4", "3,2021,1,1.0", "4,2021,1,2.2", "5,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -294,7 +294,7 @@ class TableSinkITCase extends BatchTestBase { "null,null,null,1.0", "null,null,null,2.2", "null,null,null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -329,7 +329,7 @@ class TableSinkITCase extends BatchTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -364,7 +364,7 @@ class TableSinkITCase extends BatchTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -399,7 +399,7 @@ class TableSinkITCase extends BatchTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } @@ -434,7 +434,7 @@ class TableSinkITCase extends BatchTestBase { "null,2021,null,1.0", "null,2021,null,2.2", "null,2021,null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertThat(result.sorted).isEqualTo(expected.sorted) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/ChangelogSourceITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/ChangelogSourceITCase.scala index 0f3af85aac15f..a3621f3482752 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/ChangelogSourceITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/ChangelogSourceITCase.scala @@ -115,13 +115,13 @@ class ChangelogSourceITCase( "user1,Tom,tom123@gmail.com,8.10,16.20", "user3,Bailey,bailey@qq.com,9.99,19.98", "user4,Tina,tina@gmail.com,11.30,22.60") - assertEquals(expected.sorted, TestValuesTableFactory.getResults("user_sink").sorted) + assertEquals(expected.sorted, TestValuesTableFactory.getResultsAsStrings("user_sink").sorted) sourceMode match { // verify the update_before messages haven been filtered when scanning changelog source // the CHANGELOG_SOURCE has I,UA,UB,D but no primary key, so we can not omit UB case CHANGELOG_SOURCE_WITH_EVENTS_DUPLICATE => - val rawResult = TestValuesTableFactory.getRawResults("user_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("user_sink") val hasUB = rawResult.exists(r => r.startsWith("-U")) assertFalse( s"Sink result shouldn't contain UPDATE_BEFORE, but is:\n ${rawResult.mkString("\n")}", @@ -173,7 +173,7 @@ class ChangelogSourceITCase( tEnv.executeSql(dml).await() val expected = Seq("ALL,3,29.39,tom123@gmail.com") - assertEquals(expected.sorted, TestValuesTableFactory.getResults("user_sink").sorted) + assertEquals(expected.sorted, TestValuesTableFactory.getResultsAsStrings("user_sink").sorted) } @Test @@ -202,7 +202,7 @@ class ChangelogSourceITCase( val expected = Seq("16.20,1,tom123@gmail.com", "19.98,1,bailey@qq.com", "22.60,1,tina@gmail.com") - assertEquals(expected.sorted, TestValuesTableFactory.getResults("user_sink").sorted) + assertEquals(expected.sorted, TestValuesTableFactory.getResultsAsStrings("user_sink").sorted) } @Test @@ -233,7 +233,7 @@ class ChangelogSourceITCase( val expected = Seq("user3,Bailey,bailey@qq.com,9.99,19.98", "user4,Tina,tina@gmail.com,11.30,22.60") - assertEquals(expected.sorted, TestValuesTableFactory.getResults("user_sink").sorted) + assertEquals(expected.sorted, TestValuesTableFactory.getResultsAsStrings("user_sink").sorted) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/DeduplicateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/DeduplicateITCase.scala index a0f8ab4a47733..ed7e4be891700 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/DeduplicateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/DeduplicateITCase.scala @@ -198,7 +198,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getRawResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("rowtime_sink") val expected = List( "+I(1,1,Hi,1970-01-01T00:00:00.001)", @@ -237,7 +237,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getRawResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("rowtime_sink") val expected = List( "+I(1,1,Hi,1970-01-01T00:00:00.001)", @@ -279,7 +279,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getResultsAsStrings("rowtime_sink") val expected = List("6") assertEquals(expected.sorted, rawResult.sorted) } @@ -306,7 +306,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getRawResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("rowtime_sink") val expected = List( "+I(1,1,Hi,1970-01-01T00:00:00.001)", @@ -349,7 +349,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getRawResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("rowtime_sink") val expected = List( "+I(1,1,Hi,1970-01-01T00:00:00.001)", @@ -393,7 +393,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode) """.stripMargin tEnv.executeSql(sql).await() - val rawResult = TestValuesTableFactory.getResults("rowtime_sink") + val rawResult = TestValuesTableFactory.getResultsAsStrings("rowtime_sink") val expected = List("6") assertEquals(expected.sorted, rawResult.sorted) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala index 25c0b7c124b1a..e353bb8760a84 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala @@ -114,7 +114,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("JoinDisorderChangeLog") + val result = TestValuesTableFactory.getResultsAsStrings("JoinDisorderChangeLog") val expected = List("+I[jason, 4, 22.5, 22]") assertEquals(expected.sorted, result.sorted) } @@ -139,7 +139,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("SinkDisorderChangeLog") + val result = TestValuesTableFactory.getResultsAsStrings("SinkDisorderChangeLog") val expected = List("+I[jason, 4, 22.5]") assertEquals(expected.sorted, result.sorted) } @@ -168,7 +168,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("SinkRankChangeLog") + val result = TestValuesTableFactory.getResultsAsStrings("SinkRankChangeLog") val expected = List("+I[jason, 4]") assertEquals(expected.sorted, result.sorted) } @@ -197,14 +197,14 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("sink_with_pk") + val result = TestValuesTableFactory.getResultsAsStrings("sink_with_pk") val expected = List( "+I[user1, Tom, tom123@gmail.com, 8.10]", "+I[user3, Bailey, bailey@qq.com, 9.99]", "+I[user4, Tina, tina@gmail.com, 11.30]") assertEquals(expected.sorted, result.sorted) - val rawResult = TestValuesTableFactory.getRawResults("sink_with_pk") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("sink_with_pk") val expectedRaw = List( "+I[user1, Tom, tom@gmail.com, 10.02]", "+I[user2, Jack, jack@hotmail.com, 71.20]", @@ -246,7 +246,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("zm_test") + val result = TestValuesTableFactory.getResultsAsStrings("zm_test") val expected = List( "+I[jason, 1, null, null, null, null]", "+I[jason, 1, null, null, null, null]", @@ -272,7 +272,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase | src |""".stripMargin) .await() - val actual = TestValuesTableFactory.getResults("MyCtasTable") + val actual = TestValuesTableFactory.getResultsAsStrings("MyCtasTable") val expected = List( "+I[jason, 1]", "+I[jason, 1]", @@ -295,7 +295,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase | src |""".stripMargin) statementSet.execute().await() - val actualUseStatement = TestValuesTableFactory.getResults("MyCtasTableUseStatement") + val actualUseStatement = TestValuesTableFactory.getResultsAsStrings("MyCtasTableUseStatement") Assertions.assertThat(actualUseStatement.sorted).isEqualTo(expected.sorted) } @@ -359,7 +359,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("test_sink") + val result = TestValuesTableFactory.getResultsAsStrings("test_sink") val expected = List( "+I[1, jason, 3, null, null]", "+I[2, andy, 2, null, null]", @@ -378,7 +378,7 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase |""".stripMargin) .await() - val result2 = TestValuesTableFactory.getResults("test_sink") + val result2 = TestValuesTableFactory.getResultsAsStrings("test_sink") val expected2 = List("+I[1, jason, 3, X, 43]", "+I[2, andy, 2, Y, 32]", "+I[3, clark, 1, Z, 29]") assertEquals(expected2.sorted, result2.sorted) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala index 60aa95184b673..dc97cd48aedc8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.factories.TestValuesTableFactory -import org.apache.flink.table.planner.factories.TestValuesTableFactory.{getResults, registerData} +import org.apache.flink.table.planner.factories.TestValuesTableFactory.{getResultsAsStrings, registerData} import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode import org.apache.flink.table.utils.LegacyRowResource @@ -477,7 +477,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "3,RMB,40,2020-08-15T00:03,702,2020-08-15T00:00:04", "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -495,7 +495,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "3,RMB,40,2020-08-15T00:03,702,2020-08-15T00:00:04", "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -513,7 +513,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest val expected = List( "1,Euro,12,2020-08-15T00:01,114,2020-08-15T00:00:01", "2,US Dollar,18,2020-08-16T00:03,106,2020-08-16T00:02") - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -532,7 +532,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,null,null" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -553,7 +553,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,null,null" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -574,7 +574,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,null,null" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -592,7 +592,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "3,RMB,40,2020-08-15T00:03,702,2020-08-15T00:00:04", "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -609,7 +609,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "2,US Dollar,18,2020-08-16T00:03,106,2020-08-16T00:02", "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -625,7 +625,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "1,Euro,12,2020-08-15T00:01,114,2020-08-15T00:00:01", "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -663,7 +663,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,null,null,null,null" ) - assertEquals(expected.sorted, getResults("rowtime_sink1").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_sink1").sorted) } @Test @@ -683,7 +683,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,114,2020-08-15T00:00:01", "5,RMB,40,2020-08-16T00:03,702,2020-08-15T00:00:04" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -703,7 +703,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,702,2020-08-15T00:00:04" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -723,7 +723,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,null,null", "5,RMB,40,2020-08-16T00:03,null,null" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } @Test @@ -750,7 +750,7 @@ class TemporalJoinITCase(state: StateBackendMode) extends StreamingWithStateTest "4,Euro,14,2020-08-16T00:04,118,2020-08-16T00:01", "5,RMB,40,2020-08-16T00:03,702,2020-08-15T00:00:04" ) - assertEquals(expected.sorted, getResults("rowtime_default_sink").sorted) + assertEquals(expected.sorted, getResultsAsStrings("rowtime_default_sink").sorted) } private def createSinkTable(tableName: String, columns: Option[String]): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala index 3423bf74fab89..640a0dd411308 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala @@ -75,7 +75,7 @@ class TableSinkITCase extends StreamingTestBase { .select('w.end.as('t), 'id.count.as('icnt), 'num.sum.as('nsum)) table.executeInsert("appendSink").await() - val result = TestValuesTableFactory.getResults("appendSink") + val result = TestValuesTableFactory.getResultsAsStrings("appendSink") val expected = List( "1970-01-01T00:00:00.005,4,8", "1970-01-01T00:00:00.010,5,18", @@ -108,7 +108,7 @@ class TableSinkITCase extends StreamingTestBase { "INSERT INTO appendSink /*+ OPTIONS('sink.parallelism' = '1') */(t, num, text) SELECT id, num, text FROM src") .await() - val result = TestValuesTableFactory.getResults("appendSink") + val result = TestValuesTableFactory.getResultsAsStrings("appendSink") val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") assertEquals(expected.sorted, result.sorted) } @@ -131,7 +131,7 @@ class TableSinkITCase extends StreamingTestBase { |""".stripMargin) tEnv.executeSql("INSERT INTO appendSink SELECT id, ROW(num, text) FROM src").await() - val result = TestValuesTableFactory.getResults("appendSink") + val result = TestValuesTableFactory.getResultsAsStrings("appendSink") val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") assertEquals(expected.sorted, result.sorted) } @@ -157,7 +157,7 @@ class TableSinkITCase extends StreamingTestBase { .select('c, 'g) table.executeInsert("appendSink").await() - val result = TestValuesTableFactory.getResults("appendSink") + val result = TestValuesTableFactory.getResultsAsStrings("appendSink") val expected = List("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt") assertEquals(expected.sorted, result.sorted) } @@ -186,7 +186,7 @@ class TableSinkITCase extends StreamingTestBase { .select('len, 'id.count.as('icnt), 'num.sum.as('nsum)) table.executeInsert("retractSink").await() - val result = TestValuesTableFactory.getResults("retractSink") + val result = TestValuesTableFactory.getResultsAsStrings("retractSink") val expected = List("2,1,1", "5,1,2", "11,1,2", "25,1,3", "10,7,39", "14,1,3", "9,9,41") assertEquals(expected.sorted, result.sorted) @@ -216,13 +216,13 @@ class TableSinkITCase extends StreamingTestBase { .select('w.end.as('t), 'id.count.as('icnt), 'num.sum.as('nsum)) table.executeInsert("retractSink").await() - val rawResult = TestValuesTableFactory.getRawResults("retractSink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("retractSink") assertFalse( "Received retraction messages for append only table", rawResult.exists(_.startsWith("-")) ) // maybe -U or -D - val result = TestValuesTableFactory.getResults("retractSink") + val result = TestValuesTableFactory.getResultsAsStrings("retractSink") val expected = List( "1970-01-01T00:00:00.005,4,8", "1970-01-01T00:00:00.010,5,18", @@ -261,10 +261,10 @@ class TableSinkITCase extends StreamingTestBase { .select('count, 'len.count.as('lencnt), 'cTrue) table.executeInsert("upsertSink").await() - val rawResult = TestValuesTableFactory.getRawResults("upsertSink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("upsertSink") assertTrue("Results must include delete messages", rawResult.exists(_.startsWith("-D("))) - val result = TestValuesTableFactory.getResults("upsertSink") + val result = TestValuesTableFactory.getResultsAsStrings("upsertSink") val expected = List("1,5,true", "7,1,true", "9,1,true") assertEquals(expected.sorted, result.sorted) } @@ -295,13 +295,13 @@ class TableSinkITCase extends StreamingTestBase { .select('num, 'w.end.as('window_end), 'id.count.as('icnt)) table.executeInsert("upsertSink").await() - val rawResult = TestValuesTableFactory.getRawResults("upsertSink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("upsertSink") assertFalse( "Received retraction messages for append only table", rawResult.exists(_.startsWith("-")) ) // maybe -D or -U - val result = TestValuesTableFactory.getResults("upsertSink") + val result = TestValuesTableFactory.getResultsAsStrings("upsertSink") val expected = List( "1,1970-01-01T00:00:00.005,1", "2,1970-01-01T00:00:00.005,2", @@ -341,7 +341,7 @@ class TableSinkITCase extends StreamingTestBase { .select('w.end.as('wend), 'id.count.as('cnt)) table.executeInsert("upsertSink").await() - val rawResult = TestValuesTableFactory.getRawResults("upsertSink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("upsertSink") assertFalse( "Received retraction messages for append only table", rawResult.exists(_.startsWith("-")) @@ -386,7 +386,7 @@ class TableSinkITCase extends StreamingTestBase { .select('num, 'id.count.as('cnt)) table.executeInsert("upsertSink").await() - val rawResult = TestValuesTableFactory.getRawResults("upsertSink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("upsertSink") assertFalse( "Received retraction messages for append only table", rawResult.exists(_.startsWith("-")) @@ -438,7 +438,7 @@ class TableSinkITCase extends StreamingTestBase { .where('cnt <= 3) table.executeInsert("upsertSink").await() - val result = TestValuesTableFactory.getResults("upsertSink") + val result = TestValuesTableFactory.getResultsAsStrings("upsertSink") val expected = List("1,1", "2,2", "3,3") assertEquals(expected.sorted, result.sorted) } @@ -502,7 +502,7 @@ class TableSinkITCase extends StreamingTestBase { .select('num, 'w.rowtime.as('rowtime1), 'w.rowtime.as('rowtime2)) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") assertEquals(result.size(), 10) // clean up @@ -532,7 +532,7 @@ class TableSinkITCase extends StreamingTestBase { .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5))) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") val expected = Seq("12345,55,12345") assertEquals(expected.sorted, result.sorted) } @@ -558,7 +558,7 @@ class TableSinkITCase extends StreamingTestBase { .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5))) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") val expected = Seq("12345,55,12345") assertEquals(expected.sorted, result.sorted) } @@ -608,7 +608,7 @@ class TableSinkITCase extends StreamingTestBase { |""".stripMargin) .await() - val rawResult = TestValuesTableFactory.getRawResults("changelog_sink") + val rawResult = TestValuesTableFactory.getRawResultsAsStrings("changelog_sink") val expected = List( "+I(1,user2,71.20)", "+I(1,user1,10.02)", @@ -659,7 +659,7 @@ class TableSinkITCase extends StreamingTestBase { |GROUP BY user_name |""".stripMargin) .await() - val finalResult = TestValuesTableFactory.getResults("final_sink") + val finalResult = TestValuesTableFactory.getResultsAsStrings("final_sink") val finalExpected = List("user1,28.12", "user2,71.20", "user3,32.33", "user4,9.99") assertEquals(finalExpected.sorted, finalResult.sorted) } @@ -701,7 +701,7 @@ class TableSinkITCase extends StreamingTestBase { |""".stripMargin) .await() - val result = TestValuesTableFactory.getResults("MetadataSink") + val result = TestValuesTableFactory.getResultsAsStrings("MetadataSink") val expected = List("1,book,12", "2,book,null", "3,fruit,44", "4,book,11", "4,fruit,null", "5,fruit,null") assertEquals(expected.sorted, result.sorted) @@ -815,7 +815,7 @@ class TableSinkITCase extends StreamingTestBase { |""".stripMargin) .await() val expected = List("null,0.1", "null,0.4", "null,1.0", "null,2.2", "null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -851,7 +851,7 @@ class TableSinkITCase extends StreamingTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -883,7 +883,7 @@ class TableSinkITCase extends StreamingTestBase { .await() val expected = List("1,2021,1,0.1", "2,2021,1,0.4", "3,2021,1,1.0", "4,2021,1,2.2", "5,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -919,7 +919,7 @@ class TableSinkITCase extends StreamingTestBase { "null,null,null,1.0", "null,null,null,2.2", "null,null,null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -958,7 +958,7 @@ class TableSinkITCase extends StreamingTestBase { "1,c,c1,c2,33333,12,1.0", "1,c,c1,c2,33333,12,2.2", "1,c,c1,c2,33333,12,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -995,7 +995,7 @@ class TableSinkITCase extends StreamingTestBase { "1,c,c1,c2,33333,12,1.0", "1,c,c1,c2,33333,12,2.2", "1,c,c1,c2,33333,12,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -1031,7 +1031,7 @@ class TableSinkITCase extends StreamingTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -1068,7 +1068,7 @@ class TableSinkITCase extends StreamingTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -1104,7 +1104,7 @@ class TableSinkITCase extends StreamingTestBase { "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -1140,7 +1140,7 @@ class TableSinkITCase extends StreamingTestBase { "null,2021,null,1.0", "null,2021,null,2.2", "null,2021,null,3.9") - val result = TestValuesTableFactory.getResults("testSink") + val result = TestValuesTableFactory.getResultsAsStrings("testSink") assertEquals(expected.sorted, result.sorted) } @@ -1235,7 +1235,7 @@ class TableSinkITCase extends StreamingTestBase { .build()) .build()) .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) // Derived schema @@ -1247,7 +1247,7 @@ class TableSinkITCase extends StreamingTestBase { .from("T2") .executeInsert(TableDescriptor.forConnector("values").build()) .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) // Enriched schema @@ -1269,7 +1269,7 @@ class TableSinkITCase extends StreamingTestBase { .build()) .build()) .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) TestValuesTableFactory.clearAllData() } @@ -1303,7 +1303,7 @@ class TableSinkITCase extends StreamingTestBase { ) .execute() .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) // Derived schema @@ -1319,7 +1319,7 @@ class TableSinkITCase extends StreamingTestBase { ) .execute() .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) // Enriched schema @@ -1344,7 +1344,7 @@ class TableSinkITCase extends StreamingTestBase { ) .execute() .await() - assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResults.toList) + assertEquals(Seq("+I(42)"), TestValuesTableFactory.getOnlyRawResultsAsStrings.toList) } @Test @@ -1375,7 +1375,7 @@ class TableSinkITCase extends StreamingTestBase { .select('w.end.as('t), 'id.count.as('icnt), 'num.sum.as('nsum)) table.executeInsert("sink").await() - val result = TestValuesTableFactory.getResults("sink") + val result = TestValuesTableFactory.getResultsAsStrings("sink") val expected = List( "1970-01-01T00:00:00.005,4,8", "1970-01-01T00:00:00.010,5,18", From 98e4610f09f35a942e55472b5d358ebe113b0dba Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 9 Oct 2023 14:23:47 +0200 Subject: [PATCH 035/104] [FLINK-33246][tests] Add AutoRescalingITCase. --- .../runtime/testutils/CommonTestUtils.java | 27 + .../checkpointing/AutoRescalingITCase.java | 968 ++++++++++++++++++ .../UpdateJobResourceRequirementsITCase.java | 2 +- 3 files changed, 996 insertions(+), 1 deletion(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index a101a453ff004..da01615050065 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; @@ -366,6 +367,32 @@ public static void waitForCheckpoint(JobID jobID, MiniCluster miniCluster, int n }); } + /** Wait for on more completed checkpoint. */ + public static void waitForOneMoreCheckpoint(JobID jobID, MiniCluster miniCluster) + throws Exception { + final long[] currentCheckpoint = new long[] {-1L}; + waitUntilCondition( + () -> { + AccessExecutionGraph graph = miniCluster.getExecutionGraph(jobID).get(); + CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot(); + if (snapshot != null) { + long currentCount = snapshot.getCounts().getNumberOfCompletedCheckpoints(); + if (currentCheckpoint[0] < 0L) { + currentCheckpoint[0] = currentCount; + } else { + return currentCount > currentCheckpoint[0]; + } + } else if (graph.getState().isGloballyTerminalState()) { + checkState( + graph.getFailureInfo() != null, + "Job terminated before taking required checkpoint.", + graph.getState()); + throw graph.getFailureInfo().getException(); + } + return false; + }); + } + /** * @return the path as {@link java.net.URI} to the latest checkpoint. * @throws FlinkJobNotFoundException if job not found diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java new file mode 100644 index 0000000000000..403449a388b52 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java @@ -0,0 +1,968 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.test.checkpointing; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.StateBackendOptions; +import org.apache.flink.configuration.WebOptions; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobResourceRequirements; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.apache.flink.util.Collector; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForOneMoreCheckpoint; +import static org.apache.flink.test.scheduling.UpdateJobResourceRequirementsITCase.waitForAvailableSlots; +import static org.apache.flink.test.scheduling.UpdateJobResourceRequirementsITCase.waitForRunningTasks; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test checkpoint rescaling under changing resource requirements. This test is mostly a variant of + * {@link RescalingITCase} with two main differences: (1) We rescale from checkpoints instead of + * savepoints and (2) rescaling without cancel/restart but triggered by changing resource + * requirements. + */ +@RunWith(Parameterized.class) +public class AutoRescalingITCase extends TestLogger { + + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + + private static final int numTaskManagers = 2; + private static final int slotsPerTaskManager = 2; + private static final int totalSlots = numTaskManagers * slotsPerTaskManager; + + @Parameterized.Parameters(name = "backend = {0}, buffersPerChannel = {1}") + public static Collection data() { + return Arrays.asList( + new Object[][] { + {"rocksdb", 0}, {"rocksdb", 2}, {"filesystem", 0}, {"filesystem", 2} + }); + } + + public AutoRescalingITCase(String backend, int buffersPerChannel) { + this.backend = backend; + this.buffersPerChannel = buffersPerChannel; + } + + private final String backend; + + private final int buffersPerChannel; + + private String currentBackend = null; + + enum OperatorCheckpointMethod { + NON_PARTITIONED, + CHECKPOINTED_FUNCTION, + CHECKPOINTED_FUNCTION_BROADCAST, + LIST_CHECKPOINTED + } + + private static MiniClusterWithClientResource cluster; + private static RestClusterClient restClusterClient; + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setup() throws Exception { + // detect parameter change + if (!Objects.equals(currentBackend, backend)) { + shutDownExistingCluster(); + + currentBackend = backend; + + Configuration config = new Configuration(); + + final File checkpointDir = temporaryFolder.newFolder(); + final File savepointDir = temporaryFolder.newFolder(); + + config.setString(StateBackendOptions.STATE_BACKEND, currentBackend); + config.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true); + config.setBoolean(CheckpointingOptions.LOCAL_RECOVERY, true); + config.setString( + CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); + config.setString( + CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); + config.setInteger( + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL, buffersPerChannel); + + config.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); + + // speed the test suite up + // - lower refresh interval -> controls how fast we invalidate ExecutionGraphCache + // - lower slot idle timeout -> controls how fast we return idle slots to TM + config.set(WebOptions.REFRESH_INTERVAL, 50L); + config.set(JobManagerOptions.SLOT_IDLE_TIMEOUT, 50L); + + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(config) + .setNumberTaskManagers(numTaskManagers) + .setNumberSlotsPerTaskManager(slotsPerTaskManager) + .build()); + cluster.before(); + restClusterClient = cluster.getRestClusterClient(); + } + } + + @AfterClass + public static void shutDownExistingCluster() { + if (cluster != null) { + cluster.after(); + cluster = null; + } + } + + @Test + public void testCheckpointRescalingInKeyedState() throws Exception { + testCheckpointRescalingKeyedState(false); + } + + @Test + public void testCheckpointRescalingOutKeyedState() throws Exception { + testCheckpointRescalingKeyedState(true); + } + + /** + * Tests that a job with purely keyed state can be restarted from a checkpoint with a different + * parallelism. + */ + public void testCheckpointRescalingKeyedState(boolean scaleOut) throws Exception { + final int numberKeys = 42; + final int numberElements = 1000; + final int parallelism = scaleOut ? totalSlots / 2 : totalSlots; + final int parallelism2 = scaleOut ? totalSlots : totalSlots / 2; + final int maxParallelism = 13; + + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); + + ClusterClient client = cluster.getClusterClient(); + + try { + + JobGraph jobGraph = + createJobGraphWithKeyedState( + new Configuration(), + parallelism, + maxParallelism, + numberKeys, + numberElements); + + final JobID jobID = jobGraph.getJobID(); + + client.submitJob(jobGraph).get(); + + SubtaskIndexSource.SOURCE_LATCH.trigger(); + + // wait til the sources have emitted numberElements for each key and completed a + // checkpoint + assertTrue( + SubtaskIndexFlatMapper.workCompletedLatch.await( + deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + // verify the current state + + Set> actualResult = CollectionSink.getElementsSet(); + + Set> expectedResult = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism); + + expectedResult.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelism, parallelism, keyGroupIndex), + numberElements * key)); + } + + assertEquals(expectedResult, actualResult); + + // clear the CollectionSink set for the restarted job + CollectionSink.clearElementsSet(); + + waitForAllTaskRunning(cluster.getMiniCluster(), jobGraph.getJobID(), false); + + waitForOneMoreCheckpoint(jobID, cluster.getMiniCluster()); + + SubtaskIndexSource.SOURCE_LATCH.reset(); + + JobResourceRequirements.Builder builder = JobResourceRequirements.newBuilder(); + for (JobVertex vertex : jobGraph.getVertices()) { + builder.setParallelismForJobVertex(vertex.getID(), parallelism2, parallelism2); + } + + restClusterClient.updateJobResourceRequirements(jobID, builder.build()).join(); + + waitForRunningTasks(restClusterClient, jobID, 2 * parallelism2); + waitForAvailableSlots(restClusterClient, totalSlots - parallelism2); + + SubtaskIndexSource.SOURCE_LATCH.trigger(); + + client.requestJobResult(jobID).get(); + + Set> actualResult2 = CollectionSink.getElementsSet(); + + Set> expectedResult2 = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism); + expectedResult2.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelism, parallelism2, keyGroupIndex), + key * 2 * numberElements)); + } + + assertEquals(expectedResult2, actualResult2); + + } finally { + // clear the CollectionSink set for the restarted job + CollectionSink.clearElementsSet(); + } + } + + /** + * Tests that a job cannot be restarted from a checkpoint with a different parallelism if the + * rescaled operator has non-partitioned state. + */ + @Test + public void testCheckpointRescalingNonPartitionedStateCausesException() throws Exception { + final int parallelism = totalSlots / 2; + final int parallelism2 = totalSlots; + final int maxParallelism = 13; + + ClusterClient client = cluster.getClusterClient(); + + try { + JobGraph jobGraph = + createJobGraphWithOperatorState( + parallelism, maxParallelism, OperatorCheckpointMethod.NON_PARTITIONED); + // make sure the job does not finish before we take a checkpoint + StateSourceBase.canFinishLatch = new CountDownLatch(1); + + final JobID jobID = jobGraph.getJobID(); + + client.submitJob(jobGraph).get(); + + // wait until the operator is started + waitForAllTaskRunning(cluster.getMiniCluster(), jobGraph.getJobID(), false); + // wait until the operator handles some data + StateSourceBase.workStartedLatch.await(); + + waitForOneMoreCheckpoint(jobID, cluster.getMiniCluster()); + + JobResourceRequirements.Builder builder = JobResourceRequirements.newBuilder(); + for (JobVertex vertex : jobGraph.getVertices()) { + builder.setParallelismForJobVertex(vertex.getID(), parallelism2, parallelism2); + } + + restClusterClient.updateJobResourceRequirements(jobID, builder.build()).join(); + + waitForRunningTasks(restClusterClient, jobID, parallelism2); + waitForAvailableSlots(restClusterClient, totalSlots - parallelism2); + + StateSourceBase.canFinishLatch.countDown(); + + client.requestJobResult(jobID).get(); + } catch (JobExecutionException exception) { + if (!(exception.getCause() instanceof IllegalStateException)) { + throw exception; + } + } + } + + /** + * Tests that a job with non partitioned state can be restarted from a checkpoint with a + * different parallelism if the operator with non-partitioned state are not rescaled. + */ + @Test + public void testCheckpointRescalingWithKeyedAndNonPartitionedState() throws Exception { + int numberKeys = 42; + int numberElements = 1000; + int parallelism = totalSlots / 2; + int parallelism2 = totalSlots; + int maxParallelism = 13; + + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); + + ClusterClient client = cluster.getClusterClient(); + + try { + + JobGraph jobGraph = + createJobGraphWithKeyedAndNonPartitionedOperatorState( + parallelism, + maxParallelism, + parallelism, + numberKeys, + numberElements, + numberElements); + + final JobID jobID = jobGraph.getJobID(); + + client.submitJob(jobGraph).get(); + + SubtaskIndexSource.SOURCE_LATCH.trigger(); + + // wait til the sources have emitted numberElements for each key and completed a + // checkpoint + assertTrue( + SubtaskIndexFlatMapper.workCompletedLatch.await( + deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); + + // verify the current state + + Set> actualResult = CollectionSink.getElementsSet(); + + Set> expectedResult = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism); + + expectedResult.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelism, parallelism, keyGroupIndex), + numberElements * key)); + } + + assertEquals(expectedResult, actualResult); + + // clear the CollectionSink set for the restarted job + CollectionSink.clearElementsSet(); + + waitForOneMoreCheckpoint(jobID, cluster.getMiniCluster()); + + SubtaskIndexSource.SOURCE_LATCH.reset(); + + JobResourceRequirements.Builder builder = JobResourceRequirements.newBuilder(); + for (JobVertex vertex : jobGraph.getVertices()) { + if (vertex.getMaxParallelism() >= parallelism2) { + builder.setParallelismForJobVertex(vertex.getID(), parallelism2, parallelism2); + } else { + builder.setParallelismForJobVertex( + vertex.getID(), vertex.getMaxParallelism(), vertex.getMaxParallelism()); + } + } + + restClusterClient.updateJobResourceRequirements(jobID, builder.build()).join(); + + waitForRunningTasks(restClusterClient, jobID, parallelism2); + waitForAvailableSlots(restClusterClient, totalSlots - parallelism2); + + SubtaskIndexSource.SOURCE_LATCH.trigger(); + + client.requestJobResult(jobID).get(); + + Set> actualResult2 = CollectionSink.getElementsSet(); + + Set> expectedResult2 = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism); + expectedResult2.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelism, parallelism2, keyGroupIndex), + key * 2 * numberElements)); + } + + assertEquals(expectedResult2, actualResult2); + + } finally { + // clear the CollectionSink set for the restarted job + CollectionSink.clearElementsSet(); + } + } + + @Test + public void testCheckpointRescalingInPartitionedOperatorState() throws Exception { + testCheckpointRescalingPartitionedOperatorState( + false, OperatorCheckpointMethod.CHECKPOINTED_FUNCTION); + } + + @Test + public void testCheckpointRescalingOutPartitionedOperatorState() throws Exception { + testCheckpointRescalingPartitionedOperatorState( + true, OperatorCheckpointMethod.CHECKPOINTED_FUNCTION); + } + + @Test + public void testCheckpointRescalingInBroadcastOperatorState() throws Exception { + testCheckpointRescalingPartitionedOperatorState( + false, OperatorCheckpointMethod.CHECKPOINTED_FUNCTION_BROADCAST); + } + + @Test + public void testCheckpointRescalingOutBroadcastOperatorState() throws Exception { + testCheckpointRescalingPartitionedOperatorState( + true, OperatorCheckpointMethod.CHECKPOINTED_FUNCTION_BROADCAST); + } + + /** Tests rescaling of partitioned operator state. */ + public void testCheckpointRescalingPartitionedOperatorState( + boolean scaleOut, OperatorCheckpointMethod checkpointMethod) throws Exception { + final int parallelism = scaleOut ? totalSlots : totalSlots / 2; + final int parallelism2 = scaleOut ? totalSlots / 2 : totalSlots; + final int maxParallelism = 13; + + ClusterClient client = cluster.getClusterClient(); + + int counterSize = Math.max(parallelism, parallelism2); + + if (checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION + || checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION_BROADCAST) { + PartitionedStateSource.checkCorrectSnapshot = new int[counterSize]; + PartitionedStateSource.checkCorrectRestore = new int[counterSize]; + PartitionedStateSource.checkCorrectSnapshots.clear(); + } else { + throw new UnsupportedOperationException("Unsupported method:" + checkpointMethod); + } + + JobGraph jobGraph = + createJobGraphWithOperatorState(parallelism, maxParallelism, checkpointMethod); + // make sure the job does not finish before we take the checkpoint + StateSourceBase.canFinishLatch = new CountDownLatch(1); + + final JobID jobID = jobGraph.getJobID(); + + client.submitJob(jobGraph).get(); + + // wait until the operator is started + waitForAllTaskRunning(cluster.getMiniCluster(), jobGraph.getJobID(), false); + // wait until the operator handles some data + StateSourceBase.workStartedLatch.await(); + + waitForOneMoreCheckpoint(jobID, cluster.getMiniCluster()); + + JobResourceRequirements.Builder builder = JobResourceRequirements.newBuilder(); + for (JobVertex vertex : jobGraph.getVertices()) { + builder.setParallelismForJobVertex(vertex.getID(), parallelism2, parallelism2); + } + + restClusterClient.updateJobResourceRequirements(jobID, builder.build()).join(); + + waitForRunningTasks(restClusterClient, jobID, 2 * parallelism2); + waitForAvailableSlots(restClusterClient, totalSlots - parallelism2); + + StateSourceBase.canFinishLatch.countDown(); + + client.requestJobResult(jobID).get(); + + int sumExp = 0; + int sumAct = 0; + + if (checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION) { + for (int c : PartitionedStateSource.checkCorrectSnapshot) { + sumExp += c; + } + + for (int c : PartitionedStateSource.checkCorrectRestore) { + sumAct += c; + } + } else { + for (int c : PartitionedStateSource.checkCorrectSnapshot) { + sumExp += c; + } + + for (int c : PartitionedStateSource.checkCorrectRestore) { + sumAct += c; + } + + sumExp *= parallelism2; + } + + assertEquals(sumExp, sumAct); + } + + // ------------------------------------------------------------------------------------------------------------------ + + private static void configureCheckpointing(CheckpointConfig config) { + config.setCheckpointInterval(100); + config.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); + config.enableUnalignedCheckpoints(true); + } + + private static JobGraph createJobGraphWithOperatorState( + int parallelism, int maxParallelism, OperatorCheckpointMethod checkpointMethod) { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + configureCheckpointing(env.getCheckpointConfig()); + env.setParallelism(parallelism); + env.getConfig().setMaxParallelism(maxParallelism); + env.setRestartStrategy(RestartStrategies.noRestart()); + + StateSourceBase.workStartedLatch = new CountDownLatch(parallelism); + + SourceFunction src; + + switch (checkpointMethod) { + case CHECKPOINTED_FUNCTION: + src = new PartitionedStateSource(false); + break; + case CHECKPOINTED_FUNCTION_BROADCAST: + src = new PartitionedStateSource(true); + break; + case NON_PARTITIONED: + src = new NonPartitionedStateSource(); + break; + default: + throw new IllegalArgumentException(checkpointMethod.name()); + } + + DataStream input = env.addSource(src); + + input.sinkTo(new DiscardingSink<>()); + + return env.getStreamGraph().getJobGraph(); + } + + public static JobGraph createJobGraphWithKeyedState( + Configuration configuration, + int parallelism, + int maxParallelism, + int numberKeys, + int numberElements) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.setParallelism(parallelism); + if (0 < maxParallelism) { + env.getConfig().setMaxParallelism(maxParallelism); + } + + configureCheckpointing(env.getCheckpointConfig()); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.getConfig().setUseSnapshotCompression(true); + + DataStream input = + env.addSource(new SubtaskIndexSource(numberKeys, numberElements, parallelism)) + .keyBy( + new KeySelector() { + private static final long serialVersionUID = + -7952298871120320940L; + + @Override + public Integer getKey(Integer value) { + return value; + } + }); + + SubtaskIndexFlatMapper.workCompletedLatch = new CountDownLatch(numberKeys); + + DataStream> result = + input.flatMap(new SubtaskIndexFlatMapper(numberElements)); + + result.addSink(new CollectionSink<>()); + + return env.getStreamGraph().getJobGraph(); + } + + private static JobGraph createJobGraphWithKeyedAndNonPartitionedOperatorState( + int parallelism, + int maxParallelism, + int fixedParallelism, + int numberKeys, + int numberElements, + int numberElementsAfterRestart) { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(parallelism); + env.getConfig().setMaxParallelism(maxParallelism); + configureCheckpointing(env.getCheckpointConfig()); + env.setRestartStrategy(RestartStrategies.noRestart()); + + DataStream input = + env.addSource( + new SubtaskIndexNonPartitionedStateSource( + numberKeys, + numberElements, + numberElementsAfterRestart, + parallelism)) + .setParallelism(fixedParallelism) + .setMaxParallelism(fixedParallelism) + .keyBy( + new KeySelector() { + private static final long serialVersionUID = + -7952298871120320940L; + + @Override + public Integer getKey(Integer value) { + return value; + } + }); + + SubtaskIndexFlatMapper.workCompletedLatch = new CountDownLatch(numberKeys); + + DataStream> result = + input.flatMap(new SubtaskIndexFlatMapper(numberElements)); + + result.addSink(new CollectionSink<>()); + + return env.getStreamGraph().getJobGraph(); + } + + private static class SubtaskIndexSource extends RichParallelSourceFunction { + + private static final long serialVersionUID = -400066323594122516L; + + private final int numberKeys; + + private final int originalParallelism; + protected int numberElements; + + protected int counter = 0; + + private boolean running = true; + + private static final OneShotLatch SOURCE_LATCH = new OneShotLatch(); + + SubtaskIndexSource(int numberKeys, int numberElements, int originalParallelism) { + this.numberKeys = numberKeys; + this.numberElements = numberElements; + this.originalParallelism = originalParallelism; + } + + @Override + public void run(SourceContext ctx) throws Exception { + RuntimeContext runtimeContext = getRuntimeContext(); + final int subtaskIndex = runtimeContext.getIndexOfThisSubtask(); + + boolean isRestartedOrRescaled = + runtimeContext.getNumberOfParallelSubtasks() != originalParallelism + || runtimeContext.getAttemptNumber() > 0; + while (running) { + SOURCE_LATCH.await(); + if (counter < numberElements) { + synchronized (ctx.getCheckpointLock()) { + for (int value = subtaskIndex; + value < numberKeys; + value += runtimeContext.getNumberOfParallelSubtasks()) { + ctx.collect(value); + } + + counter++; + } + } else { + if (isRestartedOrRescaled) { + running = false; + } else { + Thread.sleep(100); + } + } + } + } + + @Override + public void cancel() { + running = false; + } + } + + private static class SubtaskIndexNonPartitionedStateSource extends SubtaskIndexSource + implements ListCheckpointed { + + private static final long serialVersionUID = 8388073059042040203L; + private final int numElementsAfterRestart; + + SubtaskIndexNonPartitionedStateSource( + int numberKeys, + int numberElements, + int numElementsAfterRestart, + int originalParallelism) { + super(numberKeys, numberElements, originalParallelism); + this.numElementsAfterRestart = numElementsAfterRestart; + } + + @Override + public List snapshotState(long checkpointId, long timestamp) { + return Collections.singletonList(this.counter); + } + + @Override + public void restoreState(List state) { + if (state.size() != 1) { + throw new RuntimeException( + "Test failed due to unexpected recovered state size " + state.size()); + } + this.counter = state.get(0); + this.numberElements += numElementsAfterRestart; + } + } + + private static class SubtaskIndexFlatMapper + extends RichFlatMapFunction> + implements CheckpointedFunction { + + private static final long serialVersionUID = 5273172591283191348L; + + private static CountDownLatch workCompletedLatch = new CountDownLatch(1); + + private transient ValueState counter; + private transient ValueState sum; + + private final int numberElements; + + SubtaskIndexFlatMapper(int numberElements) { + this.numberElements = numberElements; + } + + @Override + public void flatMap(Integer value, Collector> out) + throws Exception { + + int count = counter.value() + 1; + counter.update(count); + + int s = sum.value() + value; + sum.update(s); + + if (count % numberElements == 0) { + out.collect(Tuple2.of(getRuntimeContext().getIndexOfThisSubtask(), s)); + workCompletedLatch.countDown(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) { + // all managed, nothing to do. + } + + @Override + public void initializeState(FunctionInitializationContext context) { + counter = + context.getKeyedStateStore() + .getState(new ValueStateDescriptor<>("counter", Integer.class, 0)); + sum = + context.getKeyedStateStore() + .getState(new ValueStateDescriptor<>("sum", Integer.class, 0)); + } + } + + private static class CollectionSink implements SinkFunction { + + private static final Set elements = + Collections.newSetFromMap(new ConcurrentHashMap<>()); + + private static final long serialVersionUID = -1652452958040267745L; + + public static Set getElementsSet() { + return (Set) elements; + } + + public static void clearElementsSet() { + elements.clear(); + } + + @Override + public void invoke(IN value) { + elements.add(value); + } + } + + private static class StateSourceBase extends RichParallelSourceFunction { + + private static final long serialVersionUID = 7512206069681177940L; + private static CountDownLatch workStartedLatch = new CountDownLatch(1); + private static CountDownLatch canFinishLatch = new CountDownLatch(0); + + protected volatile int counter = 0; + protected volatile boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + while (running) { + synchronized (ctx.getCheckpointLock()) { + ++counter; + ctx.collect(1); + } + + Thread.sleep(2); + + if (counter == 10) { + workStartedLatch.countDown(); + } + + if (counter >= 500) { + break; + } + } + + canFinishLatch.await(); + } + + @Override + public void cancel() { + running = false; + } + } + + private static class NonPartitionedStateSource extends StateSourceBase + implements ListCheckpointed { + + private static final long serialVersionUID = -8108185918123186841L; + + @Override + public List snapshotState(long checkpointId, long timestamp) { + return Collections.singletonList(this.counter); + } + + @Override + public void restoreState(List state) { + if (!state.isEmpty()) { + this.counter = state.get(0); + } + } + } + + private static class PartitionedStateSource extends StateSourceBase + implements CheckpointedFunction { + + private static final long serialVersionUID = -359715965103593462L; + private static final int NUM_PARTITIONS = 7; + + private transient ListState counterPartitions; + private final boolean broadcast; + + private static final ConcurrentHashMap checkCorrectSnapshots = + new ConcurrentHashMap<>(); + private static int[] checkCorrectSnapshot; + private static int[] checkCorrectRestore; + + public PartitionedStateSource(boolean broadcast) { + this.broadcast = broadcast; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + + if (getRuntimeContext().getAttemptNumber() == 0) { + int[] snapshot = + checkCorrectSnapshots.computeIfAbsent( + context.getCheckpointId(), + (x) -> new int[checkCorrectRestore.length]); + snapshot[getRuntimeContext().getIndexOfThisSubtask()] = counter; + } + + counterPartitions.clear(); + + int div = counter / NUM_PARTITIONS; + int mod = counter % NUM_PARTITIONS; + + for (int i = 0; i < NUM_PARTITIONS; ++i) { + int partitionValue = div; + if (mod > 0) { + --mod; + ++partitionValue; + } + counterPartitions.add(partitionValue); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + if (broadcast) { + this.counterPartitions = + context.getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>( + "counter_partitions", IntSerializer.INSTANCE)); + } else { + this.counterPartitions = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "counter_partitions", IntSerializer.INSTANCE)); + } + + if (context.isRestored()) { + for (int v : counterPartitions.get()) { + counter += v; + } + checkCorrectRestore[getRuntimeContext().getIndexOfThisSubtask()] = counter; + context.getRestoredCheckpointId() + .ifPresent((id) -> checkCorrectSnapshot = checkCorrectSnapshots.get(id)); + } + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/UpdateJobResourceRequirementsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/UpdateJobResourceRequirementsITCase.java index 74829f34caf1e..e0a88aaab2cd2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/UpdateJobResourceRequirementsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/UpdateJobResourceRequirementsITCase.java @@ -202,7 +202,7 @@ private void runRescalingTest( } } - private static int getNumberRunningTasks(RestClusterClient restClusterClient, JobID jobId) { + public static int getNumberRunningTasks(RestClusterClient restClusterClient, JobID jobId) { final JobDetailsInfo jobDetailsInfo = restClusterClient.getJobDetails(jobId).join(); return jobDetailsInfo.getJobVertexInfos().stream() .map(JobDetailsInfo.JobVertexDetailsInfo::getTasksPerState) From 2d74ee3966525be5ab3770a653c96a7d04d5f05a Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Thu, 19 Oct 2023 14:31:10 +0200 Subject: [PATCH 036/104] [FLINK-33341][state] Refactoring: introduce common superclass for all IncrementalKeyedStateHandle. --- .../metadata/MetadataV2V3SerializerBase.java | 2 +- .../state/AbstractIncrementalStateHandle.java | 106 +++++++++++++++ .../state/DirectoryKeyedStateHandle.java | 124 ------------------ .../state/IncrementalKeyedStateHandle.java | 3 + .../IncrementalLocalKeyedStateHandle.java | 122 ++++++++--------- .../IncrementalRemoteKeyedStateHandle.java | 71 ++-------- .../ChangelogStateBackendHandle.java | 2 +- .../checkpoint/CheckpointCoordinatorTest.java | 2 +- .../runtime/state/ChangelogTestUtils.java | 2 +- ...IncrementalRemoteKeyedStateHandleTest.java | 18 +-- .../RocksDBIncrementalRestoreOperation.java | 6 +- 11 files changed, 192 insertions(+), 266 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java index db3b5dd22d0e7..bd91e8e1fbb4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java @@ -341,7 +341,7 @@ static void serializeKeyedStateHandle(KeyedStateHandle stateHandle, DataOutputSt dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); dos.writeLong(incrementalKeyedStateHandle.getCheckpointedSize()); - serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos); + serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaDataStateHandle(), dos); serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getSharedState(), dos); serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getPrivateState(), dos); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java new file mode 100644 index 0000000000000..8c7ea74c33c88 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.runtime.state; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.UUID; + +/** Abstract superclass for all {@link IncrementalKeyedStateHandle}. */ +public abstract class AbstractIncrementalStateHandle implements IncrementalKeyedStateHandle { + private static final long serialVersionUID = 1L; + + /** The checkpoint Id. */ + protected final long checkpointId; + + /** + * UUID to identify the backend which created this state handle. This is in creating the key for + * the {@link SharedStateRegistry}. + */ + protected final UUID backendIdentifier; + + /** The key-group range covered by this state handle. */ + protected final KeyGroupRange keyGroupRange; + + /** Shared state in the incremental checkpoint. */ + protected final List sharedState; + + /** Primary meta data state of the incremental checkpoint. */ + protected final StreamStateHandle metaStateHandle; + + /** Unique id for this state handle. */ + protected final StateHandleID stateHandleId; + + public AbstractIncrementalStateHandle( + UUID backendIdentifier, + KeyGroupRange keyGroupRange, + long checkpointId, + List sharedState, + StreamStateHandle metaStateHandle, + StateHandleID stateHandleId) { + this.checkpointId = checkpointId; + this.keyGroupRange = keyGroupRange; + this.backendIdentifier = backendIdentifier; + this.sharedState = sharedState; + this.metaStateHandle = metaStateHandle; + this.stateHandleId = stateHandleId; + } + + @Override + public long getCheckpointId() { + return checkpointId; + } + + @Nonnull + @Override + public UUID getBackendIdentifier() { + return backendIdentifier; + } + + @Override + public KeyGroupRange getKeyGroupRange() { + return keyGroupRange; + } + + @Nonnull + @Override + public List getSharedStateHandles() { + return sharedState; + } + + @Nonnull + @Override + public StreamStateHandle getMetaDataStateHandle() { + return metaStateHandle; + } + + @Override + public StateHandleID getStateHandleId() { + return stateHandleId; + } + + @Override + public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { + return KeyGroupRange.EMPTY_KEY_GROUP_RANGE.equals( + getKeyGroupRange().getIntersection(keyGroupRange)) + ? null + : this; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java deleted file mode 100644 index 3f922a2e9680c..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.state; - -import javax.annotation.Nonnull; - -/** - * This class is a keyed state handle based on a directory. It combines a {@link - * DirectoryStateHandle} and a {@link KeyGroupRange}. - */ -public class DirectoryKeyedStateHandle implements KeyedStateHandle { - - private static final long serialVersionUID = 1L; - - /** The directory state handle. */ - @Nonnull private final DirectoryStateHandle directoryStateHandle; - - /** The key-group range. */ - @Nonnull private final KeyGroupRange keyGroupRange; - - private final StateHandleID stateHandleId; - - public DirectoryKeyedStateHandle( - @Nonnull DirectoryStateHandle directoryStateHandle, - @Nonnull KeyGroupRange keyGroupRange) { - - this.directoryStateHandle = directoryStateHandle; - this.keyGroupRange = keyGroupRange; - this.stateHandleId = StateHandleID.randomStateHandleId(); - } - - @Nonnull - public DirectoryStateHandle getDirectoryStateHandle() { - return directoryStateHandle; - } - - @Nonnull - @Override - public KeyGroupRange getKeyGroupRange() { - return keyGroupRange; - } - - @Override - public void discardState() throws Exception { - directoryStateHandle.discardState(); - } - - @Override - public long getStateSize() { - return directoryStateHandle.getStateSize(); - } - - @Override - public long getCheckpointedSize() { - return getStateSize(); - } - - @Override - public KeyedStateHandle getIntersection(KeyGroupRange otherKeyGroupRange) { - return this.keyGroupRange.getIntersection(otherKeyGroupRange).getNumberOfKeyGroups() > 0 - ? this - : null; - } - - @Override - public StateHandleID getStateHandleId() { - return stateHandleId; - } - - @Override - public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) { - // Nothing to do, this is for local use only. - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - DirectoryKeyedStateHandle that = (DirectoryKeyedStateHandle) o; - - if (!getDirectoryStateHandle().equals(that.getDirectoryStateHandle())) { - return false; - } - return getKeyGroupRange().equals(that.getKeyGroupRange()); - } - - @Override - public int hashCode() { - int result = getDirectoryStateHandle().hashCode(); - result = 31 * result + getKeyGroupRange().hashCode(); - return result; - } - - @Override - public String toString() { - return "DirectoryKeyedStateHandle{" - + "directoryStateHandle=" - + directoryStateHandle - + ", keyGroupRange=" - + keyGroupRange - + '}'; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java index 9ce3cbd332fcb..f162efa936bac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java @@ -42,6 +42,9 @@ public interface IncrementalKeyedStateHandle @Nonnull List getSharedStateHandles(); + @Nonnull + StreamStateHandle getMetaDataStateHandle(); + /** A Holder of StreamStateHandle and the corresponding localPath. */ final class HandleAndLocalPath implements Serializable { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java index d782cf886bd5a..f854c111c6e9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java @@ -32,22 +32,11 @@ * DirectoryStateHandle} that represents the directory of the native RocksDB snapshot, the key * groups, and a stream state handle for Flink's state meta data file. */ -public class IncrementalLocalKeyedStateHandle extends DirectoryKeyedStateHandle - implements IncrementalKeyedStateHandle { +public class IncrementalLocalKeyedStateHandle extends AbstractIncrementalStateHandle { private static final long serialVersionUID = 1L; - /** Id of the checkpoint that created this state handle. */ - @Nonnegative private final long checkpointId; - - /** UUID to identify the backend which created this state handle. */ - @Nonnull private final UUID backendIdentifier; - - /** Handle to Flink's state meta data. */ - @Nonnull private final StreamStateHandle metaDataState; - - /** All shared state handles and the corresponding localPath used by the checkpoint. */ - @Nonnull private final List sharedState; + private final DirectoryStateHandle directoryStateHandle; public IncrementalLocalKeyedStateHandle( @Nonnull UUID backendIdentifier, @@ -57,21 +46,14 @@ public IncrementalLocalKeyedStateHandle( @Nonnull StreamStateHandle metaDataState, @Nonnull List sharedState) { - super(directoryStateHandle, keyGroupRange); - this.backendIdentifier = backendIdentifier; - this.checkpointId = checkpointId; - this.metaDataState = metaDataState; - this.sharedState = new ArrayList<>(sharedState); - } - - @Nonnull - public StreamStateHandle getMetaDataState() { - return metaDataState; - } - - @Override - public long getCheckpointId() { - return checkpointId; + super( + backendIdentifier, + keyGroupRange, + checkpointId, + new ArrayList<>(sharedState), + metaDataState, + StateHandleID.randomStateHandleId()); + this.directoryStateHandle = directoryStateHandle; } @Override @@ -81,52 +63,23 @@ public CheckpointBoundKeyedStateHandle rebound(long checkpointId) { checkpointId, getDirectoryStateHandle(), getKeyGroupRange(), - getMetaDataState(), + getMetaDataStateHandle(), getSharedStateHandles()); } - @Override - @Nonnull - public UUID getBackendIdentifier() { - return backendIdentifier; - } - - @Override - @Nonnull - public List getSharedStateHandles() { - return sharedState; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - - IncrementalLocalKeyedStateHandle that = (IncrementalLocalKeyedStateHandle) o; - - return getMetaDataState().equals(that.getMetaDataState()); - } - @Override public void discardState() throws Exception { Exception collectedEx = null; try { - super.discardState(); + directoryStateHandle.discardState(); } catch (Exception e) { collectedEx = e; } try { - metaDataState.discardState(); + metaStateHandle.discardState(); } catch (Exception e) { collectedEx = ExceptionUtils.firstOrSuppressed(e, collectedEx); } @@ -138,22 +91,61 @@ public void discardState() throws Exception { @Override public long getStateSize() { - return super.getStateSize() + metaDataState.getStateSize(); + return directoryStateHandle.getStateSize() + metaStateHandle.getStateSize(); } @Override public int hashCode() { - int result = super.hashCode(); - result = 31 * result + getMetaDataState().hashCode(); + int result = directoryStateHandle.hashCode(); + result = 31 * result + getKeyGroupRange().hashCode(); + result = 31 * result + getMetaDataStateHandle().hashCode(); return result; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + IncrementalLocalKeyedStateHandle that = (IncrementalLocalKeyedStateHandle) o; + + return getKeyGroupRange().equals(that.keyGroupRange) + && getMetaDataStateHandle().equals(that.getMetaDataStateHandle()); + } + @Override public String toString() { return "IncrementalLocalKeyedStateHandle{" + "metaDataState=" - + metaDataState + + metaStateHandle + "} " - + super.toString(); + + "DirectoryKeyedStateHandle{" + + "directoryStateHandle=" + + directoryStateHandle + + ", keyGroupRange=" + + keyGroupRange + + '}'; + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) { + // Nothing to do, this is for local use only. + } + + @Override + public long getCheckpointedSize() { + return directoryStateHandle.getStateSize(); + } + + @Nonnull + public DirectoryStateHandle getDirectoryStateHandle() { + return directoryStateHandle; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java index fd524d74f2adc..86a5b59c16869 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java @@ -56,7 +56,7 @@ * should not be called from production code. This means this class is also not suited to serve as a * key, e.g. in hash maps. */ -public class IncrementalRemoteKeyedStateHandle implements IncrementalKeyedStateHandle { +public class IncrementalRemoteKeyedStateHandle extends AbstractIncrementalStateHandle { public static final long UNKNOWN_CHECKPOINTED_SIZE = -1L; @@ -65,31 +65,11 @@ public class IncrementalRemoteKeyedStateHandle implements IncrementalKeyedStateH private static final long serialVersionUID = -8328808513197388231L; - /** - * UUID to identify the backend which created this state handle. This is in creating the key for - * the {@link SharedStateRegistry}. - */ - private final UUID backendIdentifier; - - /** The key-group range covered by this state handle. */ - private final KeyGroupRange keyGroupRange; - - /** The checkpoint Id. */ - private final long checkpointId; - - /** Shared state in the incremental checkpoint. */ - private final List sharedState; - /** Private state in the incremental checkpoint. */ private final List privateState; - /** Primary meta data state of the incremental checkpoint. */ - private final StreamStateHandle metaStateHandle; - private final long persistedSizeOfThisCheckpoint; - private final StateHandleID stateHandleId; - /** * Once the shared states are registered, it is the {@link SharedStateRegistry}'s responsibility * to cleanup those shared states. But in the cases where the state handle is discarded before @@ -147,18 +127,19 @@ protected IncrementalRemoteKeyedStateHandle( StreamStateHandle metaStateHandle, long persistedSizeOfThisCheckpoint, StateHandleID stateHandleId) { - this.backendIdentifier = Preconditions.checkNotNull(backendIdentifier); - this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange); - this.checkpointId = checkpointId; - this.sharedState = Preconditions.checkNotNull(sharedState); + super( + Preconditions.checkNotNull(backendIdentifier), + Preconditions.checkNotNull(keyGroupRange), + checkpointId, + sharedState, + metaStateHandle, + stateHandleId); this.privateState = Preconditions.checkNotNull(privateState); - this.metaStateHandle = Preconditions.checkNotNull(metaStateHandle); this.sharedStateRegistry = null; this.persistedSizeOfThisCheckpoint = persistedSizeOfThisCheckpoint == UNKNOWN_CHECKPOINTED_SIZE ? getStateSize() : persistedSizeOfThisCheckpoint; - this.stateHandleId = stateHandleId; } public static IncrementalRemoteKeyedStateHandle restore( @@ -181,16 +162,6 @@ public static IncrementalRemoteKeyedStateHandle restore( stateHandleId); } - @Override - public KeyGroupRange getKeyGroupRange() { - return keyGroupRange; - } - - @Override - public long getCheckpointId() { - return checkpointId; - } - @Override public CheckpointBoundKeyedStateHandle rebound(long checkpointId) { return new IncrementalRemoteKeyedStateHandle( @@ -212,15 +183,6 @@ public List getPrivateState() { return privateState; } - public StreamStateHandle getMetaStateHandle() { - return metaStateHandle; - } - - @Nonnull - public UUID getBackendIdentifier() { - return backendIdentifier; - } - @Nonnull @Override public List getSharedStateHandles() { @@ -231,19 +193,6 @@ public SharedStateRegistry getSharedStateRegistry() { return sharedStateRegistry; } - @Override - public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { - return KeyGroupRange.EMPTY_KEY_GROUP_RANGE.equals( - this.keyGroupRange.getIntersection(keyGroupRange)) - ? null - : this; - } - - @Override - public StateHandleID getStateHandleId() { - return stateHandleId; - } - @Override public void discardState() throws Exception { @@ -390,7 +339,7 @@ public boolean equals(Object o) { if (!getPrivateState().equals(that.getPrivateState())) { return false; } - return getMetaStateHandle().equals(that.getMetaStateHandle()); + return getMetaDataStateHandle().equals(that.getMetaDataStateHandle()); } /** This method should only be called in tests! This should never serve as key in a hash map. */ @@ -402,7 +351,7 @@ public int hashCode() { result = 31 * result + (int) (getCheckpointId() ^ (getCheckpointId() >>> 32)); result = 31 * result + getSharedState().hashCode(); result = 31 * result + getPrivateState().hashCode(); - result = 31 * result + getMetaStateHandle().hashCode(); + result = 31 * result + getMetaDataStateHandle().hashCode(); result = 31 * result + getStateHandleId().hashCode(); return result; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java index 2d786c44d0d97..6c5c7feae06c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java @@ -174,7 +174,7 @@ private static KeyedStateHandle castToAbsolutePath( StreamStateHandle castMetaStateHandle = restoreFileStateHandle( - incrementalRemoteKeyedStateHandle.getMetaStateHandle()); + incrementalRemoteKeyedStateHandle.getMetaDataStateHandle()); List castSharedStates = incrementalRemoteKeyedStateHandle.getSharedState().stream() .map( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 49a22a4d435be..1174d9ee18bb2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -2987,7 +2987,7 @@ void testSharedStateRegistrationOnRestore() throws Exception { handleAndLocalPath.getHandle(), TernaryBoolean.FALSE); } - verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()) + verify(incrementalKeyedStateHandle.getMetaDataStateHandle(), never()) .discardState(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ChangelogTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ChangelogTestUtils.java index 890010e1f6def..1329665ea219c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ChangelogTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ChangelogTestUtils.java @@ -80,7 +80,7 @@ public static class IncrementalStateHandleWrapper extends IncrementalRemoteKeyed stateHandle.getCheckpointId(), stateHandle.getSharedState(), stateHandle.getPrivateState(), - stateHandle.getMetaStateHandle(), + stateHandle.getMetaDataStateHandle(), stateHandle.getCheckpointedSize(), stateHandle.getStateHandleId()); this.stateHandle = stateHandle; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandleTest.java index 9b12b3a43a15c..1422bc8bcf074 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandleTest.java @@ -61,7 +61,7 @@ void testUnregisteredDiscarding() throws Exception { verifyDiscard(handleAndLocalPath.getHandle(), TernaryBoolean.TRUE); } - verify(stateHandle.getMetaStateHandle()).discardState(); + verify(stateHandle.getMetaDataStateHandle()).discardState(); } /** @@ -130,8 +130,8 @@ void testSharedStateDeRegistration() throws Exception { verify(handleAndLocalPath.getHandle(), times(0)).discardState(); } - verify(stateHandle1.getMetaStateHandle(), times(1)).discardState(); - verify(stateHandle2.getMetaStateHandle(), times(0)).discardState(); + verify(stateHandle1.getMetaDataStateHandle(), times(1)).discardState(); + verify(stateHandle2.getMetaDataStateHandle(), times(0)).discardState(); // We discard the second stateHandle2.discardState(); @@ -146,8 +146,8 @@ void testSharedStateDeRegistration() throws Exception { verifyDiscard(handleAndLocalPath.getHandle(), TernaryBoolean.TRUE); } - verify(stateHandle1.getMetaStateHandle(), times(1)).discardState(); - verify(stateHandle2.getMetaStateHandle(), times(1)).discardState(); + verify(stateHandle1.getMetaDataStateHandle(), times(1)).discardState(); + verify(stateHandle2.getMetaDataStateHandle(), times(1)).discardState(); } /** @@ -176,7 +176,7 @@ void testSharedStateReRegistration() throws Exception { // Everything should be discarded for this handle stateHandleZ.discardState(); - verify(stateHandleZ.getMetaStateHandle(), times(1)).discardState(); + verify(stateHandleZ.getMetaDataStateHandle(), times(1)).discardState(); // Close the first registry stateRegistryA.close(); @@ -188,16 +188,16 @@ void testSharedStateReRegistration() throws Exception { // Private state should still get discarded stateHandleY.discardState(); - verify(stateHandleY.getMetaStateHandle(), times(1)).discardState(); + verify(stateHandleY.getMetaDataStateHandle(), times(1)).discardState(); // This should still be unaffected - verify(stateHandleX.getMetaStateHandle(), never()).discardState(); + verify(stateHandleX.getMetaDataStateHandle(), never()).discardState(); // We re-register the handle with a new registry SharedStateRegistry sharedStateRegistryB = spy(new SharedStateRegistryImpl()); stateHandleX.registerSharedStates(sharedStateRegistryB, 0L); stateHandleX.discardState(); - verify(stateHandleX.getMetaStateHandle(), times(1)).discardState(); + verify(stateHandleX.getMetaDataStateHandle(), times(1)).discardState(); // Should be completely discarded because it is tracked through the new registry sharedStateRegistryB.unregisterUnusedState(1L); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java index 3c36c754f3be1..a62bbb4a70b76 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java @@ -252,7 +252,7 @@ private void restoreBaseDBFromDownloadedState(StateHandleDownloadSpec downloaded stateHandle.getCheckpointId(), new DirectoryStateHandle(downloadedState.getDownloadDestination()), stateHandle.getKeyGroupRange(), - stateHandle.getMetaStateHandle(), + stateHandle.getMetaDataStateHandle(), stateHandle.getSharedState())); } @@ -260,7 +260,7 @@ private void restoreBaseDBFromDownloadedState(StateHandleDownloadSpec downloaded private void restoreBaseDBFromLocalState(IncrementalLocalKeyedStateHandle localKeyedStateHandle) throws Exception { KeyedBackendSerializationProxy serializationProxy = - readMetaData(localKeyedStateHandle.getMetaDataState()); + readMetaData(localKeyedStateHandle.getMetaDataStateHandle()); List stateMetaInfoSnapshots = serializationProxy.getStateMetaInfoSnapshots(); @@ -474,7 +474,7 @@ private RestoredDBInstance restoreTempDBInstanceFromDownloadedState( StateHandleDownloadSpec downloadRequest) throws Exception { KeyedBackendSerializationProxy serializationProxy = - readMetaData(downloadRequest.getStateHandle().getMetaStateHandle()); + readMetaData(downloadRequest.getStateHandle().getMetaDataStateHandle()); // read meta data List stateMetaInfoSnapshots = serializationProxy.getStateMetaInfoSnapshots(); From 0be231972d5d687924db22ae81d6a10049f7c43e Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 27 Oct 2023 11:01:49 +0200 Subject: [PATCH 037/104] [FLINK-33341][state] Refactoring: consolidate equals/hashCode/toString for incremental state handle classes. --- .../state/AbstractIncrementalStateHandle.java | 36 ++++++++++ .../IncrementalLocalKeyedStateHandle.java | 35 +--------- .../IncrementalRemoteKeyedStateHandle.java | 69 ++----------------- 3 files changed, 43 insertions(+), 97 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java index 8c7ea74c33c88..85f12329f5715 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java @@ -21,6 +21,7 @@ import javax.annotation.Nonnull; import java.util.List; +import java.util.Objects; import java.util.UUID; /** Abstract superclass for all {@link IncrementalKeyedStateHandle}. */ @@ -103,4 +104,39 @@ public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { ? null : this; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AbstractIncrementalStateHandle that = (AbstractIncrementalStateHandle) o; + return Objects.equals(stateHandleId, that.stateHandleId); + } + + @Override + public int hashCode() { + return stateHandleId.hashCode(); + } + + @Override + public String toString() { + return "AbstractIncrementalStateHandle{" + + "checkpointId=" + + checkpointId + + ", backendIdentifier=" + + backendIdentifier + + ", keyGroupRange=" + + keyGroupRange + + ", sharedState=" + + sharedState + + ", metaStateHandle=" + + metaStateHandle + + ", stateHandleId=" + + stateHandleId + + '}'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java index f854c111c6e9b..ac457f00622b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java @@ -94,44 +94,13 @@ public long getStateSize() { return directoryStateHandle.getStateSize() + metaStateHandle.getStateSize(); } - @Override - public int hashCode() { - int result = directoryStateHandle.hashCode(); - result = 31 * result + getKeyGroupRange().hashCode(); - result = 31 * result + getMetaDataStateHandle().hashCode(); - return result; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - - IncrementalLocalKeyedStateHandle that = (IncrementalLocalKeyedStateHandle) o; - - return getKeyGroupRange().equals(that.keyGroupRange) - && getMetaDataStateHandle().equals(that.getMetaDataStateHandle()); - } - @Override public String toString() { return "IncrementalLocalKeyedStateHandle{" - + "metaDataState=" - + metaStateHandle - + "} " - + "DirectoryKeyedStateHandle{" + "directoryStateHandle=" + directoryStateHandle - + ", keyGroupRange=" - + keyGroupRange - + '}'; + + "} " + + super.toString(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java index 86a5b59c16869..41b9a0466cc2b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java @@ -306,75 +306,16 @@ IncrementalRemoteKeyedStateHandle copy() { stateHandleId); } - /** - * This method is should only be called in tests! This should never serve as key in a hash map. - */ - @VisibleForTesting - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - IncrementalRemoteKeyedStateHandle that = (IncrementalRemoteKeyedStateHandle) o; - - if (!getStateHandleId().equals(that.getStateHandleId())) { - return false; - } - if (getCheckpointId() != that.getCheckpointId()) { - return false; - } - if (!getBackendIdentifier().equals(that.getBackendIdentifier())) { - return false; - } - if (!getKeyGroupRange().equals(that.getKeyGroupRange())) { - return false; - } - if (!getSharedState().equals(that.getSharedState())) { - return false; - } - if (!getPrivateState().equals(that.getPrivateState())) { - return false; - } - return getMetaDataStateHandle().equals(that.getMetaDataStateHandle()); - } - - /** This method should only be called in tests! This should never serve as key in a hash map. */ - @VisibleForTesting - @Override - public int hashCode() { - int result = getBackendIdentifier().hashCode(); - result = 31 * result + getKeyGroupRange().hashCode(); - result = 31 * result + (int) (getCheckpointId() ^ (getCheckpointId() >>> 32)); - result = 31 * result + getSharedState().hashCode(); - result = 31 * result + getPrivateState().hashCode(); - result = 31 * result + getMetaDataStateHandle().hashCode(); - result = 31 * result + getStateHandleId().hashCode(); - return result; - } - @Override public String toString() { return "IncrementalRemoteKeyedStateHandle{" - + "backendIdentifier=" - + backendIdentifier - + ", stateHandleId=" - + stateHandleId - + ", keyGroupRange=" - + keyGroupRange - + ", checkpointId=" - + checkpointId - + ", sharedState=" - + sharedState - + ", privateState=" + + "privateState=" + privateState - + ", metaStateHandle=" - + metaStateHandle + + ", persistedSizeOfThisCheckpoint=" + + persistedSizeOfThisCheckpoint + ", registered=" + (sharedStateRegistry != null) - + '}'; + + "} " + + super.toString(); } } From 05cdb299e26db931a328bdc4e536dfb7effdb905 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 25 Oct 2023 11:50:55 +0200 Subject: [PATCH 038/104] [FLINK-33341][state] Add support for rescaling from local keyed state to RockDBIncrementalRestoreOperation. --- .../RocksDBIncrementalCheckpointUtils.java | 8 +- .../state/StateHandleDownloadSpec.java | 12 + .../RocksDBIncrementalRestoreOperation.java | 233 +++++++++--------- 3 files changed, 129 insertions(+), 124 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java index 5412170987604..a835d10c481a8 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java @@ -175,14 +175,14 @@ public static boolean beforeThePrefixBytes(@Nonnull byte[] bytes, @Nonnull byte[ * @return The best candidate or null if no candidate was a good fit. */ @Nullable - public static KeyedStateHandle chooseTheBestStateHandleForInitial( - @Nonnull Collection restoreStateHandles, + public static T chooseTheBestStateHandleForInitial( + @Nonnull Collection restoreStateHandles, @Nonnull KeyGroupRange targetKeyGroupRange, double overlapFractionThreshold) { - KeyedStateHandle bestStateHandle = null; + T bestStateHandle = null; Score bestScore = Score.MIN; - for (KeyedStateHandle rawStateHandle : restoreStateHandles) { + for (T rawStateHandle : restoreStateHandles) { Score handleScore = stateHandleEvaluator( rawStateHandle, targetKeyGroupRange, overlapFractionThreshold); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java index 93a33fdc6fa97..5f37f84921ff4 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java @@ -18,6 +18,8 @@ package org.apache.flink.contrib.streaming.state; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import java.nio.file.Path; @@ -46,4 +48,14 @@ public IncrementalRemoteKeyedStateHandle getStateHandle() { public Path getDownloadDestination() { return downloadDestination; } + + public IncrementalLocalKeyedStateHandle createLocalStateHandleForDownloadedState() { + return new IncrementalLocalKeyedStateHandle( + stateHandle.getBackendIdentifier(), + stateHandle.getCheckpointId(), + new DirectoryStateHandle(downloadDestination), + stateHandle.getKeyGroupRange(), + stateHandle.getMetaDataStateHandle(), + stateHandle.getSharedState()); + } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java index a62bbb4a70b76..11d4756ae1384 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java @@ -35,7 +35,6 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.state.BackendBuildingException; import org.apache.flink.runtime.state.CompositeKeySerializationUtils; -import org.apache.flink.runtime.state.DirectoryStateHandle; import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; import org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath; import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; @@ -44,11 +43,9 @@ import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; -import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StateSerializerProvider; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; -import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FileUtils; import org.apache.flink.util.IOUtils; import org.apache.flink.util.Preconditions; @@ -227,35 +224,12 @@ private void restoreBaseDBFromRemoteState(IncrementalRemoteKeyedStateHandle stat new StateHandleDownloadSpec(stateHandle, tmpRestoreInstancePath); try { transferRemoteStateToLocalDirectory(Collections.singletonList(downloadRequest)); - restoreBaseDBFromDownloadedState(downloadRequest); + restoreBaseDBFromLocalState(downloadRequest.createLocalStateHandleForDownloadedState()); } finally { cleanUpPathQuietly(downloadRequest.getDownloadDestination()); } } - /** - * This helper method creates a {@link IncrementalLocalKeyedStateHandle} for state that was - * previously downloaded for a {@link IncrementalRemoteKeyedStateHandle} and then invokes the - * restore procedure for local state on the downloaded state. - * - * @param downloadedState the specification of a completed state download. - * @throws Exception for restore problems. - */ - private void restoreBaseDBFromDownloadedState(StateHandleDownloadSpec downloadedState) - throws Exception { - // since we transferred all remote state to a local directory, we can use the same code - // as for local recovery. - IncrementalRemoteKeyedStateHandle stateHandle = downloadedState.getStateHandle(); - restoreBaseDBFromLocalState( - new IncrementalLocalKeyedStateHandle( - stateHandle.getBackendIdentifier(), - stateHandle.getCheckpointId(), - new DirectoryStateHandle(downloadedState.getDownloadDestination()), - stateHandle.getKeyGroupRange(), - stateHandle.getMetaDataStateHandle(), - stateHandle.getSharedState())); - } - /** Restores RocksDB instance from local state. */ private void restoreBaseDBFromLocalState(IncrementalLocalKeyedStateHandle localKeyedStateHandle) throws Exception { @@ -304,113 +278,133 @@ private void restoreWithRescaling(Collection restoreStateHandl Preconditions.checkArgument(restoreStateHandles != null && !restoreStateHandles.isEmpty()); - Map allDownloadSpecs = - CollectionUtil.newHashMapWithExpectedSize(restoreStateHandles.size()); - - // Choose the best state handle for the initial DB - final KeyedStateHandle selectedInitialHandle = - RocksDBIncrementalCheckpointUtils.chooseTheBestStateHandleForInitial( - restoreStateHandles, keyGroupRange, overlapFractionThreshold); + final List allDownloadSpecs = new ArrayList<>(); - Preconditions.checkNotNull(selectedInitialHandle); + final List localKeyedStateHandles = + new ArrayList<>(restoreStateHandles.size()); final Path absolutInstanceBasePath = instanceBasePath.getAbsoluteFile().toPath(); // Prepare and collect all the download request to pull remote state to a local directory for (KeyedStateHandle stateHandle : restoreStateHandles) { - if (!(stateHandle instanceof IncrementalRemoteKeyedStateHandle)) { + if (stateHandle instanceof IncrementalRemoteKeyedStateHandle) { + StateHandleDownloadSpec downloadRequest = + new StateHandleDownloadSpec( + (IncrementalRemoteKeyedStateHandle) stateHandle, + absolutInstanceBasePath.resolve(UUID.randomUUID().toString())); + allDownloadSpecs.add(downloadRequest); + } else if (stateHandle instanceof IncrementalLocalKeyedStateHandle) { + localKeyedStateHandles.add((IncrementalLocalKeyedStateHandle) stateHandle); + } else { throw unexpectedStateHandleException( IncrementalRemoteKeyedStateHandle.class, stateHandle.getClass()); } - StateHandleDownloadSpec downloadRequest = - new StateHandleDownloadSpec( - (IncrementalRemoteKeyedStateHandle) stateHandle, - absolutInstanceBasePath.resolve(UUID.randomUUID().toString())); - allDownloadSpecs.put(stateHandle.getStateHandleId(), downloadRequest); } - // Process all state downloads - transferRemoteStateToLocalDirectory(allDownloadSpecs.values()); - - // Init the base DB instance with the initial state - initBaseDBForRescaling(allDownloadSpecs.remove(selectedInitialHandle.getStateHandleId())); - - // Transfer remaining key-groups from temporary instance into base DB - byte[] startKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; - CompositeKeySerializationUtils.serializeKeyGroup( - keyGroupRange.getStartKeyGroup(), startKeyGroupPrefixBytes); - - byte[] stopKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; - CompositeKeySerializationUtils.serializeKeyGroup( - keyGroupRange.getEndKeyGroup() + 1, stopKeyGroupPrefixBytes); - - // Insert all remaining state through creating temporary RocksDB instances - for (StateHandleDownloadSpec downloadRequest : allDownloadSpecs.values()) { - logger.info( - "Starting to restore from state handle: {} with rescaling.", - downloadRequest.getStateHandle()); - - try (RestoredDBInstance tmpRestoreDBInfo = - restoreTempDBInstanceFromDownloadedState(downloadRequest); - RocksDBWriteBatchWrapper writeBatchWrapper = - new RocksDBWriteBatchWrapper( - this.rocksHandle.getDb(), writeBatchSize)) { - - List tmpColumnFamilyDescriptors = - tmpRestoreDBInfo.columnFamilyDescriptors; - List tmpColumnFamilyHandles = - tmpRestoreDBInfo.columnFamilyHandles; - - // iterating only the requested descriptors automatically skips the default column - // family handle - for (int descIdx = 0; descIdx < tmpColumnFamilyDescriptors.size(); ++descIdx) { - ColumnFamilyHandle tmpColumnFamilyHandle = tmpColumnFamilyHandles.get(descIdx); - - ColumnFamilyHandle targetColumnFamilyHandle = - this.rocksHandle.getOrRegisterStateColumnFamilyHandle( - null, - tmpRestoreDBInfo.stateMetaInfoSnapshots.get(descIdx)) - .columnFamilyHandle; - - try (RocksIteratorWrapper iterator = - RocksDBOperationUtils.getRocksIterator( - tmpRestoreDBInfo.db, - tmpColumnFamilyHandle, - tmpRestoreDBInfo.readOptions)) { - - iterator.seek(startKeyGroupPrefixBytes); - - while (iterator.isValid()) { - - if (RocksDBIncrementalCheckpointUtils.beforeThePrefixBytes( - iterator.key(), stopKeyGroupPrefixBytes)) { - writeBatchWrapper.put( - targetColumnFamilyHandle, iterator.key(), iterator.value()); - } else { - // Since the iterator will visit the record according to the sorted - // order, - // we can just break here. - break; - } + allDownloadSpecs.stream() + .map(StateHandleDownloadSpec::createLocalStateHandleForDownloadedState) + .forEach(localKeyedStateHandles::add); - iterator.next(); - } - } // releases native iterator resources - } + // Choose the best state handle for the initial DB + final IncrementalLocalKeyedStateHandle selectedInitialHandle = + RocksDBIncrementalCheckpointUtils.chooseTheBestStateHandleForInitial( + localKeyedStateHandles, keyGroupRange, overlapFractionThreshold); + Preconditions.checkNotNull(selectedInitialHandle); + // Remove the selected handle from the list so that we don't restore it twice. + localKeyedStateHandles.remove(selectedInitialHandle); + + try { + // Process all state downloads + transferRemoteStateToLocalDirectory(allDownloadSpecs); + + // Init the base DB instance with the initial state + initBaseDBForRescaling(selectedInitialHandle); + + // Transfer remaining key-groups from temporary instance into base DB + byte[] startKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; + CompositeKeySerializationUtils.serializeKeyGroup( + keyGroupRange.getStartKeyGroup(), startKeyGroupPrefixBytes); + + byte[] stopKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; + CompositeKeySerializationUtils.serializeKeyGroup( + keyGroupRange.getEndKeyGroup() + 1, stopKeyGroupPrefixBytes); + + // Insert all remaining state through creating temporary RocksDB instances + for (IncrementalLocalKeyedStateHandle stateHandle : localKeyedStateHandles) { logger.info( - "Finished restoring from state handle: {} with rescaling.", - downloadRequest.getStateHandle()); - } finally { - cleanUpPathQuietly(downloadRequest.getDownloadDestination()); + "Starting to restore from state handle: {} with rescaling.", stateHandle); + + try (RestoredDBInstance tmpRestoreDBInfo = + restoreTempDBInstanceFromLocalState(stateHandle); + RocksDBWriteBatchWrapper writeBatchWrapper = + new RocksDBWriteBatchWrapper( + this.rocksHandle.getDb(), writeBatchSize)) { + + List tmpColumnFamilyDescriptors = + tmpRestoreDBInfo.columnFamilyDescriptors; + List tmpColumnFamilyHandles = + tmpRestoreDBInfo.columnFamilyHandles; + + // iterating only the requested descriptors automatically skips the default + // column + // family handle + for (int descIdx = 0; descIdx < tmpColumnFamilyDescriptors.size(); ++descIdx) { + ColumnFamilyHandle tmpColumnFamilyHandle = + tmpColumnFamilyHandles.get(descIdx); + + ColumnFamilyHandle targetColumnFamilyHandle = + this.rocksHandle.getOrRegisterStateColumnFamilyHandle( + null, + tmpRestoreDBInfo.stateMetaInfoSnapshots.get( + descIdx)) + .columnFamilyHandle; + + try (RocksIteratorWrapper iterator = + RocksDBOperationUtils.getRocksIterator( + tmpRestoreDBInfo.db, + tmpColumnFamilyHandle, + tmpRestoreDBInfo.readOptions)) { + + iterator.seek(startKeyGroupPrefixBytes); + + while (iterator.isValid()) { + + if (RocksDBIncrementalCheckpointUtils.beforeThePrefixBytes( + iterator.key(), stopKeyGroupPrefixBytes)) { + writeBatchWrapper.put( + targetColumnFamilyHandle, + iterator.key(), + iterator.value()); + } else { + // Since the iterator will visit the record according to the + // sorted + // order, + // we can just break here. + break; + } + + iterator.next(); + } + } // releases native iterator resources + } + logger.info( + "Finished restoring from state handle: {} with rescaling.", + stateHandle); + } } + } finally { + // Cleanup all download directories + allDownloadSpecs.stream() + .map(StateHandleDownloadSpec::getDownloadDestination) + .forEach(this::cleanUpPathQuietly); } } - private void initBaseDBForRescaling(StateHandleDownloadSpec downloadedInitialState) + private void initBaseDBForRescaling(IncrementalLocalKeyedStateHandle stateHandle) throws Exception { // 1. Restore base DB from selected initial handle - restoreBaseDBFromDownloadedState(downloadedInitialState); + restoreBaseDBFromLocalState(stateHandle); // 2. Clip the base DB instance try { @@ -418,7 +412,7 @@ private void initBaseDBForRescaling(StateHandleDownloadSpec downloadedInitialSta this.rocksHandle.getDb(), this.rocksHandle.getColumnFamilyHandles(), keyGroupRange, - downloadedInitialState.getStateHandle().getKeyGroupRange(), + stateHandle.getKeyGroupRange(), keyGroupPrefixBytes); } catch (RocksDBException e) { String errMsg = "Failed to clip DB after initialization."; @@ -470,11 +464,10 @@ public void close() { } } - private RestoredDBInstance restoreTempDBInstanceFromDownloadedState( - StateHandleDownloadSpec downloadRequest) throws Exception { - + private RestoredDBInstance restoreTempDBInstanceFromLocalState( + IncrementalLocalKeyedStateHandle stateHandle) throws Exception { KeyedBackendSerializationProxy serializationProxy = - readMetaData(downloadRequest.getStateHandle().getMetaDataStateHandle()); + readMetaData(stateHandle.getMetaDataStateHandle()); // read meta data List stateMetaInfoSnapshots = serializationProxy.getStateMetaInfoSnapshots(); @@ -487,7 +480,7 @@ private RestoredDBInstance restoreTempDBInstanceFromDownloadedState( RocksDB restoreDb = RocksDBOperationUtils.openDB( - downloadRequest.getDownloadDestination().toString(), + stateHandle.getDirectoryStateHandle().getDirectory().toString(), columnFamilyDescriptors, columnFamilyHandles, RocksDBOperationUtils.createColumnFamilyOptions( From a4ad86fb083f90503938a9c3d816cdda9dc22427 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 13 Oct 2023 16:27:39 +0200 Subject: [PATCH 039/104] [FLINK-33341][state] Add support for rescaling from local keyed state to PrioritizedOperatorSubtaskState. --- .../PrioritizedOperatorSubtaskState.java | 132 +++++++++++-- .../PrioritizedOperatorSubtaskStateTest.java | 184 ++++++++++++++++-- .../checkpoint/StateHandleDummyUtil.java | 29 ++- 3 files changed, 305 insertions(+), 40 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java index ef9bcd0440b12..e41bcfe73385b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java @@ -31,10 +31,14 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.stream.Collectors; /** * This class is a wrapper over multiple alternative {@link OperatorSubtaskState} that are (partial) @@ -286,14 +290,14 @@ public PrioritizedOperatorSubtaskState build() { } return new PrioritizedOperatorSubtaskState( - resolvePrioritizedAlternatives( + computePrioritizedAlternatives( jobManagerState.getManagedKeyedState(), managedKeyedAlternatives, - eqStateApprover(KeyedStateHandle::getKeyGroupRange)), - resolvePrioritizedAlternatives( + KeyedStateHandle::getKeyGroupRange), + computePrioritizedAlternatives( jobManagerState.getRawKeyedState(), rawKeyedAlternatives, - eqStateApprover(KeyedStateHandle::getKeyGroupRange)), + KeyedStateHandle::getKeyGroupRange), resolvePrioritizedAlternatives( jobManagerState.getManagedOperatorState(), managedOperatorAlternatives, @@ -313,22 +317,121 @@ public PrioritizedOperatorSubtaskState build() { restoredCheckpointId); } + /** + * This method creates an alternative recovery option by replacing as much job manager state + * with higher prioritized (=local) alternatives as possible. + * + * @param jobManagerState the state that the task got assigned from the job manager (this + * state lives in remote storage). + * @param alternativesByPriority local alternatives to the job manager state, ordered by + * priority. + * @param identityExtractor function to extract an identifier from a state object. + * @return prioritized state alternatives. + * @param the type of the state objects we process. + * @param the type of object that represents the id the state object type. + */ + + List> computePrioritizedAlternatives( + StateObjectCollection jobManagerState, + List> alternativesByPriority, + Function identityExtractor) { + + if (alternativesByPriority != null + && !alternativesByPriority.isEmpty() + && jobManagerState.hasState()) { + + Optional> mergedAlternative = + tryComputeMixedLocalAndRemoteAlternative( + jobManagerState, alternativesByPriority, identityExtractor); + + // Return the mix of local/remote state as first and pure remote state as second + // alternative (in case that we fail to recover from the local state, e.g. because + // of corruption). + if (mergedAlternative.isPresent()) { + return Arrays.asList(mergedAlternative.get(), jobManagerState); + } + } + + return Collections.singletonList(jobManagerState); + } + + /** + * This method creates an alternative recovery option by replacing as much job manager state + * with higher prioritized (=local) alternatives as possible. Returns empty Optional if the + * JM state is empty or nothing could be replaced. + * + * @param jobManagerState the state that the task got assigned from the job manager (this + * state lives in remote storage). + * @param alternativesByPriority local alternatives to the job manager state, ordered by + * priority. + * @param identityExtractor function to extract an identifier from a state object. + * @return A state collection where all JM state handles for which we could find local * + * alternatives are replaced by the alternative with the highest priority. Empty + * optional if no state could be replaced. + * @param the type of the state objects we process. + * @param the type of object that represents the id the state object type. + */ + static + Optional> + tryComputeMixedLocalAndRemoteAlternative( + StateObjectCollection jobManagerState, + List> alternativesByPriority, + Function identityExtractor) { + + List result = Collections.emptyList(); + + // Build hash index over ids of the JM state + Map indexById = + jobManagerState.stream() + .collect(Collectors.toMap(identityExtractor, Function.identity())); + + // Move through all alternative in order from high to low priority + for (StateObjectCollection alternative : alternativesByPriority) { + // Check all the state objects in the alternative if they can replace JM state + for (STATE_OBJ_TYPE stateHandle : alternative) { + // Remove the current state object's id from the index to check for a match + if (indexById.remove(identityExtractor.apply(stateHandle)) != null) { + if (result.isEmpty()) { + // Lazy init result collection + result = new ArrayList<>(jobManagerState.size()); + } + // If the id was still in the index, replace with higher prio alternative + result.add(stateHandle); + + // If the index is empty we are already done, all JM state was replaces with + // the best alternative. + if (indexById.isEmpty()) { + return Optional.of(new StateObjectCollection<>(result)); + } + } + } + } + + // Nothing useful to return + if (result.isEmpty()) { + return Optional.empty(); + } + + // Add all remaining JM state objects that we could not replace from the index to the + // final result + result.addAll(indexById.values()); + return Optional.of(new StateObjectCollection<>(result)); + } + /** * This helper method resolves the dependencies between the ground truth of the operator * state obtained from the job manager and potential alternatives for recovery, e.g. from a * task-local source. */ - protected - List> resolvePrioritizedAlternatives( - StateObjectCollection jobManagerState, - List> alternativesByPriority, - BiFunction approveFun) { + List> resolvePrioritizedAlternatives( + StateObjectCollection jobManagerState, + List> alternativesByPriority, + BiFunction approveFun) { // Nothing to resolve if there are no alternatives, or the ground truth has already no - // state, or if we can - // assume that a rescaling happened because we find more than one handle in the JM state - // (this is more a sanity - // check). + // state, or if we can assume that a rescaling happened because we find more than one + // handle in the JM state + // (this is more a sanity check). if (alternativesByPriority == null || alternativesByPriority.isEmpty() || !jobManagerState.hasState() @@ -347,8 +450,7 @@ List> resolvePrioritizedAlternatives( for (StateObjectCollection alternative : alternativesByPriority) { // We found an alternative to the JM state if it has state, we have a 1:1 - // relationship, and the - // approve-function signaled true. + // relationship, and the approve-function signaled true. if (alternative != null && alternative.hasState() && alternative.size() == 1 diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java index 11f41bc8baac4..fa892cea9daae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java @@ -27,15 +27,19 @@ import org.apache.flink.runtime.state.StateObject; import org.apache.flink.util.Preconditions; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.function.Function; +import java.util.function.IntFunction; import java.util.stream.Collectors; import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewInputChannelStateHandle; @@ -51,6 +55,107 @@ class PrioritizedOperatorSubtaskStateTest { private static final Random RANDOM = new Random(0x42); + @Test + void testTryCreateMixedLocalAndRemoteAlternative() { + testTryCreateMixedLocalAndRemoteAlternative( + StateHandleDummyUtil::createKeyedStateHandleFromSeed, + KeyedStateHandle::getKeyGroupRange); + } + + void testTryCreateMixedLocalAndRemoteAlternative( + IntFunction stateHandleFactory, Function idExtractor) { + + SH remote0 = stateHandleFactory.apply(0); + SH remote1 = stateHandleFactory.apply(1); + SH remote2 = stateHandleFactory.apply(2); + SH remote3 = stateHandleFactory.apply(3); + + List jmState = Arrays.asList(remote0, remote1, remote2, remote3); + + SH local0 = stateHandleFactory.apply(0); + SH local3a = stateHandleFactory.apply(3); + + List alternativeA = Arrays.asList(local0, local3a); + + SH local1 = stateHandleFactory.apply(1); + SH local3b = stateHandleFactory.apply(3); + SH local5 = stateHandleFactory.apply(5); + + List alternativeB = Arrays.asList(local1, local3b, local5); + + List> alternatives = + Arrays.asList( + new StateObjectCollection<>(alternativeA), + new StateObjectCollection<>(Collections.emptyList()), + new StateObjectCollection<>(alternativeB)); + + StateObjectCollection result = + PrioritizedOperatorSubtaskState.Builder.tryComputeMixedLocalAndRemoteAlternative( + new StateObjectCollection<>(jmState), alternatives, idExtractor) + .get(); + + assertThat(result).hasSameElementsAs(Arrays.asList(local0, local1, remote2, local3a)); + } + + @Test + void testTryCreateMixedLocalAndRemoteAlternativeEmptyAlternative() { + testTryCreateMixedLocalAndRemoteAlternativeEmptyAlternative( + StateHandleDummyUtil::createKeyedStateHandleFromSeed, + KeyedStateHandle::getKeyGroupRange); + } + + void testTryCreateMixedLocalAndRemoteAlternativeEmptyAlternative( + IntFunction stateHandleFactory, Function idExtractor) { + List jmState = + Arrays.asList( + stateHandleFactory.apply(0), + stateHandleFactory.apply(1), + stateHandleFactory.apply(2), + stateHandleFactory.apply(3)); + + Assertions.assertFalse( + PrioritizedOperatorSubtaskState.Builder.tryComputeMixedLocalAndRemoteAlternative( + new StateObjectCollection<>(jmState), + Collections.emptyList(), + idExtractor) + .isPresent()); + + Assertions.assertFalse( + PrioritizedOperatorSubtaskState.Builder.tryComputeMixedLocalAndRemoteAlternative( + new StateObjectCollection<>(jmState), + Collections.singletonList(new StateObjectCollection<>()), + idExtractor) + .isPresent()); + } + + @Test + void testTryCreateMixedLocalAndRemoteAlternativeEmptyJMState() { + testTryCreateMixedLocalAndRemoteAlternativeEmptyJMState( + StateHandleDummyUtil::createKeyedStateHandleFromSeed, + KeyedStateHandle::getKeyGroupRange); + } + + void testTryCreateMixedLocalAndRemoteAlternativeEmptyJMState( + IntFunction stateHandleFactory, Function idExtractor) { + List alternativeA = + Arrays.asList(stateHandleFactory.apply(0), stateHandleFactory.apply(3)); + + Assertions.assertFalse( + PrioritizedOperatorSubtaskState.Builder.tryComputeMixedLocalAndRemoteAlternative( + new StateObjectCollection<>(Collections.emptyList()), + Collections.singletonList( + new StateObjectCollection<>(alternativeA)), + idExtractor) + .isPresent()); + + Assertions.assertFalse( + PrioritizedOperatorSubtaskState.Builder.tryComputeMixedLocalAndRemoteAlternative( + new StateObjectCollection<>(Collections.emptyList()), + Collections.emptyList(), + KeyedStateHandle::getKeyGroupRange) + .isPresent()); + } + /** * This tests attempts to test (almost) the full space of significantly different options for * verifying and prioritizing {@link OperatorSubtaskState} options for local recovery over @@ -106,16 +211,17 @@ void testPrioritization() { : onlyPrimary)) .isTrue(); - assertThat( - checkResultAsExpected( - OperatorSubtaskState::getManagedKeyedState, - PrioritizedOperatorSubtaskState - ::getPrioritizedManagedKeyedState, - prioritizedOperatorSubtaskState, - primaryAndFallback.getManagedKeyedState().size() == 1 - ? validAlternatives - : onlyPrimary)) - .isTrue(); + StateObjectCollection expManagedKeyed = + computeExpectedMixedState( + orderedAlternativesList, + primaryAndFallback, + OperatorSubtaskState::getManagedKeyedState, + KeyedStateHandle::getKeyGroupRange); + + assertResultAsExpected( + expManagedKeyed, + primaryAndFallback.getManagedKeyedState(), + prioritizedOperatorSubtaskState.getPrioritizedManagedKeyedState()); assertThat( checkResultAsExpected( @@ -128,16 +234,17 @@ void testPrioritization() { : onlyPrimary)) .isTrue(); - assertThat( - checkResultAsExpected( - OperatorSubtaskState::getRawKeyedState, - PrioritizedOperatorSubtaskState - ::getPrioritizedRawKeyedState, - prioritizedOperatorSubtaskState, - primaryAndFallback.getRawKeyedState().size() == 1 - ? validAlternatives - : onlyPrimary)) - .isTrue(); + StateObjectCollection expRawKeyed = + computeExpectedMixedState( + orderedAlternativesList, + primaryAndFallback, + OperatorSubtaskState::getRawKeyedState, + KeyedStateHandle::getKeyGroupRange); + + assertResultAsExpected( + expRawKeyed, + primaryAndFallback.getRawKeyedState(), + prioritizedOperatorSubtaskState.getPrioritizedRawKeyedState()); } } } @@ -390,4 +497,41 @@ private static T deepCopy(T stateObject) { throw new IllegalStateException(); } } + + private StateObjectCollection computeExpectedMixedState( + List orderedAlternativesList, + OperatorSubtaskState primaryAndFallback, + Function> stateExtractor, + Function idExtractor) { + + List reverseAlternatives = new ArrayList<>(orderedAlternativesList); + Collections.reverse(reverseAlternatives); + + Map map = + stateExtractor.apply(primaryAndFallback).stream() + .collect(Collectors.toMap(idExtractor, Function.identity())); + + reverseAlternatives.stream() + .flatMap(x -> stateExtractor.apply(x).stream()) + .forEach(x -> map.replace(idExtractor.apply(x), x)); + + return new StateObjectCollection<>(map.values()); + } + + static void assertResultAsExpected( + StateObjectCollection expected, + StateObjectCollection primary, + List> actual) { + Assertions.assertTrue(!actual.isEmpty() && actual.size() <= 2); + Assertions.assertTrue(isSameContentUnordered(expected, actual.get(0))); + if (actual.size() == 1) { + Assertions.assertTrue(isSameContentUnordered(primary, actual.get(0))); + } else { + Assertions.assertTrue(isSameContentUnordered(primary, actual.get(1))); + } + } + + static boolean isSameContentUnordered(Collection a, Collection b) { + return a.size() == b.size() && a.containsAll(b); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java index f7e4a7ef2dc15..52a8bf032b622 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java @@ -55,7 +55,8 @@ public static OperatorStateHandle createNewOperatorStateHandle( OperatorStateHandle.StateMetaInfo metaInfo = new OperatorStateHandle.StateMetaInfo( offsets, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE); - operatorStateMetaData.put(String.valueOf(UUID.randomUUID()), metaInfo); + operatorStateMetaData.put( + String.valueOf(new UUID(random.nextLong(), random.nextLong())), metaInfo); } return new OperatorStreamStateHandle( operatorStateMetaData, createStreamStateHandle(numNamedStates, random)); @@ -65,7 +66,8 @@ private static ByteStreamStateHandle createStreamStateHandle( int numNamedStates, Random random) { byte[] streamData = new byte[numNamedStates * 4]; random.nextBytes(streamData); - return new ByteStreamStateHandle(String.valueOf(UUID.randomUUID()), streamData); + return new ByteStreamStateHandle( + String.valueOf(new UUID(random.nextLong(), random.nextLong())), streamData); } /** Creates a new test {@link KeyedStateHandle} for the given key-group. */ @@ -149,11 +151,11 @@ public static InputChannelStateHandle createNewInputChannelStateHandle( } public static ResultSubpartitionStateHandle createNewResultSubpartitionStateHandle( - int i, Random random) { + int numNamedStates, Random random) { return new ResultSubpartitionStateHandle( new ResultSubpartitionInfo(random.nextInt(), random.nextInt()), - createStreamStateHandle(i, random), - genOffsets(i, random)); + createStreamStateHandle(numNamedStates, random), + genOffsets(numNamedStates, random)); } private static ArrayList genOffsets(int size, Random random) { @@ -164,6 +166,23 @@ private static ArrayList genOffsets(int size, Random random) { return offsets; } + public static KeyedStateHandle createKeyedStateHandleFromSeed(int seed) { + return createNewKeyedStateHandle(KeyGroupRange.of(seed * 4, seed * 4 + 3)); + } + + public static OperatorStateHandle createOperatorStateHandleFromSeed(int seed) { + return createNewOperatorStateHandle(1 + (seed % 3), new Random(seed)); + } + + public static InputChannelStateHandle createInputChannelStateHandleFromSeed(int seed) { + return createNewInputChannelStateHandle(1 + (seed % 3), new Random(seed)); + } + + public static ResultSubpartitionStateHandle createResultSubpartitionStateHandleFromSeed( + int seed) { + return createNewResultSubpartitionStateHandle(1 + (seed % 3), new Random(seed)); + } + /** KeyedStateHandle that only holds a key-group information. */ private static class DummyKeyedStateHandle implements KeyedStateHandle { From dcce3764a4500b2006cd260677169d14c553a3eb Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 19 Oct 2023 17:10:55 +0200 Subject: [PATCH 040/104] [FLINK-33375] Make it possible to run ValuesSource infinitely --- .../source/DynamicFilteringValuesSource.java | 11 ++- .../DynamicFilteringValuesSourceReader.java | 34 +++++-- .../connector/source/TerminatingLogic.java | 45 +++++++++ .../flink/connector/source/ValuesSource.java | 15 ++- .../connector/source/ValuesSourceReader.java | 26 ++++-- ...ynamicFilteringValuesSourceEnumerator.java | 16 +++- .../enumerator/ValuesSourceEnumerator.java | 8 +- .../split/ValuesSourcePartitionSplit.java | 16 ++++ .../ValuesSourcePartitionSplitSerializer.java | 6 +- .../source/split/ValuesSourceSplit.java | 15 +++ .../split/ValuesSourceSplitSerializer.java | 5 +- .../factories/TestValuesTableFactory.java | 91 +++++++++++++++---- 12 files changed, 242 insertions(+), 46 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/TerminatingLogic.java diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java index 0622169b8ccb9..4a011476ecbe4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java @@ -52,16 +52,22 @@ public class DynamicFilteringValuesSource implements Source { private final TypeSerializer serializer; + private final TerminatingLogic terminatingLogic; + private final Boundedness boundedness; private Map, byte[]> serializedElements; private Map, Integer> counts; private final List dynamicFilteringFields; public DynamicFilteringValuesSource( + TerminatingLogic terminatingLogic, + Boundedness boundedness, Map, Collection> elements, TypeSerializer serializer, List dynamicFilteringFields) { this.serializer = serializer; this.dynamicFilteringFields = dynamicFilteringFields; + this.terminatingLogic = terminatingLogic; + this.boundedness = boundedness; serializeElements(serializer, elements); } @@ -91,7 +97,7 @@ private void serializeElements( @Override public Boundedness getBoundedness() { - return Boundedness.BOUNDED; + return boundedness; } @Override @@ -108,7 +114,8 @@ public SplitEnumerator createEnumerat serializedElements.keySet().stream() .map(ValuesSourcePartitionSplit::new) .collect(Collectors.toList()); - return new DynamicFilteringValuesSourceEnumerator(context, splits, dynamicFilteringFields); + return new DynamicFilteringValuesSourceEnumerator( + context, terminatingLogic, splits, dynamicFilteringFields); } @Override diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSourceReader.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSourceReader.java index 71e0dd05ee462..29ffa8a70a0c5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSourceReader.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSourceReader.java @@ -64,6 +64,7 @@ public class DynamicFilteringValuesSourceReader private transient ValuesSourcePartitionSplit currentSplit; private transient Iterator iterator; private transient boolean noMoreSplits; + private transient boolean reachedInfiniteEnd; public DynamicFilteringValuesSourceReader( Map, byte[]> serializedElements, @@ -88,6 +89,10 @@ public void start() { @Override public InputStatus pollNext(ReaderOutput output) { + if (reachedInfiniteEnd) { + return InputStatus.NOTHING_AVAILABLE; + } + if (iterator != null) { if (iterator.hasNext()) { output.collect(iterator.next()); @@ -115,23 +120,32 @@ private void finishSplit() { private InputStatus tryMoveToNextSplit() { currentSplit = remainingSplits.poll(); if (currentSplit != null) { - Map partition = currentSplit.getPartition(); - List list = - deserialize(serializedElements.get(partition), counts.get(partition)); - iterator = list.iterator(); - return InputStatus.MORE_AVAILABLE; + if (currentSplit.isInfinite()) { + this.reachedInfiniteEnd = true; + resetAvailability(); + return InputStatus.NOTHING_AVAILABLE; + } else { + Map partition = currentSplit.getPartition(); + List list = + deserialize(serializedElements.get(partition), counts.get(partition)); + iterator = list.iterator(); + return InputStatus.MORE_AVAILABLE; + } } else if (noMoreSplits) { return InputStatus.END_OF_INPUT; } else { - // ensure we are not called in a loop by resetting the availability future - if (availability.isDone()) { - availability = new CompletableFuture<>(); - } - + resetAvailability(); return InputStatus.NOTHING_AVAILABLE; } } + private void resetAvailability() { + // ensure we are not called in a loop by resetting the availability future + if (availability.isDone()) { + availability = new CompletableFuture<>(); + } + } + private List deserialize(byte[] data, int count) { List list = new ArrayList<>(); try (ByteArrayInputStream bais = new ByteArrayInputStream(data)) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/TerminatingLogic.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/TerminatingLogic.java new file mode 100644 index 0000000000000..124a2bdff582e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/TerminatingLogic.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.flink.connector.source; + +import org.apache.flink.table.planner.factories.TestValuesTableFactory; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** + * Tells sources created from {@link TestValuesTableFactory} how they should behave after they + * produced all data. It is separate from 'bounded', because even if a source is unbounded it can + * stop producing records and shutdown. + */ +public enum TerminatingLogic { + INFINITE, + FINITE; + + public static TerminatingLogic readFrom(DataInputStream in) throws IOException { + final boolean isInfinite = in.readBoolean(); + return isInfinite ? TerminatingLogic.INFINITE : TerminatingLogic.FINITE; + } + + public static void writeTo(DataOutputStream out, TerminatingLogic terminatingLogic) + throws IOException { + out.writeBoolean(terminatingLogic == TerminatingLogic.INFINITE); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java index 6d9a3e186e256..ad5c3b26b024b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java @@ -57,10 +57,19 @@ public class ValuesSource implements Source serializedElements; - public ValuesSource(Collection elements, TypeSerializer serializer) { + private final TerminatingLogic terminatingLogic; + private final Boundedness boundedness; + + public ValuesSource( + TerminatingLogic terminatingLogic, + Boundedness boundedness, + Collection elements, + TypeSerializer serializer) { Preconditions.checkState(serializer != null, "serializer not set"); this.serializedElements = serializeElements(elements, serializer); this.serializer = serializer; + this.terminatingLogic = terminatingLogic; + this.boundedness = boundedness; } private List serializeElements( @@ -82,7 +91,7 @@ private List serializeElements( @Override public Boundedness getBoundedness() { - return Boundedness.BOUNDED; + return boundedness; } @Override @@ -98,7 +107,7 @@ public SplitEnumerator createEnumerator( IntStream.range(0, serializedElements.size()) .mapToObj(ValuesSourceSplit::new) .collect(Collectors.toList()); - return new ValuesSourceEnumerator(enumContext, splits); + return new ValuesSourceEnumerator(enumContext, splits, terminatingLogic); } @Override diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSourceReader.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSourceReader.java index 84e33e529dadf..dcaca67fd1ea6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSourceReader.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSourceReader.java @@ -97,22 +97,32 @@ public void start() { public InputStatus pollNext(ReaderOutput output) throws Exception { ValuesSourceSplit currentSplit = remainingSplits.poll(); if (currentSplit != null) { - output.collect(elements.get(currentSplit.getIndex())); - // request another split - context.sendSplitRequest(); - return InputStatus.MORE_AVAILABLE; + if (currentSplit.isInfinite()) { + remainingSplits.add(currentSplit); + resetAvailability(); + return InputStatus.NOTHING_AVAILABLE; + } else { + output.collect(elements.get(currentSplit.getIndex())); + // request another split + context.sendSplitRequest(); + return InputStatus.MORE_AVAILABLE; + } } else if (noMoreSplits) { return InputStatus.END_OF_INPUT; } else { - // ensure we are not called in a loop by resetting the availability future - if (availability.isDone()) { - availability = new CompletableFuture<>(); - } + resetAvailability(); return InputStatus.NOTHING_AVAILABLE; } } + private void resetAvailability() { + // ensure we are not called in a loop by resetting the availability future + if (availability.isDone()) { + availability = new CompletableFuture<>(); + } + } + @Override public List snapshotState(long checkpointId) { return Collections.emptyList(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/DynamicFilteringValuesSourceEnumerator.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/DynamicFilteringValuesSourceEnumerator.java index 83b09b0ffcc91..793f46fa19590 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/DynamicFilteringValuesSourceEnumerator.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/DynamicFilteringValuesSourceEnumerator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.connector.source.split.ValuesSourcePartitionSplit; import org.apache.flink.table.connector.source.DynamicFilteringData; import org.apache.flink.table.connector.source.DynamicFilteringEvent; @@ -35,6 +36,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -47,15 +49,18 @@ public class DynamicFilteringValuesSourceEnumerator private final SplitEnumeratorContext context; private final List allSplits; private final List dynamicFilteringFields; + private final TerminatingLogic terminatingLogic; private transient boolean receivedDynamicFilteringEvent; private transient List remainingSplits; public DynamicFilteringValuesSourceEnumerator( SplitEnumeratorContext context, + TerminatingLogic terminatingLogic, List allSplits, List dynamicFilteringFields) { this.context = context; this.allSplits = allSplits; + this.terminatingLogic = terminatingLogic; this.dynamicFilteringFields = dynamicFilteringFields; } @@ -68,8 +73,15 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname throw new IllegalStateException("DynamicFilteringEvent has not receive"); } if (remainingSplits.isEmpty()) { - context.signalNoMoreSplits(subtaskId); - LOG.info("No more splits available for subtask {}", subtaskId); + if (terminatingLogic == TerminatingLogic.INFINITE) { + context.assignSplit( + new ValuesSourcePartitionSplit( + Collections.emptyMap(), TerminatingLogic.INFINITE), + subtaskId); + } else { + context.signalNoMoreSplits(subtaskId); + LOG.info("No more splits available for subtask {}", subtaskId); + } } else { ValuesSourcePartitionSplit split = remainingSplits.remove(0); LOG.debug("Assigned split to subtask {} : {}", subtaskId, split); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/ValuesSourceEnumerator.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/ValuesSourceEnumerator.java index 4e6f2d287c85f..8d60627198ecf 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/ValuesSourceEnumerator.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/enumerator/ValuesSourceEnumerator.java @@ -20,6 +20,7 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.connector.source.ValuesSource; import org.apache.flink.connector.source.split.ValuesSourceSplit; @@ -38,12 +39,15 @@ public class ValuesSourceEnumerator implements SplitEnumerator context; private final Queue remainingSplits; + private final TerminatingLogic terminatingLogic; public ValuesSourceEnumerator( SplitEnumeratorContext context, - List remainingSplits) { + List remainingSplits, + TerminatingLogic terminatingLogic) { this.context = context; this.remainingSplits = new ArrayDeque<>(remainingSplits); + this.terminatingLogic = terminatingLogic; } @Override @@ -54,6 +58,8 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname final ValuesSourceSplit nextSplit = remainingSplits.poll(); if (nextSplit != null) { context.assignSplit(nextSplit, subtaskId); + } else if (terminatingLogic == TerminatingLogic.INFINITE) { + context.assignSplit(new ValuesSourceSplit(-1, TerminatingLogic.INFINITE), subtaskId); } else { context.signalNoMoreSplits(subtaskId); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplit.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplit.java index a8eab5d624134..f580a92cb3d31 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplit.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplit.java @@ -20,6 +20,7 @@ import org.apache.flink.api.connector.source.SourceSplit; import org.apache.flink.connector.source.DynamicFilteringValuesSource; +import org.apache.flink.connector.source.TerminatingLogic; import java.util.Map; @@ -27,15 +28,30 @@ public class ValuesSourcePartitionSplit implements SourceSplit { private final Map partition; + private final TerminatingLogic terminatingLogic; public ValuesSourcePartitionSplit(Map partition) { + this(partition, TerminatingLogic.FINITE); + } + + public ValuesSourcePartitionSplit( + Map partition, TerminatingLogic terminatingLogic) { this.partition = partition; + this.terminatingLogic = terminatingLogic; } public Map getPartition() { return partition; } + public TerminatingLogic getTerminatingLogic() { + return terminatingLogic; + } + + public boolean isInfinite() { + return terminatingLogic == TerminatingLogic.INFINITE; + } + @Override public String splitId() { return partition.toString(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplitSerializer.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplitSerializer.java index 416c5268e889a..e79d10a503ee6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplitSerializer.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourcePartitionSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.source.split; import org.apache.flink.connector.source.DynamicFilteringValuesSource; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.util.Preconditions; @@ -50,6 +51,7 @@ public byte[] serialize(ValuesSourcePartitionSplit split) throws IOException { out.writeUTF(entry.getKey()); out.writeUTF(entry.getValue()); } + TerminatingLogic.writeTo(out, split.getTerminatingLogic()); out.flush(); return baos.toByteArray(); } @@ -68,7 +70,9 @@ public ValuesSourcePartitionSplit deserialize(int version, byte[] serialized) String value = in.readUTF(); partition.put(key, value); } - ValuesSourcePartitionSplit split = new ValuesSourcePartitionSplit(partition); + final TerminatingLogic terminatingLogic = TerminatingLogic.readFrom(in); + ValuesSourcePartitionSplit split = + new ValuesSourcePartitionSplit(partition, terminatingLogic); Preconditions.checkArgument(split.splitId().equals(splitId)); return split; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplit.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplit.java index 87d4b9bce9e1b..31f6b431e4d6a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplit.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplit.java @@ -19,15 +19,30 @@ package org.apache.flink.connector.source.split; import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.connector.source.ValuesSource; /** The split of the {@link ValuesSource}. */ public class ValuesSourceSplit implements SourceSplit { private final int index; + private final TerminatingLogic terminatingLogic; public ValuesSourceSplit(int index) { + this(index, TerminatingLogic.FINITE); + } + + public ValuesSourceSplit(int index, TerminatingLogic terminatingLogic) { this.index = index; + this.terminatingLogic = terminatingLogic; + } + + public TerminatingLogic getTerminatingLogic() { + return terminatingLogic; + } + + public boolean isInfinite() { + return terminatingLogic == TerminatingLogic.INFINITE; } public int getIndex() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplitSerializer.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplitSerializer.java index 697162e18b49a..a52f4595497c9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplitSerializer.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/split/ValuesSourceSplitSerializer.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.source.split; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.connector.source.ValuesSource; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -39,6 +40,7 @@ public byte[] serialize(ValuesSourceSplit split) throws IOException { try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream out = new DataOutputStream(baos)) { out.writeInt(split.getIndex()); + TerminatingLogic.writeTo(out, split.getTerminatingLogic()); out.flush(); return baos.toByteArray(); } @@ -49,7 +51,8 @@ public ValuesSourceSplit deserialize(int version, byte[] serialized) throws IOEx try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); DataInputStream in = new DataInputStream(bais)) { int index = in.readInt(); - return new ValuesSourceSplit(index); + final TerminatingLogic terminatingLogic = TerminatingLogic.readFrom(in); + return new ValuesSourceSplit(index, terminatingLogic); } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index c4da3ec230158..497629e26f71f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -24,11 +24,13 @@ import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.source.DynamicFilteringValuesSource; +import org.apache.flink.connector.source.TerminatingLogic; import org.apache.flink.connector.source.ValuesSource; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -313,6 +315,15 @@ private static RowKind parseRowKind(String rowKindShortString) { private static final ConfigOption BOUNDED = ConfigOptions.key("bounded").booleanType().defaultValue(false); + private static final ConfigOption TERMINATING = + ConfigOptions.key("terminating") + .booleanType() + .defaultValue(true) + .withDescription( + "Declares the behaviour of sources after all data has been" + + " produced. It is separate from 'bounded', because even if a source" + + " is unbounded it can stop producing records and shutdown."); + private static final ConfigOption CHANGELOG_MODE = ConfigOptions.key("changelog-mode") .stringType() @@ -457,6 +468,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { ChangelogMode changelogMode = parseChangelogMode(helper.getOptions().get(CHANGELOG_MODE)); String runtimeSource = helper.getOptions().get(RUNTIME_SOURCE); boolean isBounded = helper.getOptions().get(BOUNDED); + boolean isFinite = helper.getOptions().get(TERMINATING); String dataId = helper.getOptions().get(DATA_ID); String sourceClass = helper.getOptions().get(TABLE_SOURCE_CLASS); boolean isAsync = helper.getOptions().get(ASYNC_ENABLED); @@ -495,6 +507,16 @@ public DynamicTableSource createDynamicTableSource(Context context) { convertToMetadataMap( helper.getOptions().get(READABLE_METADATA), context.getClassLoader()); + if (!isFinite && isBounded) { + throw new IllegalArgumentException( + "Source can not be bounded and infinite at the same time."); + } + + TerminatingLogic terminating = + isFinite ? TerminatingLogic.FINITE : TerminatingLogic.INFINITE; + Boundedness boundedness = + isBounded ? Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED; + if (sourceClass.equals("DEFAULT")) { if (internalData) { return new TestValuesScanTableSourceWithInternalData( @@ -521,7 +543,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new TestValuesScanTableSourceWithoutProjectionPushDown( producedDataType, changelogMode, - isBounded, + boundedness, + terminating, runtimeSource, failingSource, partition2Rows, @@ -542,6 +565,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new TestValuesScanTableSourceWithWatermarkPushDown( producedDataType, changelogMode, + terminating, runtimeSource, failingSource, partition2Rows, @@ -560,7 +584,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new TestValuesScanTableSource( producedDataType, changelogMode, - isBounded, + boundedness, + terminating, runtimeSource, failingSource, partition2Rows, @@ -580,7 +605,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { context.getCatalogTable().getResolvedSchema().toPhysicalRowDataType(), producedDataType, changelogMode, - isBounded, + boundedness, + terminating, runtimeSource, failingSource, partition2Rows, @@ -679,6 +705,7 @@ public Set> optionalOptions() { DATA_ID, CHANGELOG_MODE, BOUNDED, + TERMINATING, RUNTIME_SOURCE, TABLE_SOURCE_CLASS, FAILING_SOURCE, @@ -855,7 +882,8 @@ private static class TestValuesScanTableSourceWithoutProjectionPushDown protected DataType producedDataType; protected final ChangelogMode changelogMode; - protected final boolean bounded; + protected final Boundedness boundedness; + protected final TerminatingLogic terminating; protected final String runtimeSource; protected final boolean failingSource; protected Map, Collection> data; @@ -878,7 +906,8 @@ private static class TestValuesScanTableSourceWithoutProjectionPushDown private TestValuesScanTableSourceWithoutProjectionPushDown( DataType producedDataType, ChangelogMode changelogMode, - boolean bounded, + Boundedness boundedness, + TerminatingLogic terminating, String runtimeSource, boolean failingSource, Map, Collection> data, @@ -894,7 +923,8 @@ private TestValuesScanTableSourceWithoutProjectionPushDown( @Nullable int[] projectedMetadataFields) { this.producedDataType = producedDataType; this.changelogMode = changelogMode; - this.bounded = bounded; + this.boundedness = boundedness; + this.terminating = terminating; this.runtimeSource = runtimeSource; this.failingSource = failingSource; this.data = data; @@ -930,6 +960,9 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon switch (runtimeSource) { case "SourceFunction": try { + checkArgument( + terminating != TerminatingLogic.FINITE, + "Values Source doesn't support infinite SourceFunction."); Collection values = convertToRowData(converter); final SourceFunction sourceFunction; if (failingSource) { @@ -939,7 +972,8 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon } else { sourceFunction = new FromElementsFunction<>(serializer, values); } - return SourceFunctionProvider.of(sourceFunction, bounded); + return SourceFunctionProvider.of( + sourceFunction, boundedness == Boundedness.BOUNDED); } catch (IOException e) { throw new TableException("Fail to init source function", e); } @@ -947,12 +981,18 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon checkArgument( !failingSource, "Values InputFormat Source doesn't support as failing source."); + checkArgument( + terminating != TerminatingLogic.FINITE, + "Values Source doesn't support infinite InputFormat."); Collection values = convertToRowData(converter); return InputFormatProvider.of(new CollectionInputFormat<>(values, serializer)); case "DataStream": checkArgument( !failingSource, "Values DataStream Source doesn't support as failing source."); + checkArgument( + terminating != TerminatingLogic.FINITE, + "Values Source doesn't support infinite DataStream."); try { Collection values2 = convertToRowData(converter); FromElementsFunction function = @@ -972,7 +1012,7 @@ public DataStream produceDataStream( @Override public boolean isBounded() { - return bounded; + return boundedness == Boundedness.BOUNDED; } }; } catch (IOException e) { @@ -984,13 +1024,18 @@ public boolean isBounded() { if (acceptedPartitionFilterFields == null || acceptedPartitionFilterFields.isEmpty()) { Collection values2 = convertToRowData(converter); - return SourceProvider.of(new ValuesSource(values2, serializer)); + return SourceProvider.of( + new ValuesSource(terminating, boundedness, values2, serializer)); } else { Map, Collection> partitionValues = convertToPartitionedRowData(converter); DynamicFilteringValuesSource source = new DynamicFilteringValuesSource( - partitionValues, serializer, acceptedPartitionFilterFields); + terminating, + boundedness, + partitionValues, + serializer, + acceptedPartitionFilterFields); return SourceProvider.of(source); } default: @@ -1041,7 +1086,8 @@ public DynamicTableSource copy() { return new TestValuesScanTableSourceWithoutProjectionPushDown( producedDataType, changelogMode, - bounded, + boundedness, + terminating, runtimeSource, failingSource, data, @@ -1385,7 +1431,8 @@ private static class TestValuesScanTableSource private TestValuesScanTableSource( DataType producedDataType, ChangelogMode changelogMode, - boolean bounded, + Boundedness boundedness, + TerminatingLogic terminating, String runtimeSource, boolean failingSource, Map, Collection> data, @@ -1402,7 +1449,8 @@ private TestValuesScanTableSource( super( producedDataType, changelogMode, - bounded, + boundedness, + terminating, runtimeSource, failingSource, data, @@ -1423,7 +1471,8 @@ public DynamicTableSource copy() { return new TestValuesScanTableSource( producedDataType, changelogMode, - bounded, + boundedness, + terminating, runtimeSource, failingSource, data, @@ -1464,6 +1513,7 @@ private static class TestValuesScanTableSourceWithWatermarkPushDown private TestValuesScanTableSourceWithWatermarkPushDown( DataType producedDataType, ChangelogMode changelogMode, + TerminatingLogic terminating, String runtimeSource, boolean failingSource, Map, Collection> data, @@ -1481,7 +1531,8 @@ private TestValuesScanTableSourceWithWatermarkPushDown( super( producedDataType, changelogMode, - false, + Boundedness.CONTINUOUS_UNBOUNDED, + terminating, runtimeSource, failingSource, data, @@ -1534,6 +1585,7 @@ public DynamicTableSource copy() { new TestValuesScanTableSourceWithWatermarkPushDown( producedDataType, changelogMode, + terminating, runtimeSource, failingSource, data, @@ -1574,7 +1626,8 @@ private TestValuesScanLookupTableSource( DataType originType, DataType producedDataType, ChangelogMode changelogMode, - boolean bounded, + Boundedness boundedness, + TerminatingLogic terminating, String runtimeSource, boolean failingSource, Map, Collection> data, @@ -1596,7 +1649,8 @@ private TestValuesScanLookupTableSource( super( producedDataType, changelogMode, - bounded, + boundedness, + terminating, runtimeSource, failingSource, data, @@ -1780,7 +1834,8 @@ public DynamicTableSource copy() { originType, producedDataType, changelogMode, - bounded, + boundedness, + terminating, runtimeSource, failingSource, data, From a5b4e60e563bf145afede43dda8510833d2932e4 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 24 Oct 2023 13:41:58 +0200 Subject: [PATCH 041/104] [FLINK-33375] Improve TestValues utilities for restore tests --- .../source/DynamicFilteringValuesSource.java | 5 +-- .../flink/connector/source/ValuesSource.java | 2 +- .../factories/TestValuesRuntimeFunctions.java | 34 +++++++++++++++++-- .../factories/TestValuesTableFactory.java | 13 +++++++ 4 files changed, 48 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java index 4a011476ecbe4..303bf92f546a5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/DynamicFilteringValuesSource.java @@ -120,8 +120,9 @@ public SplitEnumerator createEnumerat @Override public SplitEnumerator restoreEnumerator( - SplitEnumeratorContext context, NoOpEnumState checkpoint) { - throw new UnsupportedOperationException("Unsupported now."); + SplitEnumeratorContext context, NoOpEnumState checkpoint) + throws Exception { + return createEnumerator(context); } @Override diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java index ad5c3b26b024b..e4b24d82e7cfe 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/source/ValuesSource.java @@ -114,7 +114,7 @@ public SplitEnumerator createEnumerator( public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, NoOpEnumState checkpoint) throws Exception { - throw new UnsupportedOperationException("Unsupported now."); + return createEnumerator(enumContext); } @Override diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java index f582531f7bc53..fa79a9fed1f39 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java @@ -71,6 +71,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import static org.apache.flink.table.planner.factories.TestValuesTableFactory.RESOURCE_COUNTER; @@ -93,6 +94,10 @@ final class TestValuesRuntimeFunctions { // [table_name, [watermark]] private static final Map> watermarkHistory = new HashMap<>(); + // [table_name, [List[observer]] + private static final Map>>> + localRawResultsObservers = new HashMap<>(); + static List getRawResultsAsStrings(String tableName) { return getRawResults(tableName).stream() .map(TestValuesRuntimeFunctions::rowToString) @@ -162,6 +167,11 @@ static List getResults(String tableName) { return Collections.emptyList(); } + static void registerLocalRawResultsObserver( + String tableName, BiConsumer> observer) { + localRawResultsObservers.computeIfAbsent(tableName, n -> new ArrayList<>()).add(observer); + } + static void clearResults() { synchronized (LOCK) { globalRawResult.clear(); @@ -347,6 +357,17 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { rawResultState.update(localRawResult); } } + + protected void addLocalRawResult(Row row) { + localRawResult.add(row); + Optional.ofNullable(localRawResultsObservers.get(tableName)) + .orElse(Collections.emptyList()) + .forEach( + c -> + c.accept( + getRuntimeContext().getIndexOfThisSubtask(), + localRawResult)); + } } static class AppendingSinkFunction extends AbstractExactlyOnceSink { @@ -376,7 +397,7 @@ public void invoke(RowData value, Context context) throws Exception { } } synchronized (LOCK) { - localRawResult.add((Row) converter.toExternal(value)); + addLocalRawResult((Row) converter.toExternal(value)); } } else { throw new RuntimeException( @@ -436,7 +457,7 @@ public void invoke(RowData value, Context context) throws Exception { assertThat(row).isNotNull(); synchronized (LOCK) { - localRawResult.add(row); + addLocalRawResult(row); Row key = Row.project(row, keyIndices); key.setKind(RowKind.INSERT); @@ -538,7 +559,7 @@ public void invoke(RowData value, Context context) throws Exception { Row row = (Row) converter.toExternal(value); assertThat(row).isNotNull(); synchronized (LOCK) { - localRawResult.add(row); + addLocalRawResult(row); final Row retractRow = Row.copy(row); retractRow.setKind(RowKind.INSERT); if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) { @@ -590,6 +611,13 @@ public void writeRecord(RowData value) throws IOException { assertThat(row).isNotNull(); synchronized (LOCK) { localRawResult.add(row); + Optional.ofNullable(localRawResultsObservers.get(tableName)) + .orElse(Collections.emptyList()) + .forEach( + c -> + c.accept( + getRuntimeContext().getIndexOfThisSubtask(), + localRawResult)); } } else { throw new RuntimeException( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index 497629e26f71f..c0cdbc60c8840 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -148,6 +148,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.Function; import java.util.stream.Collectors; @@ -267,6 +268,18 @@ public static List getResults(String tableName) { return TestValuesRuntimeFunctions.getResults(tableName); } + /** + * Registers an observer for a table that gets notified of each incoming raw data for every + * subtask. It gets all rows seen so far, by a given task. + * + * @param tableName the table name of the registered table sink. + * @param observer the observer to be notified + */ + public static void registerLocalRawResultsObserver( + String tableName, BiConsumer> observer) { + TestValuesRuntimeFunctions.registerLocalRawResultsObserver(tableName, observer); + } + public static List getWatermarkOutput(String tableName) { return TestValuesRuntimeFunctions.getWatermarks(tableName); } From aa7519b91b89b626c92950f39d29d7455addd104 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 27 Oct 2023 17:06:41 +0200 Subject: [PATCH 042/104] [hotfix] Adapt the SinkTestStep to give plain access to expected results --- .../table/test/program/SinkTestStep.java | 52 ++++++++++++++++--- .../table/test/program/TableTestProgram.java | 43 +++++---------- 2 files changed, 57 insertions(+), 38 deletions(-) diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java index 42bfbb9da87cf..d195d65c1440b 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java @@ -24,31 +24,69 @@ import java.util.List; import java.util.Map; -import java.util.function.Predicate; +import java.util.stream.Collectors; /** Test step for creating a table sink. */ public final class SinkTestStep extends TableTestStep { - public final @Nullable Predicate> expectedBeforeRestore; - public final @Nullable Predicate> expectedAfterRestore; + public final @Nullable List expectedBeforeRestore; + public final @Nullable List expectedAfterRestore; + public final @Nullable List expectedBeforeRestoreStrings; + public final @Nullable List expectedAfterRestoreStrings; SinkTestStep( String name, List schemaComponents, List partitionKeys, Map options, - @Nullable Predicate> expectedBeforeRestore, - @Nullable Predicate> expectedAfterRestore) { + @Nullable List expectedBeforeRestore, + @Nullable List expectedAfterRestore, + @Nullable List expectedBeforeRestoreStrings, + @Nullable List expectedAfterRestoreStrings) { super(name, schemaComponents, partitionKeys, options); + if (expectedBeforeRestore != null && expectedAfterRestoreStrings != null) { + throw new IllegalArgumentException( + "You can not mix Row/String representation in before/after restore data."); + } + if (expectedBeforeRestoreStrings != null && expectedAfterRestore != null) { + throw new IllegalArgumentException( + "You can not mix Row/String representation in before/after restore data."); + } this.expectedBeforeRestore = expectedBeforeRestore; this.expectedAfterRestore = expectedAfterRestore; + this.expectedBeforeRestoreStrings = expectedBeforeRestoreStrings; + this.expectedAfterRestoreStrings = expectedAfterRestoreStrings; + } + + public List getExpectedBeforeRestoreAsStrings() { + if (expectedBeforeRestoreStrings != null) { + return expectedBeforeRestoreStrings; + } + + if (expectedBeforeRestore != null) { + return expectedBeforeRestore.stream().map(Row::toString).collect(Collectors.toList()); + } + + return null; + } + + public List getExpectedAfterRestoreAsStrings() { + if (expectedAfterRestoreStrings != null) { + return expectedAfterRestoreStrings; + } + + if (expectedAfterRestore != null) { + return expectedAfterRestore.stream().map(Row::toString).collect(Collectors.toList()); + } + + return null; } @Override public TestKind getKind() { - return expectedBeforeRestore == null + return expectedBeforeRestore == null && expectedBeforeRestoreStrings == null ? TestKind.SINK_WITHOUT_DATA - : expectedAfterRestore == null + : expectedAfterRestore == null && expectedAfterRestoreStrings == null ? TestKind.SINK_WITH_DATA : TestKind.SINK_WITH_RESTORE_DATA; } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java index 1b194b39ddd28..88d53dff46185 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -28,15 +28,12 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; -import org.apache.commons.collections.CollectionUtils; - import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -378,8 +375,11 @@ public Builder complete() { /** Builder pattern for {@link SinkTestStep}. */ public static class SinkBuilder extends TableBuilder { - private Predicate> expectedBeforeRestore; - private Predicate> expectedAfterRestore; + private List expectedBeforeRestore; + private List expectedAfterRestore; + + private List expectedBeforeRestoreStrings; + private List expectedAfterRestoreStrings; private SinkBuilder(String name, List targetSteps, Builder rootBuilder) { super(name, targetSteps, rootBuilder); @@ -394,46 +394,25 @@ public SinkBuilder withExpectedValues(String... expectedRows) { } public SinkBuilder withValuesBeforeRestore(Row... expectedRows) { - this.expectedBeforeRestore = equalIgnoringOrder(expectedRows); + this.expectedBeforeRestore = Arrays.asList(expectedRows); return this; } public SinkBuilder withValuesBeforeRestore(String... expectedRows) { - this.expectedBeforeRestore = equalIgnoringOrder(expectedRows); + this.expectedBeforeRestoreStrings = Arrays.asList(expectedRows); return this; } public SinkBuilder withValuesAfterRestore(Row... expectedRows) { - this.expectedAfterRestore = equalIgnoringOrder(expectedRows); + this.expectedAfterRestore = Arrays.asList(expectedRows); return this; } public SinkBuilder withValuesAfterRestore(String... expectedRows) { - this.expectedAfterRestore = equalIgnoringOrder(expectedRows); + this.expectedAfterRestoreStrings = Arrays.asList(expectedRows); return this; } - private static Predicate> equalIgnoringOrder(Row... expectedRows) { - return (actualRows) -> { - if (actualRows.size() != expectedRows.length) { - return false; - } - return CollectionUtils.isEqualCollection(actualRows, Arrays.asList(expectedRows)); - }; - } - - private static Predicate> equalIgnoringOrder(String... expectedRows) { - return (actualRows) -> { - if (actualRows.size() != expectedRows.length) { - return false; - } - final List actualRowsString = - actualRows.stream().map(Row::toString).collect(Collectors.toList()); - return CollectionUtils.isEqualCollection( - actualRowsString, Arrays.asList(expectedRows)); - }; - } - public Builder complete() { targetSteps.add( new SinkTestStep( @@ -442,7 +421,9 @@ public Builder complete() { partitionKeys, options, expectedBeforeRestore, - expectedAfterRestore)); + expectedAfterRestore, + expectedBeforeRestoreStrings, + expectedAfterRestoreStrings)); return rootBuilder; } } From d863ff38c0671255df2452c79dad88fa47e2bc0c Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 20 Oct 2023 14:47:43 +0200 Subject: [PATCH 043/104] [FLINK-33375] Implement restore test base This closes #23603 --- .../table/test/program/TableTestProgram.java | 1 + .../plan/utils/ExecNodeMetadataUtil.java | 2 +- .../factories/TestValuesTableFactory.java | 6 +- .../nodes/exec/testutils/CalcRestoreTest.java | 38 +++ .../exec/testutils/CalcTestPrograms.java | 42 +++ .../nodes/exec/testutils/RestoreTestBase.java | 245 ++++++++++++++++++ .../calc-simple/plan/calc-simple.json | 113 ++++++++ .../calc-simple/savepoint/_metadata | Bin 0 -> 7032 bytes pom.xml | 1 + 9 files changed, 444 insertions(+), 4 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/plan/calc-simple.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/savepoint/_metadata diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java index 88d53dff46185..efec060315d2f 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -114,6 +114,7 @@ private TableTestProgram( * *

The identifier is necessary to (ideally globally) identify the test program in outputs. * For example, a runner for plan tests can create directories and use the name as file names. + * The identifier must start with the name of the exec node under testing. * *

The description should give more context and should start with a verb and "s" suffix. * diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java index 5c467f29583cb..b478f637fc919 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java @@ -197,7 +197,7 @@ public static void addTestNode(Class> execNodeClass) { addToLookupMap(execNodeClass); } - private static > List extractMetadataFromAnnotation( + public static > List extractMetadataFromAnnotation( Class execNodeClass) { List metadata = new ArrayList<>(); ExecNodeMetadata annotation = execNodeClass.getDeclaredAnnotation(ExecNodeMetadata.class); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index c0cdbc60c8840..8511e30ce3cb6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -974,7 +974,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon case "SourceFunction": try { checkArgument( - terminating != TerminatingLogic.FINITE, + terminating == TerminatingLogic.FINITE, "Values Source doesn't support infinite SourceFunction."); Collection values = convertToRowData(converter); final SourceFunction sourceFunction; @@ -995,7 +995,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon !failingSource, "Values InputFormat Source doesn't support as failing source."); checkArgument( - terminating != TerminatingLogic.FINITE, + terminating == TerminatingLogic.FINITE, "Values Source doesn't support infinite InputFormat."); Collection values = convertToRowData(converter); return InputFormatProvider.of(new CollectionInputFormat<>(values, serializer)); @@ -1004,7 +1004,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon !failingSource, "Values DataStream Source doesn't support as failing source."); checkArgument( - terminating != TerminatingLogic.FINITE, + terminating == TerminatingLogic.FINITE, "Values Source doesn't support infinite DataStream."); try { Collection values2 = convertToRowData(converter); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java new file mode 100644 index 0000000000000..117316aa8e1e6 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.testutils; + +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Collections; +import java.util.List; + +/** Restore tests for {@link StreamExecCalc}. */ +public class CalcRestoreTest extends RestoreTestBase { + + public CalcRestoreTest() { + super(StreamExecCalc.class); + } + + @Override + public List programs() { + return Collections.singletonList(CalcTestPrograms.SIMPLE_CALC); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java new file mode 100644 index 0000000000000..534d50a26bb1f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.testutils; + +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecCalc}. */ +public class CalcTestPrograms { + + static final TableTestProgram SIMPLE_CALC = + TableTestProgram.of("calc-simple", "validates basic calc node") + .setupTableSource("t") + .withSchema("a BIGINT", "b DOUBLE") + .withValuesBeforeRestore(Row.of(420L, 42.0)) + .withValuesAfterRestore(Row.of(421L, 42.1)) + .complete() + .setupTableSink("sink_t") + .withSchema("a BIGINT", "b DOUBLE") + .withValuesBeforeRestore(Row.of(421L, 42.0)) + .withValuesAfterRestore(Row.of(422L, 42.1)) + .complete() + .runSql("INSERT INTO sink_t SELECT a + 1, b FROM t") + .build(); +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java new file mode 100644 index 0000000000000..23e0926140cc4 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.testutils; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.RestoreMode; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.PlanReference; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; +import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.SqlTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.table.test.program.TableTestProgramRunner; +import org.apache.flink.table.test.program.TestStep.TestKind; +import org.apache.flink.test.junit5.MiniClusterExtension; + +import org.apache.commons.collections.CollectionUtils; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Base class for implementing restore tests for {@link ExecNode}.You can generate json compiled + * plan and a savepoint for the latest node version by running {@link + * RestoreTestBase#generateTestSetupFiles(TableTestProgram)} which is disabled by default. + * + *

Note: The test base uses {@link TableConfigOptions.CatalogPlanCompilation#SCHEMA} + * because it needs to adjust source and sink properties before and after the restore. Therefore, + * the test base can not be used for testing storing table options in the compiled plan. + */ +@ExtendWith(MiniClusterExtension.class) +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class RestoreTestBase implements TableTestProgramRunner { + + private final Class execNodeUnderTest; + + protected RestoreTestBase(Class execNodeUnderTest) { + this.execNodeUnderTest = execNodeUnderTest; + } + + @Override + public EnumSet supportedSetupSteps() { + return EnumSet.of(TestKind.SOURCE_WITH_RESTORE_DATA, TestKind.SINK_WITH_RESTORE_DATA); + } + + @Override + public EnumSet supportedRunSteps() { + return EnumSet.of(TestKind.SQL); + } + + private @TempDir Path tmpDir; + + private List getAllMetadata() { + return ExecNodeMetadataUtil.extractMetadataFromAnnotation(execNodeUnderTest); + } + + private ExecNodeMetadata getLatestMetadata() { + return ExecNodeMetadataUtil.latestAnnotation(execNodeUnderTest); + } + + private Stream createSpecs() { + return getAllMetadata().stream() + .flatMap( + metadata -> + supportedPrograms().stream().map(p -> Arguments.of(metadata, p))); + } + + /** + * Execute this test to generate test files. Remember to be using the correct branch when + * generating the test files. + */ + @Disabled + @ParameterizedTest + @MethodSource("supportedPrograms") + public void generateTestSetupFiles(TableTestProgram program) throws Exception { + final TableEnvironment tEnv = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + tEnv.getConfig() + .set( + TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, + TableConfigOptions.CatalogPlanCompilation.SCHEMA); + for (SourceTestStep sourceTestStep : program.getSetupSourceTestSteps()) { + final String id = TestValuesTableFactory.registerData(sourceTestStep.dataBeforeRestore); + final Map options = new HashMap<>(); + options.put("connector", "values"); + options.put("data-id", id); + options.put("terminating", "false"); + options.put("disable-lookup", "true"); + options.put("runtime-source", "NewSource"); + sourceTestStep.apply(tEnv, options); + } + + final List> futures = new ArrayList<>(); + for (SinkTestStep sinkTestStep : program.getSetupSinkTestSteps()) { + final CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + final String tableName = sinkTestStep.name; + TestValuesTableFactory.registerLocalRawResultsObserver( + tableName, + (integer, strings) -> { + final boolean shouldTakeSavepoint = + CollectionUtils.isEqualCollection( + TestValuesTableFactory.getRawResultsAsStrings(tableName), + sinkTestStep.getExpectedBeforeRestoreAsStrings()); + if (shouldTakeSavepoint) { + future.complete(null); + } + }); + final Map options = new HashMap<>(); + options.put("connector", "values"); + options.put("disable-lookup", "true"); + options.put("sink-insert-only", "false"); + sinkTestStep.apply(tEnv, options); + } + + final SqlTestStep sqlTestStep = program.getRunSqlTestStep(); + + final CompiledPlan compiledPlan = tEnv.compilePlanSql(sqlTestStep.sql); + compiledPlan.writeToFile(getPlanPath(program, getLatestMetadata())); + + final TableResult tableResult = compiledPlan.execute(); + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + final JobClient jobClient = tableResult.getJobClient().get(); + final String savepoint = + jobClient + .stopWithSavepoint(false, tmpDir.toString(), SavepointFormatType.DEFAULT) + .get(); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); + final Path savepointPath = Paths.get(new URI(savepoint)); + final Path savepointDirPath = getSavepointPath(program, getLatestMetadata()); + Files.createDirectories(savepointDirPath); + Files.move(savepointPath, savepointDirPath, StandardCopyOption.ATOMIC_MOVE); + } + + @ParameterizedTest + @MethodSource("createSpecs") + void testRestore(ExecNodeMetadata metadata, TableTestProgram program) throws Exception { + final EnvironmentSettings settings = EnvironmentSettings.inStreamingMode(); + final SavepointRestoreSettings restoreSettings = + SavepointRestoreSettings.forPath( + getSavepointPath(program, metadata).toString(), + false, + RestoreMode.NO_CLAIM); + SavepointRestoreSettings.toConfiguration(restoreSettings, settings.getConfiguration()); + final TableEnvironment tEnv = TableEnvironment.create(settings); + tEnv.getConfig() + .set( + TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, + TableConfigOptions.CatalogPlanRestore.IDENTIFIER); + for (SourceTestStep sourceTestStep : program.getSetupSourceTestSteps()) { + final String id = TestValuesTableFactory.registerData(sourceTestStep.dataAfterRestore); + final Map options = new HashMap<>(); + options.put("connector", "values"); + options.put("data-id", id); + options.put("disable-lookup", "true"); + options.put("runtime-source", "NewSource"); + sourceTestStep.apply(tEnv, options); + } + + for (SinkTestStep sinkTestStep : program.getSetupSinkTestSteps()) { + final Map options = new HashMap<>(); + options.put("connector", "values"); + options.put("disable-lookup", "true"); + options.put("sink-insert-only", "false"); + sinkTestStep.apply(tEnv, options); + } + + final CompiledPlan compiledPlan = + tEnv.loadPlan(PlanReference.fromFile(getPlanPath(program, metadata))); + compiledPlan.execute().await(); + for (SinkTestStep sinkTestStep : program.getSetupSinkTestSteps()) { + assertThat(TestValuesTableFactory.getRawResultsAsStrings(sinkTestStep.name)) + .containsExactlyInAnyOrder( + Stream.concat( + sinkTestStep.getExpectedBeforeRestoreAsStrings() + .stream(), + sinkTestStep.getExpectedAfterRestoreAsStrings() + .stream()) + .toArray(String[]::new)); + } + } + + private Path getPlanPath(TableTestProgram program, ExecNodeMetadata metadata) { + return Paths.get( + getTestResourceDirectory(program, metadata) + "/plan/" + program.id + ".json"); + } + + private Path getSavepointPath(TableTestProgram program, ExecNodeMetadata metadata) { + return Paths.get(getTestResourceDirectory(program, metadata) + "/savepoint/"); + } + + private String getTestResourceDirectory(TableTestProgram program, ExecNodeMetadata metadata) { + return String.format( + "%s/src/test/resources/restore-tests/%s_%d/%s", + System.getProperty("user.dir"), metadata.name(), metadata.version(), program.id); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/plan/calc-simple.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/plan/calc-simple.json new file mode 100644 index 0000000000000..486b19248602b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/plan/calc-simple.json @@ -0,0 +1,113 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "DOUBLE" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` BIGINT, `b` DOUBLE>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`EXPR$0` BIGINT, `b` DOUBLE>", + "description" : "Calc(select=[(a + 1) AS EXPR$0, b])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "DOUBLE" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`EXPR$0` BIGINT, `b` DOUBLE>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[EXPR$0, b])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-simple/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..e98aee3d6c8952899b9c836d73942309f727b8ea GIT binary patch literal 7032 zcmeHMUuYaf7@xbOG5wpgO^dB<2{x@1d%O4lk9qMfx!9}MOSrqVB?aT$?p(6T-tKjG z?s_RwQiNJ5SOtCX!74@Z!3cr~KKURL1%0ygL4Bx*AQVN=f}mCWW^OOHm%F4XVi9`- zce^|D&G+{=v)_E1czVi(5bA^~80EYJ^~=7z*CXMa6wDKMUe1StQaDEVoEHkm-32A9uv#h5c9eUgQh5X#nS~^4h5zo z9y8+eS1JW<#x!X;;AU;AC}zc(lBQ6J+7QD0&&OK^fJL31|WD@ivwBr%bAL72UasNi>k2i&$;+#qZqoh%#Ky1;ES12(wtl9$I+|>}? z%|Gzvx9}5MXBz0y6(liynZk%dRM{NH6ldZ2)_V(k-aY^Nq!T%kXiJ`r)rL%pdXqD_ zj70`YF{zkzuwtMCt8%YRUcCHk-xcSx1t)SQQIDi+Wmxixt_`UemN?p4WvpUtO5|OG zNwiVbrxXdROf8DMNsuE}mx{=jipc#Xaw|lOR!r9C3|5L|1MO~BkDHy=&oIu{n@ zkj8C|&82Vwr1jVgJ%)>er%;bIKBk+n^mL7)?Wb%?kLG5GL_zD$Sw)j6>TjVglPiMU zCxHSew-hZ_HKDs~LX)t{ho44iK9MrlE5N^53mWY_6pi z+J+EVwS_@bU$N^@XbE9kP4SGZAt>)SmC zJRkh{LNE7TrwTna-yOa)gF53~F*N`AQ@`!wY#b6rJ8coJlU2jJz0R3$>uh~h&VRZ0 z)lczj9mtVDohIz@;4avx&ama}1bYH&@|%T;0l1E^6>a$jz|jG~mx(DEO0f#+Z8l72 zsG(?64AUAc3g&@_;GRXC-FH@s;8zC*EyOix5mTlT8f^q8sxVfY9p~7km!hs%;)z5m z%b~iPvCplU{jgpFwB3RI@aT9f8RtTGaUl;jxO4TLGD`J=BGq=Z-4`>5em}e4>jd;M z?VYgM*f5yX!F89ufANdU`)@$}kD_&DtdwYLt`RH=|7#)XAf_HKOn+e?g7k!!Wj z!8rEw`$NFjHl=Fj{QDmtJv94ACvYCFab{Bv^C<&-`oRi9eJB0|1FNsE>jtoBy=wjm68>`=ljB}&Ocvtr%73k)exLfUx}~Jh6R1~KObHwE5Y{9Dg&BDQZMM2b zFnlV2=bqZLqABLIy(d%3nWX~jsj<+! zEH7GSh;>r2mbJ4vta8Pg13`(xvlb_0YTqc>^f#Wn*8S$mz7Kf~W&`M0z`Qm>QJ}ey zCg}zdNrgy|3Fw-*#Asw`TCGY$QOgQWAJiIYJqS@=_oC=Xv#3^9+w~rRjI@ap_yW$# zz9uIyy@e&cg%)~q7sB6KYIy61qN&kTVknw~g0U6h{w1EE0Jvk>*OV4eT$;e5(9#;a SPOPZ1o}vdbflink-table/flink-table-planner/src/test/resources/**/*.out flink-table/flink-table-planner/src/test/resources/json/*.json flink-table/flink-table-planner/src/test/resources/jsonplan/*.json + flink-table/flink-table-planner/src/test/resources/restore-tests/** flink-yarn/src/test/resources/krb5.keytab flink-end-to-end-tests/test-scripts/test-data/** flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/hadoop/config/keystore.jks From 83ee295ae046bceebfa2dfb286afb20a814023e8 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sun, 29 Oct 2023 04:20:48 -0500 Subject: [PATCH 044/104] [FLINK-32661][sql-gateway] Fix unstable OperationRelatedITCase.testOperationRelatedApis This closes #23564. --- .../table/gateway/rest/OperationRelatedITCase.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/OperationRelatedITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/OperationRelatedITCase.java index a26799462dad1..b7e9e2fef4b9e 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/OperationRelatedITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/OperationRelatedITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.gateway.rest; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.table.gateway.api.operation.OperationHandle; @@ -43,7 +44,6 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -119,6 +119,9 @@ List submitOperation() throws Exception { SessionHandle sessionHandle = new SessionHandle(UUID.fromString(sessionHandleId)); assertThat(SQL_GATEWAY_SERVICE_EXTENSION.getSessionManager().getSession(sessionHandle)) .isNotNull(); + + OneShotLatch startLatch = new OneShotLatch(); + Thread main = Thread.currentThread(); OperationHandle operationHandle = SQL_GATEWAY_SERVICE_EXTENSION .getService() @@ -126,11 +129,15 @@ List submitOperation() throws Exception { sessionHandle, () -> { try { - TimeUnit.SECONDS.sleep(10); + startLatch.trigger(); + // keep operation in RUNNING state in response to cancel + // or close operations. + main.join(); } catch (InterruptedException ignored) { } return NotReadyResult.INSTANCE; }); + startLatch.await(); assertThat(operationHandle).isNotNull(); return Arrays.asList(sessionHandleId, operationHandle.getIdentifier().toString()); } From 329e3c183501fe99be37d663b54f1c1e43bab573 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Sat, 28 Oct 2023 15:07:32 +0200 Subject: [PATCH 045/104] [hotfix] Prepare StatefulSequenceSource for removal in 2.0 --- .../api/functions/source/StatefulSequenceSource.java | 3 +++ .../flink/connector/datagen/table/DataGenTableSource.java | 6 +----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java index a9f4d1e7023b5..983f49c7eca99 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java @@ -42,6 +42,9 @@ *

This strategy guarantees that each element will be emitted exactly-once, but elements will not * necessarily be emitted in ascending order, even for the same tasks. * + *

NOTE: this source will be removed together with the deprecated + * StreamExecutionEnvironmetn#generateSequence() method. + * * @deprecated This class is based on the {@link * org.apache.flink.streaming.api.functions.source.SourceFunction} API, which is due to be * removed. Use the new {@link org.apache.flink.api.connector.source.Source} API instead. diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenTableSource.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenTableSource.java index f9c1bf4deee09..9163c2a63f2fd 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenTableSource.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenTableSource.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.connector.datagen.table.types.RowDataGenerator; -import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource; import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator; import org.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSource; import org.apache.flink.table.connector.ChangelogMode; @@ -33,10 +32,7 @@ import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.table.types.DataType; -/** - * A {@link StreamTableSource} that emits each number from a given interval exactly once, possibly - * in parallel. See {@link StatefulSequenceSource}. - */ +/** A {@link StreamTableSource} that emits generated data rows. */ @Internal public class DataGenTableSource implements ScanTableSource, SupportsLimitPushDown { From b946ecc668342d48c2c0193ad4eff1897c75b68f Mon Sep 17 00:00:00 2001 From: JunRuiLee Date: Wed, 13 Sep 2023 18:51:27 +0800 Subject: [PATCH 046/104] [FLINK-33084][runtime] Migrate globalJobParameter to configuration. This closes #23409. --- .../flink/api/common/ExecutionConfig.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index f3fae490cf712..22aee331513cc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -154,8 +154,6 @@ public class ExecutionConfig implements Serializable, Archiveable parameters) { + configuration.set(PipelineOptions.GLOBAL_JOB_PARAMETERS, parameters); } // -------------------------------------------------------------------------------------------- @@ -985,7 +990,6 @@ public boolean equals(Object obj) { || (null != restartStrategyConfiguration && restartStrategyConfiguration.equals( other.restartStrategyConfiguration))) - && Objects.equals(globalJobParameters, other.globalJobParameters) && registeredTypesWithKryoSerializerClasses.equals( other.registeredTypesWithKryoSerializerClasses) && defaultKryoSerializerClasses.equals(other.defaultKryoSerializerClasses) @@ -1002,7 +1006,6 @@ public int hashCode() { return Objects.hash( configuration, restartStrategyConfiguration, - globalJobParameters, registeredTypesWithKryoSerializerClasses, defaultKryoSerializerClasses, registeredKryoTypes, @@ -1018,8 +1021,6 @@ public String toString() { + executionRetryDelay + ", restartStrategyConfiguration=" + restartStrategyConfiguration - + ", globalJobParameters=" - + globalJobParameters + ", registeredTypesWithKryoSerializers=" + registeredTypesWithKryoSerializers + ", registeredTypesWithKryoSerializerClasses=" @@ -1150,7 +1151,6 @@ public void configure(ReadableConfig configuration, ClassLoader classLoader) { configuration.getOptional(PipelineOptions.FORCE_KRYO).ifPresent(this::setForceKryo); configuration .getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS) - .map(MapBasedJobParameters::new) .ifPresent(this::setGlobalJobParameters); configuration @@ -1202,7 +1202,7 @@ public void configure(ReadableConfig configuration, ClassLoader classLoader) { /** * @return A copy of internal {@link #configuration}. Note it is missing all options that are * stored as plain java fields in {@link ExecutionConfig}, for example {@link - * #registeredKryoTypes} or {@link #globalJobParameters}. + * #registeredKryoTypes}. */ @Internal public Configuration toConfiguration() { From 8d21c321dda18ad20022532c861d57ee38b70fda Mon Sep 17 00:00:00 2001 From: fengjiajie Date: Mon, 30 Oct 2023 16:24:40 +0800 Subject: [PATCH 047/104] [FLINK-33360][connector/common] Clean up finishedReaders after switch to next Enumerator This closes #23593. --- .../hybrid/HybridSourceSplitEnumerator.java | 1 + .../HybridSourceSplitEnumeratorTest.java | 36 +++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java index c8b00b6a1f4ad..17c70ebbdc4d3 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java @@ -259,6 +259,7 @@ private void switchEnumerator() { if (currentEnumerator != null) { try { currentEnumerator.close(); + finishedReaders.clear(); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java index fcde32811f4ca..8b068d645b6c7 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java @@ -252,6 +252,42 @@ public void testInterceptNoMoreSplitEvent() { assertThat(context.hasNoMoreSplits(0)).isTrue(); } + @Test + public void testMultiSubtaskSwitchEnumerator() { + context = new MockSplitEnumeratorContext<>(2); + source = + HybridSource.builder(MOCK_SOURCE) + .addSource(MOCK_SOURCE) + .addSource(MOCK_SOURCE) + .build(); + + enumerator = (HybridSourceSplitEnumerator) source.createEnumerator(context); + enumerator.start(); + + registerReader(context, enumerator, SUBTASK0); + registerReader(context, enumerator, SUBTASK1); + enumerator.handleSourceEvent(SUBTASK0, new SourceReaderFinishedEvent(-1)); + enumerator.handleSourceEvent(SUBTASK1, new SourceReaderFinishedEvent(-1)); + + assertThat(getCurrentSourceIndex(enumerator)).isEqualTo(0); + enumerator.handleSourceEvent(SUBTASK0, new SourceReaderFinishedEvent(0)); + assertThat(getCurrentSourceIndex(enumerator)).isEqualTo(0); + enumerator.handleSourceEvent(SUBTASK1, new SourceReaderFinishedEvent(0)); + assertThat(getCurrentSourceIndex(enumerator)) + .as("all reader finished source-0") + .isEqualTo(1); + + enumerator.handleSourceEvent(SUBTASK0, new SourceReaderFinishedEvent(1)); + assertThat(getCurrentSourceIndex(enumerator)) + .as( + "only reader-0 has finished reading, reader-1 is not yet done, so do not switch to the next source") + .isEqualTo(1); + enumerator.handleSourceEvent(SUBTASK1, new SourceReaderFinishedEvent(1)); + assertThat(getCurrentSourceIndex(enumerator)) + .as("all reader finished source-1") + .isEqualTo(2); + } + private static class UnderlyingEnumeratorWrapper implements SplitEnumerator { private static final MockSourceSplit SPLIT_1 = new MockSourceSplit(0, 0, 1); From 957eaeda496a5a0bc80c86601217a3d643671317 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 25 Oct 2023 16:05:18 +0200 Subject: [PATCH 048/104] [FLINK-32181][docs] Enforce Maven 3.8.6 as required version --- README.md | 5 +---- pom.xml | 6 +----- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index b2691132ffa24..a510cc8996216 100644 --- a/README.md +++ b/README.md @@ -69,7 +69,7 @@ Prerequisites for building Flink: * Unix-like environment (we use Linux, Mac OS X, Cygwin, WSL) * Git -* Maven (we recommend version 3.8.6 and require at least 3.1.1) +* Maven (we require version 3.8.6) * Java 8 or 11 (Java 9 or 10 may work) ``` @@ -80,9 +80,6 @@ cd flink Flink is now installed in `build-target`. -*NOTE: Maven 3.3.x can build Flink, but will not properly shade away certain dependencies. Maven 3.1.1 creates the libraries properly. -To build unit tests with Java 8, use Java 8u51 or above to prevent failures in unit tests that use the PowerMock runner.* - ## Developing Flink The Flink committers use IntelliJ IDEA to develop the Flink codebase. diff --git a/pom.xml b/pom.xml index 3de7cb078b550..9d3176551dced 100644 --- a/pom.xml +++ b/pom.xml @@ -1396,9 +1396,6 @@ under the License. - - [3.8.6] - [1.8.0,1.8.1) @@ -1799,8 +1796,7 @@ under the License. - - [3.1.1,) + [3.8.6] ${target.java.version} From 806147c3233a47eacaa630dca5fdfef83397ab31 Mon Sep 17 00:00:00 2001 From: caicancai <2356672992@qq.com> Date: Mon, 30 Oct 2023 16:59:17 +0800 Subject: [PATCH 049/104] [FLINK-33393][doc] Fix typo in function documentation --- docs/data/sql_functions_zh.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml index 69eeccc64b3b7..dea5b499bc1e1 100644 --- a/docs/data/sql_functions_zh.yml +++ b/docs/data/sql_functions_zh.yml @@ -339,7 +339,7 @@ string: table: STRING.rtrim() description: | 返回从 STRING 中删除右边空格的字符串。 - 例如 `'This is a test String. '.ltrim()` 返回 `'This is a test String.'`。 + 例如 `'This is a test String. '.rtrim()` 返回 `'This is a test String.'`。 - sql: REPEAT(string, int) table: STRING.repeat(INT) description: | From 701e937415fbbfd5024d9f771e61e4c11ede01c5 Mon Sep 17 00:00:00 2001 From: Xiangyu Feng Date: Mon, 30 Oct 2023 19:40:07 +0800 Subject: [PATCH 050/104] [hotfix][docs] Add SQL Gateway doc reference in the Chinese version of table overview --- docs/content.zh/docs/dev/table/overview.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content.zh/docs/dev/table/overview.md b/docs/content.zh/docs/dev/table/overview.md index 05f758ac98cf4..ce62e43498619 100644 --- a/docs/content.zh/docs/dev/table/overview.md +++ b/docs/content.zh/docs/dev/table/overview.md @@ -52,6 +52,7 @@ and later use the DataStream API to build alerting based on the matched patterns * [SQL]({{< ref "docs/dev/table/sql/overview" >}}): SQL 支持的操作和语法。 * [内置函数]({{< ref "docs/dev/table/functions/systemFunctions" >}}): Table API 和 SQL 中的内置函数。 * [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}): 不用编写代码就可以尝试 Flink SQL,可以直接提交 SQL 任务到集群上。 +* [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}): SQL 提交服务,支持多个客户端从远端并发提交 SQL 任务。 * [SQL Jdbc Driver]({{< ref "docs/dev/table/jdbcDriver" >}}): 标准JDBC Driver,可以提交Flink SQL作业到Sql Gateway。 * [OLAP Quickstart]({{< ref "docs/dev/table/olap_quickstart" >}}): Flink OLAP服务搭建指南. From cc62044efc054b057d02838a02356fe7c9c9d7a2 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Mon, 30 Oct 2023 15:16:13 +0100 Subject: [PATCH 051/104] [FLINK-33334][table] Make map entries sorted by keys in json plan to have it stable for java21 This closes #23562 --- .../plan/nodes/exec/serde/JsonSerdeUtil.java | 2 + .../resources/jsonplan/testGetJsonPlan.out | 4 +- .../testComplexCalc.out | 4 +- .../testProjectPushDown.out | 4 +- .../CalcJsonPlanTest_jsonplan/testSarg.out | 8 +- .../testSimpleFilter.out | 4 +- .../testSimpleProject.out | 4 +- .../testChangelogSource.out | 8 +- .../testUpsertSource.out | 8 +- .../testCrossJoin.out | 4 +- .../testCrossJoinOverrideParameters.out | 4 +- .../testJoinWithFilter.out | 4 +- .../testLeftOuterJoinWithLiteralTrue.out | 4 +- .../testDeduplication.out | 4 +- .../testExpand.out | 8 +- ...inctAggCalls[isMiniBatchEnabled=false].out | 8 +- ...tinctAggCalls[isMiniBatchEnabled=true].out | 8 +- ...sWithGroupBy[isMiniBatchEnabled=false].out | 8 +- ...lsWithGroupBy[isMiniBatchEnabled=true].out | 8 +- ...thoutGroupBy[isMiniBatchEnabled=false].out | 8 +- ...ithoutGroupBy[isMiniBatchEnabled=true].out | 8 +- ...inedAggCalls[isMiniBatchEnabled=false].out | 8 +- ...finedAggCalls[isMiniBatchEnabled=true].out | 8 +- .../testIncrementalAggregate.out | 8 +- ...egateWithSumCountDistinctAndRetraction.out | 8 +- ...estProcessingTimeInnerJoinWithOnClause.out | 8 +- .../testRowTimeInnerJoinWithOnClause.out | 8 +- .../testInnerJoin.out | 8 +- .../testInnerJoinWithEqualPk.out | 12 +-- .../testInnerJoinWithPk.out | 12 +-- .../testLeftJoinNonEqui.out | 12 +-- .../LimitJsonPlanTest_jsonplan/testLimit.out | 8 +- .../testAggAndLeftJoinWithTryResolveMode.out | 12 +-- .../testJoinTemporalTable.out | 8 +- .../testJoinTemporalTableWithAsyncHint.out | 8 +- .../testJoinTemporalTableWithAsyncHint2.out | 8 +- ...estJoinTemporalTableWithAsyncRetryHint.out | 8 +- ...stJoinTemporalTableWithAsyncRetryHint2.out | 8 +- ...oinTemporalTableWithProjectionPushDown.out | 8 +- .../testJoinTemporalTableWithRetryHint.out | 8 +- ...inTemporalTableWithMultiJoinConditions.out | 10 +- ...estLeftJoinTemporalTableWithPostFilter.out | 10 +- ...testLeftJoinTemporalTableWithPreFilter.out | 10 +- .../testMatch.out | 16 +-- .../testSkipPastLastRow.out | 100 +++++++++--------- .../testSkipToFirst.out | 100 +++++++++--------- .../testSkipToLast.out | 100 +++++++++--------- .../testSkipToNextRow.out | 100 +++++++++--------- ...ProcTimeBoundedNonPartitionedRangeOver.out | 8 +- ...estProcTimeBoundedPartitionedRangeOver.out | 8 +- ...PartitionedRowsOverWithBuiltinProctime.out | 8 +- ...tProcTimeUnboundedPartitionedRangeOver.out | 8 +- ...ctimeBoundedDistinctPartitionedRowOver.out | 8 +- ...tinctWithNonDistinctPartitionedRowOver.out | 8 +- .../testRowTimeBoundedPartitionedRowsOver.out | 8 +- .../testPythonCalc.out | 4 +- .../testPythonFunctionInWhereClause.out | 4 +- .../testJoinWithFilter.out | 4 +- .../testPythonTableFunction.out | 4 +- .../tesPythonAggCallsWithGroupBy.out | 8 +- ...ProcTimeBoundedNonPartitionedRangeOver.out | 8 +- ...estProcTimeBoundedPartitionedRangeOver.out | 8 +- ...PartitionedRowsOverWithBuiltinProctime.out | 8 +- ...tProcTimeUnboundedPartitionedRangeOver.out | 8 +- .../testRowTimeBoundedPartitionedRowsOver.out | 8 +- .../RankJsonPlanTest_jsonplan/testRank.out | 8 +- .../SortJsonPlanTest_jsonplan/testSort.out | 10 +- .../testSortLimit.out | 8 +- ...onDeterministicFuncSinkWithDifferentPk.out | 8 +- .../testOverwrite.out | 8 +- .../testPartialInsert.out | 8 +- .../testPartitioning.out | 8 +- .../testWritingMetadata.out | 4 +- .../testFilterPushDown.out | 4 +- .../testLimitPushDown.out | 4 +- .../testPartitionPushDown.out | 4 +- .../testProjectPushDown.out | 4 +- .../testReadingMetadata.out | 4 +- .../testWatermarkPushDown.out | 4 +- .../UnionJsonPlanTest_jsonplan/testUnion.out | 8 +- .../testValues.out | 4 +- .../testWatermarkAssigner.out | 4 +- .../testDistinctSplitEnabled.out | 4 +- 83 files changed, 482 insertions(+), 480 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java index dffd134e97a84..b55fccbff2865 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java @@ -55,6 +55,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.jsontype.NamedType; @@ -107,6 +108,7 @@ public static boolean hasJsonCreatorAnnotation(Class clazz) { .getTypeFactory() .withClassLoader(JsonSerdeUtil.class.getClassLoader())); OBJECT_MAPPER_INSTANCE.configure(MapperFeature.USE_GETTERS_AS_SETTERS, false); + OBJECT_MAPPER_INSTANCE.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); OBJECT_MAPPER_INSTANCE.registerModule(createFlinkTableJacksonModule()); } diff --git a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out index 5eb4e8941c8ad..57549b10481a0 100644 --- a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out +++ b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out @@ -27,8 +27,8 @@ }, "partitionKeys": [], "options": { - "connector": "values", - "bounded": "false" + "bounded": "false", + "connector": "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out index ade6314c15f68..a9cdda84b2ad9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testProjectPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testProjectPushDown.out index 3a8f87c94b831..13d6fdbeeb8a3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testProjectPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testProjectPushDown.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSarg.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSarg.out index 2c506c6719f42..0ff3c9b31ecb2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSarg.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSarg.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -122,9 +122,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out index 73d8228c65ac9..1bb7c152eaae5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out index 19d6e6bf07f74..c05aa95d598d1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out index e7c509cf670c8..9116ad3ddb48d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out @@ -30,9 +30,9 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", "bounded" : "false", - "changelog-mode" : "I,UA,UB,D" + "changelog-mode" : "I,UA,UB,D", + "connector" : "values" } } }, @@ -123,9 +123,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out index f53f927d7e916..a45810c151d18 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out @@ -30,9 +30,9 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", "bounded" : "false", - "changelog-mode" : "I,UA,D" + "changelog-mode" : "I,UA,D", + "connector" : "values" } } }, @@ -111,9 +111,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out index 42e4499429cd8..6f30a30e3418a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out index b5d9287dbf398..dde24802d1999 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out index 366db5b841f7d..bc89928a365c4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out index c8ca94f443f6c..d064518274717 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out index 1db7a3dc67d6f..445a47a58128d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out @@ -42,9 +42,9 @@ }, "partitionKeys" : [ ], "options" : { - "disable-lookup" : "true", + "bounded" : "false", "connector" : "values", - "bounded" : "false" + "disable-lookup" : "true" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out index 22743e19db5ac..feb6c3119e58c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -360,9 +360,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out index 5af90e8debb39..72e7811b337db 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -287,9 +287,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out index c45ce2a1b36ad..7b78f56e5ed2a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -535,9 +535,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out index 6a9ffde549070..dc3ed1e0849b4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -230,9 +230,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out index 6f3abfbc2baa4..1c39ad82ffd49 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -295,9 +295,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out index 084191c1acbd5..aebfcd2a9b578 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -257,9 +257,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out index 670217206d501..3be27cfec5ce7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -331,9 +331,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out index eeed9ddd9af68..0777239ed59ec 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -223,9 +223,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out index 43c87558a63c1..bdb25ed04987b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -239,9 +239,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out index 53eab2ccd9642..46cc85e26f347 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -322,9 +322,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out index 2da16142ce53a..7a48fa0143f9a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -485,9 +485,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out index dbf27d1b24101..08846eee7452d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -354,8 +354,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out index 1632adb97d817..ab5aa86939d24 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -277,8 +277,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out index d0e30cb050920..b260494e3b3f6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -75,8 +75,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out index a2808111c7009..7c65bb5484015 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -114,8 +114,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -246,9 +246,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out index bb03b9366e866..5eb471f3a88a9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -145,8 +145,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -344,9 +344,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out index cf900d6bad687..da25c9e64b004 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -75,8 +75,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -202,9 +202,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out index 8690fc66fbdb8..ebce53e07ccf3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -143,9 +143,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testAggAndLeftJoinWithTryResolveMode.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testAggAndLeftJoinWithTryResolveMode.out index 0427d67cafb60..2b03ae977daaa 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testAggAndLeftJoinWithTryResolveMode.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testAggAndLeftJoinWithTryResolveMode.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -293,8 +293,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -381,8 +381,8 @@ }, "partitionKeys" : [ ], "options" : { - "sink-insert-only" : "false", - "connector" : "values" + "connector" : "values", + "sink-insert-only" : "false" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out index 0ab16e65b1db9..fa842045e8ee8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint.out index eeb78ad1818a3..e8c02f10f0213 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint2.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint2.out index eeb78ad1818a3..e8c02f10f0213 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint2.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncHint2.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint.out index b1ee0116b770b..8907e440458bd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint2.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint2.out index b1ee0116b770b..8907e440458bd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint2.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithAsyncRetryHint2.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out index 55df978f4d3c1..b6d416ef76e96 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithRetryHint.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithRetryHint.out index b1ee0116b770b..8907e440458bd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithRetryHint.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithRetryHint.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -242,8 +242,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithMultiJoinConditions.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithMultiJoinConditions.out index d8163b45429bb..6677518971740 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithMultiJoinConditions.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithMultiJoinConditions.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -291,8 +291,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -523,4 +523,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPostFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPostFilter.out index e102a96e014e2..c5b68ad2d09ed 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPostFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPostFilter.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -266,8 +266,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -498,4 +498,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPreFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPreFilter.out index ea581a25a1eec..ed945695d588f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPreFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testLeftJoinTemporalTableWithPreFilter.out @@ -80,8 +80,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -257,8 +257,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -489,4 +489,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out index 02783c493d92f..562fc04e8801d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out @@ -36,8 +36,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -178,7 +178,7 @@ } ], "type" : "BOOLEAN" }, - "l" : { + "C" : { "kind" : "CALL", "syntax" : "BINARY", "internalName" : "$=$1", @@ -198,12 +198,12 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "LITERAL", - "value" : "b", + "value" : "c", "type" : "VARCHAR(2147483647) NOT NULL" } ], "type" : "BOOLEAN" }, - "C" : { + "l" : { "kind" : "CALL", "syntax" : "BINARY", "internalName" : "$=$1", @@ -223,7 +223,7 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "LITERAL", - "value" : "c", + "value" : "b", "type" : "VARCHAR(2147483647) NOT NULL" } ], "type" : "BOOLEAN" @@ -328,9 +328,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out index 5121afda49410..f6327ec0b2630 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out @@ -37,8 +37,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -246,7 +246,28 @@ } }, "measures" : { - "startTime" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -254,28 +275,18 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" }, "endTime" : { "kind" : "CALL", @@ -308,7 +319,7 @@ } ], "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, - "Initial_Temp" : { + "startTime" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -316,39 +327,28 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 2, + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } }, "after" : { @@ -418,9 +418,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } @@ -465,4 +465,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out index 854473fa59508..ed49d18cacc16 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out @@ -37,8 +37,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -246,7 +246,28 @@ } }, "measures" : { - "startTime" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -254,28 +275,18 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" }, "endTime" : { "kind" : "CALL", @@ -308,7 +319,7 @@ } ], "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, - "Initial_Temp" : { + "startTime" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -316,39 +327,28 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 2, + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } }, "after" : { @@ -420,9 +420,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } @@ -467,4 +467,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out index 9fea5038705a4..d5c13f04e042b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out @@ -37,8 +37,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -246,7 +246,28 @@ } }, "measures" : { - "startTime" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -254,28 +275,18 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" }, "endTime" : { "kind" : "CALL", @@ -308,7 +319,7 @@ } ], "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, - "Initial_Temp" : { + "startTime" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -316,39 +327,28 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 2, + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } }, "after" : { @@ -420,9 +420,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } @@ -467,4 +467,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out index 40d68bff4d74d..8e6d41a62367c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out @@ -37,8 +37,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -246,7 +246,28 @@ } }, "measures" : { - "startTime" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -254,28 +275,18 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" + "type" : "INT" }, "endTime" : { "kind" : "CALL", @@ -308,7 +319,7 @@ } ], "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, - "Initial_Temp" : { + "startTime" : { "kind" : "CALL", "syntax" : "PREFIX", "internalName" : "$FINAL$1", @@ -316,39 +327,28 @@ "kind" : "CALL", "internalName" : "$FIRST$1", "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 2, + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" }, { "kind" : "LITERAL", "value" : 0, "type" : "INT NOT NULL" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } ], - "type" : "INT" + "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" } }, "after" : { @@ -418,9 +418,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } @@ -465,4 +465,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index 96a38d5f48d58..3635ab27dc215 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -385,9 +385,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index 9a90ac0406e42..2b52c554e2899 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -445,9 +445,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index 5ddebb6ae1f95..8c87e0e62e2be 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -385,9 +385,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index 12a00e1f28715..babc6ed122d0b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -419,9 +419,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out index 911a27619a85c..3c1c21c43be33 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -429,9 +429,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out index 17cd04fe9875b..0a3af51be4d20 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -525,9 +525,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out index 9b29878b0593b..42f1c3c1c1ab3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -260,9 +260,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonCalc.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonCalc.out index 0e3935fbd64c2..eab1e44c36a50 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonCalc.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonCalc.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonFunctionInWhereClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonFunctionInWhereClause.out index 44ded93003775..d305e54052825 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonFunctionInWhereClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest_jsonplan/testPythonFunctionInWhereClause.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out index 1759ba3f2a411..96cad4cd6bdbc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testPythonTableFunction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testPythonTableFunction.out index 5c7ccfcadb846..494b31e30f5c1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testPythonTableFunction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest_jsonplan/testPythonTableFunction.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest_jsonplan/tesPythonAggCallsWithGroupBy.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest_jsonplan/tesPythonAggCallsWithGroupBy.out index b9e1f6fb9effe..41cd0764ec9a7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest_jsonplan/tesPythonAggCallsWithGroupBy.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest_jsonplan/tesPythonAggCallsWithGroupBy.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -176,9 +176,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index fd0196523962c..cf7ae2c3a8709 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -378,9 +378,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index 9efabb0de721d..dc25537075fc3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -392,9 +392,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index fcf958090e660..14e85a6ea752f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -329,9 +329,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index 4efefdf2625e8..b4f591bbe3e8e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -382,9 +382,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out index a6f809538c84b..f0012a930097b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out @@ -56,8 +56,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -324,9 +324,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out index 7bcce2d7785d7..7209ea0be4acf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -157,9 +157,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest_jsonplan/testSort.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest_jsonplan/testSort.out index a0d7d070ec1de..c73bfeead3fd3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest_jsonplan/testSort.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest_jsonplan/testSort.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -122,9 +122,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } @@ -169,4 +169,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} \ No newline at end of file +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out index 7d31643fdba4e..e38d37c84e5fa 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -144,9 +144,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out index 35f561d030ed4..b8ee11efc4901 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out @@ -30,8 +30,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "changelog-mode" : "I,UA,UB,D" + "changelog-mode" : "I,UA,UB,D", + "connector" : "values" } } } @@ -111,8 +111,8 @@ }, "partitionKeys" : [ ], "options" : { - "sink-insert-only" : "false", - "connector" : "values" + "connector" : "values", + "sink-insert-only" : "false" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out index 564be270d1759..18fc2687ff367 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -60,9 +60,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "filesystem", "format" : "testcsv", - "path" : "/tmp", - "connector" : "filesystem" + "path" : "/tmp" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartialInsert.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartialInsert.out index 06f096a3e4bf8..db7fba6642904 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartialInsert.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartialInsert.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } @@ -107,9 +107,9 @@ }, "partitionKeys" : [ "c" ], "options" : { + "connector" : "filesystem", "format" : "testcsv", - "path" : "/tmp", - "connector" : "filesystem" + "path" : "/tmp" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out index f0b5a7146fc2d..d4c1415e67ac8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -95,9 +95,9 @@ }, "partitionKeys" : [ "c" ], "options" : { + "connector" : "filesystem", "format" : "testcsv", - "path" : "/tmp", - "connector" : "filesystem" + "path" : "/tmp" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out index 53fd3f65a1370..3508d24f6897b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out index 603153e900ced..87cd3712253c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out @@ -22,9 +22,9 @@ }, "partitionKeys" : [ ], "options" : { + "bounded" : "false", "connector" : "values", - "filterable-fields" : "a", - "bounded" : "false" + "filterable-fields" : "a" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out index 7a0c5148ad244..40cd6fd45e8be 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out index 8cd42971c0b97..5dd4454e395ef 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out @@ -22,9 +22,9 @@ }, "partitionKeys" : [ "p" ], "options" : { + "bounded" : "false", "connector" : "values", - "partition-list" : "p:A", - "bounded" : "false" + "partition-list" : "p:A" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out index 40ec065c6fd14..006a5a99aa095 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out @@ -22,8 +22,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out index 8b4be5346174e..574ee2cb1f401 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out @@ -27,9 +27,9 @@ }, "partitionKeys" : [ ], "options" : { + "bounded" : "false", "connector" : "values", - "readable-metadata" : "m:STRING", - "bounded" : "false" + "readable-metadata" : "m:STRING" } } }, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out index 451bb80a2c6fd..09e6aaf62cb26 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out @@ -46,9 +46,9 @@ }, "partitionKeys" : [ ], "options" : { - "disable-lookup" : "true", - "connector" : "values", "bounded" : "false", + "connector" : "values", + "disable-lookup" : "true", "enable-watermark-push-down" : "true" } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out index 6bd126eb71534..17fba7f6a3dd7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out @@ -25,8 +25,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } }, @@ -62,8 +62,8 @@ }, "partitionKeys" : [ ], "options" : { - "connector" : "values", - "bounded" : "false" + "bounded" : "false", + "connector" : "values" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out index d69774ef7fd52..77a6278b58564 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out @@ -104,9 +104,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out index 5665c7265363c..162b6b74d0cfb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out @@ -46,9 +46,9 @@ }, "partitionKeys" : [ ], "options" : { - "disable-lookup" : "true", - "connector" : "values", "bounded" : "false", + "connector" : "values", + "disable-lookup" : "true", "enable-watermark-push-down" : "false" } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out index e4d4795efab37..76428d4f2c362 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out @@ -819,9 +819,9 @@ }, "partitionKeys" : [ ], "options" : { + "connector" : "values", "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT", - "connector" : "values" + "table-sink-class" : "DEFAULT" } } } From 530ebd2f4ef59f84d2aedaf13a89b030480e3808 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 25 Oct 2023 16:00:34 +0200 Subject: [PATCH 052/104] [FLINK-32182][build] Use original japicmp plugin --- flink-scala/pom.xml | 2 +- flink-streaming-scala/pom.xml | 4 ++-- flink-table/flink-sql-jdbc-driver-bundle/pom.xml | 2 +- pom.xml | 8 ++++---- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index 8adbbab4cc3e9..c4c801e089dc6 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -343,7 +343,7 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml index c3d0e8e2cb75e..a40a76e4b0321 100644 --- a/flink-streaming-scala/pom.xml +++ b/flink-streaming-scala/pom.xml @@ -263,7 +263,7 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin @@ -305,7 +305,7 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin diff --git a/flink-table/flink-sql-jdbc-driver-bundle/pom.xml b/flink-table/flink-sql-jdbc-driver-bundle/pom.xml index 77585f330321d..d310182adff90 100644 --- a/flink-table/flink-sql-jdbc-driver-bundle/pom.xml +++ b/flink-table/flink-sql-jdbc-driver-bundle/pom.xml @@ -84,7 +84,7 @@ - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin true diff --git a/pom.xml b/pom.xml index 9d3176551dced..d288ca5f5c0e6 100644 --- a/pom.xml +++ b/pom.xml @@ -1199,7 +1199,7 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin true @@ -1470,7 +1470,7 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin @@ -2240,9 +2240,9 @@ under the License. - io.github.zentol.japicmp + com.github.siom79.japicmp japicmp-maven-plugin - 0.17.1.1_m325 + 0.17.1 From 25697476095a5b9cf38dc3b61c684d0e912b1353 Mon Sep 17 00:00:00 2001 From: JunRuiLee Date: Wed, 13 Sep 2023 16:00:52 +0800 Subject: [PATCH 053/104] =?UTF-8?q?[FLINK-33080][runtime]=20Fix=20?= =?UTF-8?q?=E2=80=98state.checkpoint-storage=E2=80=99=20not=20taking=20eff?= =?UTF-8?q?ect=20in=20job=20generation=20stage?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This closes #23408. --- .../program/StreamContextEnvironment.java | 1 + .../program/StreamContextEnvironmentTest.java | 30 +++++++++++++++++++ .../StreamExecutionEnvironment.java | 18 +++++++++++ .../StreamExecutionEnvironmentTest.java | 19 ++++++++++++ 4 files changed, 68 insertions(+) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java index 21ddca6169491..07f47573666e6 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java @@ -303,6 +303,7 @@ private void checkMainConfiguration(Configuration clusterConfigMap, List private void checkCheckpointConfig(Configuration clusterConfigMap, List errors) { CheckpointConfig expectedCheckpointConfig = new CheckpointConfig(); expectedCheckpointConfig.configure(clusterConfigMap); + configureCheckpointStorage(clusterConfigMap, expectedCheckpointConfig); checkConfigurationObject( expectedCheckpointConfig.toConfiguration(), checkpointCfg.toConfiguration(), diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java index 85f6279fc584e..1af115a402ab6 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java @@ -121,6 +121,36 @@ void testDisallowCheckpointStorage( CheckpointConfig.class.getSimpleName(), "setCheckpointStorage"); } + @ParameterizedTest + @MethodSource("provideExecutors") + void testDisallowCheckpointStorageByConfiguration( + ThrowingConsumer executor) { + final Configuration clusterConfig = new Configuration(); + + Configuration jobConfig = new Configuration(); + String disallowedPath = "file:///flink/disallowed/modification"; + jobConfig.set(CheckpointingOptions.CHECKPOINT_STORAGE, "jobmanager"); + jobConfig.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, disallowedPath); + final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + final StreamContextEnvironment environment = + new StreamContextEnvironment( + new MockExecutorServiceLoader(), + clusterConfig, + jobConfig, + classLoader, + true, + true, + false, + Collections.emptyList()); + + environment.fromCollection(Collections.singleton(1)).sinkTo(new DiscardingSink<>()); + assertThatThrownBy(() -> executor.accept(environment)) + .isInstanceOf(MutatedConfigurationException.class) + .hasMessageContainingAll( + CheckpointingOptions.CHECKPOINT_STORAGE.key(), + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + } + @ParameterizedTest @MethodSource("provideExecutors") void testAllowCheckpointStorage( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 891aabc52f671..343308edcf2d8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -73,6 +73,8 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.scheduler.ClusterDatasetCorruptedException; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.CheckpointStorageLoader; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateBackendLoader; @@ -1065,6 +1067,11 @@ public void configure(ReadableConfig configuration, ClassLoader classLoader) { config.configure(configuration, classLoader); checkpointCfg.configure(configuration); + + // here we should make sure the configured checkpoint storage will take effect + // this needs to happen after checkpointCfg#configure(...) to override the effect of + // checkpointCfg#setCheckpointStorage(checkpointDirectory) + configureCheckpointStorage(configuration, checkpointCfg); } private void registerCustomListeners( @@ -1098,6 +1105,17 @@ private void configBufferTimeout(ReadableConfig configuration) { } } + protected void configureCheckpointStorage( + ReadableConfig configuration, CheckpointConfig checkpointCfg) { + try { + Optional storageOptional = + CheckpointStorageLoader.fromConfig(configuration, userClassloader, null); + storageOptional.ifPresent(checkpointCfg::setCheckpointStorage); + } catch (DynamicCodeLoadingException e) { + throw new WrappingRuntimeException(e); + } + } + // -------------------------------------------------------------------------------------------- // Data stream creations // -------------------------------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentTest.java index 4cb61a7d4acf5..f04ce7a7883a1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentTest.java @@ -25,12 +25,16 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -473,6 +477,21 @@ public void go() { } } + @Test + void testConfigureCheckpointStorage() { + Configuration configuration = new Configuration(); + String path = "file:///valid"; + configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "jobmanager"); + configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, path); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + + CheckpointStorage storage = env.getCheckpointConfig().getCheckpointStorage(); + assertThat(storage).isInstanceOf(JobManagerCheckpointStorage.class); + assertThat(((JobManagerCheckpointStorage) storage).getCheckpointPath()) + .isEqualTo(new Path(path)); + } + ///////////////////////////////////////////////////////////// // Utilities ///////////////////////////////////////////////////////////// From 780a673d8e2c3845d685c86d95166d9169601726 Mon Sep 17 00:00:00 2001 From: Wencong Liu <104502720+WencongLiu@users.noreply.github.com> Date: Tue, 31 Oct 2023 16:48:53 +0800 Subject: [PATCH 054/104] =?UTF-8?q?[FLINK-33356][docs]=20Fix=20the=20messe?= =?UTF-8?q?d=20navigation=20bar=20on=20Flink=E2=80=99s=20official=20websit?= =?UTF-8?q?e?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This closes #23627 --- docs/themes/book | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/themes/book b/docs/themes/book index 3f1bcccbfb247..a486adf8462c0 160000 --- a/docs/themes/book +++ b/docs/themes/book @@ -1 +1 @@ -Subproject commit 3f1bcccbfb247da44ab5410a97576c0bf6da103b +Subproject commit a486adf8462c0abfc9034436ddd72927d6656809 From 81d559a3bb11722f5148a9f4ad42e60105ae27a2 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 30 Oct 2023 14:37:25 +0100 Subject: [PATCH 055/104] [hotfix][rest] Improve error message --- .../apache/flink/runtime/rest/messages/MessageHeaders.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java index 63c540834934f..122429e3c0d49 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java @@ -78,12 +78,15 @@ default Collection> getResponseTypeParameters() { * @return short description */ default String operationId() { + final String className = getClass().getSimpleName(); + if (getHttpMethod() != HttpMethodWrapper.GET) { throw new UnsupportedOperationException( - "The default implementation is only supported for GET calls. Please override 'operationId()'."); + "The default implementation is only supported for GET calls. Please override 'operationId()' in '" + + className + + "'."); } - final String className = getClass().getSimpleName(); final int headersSuffixStart = className.lastIndexOf("Headers"); if (headersSuffixStart == -1) { throw new IllegalStateException( From 9c3b18b0320c7260f1fe6e9dbbf489423a729cb4 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Tue, 31 Oct 2023 20:54:19 +0800 Subject: [PATCH 056/104] [FLINK-25055][network] Support listen and notify mechanism for remote partition request --- .../all_taskmanager_network_section.html | 10 +- ...tty_shuffle_environment_configuration.html | 10 +- .../NettyShuffleEnvironmentOptions.java | 23 +- .../network/NettyShuffleServiceFactory.java | 6 +- .../io/network/NetworkSequenceViewReader.java | 38 ++- ...reditBasedSequenceNumberingViewReader.java | 57 +++- .../netty/NettyPartitionRequestListener.java | 103 +++++++ .../network/netty/PartitionRequestQueue.java | 27 ++ .../netty/PartitionRequestServerHandler.java | 18 +- .../partition/PartitionRequestListener.java | 77 +++++ .../PartitionRequestListenerManager.java | 78 +++++ .../partition/ResultPartitionManager.java | 157 +++++++++- .../partition/ResultPartitionProvider.java | 26 ++ .../partition/consumer/InputChannel.java | 8 +- .../consumer/RemoteInputChannel.java | 25 +- .../consumer/RemoteRecoveredInputChannel.java | 4 + .../consumer/SingleInputGateFactory.java | 5 + .../consumer/UnknownInputChannel.java | 5 + .../shuffle/ShuffleEnvironmentContext.java | 10 +- .../taskexecutor/TaskManagerRunner.java | 1 + .../taskexecutor/TaskManagerServices.java | 12 +- .../NettyShuffleEnvironmentConfiguration.java | 20 ++ .../NettyShuffleEnvironmentBuilder.java | 13 + .../netty/CancelPartitionRequestTest.java | 49 ++-- ...asedPartitionRequestClientHandlerTest.java | 1 + ...tBasedSequenceNumberingViewReaderTest.java | 10 +- .../NettyPartitionRequestClientTest.java | 2 +- .../netty/PartitionRequestQueueTest.java | 111 ++++++-- .../PartitionRequestRegistrationTest.java | 267 ++++++++++++++++++ .../PartitionRequestServerHandlerTest.java | 43 +-- .../ServerTransportErrorHandlingTest.java | 22 +- .../PartitionRequestListenerManagerTest.java | 188 ++++++++++++ ...sultPartitionReleaseOnConsumptionTest.java | 6 +- .../partition/ResultPartitionFactoryTest.java | 39 +-- .../partition/ResultPartitionManagerTest.java | 146 +++++++++- .../TestingPartitionRequestListener.java | 106 +++++++ .../partition/TestingResultPartition.java | 136 +++++++++ .../TestingResultPartitionProvider.java | 135 +++++++++ .../TestingSubpartitionCreatedViewReader.java | 176 ++++++++++++ .../consumer/InputChannelBuilder.java | 10 + .../consumer/RemoteInputChannelTest.java | 26 +- .../consumer/SingleInputGateTest.java | 24 +- ...skExecutorLocalStateStoresManagerTest.java | 1 + .../taskexecutor/TaskExecutorBuilder.java | 1 + .../TaskSubmissionTestEnvironment.java | 10 + .../SingleInputGateBenchmarkFactory.java | 3 + 46 files changed, 2046 insertions(+), 199 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestRegistrationTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManagerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionRequestListener.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartition.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartitionProvider.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingSubpartitionCreatedViewReader.java diff --git a/docs/layouts/shortcodes/generated/all_taskmanager_network_section.html b/docs/layouts/shortcodes/generated/all_taskmanager_network_section.html index 81fd2233f4e23..d1d221664108b 100644 --- a/docs/layouts/shortcodes/generated/all_taskmanager_network_section.html +++ b/docs/layouts/shortcodes/generated/all_taskmanager_network_section.html @@ -182,17 +182,23 @@

+ + + + + + - + - + diff --git a/docs/layouts/shortcodes/generated/netty_shuffle_environment_configuration.html b/docs/layouts/shortcodes/generated/netty_shuffle_environment_configuration.html index 2e2ea2507688c..42419910d9b4c 100644 --- a/docs/layouts/shortcodes/generated/netty_shuffle_environment_configuration.html +++ b/docs/layouts/shortcodes/generated/netty_shuffle_environment_configuration.html @@ -170,17 +170,23 @@ + + + + + + - + - + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java index 8b2099ee8e3df..aeeb4b4725793 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java @@ -21,8 +21,12 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.description.Description; + +import java.time.Duration; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.description.TextElement.code; /** The set of configuration options relating to network stack. */ @PublicEvolving @@ -585,7 +589,7 @@ public class NettyShuffleEnvironmentOptions { .defaultValue(100) .withDeprecatedKeys("taskmanager.net.request-backoff.initial") .withDescription( - "Minimum backoff in milliseconds for partition requests of input channels."); + "Minimum backoff in milliseconds for partition requests of local input channels."); /** Maximum backoff for partition requests of input channels. */ @Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK) @@ -595,7 +599,22 @@ public class NettyShuffleEnvironmentOptions { .defaultValue(10000) .withDeprecatedKeys("taskmanager.net.request-backoff.max") .withDescription( - "Maximum backoff in milliseconds for partition requests of input channels."); + "Maximum backoff in milliseconds for partition requests of local input channels."); + + /** The timeout for partition request listener in result partition manager. */ + @Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK) + public static final ConfigOption NETWORK_PARTITION_REQUEST_TIMEOUT = + key("taskmanager.network.partition-request-timeout") + .durationType() + .defaultValue(Duration.ofSeconds(10)) + .withDescription( + Description.builder() + .text( + "Timeout for an individual partition request of remote input channels. " + + "The partition request will finally fail if the total wait time exceeds " + + "twice the value of %s.", + code(NETWORK_REQUEST_BACKOFF_MAX.key())) + .build()); // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java index fe71f47762b08..7f708b45188cc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.concurrent.ExecutorThreadFactory; +import org.apache.flink.util.concurrent.ScheduledExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,6 +87,7 @@ public NettyShuffleEnvironment createShuffleEnvironment( shuffleEnvironmentContext.getEventPublisher(), shuffleEnvironmentContext.getParentMetricGroup(), shuffleEnvironmentContext.getIoExecutor(), + shuffleEnvironmentContext.getScheduledExecutor(), shuffleEnvironmentContext.getNumberOfSlots(), shuffleEnvironmentContext.getTmpDirPaths()); } @@ -97,13 +99,15 @@ static NettyShuffleEnvironment createNettyShuffleEnvironment( TaskEventPublisher taskEventPublisher, MetricGroup metricGroup, Executor ioExecutor, + ScheduledExecutor scheduledExecutor, int numberOfSlots, String[] tmpDirPaths) { return createNettyShuffleEnvironment( config, taskExecutorResourceId, taskEventPublisher, - new ResultPartitionManager(), + new ResultPartitionManager( + config.getPartitionRequestListenerTimeout(), scheduledExecutor), metricGroup, ioExecutor, numberOfSlots, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java index fc317ef2b023a..665dc1b975009 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java @@ -18,7 +18,11 @@ package org.apache.flink.runtime.io.network; +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; @@ -34,12 +38,36 @@ */ public interface NetworkSequenceViewReader { - void requestSubpartitionView( + /** + * When the netty server receives the downstream task's partition request and the upstream task + * has registered its partition, it will process the partition request immediately, otherwise it + * will create a {@link PartitionRequestListener} for given {@link ResultPartitionID} in {@link + * ResultPartitionManager} and notify the listener when the upstream task registers its + * partition. + * + * @param partitionProvider the result partition provider + * @param resultPartitionId the result partition id + * @param subPartitionIndex the sub partition index + * @throws IOException the thrown exception + */ + void requestSubpartitionViewOrRegisterListener( ResultPartitionProvider partitionProvider, ResultPartitionID resultPartitionId, int subPartitionIndex) throws IOException; + /** + * When the {@link ResultPartitionManager} registers {@link ResultPartition}, it will get the + * {@link PartitionRequestListener} via given {@link ResultPartitionID}, and create subpartition + * view reader for downstream task. + * + * @param partition the result partition + * @param subPartitionIndex the sub partition index + * @throws IOException the thrown exception + */ + void notifySubpartitionCreated(ResultPartition partition, int subPartitionIndex) + throws IOException; + @Nullable BufferAndAvailability getNextBuffer() throws IOException; @@ -91,4 +119,12 @@ void requestSubpartitionView( InputChannelID getReceiverId(); void notifyNewBufferSize(int newBufferSize); + + /** + * When the partition request from the given downstream task is timeout, it should notify the + * reader in netty server and send {@link PartitionNotFoundException} to the task. + * + * @param partitionRequestListener the timeout message of given {@link PartitionRequestListener} + */ + void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java index 4aed5e138df2d..c12db284c13c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java @@ -22,6 +22,8 @@ import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; @@ -33,8 +35,10 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Optional; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; /** * Simple wrapper for the subpartition view used in the new network credit-based mode. @@ -55,6 +59,8 @@ class CreditBasedSequenceNumberingViewReader private volatile ResultSubpartitionView subpartitionView; + private volatile PartitionRequestListener partitionRequestListener; + /** * The status indicating whether this reader is already enqueued in the pipeline for * transferring data or not. @@ -78,27 +84,47 @@ class CreditBasedSequenceNumberingViewReader } @Override - public void requestSubpartitionView( + public void requestSubpartitionViewOrRegisterListener( ResultPartitionProvider partitionProvider, ResultPartitionID resultPartitionId, int subPartitionIndex) throws IOException { - synchronized (requestLock) { - if (subpartitionView == null) { - // This call can trigger a notification we have to - // schedule a separate task at the event loop that will - // start consuming this. Otherwise the reference to the - // view cannot be available in getNextBuffer(). - this.subpartitionView = - partitionProvider.createSubpartitionView( - resultPartitionId, subPartitionIndex, this); + checkState(subpartitionView == null, "Subpartition already requested"); + checkState( + partitionRequestListener == null, "Partition request listener already created"); + partitionRequestListener = + new NettyPartitionRequestListener( + partitionProvider, this, subPartitionIndex, resultPartitionId); + // The partition provider will create subpartitionView if resultPartition is + // registered, otherwise it will register a listener of partition request to the result + // partition manager. + Optional subpartitionViewOptional = + partitionProvider.createSubpartitionViewOrRegisterListener( + resultPartitionId, subPartitionIndex, this, partitionRequestListener); + if (subpartitionViewOptional.isPresent()) { + this.subpartitionView = subpartitionViewOptional.get(); } else { - throw new IllegalStateException("Subpartition already requested"); + // If the subpartitionView is not exist, it means that the requested partition is + // not registered. + return; } } notifyDataAvailable(); + requestQueue.notifyReaderCreated(this); + } + + @Override + public void notifySubpartitionCreated(ResultPartition partition, int subPartitionIndex) + throws IOException { + synchronized (requestLock) { + checkState(subpartitionView == null, "Subpartition already requested"); + subpartitionView = partition.createSubpartitionView(subPartitionIndex, this); + } + + notifyDataAvailable(); + requestQueue.notifyReaderCreated(this); } @Override @@ -182,6 +208,12 @@ public void notifyNewBufferSize(int newBufferSize) { subpartitionView.notifyNewBufferSize(newBufferSize); } + @Override + public void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener) { + requestQueue.notifyPartitionRequestTimeout(partitionRequestListener); + this.partitionRequestListener = null; + } + @VisibleForTesting int getNumCreditsAvailable() { return numCreditsAvailable; @@ -226,6 +258,9 @@ public Throwable getFailureCause() { @Override public void releaseAllResources() throws IOException { + if (partitionRequestListener != null) { + partitionRequestListener.releaseListener(); + } subpartitionView.releaseAllResources(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java new file mode 100644 index 0000000000000..b09756bf22d64 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Implementation of {@link PartitionRequestListener} for netty partition request. */ +public class NettyPartitionRequestListener implements PartitionRequestListener { + private final ResultPartitionProvider resultPartitionProvider; + private final NetworkSequenceViewReader reader; + private final int subPartitionIndex; + private final ResultPartitionID resultPartitionId; + private final long createTimestamp; + + public NettyPartitionRequestListener( + ResultPartitionProvider resultPartitionProvider, + NetworkSequenceViewReader reader, + int subPartitionIndex, + ResultPartitionID resultPartitionId) { + this( + resultPartitionProvider, + reader, + subPartitionIndex, + resultPartitionId, + System.currentTimeMillis()); + } + + @VisibleForTesting + public NettyPartitionRequestListener( + ResultPartitionProvider resultPartitionProvider, + NetworkSequenceViewReader reader, + int subPartitionIndex, + ResultPartitionID resultPartitionId, + long createTimestamp) { + this.resultPartitionProvider = resultPartitionProvider; + this.reader = reader; + this.subPartitionIndex = subPartitionIndex; + this.resultPartitionId = resultPartitionId; + this.createTimestamp = createTimestamp; + } + + @Override + public long getCreateTimestamp() { + return createTimestamp; + } + + @Override + public ResultPartitionID getResultPartitionId() { + return resultPartitionId; + } + + @Override + public NetworkSequenceViewReader getViewReader() { + return reader; + } + + @Override + public InputChannelID getReceiverId() { + return reader.getReceiverId(); + } + + @Override + public void notifyPartitionCreated(ResultPartition partition) throws IOException { + checkNotNull(partition); + reader.notifySubpartitionCreated(partition, subPartitionIndex); + } + + @Override + public void notifyPartitionCreatedTimeout() { + reader.notifyPartitionRequestTimeout(this); + } + + @Override + public void releaseListener() { + resultPartitionProvider.releasePartitionRequestListener(this); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index 57337fdc4f7de..31042cfac3abf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -22,6 +22,9 @@ import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; @@ -259,6 +262,26 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exc if (toRelease != null) { releaseViewReader(toRelease); } + } else if (msg instanceof PartitionRequestListener) { + PartitionRequestListener partitionRequestListener = (PartitionRequestListener) msg; + + // Send partition not found message to the downstream task when the listener is timeout. + final ResultPartitionID resultPartitionId = + partitionRequestListener.getResultPartitionId(); + final InputChannelID inputChannelId = partitionRequestListener.getReceiverId(); + availableReaders.remove(partitionRequestListener.getViewReader()); + allReaders.remove(inputChannelId); + try { + ctx.writeAndFlush( + new NettyMessage.ErrorResponse( + new PartitionNotFoundException(resultPartitionId), inputChannelId)); + } catch (Exception e) { + LOG.warn( + "Write partition not found exception to {} for result partition {} fail", + inputChannelId, + resultPartitionId, + e); + } } else { ctx.fireUserEventTriggered(msg); } @@ -393,6 +416,10 @@ private void onChannelFutureFailure(ChannelFuture future) throws Exception { } } + public void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener) { + ctx.pipeline().fireUserEventTriggered(partitionRequestListener); + } + // This listener is called after an element of the current nonEmptyReader has been // flushed. If successful, the listener triggers further processing of the // queues. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java index ccd9b1c53a872..a0d6e25aec29f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.io.network.netty.NettyMessage.ResumeConsumption; import org.apache.flink.runtime.io.network.netty.NettyMessage.SegmentId; import org.apache.flink.runtime.io.network.netty.NettyMessage.TaskEventRequest; -import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; @@ -83,19 +82,14 @@ protected void channelRead0(ChannelHandlerContext ctx, NettyMessage msg) throws LOG.debug("Read channel on {}: {}.", ctx.channel().localAddress(), request); - try { - NetworkSequenceViewReader reader; - reader = - new CreditBasedSequenceNumberingViewReader( - request.receiverId, request.credit, outboundQueue); + NetworkSequenceViewReader reader; + reader = + new CreditBasedSequenceNumberingViewReader( + request.receiverId, request.credit, outboundQueue); - reader.requestSubpartitionView( - partitionProvider, request.partitionId, request.queueIndex); + reader.requestSubpartitionViewOrRegisterListener( + partitionProvider, request.partitionId, request.queueIndex); - outboundQueue.notifyReaderCreated(reader); - } catch (PartitionNotFoundException notFound) { - respondWithError(ctx, notFound, request.receiverId); - } } // ---------------------------------------------------------------- // Task events diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java new file mode 100644 index 0000000000000..82253be5256ab --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; + +import java.io.IOException; + +/** + * When the netty server receives a downstream task's partition request event and finds its upstream + * task doesn't register its partition yet, the netty server will construct a {@link + * PartitionRequestListener} and notify the listener when the task deploys itself and registers its + * partition to {@link ResultPartitionManager}. + */ +public interface PartitionRequestListener { + + /** + * The creation timestamp of this notifier, it's used to check whether the notifier is timeout. + * + * @return the creation timestamp + */ + long getCreateTimestamp(); + + /** + * Get the result partition id of the notifier. + * + * @return the result partition id + */ + ResultPartitionID getResultPartitionId(); + + /** + * Get the view reader of the notifier. + * + * @return the view reader + */ + NetworkSequenceViewReader getViewReader(); + + /** + * Get the input channel id of the notifier. + * + * @return the input channel id + */ + InputChannelID getReceiverId(); + + /** + * Notify the partition request listener when the given partition is registered. + * + * @param partition The registered partition. + */ + void notifyPartitionCreated(ResultPartition partition) throws IOException; + + /** + * When the partition request listener is timeout, it will be notified to send {@link + * PartitionNotFoundException}. + */ + void notifyPartitionCreatedTimeout(); + + /** Release this listener. */ + void releaseListener(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java new file mode 100644 index 0000000000000..e296d767fb890 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** Manages partition request listener with input channel id. */ +public class PartitionRequestListenerManager { + private final Map listeners; + + public PartitionRequestListenerManager() { + this.listeners = new HashMap<>(); + } + + public Collection getPartitionRequestListeners() { + return listeners.values(); + } + + public void remove(InputChannelID receiverId) { + listeners.remove(receiverId); + } + + public boolean isEmpty() { + return listeners.isEmpty(); + } + + public void registerListener(PartitionRequestListener listener) { + PartitionRequestListener previous = listeners.put(listener.getReceiverId(), listener); + if (previous != null) { + throw new IllegalStateException( + "Partition request listener with receiver " + + listener.getReceiverId() + + " has been registered."); + } + } + + /** + * Remove the expire partition request listener and add it to the given timeoutListeners. + * + * @param now the timestamp + * @param timeout the timeout mills + * @param timeoutListeners the expire partition request listeners + */ + public void removeExpiration( + long now, long timeout, Collection timeoutListeners) { + Iterator> iterator = + listeners.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + PartitionRequestListener partitionRequestListener = entry.getValue(); + if ((now - partitionRequestListener.getCreateTimestamp()) > timeout) { + timeoutListeners.add(partitionRequestListener); + iterator.remove(); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java index 223052681c024..7657bf538b883 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java @@ -18,14 +18,26 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.concurrent.ScheduledExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + import java.io.IOException; import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkState; @@ -40,9 +52,36 @@ public class ResultPartitionManager implements ResultPartitionProvider { private final Map registeredPartitions = CollectionUtil.newHashMapWithExpectedSize(16); + @GuardedBy("registeredPartitions") + private final Map listenerManagers = + new HashMap<>(); + + @Nullable private ScheduledFuture partitionListenerTimeoutChecker; + + private final int partitionListenerTimeout; + private boolean isShutdown; - public void registerResultPartition(ResultPartition partition) { + @VisibleForTesting + public ResultPartitionManager() { + this(0, null); + } + + public ResultPartitionManager( + int partitionListenerTimeout, ScheduledExecutor scheduledExecutor) { + this.partitionListenerTimeout = partitionListenerTimeout; + if (partitionListenerTimeout > 0 && scheduledExecutor != null) { + this.partitionListenerTimeoutChecker = + scheduledExecutor.scheduleWithFixedDelay( + this::checkRequestPartitionListeners, + partitionListenerTimeout, + partitionListenerTimeout, + TimeUnit.MILLISECONDS); + } + } + + public void registerResultPartition(ResultPartition partition) throws IOException { + PartitionRequestListenerManager listenerManager; synchronized (registeredPartitions) { checkState(!isShutdown, "Result partition manager already shut down."); @@ -53,8 +92,16 @@ public void registerResultPartition(ResultPartition partition) { throw new IllegalStateException("Result partition already registered."); } - LOG.debug("Registered {}.", partition); + listenerManager = listenerManagers.remove(partition.getPartitionId()); + } + if (listenerManager != null) { + for (PartitionRequestListener listener : + listenerManager.getPartitionRequestListeners()) { + listener.notifyPartitionCreated(partition); + } } + + LOG.debug("Registered {}.", partition); } @Override @@ -81,7 +128,51 @@ public ResultSubpartitionView createSubpartitionView( return subpartitionView; } + @Override + public Optional createSubpartitionViewOrRegisterListener( + ResultPartitionID partitionId, + int subpartitionIndex, + BufferAvailabilityListener availabilityListener, + PartitionRequestListener partitionRequestListener) + throws IOException { + + final ResultSubpartitionView subpartitionView; + synchronized (registeredPartitions) { + final ResultPartition partition = registeredPartitions.get(partitionId); + + if (partition == null) { + listenerManagers + .computeIfAbsent(partitionId, key -> new PartitionRequestListenerManager()) + .registerListener(partitionRequestListener); + subpartitionView = null; + } else { + + LOG.debug("Requesting subpartition {} of {}.", subpartitionIndex, partition); + + subpartitionView = + partition.createSubpartitionView(subpartitionIndex, availabilityListener); + } + } + + return subpartitionView == null ? Optional.empty() : Optional.of(subpartitionView); + } + + @Override + public void releasePartitionRequestListener(PartitionRequestListener listener) { + synchronized (registeredPartitions) { + PartitionRequestListenerManager listenerManager = + listenerManagers.get(listener.getResultPartitionId()); + if (listenerManager != null) { + listenerManager.remove(listener.getReceiverId()); + if (listenerManager.isEmpty()) { + listenerManagers.remove(listener.getResultPartitionId()); + } + } + } + } + public void releasePartition(ResultPartitionID partitionId, Throwable cause) { + PartitionRequestListenerManager listenerManager; synchronized (registeredPartitions) { ResultPartition resultPartition = registeredPartitions.remove(partitionId); if (resultPartition != null) { @@ -91,6 +182,13 @@ public void releasePartition(ResultPartitionID partitionId, Throwable cause) { partitionId.getPartitionId(), partitionId.getProducerId()); } + listenerManager = listenerManagers.remove(partitionId); + } + if (listenerManager != null && !listenerManager.isEmpty()) { + for (PartitionRequestListener listener : + listenerManager.getPartitionRequestListeners()) { + listener.notifyPartitionCreatedTimeout(); + } } } @@ -106,12 +204,61 @@ public void shutdown() { registeredPartitions.clear(); + releaseListenerManagers(); + + // stop the timeout checks for the TaskManagers + if (partitionListenerTimeoutChecker != null) { + partitionListenerTimeoutChecker.cancel(false); + partitionListenerTimeoutChecker = null; + } + isShutdown = true; LOG.debug("Successful shutdown."); } } + private void releaseListenerManagers() { + for (PartitionRequestListenerManager listenerManager : listenerManagers.values()) { + for (PartitionRequestListener listener : + listenerManager.getPartitionRequestListeners()) { + listener.notifyPartitionCreatedTimeout(); + } + } + listenerManagers.clear(); + } + + /** Check whether the partition request listener is timeout. */ + private void checkRequestPartitionListeners() { + List timeoutPartitionRequestListeners = new LinkedList<>(); + synchronized (registeredPartitions) { + if (isShutdown) { + return; + } + long now = System.currentTimeMillis(); + Iterator> iterator = + listenerManagers.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = + iterator.next(); + PartitionRequestListenerManager partitionRequestListeners = entry.getValue(); + partitionRequestListeners.removeExpiration( + now, partitionListenerTimeout, timeoutPartitionRequestListeners); + if (partitionRequestListeners.isEmpty()) { + iterator.remove(); + } + } + } + for (PartitionRequestListener partitionRequestListener : timeoutPartitionRequestListeners) { + partitionRequestListener.notifyPartitionCreatedTimeout(); + } + } + + @VisibleForTesting + public Map getListenerManagers() { + return listenerManagers; + } + // ------------------------------------------------------------------------ // Notifications // ------------------------------------------------------------------------ @@ -131,6 +278,12 @@ void onConsumedPartition(ResultPartition partition) { partitionId.getPartitionId(), partitionId.getProducerId()); } + PartitionRequestListenerManager listenerManager = + listenerManagers.remove(partition.getPartitionId()); + checkState( + listenerManager == null || listenerManager.isEmpty(), + "The partition request listeners is not empty for " + + partition.getPartitionId()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java index 0cd323a3c66d4..b95cc6f8c968d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import java.io.IOException; +import java.util.Optional; /** Interface for creating result partitions. */ public interface ResultPartitionProvider { @@ -29,4 +30,29 @@ ResultSubpartitionView createSubpartitionView( int index, BufferAvailabilityListener availabilityListener) throws IOException; + + /** + * If the upstream task's partition has been registered, returns the result subpartition input + * view immediately, otherwise register the listener and return empty. + * + * @param partitionId the result partition id + * @param index the index + * @param availabilityListener the buffer availability listener + * @param partitionRequestListener the partition request listener + * @return the result subpartition view + * @throws IOException the thrown exception + */ + Optional createSubpartitionViewOrRegisterListener( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener, + PartitionRequestListener partitionRequestListener) + throws IOException; + + /** + * Release the given listener in this result partition provider. + * + * @param listener the given listener + */ + void releasePartitionRequestListener(PartitionRequestListener listener); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 1d5a444e2c4e7..f36c5d95aec56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -77,7 +77,7 @@ public abstract class InputChannel { protected final Counter numBuffersIn; /** The current backoff (in ms). */ - private int currentBackoff; + protected int currentBackoff; protected InputChannel( SingleInputGate inputGate, @@ -105,7 +105,7 @@ protected InputChannel( this.initialBackoff = initial; this.maxBackoff = max; - this.currentBackoff = initial == 0 ? -1 : 0; + this.currentBackoff = 0; this.numBytesIn = numBytesIn; this.numBuffersIn = numBuffersIn; @@ -277,12 +277,12 @@ protected int getCurrentBackoff() { */ protected boolean increaseBackoff() { // Backoff is disabled - if (currentBackoff < 0) { + if (initialBackoff == 0) { return false; } - // This is the first time backing off if (currentBackoff == 0) { + // This is the first time backing off currentBackoff = initialBackoff; return true; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index b4fdc3e3c108c..2559e02ea8c92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -100,6 +100,9 @@ public class RemoteInputChannel extends InputChannel { /** The initial number of exclusive buffers assigned to this channel. */ private final int initialCredit; + /** The milliseconds timeout for partition request listener in result partition manager. */ + private final int partitionRequestListenerTimeout; + /** The number of available buffers that have not been announced to the producer yet. */ private final AtomicInteger unannouncedCredit = new AtomicInteger(0); @@ -124,6 +127,7 @@ public RemoteInputChannel( ConnectionManager connectionManager, int initialBackOff, int maxBackoff, + int partitionRequestListenerTimeout, int networkBuffersPerChannel, Counter numBytesIn, Counter numBuffersIn, @@ -140,6 +144,7 @@ public RemoteInputChannel( numBuffersIn); checkArgument(networkBuffersPerChannel >= 0, "Must be non-negative."); + this.partitionRequestListenerTimeout = partitionRequestListenerTimeout; this.initialCredit = networkBuffersPerChannel; this.connectionId = checkNotNull(connectionId); this.connectionManager = checkNotNull(connectionManager); @@ -201,12 +206,30 @@ void retriggerSubpartitionRequest() throws IOException { if (increaseBackoff()) { partitionRequestClient.requestSubpartition( - partitionId, consumedSubpartitionIndex, this, getCurrentBackoff()); + partitionId, consumedSubpartitionIndex, this, 0); } else { failPartitionRequest(); } } + /** + * The remote task manager creates partition request listener and returns {@link + * PartitionNotFoundException} until the listener is timeout, so the backoff should add the + * timeout milliseconds if it exists. + * + * @return true, iff the operation was successful. Otherwise, false. + */ + @Override + protected boolean increaseBackoff() { + if (partitionRequestListenerTimeout > 0) { + currentBackoff += partitionRequestListenerTimeout; + return currentBackoff < 2 * maxBackoff; + } + + // Backoff is disabled + return false; + } + @Override public Optional getNextBuffer() throws IOException { checkPartitionRequestQueueInitialized(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java index 2c0efa01592ab..f63ab7564f6a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java @@ -34,6 +34,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { private final ConnectionID connectionId; private final ConnectionManager connectionManager; + private final int partitionRequestListenerTimeout; RemoteRecoveredInputChannel( SingleInputGate inputGate, @@ -44,6 +45,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { ConnectionManager connectionManager, int initialBackOff, int maxBackoff, + int partitionRequestListenerTimeout, int networkBuffersPerChannel, InputChannelMetrics metrics) { super( @@ -59,6 +61,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { this.connectionId = checkNotNull(connectionId); this.connectionManager = checkNotNull(connectionManager); + this.partitionRequestListenerTimeout = partitionRequestListenerTimeout; } @Override @@ -73,6 +76,7 @@ protected InputChannel toInputChannelInternal() throws IOException { connectionManager, initialBackoff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, numBytesIn, numBuffersIn, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index de35b31bff1f8..38f6d13ebfa6b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -79,6 +79,8 @@ public class SingleInputGateFactory { protected final int partitionRequestMaxBackoff; + protected final int partitionRequestListenerTimeout; + @Nonnull protected final ConnectionManager connectionManager; @Nonnull protected final ResultPartitionManager partitionManager; @@ -118,6 +120,7 @@ public SingleInputGateFactory( this.taskExecutorResourceId = taskExecutorResourceId; this.partitionRequestInitialBackoff = networkConfig.partitionRequestInitialBackoff(); this.partitionRequestMaxBackoff = networkConfig.partitionRequestMaxBackoff(); + this.partitionRequestListenerTimeout = networkConfig.getPartitionRequestListenerTimeout(); this.maxRequiredBuffersPerGate = networkConfig.maxRequiredBuffersPerGate(); this.configuredNetworkBuffersPerChannel = NettyShuffleUtils.getNetworkBuffersPerInputChannel( @@ -307,6 +310,7 @@ private InputChannel createInputChannel( connectionManager, partitionRequestInitialBackoff, partitionRequestMaxBackoff, + partitionRequestListenerTimeout, buffersPerChannel, metrics); }, @@ -363,6 +367,7 @@ protected InputChannel createKnownInputChannel( connectionManager, partitionRequestInitialBackoff, partitionRequestMaxBackoff, + partitionRequestListenerTimeout, buffersPerChannel, metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index 3be987b770797..af3043f0f2d5f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -56,6 +56,8 @@ class UnknownInputChannel extends InputChannel implements ChannelStateHolder { private final int maxBackoff; + private final int partitionRequestListenerTimeout; + private final int networkBuffersPerChannel; private final InputChannelMetrics metrics; @@ -72,6 +74,7 @@ public UnknownInputChannel( ConnectionManager connectionManager, int initialBackoff, int maxBackoff, + int partitionRequestListenerTimeout, int networkBuffersPerChannel, InputChannelMetrics metrics) { @@ -91,6 +94,7 @@ public UnknownInputChannel( this.metrics = checkNotNull(metrics); this.initialBackoff = initialBackoff; this.maxBackoff = maxBackoff; + this.partitionRequestListenerTimeout = partitionRequestListenerTimeout; this.networkBuffersPerChannel = networkBuffersPerChannel; } @@ -168,6 +172,7 @@ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) { connectionManager, initialBackoff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, metrics.getNumBytesInRemoteCounter(), metrics.getNumBuffersInRemoteCounter(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java index c50fcb0c0f89b..6024f60afdb66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java @@ -23,6 +23,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.io.network.TaskEventPublisher; +import org.apache.flink.util.concurrent.ScheduledExecutor; import java.net.InetAddress; import java.util.concurrent.Executor; @@ -42,6 +43,7 @@ public class ShuffleEnvironmentContext { private final String[] tmpDirPaths; private final Executor ioExecutor; + private final ScheduledExecutor scheduledExecutor; public ShuffleEnvironmentContext( Configuration configuration, @@ -53,7 +55,8 @@ public ShuffleEnvironmentContext( String[] tmpDirPaths, TaskEventPublisher eventPublisher, MetricGroup parentMetricGroup, - Executor ioExecutor) { + Executor ioExecutor, + ScheduledExecutor scheduledExecutor) { this.configuration = checkNotNull(configuration); this.taskExecutorResourceId = checkNotNull(taskExecutorResourceId); this.networkMemorySize = networkMemorySize; @@ -62,6 +65,7 @@ public ShuffleEnvironmentContext( this.eventPublisher = checkNotNull(eventPublisher); this.parentMetricGroup = checkNotNull(parentMetricGroup); this.ioExecutor = ioExecutor; + this.scheduledExecutor = scheduledExecutor; this.numberOfSlots = numberOfSlots; this.tmpDirPaths = checkNotNull(tmpDirPaths); } @@ -98,6 +102,10 @@ public Executor getIoExecutor() { return ioExecutor; } + public ScheduledExecutor getScheduledExecutor() { + return scheduledExecutor; + } + public int getNumberOfSlots() { return numberOfSlots; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 258c5ca9be2e0..f1ec64ce45194 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -638,6 +638,7 @@ public static TaskExecutor startTaskManager( taskExecutorBlobService.getPermanentBlobService(), taskManagerMetricGroup.f1, ioExecutor, + rpcService.getScheduledExecutor(), fatalErrorHandler, workingDirectory); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index da8a063618944..f4b206421aa34 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -51,6 +51,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.ScheduledExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -293,6 +294,7 @@ public void shutDown() throws FlinkException { * @param permanentBlobService permanentBlobService used by the services * @param taskManagerMetricGroup metric group of the task manager * @param ioExecutor executor for async IO operations + * @param scheduledExecutor scheduled executor in rpc service * @param fatalErrorHandler to handle class loading OOMs * @param workingDirectory the working directory of the process * @return task manager components @@ -303,6 +305,7 @@ public static TaskManagerServices fromConfiguration( PermanentBlobService permanentBlobService, MetricGroup taskManagerMetricGroup, ExecutorService ioExecutor, + ScheduledExecutor scheduledExecutor, FatalErrorHandler fatalErrorHandler, WorkingDirectory workingDirectory) throws Exception { @@ -321,7 +324,8 @@ public static TaskManagerServices fromConfiguration( taskManagerServicesConfiguration, taskEventDispatcher, taskManagerMetricGroup, - ioExecutor); + ioExecutor, + scheduledExecutor); final int listeningDataPort = shuffleEnvironment.start(); LOG.info( @@ -454,7 +458,8 @@ private static TaskSlotTable createTaskSlotTable( TaskManagerServicesConfiguration taskManagerServicesConfiguration, TaskEventDispatcher taskEventDispatcher, MetricGroup taskManagerMetricGroup, - Executor ioExecutor) + Executor ioExecutor, + ScheduledExecutor scheduledExecutor) throws FlinkException { final ShuffleEnvironmentContext shuffleEnvironmentContext = @@ -468,7 +473,8 @@ private static TaskSlotTable createTaskSlotTable( taskManagerServicesConfiguration.getTmpDirPaths(), taskEventDispatcher, taskManagerMetricGroup, - ioExecutor); + ioExecutor, + scheduledExecutor); return ShuffleServiceLoader.loadShuffleServiceFactory( taskManagerServicesConfiguration.getConfiguration()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java index ecd9c04471ae6..54a75156ecb4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java @@ -65,6 +65,8 @@ public class NettyShuffleEnvironmentConfiguration { private final int partitionRequestMaxBackoff; + private final int partitionRequestListenerTimeout; + /** * Number of network buffers to use for each outgoing/incoming channel (subpartition/input * channel). @@ -122,6 +124,7 @@ public NettyShuffleEnvironmentConfiguration( int networkBufferSize, int partitionRequestInitialBackoff, int partitionRequestMaxBackoff, + int partitionRequestListenerTimeout, int networkBuffersPerChannel, int floatingNetworkBuffersPerGate, Optional maxRequiredBuffersPerGate, @@ -148,6 +151,7 @@ public NettyShuffleEnvironmentConfiguration( this.networkBufferSize = networkBufferSize; this.partitionRequestInitialBackoff = partitionRequestInitialBackoff; this.partitionRequestMaxBackoff = partitionRequestMaxBackoff; + this.partitionRequestListenerTimeout = partitionRequestListenerTimeout; this.networkBuffersPerChannel = networkBuffersPerChannel; this.floatingNetworkBuffersPerGate = floatingNetworkBuffersPerGate; this.maxRequiredBuffersPerGate = maxRequiredBuffersPerGate; @@ -190,6 +194,10 @@ public int partitionRequestMaxBackoff() { return partitionRequestMaxBackoff; } + public int getPartitionRequestListenerTimeout() { + return partitionRequestListenerTimeout; + } + public int networkBuffersPerChannel() { return networkBuffersPerChannel; } @@ -317,6 +325,13 @@ public static NettyShuffleEnvironmentConfiguration fromConfiguration( int maxRequestBackoff = configuration.getInteger( NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX); + int listenerTimeout = + (int) + configuration + .get( + NettyShuffleEnvironmentOptions + .NETWORK_PARTITION_REQUEST_TIMEOUT) + .toMillis(); int buffersPerChannel = configuration.getInteger( @@ -419,6 +434,7 @@ public static NettyShuffleEnvironmentConfiguration fromConfiguration( pageSize, initialRequestBackoff, maxRequestBackoff, + listenerTimeout, buffersPerChannel, extraBuffersPerGate, maxRequiredBuffersPerGate, @@ -561,6 +577,7 @@ public int hashCode() { result = 31 * result + networkBufferSize; result = 31 * result + partitionRequestInitialBackoff; result = 31 * result + partitionRequestMaxBackoff; + result = 31 * result + partitionRequestListenerTimeout; result = 31 * result + networkBuffersPerChannel; result = 31 * result + floatingNetworkBuffersPerGate; result = 31 * result + requestSegmentsTimeout.hashCode(); @@ -604,6 +621,7 @@ public boolean equals(Object obj) { && Arrays.equals(this.tempDirs, that.tempDirs) && this.batchShuffleCompressionEnabled == that.batchShuffleCompressionEnabled && this.maxBuffersPerChannel == that.maxBuffersPerChannel + && this.partitionRequestListenerTimeout == that.partitionRequestListenerTimeout && Objects.equals(this.compressionCodec, that.compressionCodec) && this.maxNumberOfConnections == that.maxNumberOfConnections && this.connectionReuseEnabled == that.connectionReuseEnabled @@ -638,6 +656,8 @@ public String toString() { + compressionCodec + ", maxBuffersPerChannel=" + maxBuffersPerChannel + + ", partitionRequestListenerTimeout" + + partitionRequestListenerTimeout + ", batchShuffleReadMemoryBytes=" + batchShuffleReadMemoryBytes + ", sortShuffleMinBuffers=" diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java index 23e01a929b573..1a8f21d7c131c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.io.network.netty.NettyConfig; @@ -55,6 +56,12 @@ public class NettyShuffleEnvironmentBuilder { private int partitionRequestMaxBackoff; + private int partitionRequestTimeout = + (int) + NettyShuffleEnvironmentOptions.NETWORK_PARTITION_REQUEST_TIMEOUT + .defaultValue() + .toMillis(); + private int networkBuffersPerChannel = 2; private int floatingNetworkBuffersPerGate = 8; @@ -125,6 +132,11 @@ public NettyShuffleEnvironmentBuilder setPartitionRequestMaxBackoff( return this; } + public NettyShuffleEnvironmentBuilder setPartitionRequestTimeout(int partitionRequestTimeout) { + this.partitionRequestTimeout = partitionRequestTimeout; + return this; + } + public NettyShuffleEnvironmentBuilder setNetworkBuffersPerChannel( int networkBuffersPerChannel) { this.networkBuffersPerChannel = networkBuffersPerChannel; @@ -246,6 +258,7 @@ public NettyShuffleEnvironment build() { bufferSize, partitionRequestInitialBackoff, partitionRequestMaxBackoff, + partitionRequestTimeout, networkBuffersPerChannel, floatingNetworkBuffersPerGate, maxRequiredBuffersPerGate, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java index 7bc7e9914caa2..7bfa0d28f8501 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; @@ -34,12 +35,12 @@ import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import javax.annotation.Nullable; import java.io.IOException; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -83,20 +84,14 @@ public void testCancelPartitionRequest() throws Exception { spy(new InfiniteSubpartitionView(outboundBuffers, sync)); // Return infinite subpartition - when(partitions.createSubpartitionView( - eq(pid), eq(0), any(BufferAvailabilityListener.class))) + when(partitions.createSubpartitionViewOrRegisterListener( + eq(pid), + eq(0), + any(BufferAvailabilityListener.class), + any(PartitionRequestListener.class))) .thenAnswer( - new Answer() { - @Override - public ResultSubpartitionView answer( - InvocationOnMock invocationOnMock) throws Throwable { - BufferAvailabilityListener listener = - (BufferAvailabilityListener) - invocationOnMock.getArguments()[2]; - listener.notifyDataAvailable(); - return view; - } - }); + (Answer>) + invocationOnMock -> Optional.of(view)); NettyProtocol protocol = new NettyProtocol(partitions, mock(TaskEventDispatcher.class)); @@ -140,20 +135,20 @@ public void testDuplicateCancel() throws Exception { spy(new InfiniteSubpartitionView(outboundBuffers, sync)); // Return infinite subpartition - when(partitions.createSubpartitionView( - eq(pid), eq(0), any(BufferAvailabilityListener.class))) + when(partitions.createSubpartitionViewOrRegisterListener( + eq(pid), + eq(0), + any(BufferAvailabilityListener.class), + any(PartitionRequestListener.class))) .thenAnswer( - new Answer() { - @Override - public ResultSubpartitionView answer( - InvocationOnMock invocationOnMock) throws Throwable { - BufferAvailabilityListener listener = - (BufferAvailabilityListener) - invocationOnMock.getArguments()[2]; - listener.notifyDataAvailable(); - return view; - } - }); + (Answer>) + invocationOnMock -> { + BufferAvailabilityListener listener = + (BufferAvailabilityListener) + invocationOnMock.getArguments()[2]; + listener.notifyDataAvailable(); + return Optional.of(view); + }); NettyProtocol protocol = new NettyProtocol(partitions, mock(TaskEventDispatcher.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index 6b39979fac828..f46c25ff5a2d7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -822,6 +822,7 @@ private static class TestRemoteInputChannelForError extends RemoteInputChannel { new TestingConnectionManager(), 0, 100, + 100, 2, new SimpleCounter(), new SimpleCounter(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java index 33b903ff68899..e8a6c70704443 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.netty; import org.apache.flink.runtime.io.network.partition.NoOpResultSubpartitionView; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.TestingResultPartition; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; @@ -83,9 +83,11 @@ private CreditBasedSequenceNumberingViewReader createNetworkSequenceViewReader( CreditBasedSequenceNumberingViewReader reader = new CreditBasedSequenceNumberingViewReader( new InputChannelID(), initialCredit, queue); - reader.requestSubpartitionView( - (ignored1, ignored2, ignored3) -> new NoOpResultSubpartitionView(), - new ResultPartitionID(), + reader.notifySubpartitionCreated( + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction( + (index, listener) -> new NoOpResultSubpartitionView()) + .build(), 0); return reader; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index ad34bdf363e85..ccfef5f2442df 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -107,7 +107,7 @@ public void testRetriggerPartitionRequest() throws Exception { InputChannelBuilder.newBuilder() .setConnectionManager( mockConnectionManagerWithPartitionRequestClient(client)) - .setInitialBackoff(1) + .setPartitionRequestListenerTimeout(1) .setMaxBackoff(2) .buildRemoteChannel(inputGate); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java index 34b949742db62..1a16710a8119f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java @@ -28,16 +28,17 @@ import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; import org.apache.flink.runtime.io.network.partition.NoOpBufferAvailablityListener; import org.apache.flink.runtime.io.network.partition.NoOpResultSubpartitionView; +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.PipelinedSubpartition; import org.apache.flink.runtime.io.network.partition.PipelinedSubpartitionTest; import org.apache.flink.runtime.io.network.partition.PipelinedSubpartitionView; import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.io.network.partition.TestingResultPartition; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.runtime.io.network.util.TestBufferFactory; @@ -90,6 +91,44 @@ public static void shutdown() throws Exception { fileChannelManager.close(); } + /** Test that PartitionNotFound message will be sent to downstream in notifying timeout. */ + @Test + public void testNotifyReaderPartitionTimeout() throws Exception { + PartitionRequestQueue queue = new PartitionRequestQueue(); + EmbeddedChannel channel = new EmbeddedChannel(queue); + ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); + ResultPartitionID resultPartitionId = new ResultPartitionID(); + CreditBasedSequenceNumberingViewReader reader = + new CreditBasedSequenceNumberingViewReader(new InputChannelID(0, 0), 10, queue); + reader.requestSubpartitionViewOrRegisterListener( + resultPartitionManager, resultPartitionId, 0); + + assertEquals( + resultPartitionManager + .getListenerManagers() + .get(resultPartitionId) + .getPartitionRequestListeners() + .size(), + 1); + + reader.notifyPartitionRequestTimeout( + resultPartitionManager + .getListenerManagers() + .get(resultPartitionId) + .getPartitionRequestListeners() + .iterator() + .next()); + + channel.runPendingTasks(); + + Object read = channel.readOutbound(); + assertNotNull(read); + assertThat(read, instanceOf(NettyMessage.ErrorResponse.class)); + assertThat( + ((NettyMessage.ErrorResponse) read).cause, + instanceOf(PartitionNotFoundException.class)); + } + /** * In case of enqueuing an empty reader and a reader that actually has some buffers when channel * is not writable, on channelWritability change event should result in reading all of the @@ -106,10 +145,12 @@ public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { CreditBasedSequenceNumberingViewReader reader2 = new CreditBasedSequenceNumberingViewReader(new InputChannelID(1, 1), 10, queue); - reader1.requestSubpartitionView( - (partitionId, index, availabilityListener) -> - new EmptyAlwaysAvailableResultSubpartitionView(), - new ResultPartitionID(), + reader1.notifySubpartitionCreated( + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction( + (index, listener) -> + new EmptyAlwaysAvailableResultSubpartitionView()) + .build(), 0); reader1.notifyDataAvailable(); assertTrue(reader1.getAvailabilityAndBacklog().isAvailable()); @@ -122,10 +163,12 @@ public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { channel.runPendingTasks(); reader2.notifyDataAvailable(); - reader2.requestSubpartitionView( - (partitionId, index, availabilityListener) -> - new DefaultBufferResultSubpartitionView(buffersToWrite), - new ResultPartitionID(), + reader2.notifySubpartitionCreated( + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction( + (index, listener) -> + new DefaultBufferResultSubpartitionView(buffersToWrite)) + .build(), 0); assertTrue(reader2.getAvailabilityAndBacklog().isAvailable()); assertFalse(reader2.isRegisteredAsAvailable()); @@ -152,8 +195,10 @@ public void testReadOnlyBufferWriting() throws Exception { private void testBufferWriting(ResultSubpartitionView view) throws IOException { // setup - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); final InputChannelID receiverId = new InputChannelID(); final PartitionRequestQueue queue = new PartitionRequestQueue(); @@ -161,7 +206,7 @@ private void testBufferWriting(ResultSubpartitionView view) throws IOException { new CreditBasedSequenceNumberingViewReader(receiverId, Integer.MAX_VALUE, queue); final EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); // notify about buffer availability and encode one buffer reader.notifyDataAvailable(); @@ -246,8 +291,10 @@ public void testEnqueueReaderByNotifyingEventBuffer() throws Exception { // setup final ResultSubpartitionView view = new NextIsEventResultSubpartitionView(); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); final InputChannelID receiverId = new InputChannelID(); final PartitionRequestQueue queue = new PartitionRequestQueue(); @@ -255,7 +302,7 @@ public void testEnqueueReaderByNotifyingEventBuffer() throws Exception { new CreditBasedSequenceNumberingViewReader(receiverId, 0, queue); final EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); // block the channel so that we see an intermediate state in the test ByteBuf channelBlockingBuffer = blockChannel(channel); @@ -297,8 +344,10 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { // setup final ResultSubpartitionView view = new DefaultBufferResultSubpartitionView(10); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); final InputChannelID receiverId = new InputChannelID(); final PartitionRequestQueue queue = new PartitionRequestQueue(); @@ -307,7 +356,7 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { final EmbeddedChannel channel = new EmbeddedChannel(queue); reader.addCredit(-2); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); queue.notifyReaderCreated(reader); // block the channel so that we see an intermediate state in the test @@ -375,8 +424,10 @@ public void testEnqueueReaderByResumingConsumption() throws Exception { BufferAvailabilityListener bufferAvailabilityListener = new NoOpBufferAvailablityListener(); PipelinedSubpartitionView view = subpartition.createReadView(bufferAvailabilityListener); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); InputChannelID receiverId = new InputChannelID(); PartitionRequestQueue queue = new PartitionRequestQueue(); @@ -384,7 +435,7 @@ public void testEnqueueReaderByResumingConsumption() throws Exception { new CreditBasedSequenceNumberingViewReader(receiverId, 2, queue); EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); queue.notifyReaderCreated(reader); assertTrue(reader.getAvailabilityAndBacklog().isAvailable()); @@ -413,8 +464,10 @@ public void testAnnounceBacklog() throws Exception { PipelinedSubpartitionView view = subpartition.createReadView(new NoOpBufferAvailablityListener()); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); PartitionRequestQueue queue = new PartitionRequestQueue(); InputChannelID receiverId = new InputChannelID(); @@ -422,7 +475,7 @@ public void testAnnounceBacklog() throws Exception { new CreditBasedSequenceNumberingViewReader(receiverId, 0, queue); EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); queue.notifyReaderCreated(reader); reader.notifyDataAvailable(); @@ -459,7 +512,7 @@ private void testCancelPartitionRequest(boolean isAvailableView) throws Exceptio new CreditBasedSequenceNumberingViewReader(receiverId, 2, queue); final EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionManager, partition.getPartitionId(), 0); + reader.notifySubpartitionCreated(partition, 0); // add this reader into allReaders queue queue.notifyReaderCreated(reader); @@ -494,8 +547,10 @@ public void testNotifyNewBufferSize() throws Exception { BufferAvailabilityListener bufferAvailabilityListener = new NoOpBufferAvailablityListener(); ResultSubpartitionView view = parent.createSubpartitionView(0, bufferAvailabilityListener); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> view; + ResultPartition partition = + TestingResultPartition.newBuilder() + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); InputChannelID receiverId = new InputChannelID(); PartitionRequestQueue queue = new PartitionRequestQueue(); @@ -503,7 +558,7 @@ public void testNotifyNewBufferSize() throws Exception { new CreditBasedSequenceNumberingViewReader(receiverId, 2, queue); EmbeddedChannel channel = new EmbeddedChannel(queue); - reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); + reader.notifySubpartitionCreated(partition, 0); queue.notifyReaderCreated(reader); // when: New buffer size received. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestRegistrationTest.java new file mode 100644 index 0000000000000..3f98c8701b9b0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestRegistrationTest.java @@ -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. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; +import org.apache.flink.runtime.event.TaskEvent; +import org.apache.flink.runtime.io.network.NetworkClientHandler; +import org.apache.flink.runtime.io.network.TaskEventPublisher; +import org.apache.flink.runtime.io.network.TestingConnectionManager; +import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.io.network.partition.TestingResultPartition; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelBuilder; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder; +import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; +import org.apache.flink.testutils.TestingUtils; + +import org.apache.flink.shaded.netty4.io.netty.channel.Channel; + +import org.junit.jupiter.api.Test; + +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.connect; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.initServerAndClient; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.shutdown; +import static org.assertj.core.api.Assertions.fail; + +/** + * Tests {@link NettyMessage.PartitionRequest} before and after {@link ResultPartitionManager} + * registers given {@link ResultPartition}. + */ +class PartitionRequestRegistrationTest { + + /** + * Verifies that result partition manager registers partition before receive partition request. + */ + @Test + void testRegisterResultPartitionBeforeRequest() throws Exception { + final TestPooledBufferProvider outboundBuffers = new TestPooledBufferProvider(16); + final CountDownLatch sync = new CountDownLatch(1); + final ResultSubpartitionView view = + new CancelPartitionRequestTest.InfiniteSubpartitionView(outboundBuffers, sync); + + ResultPartitionManager partitionManager = new ResultPartitionManager(); + ResultPartition resultPartition = + TestingResultPartition.newBuilder() + .setResultPartitionManager(partitionManager) + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); + + // Register result partition before request + partitionManager.registerResultPartition(resultPartition); + + NettyTestUtil.NettyServerAndClient serverAndClient = null; + try { + + NettyProtocol protocol = + new NettyProtocol(partitionManager, new NoOpTaskEventPublisher()); + + serverAndClient = initServerAndClient(protocol); + + Channel ch = connect(serverAndClient); + + // Request for non-existing input channel => results in cancel request + ch.writeAndFlush( + new NettyMessage.PartitionRequest( + resultPartition.getPartitionId(), + 0, + new InputChannelID(), + Integer.MAX_VALUE)) + .await(); + + // Wait for the notification + if (!sync.await(TestingUtils.TESTING_DURATION.toMillis(), TimeUnit.MILLISECONDS)) { + fail( + "Timed out after waiting for " + + TestingUtils.TESTING_DURATION.toMillis() + + " ms to be notified about cancelled partition."); + } + } finally { + shutdown(serverAndClient); + } + } + + /** + * Verifies that result partition manager registers partition after receive partition request. + */ + @Test + void testRegisterResultPartitionAfterRequest() throws Exception { + final TestPooledBufferProvider outboundBuffers = new TestPooledBufferProvider(16); + final CountDownLatch sync = new CountDownLatch(1); + final ResultSubpartitionView view = + new CancelPartitionRequestTest.InfiniteSubpartitionView(outboundBuffers, sync); + + ResultPartitionManager partitionManager = new ResultPartitionManager(); + ResultPartition resultPartition = + TestingResultPartition.newBuilder() + .setResultPartitionManager(partitionManager) + .setCreateSubpartitionViewFunction((index, listener) -> view) + .build(); + + NettyTestUtil.NettyServerAndClient serverAndClient = null; + try { + + NettyProtocol protocol = + new NettyProtocol(partitionManager, new NoOpTaskEventPublisher()); + + serverAndClient = initServerAndClient(protocol); + + Channel ch = connect(serverAndClient); + + // Request for non-existing input channel => results in cancel request + ch.writeAndFlush( + new NettyMessage.PartitionRequest( + resultPartition.getPartitionId(), + 0, + new InputChannelID(), + Integer.MAX_VALUE)) + .await(); + + // Register result partition after partition request + partitionManager.registerResultPartition(resultPartition); + + // Wait for the notification + if (!sync.await(TestingUtils.TESTING_DURATION.toMillis(), TimeUnit.MILLISECONDS)) { + fail( + "Timed out after waiting for " + + TestingUtils.TESTING_DURATION.toMillis() + + " ms to be notified about cancelled partition."); + } + } finally { + shutdown(serverAndClient); + } + } + + /** Verifies that result partition manager notifier timeout. */ + @Test + void testPartitionRequestNotifierTimeout() throws Exception { + final ResultPartitionID pid = new ResultPartitionID(); + final CountDownLatch sync = new CountDownLatch(1); + + NettyTestUtil.NettyServerAndClient serverAndClient = null; + try { + ResultPartitionProvider partitions = + new ResultPartitionProvider() { + @Override + public ResultSubpartitionView createSubpartitionView( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener) { + return null; + } + + @Override + public Optional + createSubpartitionViewOrRegisterListener( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener, + PartitionRequestListener partitionRequestListener) { + partitionRequestListener.notifyPartitionCreatedTimeout(); + return Optional.empty(); + } + + @Override + public void releasePartitionRequestListener( + PartitionRequestListener listener) {} + }; + + NettyProtocol protocol = new NettyProtocol(partitions, new NoOpTaskEventPublisher()); + + serverAndClient = initServerAndClient(protocol); + + Channel ch = connect(serverAndClient); + + NetworkClientHandler clientHandler = ch.pipeline().get(NetworkClientHandler.class); + RemoteInputChannel remoteInputChannel = + new TestRemoteInputChannelForPartitionNotFound(sync); + clientHandler.addInputChannel(remoteInputChannel); + + // Request for non-existing input channel => results in cancel request + ch.writeAndFlush( + new NettyMessage.PartitionRequest( + pid, + 0, + remoteInputChannel.getInputChannelId(), + Integer.MAX_VALUE)) + .await(); + + // Wait for the notification + if (!sync.await(TestingUtils.TESTING_DURATION.toMillis(), TimeUnit.MILLISECONDS)) { + fail( + "Timed out after waiting for " + + TestingUtils.TESTING_DURATION.toMillis() + + " ms to be notified about cancelled partition."); + } + } finally { + shutdown(serverAndClient); + } + } + + /** + * The test remote input channel to count down the latch when it receives partition not found + * exception. + */ + private static class TestRemoteInputChannelForPartitionNotFound extends RemoteInputChannel { + private final CountDownLatch latch; + + TestRemoteInputChannelForPartitionNotFound(CountDownLatch latch) { + super( + new SingleInputGateBuilder().setNumberOfChannels(1).build(), + 0, + new ResultPartitionID(), + 0, + InputChannelBuilder.STUB_CONNECTION_ID, + new TestingConnectionManager(), + 0, + 100, + 10000, + 2, + new SimpleCounter(), + new SimpleCounter(), + ChannelStateWriter.NO_OP); + this.latch = latch; + } + + @Override + public void onFailedPartitionRequest() { + latch.countDown(); + } + } + + /** A testing implementation of {@link TaskEventPublisher} without operation. */ + private static class NoOpTaskEventPublisher implements TaskEventPublisher { + @Override + public boolean publish(ResultPartitionID partitionId, TaskEvent event) { + return true; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java index 67cf02554a759..d7941ef352df1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java @@ -21,15 +21,10 @@ import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; -import org.apache.flink.runtime.io.network.netty.NettyMessage.PartitionRequest; import org.apache.flink.runtime.io.network.netty.NettyMessage.ResumeConsumption; -import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.PartitionTestUtils; import org.apache.flink.runtime.io.network.partition.ResultPartition; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.util.TestLogger; @@ -41,46 +36,13 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** Tests for {@link PartitionRequestServerHandler}. */ public class PartitionRequestServerHandlerTest extends TestLogger { - /** - * Tests that {@link PartitionRequestServerHandler} responds {@link ErrorResponse} with wrapped - * {@link PartitionNotFoundException} after receiving invalid {@link PartitionRequest}. - */ - @Test - public void testResponsePartitionNotFoundException() { - final PartitionRequestServerHandler serverHandler = - new PartitionRequestServerHandler( - new ResultPartitionManager(), - new TaskEventDispatcher(), - new PartitionRequestQueue()); - final EmbeddedChannel channel = new EmbeddedChannel(serverHandler); - final ResultPartitionID partitionId = new ResultPartitionID(); - - // Write the message of partition request to server - channel.writeInbound(new PartitionRequest(partitionId, 0, new InputChannelID(), 2)); - channel.runPendingTasks(); - - // Read the response message after handling partition request - final Object msg = channel.readOutbound(); - assertThat(msg, instanceOf(ErrorResponse.class)); - - final ErrorResponse err = (ErrorResponse) msg; - assertThat(err.cause, instanceOf(PartitionNotFoundException.class)); - - final ResultPartitionID actualPartitionId = - ((PartitionNotFoundException) err.cause).getPartitionId(); - assertThat(partitionId, is(actualPartitionId)); - } - @Test public void testResumeConsumption() { final InputChannelID inputChannelID = new InputChannelID(); @@ -108,9 +70,6 @@ public void testAcknowledgeAllRecordsProcessed() throws IOException { ResultPartition resultPartition = PartitionTestUtils.createPartition(ResultPartitionType.PIPELINED_BOUNDED); - ResultPartitionProvider partitionProvider = - (partitionId, index, availabilityListener) -> - resultPartition.createSubpartitionView(index, availabilityListener); // Creates the netty network handler stack. PartitionRequestQueue partitionRequestQueue = new PartitionRequestQueue(); @@ -125,7 +84,7 @@ public void testAcknowledgeAllRecordsProcessed() throws IOException { NetworkSequenceViewReader viewReader = new CreditBasedSequenceNumberingViewReader( inputChannelID, 2, partitionRequestQueue); - viewReader.requestSubpartitionView(partitionProvider, resultPartition.getPartitionId(), 0); + viewReader.notifySubpartitionCreated(resultPartition, 0); partitionRequestQueue.notifyReaderCreated(viewReader); // Write the message to acknowledge all records are processed to server diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java index c8faf422fc088..78bfa73e600da 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.io.network.netty; import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.netty.CancelPartitionRequestTest.InfiniteSubpartitionView; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; +import org.apache.flink.runtime.io.network.partition.PartitionRequestListener; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; @@ -36,6 +36,7 @@ import org.junit.Test; import org.mockito.stubbing.Answer; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -59,19 +60,18 @@ public void testRemoteClose() throws Exception { final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); - when(partitionManager.createSubpartitionView( + when(partitionManager.createSubpartitionViewOrRegisterListener( any(ResultPartitionID.class), anyInt(), - any(BufferAvailabilityListener.class))) + any(BufferAvailabilityListener.class), + any(PartitionRequestListener.class))) .thenAnswer( - (Answer) - invocationOnMock -> { - BufferAvailabilityListener listener = - (BufferAvailabilityListener) - invocationOnMock.getArguments()[2]; - listener.notifyDataAvailable(); - return new InfiniteSubpartitionView(outboundBuffers, sync); - }); + (Answer>) + invocationOnMock -> + Optional.of( + new CancelPartitionRequestTest + .InfiniteSubpartitionView( + outboundBuffers, sync))); NettyProtocol protocol = new NettyProtocol(partitionManager, mock(TaskEventDispatcher.class)) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManagerTest.java new file mode 100644 index 0000000000000..40acdcb98be6f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManagerTest.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.io.network.netty.NettyPartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test case for {@link PartitionRequestListenerManager}. */ +class PartitionRequestListenerManagerTest extends TestLogger { + /** Test add listener to {@link PartitionRequestListenerManager}. */ + @Test + void testAddListener() { + PartitionRequestListenerManager partitionRequestListenerManager = + new PartitionRequestListenerManager(); + assertThat(partitionRequestListenerManager.isEmpty()).isTrue(); + + List listenerList = new ArrayList<>(); + NettyPartitionRequestListener listener1 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 0, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener1); + listenerList.add(listener1); + + NettyPartitionRequestListener listener2 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 1, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener2); + listenerList.add(listener2); + + NettyPartitionRequestListener listener3 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 2, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener3); + listenerList.add(listener3); + + assertThat(partitionRequestListenerManager.getPartitionRequestListeners()) + .hasSize(listenerList.size()); + assertThat(listenerList) + .containsAll(partitionRequestListenerManager.getPartitionRequestListeners()); + } + + /** + * Test remove listener from {@link PartitionRequestListenerManager} by {@link InputChannelID}. + */ + @Test + void testRemoveListener() { + PartitionRequestListenerManager partitionRequestListenerManager = + new PartitionRequestListenerManager(); + assertThat(partitionRequestListenerManager.isEmpty()).isTrue(); + + List listenerList = new ArrayList<>(); + NettyPartitionRequestListener listener1 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 0, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener1); + + NettyPartitionRequestListener listener2 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 1, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener2); + listenerList.add(listener2); + + NettyPartitionRequestListener listener3 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 2, + new ResultPartitionID()); + partitionRequestListenerManager.registerListener(listener3); + listenerList.add(listener3); + + partitionRequestListenerManager.remove(listener1.getReceiverId()); + assertThat(partitionRequestListenerManager.getPartitionRequestListeners()) + .hasSize(listenerList.size()); + assertThat(listenerList) + .containsAll(partitionRequestListenerManager.getPartitionRequestListeners()); + } + + /** Test remove expire listeners from {@link PartitionRequestListenerManager}. */ + @Test + void testRemoveExpiration() { + PartitionRequestListenerManager partitionRequestListenerManager = + new PartitionRequestListenerManager(); + assertThat(partitionRequestListenerManager.isEmpty()).isTrue(); + + List listenerList = new ArrayList<>(); + List expireListenerList = new ArrayList<>(); + NettyPartitionRequestListener listener1 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 0, + new ResultPartitionID(), + 0L); + partitionRequestListenerManager.registerListener(listener1); + expireListenerList.add(listener1); + + NettyPartitionRequestListener listener2 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 1, + new ResultPartitionID(), + 0L); + partitionRequestListenerManager.registerListener(listener2); + expireListenerList.add(listener2); + + long currentTimestamp = System.currentTimeMillis(); + NettyPartitionRequestListener listener3 = + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .build(), + 2, + new ResultPartitionID(), + currentTimestamp); + partitionRequestListenerManager.registerListener(listener3); + listenerList.add(listener3); + + List removeExpireListenerList = new ArrayList<>(); + partitionRequestListenerManager.removeExpiration( + currentTimestamp, 1L, removeExpireListenerList); + + assertThat(partitionRequestListenerManager.getPartitionRequestListeners()) + .hasSize(listenerList.size()); + assertThat(listenerList) + .containsAll(partitionRequestListenerManager.getPartitionRequestListeners()); + + assertThat(removeExpireListenerList).hasSize(expireListenerList.size()); + assertThat(expireListenerList).containsAll(removeExpireListenerList); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java index ed169f9322a75..8c3c67815ac21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java @@ -32,7 +32,7 @@ public class PipelinedResultPartitionReleaseOnConsumptionTest extends TestLogger { @Test - public void testConsumptionBasedPartitionRelease() { + public void testConsumptionBasedPartitionRelease() throws IOException { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() @@ -70,7 +70,7 @@ public void testConsumptionBeforePartitionClose() throws IOException { } @Test - public void testMultipleReleaseCallsAreIdempotent() { + public void testMultipleReleaseCallsAreIdempotent() throws IOException { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() @@ -87,7 +87,7 @@ public void testMultipleReleaseCallsAreIdempotent() { } @Test - public void testReleaseAfterIdempotentCalls() { + public void testReleaseAfterIdempotentCalls() throws IOException { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java index 8e3b2263fbee6..d8b2a051f2dfe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java @@ -38,6 +38,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import java.io.IOException; import java.util.Optional; import java.util.concurrent.Executors; @@ -64,7 +65,7 @@ static void shutdown() throws Exception { } @Test - void testBoundedBlockingSubpartitionsCreated() { + void testBoundedBlockingSubpartitionsCreated() throws IOException { final BoundedBlockingResultPartition resultPartition = (BoundedBlockingResultPartition) createResultPartition(ResultPartitionType.BLOCKING); @@ -73,7 +74,7 @@ void testBoundedBlockingSubpartitionsCreated() { } @Test - void testPipelinedSubpartitionsCreated() { + void testPipelinedSubpartitionsCreated() throws IOException { final PipelinedResultPartition resultPartition = (PipelinedResultPartition) createResultPartition(ResultPartitionType.PIPELINED); assertThat(resultPartition.subpartitions) @@ -81,26 +82,26 @@ void testPipelinedSubpartitionsCreated() { } @Test - void testSortMergePartitionCreated() { + void testSortMergePartitionCreated() throws IOException { ResultPartition resultPartition = createResultPartition(ResultPartitionType.BLOCKING, 1); assertThat(resultPartition).isInstanceOf(SortMergeResultPartition.class); } @Test - void testHybridFullResultPartitionOfLegacyModeCreated() { + void testHybridFullResultPartitionOfLegacyModeCreated() throws IOException { ResultPartition resultPartition = createResultPartition(ResultPartitionType.HYBRID_FULL); assertThat(resultPartition).isInstanceOf(HsResultPartition.class); } @Test - void testHybridSelectiveResultPartitionOfLegacyModeCreated() { + void testHybridSelectiveResultPartitionOfLegacyModeCreated() throws IOException { ResultPartition resultPartition = createResultPartition(ResultPartitionType.HYBRID_SELECTIVE); assertThat(resultPartition).isInstanceOf(HsResultPartition.class); } @Test - void testHybridFullResultPartitionOfNewModeCreated() { + void testHybridFullResultPartitionOfNewModeCreated() throws IOException { ResultPartition resultPartition = createResultPartition( ResultPartitionType.HYBRID_FULL, createTieredResultPartitionFactory()); @@ -108,7 +109,7 @@ void testHybridFullResultPartitionOfNewModeCreated() { } @Test - void testHybridSelectiveResultPartitionOfNewModeCreated() { + void testHybridSelectiveResultPartitionOfNewModeCreated() throws IOException { ResultPartition resultPartition = createResultPartition( ResultPartitionType.HYBRID_SELECTIVE, createTieredResultPartitionFactory()); @@ -116,7 +117,7 @@ void testHybridSelectiveResultPartitionOfNewModeCreated() { } @Test - void testNoReleaseOnConsumptionForBoundedBlockingPartition() { + void testNoReleaseOnConsumptionForBoundedBlockingPartition() throws IOException { final ResultPartition resultPartition = createResultPartition(ResultPartitionType.BLOCKING); resultPartition.onConsumedSubpartition(0); @@ -125,7 +126,7 @@ void testNoReleaseOnConsumptionForBoundedBlockingPartition() { } @Test - void testNoReleaseOnConsumptionForSortMergePartition() { + void testNoReleaseOnConsumptionForSortMergePartition() throws IOException { final ResultPartition resultPartition = createResultPartition(ResultPartitionType.BLOCKING, 1); @@ -135,7 +136,7 @@ void testNoReleaseOnConsumptionForSortMergePartition() { } @Test - void testNoReleaseOnConsumptionForHybridFullPartitionOfLegacyMode() { + void testNoReleaseOnConsumptionForHybridFullPartitionOfLegacyMode() throws IOException { final ResultPartition resultPartition = createResultPartition(ResultPartitionType.HYBRID_FULL); @@ -145,7 +146,7 @@ void testNoReleaseOnConsumptionForHybridFullPartitionOfLegacyMode() { } @Test - void testNoReleaseOnConsumptionForHybridSelectivePartitionOfLegacyMode() { + void testNoReleaseOnConsumptionForHybridSelectivePartitionOfLegacyMode() throws IOException { final ResultPartition resultPartition = createResultPartition(ResultPartitionType.HYBRID_SELECTIVE); @@ -155,7 +156,7 @@ void testNoReleaseOnConsumptionForHybridSelectivePartitionOfLegacyMode() { } @Test - void testNoReleaseOnConsumptionForHybridFullPartitionOfNewMode() { + void testNoReleaseOnConsumptionForHybridFullPartitionOfNewMode() throws IOException { ResultPartition resultPartition = createResultPartition( ResultPartitionType.HYBRID_FULL, createTieredResultPartitionFactory()); @@ -166,7 +167,7 @@ void testNoReleaseOnConsumptionForHybridFullPartitionOfNewMode() { } @Test - void testNoReleaseOnConsumptionForHybridSelectivePartitionOfNewMode() { + void testNoReleaseOnConsumptionForHybridSelectivePartitionOfNewMode() throws IOException { ResultPartition resultPartition = createResultPartition( ResultPartitionType.HYBRID_SELECTIVE, createTieredResultPartitionFactory()); @@ -176,18 +177,19 @@ void testNoReleaseOnConsumptionForHybridSelectivePartitionOfNewMode() { assertThat(resultPartition.isReleased()).isFalse(); } - private static ResultPartition createResultPartition(ResultPartitionType partitionType) { + private static ResultPartition createResultPartition(ResultPartitionType partitionType) + throws IOException { return createResultPartition(partitionType, Integer.MAX_VALUE, false, Optional.empty()); } private static ResultPartition createResultPartition( - ResultPartitionType partitionType, - Optional tieredStorage) { + ResultPartitionType partitionType, Optional tieredStorage) + throws IOException { return createResultPartition(partitionType, Integer.MAX_VALUE, false, tieredStorage); } private static ResultPartition createResultPartition( - ResultPartitionType partitionType, int sortShuffleMinParallelism) { + ResultPartitionType partitionType, int sortShuffleMinParallelism) throws IOException { return createResultPartition( partitionType, sortShuffleMinParallelism, false, Optional.empty()); } @@ -196,7 +198,8 @@ private static ResultPartition createResultPartition( ResultPartitionType partitionType, int sortShuffleMinParallelism, boolean isBroadcast, - Optional tieredStorage) { + Optional tieredStorage) + throws IOException { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartitionFactory factory = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java index b9e1dc4a6ae2b..04171937552ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java @@ -18,15 +18,22 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.io.network.netty.NettyPartitionRequestListener; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.verifyCreateSubpartitionViewThrowsException; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link ResultPartitionManager}. */ -public class ResultPartitionManagerTest extends TestLogger { +class ResultPartitionManagerTest extends TestLogger { /** * Tests that {@link ResultPartitionManager#createSubpartitionView(ResultPartitionID, int, @@ -34,7 +41,7 @@ public class ResultPartitionManagerTest extends TestLogger { * was not registered before. */ @Test - public void testThrowPartitionNotFoundException() throws Exception { + void testThrowPartitionNotFoundException() { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); @@ -46,7 +53,7 @@ public void testThrowPartitionNotFoundException() throws Exception { * BufferAvailabilityListener)} successful if this partition was already registered before. */ @Test - public void testCreateViewForRegisteredPartition() throws Exception { + void testCreateViewForRegisteredPartition() throws Exception { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); @@ -55,13 +62,91 @@ public void testCreateViewForRegisteredPartition() throws Exception { partition.getPartitionId(), 0, new NoOpBufferAvailablityListener()); } + /** + * {@link ResultPartitionManager} creates subpartition view reader after the given partition is + * registered. + */ + @Test + void testCreateSubpartitionViewAfterRegisteredPartition() throws Exception { + final ResultPartitionManager partitionManager = new ResultPartitionManager(); + final ResultPartition partition = createPartition(); + + assertThat(partitionManager.getListenerManagers().isEmpty()).isTrue(); + + partitionManager.registerResultPartition(partition); + PartitionRequestListener partitionRequestListener = + TestingPartitionRequestListener.newBuilder().build(); + assertThat( + partitionManager.createSubpartitionViewOrRegisterListener( + partition.getPartitionId(), + 0, + new NoOpBufferAvailablityListener(), + partitionRequestListener)) + .isPresent(); + assertThat(partitionManager.getListenerManagers().isEmpty()).isTrue(); + } + + /** + * The {@link ResultPartitionManager} registers {@link PartitionRequestListener} before specify + * {@link ResultPartition} is registered. When the {@link ResultPartition} is registered, the + * {@link ResultPartitionManager} will find the listener and create partition view reader. an + */ + @Test + void testRegisterPartitionListenerBeforeRegisteredPartition() throws Exception { + final ResultPartitionManager partitionManager = new ResultPartitionManager(); + final ResultPartition partition = createPartition(); + + assertThat(partitionManager.getListenerManagers().isEmpty()).isTrue(); + + final CompletableFuture notifySubpartitionCreatedFuture = + new CompletableFuture<>(); + PartitionRequestListener partitionRequestListener = + TestingPartitionRequestListener.newBuilder() + .setResultPartitionId(partition.getPartitionId()) + .setNetworkSequenceViewReader( + TestingSubpartitionCreatedViewReader.newBuilder() + .setNotifySubpartitionCreatedConsumer( + tuple -> + notifySubpartitionCreatedFuture.complete( + tuple.f0)) + .build()) + .build(); + assertThat( + partitionManager.createSubpartitionViewOrRegisterListener( + partition.getPartitionId(), + 0, + new NoOpBufferAvailablityListener(), + partitionRequestListener)) + .isNotPresent(); + assertThat(partitionManager.getListenerManagers()).hasSize(1); + + // Check if the partition request listener is registered. + PartitionRequestListenerManager listenerManager = + partitionManager.getListenerManagers().get(partition.getPartitionId()); + assertThat(listenerManager).isNotNull(); + assertThat(listenerManager.isEmpty()).isFalse(); + assertThat(listenerManager.getPartitionRequestListeners()).hasSize(1); + PartitionRequestListener listener = + listenerManager.getPartitionRequestListeners().iterator().next(); + assertThat(listener.getResultPartitionId()).isEqualTo(partition.getPartitionId()); + assertThat(notifySubpartitionCreatedFuture).isNotDone(); + + partitionManager.registerResultPartition(partition); + + // Check if the listener is notified. + ResultPartition notifyPartition = + notifySubpartitionCreatedFuture.get(10, TimeUnit.MILLISECONDS); + assertThat(partition.getPartitionId()).isEqualTo(notifyPartition.getPartitionId()); + assertThat(partitionManager.getListenerManagers().isEmpty()).isTrue(); + } + /** * Tests {@link ResultPartitionManager#createSubpartitionView(ResultPartitionID, int, * BufferAvailabilityListener)} would throw a {@link PartitionNotFoundException} if this * partition was already released before. */ @Test - public void testCreateViewForReleasedPartition() throws Exception { + void testCreateViewForReleasedPartition() throws Exception { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); @@ -70,4 +155,55 @@ public void testCreateViewForReleasedPartition() throws Exception { verifyCreateSubpartitionViewThrowsException(partitionManager, partition.getPartitionId()); } + + /** Test notifier timeout in {@link ResultPartitionManager}. */ + @Test + void testCreateViewReaderForNotifierTimeout() throws Exception { + ManuallyTriggeredScheduledExecutor scheduledExecutor = + new ManuallyTriggeredScheduledExecutor(); + final ResultPartitionManager partitionManager = + new ResultPartitionManager(1000000, scheduledExecutor); + final ResultPartition partition1 = createPartition(); + final ResultPartition partition2 = createPartition(); + + CompletableFuture timeoutFuture1 = new CompletableFuture<>(); + CompletableFuture timeoutFuture2 = new CompletableFuture<>(); + partitionManager.createSubpartitionViewOrRegisterListener( + partition1.getPartitionId(), + 0, + new NoOpBufferAvailablityListener(), + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .setPartitionRequestListenerTimeoutConsumer( + timeoutFuture1::complete) + .build(), + 0, + partition1.getPartitionId(), + 0L)); + partitionManager.createSubpartitionViewOrRegisterListener( + partition2.getPartitionId(), + 0, + new NoOpBufferAvailablityListener(), + new NettyPartitionRequestListener( + TestingResultPartitionProvider.newBuilder().build(), + TestingSubpartitionCreatedViewReader.newBuilder() + .setReceiverId(new InputChannelID()) + .setPartitionRequestListenerTimeoutConsumer( + timeoutFuture2::complete) + .build(), + 0, + partition2.getPartitionId())); + scheduledExecutor.triggerScheduledTasks(); + + assertThat(timeoutFuture1.isDone()).isTrue(); + assertThat(partition1.getPartitionId()) + .isEqualTo(timeoutFuture1.get().getResultPartitionId()); + assertThat(timeoutFuture2.isDone()).isFalse(); + assertThat(partitionManager.getListenerManagers().get(partition1.getPartitionId())) + .isNull(); + assertThat(partitionManager.getListenerManagers().get(partition2.getPartitionId())) + .isNotNull(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionRequestListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionRequestListener.java new file mode 100644 index 0000000000000..e8341e09f7d3e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionRequestListener.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; + +import java.io.IOException; + +/** {@link PartitionRequestListener} implementation for testing purposes. */ +public class TestingPartitionRequestListener implements PartitionRequestListener { + private final long createTimestamp; + private final ResultPartitionID resultPartitionId; + private final InputChannelID inputChannelId; + private final NetworkSequenceViewReader reader; + + private TestingPartitionRequestListener( + ResultPartitionID resultPartitionId, + InputChannelID inputChannelId, + NetworkSequenceViewReader reader) { + this.createTimestamp = System.currentTimeMillis(); + this.resultPartitionId = resultPartitionId; + this.inputChannelId = inputChannelId; + this.reader = reader; + } + + @Override + public long getCreateTimestamp() { + return createTimestamp; + } + + @Override + public ResultPartitionID getResultPartitionId() { + return resultPartitionId; + } + + @Override + public NetworkSequenceViewReader getViewReader() { + return reader; + } + + @Override + public InputChannelID getReceiverId() { + return inputChannelId; + } + + @Override + public void notifyPartitionCreated(ResultPartition partition) throws IOException { + reader.notifySubpartitionCreated(partition, 0); + } + + @Override + public void notifyPartitionCreatedTimeout() {} + + @Override + public void releaseListener() {} + + public static TestingPartitionRequestListenerBuilder newBuilder() { + return new TestingPartitionRequestListenerBuilder(); + } + + /** Factory for {@link TestingPartitionRequestListener}. */ + public static class TestingPartitionRequestListenerBuilder { + private ResultPartitionID resultPartitionId = new ResultPartitionID(); + private InputChannelID inputChannelId = new InputChannelID(); + private NetworkSequenceViewReader reader = null; + + public TestingPartitionRequestListenerBuilder setResultPartitionId( + ResultPartitionID resultPartitionId) { + this.resultPartitionId = resultPartitionId; + return this; + } + + public TestingPartitionRequestListenerBuilder setInputChannelId( + InputChannelID inputChannelId) { + this.inputChannelId = inputChannelId; + return this; + } + + public TestingPartitionRequestListenerBuilder setNetworkSequenceViewReader( + NetworkSequenceViewReader reader) { + this.reader = reader; + return this; + } + + public TestingPartitionRequestListener build() { + return new TestingPartitionRequestListener(resultPartitionId, inputChannelId, reader); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartition.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartition.java new file mode 100644 index 0000000000000..d29491739e346 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartition.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.event.AbstractEvent; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** {@link ResultPartition} class for testing purposes. */ +public class TestingResultPartition extends ResultPartition { + private final CreateSubpartitionView createSubpartitionViewFunction; + + public TestingResultPartition( + ResultPartitionID partitionId, + ResultPartitionManager partitionManager, + CreateSubpartitionView createSubpartitionViewFunction) { + super( + "test", + 0, + partitionId, + ResultPartitionType.PIPELINED, + 0, + 0, + partitionManager, + null, + null); + this.createSubpartitionViewFunction = createSubpartitionViewFunction; + } + + @Override + public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {} + + @Override + public void broadcastRecord(ByteBuffer record) throws IOException {} + + @Override + public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {} + + @Override + public void alignedBarrierTimeout(long checkpointId) throws IOException {} + + @Override + public void abortCheckpoint(long checkpointId, CheckpointException cause) {} + + @Override + public ResultSubpartitionView createSubpartitionView( + int index, BufferAvailabilityListener availabilityListener) throws IOException { + return createSubpartitionViewFunction.createSubpartitionView(index, availabilityListener); + } + + @Override + public void flushAll() {} + + @Override + public void flush(int subpartitionIndex) {} + + @Override + protected void setupInternal() throws IOException {} + + @Override + public int getNumberOfQueuedBuffers() { + return 0; + } + + @Override + public long getSizeOfQueuedBuffersUnsafe() { + return 0; + } + + @Override + public int getNumberOfQueuedBuffers(int targetSubpartition) { + return 0; + } + + @Override + protected void releaseInternal() {} + + public static TestingResultPartitionBuilder newBuilder() { + return new TestingResultPartitionBuilder(); + } + + /** Factory for {@link TestingResultPartition}. */ + public static class TestingResultPartitionBuilder { + private ResultPartitionID resultPartitionId = new ResultPartitionID(); + private ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); + private CreateSubpartitionView createSubpartitionViewFunction = + (index, availabilityListener) -> null; + + public TestingResultPartitionBuilder setCreateSubpartitionViewFunction( + CreateSubpartitionView createSubpartitionViewFunction) { + this.createSubpartitionViewFunction = createSubpartitionViewFunction; + return this; + } + + public TestingResultPartitionBuilder setResultPartitionID( + ResultPartitionID resultPartitionId) { + this.resultPartitionId = resultPartitionId; + return this; + } + + public TestingResultPartitionBuilder setResultPartitionManager( + ResultPartitionManager resultPartitionManager) { + this.resultPartitionManager = resultPartitionManager; + return this; + } + + public TestingResultPartition build() { + return new TestingResultPartition( + resultPartitionId, resultPartitionManager, createSubpartitionViewFunction); + } + } + + /** Testing interface for createSubpartitionView. */ + public interface CreateSubpartitionView { + ResultSubpartitionView createSubpartitionView( + int index, BufferAvailabilityListener availabilityListener) throws IOException; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartitionProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartitionProvider.java new file mode 100644 index 0000000000000..d5f10ad599f39 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingResultPartitionProvider.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import java.io.IOException; +import java.util.Optional; + +/** {@link ResultPartitionProvider} implementation for testing purposes. */ +public class TestingResultPartitionProvider implements ResultPartitionProvider { + private final CreateSubpartitionView createSubpartitionViewFunction; + private final CreateSubpartitionViewOrRegisterListener + createSubpartitionViewOrRegisterListenerFunction; + private final ReleasePartitionRequestListener releasePartitionRequestListenerConsumer; + + public TestingResultPartitionProvider( + CreateSubpartitionView createSubpartitionViewFunction, + CreateSubpartitionViewOrRegisterListener + createSubpartitionViewOrRegisterListenerFunction, + ReleasePartitionRequestListener releasePartitionRequestListenerConsumer) { + this.createSubpartitionViewFunction = createSubpartitionViewFunction; + this.createSubpartitionViewOrRegisterListenerFunction = + createSubpartitionViewOrRegisterListenerFunction; + this.releasePartitionRequestListenerConsumer = releasePartitionRequestListenerConsumer; + } + + @Override + public ResultSubpartitionView createSubpartitionView( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener) + throws IOException { + return createSubpartitionViewFunction.createSubpartitionView( + partitionId, index, availabilityListener); + } + + @Override + public Optional createSubpartitionViewOrRegisterListener( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener, + PartitionRequestListener notifier) + throws IOException { + return createSubpartitionViewOrRegisterListenerFunction + .createSubpartitionViewOrRegisterListener( + partitionId, index, availabilityListener, notifier); + } + + @Override + public void releasePartitionRequestListener(PartitionRequestListener notifier) { + releasePartitionRequestListenerConsumer.releasePartitionRequestListener(notifier); + } + + public static TestingResultPartitionProviderBuilder newBuilder() { + return new TestingResultPartitionProviderBuilder(); + } + + /** Factory for {@link TestingResultPartitionProvider}. */ + public static class TestingResultPartitionProviderBuilder { + private CreateSubpartitionView createSubpartitionViewFunction = + (resultPartitionID, index, availabilityListener) -> null; + private CreateSubpartitionViewOrRegisterListener + createSubpartitionViewOrRegisterListenerFunction = + (partitionId, index, availabilityListener, partitionRequestListener) -> + Optional.empty(); + private ReleasePartitionRequestListener releasePartitionRequestListenerConsumer = + listener -> {}; + + public TestingResultPartitionProviderBuilder setCreateSubpartitionViewFunction( + CreateSubpartitionView createSubpartitionViewFunction) { + this.createSubpartitionViewFunction = createSubpartitionViewFunction; + return this; + } + + public TestingResultPartitionProviderBuilder setCreateSubpartitionViewOrNotifyFunction( + CreateSubpartitionViewOrRegisterListener + createSubpartitionViewOrRegisterListenerFunction) { + this.createSubpartitionViewOrRegisterListenerFunction = + createSubpartitionViewOrRegisterListenerFunction; + return this; + } + + public TestingResultPartitionProviderBuilder setReleasePartitionRequestListenerConsumer( + ReleasePartitionRequestListener releasePartitionRequestListenerConsumer) { + this.releasePartitionRequestListenerConsumer = releasePartitionRequestListenerConsumer; + return this; + } + + public TestingResultPartitionProvider build() { + return new TestingResultPartitionProvider( + createSubpartitionViewFunction, + createSubpartitionViewOrRegisterListenerFunction, + releasePartitionRequestListenerConsumer); + } + } + + /** Testing interface for createSubpartitionView. */ + public interface CreateSubpartitionView { + ResultSubpartitionView createSubpartitionView( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener) + throws IOException; + } + + /** Testing interface for createSubpartitionViewOrRegisterListener. */ + public interface CreateSubpartitionViewOrRegisterListener { + Optional createSubpartitionViewOrRegisterListener( + ResultPartitionID partitionId, + int index, + BufferAvailabilityListener availabilityListener, + PartitionRequestListener partitionRequestListener) + throws IOException; + } + + /** Testing interface for releasePartitionRequestListener. */ + public interface ReleasePartitionRequestListener { + void releasePartitionRequestListener(PartitionRequestListener listener); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingSubpartitionCreatedViewReader.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingSubpartitionCreatedViewReader.java new file mode 100644 index 0000000000000..bc5334f642023 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingSubpartitionCreatedViewReader.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.runtime.io.network.partition; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.io.network.NetworkSequenceViewReader; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; + +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.util.function.Consumer; + +/** Testing view reader for partition request notifier. */ +public class TestingSubpartitionCreatedViewReader implements NetworkSequenceViewReader { + private final InputChannelID receiverId; + private final Consumer partitionRequestListenerTimeoutConsumer; + private final Consumer> notifySubpartitionCreatedConsumer; + + private TestingSubpartitionCreatedViewReader( + InputChannelID receiverId, + Consumer partitionRequestListenerTimeoutConsumer, + Consumer> notifySubpartitionCreatedConsumer) { + this.receiverId = receiverId; + this.partitionRequestListenerTimeoutConsumer = partitionRequestListenerTimeoutConsumer; + this.notifySubpartitionCreatedConsumer = notifySubpartitionCreatedConsumer; + } + + @Override + public void notifySubpartitionCreated(ResultPartition partition, int subPartitionIndex) + throws IOException { + notifySubpartitionCreatedConsumer.accept(Tuple2.of(partition, subPartitionIndex)); + } + + @Override + public void requestSubpartitionViewOrRegisterListener( + ResultPartitionProvider partitionProvider, + ResultPartitionID resultPartitionId, + int subPartitionIndex) + throws IOException { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public InputChannel.BufferAndAvailability getNextBuffer() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean needAnnounceBacklog() { + throw new UnsupportedOperationException(); + } + + @Override + public void addCredit(int creditDeltas) { + throw new UnsupportedOperationException(); + } + + @Override + public void notifyRequiredSegmentId(int segmentId) { + throw new UnsupportedOperationException(); + } + + @Override + public void resumeConsumption() { + throw new UnsupportedOperationException(); + } + + @Override + public void acknowledgeAllRecordsProcessed() { + throw new UnsupportedOperationException(); + } + + @Override + public ResultSubpartitionView.AvailabilityWithBacklog getAvailabilityAndBacklog() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isRegisteredAsAvailable() { + throw new UnsupportedOperationException(); + } + + @Override + public void setRegisteredAsAvailable(boolean isRegisteredAvailable) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isReleased() { + throw new UnsupportedOperationException(); + } + + @Override + public void releaseAllResources() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public Throwable getFailureCause() { + throw new UnsupportedOperationException(); + } + + @Override + public InputChannelID getReceiverId() { + return receiverId; + } + + @Override + public void notifyNewBufferSize(int newBufferSize) { + throw new UnsupportedOperationException(); + } + + @Override + public void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener) { + partitionRequestListenerTimeoutConsumer.accept(partitionRequestListener); + } + + public static TestingSubpartitionCreatedViewReaderBuilder newBuilder() { + return new TestingSubpartitionCreatedViewReaderBuilder(); + } + + /** Builder for {@link TestingSubpartitionCreatedViewReader}. */ + public static class TestingSubpartitionCreatedViewReaderBuilder { + private InputChannelID receiverId; + private Consumer partitionRequestListenerTimeoutConsumer = + listener -> {}; + private Consumer> notifySubpartitionCreatedConsumer = + tuple -> {}; + + public TestingSubpartitionCreatedViewReaderBuilder setReceiverId( + InputChannelID receiverId) { + this.receiverId = receiverId; + return this; + } + + public TestingSubpartitionCreatedViewReaderBuilder + setPartitionRequestListenerTimeoutConsumer( + Consumer + partitionRequestListenerTimeoutConsumer) { + this.partitionRequestListenerTimeoutConsumer = partitionRequestListenerTimeoutConsumer; + return this; + } + + public TestingSubpartitionCreatedViewReaderBuilder setNotifySubpartitionCreatedConsumer( + Consumer> notifySubpartitionCreatedConsumer) { + this.notifySubpartitionCreatedConsumer = notifySubpartitionCreatedConsumer; + return this; + } + + public TestingSubpartitionCreatedViewReader build() { + return new TestingSubpartitionCreatedViewReader( + receiverId, + partitionRequestListenerTimeoutConsumer, + notifySubpartitionCreatedConsumer); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index 622a1392a0394..3f85ff0650c11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -52,6 +52,7 @@ public class InputChannelBuilder { private ConnectionManager connectionManager = new TestingConnectionManager(); private int initialBackoff = 0; private int maxBackoff = 0; + private int partitionRequestListenerTimeout = 0; private int networkBuffersPerChannel = 2; private InputChannelMetrics metrics = InputChannelTestUtils.newUnregisteredInputChannelMetrics(); @@ -100,6 +101,12 @@ public InputChannelBuilder setMaxBackoff(int maxBackoff) { return this; } + public InputChannelBuilder setPartitionRequestListenerTimeout( + int partitionRequestListenerTimeout) { + this.partitionRequestListenerTimeout = partitionRequestListenerTimeout; + return this; + } + public InputChannelBuilder setNetworkBuffersPerChannel(int networkBuffersPerChannel) { this.networkBuffersPerChannel = networkBuffersPerChannel; return this; @@ -136,6 +143,7 @@ UnknownInputChannel buildUnknownChannel(SingleInputGate inputGate) { connectionManager, initialBackoff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, metrics); channel.setChannelStateWriter(stateWriter); @@ -167,6 +175,7 @@ public RemoteInputChannel buildRemoteChannel(SingleInputGate inputGate) { connectionManager, initialBackoff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, metrics.getNumBytesInRemoteCounter(), metrics.getNumBuffersInRemoteCounter(), @@ -201,6 +210,7 @@ public RemoteRecoveredInputChannel buildRemoteRecoveredChannel(SingleInputGate i connectionManager, initialBackoff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, metrics); channel.setChannelStateWriter(stateWriter); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 8b2e42e5544a9..65d0d2bdfe8b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -356,9 +356,10 @@ public void testRetriggerWithoutPartitionRequest() throws Exception { } @Test - public void testPartitionRequestExponentialBackoff() throws Exception { - // Start with initial backoff, then keep doubling, and cap at max. - int[] expectedDelays = {500, 1000, 2000, 3000}; + public void testPartitionRequestLinearBackoff() throws Exception { + // Start with initial backoff, then keep adding the partition request timeout, and cap at + // max. + int[] expectedDelays = {500, 1000, 1500, 2000}; // Setup SingleInputGate inputGate = createSingleInputGate(1); @@ -366,17 +367,16 @@ public void testPartitionRequestExponentialBackoff() throws Exception { TestVerifyPartitionRequestClient client = new TestVerifyPartitionRequestClient(); ConnectionManager connectionManager = new TestVerifyConnectionManager(client); RemoteInputChannel ch = - createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 3000); + createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 1000); // Initial request ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); - // Request subpartition and verify that the actual requests are delayed. + // Request subpartition and verify that the actual back off. for (int expected : expectedDelays) { ch.retriggerSubpartitionRequest(); - - client.verifyResult(partitionId, 0, expected); + assertEquals(expected, ch.getCurrentBackoff()); } // Exception after backoff is greater than the maximum backoff. @@ -402,9 +402,9 @@ public void testPartitionRequestSingleBackoff() throws Exception { ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); - // Initial delay for second request + // The current backoff for second request ch.retriggerSubpartitionRequest(); - client.verifyResult(partitionId, 0, 500); + assertEquals(500, ch.getCurrentBackoff()); // Exception after backoff is greater than the maximum backoff. try { @@ -1744,11 +1744,11 @@ private RemoteInputChannel createRemoteInputChannel( private RemoteInputChannel createRemoteInputChannel( SingleInputGate inputGate, int consumedSubpartitionIndex, - int initialBackoff, + int partitionRequestTimeout, int maxBackoff) { return InputChannelBuilder.newBuilder() .setConsumedSubpartitionIndex(consumedSubpartitionIndex) - .setInitialBackoff(initialBackoff) + .setPartitionRequestListenerTimeout(partitionRequestTimeout) .setMaxBackoff(maxBackoff) .buildRemoteChannel(inputGate); } @@ -1757,10 +1757,10 @@ private RemoteInputChannel createRemoteInputChannel( SingleInputGate inputGate, ConnectionManager connectionManager, ResultPartitionID partitionId, - int initialBackoff, + int partitionRequestTimeout, int maxBackoff) { return InputChannelBuilder.newBuilder() - .setInitialBackoff(initialBackoff) + .setPartitionRequestListenerTimeout(partitionRequestTimeout) .setMaxBackoff(maxBackoff) .setPartitionId(partitionId) .setConnectionManager(connectionManager) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 0f03b3d223d83..0e452f0dde770 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -635,11 +635,13 @@ void testRequestBackoffConfiguration() throws Exception { }; int initialBackoff = 137; + int partitionRequestTimeout = 600; int maxBackoff = 1001; final NettyShuffleEnvironment netEnv = new NettyShuffleEnvironmentBuilder() .setPartitionRequestInitialBackoff(initialBackoff) + .setPartitionRequestTimeout(partitionRequestTimeout) .setPartitionRequestMaxBackoff(maxBackoff) .build(); @@ -675,14 +677,10 @@ void testRequestBackoffConfiguration() throws Exception { InputChannel localChannel = channelMap.get(createSubpartitionInfo(partitionIds[0])); assertThat(localChannel.getClass()).isEqualTo(LocalInputChannel.class); - InputChannel remoteChannel = channelMap.get(createSubpartitionInfo(partitionIds[1])); - assertThat(remoteChannel.getClass()).isEqualTo(RemoteInputChannel.class); - InputChannel unknownChannel = channelMap.get(createSubpartitionInfo(partitionIds[2])); assertThat(unknownChannel.getClass()).isEqualTo(UnknownInputChannel.class); - InputChannel[] channels = - new InputChannel[] {localChannel, remoteChannel, unknownChannel}; + InputChannel[] channels = new InputChannel[] {localChannel, unknownChannel}; for (InputChannel ch : channels) { assertThat(ch.getCurrentBackoff()).isEqualTo(0); @@ -700,6 +698,22 @@ void testRequestBackoffConfiguration() throws Exception { assertThat(ch.increaseBackoff()).isFalse(); } + + InputChannel remoteChannel = channelMap.get(createSubpartitionInfo(partitionIds[1])); + assertThat(remoteChannel.getClass()).isEqualTo(RemoteInputChannel.class); + + assertThat(remoteChannel.getCurrentBackoff()).isEqualTo(0); + + assertThat(remoteChannel.increaseBackoff()).isTrue(); + assertThat(remoteChannel.getCurrentBackoff()).isEqualTo(partitionRequestTimeout); + + assertThat(remoteChannel.increaseBackoff()).isTrue(); + assertThat(remoteChannel.getCurrentBackoff()).isEqualTo(partitionRequestTimeout * 2); + + assertThat(remoteChannel.increaseBackoff()).isTrue(); + assertThat(remoteChannel.getCurrentBackoff()).isEqualTo(partitionRequestTimeout * 3); + + assertThat(remoteChannel.increaseBackoff()).isFalse(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java index 11a511831d585..c7a86387657eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java @@ -415,6 +415,7 @@ private TaskManagerServices createTaskManagerServices( VoidPermanentBlobService.INSTANCE, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), Executors.newDirectExecutorService(), + null, throwable -> {}, workingDirectory); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorBuilder.java index 8051e538dfddb..73f0b7d6e9a14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorBuilder.java @@ -139,6 +139,7 @@ public TaskExecutor build() throws Exception { VoidPermanentBlobService.INSTANCE, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), Executors.newDirectExecutorService(), + rpcService.getScheduledExecutor(), throwable -> {}, workingDirectory); } else { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index 7b152a9adc112..0cbf60279c593 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -309,6 +310,15 @@ private TestingTaskExecutor createTaskExecutor( configuration.getInteger( NettyShuffleEnvironmentOptions .NETWORK_REQUEST_BACKOFF_MAX)) + .setResultPartitionManager( + new ResultPartitionManager( + (int) + configuration + .get( + NettyShuffleEnvironmentOptions + .NETWORK_PARTITION_REQUEST_TIMEOUT) + .toMillis(), + testingRpcService.getScheduledExecutor())) .setNettyConfig(localCommunication ? null : nettyConfig) .build(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java index 49f56cbb840bc..1754184138797 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java @@ -92,6 +92,7 @@ protected InputChannel createKnownInputChannel( connectionManager, partitionRequestInitialBackoff, partitionRequestMaxBackoff, + partitionRequestListenerTimeout, configuredNetworkBuffersPerChannel, metrics); } @@ -165,6 +166,7 @@ public TestRemoteInputChannel( ConnectionManager connectionManager, int initialBackOff, int maxBackoff, + int partitionRequestListenerTimeout, int networkBuffersPerChannel, InputChannelMetrics metrics) { super( @@ -176,6 +178,7 @@ public TestRemoteInputChannel( connectionManager, initialBackOff, maxBackoff, + partitionRequestListenerTimeout, networkBuffersPerChannel, metrics.getNumBytesInRemoteCounter(), metrics.getNumBuffersInRemoteCounter(), From 29b5213aae0f3dbd108909881083dc114cfc15aa Mon Sep 17 00:00:00 2001 From: yuxiang <384669270@qq.com> Date: Tue, 31 Oct 2023 21:31:20 +0800 Subject: [PATCH 057/104] [FLINK-26624][runtime] Running HA (hashmap, async) end-to-end test failed on azure due to unable to find master logs (#23614) Co-authored-by: yu <13485876233> --- flink-end-to-end-tests/test-scripts/common_ha.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_ha.sh b/flink-end-to-end-tests/test-scripts/common_ha.sh index 23f89eea0bb42..a18bd15da46f8 100644 --- a/flink-end-to-end-tests/test-scripts/common_ha.sh +++ b/flink-end-to-end-tests/test-scripts/common_ha.sh @@ -49,7 +49,7 @@ function verify_num_occurences_in_logs() { local text="$2" local expected_no="$3" - local actual_no=$(grep -r --include "*${log_pattern}*.log*" -e "${text}" "$FLINK_LOG_DIR/" | cut -d ":" -f 1 | sed "s/\.[0-9]\{1,\}$//g" | uniq | wc -l) + local actual_no=$(grep -r --include "*${log_pattern}*.log*" -e "${text}" "$FLINK_LOG_DIR/" | cut -d ":" -f 1 | sed "s/\.[0-9]\{1,\}$//g" | sort -u | wc -l) [[ "${expected_no}" -eq "${actual_no}" ]] } From bfe6f19173fc89e552a04edcd24b508af6155d93 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 31 Oct 2023 12:13:22 +0100 Subject: [PATCH 058/104] [FLINK-25809][table] Improve readability of TableTestPrograms --- .../table/test/program/SinkTestStep.java | 60 +++++ .../table/test/program/SourceTestStep.java | 42 ++++ .../table/test/program/TableTestProgram.java | 208 ++---------------- .../table/test/program/TableTestStep.java | 61 +++++ .../exec/testutils/CalcTestPrograms.java | 24 +- .../program/TableTestProgramRunnerTest.java | 21 +- 6 files changed, 207 insertions(+), 209 deletions(-) diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java index d195d65c1440b..11ae1bb055d15 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java @@ -22,6 +22,7 @@ import javax.annotation.Nullable; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -58,6 +59,11 @@ public final class SinkTestStep extends TableTestStep { this.expectedAfterRestoreStrings = expectedAfterRestoreStrings; } + /** Builder for creating a {@link SinkTestStep}. */ + public static SinkTestStep.Builder newBuilder(String name) { + return new SinkTestStep.Builder(name); + } + public List getExpectedBeforeRestoreAsStrings() { if (expectedBeforeRestoreStrings != null) { return expectedBeforeRestoreStrings; @@ -90,4 +96,58 @@ public TestKind getKind() { ? TestKind.SINK_WITH_DATA : TestKind.SINK_WITH_RESTORE_DATA; } + + /** Builder pattern for {@link SinkTestStep}. */ + public static final class Builder extends AbstractBuilder { + + private List expectedBeforeRestore; + private List expectedAfterRestore; + + private List expectedBeforeRestoreStrings; + private List expectedAfterRestoreStrings; + + private Builder(String name) { + super(name); + } + + public Builder consumedValues(Row... expectedRows) { + return consumedBeforeRestore(expectedRows); + } + + public Builder consumedValues(String... expectedRows) { + return consumedBeforeRestore(expectedRows); + } + + public Builder consumedBeforeRestore(Row... expectedRows) { + this.expectedBeforeRestore = Arrays.asList(expectedRows); + return this; + } + + public Builder consumedBeforeRestore(String... expectedRows) { + this.expectedBeforeRestoreStrings = Arrays.asList(expectedRows); + return this; + } + + public Builder consumedAfterRestore(Row... expectedRows) { + this.expectedAfterRestore = Arrays.asList(expectedRows); + return this; + } + + public Builder consumedAfterRestore(String... expectedRows) { + this.expectedAfterRestoreStrings = Arrays.asList(expectedRows); + return this; + } + + public SinkTestStep build() { + return new SinkTestStep( + name, + schemaComponents, + partitionKeys, + options, + expectedBeforeRestore, + expectedAfterRestore, + expectedBeforeRestoreStrings, + expectedAfterRestoreStrings); + } + } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java index eec3b1677b010..6653eb174c5ec 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java @@ -20,6 +20,8 @@ import org.apache.flink.types.Row; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; @@ -41,6 +43,11 @@ public final class SourceTestStep extends TableTestStep { this.dataAfterRestore = dataAfterRestore; } + /** Builder for creating a {@link SourceTestStep}. */ + public static Builder newBuilder(String name) { + return new Builder(name); + } + @Override public TestKind getKind() { return dataBeforeRestore.isEmpty() @@ -49,4 +56,39 @@ public TestKind getKind() { ? TestKind.SOURCE_WITH_DATA : TestKind.SOURCE_WITH_RESTORE_DATA; } + + /** Builder pattern for {@link SourceTestStep}. */ + public static final class Builder extends AbstractBuilder { + + private final List dataBeforeRestore = new ArrayList<>(); + private final List dataAfterRestore = new ArrayList<>(); + + private Builder(String name) { + super(name); + } + + public Builder producedValues(Row... data) { + return producedBeforeRestore(data); + } + + public Builder producedBeforeRestore(Row... data) { + this.dataBeforeRestore.addAll(Arrays.asList(data)); + return this; + } + + public Builder producedAfterRestore(Row... data) { + this.dataAfterRestore.addAll(Arrays.asList(data)); + return this; + } + + public SourceTestStep build() { + return new SourceTestStep( + name, + schemaComponents, + partitionKeys, + options, + dataBeforeRestore, + dataAfterRestore); + } + } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java index efec060315d2f..731c967bba108 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -19,21 +19,16 @@ package org.apache.flink.table.test.program; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigurationUtils; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.test.program.FunctionTestStep.FunctionBehavior; import org.apache.flink.table.test.program.FunctionTestStep.FunctionPersistence; import org.apache.flink.table.test.program.TestStep.TestKind; -import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; /** @@ -250,205 +245,40 @@ public Builder setupCatalogFunction( return this; } - /** Setup step for building a table source. */ - public SourceBuilder setupTableSource(String name) { - return new SourceBuilder(name, setupSteps, this); - } - - /** Setup step for building a table sink. */ - public SinkBuilder setupTableSink(String name) { - return new SinkBuilder(name, setupSteps, this); - } - - /** Run step for executing SQL. */ - public Builder runSql(String sql) { - this.runSteps.add(new SqlTestStep(sql)); - return this; - } - - /** Run step for executing a statement set. */ - public StatementSetBuilder runStatementSet() { - return new StatementSetBuilder(runSteps, this); - } - - public TableTestProgram build() { - return new TableTestProgram(id, description, setupSteps, runSteps); - } - } - - /** Builder pattern for {@link SourceTestStep} and {@link SinkTestStep}. */ - @SuppressWarnings("unchecked") - private static class TableBuilder> { - - protected final String name; - protected final List targetSteps; - protected final Builder rootBuilder; - - protected final List schemaComponents = new ArrayList<>(); - protected final List partitionKeys = new ArrayList<>(); - protected final Map options = new HashMap<>(); - - private TableBuilder(String name, List targetSteps, Builder rootBuilder) { - this.name = name; - this.targetSteps = targetSteps; - this.rootBuilder = rootBuilder; - } - - /** - * Define the schema like you would in SQL e.g. "my_col INT", "PRIMARY KEY (uid) NOT - * ENFORCED", or "WATERMARK FOR ts AS ts". - */ - public SpecificBuilder withSchema(String... schemaComponents) { - this.schemaComponents.addAll(Arrays.asList(schemaComponents)); - return (SpecificBuilder) this; - } - /** - * Unless the test requires a very specific configuration, try to avoid calling this method - * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. - */ - public SpecificBuilder withOptions(Map options) { - this.options.putAll(options); - return (SpecificBuilder) this; - } - - /** - * Unless the test requires a very specific configuration, try to avoid calling this method - * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + * Setup step for a table source. + * + *

Use {@link SourceTestStep.Builder} to construct this step. */ - public SpecificBuilder withOption(String key, String value) { - this.options.put(key, value); - return (SpecificBuilder) this; + public Builder setupTableSource(SourceTestStep sourceTestStep) { + setupSteps.add(sourceTestStep); + return this; } /** - * Unless the test requires a very specific configuration, try to avoid calling this method - * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + * Setup step for a table sink. + * + *

Use {@link SinkTestStep.Builder} to construct this step. */ - public SpecificBuilder withOption(ConfigOption option, String value) { - this.options.put(option.key(), ConfigurationUtils.convertValue(value, String.class)); - return (SpecificBuilder) this; - } - - public SpecificBuilder withPartitionKeys(String... partitionKeys) { - this.partitionKeys.addAll(Arrays.asList(partitionKeys)); - return (SpecificBuilder) this; - } - } - - /** Builder pattern for {@link SourceTestStep}. */ - public static class SourceBuilder extends TableBuilder { - - private final List dataBeforeRestore = new ArrayList<>(); - private final List dataAfterRestore = new ArrayList<>(); - - private SourceBuilder(String name, List targetSteps, Builder rootBuilder) { - super(name, targetSteps, rootBuilder); - } - - public SourceBuilder withValues(Row... data) { - return withValuesBeforeRestore(data); - } - - public SourceBuilder withValuesBeforeRestore(Row... data) { - this.dataBeforeRestore.addAll(Arrays.asList(data)); - return this; - } - - public SourceBuilder withValuesAfterRestore(Row... data) { - this.dataAfterRestore.addAll(Arrays.asList(data)); - return this; - } - - public Builder complete() { - targetSteps.add( - new SourceTestStep( - name, - schemaComponents, - partitionKeys, - options, - dataBeforeRestore, - dataAfterRestore)); - return rootBuilder; - } - } - - /** Builder pattern for {@link SinkTestStep}. */ - public static class SinkBuilder extends TableBuilder { - - private List expectedBeforeRestore; - private List expectedAfterRestore; - - private List expectedBeforeRestoreStrings; - private List expectedAfterRestoreStrings; - - private SinkBuilder(String name, List targetSteps, Builder rootBuilder) { - super(name, targetSteps, rootBuilder); - } - - public SinkBuilder withExpectedValues(Row... expectedRows) { - return withValuesBeforeRestore(expectedRows); - } - - public SinkBuilder withExpectedValues(String... expectedRows) { - return withValuesBeforeRestore(expectedRows); - } - - public SinkBuilder withValuesBeforeRestore(Row... expectedRows) { - this.expectedBeforeRestore = Arrays.asList(expectedRows); - return this; - } - - public SinkBuilder withValuesBeforeRestore(String... expectedRows) { - this.expectedBeforeRestoreStrings = Arrays.asList(expectedRows); + public Builder setupTableSink(SinkTestStep sinkTestStep) { + setupSteps.add(sinkTestStep); return this; } - public SinkBuilder withValuesAfterRestore(Row... expectedRows) { - this.expectedAfterRestore = Arrays.asList(expectedRows); - return this; - } - - public SinkBuilder withValuesAfterRestore(String... expectedRows) { - this.expectedAfterRestoreStrings = Arrays.asList(expectedRows); + /** Run step for executing SQL. */ + public Builder runSql(String sql) { + this.runSteps.add(new SqlTestStep(sql)); return this; } - public Builder complete() { - targetSteps.add( - new SinkTestStep( - name, - schemaComponents, - partitionKeys, - options, - expectedBeforeRestore, - expectedAfterRestore, - expectedBeforeRestoreStrings, - expectedAfterRestoreStrings)); - return rootBuilder; - } - } - - /** Builder pattern for {@link StatementSetTestStep}. */ - public static class StatementSetBuilder { - - private final List targetSteps; - private final Builder rootBuilder; - private final List statements = new ArrayList<>(); - - private StatementSetBuilder(List targetSteps, Builder rootBuilder) { - this.targetSteps = targetSteps; - this.rootBuilder = rootBuilder; - } - - public StatementSetBuilder withSql(String sql) { - this.statements.add(sql); + /** Run step for executing a statement set. */ + public Builder runStatementSet(String... sql) { + this.runSteps.add(new StatementSetTestStep(Arrays.asList(sql))); return this; } - public Builder complete() { - this.targetSteps.add(new StatementSetTestStep(statements)); - return rootBuilder; + public TableTestProgram build() { + return new TableTestProgram(id, description, setupSteps, runSteps); } } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java index bff700d9a159b..1d0207f71261e 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java @@ -18,9 +18,13 @@ package org.apache.flink.table.test.program; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -70,4 +74,61 @@ public TableResult apply(TableEnvironment env, Map extraOptions) return env.executeSql(createTable); } + + /** Builder pattern for {@link SourceTestStep} and {@link SinkTestStep}. */ + @SuppressWarnings("unchecked") + protected abstract static class AbstractBuilder< + SpecificBuilder extends AbstractBuilder> { + + protected final String name; + + protected final List schemaComponents = new ArrayList<>(); + protected final List partitionKeys = new ArrayList<>(); + protected final Map options = new HashMap<>(); + + protected AbstractBuilder(String name) { + this.name = name; + } + + /** + * Define the schema like you would in SQL e.g. "my_col INT", "PRIMARY KEY (uid) NOT + * ENFORCED", or "WATERMARK FOR ts AS ts". + */ + public SpecificBuilder addSchema(String... schemaComponents) { + this.schemaComponents.addAll(Arrays.asList(schemaComponents)); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder addOptions(Map options) { + this.options.putAll(options); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder addOption(String key, String value) { + this.options.put(key, value); + return (SpecificBuilder) this; + } + + /** + * Unless the test requires a very specific configuration, try to avoid calling this method + * and fill in options later via {@link TableTestStep#apply(TableEnvironment, Map)}. + */ + public SpecificBuilder addOption(ConfigOption option, String value) { + this.options.put(option.key(), ConfigurationUtils.convertValue(value, String.class)); + return (SpecificBuilder) this; + } + + public SpecificBuilder addPartitionKeys(String... partitionKeys) { + this.partitionKeys.addAll(Arrays.asList(partitionKeys)); + return (SpecificBuilder) this; + } + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java index 534d50a26bb1f..feb151ba9ef84 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java @@ -19,6 +19,8 @@ package org.apache.flink.table.planner.plan.nodes.exec.testutils; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; import org.apache.flink.table.test.program.TableTestProgram; import org.apache.flink.types.Row; @@ -27,16 +29,18 @@ public class CalcTestPrograms { static final TableTestProgram SIMPLE_CALC = TableTestProgram.of("calc-simple", "validates basic calc node") - .setupTableSource("t") - .withSchema("a BIGINT", "b DOUBLE") - .withValuesBeforeRestore(Row.of(420L, 42.0)) - .withValuesAfterRestore(Row.of(421L, 42.1)) - .complete() - .setupTableSink("sink_t") - .withSchema("a BIGINT", "b DOUBLE") - .withValuesBeforeRestore(Row.of(421L, 42.0)) - .withValuesAfterRestore(Row.of(422L, 42.1)) - .complete() + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("a BIGINT", "b DOUBLE") + .producedBeforeRestore(Row.of(420L, 42.0)) + .producedAfterRestore(Row.of(421L, 42.1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a BIGINT", "b DOUBLE") + .consumedBeforeRestore(Row.of(421L, 42.0)) + .consumedAfterRestore(Row.of(422L, 42.1)) + .build()) .runSql("INSERT INTO sink_t SELECT a + 1, b FROM t") .build(); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java index e7b58aaa86439..a0d114c1178ec 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/test/program/TableTestProgramRunnerTest.java @@ -97,14 +97,16 @@ void testSqlStep() { void testTableStep() { final TableTestProgram program = TableTestProgram.of(ID, DESCRIPTION) - .setupTableSource("MyTableSource") - .withSchema("i INT") - .withOption("connector", "datagen") - .complete() - .setupTableSource("MyTableSink") - .withSchema("i INT") - .withOption("connector", "blackhole") - .complete() + .setupTableSource( + SourceTestStep.newBuilder("MyTableSource") + .addSchema("i INT") + .addOption("connector", "datagen") + .build()) + .setupTableSink( + SinkTestStep.newBuilder("MyTableSink") + .addSchema("i INT") + .addOption("connector", "blackhole") + .build()) .build(); assertThat(program.setupSteps).hasSize(2); @@ -127,8 +129,7 @@ void testTableStep() { "CREATE TABLE `default_catalog`.`default_database`.`MyTableSink` (\n" + " `i` INT\n" + ") WITH (\n" - + " 'connector' = 'blackhole',\n" - + " 'number-of-rows' = '3'\n" + + " 'connector' = 'blackhole'\n" + ")\n"); } From eef22705dd94fcc9520b8defc11cc562f93cee84 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Wed, 1 Nov 2023 17:26:49 +0800 Subject: [PATCH 059/104] [hotfix][python] Fix Kafka csv example --- .../examples/datastream/connectors/kafka_csv_format.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py index 4dbb243fcf984..39c134a8ed336 100644 --- a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py +++ b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py @@ -21,8 +21,7 @@ from pyflink.common import Types from pyflink.datastream import StreamExecutionEnvironment from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer -from pyflink.datastream.formats.csv import CsvRowSerializationSchema -from pyflink.datastream.formats.json import JsonRowDeserializationSchema +from pyflink.datastream.formats.csv import CsvRowSerializationSchema, CsvRowDeserializationSchema # Make sure that the Kafka cluster is started and the topic 'test_csv_topic' is @@ -46,9 +45,8 @@ def write_to_kafka(env): def read_from_kafka(env): - deserialization_schema = JsonRowDeserializationSchema.Builder() \ - .type_info(Types.ROW([Types.INT(), Types.STRING()])) \ - .build() + type_info = Types.ROW([Types.INT(), Types.STRING()]) + deserialization_schema = CsvRowDeserializationSchema.Builder(type_info).build() kafka_consumer = FlinkKafkaConsumer( topics='test_csv_topic', From dd47530f1eee829242c1c7a5fab9b4b8553cba5a Mon Sep 17 00:00:00 2001 From: Fang Yong Date: Wed, 1 Nov 2023 19:00:40 +0800 Subject: [PATCH 060/104] [FLINK-32309][sql-gateway] Use independent resource manager for table environment (#22768) --- .../src/test/resources/sql/function.q | 31 +++++++---- .../service/operation/OperationExecutor.java | 54 +++++++++++++++++-- .../gateway/service/result/ResultFetcher.java | 15 ++++++ .../flink/table/catalog/FunctionCatalog.java | 37 +++++++++++-- .../flink/table/resource/ResourceManager.java | 52 +++++++++++++----- .../table/resource/ResourceManagerTest.java | 19 +++++++ 6 files changed, 179 insertions(+), 29 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/resources/sql/function.q b/flink-table/flink-sql-client/src/test/resources/sql/function.q index 320fbefc8989b..e3e21ed8ccdf5 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/function.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/function.q @@ -306,6 +306,11 @@ create function upperudf AS 'UpperUDF' using jar '$VAR_UDF_JAR_PATH'; [INFO] Execute statement succeed. !info +# `SHOW JARS` does not list the jars being used by function, it only list all the jars added by `ADD JAR` +SHOW JARS; +Empty set +!ok + # run a query to verify the registered UDF works SELECT id, upperudf(str) FROM (VALUES (1, 'hello world'), (2, 'hi')) as T(id, str); +----+-------------+--------------------------------+ @@ -317,19 +322,27 @@ SELECT id, upperudf(str) FROM (VALUES (1, 'hello world'), (2, 'hi')) as T(id, st Received a total of 2 rows !ok +# Each query registers its jar to resource manager could not affect the session in sql gateway SHOW JARS; -+-$VAR_UDF_JAR_PATH_DASH-----+ -| $VAR_UDF_JAR_PATH_SPACEjars | -+-$VAR_UDF_JAR_PATH_DASH-----+ -| $VAR_UDF_JAR_PATH | -+-$VAR_UDF_JAR_PATH_DASH-----+ -1 row in set +Empty set !ok -REMOVE JAR '$VAR_UDF_JAR_PATH'; -[INFO] Execute statement succeed. -!info +# Show all users functions which should not add function jars to session resource manager +show user functions; ++---------------+ +| function name | ++---------------+ +| func11 | +| func3 | +| func4 | +| temp_upperudf | +| tmp_func | +| upperudf | ++---------------+ +6 rows in set +!ok +# Show functions will not affect the session in sql gateway SHOW JARS; Empty set !ok diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index 96a9003f81676..1d29e555a7463 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -37,6 +37,7 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; +import org.apache.flink.table.api.internal.ExecutableOperationContextImpl; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.CatalogBaseTable.TableKind; @@ -70,10 +71,12 @@ import org.apache.flink.table.operations.CompileAndExecutePlanOperation; import org.apache.flink.table.operations.DeleteFromFilterOperation; import org.apache.flink.table.operations.EndStatementSetOperation; +import org.apache.flink.table.operations.ExecutableOperation; import org.apache.flink.table.operations.LoadModuleOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.operations.ShowFunctionsOperation; import org.apache.flink.table.operations.StatementSetOperation; import org.apache.flink.table.operations.UnloadModuleOperation; import org.apache.flink.table.operations.UseOperation; @@ -82,10 +85,13 @@ import org.apache.flink.table.operations.command.RemoveJarOperation; import org.apache.flink.table.operations.command.ResetOperation; import org.apache.flink.table.operations.command.SetOperation; +import org.apache.flink.table.operations.command.ShowJarsOperation; import org.apache.flink.table.operations.command.ShowJobsOperation; import org.apache.flink.table.operations.command.StopJobOperation; import org.apache.flink.table.operations.ddl.AlterOperation; +import org.apache.flink.table.operations.ddl.CreateCatalogFunctionOperation; import org.apache.flink.table.operations.ddl.CreateOperation; +import org.apache.flink.table.operations.ddl.CreateTempSystemFunctionOperation; import org.apache.flink.table.operations.ddl.DropOperation; import org.apache.flink.table.resource.ResourceManager; import org.apache.flink.table.utils.DateTimeUtils; @@ -110,6 +116,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; import static org.apache.flink.table.api.internal.TableResultInternal.TABLE_RESULT_OK; import static org.apache.flink.table.gateway.service.utils.Constants.COMPLETION_CANDIDATES; import static org.apache.flink.table.gateway.service.utils.Constants.JOB_ID; @@ -183,7 +191,8 @@ public ResultFetcher configureSession(OperationHandle handle, String statement) public ResultFetcher executeStatement(OperationHandle handle, String statement) { // Instantiate the TableEnvironment lazily - TableEnvironmentInternal tableEnv = getTableEnvironment(); + ResourceManager resourceManager = sessionContext.getSessionState().resourceManager.copy(); + TableEnvironmentInternal tableEnv = getTableEnvironment(resourceManager); List parsedOperations = tableEnv.getParser().parse(statement); if (parsedOperations.size() > 1) { throw new UnsupportedOperationException( @@ -197,14 +206,15 @@ public ResultFetcher executeStatement(OperationHandle handle, String statement) try { SqlGatewayStreamExecutionEnvironment.setAsContext( sessionContext.getUserClassloader()); - return executeOperation(tableEnv, handle, op); + return executeOperation(tableEnv, handle, op).withResourceManager(resourceManager); } finally { SqlGatewayStreamExecutionEnvironment.unsetAsContext(); } } else { return sessionContext.isStatementSetState() ? executeOperationInStatementSetState(tableEnv, handle, op) - : executeOperation(tableEnv, handle, op); + .withResourceManager(resourceManager) + : executeOperation(tableEnv, handle, op).withResourceManager(resourceManager); } } @@ -315,6 +325,10 @@ public ResultFetcher getCompletionHints( // -------------------------------------------------------------------------------------------- public TableEnvironmentInternal getTableEnvironment() { + return getTableEnvironment(sessionContext.getSessionState().resourceManager); + } + + public TableEnvironmentInternal getTableEnvironment(ResourceManager resourceManager) { // checks the value of RUNTIME_MODE Configuration operationConfig = sessionContext.getSessionConf().clone(); operationConfig.addAll(executionConfig); @@ -342,8 +356,8 @@ public TableEnvironmentInternal getTableEnvironment() { executor, sessionContext.getSessionState().catalogManager, sessionContext.getSessionState().moduleManager, - sessionContext.getSessionState().resourceManager, - sessionContext.getSessionState().functionCatalog); + resourceManager, + sessionContext.getSessionState().functionCatalog.copy(resourceManager)); } private static Executor lookupExecutor( @@ -440,11 +454,41 @@ private ResultFetcher executeOperation( return callShowJobsOperation(tableEnv, handle, (ShowJobsOperation) op); } else if (op instanceof RemoveJarOperation) { return callRemoveJar(handle, ((RemoveJarOperation) op).getPath()); + } else if (op instanceof AddJarOperation + || op instanceof ShowJarsOperation + || op instanceof CreateTempSystemFunctionOperation + || op instanceof CreateCatalogFunctionOperation + || op instanceof ShowFunctionsOperation) { + return callExecutableOperation(handle, (ExecutableOperation) op); } else { return callOperation(tableEnv, handle, op); } } + private ResultFetcher callExecutableOperation(OperationHandle handle, ExecutableOperation op) { + TableResultInternal result = + op.execute( + new ExecutableOperationContextImpl( + sessionContext.getSessionState().catalogManager, + sessionContext.getSessionState().functionCatalog, + sessionContext.getSessionState().moduleManager, + sessionContext.getSessionState().resourceManager, + tableConfig(), + sessionContext.getSessionConf().get(RUNTIME_MODE) == STREAMING)); + return ResultFetcher.fromTableResult(handle, result, false); + } + + private TableConfig tableConfig() { + Configuration operationConfig = sessionContext.getSessionConf().clone(); + operationConfig.addAll(executionConfig); + + TableConfig tableConfig = TableConfig.getDefault(); + tableConfig.setRootConfiguration(sessionContext.getDefaultContext().getFlinkConfig()); + tableConfig.addConfiguration(operationConfig); + + return tableConfig; + } + private ResultFetcher callSetOperation( TableEnvironmentInternal tableEnv, OperationHandle handle, SetOperation setOp) { if (setOp.getKey().isPresent() && setOp.getValue().isPresent()) { diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java index 9fa810c9c8cba..f901091f18ed8 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java @@ -30,6 +30,7 @@ import org.apache.flink.table.gateway.api.results.ResultSet; import org.apache.flink.table.gateway.api.results.ResultSetImpl; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; +import org.apache.flink.table.resource.ResourceManager; import org.apache.flink.table.utils.print.RowDataToStringConverter; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.CollectionUtil; @@ -39,6 +40,7 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -78,6 +80,7 @@ public class ResultFetcher { private long currentToken = 0; private boolean noMoreResults = false; + @Nullable private ResourceManager resourceManager; private ResultFetcher( OperationHandle operationHandle, @@ -181,8 +184,20 @@ public static ResultFetcher fromResults( return new ResultFetcher(operationHandle, resultSchema, results, jobID, resultKind); } + public ResultFetcher withResourceManager(ResourceManager resourceManager) { + this.resourceManager = resourceManager; + return this; + } + public void close() { resultStore.close(); + if (resourceManager != null) { + try { + resourceManager.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } } public ResolvedSchema getResultSchema() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java index 8eb8f82351f6a..55507ad1e6d78 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java @@ -78,9 +78,8 @@ public final class FunctionCatalog { private final CatalogManager catalogManager; private final ModuleManager moduleManager; - private final Map tempSystemFunctions = new LinkedHashMap<>(); - private final Map tempCatalogFunctions = - new LinkedHashMap<>(); + private final Map tempSystemFunctions; + private final Map tempCatalogFunctions; /** * Temporary utility until the new type inference is fully functional. It needs to be set by the @@ -93,10 +92,31 @@ public FunctionCatalog( ResourceManager resourceManager, CatalogManager catalogManager, ModuleManager moduleManager) { + this( + config, + resourceManager, + catalogManager, + moduleManager, + new LinkedHashMap<>(), + new LinkedHashMap<>(), + null); + } + + private FunctionCatalog( + ReadableConfig config, + ResourceManager resourceManager, + CatalogManager catalogManager, + ModuleManager moduleManager, + Map tempSystemFunctions, + Map tempCatalogFunctions, + PlannerTypeInferenceUtil plannerTypeInferenceUtil) { this.config = checkNotNull(config); this.resourceManager = checkNotNull(resourceManager); this.catalogManager = checkNotNull(catalogManager); this.moduleManager = checkNotNull(moduleManager); + this.tempSystemFunctions = tempSystemFunctions; + this.tempCatalogFunctions = tempCatalogFunctions; + this.plannerTypeInferenceUtil = plannerTypeInferenceUtil; } public void setPlannerTypeInferenceUtil(PlannerTypeInferenceUtil plannerTypeInferenceUtil) { @@ -777,6 +797,17 @@ public void registerFunctionJarResources(String functionName, List } } + public FunctionCatalog copy(ResourceManager newResourceManager) { + return new FunctionCatalog( + config, + newResourceManager, + catalogManager, + moduleManager, + tempSystemFunctions, + tempCatalogFunctions, + plannerTypeInferenceUtil); + } + private void registerCatalogFunction( ObjectIdentifier identifier, CatalogFunction catalogFunction, boolean ignoreIfExists) { final ObjectIdentifier normalizedIdentifier = diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/resource/ResourceManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/resource/ResourceManager.java index 5de2b5d6ef4b9..307380af41b8a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/resource/ResourceManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/resource/ResourceManager.java @@ -69,10 +69,12 @@ public class ResourceManager implements Closeable { private static final String JAR_SUFFIX = "jar"; private static final String FILE_SCHEME = "file"; - private final Path localResourceDir; + protected final Path localResourceDir; /** Resource infos for functions. */ private final Map functionResourceInfos; + private final boolean cleanLocalResource; + protected final Map resourceInfos; protected final MutableURLClassLoader userClassLoader; @@ -84,13 +86,27 @@ public static ResourceManager createResourceManager( } public ResourceManager(ReadableConfig config, MutableURLClassLoader userClassLoader) { - this.localResourceDir = + this( new Path( config.get(TableConfigOptions.RESOURCES_DOWNLOAD_DIR), - String.format("flink-table-%s", UUID.randomUUID())); - this.functionResourceInfos = new HashMap<>(); - this.resourceInfos = new HashMap<>(); + String.format("flink-table-%s", UUID.randomUUID())), + new HashMap<>(), + new HashMap<>(), + userClassLoader, + true); + } + + private ResourceManager( + Path localResourceDir, + Map resourceInfos, + Map functionResourceInfos, + MutableURLClassLoader userClassLoader, + boolean cleanLocalResource) { + this.localResourceDir = localResourceDir; + this.functionResourceInfos = functionResourceInfos; + this.resourceInfos = resourceInfos; this.userClassLoader = userClassLoader; + this.cleanLocalResource = cleanLocalResource; } /** @@ -232,6 +248,15 @@ public void addJarConfiguration(TableConfig tableConfig) { tableConfig.set(PipelineOptions.JARS, new ArrayList<>(jarFiles)); } + public ResourceManager copy() { + return new ResourceManager( + localResourceDir, + new HashMap<>(resourceInfos), + new HashMap<>(functionResourceInfos), + userClassLoader.copy(), + false); + } + @Override public void close() throws IOException { resourceInfos.clear(); @@ -245,14 +270,17 @@ public void close() throws IOException { exception = e; } - FileSystem fileSystem = FileSystem.getLocalFileSystem(); - try { - if (fileSystem.exists(localResourceDir)) { - fileSystem.delete(localResourceDir, true); + if (cleanLocalResource) { + FileSystem fileSystem = FileSystem.getLocalFileSystem(); + try { + if (fileSystem.exists(localResourceDir)) { + fileSystem.delete(localResourceDir, true); + } + } catch (IOException ioe) { + LOG.debug( + String.format("Error while delete directory [%s].", localResourceDir), ioe); + exception = ExceptionUtils.firstOrSuppressed(ioe, exception); } - } catch (IOException ioe) { - LOG.debug(String.format("Error while delete directory [%s].", localResourceDir), ioe); - exception = ExceptionUtils.firstOrSuppressed(ioe, exception); } if (exception != null) { diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/resource/ResourceManagerTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/resource/ResourceManagerTest.java index 42305d806af06..400e865b41624 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/resource/ResourceManagerTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/resource/ResourceManagerTest.java @@ -431,6 +431,25 @@ void testRegisterFunctionWithResource() { assertThat(functionResourceInfos.containsKey(resourceUri)).isFalse(); } + @Test + void testCloseCopiedResourceManager() throws Exception { + ResourceUri resourceUri = new ResourceUri(ResourceType.JAR, udfJar.getPath()); + resourceManager.declareFunctionResources(Collections.singleton(resourceUri)); + resourceManager.registerJarResources(Collections.singletonList(resourceUri)); + assertThat(resourceManager.functionResourceInfos().size()).isEqualTo(1); + assertThat(resourceManager.resourceInfos.size()).isEqualTo(1); + + ResourceManager copiedResourceManager = resourceManager.copy(); + assertThat(copiedResourceManager.functionResourceInfos().size()).isEqualTo(1); + assertThat(copiedResourceManager.resourceInfos.size()).isEqualTo(1); + copiedResourceManager.close(); + assertThat(copiedResourceManager.functionResourceInfos().size()).isEqualTo(0); + assertThat(copiedResourceManager.resourceInfos.size()).isEqualTo(0); + + assertThat(resourceManager.functionResourceInfos().size()).isEqualTo(1); + assertThat(resourceManager.resourceInfos.size()).isEqualTo(1); + } + @Test public void testCloseResourceManagerCleanDownloadedResources() throws Exception { resourceManager.close(); From 011e3ae19b1c761f8f60eb323aa76c7fc0323f76 Mon Sep 17 00:00:00 2001 From: yuxiang <384669270@qq.com> Date: Wed, 1 Nov 2023 20:43:38 +0800 Subject: [PATCH 061/104] [FLINK-32107][test] Adds retry to artifact downloads (#23528) Co-authored-by: yu <13485876233> --- .../test-scripts/common_kubernetes.sh | 54 ++++++++++++------- 1 file changed, 36 insertions(+), 18 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh index 59b4ea70f3472..c12fa071fe42d 100755 --- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh +++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh @@ -21,10 +21,12 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_docker.sh CONTAINER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts -MINIKUBE_START_RETRIES=3 -MINIKUBE_START_BACKOFF=5 +RETRY_COUNT=3 +RETRY_BACKOFF_TIME=5 RESULT_HASH="e682ec6622b5e83f2eb614617d5ab2cf" MINIKUBE_VERSION="v1.28.0" +CRICTL_VERSION="v1.24.2" +CRI_DOCKERD_VERSION="0.2.3" NON_LINUX_ENV_NOTE="****** Please start/stop minikube manually in non-linux environment. ******" @@ -39,8 +41,9 @@ function setup_kubernetes_for_linux { if ! [ -x "$(command -v kubectl)" ]; then echo "Installing kubectl ..." local version=$(curl -s https://storage.googleapis.com/kubernetes-release/release/stable.txt) - curl -Lo kubectl https://storage.googleapis.com/kubernetes-release/release/$version/bin/linux/$arch/kubectl && \ - chmod +x kubectl && sudo mv kubectl /usr/local/bin/ + download_kubectl_url="https://storage.googleapis.com/kubernetes-release/release/$version/bin/linux/$arch/kubectl" + retry_download "${download_kubectl_url}" + chmod +x kubectl && sudo mv kubectl /usr/local/bin/ fi # Download minikube when it is not installed beforehand. if [ -x "$(command -v minikube)" ] && [[ "$(minikube version | grep -c $MINIKUBE_VERSION)" == "0" ]]; then @@ -50,32 +53,32 @@ function setup_kubernetes_for_linux { if ! [ -x "$(command -v minikube)" ]; then echo "Installing minikube $MINIKUBE_VERSION ..." - curl -Lo minikube https://storage.googleapis.com/minikube/releases/$MINIKUBE_VERSION/minikube-linux-$arch && \ - chmod +x minikube && sudo mv minikube /usr/bin/minikube + download_minikube_url="https://storage.googleapis.com/minikube/releases/$MINIKUBE_VERSION/minikube-linux-$arch" + retry_download "${download_minikube_url}" + chmod +x "minikube-linux-$arch" && sudo mv "minikube-linux-$arch" /usr/bin/minikube fi # conntrack is required for minikube 1.9 and later sudo apt-get install conntrack # crictl is required for cri-dockerd - local crictl_version crictl_archive - crictl_version="v1.24.2" - crictl_archive="crictl-$crictl_version-linux-${arch}.tar.gz" - wget -nv "https://github.com/kubernetes-sigs/cri-tools/releases/download/${crictl_version}/${crictl_archive}" + local crictl_archive + crictl_archive="crictl-$CRICTL_VERSION-linux-${arch}.tar.gz" + download_crictl_url="https://github.com/kubernetes-sigs/cri-tools/releases/download/${CRICTL_VERSION}/${crictl_archive}" + retry_download "${download_crictl_url}" sudo tar zxvf ${crictl_archive} -C /usr/local/bin rm -f ${crictl_archive} # cri-dockerd is required to use Kubernetes 1.24+ and the none driver - local cri_dockerd_version cri_dockerd_archive cri_dockerd_binary - cri_dockerd_version="0.2.3" - cri_dockerd_archive="cri-dockerd-${cri_dockerd_version}.${arch}.tgz" + local cri_dockerd_archive cri_dockerd_binary + cri_dockerd_archive="cri-dockerd-${CRI_DOCKERD_VERSION}.${arch}.tgz" cri_dockerd_binary="cri-dockerd" - wget -nv "https://github.com/Mirantis/cri-dockerd/releases/download/v${cri_dockerd_version}/${cri_dockerd_archive}" + retry_download "https://github.com/Mirantis/cri-dockerd/releases/download/v${CRI_DOCKERD_VERSION}/${cri_dockerd_archive}" tar xzvf $cri_dockerd_archive "cri-dockerd/${cri_dockerd_binary}" --strip-components=1 sudo install -o root -g root -m 0755 "${cri_dockerd_binary}" "/usr/local/bin/${cri_dockerd_binary}" rm ${cri_dockerd_binary} - wget -nv https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${cri_dockerd_version}/packaging/systemd/cri-docker.service - wget -nv https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${cri_dockerd_version}/packaging/systemd/cri-docker.socket + retry_download "https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${CRI_DOCKERD_VERSION}/packaging/systemd/cri-docker.service" + retry_download "https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${CRI_DOCKERD_VERSION}/packaging/systemd/cri-docker.socket" sudo mv cri-docker.socket cri-docker.service /etc/systemd/system/ sudo sed -i -e "s,/usr/bin/${cri_dockerd_binary},/usr/local/bin/${cri_dockerd_binary}," /etc/systemd/system/cri-docker.service @@ -87,6 +90,21 @@ function setup_kubernetes_for_linux { sudo sysctl fs.protected_regular=0 } +function retry_download { + if [[ "$#" != 1 ]]; then + echo "Fatal error: No parameter or too many parameters passed: $@" + exit 1; + fi + + local download_url download_command + download_url="$1" + download_command="wget -nv ${download_url}" + if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} "${download_command}"; then + echo "ERROR: Download failed repeatedly after ${RETRY_COUNT} tries. Aborting..." + exit 1 + fi +} + function check_kubernetes_status { minikube status return $? @@ -138,7 +156,7 @@ function start_kubernetes { echo "The mounting process is running with pid $minikube_mount_pid" else setup_kubernetes_for_linux - if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} start_kubernetes_if_not_running; then + if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} start_kubernetes_if_not_running; then echo "Could not start minikube. Aborting..." exit 1 fi @@ -153,7 +171,7 @@ function stop_kubernetes { else echo "Stopping minikube ..." stop_command="minikube stop" - if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} "${stop_command}"; then + if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} "${stop_command}"; then echo "Could not stop minikube. Aborting..." exit 1 fi From 56b1be467041bbe8c18140083279ccf0416f54fe Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 1 Nov 2023 10:24:48 +0100 Subject: [PATCH 062/104] [hotfix] Update docs build CI image --- .github/workflows/docs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index 3353fdc693b6d..26b29a1c9ca36 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -48,7 +48,7 @@ jobs: fi - name: Build documentation run: | - docker run --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11 bash -c "cd /root/flink && ./.github/workflows/docs.sh" + docker run --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" - name: Upload documentation uses: burnett01/rsync-deployments@5.2 with: From 0854084e08b529a22ce7851630423c1107e7a822 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 1 Nov 2023 16:47:48 +0100 Subject: [PATCH 063/104] [hotfix][docs] Fix docker/git permission issue --- .github/workflows/docs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index 26b29a1c9ca36..ff59e8cd4f411 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -48,7 +48,7 @@ jobs: fi - name: Build documentation run: | - docker run --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" + docker run -user 1001 --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" - name: Upload documentation uses: burnett01/rsync-deployments@5.2 with: From d5b24795d54c9b3a550e5c7b0957631778dfaf6f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 1 Nov 2023 16:49:19 +0100 Subject: [PATCH 064/104] [hotfix][docs][ci] Fix typo --- .github/workflows/docs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index ff59e8cd4f411..17d927a0eecc8 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -48,7 +48,7 @@ jobs: fi - name: Build documentation run: | - docker run -user 1001 --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" + docker run --user 1001 --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" - name: Upload documentation uses: burnett01/rsync-deployments@5.2 with: From 24c1954cebc0e62655a9621d5a2adef81db84b5d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 1 Nov 2023 16:58:01 +0100 Subject: [PATCH 065/104] [hotfix][docs] Fix git permission issue attempt #2 --- .github/workflows/docs.sh | 2 ++ .github/workflows/docs.yml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/docs.sh b/.github/workflows/docs.sh index b8a088d31d1c9..1d08bbbafef26 100755 --- a/.github/workflows/docs.sh +++ b/.github/workflows/docs.sh @@ -22,6 +22,8 @@ mvn --version java -version javadoc -J-version +# workaround for a git security patch +git config --global --add safe.directory /root/flink git submodule update --init --recursive cd docs diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index 17d927a0eecc8..26754d2c1e6f6 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -48,7 +48,7 @@ jobs: fi - name: Build documentation run: | - docker run --user 1001 --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" + docker run --rm --volume "$PWD:/root/flink" chesnay/flink-ci:java_8_11_17_maven_386_v2 bash -c "cd /root/flink && ./.github/workflows/docs.sh" - name: Upload documentation uses: burnett01/rsync-deployments@5.2 with: From 817e3f2b964fa5d86e207d6aa3065f139ec84402 Mon Sep 17 00:00:00 2001 From: Xiangyu Feng Date: Wed, 1 Nov 2023 19:16:08 +0800 Subject: [PATCH 066/104] [FLINK-33235][doc] Update OLAP Quickstart doc --- .../content/docs/dev/table/olap_quickstart.md | 181 +++++++++--------- docs/content/docs/dev/table/overview.md | 2 +- docs/static/fig/olap-architecture.svg | 21 ++ 3 files changed, 117 insertions(+), 87 deletions(-) create mode 100644 docs/static/fig/olap-architecture.svg diff --git a/docs/content/docs/dev/table/olap_quickstart.md b/docs/content/docs/dev/table/olap_quickstart.md index e0b3afba2bc57..e5084e065c203 100644 --- a/docs/content/docs/dev/table/olap_quickstart.md +++ b/docs/content/docs/dev/table/olap_quickstart.md @@ -1,5 +1,5 @@ --- -title: "Quickstart for Flink OLAP" +title: "OLAP Quickstart" weight: 91 type: docs aliases: @@ -24,32 +24,41 @@ specific language governing permissions and limitations under the License. --> -# Introduction +# OLAP Quickstart -Flink OLAP has already added to [Apache Flink Roadmap](https://flink.apache.org/roadmap/). It means Flink can not only support streaming and batch computing, but also support OLAP(On-Line Analytical Processing). This page will show how to quickly set up a Flink OLAP service, and will introduce some best practices. +OLAP (OnLine Analysis Processing) is a key technology in the field of data analysis, it is generally used to perform complex queries on large data sets with latencies in seconds. Now Flink can not only support streaming and batch computing, but also supports users to deploy it as an OLAP computing service. This page will show you how to quickly set up a local Flink OLAP service, and will also introduce some best practices helping you deploy Flink OLAP service in production. -## Architecture +## Architecture Introduction +This chapter will introduce you to the overall architecture of Flink OLAP service and the advantages of using it. -The Flink OLAP service consists of three parts: Client, Flink SQL Gateway, Flink Session Cluster. +### Architecture -* **Client**: Could be any client that can interact with Flink SQL Gateway, such as SQL client, Flink JDBC driver and so on. -* **Flink SQL Gateway**: The SQL Gateway provides an easy way to submit the Flink Job, look up the metadata, and analyze table stats. -* **Flink Session Cluster**: We choose session clusters to run OLAP queries, mainly to avoid the overhead of cluster startup. +Flink OLAP service consists of three parts: Client, Flink SQL Gateway and Flink Session Cluster. -## Advantage +* **Client**: Could be any client that can interact with [Flink SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}), such as [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}), [Flink JDBC Driver]({{< ref "docs/dev/table/jdbcDriver" >}}) and so on. +* **Flink SQL Gateway**: The SQL Gateway provides an easy way to parse the sql query, look up the metadata, analyze table stats, optimize the plan and submit JobGraphs to cluster. +* **Flink Session Cluster**: OLAP queries run on [session cluster]({{< ref "/docs/deployment/resource-providers/native_kubernetes#starting-a-flink-session-on-kubernetes" >}}), mainly to avoid the overhead of cluster startup. + +{{< img src="/fig/olap-architecture.svg" alt="Illustration of Flink OLAP Architecture" width="85%" >}} + +### Advantage * **Massively Parallel Processing** - * Flink OLAP runs naturally as an MPP(Massively Parallel Processing) system, which supports low-latency ad-hoc queries + * Flink OLAP runs naturally as a massively parallel processing system, which enables planners to easily adjust the job parallelism to fulfill queries' latency requirement under different data sizes. +* **Elastic Resource Management** + * Flink's resource management supports min/max scaling, which means the session cluster can allocate the resource according to workload dynamically. * **Reuse Connectors** - * Flink OLAP can reuse rich connectors in Flink ecosystem. + * Flink OLAP can reuse the rich [Connectors]({{< ref "docs/connectors/table/overview" >}}) in Flink ecosystem. * **Unified Engine** * Unified computing engine for Streaming/Batch/OLAP. -# Deploying in Local Mode +## Deploying in Local Mode + +In this chapter, you will learn how to build Flink OLAP services locally. -## Downloading Flink +### Downloading Flink -The same as [Local Installation]({{< ref "docs/try-flink/local_installation" >}}). Flink runs on all UNIX-like environments, i.e. Linux, Mac OS X, and Cygwin (for Windows). We need to have at least Java 11 installed, Java 17 is more recommended in OLAP scenario. To check the Java version installed, type in your terminal: +The same as [Local Installation]({{< ref "docs/try-flink/local_installation" >}}). Flink runs on all UNIX-like environments, i.e. Linux, Mac OS X, and Cygwin (for Windows). User need to have at __Java 11__ installed. To check the Java version installed, user can type in the terminal: ``` java -version @@ -61,7 +70,7 @@ Next, [Download](https://flink.apache.org/downloads/) the latest binary release tar -xzf flink-*.tgz ``` -## Starting a local cluster +### Starting a local cluster To start a local cluster, run the bash script that comes with Flink: @@ -69,9 +78,9 @@ To start a local cluster, run the bash script that comes with Flink: ./bin/start-cluster.sh ``` -You should be able to navigate to the web UI at localhost:8081 to view the Flink dashboard and see that the cluster is up and running. +You should be able to navigate to the web UI at http://localhost:8081 to view the Flink dashboard and see that the cluster is up and running. -## Start a SQL Client CLI +### Start a SQL Client CLI You can start the CLI with an embedded gateway by calling: @@ -79,7 +88,7 @@ You can start the CLI with an embedded gateway by calling: ./bin/sql-client.sh ``` -## Running Queries +### Running Queries You could simply execute queries in CLI and retrieve the results. @@ -102,98 +111,98 @@ GROUP BY buyer ORDER BY total_cost LIMIT 3; ``` -And then you could find job detail information in web UI at localhost:8081. +And then you could find job detail information in web UI at http://localhost:8081. -# Deploying in Production +## Deploying in Production This section guides you through setting up a production ready Flink OLAP service. -## Cluster Deployment +### Client -In production, we recommend to use Flink Session Cluster, Flink SQL Gateway and Flink JDBC Driver to build an OLAP service. +#### Flink JDBC Driver -### Session Cluster +You should use Flink JDBC Driver when submitting queries to SQL Gateway since it provides low-level connection management. When used in production, you should pay attention to reuse the JDBC connection to avoid frequently creating/closing sessions in the Gateway and then reduce the E2E query latency. For detailed information, please refer to the [Flink JDBC Driver]({{ }}). -For Flink Session Cluster, we recommend to deploy Flink on native Kubernetes using session mode. Kubernetes is a popular container-orchestration system for automating computer application deployment, scaling, and management. By deploying on native Kubernetes, Flink Session Cluster is able to dynamically allocate and de-allocate TaskManagers. For more information, please refer to [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes">}}). +### Cluster Deployment -### SQL Gateway +In production, you should use Flink Session Cluster, Flink SQL Gateway to build an OLAP service. -For Flink SQL Gateway, we recommend deploying it as a stateless microservice and register this on the service discovery component. For more information, please refer to the [SQL Gateway Overview]({{< ref "docs/dev/table/sql-gateway/overview">}}). +#### Session Cluster -### Flink JDBC Driver +For Flink Session Cluster, you can deploy it on Native Kubernetes using session mode. Kubernetes is a popular container-orchestration system for automating computer application deployment, scaling, and management. By deploying on Native Kubernetes, Flink Session Cluster is able to dynamically allocate and de-allocate TaskManagers. For more information, please refer to [Native Kubernetes]({{ < ref "docs/deployment/resource-providers/native_kubernetes"> }}). Furthermore, you can config the option [slotmanager.number-of-slots.min]({{< ref "docs/deployment/config#slotmanager-number-of-slots-min" >}}) in session cluster. This will help you significantly reduce the cold start time of your query. For detailed information, please refer to [FLIP-362](https://cwiki.apache.org/confluence/display/FLINK/FLIP-362%3A+Support+minimum+resource+limitation). -When submitting queries to SQL Gateway, we recommend using Flink JDBC Driver since it provides low-level connection management. When used in production, we need to pay attention to reuse the JDBC connection to avoid frequently creating/closing sessions in the Gateway. For more information, please refer to the [Flink JDBC Driver]({{{}}}). +#### SQL Gateway -## Datasource Configurations +For Flink SQL Gateway, you should deploy it as a stateless microservice and register the instance on service discovery component. Through this way, client can balance the query between instances easily. For more information, please refer to [SQL Gateway Overview]({{< ref "docs/dev/table/sql-gateway/overview">}}). -### Catalogs +### Datasource Configurations -In OLAP scenario, we recommend using FileCatalogStore in the catalog configuration introduced in [FLIP-295](https://cwiki.apache.org/confluence/display/FLINK/FLIP-295%3A+Support+lazy+initialization+of+catalogs+and+persistence+of+catalog+configurations). As a long running service, Flink OLAP cluster's catalog information will not change frequently and can be re-used cross sessions. For more information, please refer to the [Catalog Store]({{< ref "docs/dev/table/catalogs#catalog-store">}}). +#### Catalogs -### Connectors +In OLAP scenario, you should configure `FileCatalogStore` provided by [Catalogs]({{< ref "docs/dev/table/catalogs">}}) as the catalog used by cluster. As a long-running service, Flink OLAP cluster's catalog information will not change frequently and should be re-used cross sessions to reduce the cold-start cost. For more information, please refer to the [Catalog Store]({{< ref "docs/dev/table/catalogs#catalog-store">}}). -Both Session Cluster and SQL Gateway rely on connectors to analyze table stats and read data from the configured data source. To add connectors, please refer to the [Connectors and Formats]({{< ref "docs/connectors/table/overview">}}). +#### Connectors -## Cluster Configurations +Both Session Cluster and SQL Gateway rely on connectors to analyze table stats and read data from the configured data source. To add connectors, please refer to the [Connectors]({{< ref "docs/connectors/table/overview">}}). -In OLAP scenario, we picked out a few configurations that can help improve user usability and query performance. +### Recommended Cluster Configurations -### SQL&Table Options +In OLAP scenario, appropriate configurations that can greatly help users improve the overall usability and query performance. Here are some recommended production configurations: -| Parameters | Default | Recommended | -|:-------------------------------------|:--------|:------------| -| table.optimizer.join-reorder-enabled | false | true | -| pipeline.object-reuse | false | true | +#### SQL&Table Options -### Runtime Options +| Parameters | Default | Recommended | +|:---------------------------------------------------------------------------------------------------------------|:--------|:------------| +| [table.optimizer.join-reorder-enabled]({{}}) | false | true | +| [pipeline.object-reuse]({{< ref "docs/deployment/config#pipeline-object-reuse" >}}) | false | true | -| Parameters | Default | Recommended | -|:-----------------------------|:-----------------------|:------------------------------------------------------------------------------------------------------------------------------------------| -| execution.runtime-mode | STREAMING | BATCH | -| execution.batch-shuffle-mode | ALL_EXCHANGES_BLOCKING | ALL_EXCHANGES_PIPELINED | -| env.java.opts.all | {default value} | {default value} -XX:PerMethodRecompilationCutoff=10000 -XX:PerBytecodeRecompilationCutoff=10000-XX:ReservedCodeCacheSize=512M -XX:+UseZGC | -| JDK Version | 11 | 17 | +#### Runtime Options -We strongly recommend using JDK17 with ZGC in OLAP scenario in order to provide zero gc stw and solve the issue described in [FLINK-32746](https://issues.apache.org/jira/browse/FLINK-32746). +| Parameters | Default | Recommended | +|:--------------------------------------------------------------------------------------------------|:-----------------------|:------------------------------------------------------------------------------------------------------------------------------------------| +| [execution.runtime-mode]({{< ref "docs/deployment/config#execution-runtime-mode" >}}) | STREAMING | BATCH | +| [execution.batch-shuffle-mode]({{< ref "docs/deployment/config#execution-batch-shuffle-mode" >}}) | ALL_EXCHANGES_BLOCKING | ALL_EXCHANGES_PIPELINED | +| [env.java.opts.all]({{< ref "docs/deployment/config#env-java-opts-all" >}}) | {default value} | {default value} -XX:PerMethodRecompilationCutoff=10000 -XX:PerBytecodeRecompilationCutoff=10000-XX:ReservedCodeCacheSize=512M -XX:+UseZGC | +| JDK Version | 11 | 17 | -### Scheduling Options +Using JDK17 within ZGC can greatly help optimize the metaspace garbage collection issue, detailed information can be found in [FLINK-32746](https://issues.apache.org/jira/browse/FLINK-32746). Meanwhile, ZGC can provide close to zero application pause time when collecting garbage objects in memory. Additionally, OLAP queries need to be executed in `BATCH` mode because both `Pipelined` and `Blocking` edges may appear in the execution plan of an OLAP query. Batch scheduler allows queries to be scheduled in stages, which could avoid scheduling deadlocks in this scenario. + +#### Scheduling Options | Parameters | Default | Recommended | -|:---------------------------------------------------------|:------------------|:------------------| -| jobmanager.scheduler | Default | Default | -| jobmanager.execution.failover-strategy | region | full | -| restart-strategy.type | (none) | disable | -| jobstore.type | File | Memory | -| jobstore.max-capacity | Integer.MAX_VALUE | 500 | - -We would like to highlight the usage of `PipelinedRegionSchedulingStrategy`. Since many OLAP queries will have blocking edges in their jobGraph. - -### Network Options - -| Parameters | Default | Recommended | -|:------------------------------------|:-----------|:---------------| -| rest.server.numThreads | 4 | 32 | -| web.refresh-interval | 3000 | 300000 | -| pekko.framesize | 10485760b | 104857600b | - -### ResourceManager Options - -| Parameters | Default | Recommended | -|:-------------------------------------|:----------|:---------------| -| kubernetes.jobmanager.replicas | 1 | 2 | -| kubernetes.jobmanager.cpu.amount | 1.0 | 16.0 | -| jobmanager.memory.process.size | (none) | 65536m | -| jobmanager.memory.jvm-overhead.max | 1g | 6144m | -| kubernetes.taskmanager.cpu.amount | (none) | 16 | -| taskmanager.numberOfTaskSlots | 1 | 32 | -| taskmanager.memory.process.size | (none) | 65536m | -| taskmanager.memory.managed.size | (none) | 65536m | - -We prefer to use large taskManager pods in OLAP since this can put more computation in local and reduce network/deserialization/serialization overhead. Meanwhile, since JobManager is a single point of calculation in OLAP scenario, we also prefer large pod. - -# Future Work -There is a big margin for improvement in Flink OLAP, both in usability and query performance, and we trace all of them in underlying tickets. +|:------------------------------------------------------------------------------------------------------------------------|:------------------|:--------| +| [jobmanager.scheduler]({{< ref "docs/deployment/config#jobmanager-scheduler" >}}) | Default | Default | +| [jobmanager.execution.failover-strategy]({{< ref "docs/deployment/config#jobmanager-execution-failover-strategy-1" >}}) | region | full | +| [restart-strategy.type]({{< ref "docs/deployment/config#restart-strategy-type" >}}) | (none) | disable | +| [jobstore.type]({{< ref "docs/deployment/config#jobstore-type" >}}) | File | Memory | +| [jobstore.max-capacity]({{< ref "docs/deployment/config#jobstore-max-capacity" >}}) | Integer.MAX_VALUE | 500 | + + +#### Network Options + +| Parameters | Default | Recommended | +|:--------------------------------------------------------------------------------------|:-----------|:---------------| +| [rest.server.numThreads]({{< ref "docs/deployment/config#rest-server-numthreads" >}}) | 4 | 32 | +| [web.refresh-interval]({{< ref "docs/deployment/config#web-refresh-interval" >}}) | 3000 | 300000 | +| [pekko.framesize]({{< ref "docs/deployment/config#pekko-framesize" >}}) | 10485760b | 104857600b | + +#### ResourceManager Options + +| Parameters | Default | Recommended | +|:-------------------------------------------------------------------|:--------|:----------------------------------------| +| [kubernetes.jobmanager.replicas]({{< ref "docs/deployment/config#kubernetes-jobmanager-replicas" >}}) | 1 | 2 | +| [kubernetes.jobmanager.cpu.amount]({{< ref "docs/deployment/config#kubernetes-jobmanager-cpu-amount" >}}) | 1.0 | 16.0 | +| [jobmanager.memory.process.size]({{< ref "docs/deployment/config#jobmanager-memory-process-size" >}}) | (none) | 32g | +| [jobmanager.memory.jvm-overhead.max]({{< ref "docs/deployment/config#jobmanager-memory-jvm-overhead-max" >}}) | 1g | 3g | +| [kubernetes.taskmanager.cpu.amount]({{< ref "docs/deployment/config#kubernetes-taskmanager-cpu-amount" >}}) | (none) | 16 | +| [taskmanager.numberOfTaskSlots]({{< ref "docs/deployment/config#taskmanager-numberoftaskslots" >}}) | 1 | 32 | +| [taskmanager.memory.process.size]({{< ref "docs/deployment/config#taskmanager-memory-process-size" >}}) | (none) | 65536m | +| [taskmanager.memory.managed.size]({{< ref "docs/deployment/config#taskmanager-memory-managed-size" >}}) | (none) | 16384m | +| [slotmanager.number-of-slots.min]({{< ref "docs/deployment/config#slotmanager-number-of-slots-min" >}}) | 0 | {taskManagerNumber * numberOfTaskSlots} | + +You can configure `slotmanager.number-of-slots.min` to a proper value as the reserved resource pool serving OLAP queries. TaskManager should configure with a large resource specification in OLAP scenario since this can put more computations in local and reduce network/deserialization/serialization overhead. Meanwhile, as a single point of calculation in OLAP, JobManager also prefer large resource specification. + +## Future Work +Flink OLAP is now part of [Apache Flink Roadmap](https://flink.apache.org/what-is-flink/roadmap/), which means the community will keep putting efforts to improve Flink OLAP, both in usability and query performance. Relevant work are traced in underlying tickets: - https://issues.apache.org/jira/browse/FLINK-25318 -- https://issues.apache.org/jira/browse/FLINK-32898 - -Furthermore, we are adding relevant OLAP benchmarks to the Flink repository such as [flink-benchmarks](https://github.com/apache/flink-benchmarks). \ No newline at end of file +- https://issues.apache.org/jira/browse/FLINK-32898 \ No newline at end of file diff --git a/docs/content/docs/dev/table/overview.md b/docs/content/docs/dev/table/overview.md index 6715d064c2faf..69bc97a1ccd19 100644 --- a/docs/content/docs/dev/table/overview.md +++ b/docs/content/docs/dev/table/overview.md @@ -60,7 +60,7 @@ Where to go next? * [Built-in Functions]({{< ref "docs/dev/table/functions/systemFunctions" >}}): Supported functions in Table API and SQL. * [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}): Play around with Flink SQL and submit a table program to a cluster without programming knowledge. * [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}): A service that enables the multiple clients to execute SQL from the remote in concurrency. -* [SQL JDBC Driver]({{< ref "docs/dev/table/jdbcDriver" >}}): A JDBC Driver that submits SQL statements to sql-gateway. * [OLAP Quickstart]({{< ref "docs/dev/table/olap_quickstart" >}}): A quickstart to show how to set up a Flink OLAP service. +* [SQL JDBC Driver]({{< ref "docs/dev/table/jdbcDriver" >}}): A JDBC Driver that submits SQL statements to sql-gateway. {{< top >}} diff --git a/docs/static/fig/olap-architecture.svg b/docs/static/fig/olap-architecture.svg new file mode 100644 index 0000000000000..2b39666ab86fe --- /dev/null +++ b/docs/static/fig/olap-architecture.svg @@ -0,0 +1,21 @@ + + + + \ No newline at end of file From ff5d8f08662bd50479039914e7a8b85ca539e6c1 Mon Sep 17 00:00:00 2001 From: Xiangyu Feng Date: Wed, 1 Nov 2023 19:16:15 +0800 Subject: [PATCH 067/104] [FLINK-33235][doc] Translate OLAP Quickstart to Chinese --- .../docs/dev/table/olap_quickstart.md | 185 +++++++++--------- docs/content.zh/docs/dev/table/overview.md | 3 +- 2 files changed, 98 insertions(+), 90 deletions(-) diff --git a/docs/content.zh/docs/dev/table/olap_quickstart.md b/docs/content.zh/docs/dev/table/olap_quickstart.md index e0b3afba2bc57..e109dfe71eebd 100644 --- a/docs/content.zh/docs/dev/table/olap_quickstart.md +++ b/docs/content.zh/docs/dev/table/olap_quickstart.md @@ -1,5 +1,5 @@ --- -title: "Quickstart for Flink OLAP" +title: "OLAP Quickstart" weight: 91 type: docs aliases: @@ -24,64 +24,73 @@ specific language governing permissions and limitations under the License. --> -# Introduction +# OLAP 搭建指南 -Flink OLAP has already added to [Apache Flink Roadmap](https://flink.apache.org/roadmap/). It means Flink can not only support streaming and batch computing, but also support OLAP(On-Line Analytical Processing). This page will show how to quickly set up a Flink OLAP service, and will introduce some best practices. +OLAP(OnLine Analysis Processing)是数据分析领域的一项关键技术,通常被用来对较大的数据集进行秒级的复杂查询分析。Flink 作为一款流批一体的计算引擎,现在也同样支持用户将其作为一个 OLAP 计算服务来部署。本文将会帮助你在本地快速搭建起一个 Flink OLAP 集群并试用。同时,也会介绍一些在实际生产环境中使用 Flink 作为 OLAP 计算服务的实践。 -## Architecture +## 架构介绍 -The Flink OLAP service consists of three parts: Client, Flink SQL Gateway, Flink Session Cluster. +本章节将会介绍 Flink 作为一个 OLAP 服务的总体架构及其在使用上的优势。 -* **Client**: Could be any client that can interact with Flink SQL Gateway, such as SQL client, Flink JDBC driver and so on. -* **Flink SQL Gateway**: The SQL Gateway provides an easy way to submit the Flink Job, look up the metadata, and analyze table stats. -* **Flink Session Cluster**: We choose session clusters to run OLAP queries, mainly to avoid the overhead of cluster startup. +### 架构 -## Advantage +Flink OLAP 服务整体由3个部分组成,包括:客户端,Flink SQL Gateway 和 Flink Session Cluster。 -* **Massively Parallel Processing** - * Flink OLAP runs naturally as an MPP(Massively Parallel Processing) system, which supports low-latency ad-hoc queries -* **Reuse Connectors** - * Flink OLAP can reuse rich connectors in Flink ecosystem. -* **Unified Engine** - * Unified computing engine for Streaming/Batch/OLAP. +* **客户端**: 可以是任何可以和 [Flink SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 交互的客户端,包括:[SQL Client]({{< ref "docs/dev/table/sqlClient" >}}),[Flink JDBC Driver]({{< ref "docs/dev/table/jdbcDriver" >}}) 等等; +* **Flink SQL Gateway**: Flink SQL Gateway 服务主要用作 SQL 解析、元数据获取、统计信息分析、Plan 优化和集群作业提交; +* **Flink Session Cluster**: OLAP 查询建议运行在 [Session 集群]({{< ref "/docs/deployment/resource-providers/native_kubernetes#starting-a-flink-session-on-kubernetes" >}})上,主要是可以减少集群启动时的额外开销; -# Deploying in Local Mode +{{< img src="/fig/olap-architecture.svg" alt="Illustration of Flink OLAP Architecture" width="85%" >}} -## Downloading Flink +### 优势 -The same as [Local Installation]({{< ref "docs/try-flink/local_installation" >}}). Flink runs on all UNIX-like environments, i.e. Linux, Mac OS X, and Cygwin (for Windows). We need to have at least Java 11 installed, Java 17 is more recommended in OLAP scenario. To check the Java version installed, type in your terminal: +* **并行计算架构** + * Flink 是天然的并行计算架构,执行 OLAP 查询时可以方便的通过调整并发来满足不同数据规模下的低延迟查询性能要求 +* **弹性资源管理** + * Flink 的集群资源具有良好的 Min、Max 扩缩容能力,可以根据集群负载动态调整所使用的资源 +* **生态丰富** + * Flink OLAP 可以复用 Flink 生态中丰富的 [连接器]({{< ref "docs/connectors/table/overview" >}}) +* **统一引擎** + * 支持流 / 批 / OLAP 的统一计算引擎 + +## 本地运行 +本章将指导用户如何在本地试用 Flink OLAP 服务。 + +### 下载 Flink + +这里的方法和[本地安装]({{< ref "docs/try-flink/local_installation" >}})中记录的步骤类似。Flink 可以运行在任何类 UNIX 的操作系统下面, 例如:Linux, Mac OS X 和 Cygwin (for Windows)。你需要在本地安装好 __Java 11__,可以通过下述命令行的方式检查安装好的 Java 版本: ``` java -version ``` -Next, [Download](https://flink.apache.org/downloads/) the latest binary release of Flink, then extract the archive: +下一步, [下载](https://flink.apache.org/downloads/) Flink 最新的二进制包并进行解压: ``` tar -xzf flink-*.tgz ``` -## Starting a local cluster +### 启动本地集群 -To start a local cluster, run the bash script that comes with Flink: +运行下述脚本,即可在本地启动集群: ``` ./bin/start-cluster.sh ``` -You should be able to navigate to the web UI at localhost:8081 to view the Flink dashboard and see that the cluster is up and running. +用户可以导航到本地的 Web UI(http://localhost:8081)来查看 Flink Dashboard 并检查集群是否已拉起和正在运行。 -## Start a SQL Client CLI +### 启动 SQL Client -You can start the CLI with an embedded gateway by calling: +用户可以通过运行下述命令,用命令行启动内嵌了 Gateway 的 SQL Client: ``` ./bin/sql-client.sh ``` -## Running Queries +### 运行 SQL 查询 -You could simply execute queries in CLI and retrieve the results. +通过命令行,用户可以方便的提交查询并获取结果: ``` SET 'sql-client.execution.result-mode' = 'tableau'; @@ -102,98 +111,98 @@ GROUP BY buyer ORDER BY total_cost LIMIT 3; ``` -And then you could find job detail information in web UI at localhost:8081. +具体的作业运行信息你可以通过访问本地的 Web UI(http://localhost:8081)来获取。 -# Deploying in Production +## 生产环境部署 -This section guides you through setting up a production ready Flink OLAP service. +这个章节会向你介绍一些在生产环境中使用 Flink OLAP 服务的建议。 -## Cluster Deployment +### 客户端 -In production, we recommend to use Flink Session Cluster, Flink SQL Gateway and Flink JDBC Driver to build an OLAP service. +#### Flink JDBC Driver -### Session Cluster +Flink JDBC Driver 提供了底层的连接管理能力,方便用户使用并向 SQL Gateway 提交查询请求。在实际的生产使用中,用户需要注意如何复用 JDBC 连接,来避免 Gateway 频繁的执行 Session 相关的创建及关闭操作,从而减少端到端的作业耗时。详细信息可以参考文档 [Flink JDBC Driver]({{ }})。 -For Flink Session Cluster, we recommend to deploy Flink on native Kubernetes using session mode. Kubernetes is a popular container-orchestration system for automating computer application deployment, scaling, and management. By deploying on native Kubernetes, Flink Session Cluster is able to dynamically allocate and de-allocate TaskManagers. For more information, please refer to [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes">}}). +### 集群部署 -### SQL Gateway +在生产环境中,建议使用 Flink Session 集群、Flink SQL Gateway 来搭建 OLAP 服务。 -For Flink SQL Gateway, we recommend deploying it as a stateless microservice and register this on the service discovery component. For more information, please refer to the [SQL Gateway Overview]({{< ref "docs/dev/table/sql-gateway/overview">}}). +#### Session Cluster -### Flink JDBC Driver +Flink Session 集群建议搭建在 Native Kubernetes 环境下,使用 Session 模式运行。K8S 作为一个流行的容器编排系统可以自动化的支持不同计算程序的部署、扩展和管理。通过将集群部署在 Native Kubernetes 上,Flink Session 集群支持动态的增减 TaskManagers。详细信息可以参考 [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes">}})。同时,你可以在 Session 集群中配置 [slotmanager.number-of-slots.min]({{< ref "docs/deployment/config#slotmanager-number-of-slots-min" >}}),这个可以帮助你显著减少 OLAP 查询执行的冷启动时间,详情请参阅 [FLIP-362](https://cwiki.apache.org/confluence/display/FLINK/FLIP-362%3A+Support+minimum+resource+limitation)。 -When submitting queries to SQL Gateway, we recommend using Flink JDBC Driver since it provides low-level connection management. When used in production, we need to pay attention to reuse the JDBC connection to avoid frequently creating/closing sessions in the Gateway. For more information, please refer to the [Flink JDBC Driver]({{{}}}). +#### Flink SQL Gateway -## Datasource Configurations +对于 Flink SQL Gateway,用户可以将其部署为无状态的微服务并注册到服务发现的组件上来对外提供服务,方便客户端可以进行负载均衡。详细信息可以参考 [SQL Gateway Overview]({{< ref "docs/dev/table/sql-gateway/overview">}})。 -### Catalogs +### 数据源配置 -In OLAP scenario, we recommend using FileCatalogStore in the catalog configuration introduced in [FLIP-295](https://cwiki.apache.org/confluence/display/FLINK/FLIP-295%3A+Support+lazy+initialization+of+catalogs+and+persistence+of+catalog+configurations). As a long running service, Flink OLAP cluster's catalog information will not change frequently and can be re-used cross sessions. For more information, please refer to the [Catalog Store]({{< ref "docs/dev/table/catalogs#catalog-store">}}). +#### Catalogs -### Connectors +在 OLAP 场景下,集群建议配置 [Catalogs]({{< ref "docs/dev/table/catalogs">}}) 中提供的 FileCatalogStore 作为 Catalog 选项。作为一个常驻服务,Flink OLAP 集群的元信息通常不会频繁变更而且需要支持跨 Session 的复用,这样可以减少元信息加载的冷启动时间。详细信息可以参考文档 [Catalog Store]({{< ref "docs/dev/table/catalogs#catalog-store">}})。 -Both Session Cluster and SQL Gateway rely on connectors to analyze table stats and read data from the configured data source. To add connectors, please refer to the [Connectors and Formats]({{< ref "docs/connectors/table/overview">}}). +#### 连接器 -## Cluster Configurations +Session Cluster 和 SQL Gateway 都依赖连接器来获取表的元信息同时从配置好的数据源读取数据,详细信息可以参考文档 [连接器]({{< ref "docs/connectors/table/overview" >}})。 -In OLAP scenario, we picked out a few configurations that can help improve user usability and query performance. +### 推荐参数配置 -### SQL&Table Options +对于 OLAP 场景,合理的参数配置可以帮助用户较大的提升服务总体的可用性和查询性能,下面列了一些生产环境建议的参数配置。 -| Parameters | Default | Recommended | -|:-------------------------------------|:--------|:------------| -| table.optimizer.join-reorder-enabled | false | true | -| pipeline.object-reuse | false | true | +#### SQL&Table 参数 -### Runtime Options +| 参数名称 | 默认值 | 推荐值 | +|:---------------------------------------------------------------------------------------------------------------|:------|:-----| +| [table.optimizer.join-reorder-enabled]({{}}) | false | true | +| [pipeline.object-reuse]({{< ref "docs/deployment/config#pipeline-object-reuse" >}}) | false | true | -| Parameters | Default | Recommended | -|:-----------------------------|:-----------------------|:------------------------------------------------------------------------------------------------------------------------------------------| -| execution.runtime-mode | STREAMING | BATCH | -| execution.batch-shuffle-mode | ALL_EXCHANGES_BLOCKING | ALL_EXCHANGES_PIPELINED | -| env.java.opts.all | {default value} | {default value} -XX:PerMethodRecompilationCutoff=10000 -XX:PerBytecodeRecompilationCutoff=10000-XX:ReservedCodeCacheSize=512M -XX:+UseZGC | -| JDK Version | 11 | 17 | +#### Runtime 参数 -We strongly recommend using JDK17 with ZGC in OLAP scenario in order to provide zero gc stw and solve the issue described in [FLINK-32746](https://issues.apache.org/jira/browse/FLINK-32746). +| 参数名称 | 默认值 | 推荐值 | +|:---------------------------------------------------------------------------------------------------------------------------------------------------------|:-----------------------|:------------------------------------------------------------------------------------------------------------------------------------------| +| [execution.runtime-mode]({{< ref "docs/deployment/config#execution-runtime-mode" >}}) | STREAMING | BATCH | +| [execution.batch-shuffle-mode]({{< ref "docs/deployment/config#execution-batch-shuffle-mode" >}}) | ALL_EXCHANGES_BLOCKING | ALL_EXCHANGES_PIPELINED | +| [env.java.opts.all]({{< ref "docs/deployment/config#env-java-opts-all" >}}) | {default value} | {default value} -XX:PerMethodRecompilationCutoff=10000 -XX:PerBytecodeRecompilationCutoff=10000-XX:ReservedCodeCacheSize=512M -XX:+UseZGC | +| JDK Version | 11 | 17 | -### Scheduling Options +推荐在 OLAP 生产环境中使用 JDK17 和 ZGC,ZGC 可以优化 Metaspace 区垃圾回收的问题,详见 [FLINK-32746](https://issues.apache.org/jira/browse/FLINK-32746)。同时 ZGC 在堆内内存垃圾回收时可以提供接近0毫秒的应用程序暂停时间。OLAP 查询在执行时需要使用批模式,因为 OLAP 查询的执行计划中可能同时出现 Pipelined 和 Blocking 属性的边。批模式下的调度器支持对作业分阶段调度,可以避免出现调度死锁问题。 -| Parameters | Default | Recommended | -|:---------------------------------------------------------|:------------------|:------------------| -| jobmanager.scheduler | Default | Default | -| jobmanager.execution.failover-strategy | region | full | -| restart-strategy.type | (none) | disable | -| jobstore.type | File | Memory | -| jobstore.max-capacity | Integer.MAX_VALUE | 500 | +#### Scheduling 参数 -We would like to highlight the usage of `PipelinedRegionSchedulingStrategy`. Since many OLAP queries will have blocking edges in their jobGraph. +| 参数名称 | 默认值 | 推荐值 | +|:------------------------------------------------------------------------------------------------------------------------|:------------------|:--------| +| [jobmanager.scheduler]({{< ref "docs/deployment/config#jobmanager-scheduler" >}}) | Default | Default | +| [jobmanager.execution.failover-strategy]({{< ref "docs/deployment/config#jobmanager-execution-failover-strategy-1" >}}) | region | full | +| [restart-strategy.type]({{< ref "docs/deployment/config#restart-strategy-type" >}}) | (none) | disable | +| [jobstore.type]({{< ref "docs/deployment/config#jobstore-type" >}}) | File | Memory | +| [jobstore.max-capacity]({{< ref "docs/deployment/config#jobstore-max-capacity" >}}) | Integer.MAX_VALUE | 500 | -### Network Options -| Parameters | Default | Recommended | -|:------------------------------------|:-----------|:---------------| -| rest.server.numThreads | 4 | 32 | -| web.refresh-interval | 3000 | 300000 | -| pekko.framesize | 10485760b | 104857600b | +#### 网络参数 -### ResourceManager Options +| 参数名称 | 默认值 | 推荐值 | +|:--------------------------------------------------------------------------------------|:----------|:-----------| +| [rest.server.numThreads]({{< ref "docs/deployment/config#rest-server-numthreads" >}}) | 4 | 32 | +| [web.refresh-interval]({{< ref "docs/deployment/config#web-refresh-interval" >}}) | 3000 | 300000 | +| [pekko.framesize]({{< ref "docs/deployment/config#pekko-framesize" >}}) | 10485760b | 104857600b | -| Parameters | Default | Recommended | -|:-------------------------------------|:----------|:---------------| -| kubernetes.jobmanager.replicas | 1 | 2 | -| kubernetes.jobmanager.cpu.amount | 1.0 | 16.0 | -| jobmanager.memory.process.size | (none) | 65536m | -| jobmanager.memory.jvm-overhead.max | 1g | 6144m | -| kubernetes.taskmanager.cpu.amount | (none) | 16 | -| taskmanager.numberOfTaskSlots | 1 | 32 | -| taskmanager.memory.process.size | (none) | 65536m | -| taskmanager.memory.managed.size | (none) | 65536m | +#### 资源管理参数 -We prefer to use large taskManager pods in OLAP since this can put more computation in local and reduce network/deserialization/serialization overhead. Meanwhile, since JobManager is a single point of calculation in OLAP scenario, we also prefer large pod. +| 参数名称 | 默认值 | 推荐值 | +|:--------------------------------------------------------------------------------------------------------------|:-------|:----------------------------------------| +| [kubernetes.jobmanager.replicas]({{< ref "docs/deployment/config#kubernetes-jobmanager-replicas" >}}) | 1 | 2 | +| [kubernetes.jobmanager.cpu.amount]({{< ref "docs/deployment/config#kubernetes-jobmanager-cpu-amount" >}}) | 1.0 | 16.0 | +| [jobmanager.memory.process.size]({{< ref "docs/deployment/config#jobmanager-memory-process-size" >}}) | (none) | 32g | +| [jobmanager.memory.jvm-overhead.max]({{< ref "docs/deployment/config#jobmanager-memory-jvm-overhead-max" >}}) | 1g | 3g | +| [kubernetes.taskmanager.cpu.amount]({{< ref "docs/deployment/config#kubernetes-taskmanager-cpu-amount" >}}) | (none) | 16 | +| [taskmanager.numberOfTaskSlots]({{< ref "docs/deployment/config#taskmanager-numberoftaskslots" >}}) | 1 | 32 | +| [taskmanager.memory.process.size]({{< ref "docs/deployment/config#taskmanager-memory-process-size" >}}) | (none) | 65536m | +| [taskmanager.memory.managed.size]({{< ref "docs/deployment/config#taskmanager-memory-managed-size" >}}) | (none) | 16384m | +| [slotmanager.number-of-slots.min]({{< ref "docs/deployment/config#slotmanager-number-of-slots-min" >}}) | 0 | {taskManagerNumber * numberOfTaskSlots} | -# Future Work -There is a big margin for improvement in Flink OLAP, both in usability and query performance, and we trace all of them in underlying tickets. +用户可以根据实际的生产情况把 `slotmanager.number-of-slots.min` 配置为一个合理值,并将其用作集群的预留资源池从而支持 OLAP 查询。在 OLAP 场景下,TaskManager 建议配置为较大的资源规格,因为这样可以把更多的计算放到本地从而减少网络 / 序列化 / 反序列化的开销。JobManager 因为是 OLAP 场景下的计算单点,也建议使用较大的资源规格。 + +## Future Work +作为 [Apache Flink Roadmap](https://flink.apache.org/what-is-flink/roadmap/) 的一部分,社区将会持续优化 Flink 在 OLAP 场景下易用性与可用性,提升查询性能与集群承载能力。相关的工作我们都会通过下面的 jira 追踪进展: - https://issues.apache.org/jira/browse/FLINK-25318 - https://issues.apache.org/jira/browse/FLINK-32898 - -Furthermore, we are adding relevant OLAP benchmarks to the Flink repository such as [flink-benchmarks](https://github.com/apache/flink-benchmarks). \ No newline at end of file diff --git a/docs/content.zh/docs/dev/table/overview.md b/docs/content.zh/docs/dev/table/overview.md index ce62e43498619..13d6dc88d73c5 100644 --- a/docs/content.zh/docs/dev/table/overview.md +++ b/docs/content.zh/docs/dev/table/overview.md @@ -53,8 +53,7 @@ and later use the DataStream API to build alerting based on the matched patterns * [内置函数]({{< ref "docs/dev/table/functions/systemFunctions" >}}): Table API 和 SQL 中的内置函数。 * [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}): 不用编写代码就可以尝试 Flink SQL,可以直接提交 SQL 任务到集群上。 * [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}): SQL 提交服务,支持多个客户端从远端并发提交 SQL 任务。 +* [OLAP Quickstart]({{< ref "docs/dev/table/olap_quickstart" >}}): Flink OLAP 服务搭建指南。 * [SQL Jdbc Driver]({{< ref "docs/dev/table/jdbcDriver" >}}): 标准JDBC Driver,可以提交Flink SQL作业到Sql Gateway。 -* [OLAP Quickstart]({{< ref "docs/dev/table/olap_quickstart" >}}): Flink OLAP服务搭建指南. - {{< top >}} From 32652e1ea10d2578f583f37aeb329e34ad0464c7 Mon Sep 17 00:00:00 2001 From: caodizhou Date: Tue, 24 Oct 2023 20:59:13 +0800 Subject: [PATCH 068/104] [FLINK-33160][network] Log the remote address when an exception occurs in the PartitionRequestQueue --- .../runtime/io/network/netty/PartitionRequestQueue.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index 31042cfac3abf..933e122899adf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -381,7 +381,10 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } private void handleException(Channel channel, Throwable cause) throws IOException { - LOG.error("Encountered error while consuming partitions", cause); + LOG.error( + "Encountered error while consuming partitions (connection to {})", + channel.remoteAddress(), + cause); fatalError = true; releaseAllResources(); From d9342f50ff002638dc11fef2b72ffd551e38a2d4 Mon Sep 17 00:00:00 2001 From: gongzhongqiang <764629910@qq.com> Date: Thu, 2 Nov 2023 16:08:36 +0800 Subject: [PATCH 069/104] [FLINK-32767][docs] Fix table path expression in SHOW CREATE TABLE usage This closes #23152. --- docs/content.zh/docs/dev/table/sql/show.md | 2 +- docs/content/docs/dev/table/sql/show.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/show.md b/docs/content.zh/docs/dev/table/sql/show.md index c6464b7081597..ca004ab19307e 100644 --- a/docs/content.zh/docs/dev/table/sql/show.md +++ b/docs/content.zh/docs/dev/table/sql/show.md @@ -608,7 +608,7 @@ show tables; ## SHOW CREATE TABLE ```sql -SHOW CREATE TABLE [catalog_name.][db_name.]table_name +SHOW CREATE TABLE [[catalog_name.]db_name.]table_name ``` 展示创建指定表的 create 语句。 diff --git a/docs/content/docs/dev/table/sql/show.md b/docs/content/docs/dev/table/sql/show.md index 310027245026d..bb5d4af3e2d9e 100644 --- a/docs/content/docs/dev/table/sql/show.md +++ b/docs/content/docs/dev/table/sql/show.md @@ -608,7 +608,7 @@ show tables; ## SHOW CREATE TABLE ```sql -SHOW CREATE TABLE +SHOW CREATE TABLE [[catalog_name.]db_name.]table_name ``` Show create table statement for specified table. From 9fcea6e61a99c673205baf21a1159647099fdf67 Mon Sep 17 00:00:00 2001 From: Wencong Liu Date: Wed, 6 Sep 2023 10:41:21 +0800 Subject: [PATCH 070/104] [FLINK-33041][docs] Add an article in English website to guide users to migrate their DataSet jobs to DataStream This closes #23362 --- .../docs/dev/datastream/dataset_migration.md | 774 ++++++++++++++++++ .../docs/dev/datastream/dataset_migration.md | 774 ++++++++++++++++++ 2 files changed, 1548 insertions(+) create mode 100644 docs/content.zh/docs/dev/datastream/dataset_migration.md create mode 100644 docs/content/docs/dev/datastream/dataset_migration.md diff --git a/docs/content.zh/docs/dev/datastream/dataset_migration.md b/docs/content.zh/docs/dev/datastream/dataset_migration.md new file mode 100644 index 0000000000000..25d1cb8f9a26b --- /dev/null +++ b/docs/content.zh/docs/dev/datastream/dataset_migration.md @@ -0,0 +1,774 @@ +--- +title: "How to Migrate from DataSet to DataStream" +weight: 302 +type: docs +--- + + +# How to Migrate from DataSet to DataStream + +The DataSet API has been formally deprecated and will no longer receive active maintenance and support. It will be removed in the +Flink 2.0 version. Flink users are recommended to migrate from the DataSet API to the DataStream API, Table API and SQL for their +data processing requirements. + +Noticed that APIs in DataStream do not always match those in DataSet exactly. The purpose of this document is to help users understand +how to achieve the same data processing behaviors with DataStream APIs as using DataSet APIs. + +According to the changes in coding and execution efficiency that are required for migration, we categorized DataSet APIs into 4 categories: + +- Category 1: APIs that have exact equivalent in DataStream, which requires barely any changes to migrate. + +- Category 2: APIs whose behavior can be achieved by other APIs with different semantics in DataStream, which might require some code changes for +migration but will result in the same execution efficiency. + +- Category 3: APIs whose behavior can be achieved by other APIs with different semantics in DataStream, with potentially additional cost in execution efficiency. + +- Category 4: APIs whose behaviors are not supported by DataStream API. + +The subsequent sections will first introduce how to set the execution environment and source/sink, then provide detailed explanations on how to migrate +each category of DataSet APIs to the DataStream APIs, highlighting the specific considerations and challenges associated with each +category. + + +## Setting the execution environment + +The first step of migrating an application from DataSet API to DataStream API is to replace `ExecutionEnvironment` with `StreamExecutionEnvironment`. + +

Heap.Max The maximum amount of heap memory that can be used for memory management (in bytes).
- This value might not be necessarily equal to the maximum value specified through -Xmx or - the equivalent Flink configuration parameter. Some GC algorithms allocate heap memory that won't + This value might not be necessarily equal to the maximum value specified through -Xmx or + the equivalent Flink configuration parameter. Some GC algorithms allocate heap memory that won't be available to the user code and, therefore, not being exposed through the heap metrics.
Gauge
Job-/TaskManagerStatus.JVM.GarbageCollector<GarbageCollector>.CountThe total number of collections that have occurred.Job-/TaskManagerStatus.JVM.GarbageCollector<Collector/All>.CountThe total number of collections that have occurred for the given (or all) collector.Gauge
<Collector/All>.TimeThe total time spent performing garbage collection for the given (or all) collector. Gauge
<GarbageCollector>.TimeThe total time spent performing garbage collection.<Collector/All>.TimeMsPerSecondThe time (in milliseconds) spent garbage collecting per second for the given (or all) collector. Gauge
Operator loopFrequencyHz stream, shardIdThe number of calls to getRecords in one second. + The number of calls to getRecords in one second. Gauge
numEffectiveSpeculativeExecutionsNumber of effective speculative execution attempts, i.e. speculative execution attempts which + Number of effective speculative execution attempts, i.e. speculative execution attempts which finish earlier than their corresponding original attempts. Counter
String The checkpoint storage implementation to be used to checkpoint state.
The implementation can be specified either via their shortcut name, or via the class name of a CheckpointStorageFactory. If a factory is specified it is instantiated via its zero argument constructor and its CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader) method is called.
Recognized shortcut names are 'jobmanager' and 'filesystem'.
state.checkpoint.cleaner.parallel-mode
trueBooleanOption whether to discard a checkpoint's states in parallel using the ExecutorService passed into the cleaner
state.checkpoints.dir
(none)Boolean This option configures local recovery for this state backend. By default, local recovery is deactivated. Local recovery currently only covers keyed state backends (including both the EmbeddedRocksDBStateBackend and the HashMapStateBackend).
state.checkpoint.cleaner.parallel-mode
trueBooleanOption whether to discard a checkpoint's states in parallel using the ExecutorService passed into the cleaner
state.checkpoints.num-retained
1Duration The maximum time the JobManager will wait to acquire all required resources after a job submission or restart. Once elapsed it will try to run the job with a lower parallelism, or fail if the minimum amount of resources could not be acquired.
Increasing this value will make the cluster more resilient against temporary resources shortages (e.g., there is more time for a failed TaskManager to be restarted).
Setting a negative duration will disable the resource timeout: The JobManager will wait indefinitely for resources to appear.
If scheduler-mode is configured to REACTIVE, this configuration value will default to a negative value to disable the resource timeout.
jobmanager.adaptive-scheduler.scaling-interval.max
(none)DurationDetermines the maximum interval time after which a scaling operation is forced even if the jobmanager.adaptive-scheduler.min-parallelism-increase aren't met. The scaling operation will be ignored when the resource hasn't changed. This option is disabled by default.
jobmanager.adaptive-scheduler.scaling-interval.min
30 sDurationDetermines the minimum time between scaling operations.
jobmanager.archive.fs.dir
(none)Duration The maximum time the JobManager will wait to acquire all required resources after a job submission or restart. Once elapsed it will try to run the job with a lower parallelism, or fail if the minimum amount of resources could not be acquired.
Increasing this value will make the cluster more resilient against temporary resources shortages (e.g., there is more time for a failed TaskManager to be restarted).
Setting a negative duration will disable the resource timeout: The JobManager will wait indefinitely for resources to appear.
If scheduler-mode is configured to REACTIVE, this configuration value will default to a negative value to disable the resource timeout.
jobmanager.adaptive-scheduler.scaling-interval.max
(none)DurationDetermines the maximum interval time after which a scaling operation is forced even if the jobmanager.adaptive-scheduler.min-parallelism-increase aren't met. The scaling operation will be ignored when the resource hasn't changed. This option is disabled by default.
jobmanager.adaptive-scheduler.scaling-interval.min
30 sDurationDetermines the minimum time between scaling operations.
jobmanager.partition.hybrid.partition-data-consume-constraint
(none)Duration The maximum time the JobManager will wait to acquire all required resources after a job submission or restart. Once elapsed it will try to run the job with a lower parallelism, or fail if the minimum amount of resources could not be acquired.
Increasing this value will make the cluster more resilient against temporary resources shortages (e.g., there is more time for a failed TaskManager to be restarted).
Setting a negative duration will disable the resource timeout: The JobManager will wait indefinitely for resources to appear.
If scheduler-mode is configured to REACTIVE, this configuration value will default to a negative value to disable the resource timeout.
jobmanager.adaptive-scheduler.scaling-interval.max
(none)DurationDetermines the maximum interval time after which a scaling operation is forced even if the jobmanager.adaptive-scheduler.min-parallelism-increase aren't met. The scaling operation will be ignored when the resource hasn't changed. This option is disabled by default.
jobmanager.adaptive-scheduler.scaling-interval.min
30 sDurationDetermines the minimum time between scaling operations.
jobmanager.archive.fs.dir
(none)String The Netty transport type, either "nio" or "epoll". The "auto" means selecting the property mode automatically based on the platform. Note that the "epoll" mode can get better performance, less GC and have more advanced features which are only available on modern Linux.
taskmanager.network.partition-request-timeout
10 sDurationTimeout for an individual partition request of remote input channels. The partition request will finally fail if the total wait time exceeds twice the value of taskmanager.network.request-backoff.max.
taskmanager.network.request-backoff.initial
100 IntegerMinimum backoff in milliseconds for partition requests of input channels.Minimum backoff in milliseconds for partition requests of local input channels.
taskmanager.network.request-backoff.max
10000 IntegerMaximum backoff in milliseconds for partition requests of input channels.Maximum backoff in milliseconds for partition requests of local input channels.
taskmanager.network.retries
String The Netty transport type, either "nio" or "epoll". The "auto" means selecting the property mode automatically based on the platform. Note that the "epoll" mode can get better performance, less GC and have more advanced features which are only available on modern Linux.
taskmanager.network.partition-request-timeout
10 sDurationTimeout for an individual partition request of remote input channels. The partition request will finally fail if the total wait time exceeds twice the value of taskmanager.network.request-backoff.max.
taskmanager.network.request-backoff.initial
100 IntegerMinimum backoff in milliseconds for partition requests of input channels.Minimum backoff in milliseconds for partition requests of local input channels.
taskmanager.network.request-backoff.max
10000 IntegerMaximum backoff in milliseconds for partition requests of input channels.Maximum backoff in milliseconds for partition requests of local input channels.
taskmanager.network.retries
+ + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Create the execution environment +ExecutionEnvironment.getExecutionEnvironment(); +// Create the local execution environment +ExecutionEnvironment.createLocalEnvironment(); +// Create the collection environment +new CollectionEnvironment(); +// Create the remote environment +ExecutionEnvironment.createRemoteEnvironment(String host, int port, String... jarFiles); + {{< /highlight >}} + + {{< highlight "java" >}} +// Create the execution environment +StreamExecutionEnvironment.getExecutionEnvironment(); +// Create the local execution environment +StreamExecutionEnvironment.createLocalEnvironment(); +// The collection environment is not supported. +// Create the remote environment +StreamExecutionEnvironment.createRemoteEnvironment(String host, int port, String... jarFiles); + {{< /highlight >}} +
+ +Unlike DataSet, DataStream supports processing on both bounded and unbounded data streams. Thus, user needs to explicitly set the execution mode +to `RuntimeExecutionMode.BATCH` if that is expected. + +```java +StreamExecutionEnvironment executionEnvironment = // [...]; +executionEnvironment.setRuntimeMode(RuntimeExecutionMode.BATCH); +``` + +## Using the streaming sources and sinks + +### Sources + +The DataStream API uses `DataStreamSource` to read records from external system, while the DataSet API uses the `DataSource`. + + + + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Read data from file +DataSource<> source = ExecutionEnvironment.readFile(inputFormat, filePath); +// Read data from collection +DataSource<> source = ExecutionEnvironment.fromCollection(data); +// Read data from inputformat +DataSource<> source = ExecutionEnvironment.createInput(inputFormat) + {{< /highlight >}} + + {{< highlight "java" >}} +// Read data from file +DataStreamSource<> source = StreamExecutionEnvironment.readFile(inputFormat, filePath); +// Read data from collection +DataStreamSource<> source = StreamExecutionEnvironment.fromCollection(data); +// Read data from inputformat +DataStreamSource<> source = StreamExecutionEnvironment.createInput(inputFormat) + {{< /highlight >}} +
+ +### Sinks + +The DataStream API uses `DataStreamSink` to write records to external system, while the +DataSet API uses the `DataSink`. + + + + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Write to outputformat +DataSink<> sink = dataSet.output(outputFormat); +// Write to csv file +DataSink<> sink = dataSet.writeAsCsv(filePath); +// Write to text file +DataSink<> sink = dataSet.writeAsText(filePath); + {{< /highlight >}} + + {{< highlight "java" >}} +// Write to sink +DataStreamSink<> sink = dataStream.sinkTo(sink) +// Write to csv file +DataStreamSink<> sink = dataStream.writeAsCsv(path); +// Write to text file +DataStreamSink<> sink = dataStream.writeAsText(path); + {{< /highlight >}} +
+ +If you are looking for pre-defined source and sink connectors of DataStream, please check the [Connector Docs]({{< ref "docs/connectors/datastream/overview" >}}) + +## Migrating DataSet APIs + +### Category 1 + +For Category 1, these DataSet APIs have exact equivalent in DataStream, which requires barely any changes to migrate. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
Map + {{< highlight "java" >}} +dataSet.map(new MapFunction<>(){ +// implement user-defined map logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.map(new MapFunction<>(){ +// implement user-defined map logic +}); + {{< /highlight >}} +
FlatMap + {{< highlight "java" >}} +dataSet.flatMap(new FlatMapFunction<>(){ +// implement user-defined flatmap logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.flatMap(new FlatMapFunction<>(){ +// implement user-defined flatmap logic +}); + {{< /highlight >}} +
Filter + {{< highlight "java" >}} +dataSet.filter(new FilterFunction<>(){ +// implement user-defined filter logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.filter(new FilterFunction<>(){ +// implement user-defined filter logic +}); + {{< /highlight >}} +
Union + {{< highlight "java" >}} +dataSet1.union(dataSet2); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream1.union(dataStream2); + {{< /highlight >}} +
Rebalance + {{< highlight "java" >}} +dataSet.rebalance(); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.rebalance(); + {{< /highlight >}} +
Project + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.project(2,0); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.project(2,0); + {{< /highlight >}} +
Reduce on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.groupBy(value -> value.f0) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.keyBy(value -> value.f0) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} +
Aggregate on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +// compute sum of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(SUM, 1); +// compute min of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(MIN, 1); +// compute max of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(MAX, 1); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// compute sum of the second field +dataStream.keyBy(value -> value.f0) + .sum(1); +// compute min of the second field +dataStream.keyBy(value -> value.f0) + .min(1); +// compute max of the second field +dataStream.keyBy(value -> value.f0) + .max(1); + {{< /highlight >}} +
+ +### Category 2 + +For category 2, the behavior of these DataSet APIs can be achieved by other APIs with different semantics in DataStream, which might require some code changes for +migration but will result in the same execution efficiency. + +Operations on a full DataSet correspond to the global window aggregation in DataStream with a custom window that is triggered at the end of the inputs. The `EndOfStreamWindows` +in the [Appendix]({{< ref "docs/dev/datastream/dataset_migration#endofstreamwindows" >}}) shows how such a window can be implemented. We will reuse it in the rest of this document. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
Distinct + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.distinct(); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.keyBy(value -> value) + .reduce((value1, value2) -> value1); + {{< /highlight >}} +
Hash-Partition + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.partitionByHash(value -> value.f0); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// partition by the hashcode of key +dataStream.partitionCustom( + (key, numSubpartition) -> key.hashCode() % numSubpartition, + value -> value.f0); + {{< /highlight >}} +
Reduce on Full DataSet + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.windowAll(EndOfStreamWindows.get()) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} +
Aggregate on Full DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +// compute sum of the second field +dataSet.aggregate(SUM, 1); +// compute min of the second field +dataSet.aggregate(MIN, 1); +// compute max of the second field +dataSet.aggregate(MAX, 1); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// compute sum of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .sum(1); +// compute min of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .min(1); +// compute max of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .max(1); + {{< /highlight >}} +
GroupReduce on Full DataSet + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.reduceGroup(new GroupReduceFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.windowAll(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} +
GroupReduce on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.groupBy(value -> value.f0) + .reduceGroup(new GroupReduceFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.keyBy(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} +
First-n + {{< highlight "java" >}} +dataSet.first(n) + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.windowAll(EndOfStreamWindows.get()) + .apply(new AllWindowFunction<>(){ + // implement first-n logic + }); + {{< /highlight >}} +
Join + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +dataSet1.join(dataSet2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .with(new JoinFunction<>(){ + // implement user-defined join logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream1 = // [...] +DataStream> dataStream2 = // [...] +dataStream1.join(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get())) + .apply(new JoinFunction<>(){ + // implement user-defined join logic + }); + {{< /highlight >}} +
CoGroup + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +dataSet1.coGroup(dataSet2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .with(new CoGroupFunction<>(){ + // implement user-defined co group logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream1 = // [...] +DataStream> dataStream2 = // [...] +dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get())) + .apply(new CoGroupFunction<>(){ + // implement user-defined co group logic + }); + {{< /highlight >}} +
OuterJoin + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +// left outer join +dataSet1.leftOuterJoin(dataSet2) + .where(dataSet1.f0) + .equalTo(dataSet2.f0) + .with(new JoinFunction<>(){ + // implement user-defined left outer join logic + }); +// right outer join +dataSet1.rightOuterJoin(dataSet2) + .where(dataSet1.f0) + .equalTo(dataSet2.f0) + .with(new JoinFunction<>(){ + // implement user-defined right outer join logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} + DataStream> dataStream1 = // [...] + DataStream> dataStream2 = // [...] + // left outer join + dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply((leftIterable, rightInterable, collector) -> { + if(!rightInterable.iterator().hasNext()){ + // implement user-defined left outer join logic + } + }); + // right outer join + dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply((leftIterable, rightInterable, collector) -> { + if(!leftIterable.iterator().hasNext()){ + // implement user-defined right outer join logic + } + }); + {{< /highlight >}} +
+ +### Category 3 + +For category 3, the behavior of these DataSet APIs can be achieved by other APIs with different semantics in DataStream, with potentially additional cost in execution efficiency. + +Currently, DataStream API does not directly support aggregations on non-keyed streams (subtask-scope aggregations). In order to do so, we need to first assign the subtask id +to the records, then turn the stream into a keyed stream. The `AddSubtaskIdMapFunction` in the [Appendix]({{< ref "docs/dev/datastream/dataset_migration#addsubtaskidmapfunction" >}}) shows how +to do that, and we will reuse it in the rest of this document. + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
MapPartition/SortPartition + {{< highlight "java" >}} +DataSet dataSet = // [...] +// MapPartition +dataSet.mapPartition(new MapPartitionFunction<>(){ + // implement user-defined map partition logic + }); +// SortPartition +dataSet.sortPartition(0, Order.ASCENDING); +dataSet.sortPartition(0, Order.DESCENDING); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +// assign subtask ID to all records +DataStream> dataStream1 = dataStream.map(new AddSubtaskIDMapFunction()); +dataStream1.keyBy(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined map partition or sort partition logic + }); + {{< /highlight >}} +
Cross + {{< highlight "java" >}} +DataSet dataSet1 = // [...] +DataSet dataSet2 = // [...] +// Cross +dataSet1.cross(dataSet2) + .with(new CrossFunction<>(){ + // implement user-defined cross logic + }) + {{< /highlight >}} + + {{< highlight "java" >}} +// the parallelism of dataStream1 and dataStream2 should be same +DataStream dataStream1 = // [...] +DataStream dataStream2 = // [...] +DataStream> datastream3 = dataStream1.broadcast().map(new AddSubtaskIDMapFunction()); +DataStream> datastream4 = dataStream2.map(new AddSubtaskIDMapFunction()); +// join the two streams according to the subtask ID +dataStream3.join(dataStream4) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new JoinFunction<>(){ + // implement user-defined cross logic + }) + {{< /highlight >}} +
+ +### Category 4 + +The behaviors of the following DataSet APIs are not supported by DataStream. + +* RangePartition +* GroupCombine + + +## Appendix + +#### EndOfStreamWindows + +The following code shows the example of `EndOfStreamWindows`. + +```java +public class EndOfStreamWindows extends WindowAssigner { + private static final long serialVersionUID = 1L; + + private static final EndOfStreamWindows INSTANCE = new EndOfStreamWindows(); + + private static final TimeWindow TIME_WINDOW_INSTANCE = + new TimeWindow(Long.MIN_VALUE, Long.MAX_VALUE); + + private EndOfStreamWindows() {} + + public static EndOfStreamWindows get() { + return INSTANCE; + } + + @Override + public Collection assignWindows( + Object element, long timestamp, WindowAssignerContext context) { + return Collections.singletonList(TIME_WINDOW_INSTANCE); + } + + @Override + public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { + return new EndOfStreamTrigger(); + } + + @Override + public String toString() { + return "EndOfStreamWindows()"; + } + + @Override + public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { + return new TimeWindow.Serializer(); + } + + @Override + public boolean isEventTime() { + return true; + } + + @Internal + public static class EndOfStreamTrigger extends Trigger { + @Override + public TriggerResult onElement( + Object element, long timestamp, TimeWindow window, TriggerContext ctx) + throws Exception { + return TriggerResult.CONTINUE; + } + + @Override + public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) { + return time == window.maxTimestamp() ? TriggerResult.FIRE : TriggerResult.CONTINUE; + } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception {} + + @Override + public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) { + return TriggerResult.CONTINUE; + } + } +} +``` + +#### AddSubtaskIDMapFunction + +The following code shows the example of `AddSubtaskIDMapFunction`. +```java +public static class AddSubtaskIDMapFunction extends RichMapFunction> { + @Override + public Tuple2 map(T value) { + return Tuple2.of(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()), value); + } +} +``` + +{{< top >}} diff --git a/docs/content/docs/dev/datastream/dataset_migration.md b/docs/content/docs/dev/datastream/dataset_migration.md new file mode 100644 index 0000000000000..25d1cb8f9a26b --- /dev/null +++ b/docs/content/docs/dev/datastream/dataset_migration.md @@ -0,0 +1,774 @@ +--- +title: "How to Migrate from DataSet to DataStream" +weight: 302 +type: docs +--- + + +# How to Migrate from DataSet to DataStream + +The DataSet API has been formally deprecated and will no longer receive active maintenance and support. It will be removed in the +Flink 2.0 version. Flink users are recommended to migrate from the DataSet API to the DataStream API, Table API and SQL for their +data processing requirements. + +Noticed that APIs in DataStream do not always match those in DataSet exactly. The purpose of this document is to help users understand +how to achieve the same data processing behaviors with DataStream APIs as using DataSet APIs. + +According to the changes in coding and execution efficiency that are required for migration, we categorized DataSet APIs into 4 categories: + +- Category 1: APIs that have exact equivalent in DataStream, which requires barely any changes to migrate. + +- Category 2: APIs whose behavior can be achieved by other APIs with different semantics in DataStream, which might require some code changes for +migration but will result in the same execution efficiency. + +- Category 3: APIs whose behavior can be achieved by other APIs with different semantics in DataStream, with potentially additional cost in execution efficiency. + +- Category 4: APIs whose behaviors are not supported by DataStream API. + +The subsequent sections will first introduce how to set the execution environment and source/sink, then provide detailed explanations on how to migrate +each category of DataSet APIs to the DataStream APIs, highlighting the specific considerations and challenges associated with each +category. + + +## Setting the execution environment + +The first step of migrating an application from DataSet API to DataStream API is to replace `ExecutionEnvironment` with `StreamExecutionEnvironment`. + + + + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Create the execution environment +ExecutionEnvironment.getExecutionEnvironment(); +// Create the local execution environment +ExecutionEnvironment.createLocalEnvironment(); +// Create the collection environment +new CollectionEnvironment(); +// Create the remote environment +ExecutionEnvironment.createRemoteEnvironment(String host, int port, String... jarFiles); + {{< /highlight >}} + + {{< highlight "java" >}} +// Create the execution environment +StreamExecutionEnvironment.getExecutionEnvironment(); +// Create the local execution environment +StreamExecutionEnvironment.createLocalEnvironment(); +// The collection environment is not supported. +// Create the remote environment +StreamExecutionEnvironment.createRemoteEnvironment(String host, int port, String... jarFiles); + {{< /highlight >}} +
+ +Unlike DataSet, DataStream supports processing on both bounded and unbounded data streams. Thus, user needs to explicitly set the execution mode +to `RuntimeExecutionMode.BATCH` if that is expected. + +```java +StreamExecutionEnvironment executionEnvironment = // [...]; +executionEnvironment.setRuntimeMode(RuntimeExecutionMode.BATCH); +``` + +## Using the streaming sources and sinks + +### Sources + +The DataStream API uses `DataStreamSource` to read records from external system, while the DataSet API uses the `DataSource`. + + + + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Read data from file +DataSource<> source = ExecutionEnvironment.readFile(inputFormat, filePath); +// Read data from collection +DataSource<> source = ExecutionEnvironment.fromCollection(data); +// Read data from inputformat +DataSource<> source = ExecutionEnvironment.createInput(inputFormat) + {{< /highlight >}} + + {{< highlight "java" >}} +// Read data from file +DataStreamSource<> source = StreamExecutionEnvironment.readFile(inputFormat, filePath); +// Read data from collection +DataStreamSource<> source = StreamExecutionEnvironment.fromCollection(data); +// Read data from inputformat +DataStreamSource<> source = StreamExecutionEnvironment.createInput(inputFormat) + {{< /highlight >}} +
+ +### Sinks + +The DataStream API uses `DataStreamSink` to write records to external system, while the +DataSet API uses the `DataSink`. + + + + + + + + + + + + + + +
DataSetDataStream
+ {{< highlight "java" >}} +// Write to outputformat +DataSink<> sink = dataSet.output(outputFormat); +// Write to csv file +DataSink<> sink = dataSet.writeAsCsv(filePath); +// Write to text file +DataSink<> sink = dataSet.writeAsText(filePath); + {{< /highlight >}} + + {{< highlight "java" >}} +// Write to sink +DataStreamSink<> sink = dataStream.sinkTo(sink) +// Write to csv file +DataStreamSink<> sink = dataStream.writeAsCsv(path); +// Write to text file +DataStreamSink<> sink = dataStream.writeAsText(path); + {{< /highlight >}} +
+ +If you are looking for pre-defined source and sink connectors of DataStream, please check the [Connector Docs]({{< ref "docs/connectors/datastream/overview" >}}) + +## Migrating DataSet APIs + +### Category 1 + +For Category 1, these DataSet APIs have exact equivalent in DataStream, which requires barely any changes to migrate. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
Map + {{< highlight "java" >}} +dataSet.map(new MapFunction<>(){ +// implement user-defined map logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.map(new MapFunction<>(){ +// implement user-defined map logic +}); + {{< /highlight >}} +
FlatMap + {{< highlight "java" >}} +dataSet.flatMap(new FlatMapFunction<>(){ +// implement user-defined flatmap logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.flatMap(new FlatMapFunction<>(){ +// implement user-defined flatmap logic +}); + {{< /highlight >}} +
Filter + {{< highlight "java" >}} +dataSet.filter(new FilterFunction<>(){ +// implement user-defined filter logic +}); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.filter(new FilterFunction<>(){ +// implement user-defined filter logic +}); + {{< /highlight >}} +
Union + {{< highlight "java" >}} +dataSet1.union(dataSet2); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream1.union(dataStream2); + {{< /highlight >}} +
Rebalance + {{< highlight "java" >}} +dataSet.rebalance(); + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.rebalance(); + {{< /highlight >}} +
Project + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.project(2,0); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.project(2,0); + {{< /highlight >}} +
Reduce on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.groupBy(value -> value.f0) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.keyBy(value -> value.f0) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} +
Aggregate on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +// compute sum of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(SUM, 1); +// compute min of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(MIN, 1); +// compute max of the second field +dataSet.groupBy(value -> value.f0) + .aggregate(MAX, 1); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// compute sum of the second field +dataStream.keyBy(value -> value.f0) + .sum(1); +// compute min of the second field +dataStream.keyBy(value -> value.f0) + .min(1); +// compute max of the second field +dataStream.keyBy(value -> value.f0) + .max(1); + {{< /highlight >}} +
+ +### Category 2 + +For category 2, the behavior of these DataSet APIs can be achieved by other APIs with different semantics in DataStream, which might require some code changes for +migration but will result in the same execution efficiency. + +Operations on a full DataSet correspond to the global window aggregation in DataStream with a custom window that is triggered at the end of the inputs. The `EndOfStreamWindows` +in the [Appendix]({{< ref "docs/dev/datastream/dataset_migration#endofstreamwindows" >}}) shows how such a window can be implemented. We will reuse it in the rest of this document. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
Distinct + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.distinct(); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.keyBy(value -> value) + .reduce((value1, value2) -> value1); + {{< /highlight >}} +
Hash-Partition + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.partitionByHash(value -> value.f0); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// partition by the hashcode of key +dataStream.partitionCustom( + (key, numSubpartition) -> key.hashCode() % numSubpartition, + value -> value.f0); + {{< /highlight >}} +
Reduce on Full DataSet + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.windowAll(EndOfStreamWindows.get()) + .reduce(new ReduceFunction<>(){ + // implement user-defined reduce logic + }); + {{< /highlight >}} +
Aggregate on Full DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +// compute sum of the second field +dataSet.aggregate(SUM, 1); +// compute min of the second field +dataSet.aggregate(MIN, 1); +// compute max of the second field +dataSet.aggregate(MAX, 1); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +// compute sum of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .sum(1); +// compute min of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .min(1); +// compute max of the second field +dataStream.windowAll(EndOfStreamWindows.get()) + .max(1); + {{< /highlight >}} +
GroupReduce on Full DataSet + {{< highlight "java" >}} +DataSet dataSet = // [...] +dataSet.reduceGroup(new GroupReduceFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +dataStream.windowAll(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} +
GroupReduce on Grouped DataSet + {{< highlight "java" >}} +DataSet> dataSet = // [...] +dataSet.groupBy(value -> value.f0) + .reduceGroup(new GroupReduceFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream = // [...] +dataStream.keyBy(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined group reduce logic + }); + {{< /highlight >}} +
First-n + {{< highlight "java" >}} +dataSet.first(n) + {{< /highlight >}} + + {{< highlight "java" >}} +dataStream.windowAll(EndOfStreamWindows.get()) + .apply(new AllWindowFunction<>(){ + // implement first-n logic + }); + {{< /highlight >}} +
Join + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +dataSet1.join(dataSet2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .with(new JoinFunction<>(){ + // implement user-defined join logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream1 = // [...] +DataStream> dataStream2 = // [...] +dataStream1.join(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get())) + .apply(new JoinFunction<>(){ + // implement user-defined join logic + }); + {{< /highlight >}} +
CoGroup + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +dataSet1.coGroup(dataSet2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .with(new CoGroupFunction<>(){ + // implement user-defined co group logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream> dataStream1 = // [...] +DataStream> dataStream2 = // [...] +dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get())) + .apply(new CoGroupFunction<>(){ + // implement user-defined co group logic + }); + {{< /highlight >}} +
OuterJoin + {{< highlight "java" >}} +DataSet> dataSet1 = // [...] +DataSet> dataSet2 = // [...] +// left outer join +dataSet1.leftOuterJoin(dataSet2) + .where(dataSet1.f0) + .equalTo(dataSet2.f0) + .with(new JoinFunction<>(){ + // implement user-defined left outer join logic + }); +// right outer join +dataSet1.rightOuterJoin(dataSet2) + .where(dataSet1.f0) + .equalTo(dataSet2.f0) + .with(new JoinFunction<>(){ + // implement user-defined right outer join logic + }); + {{< /highlight >}} + + {{< highlight "java" >}} + DataStream> dataStream1 = // [...] + DataStream> dataStream2 = // [...] + // left outer join + dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply((leftIterable, rightInterable, collector) -> { + if(!rightInterable.iterator().hasNext()){ + // implement user-defined left outer join logic + } + }); + // right outer join + dataStream1.coGroup(dataStream2) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply((leftIterable, rightInterable, collector) -> { + if(!leftIterable.iterator().hasNext()){ + // implement user-defined right outer join logic + } + }); + {{< /highlight >}} +
+ +### Category 3 + +For category 3, the behavior of these DataSet APIs can be achieved by other APIs with different semantics in DataStream, with potentially additional cost in execution efficiency. + +Currently, DataStream API does not directly support aggregations on non-keyed streams (subtask-scope aggregations). In order to do so, we need to first assign the subtask id +to the records, then turn the stream into a keyed stream. The `AddSubtaskIdMapFunction` in the [Appendix]({{< ref "docs/dev/datastream/dataset_migration#addsubtaskidmapfunction" >}}) shows how +to do that, and we will reuse it in the rest of this document. + + + + + + + + + + + + + + + + + + + + + +
OperationsDataSetDataStream
MapPartition/SortPartition + {{< highlight "java" >}} +DataSet dataSet = // [...] +// MapPartition +dataSet.mapPartition(new MapPartitionFunction<>(){ + // implement user-defined map partition logic + }); +// SortPartition +dataSet.sortPartition(0, Order.ASCENDING); +dataSet.sortPartition(0, Order.DESCENDING); + {{< /highlight >}} + + {{< highlight "java" >}} +DataStream dataStream = // [...] +// assign subtask ID to all records +DataStream> dataStream1 = dataStream.map(new AddSubtaskIDMapFunction()); +dataStream1.keyBy(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new WindowFunction<>(){ + // implement user-defined map partition or sort partition logic + }); + {{< /highlight >}} +
Cross + {{< highlight "java" >}} +DataSet dataSet1 = // [...] +DataSet dataSet2 = // [...] +// Cross +dataSet1.cross(dataSet2) + .with(new CrossFunction<>(){ + // implement user-defined cross logic + }) + {{< /highlight >}} + + {{< highlight "java" >}} +// the parallelism of dataStream1 and dataStream2 should be same +DataStream dataStream1 = // [...] +DataStream dataStream2 = // [...] +DataStream> datastream3 = dataStream1.broadcast().map(new AddSubtaskIDMapFunction()); +DataStream> datastream4 = dataStream2.map(new AddSubtaskIDMapFunction()); +// join the two streams according to the subtask ID +dataStream3.join(dataStream4) + .where(value -> value.f0) + .equalTo(value -> value.f0) + .window(EndOfStreamWindows.get()) + .apply(new JoinFunction<>(){ + // implement user-defined cross logic + }) + {{< /highlight >}} +
+ +### Category 4 + +The behaviors of the following DataSet APIs are not supported by DataStream. + +* RangePartition +* GroupCombine + + +## Appendix + +#### EndOfStreamWindows + +The following code shows the example of `EndOfStreamWindows`. + +```java +public class EndOfStreamWindows extends WindowAssigner { + private static final long serialVersionUID = 1L; + + private static final EndOfStreamWindows INSTANCE = new EndOfStreamWindows(); + + private static final TimeWindow TIME_WINDOW_INSTANCE = + new TimeWindow(Long.MIN_VALUE, Long.MAX_VALUE); + + private EndOfStreamWindows() {} + + public static EndOfStreamWindows get() { + return INSTANCE; + } + + @Override + public Collection assignWindows( + Object element, long timestamp, WindowAssignerContext context) { + return Collections.singletonList(TIME_WINDOW_INSTANCE); + } + + @Override + public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { + return new EndOfStreamTrigger(); + } + + @Override + public String toString() { + return "EndOfStreamWindows()"; + } + + @Override + public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { + return new TimeWindow.Serializer(); + } + + @Override + public boolean isEventTime() { + return true; + } + + @Internal + public static class EndOfStreamTrigger extends Trigger { + @Override + public TriggerResult onElement( + Object element, long timestamp, TimeWindow window, TriggerContext ctx) + throws Exception { + return TriggerResult.CONTINUE; + } + + @Override + public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) { + return time == window.maxTimestamp() ? TriggerResult.FIRE : TriggerResult.CONTINUE; + } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception {} + + @Override + public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) { + return TriggerResult.CONTINUE; + } + } +} +``` + +#### AddSubtaskIDMapFunction + +The following code shows the example of `AddSubtaskIDMapFunction`. +```java +public static class AddSubtaskIDMapFunction extends RichMapFunction> { + @Override + public Tuple2 map(T value) { + return Tuple2.of(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()), value); + } +} +``` + +{{< top >}} From 898935d75b2fc073bcee61758b3b0443cba584ca Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Mon, 30 Oct 2023 17:20:41 -0700 Subject: [PATCH 071/104] [FLINK-33422] Implement restore tests for Calc node --- .../table/test/program/TableTestProgram.java | 5 + .../nodes/exec/testutils/CalcRestoreTest.java | 11 +- .../exec/testutils/CalcTestPrograms.java | 166 +++++++++++ .../nodes/exec/testutils/RestoreTestBase.java | 13 +- .../plan/calc-filter-pushdown.json | 92 ++++++ .../calc-filter-pushdown/savepoint/_metadata | Bin 0 -> 6981 bytes .../calc-filter/plan/calc-filter.json | 141 +++++++++ .../calc-filter/savepoint/_metadata | Bin 0 -> 9372 bytes .../plan/calc-project-pushdown.json | 135 +++++++++ .../calc-project-pushdown/savepoint/_metadata | Bin 0 -> 7090 bytes .../calc-sarg/plan/calc-sarg.json | 132 +++++++++ .../calc-sarg/savepoint/_metadata | Bin 0 -> 5874 bytes .../plan/calc-udf-complex.json | 272 ++++++++++++++++++ .../calc-udf-complex/savepoint/_metadata | Bin 0 -> 13385 bytes .../calc-udf-simple/plan/calc-udf-simple.json | 111 +++++++ .../calc-udf-simple/savepoint/_metadata | Bin 0 -> 7016 bytes 16 files changed, 1073 insertions(+), 5 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/plan/calc-filter-pushdown.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/plan/calc-filter.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-project-pushdown/plan/calc-project-pushdown.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-project-pushdown/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/plan/calc-sarg.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/plan/calc-udf-complex.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/plan/calc-udf-simple.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/savepoint/_metadata diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java index 731c967bba108..5af08b2999888 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -103,6 +103,11 @@ private TableTestProgram( this.runSteps = runSteps; } + @Override + public String toString() { + return id; + } + /** * Entrypoint for a {@link TableTestProgram} that forces an identifier and description of the * test program. diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java index 117316aa8e1e6..2b408bfcd675d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcRestoreTest.java @@ -21,7 +21,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; import org.apache.flink.table.test.program.TableTestProgram; -import java.util.Collections; +import java.util.Arrays; import java.util.List; /** Restore tests for {@link StreamExecCalc}. */ @@ -33,6 +33,13 @@ public CalcRestoreTest() { @Override public List programs() { - return Collections.singletonList(CalcTestPrograms.SIMPLE_CALC); + return Arrays.asList( + CalcTestPrograms.SIMPLE_CALC, + CalcTestPrograms.CALC_FILTER, + CalcTestPrograms.CALC_FILTER_PUSHDOWN, + CalcTestPrograms.CALC_PROJECT_PUSHDOWN, + CalcTestPrograms.CALC_SARG, + CalcTestPrograms.CALC_UDF_SIMPLE, + CalcTestPrograms.CALC_UDF_COMPLEX); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java index feb151ba9ef84..51d23f3183cd9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/CalcTestPrograms.java @@ -19,11 +19,18 @@ package org.apache.flink.table.planner.plan.nodes.exec.testutils; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc1; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc2; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.UdfWithOpen; import org.apache.flink.table.test.program.SinkTestStep; import org.apache.flink.table.test.program.SourceTestStep; import org.apache.flink.table.test.program.TableTestProgram; import org.apache.flink.types.Row; +import java.time.LocalDateTime; + /** {@link TableTestProgram} definitions for testing {@link StreamExecCalc}. */ public class CalcTestPrograms { @@ -43,4 +50,163 @@ public class CalcTestPrograms { .build()) .runSql("INSERT INTO sink_t SELECT a + 1, b FROM t") .build(); + + static final TableTestProgram CALC_PROJECT_PUSHDOWN = + TableTestProgram.of( + "calc-project-pushdown", "validates calc node with project pushdown") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a BIGINT", "b DOUBLE") + .addOption("filterable-fields", "a") + .producedBeforeRestore(Row.of(421L, 42.1)) + .producedAfterRestore(Row.of(421L, 42.1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a BIGINT", "a1 VARCHAR") + .consumedBeforeRestore(Row.of(421L, "421")) + .consumedAfterRestore(Row.of(421L, "421")) + .build()) + .runSql( + "INSERT INTO sink_t SELECT a, CAST(a AS VARCHAR) FROM source_t WHERE a > CAST(1 AS BIGINT)") + .build(); + + static final TableTestProgram CALC_FILTER = + TableTestProgram.of("calc-filter", "validates calc node with filter") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a BIGINT", "b INT", "c DOUBLE", "d VARCHAR") + .producedBeforeRestore(Row.of(420L, 1, 42.0, "hello")) + .producedAfterRestore(Row.of(420L, 1, 42.0, "hello")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a BIGINT", "b INT", "c DOUBLE", "d VARCHAR") + .consumedBeforeRestore(Row.of(420L, 1, 42.0, "hello")) + .consumedAfterRestore(Row.of(420L, 1, 42.0, "hello")) + .build()) + .runSql("INSERT INTO sink_t SELECT * FROM source_t WHERE b > 0") + .build(); + + static final TableTestProgram CALC_FILTER_PUSHDOWN = + TableTestProgram.of("calc-filter-pushdown", "validates calc node with filter pushdown") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a BIGINT", "b DOUBLE") + .addOption("filterable-fields", "a") + .producedBeforeRestore(Row.of(421L, 42.1)) + .producedAfterRestore(Row.of(421L, 42.1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a BIGINT", "b DOUBLE") + .consumedBeforeRestore(Row.of(421L, 42.1)) + .consumedAfterRestore(Row.of(421L, 42.1)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT a, b FROM source_t WHERE a > CAST(420 AS BIGINT)") + .build(); + + static final TableTestProgram CALC_SARG = + TableTestProgram.of("calc-sarg", "validates calc node with Sarg") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a INT") + .addOption("filterable-fields", "a") + .producedBeforeRestore(Row.of(1)) + .producedAfterRestore(Row.of(1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a INT") + .consumedBeforeRestore(Row.of(1)) + .consumedAfterRestore(Row.of(1)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT a FROM source_t WHERE a = 1 or a = 2 or a is null") + .build(); + + static final TableTestProgram CALC_UDF_SIMPLE = + TableTestProgram.of("calc-udf-simple", "validates calc node with simple UDF") + .setupTemporaryCatalogFunction("udf1", JavaFunc0.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a INT") + .producedBeforeRestore(Row.of(5)) + .producedAfterRestore(Row.of(5)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a INT", "a1 BIGINT") + .consumedBeforeRestore(Row.of(5, 6L)) + .consumedAfterRestore(Row.of(5, 6L)) + .build()) + .runSql("INSERT INTO sink_t SELECT a, udf1(a) FROM source_t") + .build(); + + static final TableTestProgram CALC_UDF_COMPLEX = + TableTestProgram.of("calc-udf-complex", "validates calc node with complex UDFs") + .setupTemporaryCatalogFunction("udf1", JavaFunc0.class) + .setupTemporaryCatalogFunction("udf2", JavaFunc1.class) + .setupTemporarySystemFunction("udf3", JavaFunc2.class) + .setupTemporarySystemFunction("udf4", UdfWithOpen.class) + .setupCatalogFunction("udf5", JavaFunc5.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema( + "a BIGINT, b INT NOT NULL, c VARCHAR, d TIMESTAMP(3)") + .producedBeforeRestore( + Row.of( + 5L, + 11, + "hello world", + LocalDateTime.of(2023, 12, 16, 1, 1, 1, 123))) + .producedAfterRestore( + Row.of( + 5L, + 11, + "hello world", + LocalDateTime.of(2023, 12, 16, 1, 1, 1, 123))) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema( + "a BIGINT", + "a1 VARCHAR", + "b INT NOT NULL", + "b1 VARCHAR", + "c1 VARCHAR", + "c2 VARCHAR", + "d1 TIMESTAMP(3)") + .consumedBeforeRestore( + Row.of( + 5L, + "5", + 11, + "11 and 11 and 1702688461000", + "hello world11", + "$hello", + LocalDateTime.of(2023, 12, 16, 01, 01, 00, 0))) + .consumedAfterRestore( + Row.of( + 5L, + "5", + 11, + "11 and 11 and 1702688461000", + "hello world11", + "$hello", + LocalDateTime.of(2023, 12, 16, 01, 01, 00, 0))) + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "a, " + + "cast(a as VARCHAR) as a1, " + + "b, " + + "udf2(b, b, d) as b1, " + + "udf3(c, b) as c1, " + + "udf4(substring(c, 1, 5)) as c2, " + + "udf5(d, 1000) as d1 " + + "from source_t where " + + "(udf1(a) > 0 or (a * b) < 100) and b > 10") + .build(); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java index 23e0926140cc4..f5c370668e22a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java @@ -88,7 +88,10 @@ protected RestoreTestBase(Class execNodeUnderTest) { @Override public EnumSet supportedSetupSteps() { - return EnumSet.of(TestKind.SOURCE_WITH_RESTORE_DATA, TestKind.SINK_WITH_RESTORE_DATA); + return EnumSet.of( + TestKind.FUNCTION, + TestKind.SOURCE_WITH_RESTORE_DATA, + TestKind.SINK_WITH_RESTORE_DATA); } @Override @@ -110,7 +113,7 @@ private Stream createSpecs() { return getAllMetadata().stream() .flatMap( metadata -> - supportedPrograms().stream().map(p -> Arguments.of(metadata, p))); + supportedPrograms().stream().map(p -> Arguments.of(p, metadata))); } /** @@ -161,6 +164,8 @@ public void generateTestSetupFiles(TableTestProgram program) throws Exception { sinkTestStep.apply(tEnv, options); } + program.getSetupFunctionTestSteps().forEach(s -> s.apply(tEnv)); + final SqlTestStep sqlTestStep = program.getRunSqlTestStep(); final CompiledPlan compiledPlan = tEnv.compilePlanSql(sqlTestStep.sql); @@ -182,7 +187,7 @@ public void generateTestSetupFiles(TableTestProgram program) throws Exception { @ParameterizedTest @MethodSource("createSpecs") - void testRestore(ExecNodeMetadata metadata, TableTestProgram program) throws Exception { + void testRestore(TableTestProgram program, ExecNodeMetadata metadata) throws Exception { final EnvironmentSettings settings = EnvironmentSettings.inStreamingMode(); final SavepointRestoreSettings restoreSettings = SavepointRestoreSettings.forPath( @@ -213,6 +218,8 @@ void testRestore(ExecNodeMetadata metadata, TableTestProgram program) throws Exc sinkTestStep.apply(tEnv, options); } + program.getSetupFunctionTestSteps().forEach(s -> s.apply(tEnv)); + final CompiledPlan compiledPlan = tEnv.loadPlan(PlanReference.fromFile(getPlanPath(program, metadata))); compiledPlan.execute().await(); diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/plan/calc-filter-pushdown.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/plan/calc-filter-pushdown.json new file mode 100644 index 0000000000000..9327810d35e79 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/plan/calc-filter-pushdown.json @@ -0,0 +1,92 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "DOUBLE" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "LITERAL", + "value" : 420, + "type" : "BIGINT NOT NULL" + } ], + "type" : "BOOLEAN" + } ] + } ] + }, + "outputType" : "ROW<`a` BIGINT, `b` DOUBLE>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[>(a, 420:BIGINT)]]], fields=[a, b])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "DOUBLE" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` DOUBLE>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter-pushdown/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c2093c6d15df96c5939a33f0de8185e1209398fa GIT binary patch literal 6981 zcmeHMTZkJ~7(U5v*WR|9wp+2aEsNb&ip@+ixoqNsnCzxDb($D4x4NJhViVTGc&Ub9{k6Th&z>4ToYv zJQj!tg7LAi5Df*#cnjA^#ZYzX7a~|HDt=i|%Fam0Q3%`L?)n=xFQ?gU3T%wT8&E^Y~Bmo0&b4 zolGHLeo9OiS{*5Sp6_HroKDT=Q@M0POwXip(CS5|VU{>t!O|Sz6iwCV;YQ^oqg*z0 zjxJP4m8zP>6|qG)Y?^q1OD<3{WvXSBsx^{eO?hIfSW_2?Il5M7UdI({&Y&Q-4}NV& z*0*$V(I0v0^jD)-#790DgidDd5*ySOQ7;;{Kuf<#sEH*CXRE4F%QbHJ2ej20va)N} zmVZ0$a!@8xL<#rC@4gcA@qtAMQGQ9pg+WYb>set(4XuGJ;gR59Hy0fV*yo2A^%kSFvXoF>dRYRZ9um!-|8^9XYOT67QE}|`(QBozWF|{ZXM3BQbmI`k#72ft2Z(G4TV|lSP zXS`minP^X^dbZg)<2++69DQEY57KN~lM5MK25CbsLr>t!_!%_h^iLTUEImV~Xy+N1 z(%Ir!B2mz~e_qvPiblGq%jEJP_i-2jBezu?sHV`}E}y?zZ^n!Y=7dw>@3?j&0J^od{g+b|x9 zqI+EtZVhXMcV`RP-qzXrs$cwk-%B6km-~<>jruLvT3}- zQ!N*X`OLK!_icIM%S#NYVr#YE!w^U8>%)NSn$j?H>D`ZxAD;iMACM=T$ZX0Hd&;Oi z{ZJjD;ZuKrfsMykVU;L#K5&31%u3hurlY0paRXd*9yNc53IDl`$@6b)Ob+Ak$ba3b zewX^Mv7}v0_oAr|rmvn=vvuksdi!k~%RAT4>Zpwfxo}rDv>~rdn0x6aOO<-*rkq^p zrJKEUlUp-iZRL@@bd$Z$HeZB#>1Hq8wCe|4cP8AW=~CkbtCwy%g{3>KBMkfx(oOdI z&)$3O<4vc&``HaRwx%A1#E8Q6VN(iCr`X zAg>o~oIyTWwIpoH6IhoB6lUZZH0U%P#qg;B?uVN9s;*jd?v{MDXjRLsr3sU-S{0(p ztZ6_o;0pv?Y^v&#hM}}7L+!1l8><*}EiXD|@U5YIGizsaSmlZx9t0(d;Ch^pZ+@d- z(_ee`=EgTp55I5MU^at}Wo*|*D2j9z(j>zqJgE~2G66&9mw}EhPit0bDC%0l8HQRT zZ3Q7=*S#n@)+wr+)o#5d6NyYVlb%S3 rP%w5PJhF@l3V=ISd`;;>#pMYs3SF&n>%@vGn<%;`Kb@aSWhVaub9R<+ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/plan/calc-filter.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/plan/calc-filter.json new file mode 100644 index 0000000000000..b1d3b7bc1f984 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/plan/calc-filter.json @@ -0,0 +1,141 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT" + }, { + "name" : "c", + "dataType" : "DOUBLE" + }, { + "name" : "d", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a` BIGINT, `b` INT, `c` DOUBLE, `d` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[]]], fields=[a, b, c, d])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT, `c` DOUBLE, `d` VARCHAR(2147483647)>", + "description" : "Calc(select=[a, b, c, d], where=[(b > 0)])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT" + }, { + "name" : "c", + "dataType" : "DOUBLE" + }, { + "name" : "d", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT, `c` DOUBLE, `d` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c, d])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-filter/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..fcefcf935ee3aca12565f37e8ee49d5f2f82a96d GIT binary patch literal 9372 zcmeHNUx*t;7@y?gwbizFt%um!9@U=WpJq4zFY#VXa%me)b4QY^as|)H?j*Op&2G9o zNpnT&6;Ue%TR|Uuuofx!Uyw)&P>e`^qX zc3hwSyyNwY{|tRq|M1=C4}E#ZtI)+pF1gSyIamt#C0vvuCE_c|rSXUqEs~f&T$GD} z01m*gjn;6u@vAk7i?QZHTk?9%kVuYTnHYJBDX~VKDJd$sU#w$8Ea@tAnqpZK{aRo? za;zwa>qkoAU|o_zy>1DQ-sr*iSaRBkR`h!;|Glj(dRmrl$S(o@;F15=YJG@PH#qziUOiqj1rjAv$2 zbNN&*9nYj^Q#ojLBSSaJ0bb(6H*z zb7xbVb~k;;o$tbr=myh3_pLG$GcQxvs}NN-Co#nZc>ndy3p?I8@yeVVxiV;TiS@O* zOp1mxhjAT?%qGQ*V$z9*f$qF4dW-nt>0d_9x}U7LkvoHiBwee6!z;R$RIv%bTbjTs z*2*Gpn#iD=RK2W7SY>Qc6v)6lV$)MZ?kOUdm&jKkT4TkGO*7G`)(y11lRdudoPLxU zEnEX`M^Eok8o_%WHX^uPVXbHiME9u-!%wvl;M zXa3bV@yX7YKEN*zAXgfBObGEHF4**sGWYhtdcr-)&9aFxxbI~iZOI0VV*m_aCZ=R4 z)uvHzb=Y*C8j4nCHm!kEFbUiS&rIX&+0&?kTwNHr5PRYxp_J1aZ8uKTU{IS}$JuR| zqQOLZUpiahrn;TM7kf7RD`Kw$;MBfU&IXry=6zDHfhk{s_sOZ5L?&g!%K+b-uN}lj zavmG)_@MGZm1t!;zd}gs4#soI{qfurmo8o6UEBDax3*cv<%52l>J>$51?TN2^Y{F5 zYHttx_b$(-&UcdD9DD zpJGT=&ef=kA@1VOVZhyDlYquZnHXEYId}f|6VF5eI^BY1G*M17#_8``rgu!)RLas@ z?>%_W!k-=>pX?;Rmy_@5M&2~_jvc$BrES9oZ?-n;tMR2?&Fz|ncGtBC?N-dNYsxvg`_lFKpK?Frb(~?$PL&v@CXx}5^IWu*Ui#>0ks;VRO6f~@5tVPdm z)+QJ2$lqIs#!&f}7WkJ7LUQ?W+R&@SpbFuYb%1fZ(fV05EGwpj4LON52{ z?Zxmp1-uWn-W5$T=bbJ2TG6akSW62gUo)#jlUdVXNr$gA;9)~elT-`^Xc;P$HgBvn z(X|L`Ekm@2iY+xV8dl$9mkU8L#NUe$@~sa!82PzpK3#wQk&$$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "BIGINT NOT NULL" + } ], + "type" : "BOOLEAN" + } ] + }, { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 0 ] ], + "producedType" : "ROW<`a` BIGINT> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a` BIGINT> NOT NULL" + } ] + }, + "outputType" : "ROW<`a` BIGINT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[>(a, 1:BIGINT)], project=[a], metadata=[]]], fields=[a])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `EXPR$1` VARCHAR(2147483647)>", + "description" : "Calc(select=[a, CAST(a AS VARCHAR(2147483647)) AS EXPR$1])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "a1", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `EXPR$1` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, EXPR$1])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-project-pushdown/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-project-pushdown/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..e259d1e9f351c0a3c58a79ec9b70618361ee378f GIT binary patch literal 7090 zcmeHMe`p(39Dhl>uKU?;>xP@#D6Sg}m%HSWG^zb7O;Z=m(vqaz=)n4N_b$Ela+lq` zWNDGvF_@EqQ_w&B!#T#pKdc~#;9veFRnWg2`$PR>B7!g!Lb!Yd{j~ks1$%6sjaP}- zv!mVkb?k%R?D6;e1ivf_K`9`DVz=SySW#_e z8aQQiES8F7MpIMiOe7PV8jhzksrb-%CO(>+IyO2SL*41IL_AX)h*|TxCnAaQ*i<@} ziboRh$yf?{-N?|59ES^7oF<&CD%uRZD4eM0^SZ{-*#aq2MK!rBHVKCf1J80pvy_Y( zN?xH#i9}denivXJl{sSctu~m}aKW6`DadVutFHOK{$~H;iLw@5{Ql(^uFNdJpoIZO zR2NY<>M=n}k3pz`MGAMbs8Z8GxAQRCY?Cw$#a!s&#yu?iyg`4+>+$=&K2JcvVb6e1 zCZ2%-SrS81wqFn>gmz-kAL72UasNf=CmKReaaJXqQPilCCywaH%9I!yRvm$E?5Ydz zmLILElpDDcXsgV|szWA4-HB7U zgn0%^KB1U&uxy}%t8%YSUcCHU&lUIiyc@X_s7utf5-fQ|*P<#mf$_EqV-;&T-s&1m zpiQctQ$(yXwI~u2AcwCk72aAZyyY+6vVym1`9y8bV7X8-(4I#1EVEPk8HQSDZEn;K z(kxqtW|KG%(z+am9>ayflc>ucAJa`(db&o@j*||hN3*AhNI~oN8AX#Q>TRMflgoqL zXMqAJw-#(xRiV2bLKCpch;$U2(@cE10AW1RXo&@wN*HYiaKN8AtYR8+S#j*JbT?0< zv&aPz^R9`J!;>{;?n$&!F_T4AWugZ7WpumHw#OB&+ zq3sBPRa+S}IffuErWNzf!2(5XsnO%$!$TGk!!H-rRTV-8p)~u(wY)AVvSJwpxW3zU z(D%_#uXJ19YgeJC>boQNXHa9jdkD>c@!aqGEjBKZqFs&%*T^d2-BIIgZR>1(mCt>( z@6FHfTW!b{N9`u;@!&4lsLrtE?FM^-Yx0|oi9WcGuoZ3l2Efqg^6p zr>UW6IfiKs76tRb&Xo>#=`4!7rG`tTof2Lv~uky+!0&BuYsgbuE}^uw!PUOsRO@bC;;SHfzM0C2l& zz)cqOSz=sRcw^tD*T1{OI2A0d_PZFzUhDZV@O4b71nKH8*TZvz9}d-~^jT95S>_?s zbNo**tnwMltPrin4-eplS?BofH1yPcb^w$1C+D9aVy*FhXd9I4VQf$~@@@b9JBWWY zJ?UK84&b}dSOf4^PPW-{b>P1KwGQ-?4O~~s<0?ya*@|k`S;v2QHr>iaSsvBOMWvzH zRxaAgMY&az*;%0<~>w|Yuy<)W=zlpSR5hlocrgKFiXbb@4 zkgfI(%S53C1RuMdhikPlMq@YokP~jmoU2DrD2IzPRym)GEH-fRRN^Vq7YP!wz|!HK#-cv2=JqzSslFEJWfnpUmvP}H>c(*spXS_?wN zs+du9v{6(utDSoHLAu(&348&kYkz|inBLry-dq#CEf>PiTB?5=qLJiiG9Ha2pww(f nxOa&sC;;wQ_CHJ$C@xK4QD|z7Qzurj*+9|y^muwKmK^>YxrMy< literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/plan/calc-sarg.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/plan/calc-sarg.json new file mode 100644 index 0000000000000..b8d7145648172 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/plan/calc-sarg.json @@ -0,0 +1,132 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[]]], fields=[a])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "TRUE" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT>", + "description" : "Calc(select=[a], where=[SEARCH(a, Sarg[1, 2; NULL AS TRUE])])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-sarg/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..98082d5d1c95b8bafdea2ea48d4080173ae7dba2 GIT binary patch literal 5874 zcmeHLO>7%Q6dpTiNq^(EQAJBzMUWyP>Ro%CwG(o35{Fv2c4Rwkl}d!!-C1Xw^{%@+ zcAW}UsgT+t1gH`mI8dOf#DO4jLE^*-1te}%Jpcz#FGv+!pi(7N5O3DI*>&xN^gzWS zBduq5=FR*2-i+Uy>?@ThLZ}xW)%Ld*iY@x5VZ>BK96= zw|+nL$;#jTKQ+Gj^4ggnpS}T8jB+W229<;=#APhWNtMJ@MNK8;5s8e(MM;qa0SiF3 z(sB&LL611 zsv^bYn21v;S>j0&$A!3nqbZRn(P1?{8XX-G)u<>X#l+}HA~`IGpx9k_wyvo*GYy*eQjpsNPg_fT|HF}$(@kSy<<9G`USC*-NhbyvQ5&LR zG+=|4QHxLu%M^OMu2VZe7gD6Q^Q= zI*N%Pr4WjI1o{K+Ya91pK!36WdWt2Ta8})*T9pLQ&on8q46Fx$?i^}^ch@h@_FMi9 zZLux%{2G#2eA(QTMs&rV#1xm|``%~Ehd#aZ!CV-Ha%i{8=DaDJqDby6ZeX4TC7;u5 zI@Yw%(RI0di&x(MW8ixDwQ3lJbEsc7jRtIa%`_%-Yy;yxEyg-FD!el^mP0#qv!cmZ zXKGO-NE*3MQwE<%5z%+ExLc@=$5?kx4h4vx@R_$h? z$*~CHGDf``oUBpQQ!JbUA3p04vH0aCOkD@a07`pk#;BT#rfQB+5Z4d-kH$a$^{t5G zyi>A8?Y&u z2cCj=hB$lgZPviAA-v(p%o?;kGEtN9*%i9Tx(G#2Wb@_BiA62IJR^i5I2ivhoCSOQ5CLUKg;;yc7Z1YD7AgHA@p$8vzarn|GUkcK^b4%%o#Rw z25#M?!%qe_a|TwFc-d^1zp>4n!LRo>bB5NXNvm>ZnW-N-f%@f9i-pIQGq6NF!hTp| z34`0Zjd^p>yP4ofM;7DdnQ)yXFF4m>+{EhZv}M+aMK$7F*8rPv81>Dfh@#mtwv-8M z$OK9vWftvnho&%G(!qDX_pKS4Js%t?)g`-LWg{M&Qr)f*Lt#T(RTJ+1;ALCgkaY}~ z8VXdszHY1<=vvsjX7D~K@17$-5n7%RjtwtMBINfiJ=cs-Nwk zV--7v2}NRO2`ifx;YpLoaD1BvzshK8b(vQmQPj0|8h|RF`XHp8ij$(_ouax~9n^aW z(ufXD;0ws26CF-qdULCKb6xayTnIlKsh+n@r1OP*b|RgFs~Q*K!Bw810J!6!d(1AO axVnI$(AAotPVAiD21Q|MrZk<&PyPcK{x7ot literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/plan/calc-udf-complex.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/plan/calc-udf-complex.json new file mode 100644 index 0000000000000..ac6119b496604 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/plan/calc-udf-complex.json @@ -0,0 +1,272 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "d", + "dataType" : "TIMESTAMP(3)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[]]], fields=[a, b, c, d])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`udf2`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "systemName" : "udf3", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "systemName" : "udf4", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$SUBSTRING$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`udf5`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : 1000, + "type" : "INT NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`udf1`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "BIGINT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$*$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "BIGINT" + }, { + "kind" : "LITERAL", + "value" : 100, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 10, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `a1` VARCHAR(2147483647), `b` INT NOT NULL, `b1` VARCHAR(2147483647), `c1` VARCHAR(2147483647), `c2` VARCHAR(2147483647), `d1` TIMESTAMP(3)>", + "description" : "Calc(select=[a, CAST(a AS VARCHAR(2147483647)) AS a1, b, udf2(b, b, d) AS b1, udf3(c, b) AS c1, udf4(SUBSTRING(c, 1, 5)) AS c2, udf5(d, 1000) AS d1], where=[(((udf1(a) > 0) OR ((a * b) < 100)) AND (b > 10))])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "a1", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "b1", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "c1", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "d1", + "dataType" : "TIMESTAMP(3)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `a1` VARCHAR(2147483647), `b` INT NOT NULL, `b1` VARCHAR(2147483647), `c1` VARCHAR(2147483647), `c2` VARCHAR(2147483647), `d1` TIMESTAMP(3)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, a1, b, b1, c1, c2, d1])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-complex/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..79c262d72a5674c830c25f8d61ecd1f1c22e7ea2 GIT binary patch literal 13385 zcmeGjZEPE7`L&ZK?fQ|l3xl?66>V81p!UVzHbDX>bz2YD3AU3?sYLqv?zMC6`OdvN zyH15lDx_{`6Wb*C!4I~osz8V$RYH9H;ujPo_{FpzfY7FD64Epz#41ffgYrDD&-OXC zQ)e`0h&$sXN*ME9{;VI*1@7=q3=kr$E zI<_}DVcB-Azxn+aKD>YAiJfo0tNe2K=5=Vop6l49$B;Mv{^Squ{%h>5>TiDW>bXBX zb<^s(?jn;?py2n3G%tn=Y`P#7B4II_XE9$WFXjDy>W8j-HUMrQaUnM@vOlX;bdhBk zm6)D2smY)k8(uBQ3j3T;rMghi6lgSrqAK`Q|7!S3UJ6w&6+(fkC`|gH~BtlU36b$p@$X5f2Uf^RXB$ghDY<2>61G zi81C0<-^Pq4EO>b>YomId<7{y9S+8%pooOM3-?G*vXM0q%KSni6VJ_OR6&+T&14I@T#`+>%Hmj+Wx7li`3lpg zHfzkPv}~+uCg2@{yX^}6;g3h}K37xc?%w;xS6{n$8yY*zhHKPKCJdm`qcc;dq6tr< zqL>B(vowGF&t#9)Wn+`)%Y)W?l;s1lV4&ckp-|Wp^oc=FjD|y=0`pCWK=FZy7$L;- zDP?zgm{+f;rd(n+?I&s`(^ab26x~_Xrry1O;qG_)uVjazlh5|R zg!W~4Ps&V@j1$zPIe5Q+=k}97y7u~to48VBUjf@1O@>LvQx|BJ3TTu$Zka|$|6;eER*|MC2JwGffii!ZCL0tsoo+tdXla32v1n9BYw94iExd?L*V_q6K*q zYNcdvESqE~Gk+Fjc#L;K`}NLgiUKMFRT@)^YDtsig3Or$yZ&(O8Q)Ld`{p>8-mn5S z4cQ%kIDi)5gR^A)w_o|-Cw+0^*-K$%%^qquiCZWANkI&>34^SD@le8 znBzfQgp6KB_Z|g)LVd~2vWY2po3Jy z0*9fD8prZ*tqgK?QJ_NTi;7vfm{iSn<0Lf%wbykOALAw&nN6NbrgPj>4*_^L^#T0; z40eg?bE{NuM*wmgQJ7jZS2ti9U5aPsPRBD}d+^{v6P+s$*!uIcUaXJ?E2F?i;#4Bj z1n$oJczMc&%pPk)>=rj`^YCH{dsc} zCZE$bi1=_Wc_sn7q%#XEXOp?p^NYC^tXWx}Pvf3w2>gDFlp_|)c)8c3AorVOG?O^B zn2KjO{)1dtpF#&|y2<7%VBFM7ve;Ou4!w~5^gpgV?RUehg7Dq2h}h6Hi?5I+u!e9a#Z%shyN-59u>$kssWcRn;ei0#+IaNow>L+=57U0^3)Sd0d z-)=>(%=~z^HDsf;;IREIOkjqr$n*pEZru9d+VfGMDA}M0iBE7;pCMR2UPp_?XL+#$ z^hav~fPFWf?AV&qCUf5XIQPR&N#_nO_yeb zgEJ%&7vu87gU}GZexXLl#MytsYkh;3>){8TTPm0dQQx*5RYy(Rh6;?y+D!c?b6vkP z*}#@w*C3W&YfiW1_h{zw|4&KU*DOn%A9=C**hz38+rY-6>=+|IT6t|V%~p5RU5wH3 z?bhkj5hu4W`+?j#?+s+p>0+H*+-q*%F8O0O^$tWMgEGwH=-3+e4z%JH z-?lSse{I_qE_Xof6~n7Fo@yH&QqPj|rYO*NbZY(%%cwZYtOn*Z0~pYM=)FP-Aa zgdeCksI^DEO$&Hj8S03WJ5gqgkUCLjX?D$tGCNUb@1|kuR`}S7GUI2g#`g~=%Dh=5 zd9yYr%4{v?PL!EAQD)*qnIF{$-hmu1ohUPIo}0_yqYdmiQRXcr8#qyBa~qA(i84D; z=7(n$4P?=AqRg1H`mjX2LB{=^D6?AJ5RP=B%#g*lRheve6nk({W(YMCOd90Z4(l1J zKd<(c;}kM1zjsM-IN(mX$L@VH`%#N z#wFPhsV>b?RRqB>X_@S`s!me)BLlpTHQr@aHdgI5*-GB1l(41&CR;JeOqH-|q@cn7 zBcQOWqKXQIFA5T52DfO#uaI4npDi|oR#&0%fdR*0UM*fO0{MZyeuR*1{DT6A-+2Cw zowuHw_&LuIeGF(UQJ%SJl2B)2ps49gU^ON}z*19%CZLneVT~M8lXT55oq$Yev!#SM z&&oB)=}uJLxVFGpP^n~M;B`pGcfH5X6Mr;~H>6r{6T o8lG&z1Oh-D8~*Fj#T1(ZPzznTvD1V}aJ%4(XLd2WkVv2SH+yEA!2kdN literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/plan/calc-udf-simple.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/plan/calc-udf-simple.json new file mode 100644 index 0000000000000..894d252d5b18b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/plan/calc-udf-simple.json @@ -0,0 +1,111 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`udf1`", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + } ], + "type" : "BIGINT" + } ], + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `EXPR$1` BIGINT NOT NULL>", + "description" : "Calc(select=[a, udf1(CAST(a AS BIGINT)) AS EXPR$1])" + }, { + "id" : 3, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "a1", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `EXPR$1` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, EXPR$1])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-calc_1/calc-udf-simple/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c110557cab5c1aa8b07b12483d9cba986fcf1f35 GIT binary patch literal 7016 zcmeHMUu+ab7@xbATK={`ML{5q6cQr0dw+WW?9;Wq!s&5s?yf*eVx8Tcy)NC`J$C1g zD>0H~(Dm=JwY5)wt@H+#FiJ?>gch(zN} za<{uP-+X_6GyBcAsaFd=git5^So72_sAK--2N&+&-LX$-tFd$+^q!&LL2%{jenh$Ks}M2sB8l-N|4Wkn?~@)d0GvaZ6Q$rm&}pao|lrv)io zIVp!j6;T|FmQDq=m>*izlHIB}eNSP|C$qCpb95xI*#|_U>GGQo1 zg(?-2U|l(4C|Ffa6Jubd!JLLm=8R53ZU_7tTIk#Fb}t^QY9ouc-+1lH>;ep07+^#V z5%r=z6SVXhgc?|+a5c*+HC=Rlx6lTgq-p364|>9JkBc}Mmf}89mV>@QNe=kpQrPbc zM1^Q17>osj0U4pEp8@?L?#mnZPlWzRQ|KucRKgi$jVeXrihi<6iJ@WD73k)!hVX9u z$y$C3zo0dyfnHcf62q6tjVVNx%u!5n9-i-hys-15v+qxPktdBd$!yFSGAZg!pTHH& zGf?tr#iT=31MOXryEA$5(r}*`u~y)%uAwwq zuj&Ow#41yZqF@^2@U^AFTT6wv{KZ>V@HQ=wwCM}A7V0v&Jji_+ zD1dTP$!6sU-R=^ahE+zSL)e^Q;;SVH))D(~!rCW0$47c@mvL z9*CHKRg7GotTXdYp>>LxDXS_IokrbEcvjabMUfU3Gos}K#O8r%=!}5|A5|qb*H#N{ zK?tnc#GuJB1aV2NTyzhXDC)?L9|a%ovxpdexe;AeA!HCrb6`>{>XITWmQjG~`#pOD zpZxr0ujRc?6?z=s9e6l{I^*rbX#T5L?(MeNctnb}xguOAtA%%KowK#Av-MRy`^~O* zzrgQxAWsT)ny|-%yI`X_#g?}h>RZ#E_d;5x=uwCx)JM+X33BBp34r5dQWxGMWx>+4HR-1jd7i|s(j-w@v(vHgNbrG&15N3JI$1a=h|szJI&-)%v0<6V>``c=UC?e z)FQcHnNd5r!Ipglhl?Ah0WIYa>T-huUO=4YLvJPJy;AU-E6IBeQR0(Qrb=_DEqGdVJ zHiNH|@=mtS=CDc?YaRqOiNI={kaNCCu<2LN-{`)6tnX8+=CTfSEMlu1LQ%N6a3<;o z;YpQt?nRp+AUP_w93R=f2cfK0TB6Zis-$Dt-C zFul1Yy}1^8TP}pZ)zs|PjwCYUnbb%k4FzI5!u?A;K>={b17A;CKyhgTi$Y6l+&ZyT PVJ(!Ra+A4 Date: Mon, 23 Oct 2023 17:48:27 +0800 Subject: [PATCH 072/104] [FLINK-33323][network] Make the batchShuffleReadIOExecutor shut down before the batchShuffleReadBufferPool is destroyed --- .../runtime/io/network/NettyShuffleEnvironment.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java index 2faff9650d3e9..f3ae8ff7d2b42 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java @@ -61,6 +61,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_INPUT; import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_OUTPUT; @@ -68,6 +69,7 @@ import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerDebloatingTaskMetrics; import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerInputMetrics; import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerOutputMetrics; +import static org.apache.flink.util.ExecutorUtils.gracefulShutdown; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -409,15 +411,15 @@ public void close() { } try { - batchShuffleReadBufferPool.destroy(); + gracefulShutdown(10, TimeUnit.SECONDS, batchShuffleReadIOExecutor); } catch (Throwable t) { - LOG.warn("Cannot shut down batch shuffle read buffer pool properly.", t); + LOG.warn("Cannot shut down batch shuffle read IO executor properly.", t); } try { - batchShuffleReadIOExecutor.shutdown(); + batchShuffleReadBufferPool.destroy(); } catch (Throwable t) { - LOG.warn("Cannot shut down batch shuffle read IO executor properly.", t); + LOG.warn("Cannot shut down batch shuffle read buffer pool properly.", t); } isClosed = true; From 5da6b3eff29c14931c86b2018331c40ed256420f Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 31 Oct 2023 10:22:54 +0100 Subject: [PATCH 073/104] [FLINK-33407] Remove old expression stack leftovers for time functions --- .../PlannerExpressionConverter.scala | 40 --- .../table/planner/expressions/time.scala | 309 ------------------ 2 files changed, 349 deletions(-) delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/time.scala diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 6a0d7b028ed91..8983fbbd64a20 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -147,46 +147,6 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp assert(args.size == 1) Collect(args.head) - case EXTRACT => - assert(args.size == 2) - Extract(args.head, args.last) - - case CURRENT_DATE => - assert(args.isEmpty) - CurrentDate() - - case CURRENT_TIME => - assert(args.isEmpty) - CurrentTime() - - case CURRENT_TIMESTAMP => - assert(args.isEmpty) - CurrentTimestamp() - - case LOCAL_TIME => - assert(args.isEmpty) - LocalTime() - - case LOCAL_TIMESTAMP => - assert(args.isEmpty) - LocalTimestamp() - - case TEMPORAL_OVERLAPS => - assert(args.size == 4) - TemporalOverlaps(args.head, args(1), args(2), args.last) - - case DATE_FORMAT => - assert(args.size == 2) - DateFormat(args.head, args.last) - - case TIMESTAMP_DIFF => - assert(args.size == 3) - TimestampDiff(args.head, args(1), args.last) - - case TO_TIMESTAMP_LTZ => - assert(args.size == 2) - ToTimestampLtz(args.head, args.last) - case AT => assert(args.size == 2) ItemAt(args.head, args.last) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/time.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/time.scala deleted file mode 100644 index 19295dd36aa12..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/time.scala +++ /dev/null @@ -1,309 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.table.planner.expressions - -import org.apache.flink.api.common.typeinfo.{LocalTimeTypeInfo, SqlTimeTypeInfo, TypeInformation} -import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ -import org.apache.flink.table.planner.calcite.FlinkRelBuilder -import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable -import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils -import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils.isTimeInterval -import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} -import org.apache.flink.table.runtime.typeutils.LegacyLocalDateTimeTypeInfo -import org.apache.flink.table.typeutils.TimeIntervalTypeInfo - -import org.apache.calcite.rex._ - -case class Extract(timeIntervalUnit: PlannerExpression, temporal: PlannerExpression) - extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = timeIntervalUnit :: temporal :: Nil - - override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO - - override private[flink] def validateInput(): ValidationResult = { - if (!TypeInfoCheckUtils.isTemporal(temporal.resultType)) { - return ValidationFailure( - s"Extract operator requires Temporal input, " + - s"but $temporal is of type ${temporal.resultType}") - } - - timeIntervalUnit match { - case SymbolPlannerExpression(PlannerTimeIntervalUnit.YEAR) | SymbolPlannerExpression( - PlannerTimeIntervalUnit.QUARTER) | SymbolPlannerExpression( - PlannerTimeIntervalUnit.MONTH) | SymbolPlannerExpression(PlannerTimeIntervalUnit.WEEK) | - SymbolPlannerExpression(PlannerTimeIntervalUnit.DAY) - if temporal.resultType == SqlTimeTypeInfo.DATE - || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP - || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE - || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME - || temporal.resultType.isInstanceOf[LegacyLocalDateTimeTypeInfo] - || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS - || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MONTHS => - ValidationSuccess - - case SymbolPlannerExpression(PlannerTimeIntervalUnit.HOUR) | SymbolPlannerExpression( - PlannerTimeIntervalUnit.MINUTE) | SymbolPlannerExpression( - PlannerTimeIntervalUnit.SECOND) - if temporal.resultType == SqlTimeTypeInfo.TIME - || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP - || temporal.resultType == LocalTimeTypeInfo.LOCAL_TIME - || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME - || temporal.resultType.isInstanceOf[LegacyLocalDateTimeTypeInfo] - || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS => - ValidationSuccess - - case _ => - ValidationFailure( - s"Extract operator does not support unit '$timeIntervalUnit' for input" + - s" of type '${temporal.resultType}'.") - } - } - - override def toString: String = s"($temporal).extract($timeIntervalUnit)" -} - -abstract class CurrentTimePoint(targetType: TypeInformation[_], local: Boolean) - extends LeafExpression { - - override private[flink] def resultType: TypeInformation[_] = targetType - - override private[flink] def validateInput(): ValidationResult = { - if (!TypeInfoCheckUtils.isTimePoint(targetType)) { - ValidationFailure( - s"CurrentTimePoint operator requires Time Point target type, " + - s"but get $targetType.") - } else if (local && targetType == SqlTimeTypeInfo.DATE) { - ValidationFailure( - s"Localized CurrentTimePoint operator requires Time or Timestamp target " + - s"type, but get $targetType.") - } else { - ValidationSuccess - } - } - - override def toString: String = if (local) { - s"local$targetType()" - } else { - s"current$targetType()" - } -} - -case class CurrentDate() extends CurrentTimePoint(SqlTimeTypeInfo.DATE, local = false) - -case class CurrentTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = false) - -case class CurrentTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = false) - -case class LocalTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = true) - -case class LocalTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = true) - -/** Determines whether two anchored time intervals overlap. */ -case class TemporalOverlaps( - leftTimePoint: PlannerExpression, - leftTemporal: PlannerExpression, - rightTimePoint: PlannerExpression, - rightTemporal: PlannerExpression) - extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = - Seq(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal) - - override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO - - override private[flink] def validateInput(): ValidationResult = { - if (!TypeInfoCheckUtils.isTimePoint(leftTimePoint.resultType)) { - return ValidationFailure( - s"TemporalOverlaps operator requires leftTimePoint to be of type " + - s"Time Point, but get ${leftTimePoint.resultType}.") - } - if (!TypeInfoCheckUtils.isTimePoint(rightTimePoint.resultType)) { - return ValidationFailure( - s"TemporalOverlaps operator requires rightTimePoint to be of " + - s"type Time Point, but get ${rightTimePoint.resultType}.") - } - if (leftTimePoint.resultType != rightTimePoint.resultType) { - return ValidationFailure( - s"TemporalOverlaps operator requires leftTimePoint and " + - s"rightTimePoint to be of same type.") - } - - // leftTemporal is point, then it must be comparable with leftTimePoint - if (TypeInfoCheckUtils.isTimePoint(leftTemporal.resultType)) { - if (leftTemporal.resultType != leftTimePoint.resultType) { - return ValidationFailure( - s"TemporalOverlaps operator requires leftTemporal and " + - s"leftTimePoint to be of same type if leftTemporal is of type Time Point.") - } - } else if (!isTimeInterval(leftTemporal.resultType)) { - return ValidationFailure( - s"TemporalOverlaps operator requires leftTemporal to be of " + - s"type Time Point or Time Interval.") - } - - // rightTemporal is point, then it must be comparable with rightTimePoint - if (TypeInfoCheckUtils.isTimePoint(rightTemporal.resultType)) { - if (rightTemporal.resultType != rightTimePoint.resultType) { - return ValidationFailure( - s"TemporalOverlaps operator requires rightTemporal and " + - s"rightTimePoint to be of same type if rightTemporal is of type Time Point.") - } - } else if (!isTimeInterval(rightTemporal.resultType)) { - return ValidationFailure( - s"TemporalOverlaps operator requires rightTemporal to be of " + - s"type Time Point or Time Interval.") - } - ValidationSuccess - } - - override def toString: String = s"temporalOverlaps(${children.mkString(", ")})" - - /** - * Standard conversion of the OVERLAPS operator. Source: - * [[org.apache.calcite.sql2rel.StandardConvertletTable#convertOverlaps()]] - */ - private def convertOverlaps( - leftP: RexNode, - leftT: RexNode, - rightP: RexNode, - rightT: RexNode, - relBuilder: FlinkRelBuilder): RexNode = { - val convLeftT = convertOverlapsEnd(relBuilder, leftP, leftT, leftTemporal.resultType) - val convRightT = convertOverlapsEnd(relBuilder, rightP, rightT, rightTemporal.resultType) - - // sort end points into start and end, such that (s0 <= e0) and (s1 <= e1). - val (s0, e0) = buildSwap(relBuilder, leftP, convLeftT) - val (s1, e1) = buildSwap(relBuilder, rightP, convRightT) - - // (e0 >= s1) AND (e1 >= s0) - val leftPred = relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, e0, s1) - val rightPred = relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, e1, s0) - relBuilder.call(FlinkSqlOperatorTable.AND, leftPred, rightPred) - } - - private def convertOverlapsEnd( - relBuilder: FlinkRelBuilder, - start: RexNode, - end: RexNode, - endType: TypeInformation[_]) = { - if (isTimeInterval(endType)) { - relBuilder.call(FlinkSqlOperatorTable.DATETIME_PLUS, start, end) - } else { - end - } - } - - private def buildSwap(relBuilder: FlinkRelBuilder, start: RexNode, end: RexNode) = { - val le = relBuilder.call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, start, end) - val l = relBuilder.call(FlinkSqlOperatorTable.CASE, le, start, end) - val r = relBuilder.call(FlinkSqlOperatorTable.CASE, le, end, start) - (l, r) - } -} - -case class DateFormat(timestamp: PlannerExpression, format: PlannerExpression) - extends PlannerExpression { - override private[flink] def children = timestamp :: format :: Nil - - override def toString: String = s"$timestamp.dateFormat($format)" - - override private[flink] def resultType = STRING_TYPE_INFO -} - -case class TimestampDiff( - timePointUnit: PlannerExpression, - timePoint1: PlannerExpression, - timePoint2: PlannerExpression) - extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = - timePointUnit :: timePoint1 :: timePoint2 :: Nil - - override private[flink] def validateInput(): ValidationResult = { - if (!TypeInfoCheckUtils.isTimePoint(timePoint1.resultType)) { - return ValidationFailure( - s"$this requires an input time point type, " + - s"but timePoint1 is of type '${timePoint1.resultType}'.") - } - - if (!TypeInfoCheckUtils.isTimePoint(timePoint2.resultType)) { - return ValidationFailure( - s"$this requires an input time point type, " + - s"but timePoint2 is of type '${timePoint2.resultType}'.") - } - - timePointUnit match { - case SymbolPlannerExpression(PlannerTimePointUnit.YEAR) | SymbolPlannerExpression( - PlannerTimePointUnit.QUARTER) | SymbolPlannerExpression(PlannerTimePointUnit.MONTH) | - SymbolPlannerExpression(PlannerTimePointUnit.WEEK) | SymbolPlannerExpression( - PlannerTimePointUnit.DAY) | SymbolPlannerExpression(PlannerTimePointUnit.HOUR) | - SymbolPlannerExpression(PlannerTimePointUnit.MINUTE) | SymbolPlannerExpression( - PlannerTimePointUnit.SECOND) - if timePoint1.resultType == SqlTimeTypeInfo.DATE - || timePoint1.resultType == SqlTimeTypeInfo.TIMESTAMP - || timePoint2.resultType == SqlTimeTypeInfo.DATE - || timePoint2.resultType == SqlTimeTypeInfo.TIMESTAMP - || timePoint1.resultType == LocalTimeTypeInfo.LOCAL_DATE - || timePoint1.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME - || timePoint2.resultType == LocalTimeTypeInfo.LOCAL_DATE - || timePoint2.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME => - ValidationSuccess - - case _ => - ValidationFailure( - s"$this operator does not support unit '$timePointUnit'" + - s" for input of type ('${timePoint1.resultType}', '${timePoint2.resultType}').") - } - } - - override def toString: String = s"timestampDiff(${children.mkString(", ")})" - - override private[flink] def resultType = INT_TYPE_INFO -} - -case class ToTimestampLtz(numericEpochTime: PlannerExpression, precision: PlannerExpression) - extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = - numericEpochTime :: precision :: Nil - - override private[flink] def validateInput(): ValidationResult = { - if ( - TypeInfoCheckUtils.assertNumericExpr(numericEpochTime.resultType, "toTimestampLtz").isFailure - ) { - return ValidationFailure( - s"$this requires numeric type for the first input, " + - s"but the actual type '${numericEpochTime.resultType}'.") - } - if ( - TypeInfoCheckUtils - .assertNumericExpr(precision.resultType, "toTimestampLtz") - .isFailure - ) { - return ValidationFailure( - s"$this requires numeric type for the second input, " + - s"but the actual type '${precision.resultType}'.") - } - ValidationSuccess - } - - override def toString: String = s"toTimestampLtz(${children.mkString(", ")})" - - override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP -} From 7295c3bcf92c0e106d3e91c57b6492030c760a25 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Fri, 3 Nov 2023 12:51:42 +0100 Subject: [PATCH 074/104] [FLINK-33309] Set `-Djava.security.manager=allow` on JDK 18+ --- flink-core/pom.xml | 21 ++++++++++++++++++++- flink-dist/src/main/flink-bin/bin/config.sh | 6 ++++++ flink-runtime/pom.xml | 21 ++++++++++++++++++++- flink-streaming-java/pom.xml | 21 ++++++++++++++++++++- 4 files changed, 66 insertions(+), 3 deletions(-) diff --git a/flink-core/pom.xml b/flink-core/pom.xml index ce90b2a1e6d16..aee9443281585 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -34,7 +34,14 @@ under the License. jar - + + ${surefire.module.config.jdk21} --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + -Djava.security.manager=allow + +
diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index f09c68d493fb9..dcd48a256f72a 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -334,6 +334,12 @@ if [ -z "${FLINK_ENV_JAVA_OPTS}" ]; then # Remove leading and ending double quotes (if present) of value FLINK_ENV_JAVA_OPTS="-XX:+IgnoreUnrecognizedVMOptions $( echo "${FLINK_ENV_JAVA_OPTS}" | sed -e 's/^"//' -e 's/"$//' )" + + JAVA_SPEC_VERSION=`${JAVA_HOME}/bin/java -XshowSettings:properties 2>&1 | grep "java.specification.version" | cut -d "=" -f 2 | tr -d '[:space:]' | rev | cut -d "." -f 1 | rev` + if [[ $(( $JAVA_SPEC_VERSION > 17 )) == 1 ]]; then + # set security manager property to allow calls to System.setSecurityManager() at runtime + FLINK_ENV_JAVA_OPTS="$FLINK_ENV_JAVA_OPTS -Djava.security.manager=allow" + fi fi if [ -z "${FLINK_ENV_JAVA_OPTS_JM}" ]; then diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 359a0e181cb7e..c6bd2e909644e 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -34,7 +34,14 @@ under the License. jar - + + ${surefire.module.config.jdk21} --add-opens=java.base/java.util=ALL-UNNAMED + -Djava.security.manager=allow + + diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index f66ed3b48b819..4a7644dfb453b 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -34,7 +34,14 @@ under the License. jar - + + ${surefire.module.config.jdk21} --add-opens=java.base/java.lang=ALL-UNNAMED + -Djava.security.manager=allow + + From f6b662f83deb80572773617f7eb202fa05388198 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 31 Oct 2023 12:55:13 +0100 Subject: [PATCH 075/104] [FLINK-33412] Implement type inference for reinterpret_cast function --- .../functions/BuiltInFunctionDefinitions.java | 3 +- .../ReinterpretCastInputTypeStrategy.java | 98 +++++++++++++++++++ .../SpecificInputTypeStrategies.java | 2 + .../types/logical/utils/LogicalTypeCasts.java | 48 +++++++++ .../inference/InputTypeStrategiesTest.java | 29 +++++- .../InputTypeStrategiesTestBase.java | 19 ++-- .../PlannerExpressionConverter.scala | 8 -- .../planner/expressions/Reinterpret.scala | 49 ---------- .../planner/typeutils/TypeCoercion.scala | 81 --------------- 9 files changed, 185 insertions(+), 152 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ReinterpretCastInputTypeStrategy.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/Reinterpret.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index e653d1d646331..2d49dc30725e0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -2254,7 +2254,8 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) BuiltInFunctionDefinition.newBuilder() .name("reinterpretCast") .kind(SCALAR) - .outputTypeStrategy(TypeStrategies.MISSING) + .inputTypeStrategy(SpecificInputTypeStrategies.REINTERPRET_CAST) + .outputTypeStrategy(TypeStrategies.argument(1)) .build(); public static final BuiltInFunctionDefinition AS = diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ReinterpretCastInputTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ReinterpretCastInputTypeStrategy.java new file mode 100644 index 0000000000000..d33ddcdb452d8 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ReinterpretCastInputTypeStrategy.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.inference.ArgumentCount; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.ConstantArgumentCount; +import org.apache.flink.table.types.inference.InputTypeStrategy; +import org.apache.flink.table.types.inference.Signature; +import org.apache.flink.table.types.logical.LegacyTypeInformationType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * {@link InputTypeStrategy} specific for {@link BuiltInFunctionDefinitions#REINTERPRET_CAST}. + * + *

It expects three arguments where the type of first one must be reinterpretable as the type of + * the second one. The second one must be a type literal. The third a BOOLEAN literal if the + * reinterpretation may result in an overflow. + */ +@Internal +public final class ReinterpretCastInputTypeStrategy implements InputTypeStrategy { + @Override + public ArgumentCount getArgumentCount() { + return ConstantArgumentCount.of(3); + } + + @Override + public Optional> inferInputTypes( + CallContext callContext, boolean throwOnFailure) { + final List argumentDataTypes = callContext.getArgumentDataTypes(); + + // check for type literal + if (!callContext.isArgumentLiteral(1) + || !callContext.getArgumentValue(1, DataType.class).isPresent()) { + return callContext.fail( + throwOnFailure, "Expected type literal for the second argument."); + } + + if (!argumentDataTypes.get(2).getLogicalType().is(LogicalTypeRoot.BOOLEAN) + || !callContext.isArgumentLiteral(2) + || callContext.isArgumentNull(2)) { + return callContext.fail( + throwOnFailure, "Not null boolean literal expected for overflow."); + } + + final LogicalType fromType = argumentDataTypes.get(0).getLogicalType(); + final LogicalType toType = argumentDataTypes.get(1).getLogicalType(); + + // A hack to support legacy types. To be removed when we drop the legacy types. + if (fromType instanceof LegacyTypeInformationType) { + return Optional.of(argumentDataTypes); + } + if (!LogicalTypeCasts.supportsReinterpretCast(fromType, toType)) { + return callContext.fail( + throwOnFailure, + "Unsupported reinterpret cast from '%s' to '%s'.", + fromType, + toType); + } + + return Optional.of(argumentDataTypes); + } + + @Override + public List getExpectedSignatures(FunctionDefinition definition) { + return Collections.singletonList( + Signature.of( + Signature.Argument.ofGroup("ANY"), + Signature.Argument.ofGroup("TYPE LITERAL"), + Signature.Argument.ofGroup("TRUE | FALSE"))); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java index 56809c7e69ce0..b004ce20e9dda 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java @@ -50,6 +50,8 @@ public final class SpecificInputTypeStrategies { /** See {@link CastInputTypeStrategy}. */ public static final InputTypeStrategy CAST = new CastInputTypeStrategy(); + public static final InputTypeStrategy REINTERPRET_CAST = new ReinterpretCastInputTypeStrategy(); + /** See {@link MapInputTypeStrategy}. */ public static final InputTypeStrategy MAP = new MapInputTypeStrategy(); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java index 72321b8accf51..54c9c7d12603c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java @@ -288,6 +288,54 @@ public static boolean supportsExplicitCast(LogicalType sourceType, LogicalType t return supportsCasting(sourceType, targetType, true); } + /** + * Returns whether the source type can be reinterpreted as the target type. + * + *

Reinterpret casts correspond to the SQL reinterpret_cast and represent the logic behind a + * {@code REINTERPRET_CAST(sourceType AS targetType)} operation. + */ + public static boolean supportsReinterpretCast(LogicalType sourceType, LogicalType targetType) { + if (sourceType.getTypeRoot() == targetType.getTypeRoot()) { + return true; + } + + switch (sourceType.getTypeRoot()) { + case INTEGER: + switch (targetType.getTypeRoot()) { + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTERVAL_YEAR_MONTH: + return true; + default: + return false; + } + case BIGINT: + switch (targetType.getTypeRoot()) { + case TIMESTAMP_WITHOUT_TIME_ZONE: + case INTERVAL_DAY_TIME: + return true; + default: + return false; + } + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTERVAL_YEAR_MONTH: + switch (targetType.getTypeRoot()) { + case INTEGER: + case BIGINT: + return true; + default: + return false; + } + + case TIMESTAMP_WITHOUT_TIME_ZONE: + case INTERVAL_DAY_TIME: + return targetType.getTypeRoot() == BIGINT; + default: + return false; + } + } + // -------------------------------------------------------------------------------------------- private static boolean supportsCasting( diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java index 14c7c89a45119..3a9139a645fe2 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java @@ -636,7 +636,34 @@ ANY, explicit(DataTypes.INT()) .expectSignature("f(, )") .expectArgumentTypes( DataTypes.ARRAY(DataTypes.INT().notNull()).notNull(), - DataTypes.INT())); + DataTypes.INT()), + TestSpec.forStrategy( + "Reinterpret_cast strategy", + SpecificInputTypeStrategies.REINTERPRET_CAST) + .calledWithArgumentTypes( + DataTypes.DATE(), DataTypes.BIGINT(), DataTypes.BOOLEAN().notNull()) + .calledWithLiteralAt(1, DataTypes.BIGINT()) + .calledWithLiteralAt(2, true) + .expectSignature("f(, , )") + .expectArgumentTypes( + DataTypes.DATE(), + DataTypes.BIGINT(), + DataTypes.BOOLEAN().notNull()), + TestSpec.forStrategy( + "Reinterpret_cast strategy non literal overflow", + SpecificInputTypeStrategies.REINTERPRET_CAST) + .calledWithArgumentTypes( + DataTypes.DATE(), DataTypes.BIGINT(), DataTypes.BOOLEAN().notNull()) + .calledWithLiteralAt(1, DataTypes.BIGINT()) + .expectErrorMessage("Not null boolean literal expected for overflow."), + TestSpec.forStrategy( + "Reinterpret_cast strategy not supported cast", + SpecificInputTypeStrategies.REINTERPRET_CAST) + .calledWithArgumentTypes( + DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN().notNull()) + .calledWithLiteralAt(1, DataTypes.BIGINT()) + .calledWithLiteralAt(2, true) + .expectErrorMessage("Unsupported reinterpret cast from 'INT' to 'BIGINT'")); } /** Simple pojo that should be converted to a Structured type. */ diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTestBase.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTestBase.java index 73c9e61011975..06ff25f9a9a72 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTestBase.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTestBase.java @@ -35,7 +35,9 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -92,15 +94,11 @@ private TypeInferenceUtil.Result runTypeInference( callContextMock.argumentDataTypes = actualArgumentTypes; callContextMock.argumentLiterals = IntStream.range(0, actualArgumentTypes.size()) - .mapToObj(i -> testSpec.literalPos != null && i == testSpec.literalPos) + .mapToObj(i -> testSpec.literals.containsKey(i)) .collect(Collectors.toList()); callContextMock.argumentValues = IntStream.range(0, actualArgumentTypes.size()) - .mapToObj( - i -> - (testSpec.literalPos != null && i == testSpec.literalPos) - ? Optional.ofNullable(testSpec.literalValue) - : Optional.empty()) + .mapToObj(i -> Optional.ofNullable(testSpec.literals.get(i))) .collect(Collectors.toList()); callContextMock.argumentNulls = IntStream.range(0, actualArgumentTypes.size()) @@ -161,9 +159,7 @@ protected static class TestSpec { private List> actualArgumentTypes = new ArrayList<>(); - private @Nullable Integer literalPos; - - private @Nullable Object literalValue; + private Map literals = new HashMap<>(); private @Nullable InputTypeStrategy surroundingStrategy; @@ -207,13 +203,12 @@ public TestSpec calledWithArgumentTypes(AbstractDataType... dataTypes) { } public TestSpec calledWithLiteralAt(int pos) { - this.literalPos = pos; + this.literals.put(pos, null); return this; } public TestSpec calledWithLiteralAt(int pos, Object value) { - this.literalPos = pos; - this.literalValue = value; + this.literals.put(pos, value); return this; } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 8983fbbd64a20..953da5ede4f31 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -69,14 +69,6 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp // special case: requires individual handling of child expressions func match { - case REINTERPRET_CAST => - assert(children.size == 3) - return Reinterpret( - children.head.accept(this), - fromDataTypeToTypeInfo(children(1).asInstanceOf[TypeLiteralExpression].getOutputDataType), - getValue[Boolean](children(2).accept(this)) - ) - case WINDOW_START => assert(children.size == 1) val windowReference = translateWindowReference(children.head) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/Reinterpret.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/Reinterpret.scala deleted file mode 100644 index 13998c7b7d6dd..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/Reinterpret.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.table.planner.expressions - -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.planner.typeutils.TypeCoercion -import org.apache.flink.table.planner.validate._ -import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType - -case class Reinterpret( - child: PlannerExpression, - resultType: TypeInformation[_], - checkOverflow: Boolean) - extends UnaryExpression { - - override def toString = s"$child.reinterpret($resultType)" - - override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = { - val child: PlannerExpression = anyRefs.head.asInstanceOf[PlannerExpression] - copy(child, resultType).asInstanceOf[this.type] - } - - override private[flink] def validateInput(): ValidationResult = { - if ( - TypeCoercion.canReinterpret( - fromTypeInfoToLogicalType(child.resultType), - fromTypeInfoToLogicalType(resultType)) - ) { - ValidationSuccess - } else { - ValidationFailure(s"Unsupported reinterpret from ${child.resultType} to $resultType") - } - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/typeutils/TypeCoercion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/typeutils/TypeCoercion.scala index 6e284464225d2..c6a06b8c64eb1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/typeutils/TypeCoercion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/typeutils/TypeCoercion.scala @@ -53,85 +53,4 @@ object TypeCoercion { case _ => false } - - /** - * All the supported cast types in flink-table. - * - * Note: No distinction between explicit and implicit conversions Note: This is a subset of - * SqlTypeAssignmentRule Note: This may lose type during the cast. - */ - def canCast(from: LogicalType, to: LogicalType): Boolean = - (from.getTypeRoot, to.getTypeRoot) match { - case (_, _) if from == to => true - - case (_, VARCHAR | CHAR) => true - - case (VARCHAR | CHAR, _) if isNumeric(to) => true - case (VARCHAR | CHAR, BOOLEAN) => true - case (VARCHAR | CHAR, DECIMAL) => true - case (VARCHAR | CHAR, DATE) => true - case (VARCHAR | CHAR, TIME_WITHOUT_TIME_ZONE) => true - case (VARCHAR | CHAR, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (VARCHAR | CHAR, TIMESTAMP_WITH_LOCAL_TIME_ZONE) => true - - case (BOOLEAN, _) if isNumeric(to) => true - case (BOOLEAN, DECIMAL) => true - case (_, BOOLEAN) if isNumeric(from) => true - case (DECIMAL, BOOLEAN) => true - - case (_, _) if isNumeric(from) && isNumeric(to) => true - case (_, DECIMAL) if isNumeric(from) => true - case (DECIMAL, _) if isNumeric(to) => true - case (DECIMAL, DECIMAL) => true - case (INTEGER, DATE) => true - case (INTEGER, TIME_WITHOUT_TIME_ZONE) => true - case (TINYINT, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (SMALLINT, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (INTEGER, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (DOUBLE, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (FLOAT, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (INTEGER, INTERVAL_YEAR_MONTH) => true - case (BIGINT, INTERVAL_DAY_TIME) => true - - case (DATE, TIME_WITHOUT_TIME_ZONE) => false - case (TIME_WITHOUT_TIME_ZONE, DATE) => false - case (_, _) if isTimePoint(from) && isTimePoint(to) => true - case (DATE, INTEGER) => true - case (TIME_WITHOUT_TIME_ZONE, INTEGER) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, TINYINT) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, INTEGER) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, SMALLINT) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, BIGINT) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, DOUBLE) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, FLOAT) => true - - case (INTERVAL_YEAR_MONTH, INTEGER) => true - case (INTERVAL_DAY_TIME, BIGINT) => true - - case _ => false - } - - /** All the supported reinterpret types in flink-table. */ - def canReinterpret(from: LogicalType, to: LogicalType): Boolean = - (from.getTypeRoot, to.getTypeRoot) match { - case (_, _) if from == to => true - - case (DATE, INTEGER) => true - case (TIME_WITHOUT_TIME_ZONE, INTEGER) => true - case (TIMESTAMP_WITHOUT_TIME_ZONE, BIGINT) => true - case (INTEGER, DATE) => true - case (INTEGER, TIME_WITHOUT_TIME_ZONE) => true - case (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => true - case (INTEGER, INTERVAL_YEAR_MONTH) => true - case (BIGINT, INTERVAL_DAY_TIME) => true - case (INTERVAL_YEAR_MONTH, INTEGER) => true - case (INTERVAL_DAY_TIME, BIGINT) => true - - case (DATE, BIGINT) => true - case (TIME_WITHOUT_TIME_ZONE, BIGINT) => true - case (INTERVAL_YEAR_MONTH, BIGINT) => true - - case _ => false - } } From 96a142866a042598bfe85407b46b0871a7b8993a Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 3 Nov 2023 11:02:19 +0100 Subject: [PATCH 076/104] [FLINK-33447][table-planner] Avoid CompiledPlan recompilation during loading --- .../plan/ExecNodeGraphInternalPlan.java | 15 ++++++++---- .../planner/delegation/StreamPlanner.scala | 24 ++++++++++--------- 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/ExecNodeGraphInternalPlan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/ExecNodeGraphInternalPlan.java index 7f138c6517a69..b921535b8b810 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/ExecNodeGraphInternalPlan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/ExecNodeGraphInternalPlan.java @@ -34,17 +34,21 @@ import java.nio.file.Files; import java.nio.file.StandardOpenOption; import java.util.List; +import java.util.function.Supplier; import java.util.stream.Collectors; /** Implementation of {@link CompiledPlan} backed by an {@link ExecNodeGraph}. */ @Internal public class ExecNodeGraphInternalPlan implements InternalPlan { - private final String serializedPlan; + private final Supplier serializedPlanSupplier; private final ExecNodeGraph execNodeGraph; - public ExecNodeGraphInternalPlan(String serializedPlan, ExecNodeGraph execNodeGraph) { - this.serializedPlan = serializedPlan; + private String serializedPlan; + + public ExecNodeGraphInternalPlan( + Supplier serializedPlanSupplier, ExecNodeGraph execNodeGraph) { + this.serializedPlanSupplier = serializedPlanSupplier; this.execNodeGraph = execNodeGraph; } @@ -54,6 +58,9 @@ public ExecNodeGraph getExecNodeGraph() { @Override public String asJsonString() { + if (serializedPlan == null) { + serializedPlan = serializedPlanSupplier.get(); + } return serializedPlan; } @@ -78,7 +85,7 @@ public void writeToFile(File file, boolean ignoreIfExists, boolean failIfExists) Files.createDirectories(file.toPath().getParent()); Files.write( file.toPath(), - serializedPlan.getBytes(StandardCharsets.UTF_8), + asJsonString().getBytes(StandardCharsets.UTF_8), StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING, StandardOpenOption.WRITE); diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala index 1029475b74827..fb32326f11787 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.dag.Transformation import org.apache.flink.configuration.ExecutionOptions import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader import org.apache.flink.streaming.api.graph.StreamGraph -import org.apache.flink.table.api.{ExplainDetail, ExplainFormat, PlanReference, TableConfig, TableException} +import org.apache.flink.table.api._ import org.apache.flink.table.api.PlanReference.{ContentPlanReference, FilePlanReference, ResourcePlanReference} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.flink.table.delegation.{Executor, InternalPlan} @@ -190,10 +190,12 @@ class StreamPlanner( } new ExecNodeGraphInternalPlan( - JsonSerdeUtil - .createObjectWriter(ctx) - .withDefaultPrettyPrinter() - .writeValueAsString(execNodeGraph), + // ensures that the JSON output is always normalized + () => + JsonSerdeUtil + .createObjectWriter(ctx) + .withDefaultPrettyPrinter() + .writeValueAsString(execNodeGraph), execNodeGraph) } @@ -204,12 +206,12 @@ class StreamPlanner( val execGraph = translateToExecNodeGraph(optimizedRelNodes, isCompiled = true) afterTranslation() - new ExecNodeGraphInternalPlan( - JsonSerdeUtil - .createObjectWriter(createSerdeContext) - .withDefaultPrettyPrinter() - .writeValueAsString(execGraph), - execGraph) + val compiledJson = JsonSerdeUtil + .createObjectWriter(createSerdeContext) + .withDefaultPrettyPrinter() + .writeValueAsString(execGraph) + + new ExecNodeGraphInternalPlan(() => compiledJson, execGraph) } override def translatePlan(plan: InternalPlan): util.List[Transformation[_]] = { From fccf26bb449b90b617c6523db17eab98a544c0a7 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Sun, 5 Nov 2023 15:52:45 +0100 Subject: [PATCH 077/104] [FLINK-33300] Upgrade testcontainers from 1.18.3 to 1.19.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d288ca5f5c0e6..66ceba2669c4b 100644 --- a/pom.xml +++ b/pom.xml @@ -160,7 +160,7 @@ under the License. 2.43.0 3.21.7 3.14.9 - 1.18.3 + 1.19.1 1.8.0 false validate From cee2611eac9f7845e97ce54742876c47f25e7e2b Mon Sep 17 00:00:00 2001 From: Zakelly Date: Thu, 7 Sep 2023 17:05:19 +0800 Subject: [PATCH 078/104] [FLINK-5865][state] Unwrapping exceptions and throw the original ones in state interfaces (#23371) --- .../state/ListDelimitedSerializer.java | 18 ++-- .../state/heap/HeapAggregatingState.java | 12 +-- .../runtime/state/heap/HeapListState.java | 3 +- .../runtime/state/heap/HeapReducingState.java | 10 +-- .../state/AbstractRocksDBAppendingState.java | 31 +++---- .../state/RocksDBAggregatingState.java | 87 +++++++++--------- .../streaming/state/RocksDBListState.java | 88 ++++++++----------- .../streaming/state/RocksDBReducingState.java | 84 +++++++++--------- .../streaming/state/RocksDBValueState.java | 13 ++- .../BatchExecutionKeyAggregatingState.java | 16 +--- .../state/BatchExecutionKeyReducingState.java | 18 ++-- 11 files changed, 159 insertions(+), 221 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java index ddfb9163cb085..0574c6f491ea1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import java.io.IOException; @@ -39,7 +38,8 @@ public final class ListDelimitedSerializer { private final DataInputDeserializer dataInputView = new DataInputDeserializer(); private final DataOutputSerializer dataOutputView = new DataOutputSerializer(128); - public List deserializeList(byte[] valueBytes, TypeSerializer elementSerializer) { + public List deserializeList(byte[] valueBytes, TypeSerializer elementSerializer) + throws IOException { if (valueBytes == null) { return null; } @@ -76,17 +76,13 @@ public byte[] serializeList(List valueList, TypeSerializer elementSeri /** Deserializes a single element from a serialized list. */ public static T deserializeNextElement( - DataInputDeserializer in, TypeSerializer elementSerializer) { - try { + DataInputDeserializer in, TypeSerializer elementSerializer) throws IOException { + if (in.available() > 0) { + T element = elementSerializer.deserialize(in); if (in.available() > 0) { - T element = elementSerializer.deserialize(in); - if (in.available() > 0) { - in.readByte(); - } - return element; + in.readByte(); } - } catch (IOException e) { - throw new FlinkRuntimeException("Unexpected list element deserialization failure", e); + return element; } return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java index fbf8914f87d6a..59a86de87f992 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java @@ -28,8 +28,6 @@ import org.apache.flink.runtime.state.internal.InternalAggregatingState; import org.apache.flink.util.Preconditions; -import java.io.IOException; - /** * Heap-backed partitioned {@link AggregatingState} that is snapshotted into files. * @@ -93,20 +91,14 @@ public OUT get() { } @Override - public void add(IN value) throws IOException { + public void add(IN value) throws Exception { final N namespace = currentNamespace; if (value == null) { clear(); return; } - - try { - stateTable.transform(namespace, value, aggregateTransformation); - } catch (Exception e) { - throw new IOException( - "Exception while applying AggregateFunction in aggregating state", e); - } + stateTable.transform(namespace, value, aggregateTransformation); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java index 0f32d0bbc270a..919700f539d6a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java @@ -30,6 +30,7 @@ import org.apache.flink.util.Preconditions; import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -106,7 +107,7 @@ public byte[] getSerializedValue( final TypeSerializer safeKeySerializer, final TypeSerializer safeNamespaceSerializer, final TypeSerializer> safeValueSerializer) - throws Exception { + throws IOException { Preconditions.checkNotNull(serializedKeyAndNamespace); Preconditions.checkNotNull(safeKeySerializer); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java index 18739252ac516..b7926af7d6fc7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java @@ -28,8 +28,6 @@ import org.apache.flink.runtime.state.internal.InternalReducingState; import org.apache.flink.util.Preconditions; -import java.io.IOException; - /** * Heap-backed partitioned {@link ReducingState} that is snapshotted into files. * @@ -89,18 +87,14 @@ public V get() { } @Override - public void add(V value) throws IOException { + public void add(V value) throws Exception { if (value == null) { clear(); return; } - try { - stateTable.transform(currentNamespace, value, reduceTransformation); - } catch (Exception e) { - throw new IOException("Exception while applying ReduceFunction in reducing state", e); - } + stateTable.transform(currentNamespace, value, reduceTransformation); } // ------------------------------------------------------------------------ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java index c5373f83b8bc1..5328a62d7b0aa 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBAppendingState.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.internal.InternalAppendingState; -import org.apache.flink.util.FlinkRuntimeException; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.RocksDBException; @@ -50,34 +49,26 @@ protected AbstractRocksDBAppendingState( } @Override - public SV getInternal() { + public SV getInternal() throws IOException, RocksDBException { return getInternal(getKeyBytes()); } - SV getInternal(byte[] key) { - try { - byte[] valueBytes = backend.db.get(columnFamily, key); - if (valueBytes == null) { - return null; - } - dataInputView.setBuffer(valueBytes); - return valueSerializer.deserialize(dataInputView); - } catch (IOException | RocksDBException e) { - throw new FlinkRuntimeException("Error while retrieving data from RocksDB", e); + SV getInternal(byte[] key) throws IOException, RocksDBException { + byte[] valueBytes = backend.db.get(columnFamily, key); + if (valueBytes == null) { + return null; } + dataInputView.setBuffer(valueBytes); + return valueSerializer.deserialize(dataInputView); } @Override - public void updateInternal(SV valueToStore) { + public void updateInternal(SV valueToStore) throws RocksDBException { updateInternal(getKeyBytes(), valueToStore); } - void updateInternal(byte[] key, SV valueToStore) { - try { - // write the new value to RocksDB - backend.db.put(columnFamily, writeOptions, key, getValueBytes(valueToStore)); - } catch (RocksDBException e) { - throw new FlinkRuntimeException("Error while adding value to RocksDB", e); - } + void updateInternal(byte[] key, SV valueToStore) throws RocksDBException { + // write the new value to RocksDB + backend.db.put(columnFamily, writeOptions, key, getValueBytes(valueToStore)); } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java index ba1e0e97ef219..fc556cae3f70b 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java @@ -27,10 +27,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; import org.apache.flink.runtime.state.internal.InternalAggregatingState; -import org.apache.flink.util.FlinkRuntimeException; import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import java.io.IOException; import java.util.Collection; /** @@ -87,7 +88,7 @@ public TypeSerializer getValueSerializer() { } @Override - public R get() { + public R get() throws IOException, RocksDBException { ACC accumulator = getInternal(); if (accumulator == null) { return null; @@ -96,7 +97,7 @@ public R get() { } @Override - public void add(T value) { + public void add(T value) throws IOException, RocksDBException { byte[] key = getKeyBytes(); ACC accumulator = getInternal(key); accumulator = accumulator == null ? aggFunction.createAccumulator() : accumulator; @@ -104,60 +105,56 @@ public void add(T value) { } @Override - public void mergeNamespaces(N target, Collection sources) { + public void mergeNamespaces(N target, Collection sources) + throws IOException, RocksDBException { if (sources == null || sources.isEmpty()) { return; } - try { - ACC current = null; - - // merge the sources to the target - for (N source : sources) { - if (source != null) { - setCurrentNamespace(source); - final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); - final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); - - if (valueBytes != null) { - backend.db.delete(columnFamily, writeOptions, sourceKey); - dataInputView.setBuffer(valueBytes); - ACC value = valueSerializer.deserialize(dataInputView); - - if (current != null) { - current = aggFunction.merge(current, value); - } else { - current = value; - } - } - } - } + ACC current = null; - // if something came out of merging the sources, merge it or write it to the target - if (current != null) { - setCurrentNamespace(target); - // create the target full-binary-key - final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); - final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); + // merge the sources to the target + for (N source : sources) { + if (source != null) { + setCurrentNamespace(source); + final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); - if (targetValueBytes != null) { - // target also had a value, merge - dataInputView.setBuffer(targetValueBytes); + if (valueBytes != null) { + backend.db.delete(columnFamily, writeOptions, sourceKey); + dataInputView.setBuffer(valueBytes); ACC value = valueSerializer.deserialize(dataInputView); - current = aggFunction.merge(current, value); + if (current != null) { + current = aggFunction.merge(current, value); + } else { + current = value; + } } + } + } + + // if something came out of merging the sources, merge it or write it to the target + if (current != null) { + setCurrentNamespace(target); + // create the target full-binary-key + final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); - // serialize the resulting value - dataOutputView.clear(); - valueSerializer.serialize(current, dataOutputView); + if (targetValueBytes != null) { + // target also had a value, merge + dataInputView.setBuffer(targetValueBytes); + ACC value = valueSerializer.deserialize(dataInputView); - // write the resulting value - backend.db.put( - columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); + current = aggFunction.merge(current, value); } - } catch (Exception e) { - throw new FlinkRuntimeException("Error while merging state in RocksDB", e); + + // serialize the resulting value + dataOutputView.clear(); + valueSerializer.serialize(current, dataOutputView); + + // write the resulting value + backend.db.put(columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java index 553d86a63ce36..b5339fd4d25af 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java @@ -108,34 +108,26 @@ public TypeSerializer> getValueSerializer() { } @Override - public Iterable get() { + public Iterable get() throws IOException, RocksDBException { return getInternal(); } @Override - public List getInternal() { - try { - byte[] key = serializeCurrentKeyWithGroupAndNamespace(); - byte[] valueBytes = backend.db.get(columnFamily, key); - return listSerializer.deserializeList(valueBytes, elementSerializer); - } catch (RocksDBException e) { - throw new FlinkRuntimeException("Error while retrieving data from RocksDB", e); - } + public List getInternal() throws IOException, RocksDBException { + byte[] key = serializeCurrentKeyWithGroupAndNamespace(); + byte[] valueBytes = backend.db.get(columnFamily, key); + return listSerializer.deserializeList(valueBytes, elementSerializer); } @Override - public void add(V value) { + public void add(V value) throws IOException, RocksDBException { Preconditions.checkNotNull(value, "You cannot add null to a ListState."); - try { - backend.db.merge( - columnFamily, - writeOptions, - serializeCurrentKeyWithGroupAndNamespace(), - serializeValue(value, elementSerializer)); - } catch (Exception e) { - throw new FlinkRuntimeException("Error while adding data to RocksDB", e); - } + backend.db.merge( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + serializeValue(value, elementSerializer)); } @Override @@ -169,43 +161,35 @@ public void mergeNamespaces(N target, Collection sources) { } @Override - public void update(List valueToStore) { + public void update(List valueToStore) throws IOException, RocksDBException { updateInternal(valueToStore); } @Override - public void updateInternal(List values) { + public void updateInternal(List values) throws IOException, RocksDBException { Preconditions.checkNotNull(values, "List of values to add cannot be null."); if (!values.isEmpty()) { - try { - backend.db.put( - columnFamily, - writeOptions, - serializeCurrentKeyWithGroupAndNamespace(), - listSerializer.serializeList(values, elementSerializer)); - } catch (IOException | RocksDBException e) { - throw new FlinkRuntimeException("Error while updating data to RocksDB", e); - } + backend.db.put( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + listSerializer.serializeList(values, elementSerializer)); } else { clear(); } } @Override - public void addAll(List values) { + public void addAll(List values) throws IOException, RocksDBException { Preconditions.checkNotNull(values, "List of values to add cannot be null."); if (!values.isEmpty()) { - try { - backend.db.merge( - columnFamily, - writeOptions, - serializeCurrentKeyWithGroupAndNamespace(), - listSerializer.serializeList(values, elementSerializer)); - } catch (IOException | RocksDBException e) { - throw new FlinkRuntimeException("Error while updating data to RocksDB", e); - } + backend.db.merge( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + listSerializer.serializeList(values, elementSerializer)); } } @@ -309,19 +293,21 @@ public byte[] filterOrTransform(@Nullable byte[] value) { in.setBuffer(value); T next; int prevPosition = 0; - while ((next = ListDelimitedSerializer.deserializeNextElement(in, elementSerializer)) - != null) { - T transformedElement = elementTransformer.filterOrTransform(next); - if (transformedElement != null) { - if (transformStrategy == STOP_ON_FIRST_INCLUDED) { - return Arrays.copyOfRange(value, prevPosition, value.length); - } else { - result.add(transformedElement); + try { + while ((next = + ListDelimitedSerializer.deserializeNextElement( + in, elementSerializer)) + != null) { + T transformedElement = elementTransformer.filterOrTransform(next); + if (transformedElement != null) { + if (transformStrategy == STOP_ON_FIRST_INCLUDED) { + return Arrays.copyOfRange(value, prevPosition, value.length); + } else { + result.add(transformedElement); + } } + prevPosition = in.getPosition(); } - prevPosition = in.getPosition(); - } - try { return result.isEmpty() ? null : listSerializer.serializeList(result, elementSerializer); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java index f50f11e9d72ad..e791b3f591a7e 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java @@ -27,10 +27,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; import org.apache.flink.runtime.state.internal.InternalReducingState; -import org.apache.flink.util.FlinkRuntimeException; import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import java.io.IOException; import java.util.Collection; /** @@ -84,7 +85,7 @@ public TypeSerializer getValueSerializer() { } @Override - public V get() { + public V get() throws IOException, RocksDBException { return getInternal(); } @@ -97,60 +98,55 @@ public void add(V value) throws Exception { } @Override - public void mergeNamespaces(N target, Collection sources) { + public void mergeNamespaces(N target, Collection sources) throws Exception { if (sources == null || sources.isEmpty()) { return; } - try { - V current = null; - - // merge the sources to the target - for (N source : sources) { - if (source != null) { - setCurrentNamespace(source); - final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); - final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); - - if (valueBytes != null) { - backend.db.delete(columnFamily, writeOptions, sourceKey); - dataInputView.setBuffer(valueBytes); - V value = valueSerializer.deserialize(dataInputView); - - if (current != null) { - current = reduceFunction.reduce(current, value); - } else { - current = value; - } - } - } - } + V current = null; - // if something came out of merging the sources, merge it or write it to the target - if (current != null) { - // create the target full-binary-key - setCurrentNamespace(target); - final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); - final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); + // merge the sources to the target + for (N source : sources) { + if (source != null) { + setCurrentNamespace(source); + final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); - if (targetValueBytes != null) { - dataInputView.setBuffer(targetValueBytes); - // target also had a value, merge + if (valueBytes != null) { + backend.db.delete(columnFamily, writeOptions, sourceKey); + dataInputView.setBuffer(valueBytes); V value = valueSerializer.deserialize(dataInputView); - current = reduceFunction.reduce(current, value); + if (current != null) { + current = reduceFunction.reduce(current, value); + } else { + current = value; + } } + } + } + + // if something came out of merging the sources, merge it or write it to the target + if (current != null) { + // create the target full-binary-key + setCurrentNamespace(target); + final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); - // serialize the resulting value - dataOutputView.clear(); - valueSerializer.serialize(current, dataOutputView); + if (targetValueBytes != null) { + dataInputView.setBuffer(targetValueBytes); + // target also had a value, merge + V value = valueSerializer.deserialize(dataInputView); - // write the resulting value - backend.db.put( - columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); + current = reduceFunction.reduce(current, value); } - } catch (Exception e) { - throw new FlinkRuntimeException("Error while merging state in RocksDB", e); + + // serialize the resulting value + dataOutputView.clear(); + valueSerializer.serialize(current, dataOutputView); + + // write the resulting value + backend.db.put(columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java index 3cc780263cfdd..958efad068ab2 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java @@ -25,7 +25,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; import org.apache.flink.runtime.state.internal.InternalValueState; -import org.apache.flink.util.FlinkRuntimeException; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.RocksDBException; @@ -77,7 +76,7 @@ public TypeSerializer getValueSerializer() { } @Override - public V value() { + public V value() throws IOException { try { byte[] valueBytes = backend.db.get(columnFamily, serializeCurrentKeyWithGroupAndNamespace()); @@ -87,13 +86,13 @@ public V value() { } dataInputView.setBuffer(valueBytes); return valueSerializer.deserialize(dataInputView); - } catch (IOException | RocksDBException e) { - throw new FlinkRuntimeException("Error while retrieving data from RocksDB.", e); + } catch (RocksDBException e) { + throw new IOException("Error while retrieving data from RocksDB.", e); } } @Override - public void update(V value) { + public void update(V value) throws IOException { if (value == null) { clear(); return; @@ -105,8 +104,8 @@ public void update(V value) { writeOptions, serializeCurrentKeyWithGroupAndNamespace(), serializeValue(value)); - } catch (Exception e) { - throw new FlinkRuntimeException("Error while adding data to RocksDB", e); + } catch (RocksDBException e) { + throw new IOException("Error while adding data to RocksDB", e); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyAggregatingState.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyAggregatingState.java index 08e34f04b38e9..7dec9f21d373c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyAggregatingState.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyAggregatingState.java @@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.internal.InternalAggregatingState; -import java.io.IOException; - /** An {@link AggregatingState} which keeps value for a single key at a time. */ class BatchExecutionKeyAggregatingState extends MergingAbstractBatchExecutionKeyState @@ -52,21 +50,15 @@ public OUT get() { } @Override - public void add(IN value) throws IOException { + public void add(IN value) { if (value == null) { clear(); return; } - - try { - if (getCurrentNamespaceValue() == null) { - setCurrentNamespaceValue(aggFunction.createAccumulator()); - } - setCurrentNamespaceValue(aggFunction.add(value, getCurrentNamespaceValue())); - } catch (Exception e) { - throw new IOException( - "Exception while applying AggregateFunction in aggregating state", e); + if (getCurrentNamespaceValue() == null) { + setCurrentNamespaceValue(aggFunction.createAccumulator()); } + setCurrentNamespaceValue(aggFunction.add(value, getCurrentNamespaceValue())); } @SuppressWarnings("unchecked") diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyReducingState.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyReducingState.java index 54a7bef89171d..8ff81b6338906 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyReducingState.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyReducingState.java @@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.internal.InternalReducingState; -import java.io.IOException; - /** A {@link ReducingState} which keeps value for a single key at a time. */ class BatchExecutionKeyReducingState extends MergingAbstractBatchExecutionKeyState @@ -50,21 +48,17 @@ public T get() { } @Override - public void add(T value) throws IOException { + public void add(T value) throws Exception { if (value == null) { clear(); return; } - try { - T currentNamespaceValue = getCurrentNamespaceValue(); - if (currentNamespaceValue != null) { - setCurrentNamespaceValue(reduceFunction.reduce(currentNamespaceValue, value)); - } else { - setCurrentNamespaceValue(value); - } - } catch (Exception e) { - throw new IOException("Exception while applying ReduceFunction in reducing state", e); + T currentNamespaceValue = getCurrentNamespaceValue(); + if (currentNamespaceValue != null) { + setCurrentNamespaceValue(reduceFunction.reduce(currentNamespaceValue, value)); + } else { + setCurrentNamespaceValue(value); } } From 04b98e6e80ca3a68324775166dedb7d58ae4d60b Mon Sep 17 00:00:00 2001 From: fengjiajie Date: Mon, 6 Nov 2023 14:21:40 +0800 Subject: [PATCH 079/104] [FLINK-33171][table-planner] Consistent implicit type coercion support for equal and non-equal comparisons for codegen This closes #23478 --- .../codegen/calls/ScalarOperatorGens.scala | 202 ++++++++-------- .../expressions/ScalarOperatorsTest.scala | 226 ++++++++++++++++++ .../utils/ExpressionTestBase.scala | 1 + .../utils/ScalarOperatorsTestBase.scala | 41 +++- .../runtime/batch/sql/MiscITCase.scala | 92 +++++++ 5 files changed, 455 insertions(+), 107 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 084b3aa503ec0..230b09adccc11 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -345,44 +345,83 @@ object ScalarOperatorGens { } } - def generateEquals( + private def wrapExpressionIfNonEq( + isNonEq: Boolean, + equalsExpr: GeneratedExpression, + resultType: LogicalType): GeneratedExpression = { + if (isNonEq) { + GeneratedExpression( + s"(!${equalsExpr.resultTerm})", + equalsExpr.nullTerm, + equalsExpr.code, + resultType) + } else { + equalsExpr + } + } + + private def generateEqualAndNonEqual( ctx: CodeGeneratorContext, left: GeneratedExpression, right: GeneratedExpression, + operator: String, resultType: LogicalType): GeneratedExpression = { + checkImplicitConversionValidity(left, right) + + val nonEq = operator match { + case "==" => false + case "!=" => true + case _ => throw new CodeGenException(s"Unsupported boolean comparison '$operator'.") + } val canEqual = isInteroperable(left.resultType, right.resultType) + if (isCharacterString(left.resultType) && isCharacterString(right.resultType)) { generateOperatorIfNotNull(ctx, resultType, left, right)( - (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)") + (leftTerm, rightTerm) => s"${if (nonEq) "!" else ""}$leftTerm.equals($rightTerm)") } // numeric types else if (isNumeric(left.resultType) && isNumeric(right.resultType)) { - generateComparison(ctx, "==", left, right, resultType) + generateComparison(ctx, operator, left, right, resultType) } // array types else if (isArray(left.resultType) && canEqual) { - generateArrayComparison(ctx, left, right, resultType) + wrapExpressionIfNonEq( + nonEq, + generateArrayComparison(ctx, left, right, resultType), + resultType) } // map types else if (isMap(left.resultType) && canEqual) { val mapType = left.resultType.asInstanceOf[MapType] - generateMapComparison(ctx, left, right, mapType.getKeyType, mapType.getValueType, resultType) + wrapExpressionIfNonEq( + nonEq, + generateMapComparison( + ctx, + left, + right, + mapType.getKeyType, + mapType.getValueType, + resultType), + resultType) } // multiset types else if (isMultiset(left.resultType) && canEqual) { val multisetType = left.resultType.asInstanceOf[MultisetType] - generateMapComparison( - ctx, - left, - right, - multisetType.getElementType, - new IntType(false), + wrapExpressionIfNonEq( + nonEq, + generateMapComparison( + ctx, + left, + right, + multisetType.getElementType, + new IntType(false), + resultType), resultType) } // comparable types of same type else if (isComparable(left.resultType) && canEqual) { - generateComparison(ctx, "==", left, right, resultType) + generateComparison(ctx, operator, left, right, resultType) } // generic types of same type else if (isRaw(left.resultType) && canEqual) { @@ -399,7 +438,7 @@ object ScalarOperatorGens { | ${left.resultTerm}.ensureMaterialized($ser); | ${right.resultTerm}.ensureMaterialized($ser); | $resultTerm = - | ${left.resultTerm}.getBinarySection(). + | ${if (nonEq) "!" else ""}${left.resultTerm}.getBinarySection(). | equals(${right.resultTerm}.getBinarySection()); |} |""".stripMargin @@ -407,43 +446,58 @@ object ScalarOperatorGens { } // support date/time/timestamp equalTo string. // for performance, we cast literal string to literal time. - else if (isTimePoint(left.resultType) && isCharacterString(right.resultType)) { - if (right.literal) { - generateEquals(ctx, left, generateCastLiteral(ctx, right, left.resultType), resultType) - } else { - generateEquals( - ctx, - left, - generateCast(ctx, right, left.resultType, nullOnFailure = true), - resultType) - } - } else if (isTimePoint(right.resultType) && isCharacterString(left.resultType)) { - if (left.literal) { - generateEquals(ctx, generateCastLiteral(ctx, left, right.resultType), right, resultType) - } else { - generateEquals( - ctx, - generateCast(ctx, left, right.resultType, nullOnFailure = true), - right, - resultType) - } + else if ( + (isTimePoint(left.resultType) && isCharacterString(right.resultType)) || (isTimePoint( + right.resultType) && isCharacterString(left.resultType)) + ) { + val (newLeft, newRight) = + if (isTimePoint(left.resultType)) (left, right) + else (right, left) + generateEqualAndNonEqual( + ctx, + newLeft, + if (newRight.literal) { + generateCastLiteral(ctx, newRight, newLeft.resultType) + } else { + generateCast(ctx, newRight, newLeft.resultType, nullOnFailure = true) + }, + operator, + resultType + ) } // non comparable types else { - generateOperatorIfNotNull(ctx, resultType, left, right) { - if (isReference(left.resultType)) { - (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)" - } else if (isReference(right.resultType)) { - (leftTerm, rightTerm) => s"$rightTerm.equals($leftTerm)" - } else { - throw new CodeGenException( - s"Incomparable types: ${left.resultType} and " + - s"${right.resultType}") - } + val (newLeft, newRight) = if (isReference(left.resultType)) { + (left, right) + } else if (isReference(right.resultType)) { + (right, left) + } else { + throw new CodeGenException( + s"Incomparable types: ${left.resultType} and " + + s"${right.resultType}") + } + generateOperatorIfNotNull(ctx, resultType, newLeft, newRight) { + (leftTerm, rightTerm) => s"${if (nonEq) "!" else ""}$leftTerm.equals($rightTerm)" } } } + def generateEquals( + ctx: CodeGeneratorContext, + left: GeneratedExpression, + right: GeneratedExpression, + resultType: LogicalType): GeneratedExpression = { + generateEqualAndNonEqual(ctx, left, right, "==", resultType) + } + + def generateNotEquals( + ctx: CodeGeneratorContext, + left: GeneratedExpression, + right: GeneratedExpression, + resultType: LogicalType): GeneratedExpression = { + generateEqualAndNonEqual(ctx, left, right, "!=", resultType) + } + def generateIsDistinctFrom( ctx: CodeGeneratorContext, left: GeneratedExpression, @@ -474,68 +528,6 @@ object ScalarOperatorGens { ) } - def generateNotEquals( - ctx: CodeGeneratorContext, - left: GeneratedExpression, - right: GeneratedExpression, - resultType: LogicalType): GeneratedExpression = { - checkImplicitConversionValidity(left, right) - if (isCharacterString(left.resultType) && isCharacterString(right.resultType)) { - generateOperatorIfNotNull(ctx, resultType, left, right)( - (leftTerm, rightTerm) => s"!$leftTerm.equals($rightTerm)") - } - // numeric types - else if (isNumeric(left.resultType) && isNumeric(right.resultType)) { - generateComparison(ctx, "!=", left, right, resultType) - } - // temporal types - else if ( - isTemporal(left.resultType) && - isInteroperable(left.resultType, right.resultType) - ) { - generateComparison(ctx, "!=", left, right, resultType) - } - // array types - else if (isArray(left.resultType) && isInteroperable(left.resultType, right.resultType)) { - val equalsExpr = generateEquals(ctx, left, right, resultType) - GeneratedExpression( - s"(!${equalsExpr.resultTerm})", - equalsExpr.nullTerm, - equalsExpr.code, - resultType) - } - // map types - else if (isMap(left.resultType) && isInteroperable(left.resultType, right.resultType)) { - val equalsExpr = generateEquals(ctx, left, right, resultType) - GeneratedExpression( - s"(!${equalsExpr.resultTerm})", - equalsExpr.nullTerm, - equalsExpr.code, - resultType) - } - // comparable types - else if ( - isComparable(left.resultType) && - isInteroperable(left.resultType, right.resultType) - ) { - generateComparison(ctx, "!=", left, right, resultType) - } - // non-comparable types - else { - generateOperatorIfNotNull(ctx, resultType, left, right) { - if (isReference(left.resultType)) { - (leftTerm, rightTerm) => s"!($leftTerm.equals($rightTerm))" - } else if (isReference(right.resultType)) { - (leftTerm, rightTerm) => s"!($rightTerm.equals($leftTerm))" - } else { - throw new CodeGenException( - s"Incomparable types: ${left.resultType} and " + - s"${right.resultType}") - } - } - } - } - /** Generates comparison code for numeric types and comparable types of same type. */ def generateComparison( ctx: CodeGeneratorContext, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala index 912b138fda5d3..44fac362f8f65 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala @@ -181,12 +181,21 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { testSqlApi("'12:34:56' = f21", "TRUE") testSqlApi("'13:34:56' = f21", "FALSE") + testSqlApi("TYPEOF(f22)", "TIMESTAMP(6)") testSqlApi("f22 = '1996-11-10 12:34:56'", "TRUE") testSqlApi("f22 = '1996-11-10 12:34:57'", "FALSE") testSqlApi("f22 = cast(null as string)", "NULL") testSqlApi("'1996-11-10 12:34:56' = f22", "TRUE") testSqlApi("'1996-11-10 12:34:57' = f22", "FALSE") testSqlApi("cast(null as string) = f22", "NULL") + + testSqlApi("TYPEOF(f23)", "TIMESTAMP_LTZ(6)") + testSqlApi("f23 = '1996-11-10 12:34:56'", "TRUE") + testSqlApi("f23 = '1996-11-10 12:34:57'", "FALSE") + testSqlApi("f23 = cast(null as string)", "NULL") + testSqlApi("'1996-11-10 12:34:56' = f23", "TRUE") + testSqlApi("'1996-11-10 12:34:57' = f23", "FALSE") + testSqlApi("cast(null as string) = f23", "NULL") } @Test @@ -226,6 +235,223 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { "date_format(f22 + interval '1' second, 'yyyy-MM-dd HH:mm:ss') = cast(f22 as timestamp_ltz)", "FALSE") testSqlApi("uuid() = cast(f22 as timestamp_ltz)", "NULL") + + testSqlApi("f23 = date_format(f23, 'yyyy-MM-dd HH:mm:ss')", "TRUE") + testSqlApi("f23 = date_format(f23 + interval '1' second, 'yyyy-MM-dd HH:mm:ss')", "FALSE") + testSqlApi("f23 = uuid()", "NULL") + testSqlApi("date_format(f23, 'yyyy-MM-dd HH:mm:ss') = f23", "TRUE") + testSqlApi("date_format(f23 + interval '1' second, 'yyyy-MM-dd HH:mm:ss') = f23", "FALSE") + testSqlApi("uuid() = f23", "NULL") + } + + @Test + def testTimePointTypeNotEqualsString(): Unit = { + testSqlApi("NOT(f15 = '1996-11-10')", "FALSE") + testSqlApi("NOT(f15 = '1996-11-11')", "TRUE") + testSqlApi("NOT('1996-11-10' = f15)", "FALSE") + testSqlApi("NOT('1996-11-11' = f15)", "TRUE") + + testSqlApi("NOT(f21 = '12:34:56')", "FALSE") + testSqlApi("NOT(f21 = '13:34:56')", "TRUE") + testSqlApi("NOT('12:34:56' = f21)", "FALSE") + testSqlApi("NOT('13:34:56' = f21)", "TRUE") + + testSqlApi("TYPEOF(f22)", "TIMESTAMP(6)") + testSqlApi("NOT(f22 = '1996-11-10 12:34:56')", "FALSE") + testSqlApi("NOT(f22 = '1996-11-10 12:34:57')", "TRUE") + testSqlApi("NOT(f22 = cast(null as string))", "NULL") + testSqlApi("NOT('1996-11-10 12:34:56' = f22)", "FALSE") + testSqlApi("NOT('1996-11-10 12:34:57' = f22)", "TRUE") + testSqlApi("NOT(cast(null as string) = f22)", "NULL") + + testSqlApi("TYPEOF(f23)", "TIMESTAMP_LTZ(6)") + testSqlApi("f23 = '1996-11-10 12:34:56'", "TRUE") + testSqlApi("f23 = '1996-11-10 12:34:57'", "FALSE") + testSqlApi("f23 = cast(null as string)", "NULL") + testSqlApi("'1996-11-10 12:34:56' = f23", "TRUE") + testSqlApi("'1996-11-10 12:34:57' = f23", "FALSE") + testSqlApi("cast(null as string) = f23", "NULL") + + testSqlApi("NOT(f15 = date_format(cast(f15 as timestamp), 'yyyy-MM-dd'))", "FALSE") + testSqlApi( + "NOT(f15 = date_format(cast(f15 as timestamp) + interval '1' day, 'yyyy-MM-dd'))", + "TRUE") + testSqlApi("NOT(f15 = uuid())", "NULL") + testSqlApi("NOT(date_format(cast(f15 as timestamp), 'yyyy-MM-dd') = f15)", "FALSE") + testSqlApi( + "NOT(date_format(cast(f15 as timestamp) + interval '1' day, 'yyyy-MM-dd')) = f15", + "TRUE") + testSqlApi("NOT(uuid() = f15)", "NULL") + + testSqlApi("NOT(f21 = date_format(cast(f21 as timestamp), 'HH:mm:ss'))", "FALSE") + testSqlApi( + "NOT(f21 = date_format(cast(f21 as timestamp) + interval '1' hour, 'HH:mm:ss'))", + "TRUE") + testSqlApi("NOT(f21 = uuid())", "NULL") + testSqlApi("NOT(date_format(cast(f21 as timestamp), 'HH:mm:ss') = f21)", "FALSE") + testSqlApi( + "NOT(date_format(cast(f21 as timestamp) + interval '1' hour, 'HH:mm:ss') = f21)", + "TRUE") + testSqlApi("NOT(uuid() = f21)", "NULL") + + testSqlApi("NOT(f22 = date_format(f22, 'yyyy-MM-dd HH:mm:ss'))", "FALSE") + testSqlApi("NOT(f22 = date_format(f22 + interval '1' second, 'yyyy-MM-dd HH:mm:ss'))", "TRUE") + testSqlApi("NOT(f22 = uuid())", "NULL") + testSqlApi("NOT(date_format(f22, 'yyyy-MM-dd HH:mm:ss') = f22)", "FALSE") + testSqlApi("NOT(date_format(f22 + interval '1' second, 'yyyy-MM-dd HH:mm:ss') = f22)", "TRUE") + testSqlApi("NOT(uuid() = f22)", "NULL") + + testSqlApi("NOT(cast(f22 as timestamp_ltz) = date_format(f22, 'yyyy-MM-dd HH:mm:ss'))", "FALSE") + testSqlApi( + "NOT(cast(f22 as timestamp_ltz) = date_format(f22 + interval '1' second, 'yyyy-MM-dd HH:mm:ss'))", + "TRUE") + testSqlApi("NOT(cast(f22 as timestamp_ltz) = uuid())", "NULL") + testSqlApi("NOT(date_format(f22, 'yyyy-MM-dd HH:mm:ss') = cast(f22 as timestamp_ltz))", "FALSE") + testSqlApi( + "NOT(date_format(f22 + interval '1' second, 'yyyy-MM-dd HH:mm:ss') = cast(f22 as timestamp_ltz))", + "TRUE") + testSqlApi("NOT(uuid() = cast(f22 as timestamp_ltz))", "NULL") + + testSqlApi("NOT(f23 = date_format(f23, 'yyyy-MM-dd HH:mm:ss'))", "FALSE") + testSqlApi("NOT(f23 = date_format(f23 + interval '1' second, 'yyyy-MM-dd HH:mm:ss'))", "TRUE") + testSqlApi("NOT(f23 = uuid())", "NULL") + testSqlApi("NOT(date_format(f23, 'yyyy-MM-dd HH:mm:ss') = f23)", "FALSE") + testSqlApi("NOT(date_format(f23 + interval '1' second, 'yyyy-MM-dd HH:mm:ss') = f23)", "TRUE") + testSqlApi("NOT(uuid() = f23)", "NULL") + } + + @Test + def testMoreEqualAndNonEqual(): Unit = { + // character string + testSqlApi("f10 = 'String'", "TRUE") + testSqlApi("f10 = 'string'", "FALSE") + testSqlApi("f10 = NULL", "NULL") + testSqlApi("f10 = CAST(NULL AS STRING)", "NULL") + testSqlApi("'String' = f10", "TRUE") + testSqlApi("'string' = f10", "FALSE") + testSqlApi("NULL = f10", "NULL") + testSqlApi("CAST(NULL AS STRING) = f10", "NULL") + + testSqlApi("NOT(f10 = 'String')", "FALSE") + testSqlApi("NOT(f10 = 'string')", "TRUE") + testSqlApi("NOT(f10 = NULL)", "NULL") + testSqlApi("NOT(f10 = CAST(NULL AS STRING))", "NULL") + testSqlApi("NOT('String' = f10)", "FALSE") + testSqlApi("NOT('string' = f10)", "TRUE") + testSqlApi("NOT(NULL = f10)", "NULL") + testSqlApi("NOT(CAST(NULL AS STRING) = f10)", "NULL") + + // numeric types + testSqlApi("f2 = 1", "TRUE") + testSqlApi("f2 = 2", "FALSE") + testSqlApi("f2 = NULL", "NULL") + testSqlApi("f2 = CAST(NULL AS INT)", "NULL") + testSqlApi("1 = f2", "TRUE") + testSqlApi("2 = f2", "FALSE") + testSqlApi("NULL = f2", "NULL") + testSqlApi("CAST(NULL AS INT) = f2", "NULL") + + testSqlApi("NOT(f2 = 1)", "FALSE") + testSqlApi("NOT(f2 = 2)", "TRUE") + testSqlApi("NOT(f2 = NULL)", "NULL") + testSqlApi("NOT(f2 = CAST(NULL AS INT))", "NULL") + testSqlApi("NOT(1 = f2)", "FALSE") + testSqlApi("NOT(2 = f2)", "TRUE") + testSqlApi("NOT(NULL = f2)", "NULL") + testSqlApi("NOT(CAST(NULL AS INT) = f2)", "NULL") + + // array + testSqlApi("f24 = ARRAY['hello', 'world']", "TRUE") + testSqlApi("f24 = ARRAY['hello1', 'world']", "FALSE") + testSqlApi("f24 = NULL", "NULL") + testSqlApi("f24 = CAST(NULL AS ARRAY)", "NULL") + testSqlApi("ARRAY['hello', 'world'] = f24", "TRUE") + testSqlApi("ARRAY['hello1', 'world'] = f24", "FALSE") + testSqlApi("NULL = f24", "NULL") + testSqlApi("CAST(NULL AS ARRAY) = f24", "NULL") + + testSqlApi("NOT(f24 = ARRAY['hello', 'world'])", "FALSE") + testSqlApi("NOT(f24 = ARRAY['hello1', 'world'])", "TRUE") + testSqlApi("NOT(f24 = NULL)", "NULL") + testSqlApi("NOT(f24 = CAST(NULL AS ARRAY))", "NULL") + testSqlApi("NOT(ARRAY['hello', 'world'] = f24)", "FALSE") + testSqlApi("NOT(ARRAY['hello1', 'world'] = f24)", "TRUE") + testSqlApi("NOT(NULL = f24)", "NULL") + testSqlApi("NOT(CAST(NULL AS ARRAY)) = f24", "NULL") + + // map + testSqlApi("f25 = MAP['a', 1, 'b', 2]", "TRUE") + testSqlApi("f25 = MAP['a', 3, 'b', 2]", "FALSE") + testSqlApi("f25 = NULL", "NULL") + testSqlApi("f25 = CAST(NULL AS MAP)", "NULL") + testSqlApi("MAP['a', 1, 'b', 2] = f25", "TRUE") + testSqlApi("MAP['a', 3, 'b', 2] = f25", "FALSE") + testSqlApi("NULL = f25", "NULL") + testSqlApi("CAST(NULL AS MAP) = f25", "NULL") + + testSqlApi("NOT(f25 = MAP['a', 1, 'b', 2])", "FALSE") + testSqlApi("NOT(f25 = MAP['a', 3, 'b', 2])", "TRUE") + testSqlApi("NOT(f25 = NULL)", "NULL") + testSqlApi("NOT(f25 = CAST(NULL AS MAP))", "NULL") + testSqlApi("NOT(MAP['a', 1, 'b', 2] = f25)", "FALSE") + testSqlApi("NOT(MAP['a', 3, 'b', 2] = f25)", "TRUE") + testSqlApi("NOT(NULL = f25)", "NULL") + testSqlApi("NOT(CAST(NULL AS MAP) = f25)", "NULL") + + // raw + testSqlApi("f27 = f29", "TRUE") + testSqlApi("f27 = f28", "FALSE") + testSqlApi("f27 = NULL", "NULL") + testSqlApi("f29 = f27", "TRUE") + testSqlApi("f28 = f27", "FALSE") + testSqlApi("NULL = f27", "NULL") + + testSqlApi("NOT(f27 = f29)", "FALSE") + testSqlApi("NOT(f27 = f28)", "TRUE") + testSqlApi("NOT(f27 = NULL)", "NULL") + testSqlApi("NOT(f29 = f27)", "FALSE") + testSqlApi("NOT(f28 = f27)", "TRUE") + testSqlApi("NOT(NULL = f27)", "NULL") + + // non comparable types + testSqlApi("f30 = ROW('abc', 'def')", "TRUE") + testSqlApi("f30 = ROW('abc', 'xyz')", "FALSE") + testSqlApi("f30 = NULL", "NULL") + testSqlApi("f30 = CAST(NULL AS ROW)", "NULL") + testSqlApi("ROW('abc', 'def') = f30", "TRUE") + testSqlApi("ROW('abc', 'xyz') = f30", "FALSE") + testSqlApi("CAST(NULL AS ROW) = f30", "NULL") + + testSqlApi("NOT(f30 = ROW('abc', 'def'))", "FALSE") + testSqlApi("NOT(f30 = ROW('abc', 'xyz'))", "TRUE") + testSqlApi("NOT(f30 = NULL)", "NULL") + testSqlApi("NOT(f30 = CAST(NULL AS ROW))", "NULL") + testSqlApi("NOT(ROW('abc', 'def') = f30)", "FALSE") + testSqlApi("NOT(ROW('abc', 'xyz') = f30)", "TRUE") + testSqlApi("NOT(CAST(NULL AS ROW) = f30)", "NULL") + + // time interval, comparable + testSqlApi("f31 = f33", "TRUE") + testSqlApi("f31 = f32", "FALSE") + testSqlApi("f31 = NULL", "NULL") + testSqlApi("f31 = f34", "NULL") + testSqlApi("f31 = CAST(NULL AS INTERVAL DAY)", "NULL") + testSqlApi("f33 = f31", "TRUE") + testSqlApi("f32 = f31", "FALSE") + testSqlApi("NULL = f31", "NULL") + testSqlApi("f34 = f31", "NULL") + testSqlApi("CAST(NULL AS INTERVAL DAY) = f31", "NULL") + + testSqlApi("NOT(f31 = f33)", "FALSE") + testSqlApi("NOT(f31 = f32)", "TRUE") + testSqlApi("NOT(f31 = NULL)", "NULL") + testSqlApi("NOT(f31 = f34)", "NULL") + testSqlApi("NOT(f31 = CAST(NULL AS INTERVAL DAY))", "NULL") + testSqlApi("NOT(f33 = f31)", "FALSE") + testSqlApi("NOT(f32 = f31)", "TRUE") + testSqlApi("NOT(NULL = f31)", "NULL") + testSqlApi("NOT(f34 = f31)", "NULL") + testSqlApi("NOT(CAST(NULL AS INTERVAL DAY) = f31)", "NULL") } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 48e972d275ab7..d46ab187b75ca 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -247,6 +247,7 @@ abstract class ExpressionTestBase(isStreaming: Boolean = true) { val converter = DataStructureConverters .getConverter(resolvedDataType) .asInstanceOf[DataStructureConverter[RowData, Row]] + converter.open(getClass.getClassLoader) converter.toInternalOrNull(testData) } try { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala index b32c5618f4521..78d91c3562d79 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala @@ -21,13 +21,18 @@ import org.apache.flink.table.api.DataTypes import org.apache.flink.table.data.DecimalDataUtils import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.utils.DateTimeTestUtil._ +import org.apache.flink.table.planner.utils.TableConfigUtils import org.apache.flink.table.types.AbstractDataType import org.apache.flink.types.Row +import java.time.{DayOfWeek, Duration} + +import scala.collection.JavaConverters.mapAsJavaMapConverter + abstract class ScalarOperatorsTestBase extends ExpressionTestBase { override def testData: Row = { - val testData = new Row(23) + val testData = new Row(35) testData.setField(0, 1: Byte) testData.setField(1, 1: Short) testData.setField(2, 1) @@ -51,6 +56,22 @@ abstract class ScalarOperatorsTestBase extends ExpressionTestBase { testData.setField(20, "who".getBytes()) testData.setField(21, localTime("12:34:56")) testData.setField(22, localDateTime("1996-11-10 12:34:56")) + testData.setField( + 23, + localDateTime("1996-11-10 12:34:56") + .atZone(TableConfigUtils.getLocalTimeZone(tableConfig)) + .toInstant) + testData.setField(24, Array("hello", "world")) + testData.setField(25, Map("a" -> 1, "b" -> 2).asJava) + testData.setField(26, Map("a" -> 1, "b" -> 2).asJava) + testData.setField(27, DayOfWeek.SUNDAY) + testData.setField(28, DayOfWeek.MONDAY) + testData.setField(29, DayOfWeek.SUNDAY) + testData.setField(30, Row.of("abc", "def")) + testData.setField(31, Duration.ofDays(2)) + testData.setField(32, Duration.ofDays(3)) + testData.setField(33, Duration.ofDays(2)) + testData.setField(34, null) testData } @@ -82,7 +103,23 @@ abstract class ScalarOperatorsTestBase extends ExpressionTestBase { DataTypes.FIELD("f19", DataTypes.VARBINARY(200).notNull()), DataTypes.FIELD("f20", DataTypes.VARBINARY(200)), DataTypes.FIELD("f21", DataTypes.TIME()), - DataTypes.FIELD("f22", DataTypes.TIMESTAMP()) + DataTypes.FIELD("f22", DataTypes.TIMESTAMP()), + DataTypes.FIELD("f23", DataTypes.TIMESTAMP_LTZ()), + DataTypes.FIELD("f24", DataTypes.ARRAY(DataTypes.STRING())), + DataTypes.FIELD("f25", DataTypes.MAP(DataTypes.STRING(), DataTypes.INT())), + DataTypes.FIELD("f26", DataTypes.MULTISET(DataTypes.STRING())), + DataTypes.FIELD("f27", DataTypes.RAW(classOf[DayOfWeek])), + DataTypes.FIELD("f28", DataTypes.RAW(classOf[DayOfWeek])), + DataTypes.FIELD("f29", DataTypes.RAW(classOf[DayOfWeek])), + DataTypes.FIELD( + "f30", + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.STRING()), + DataTypes.FIELD("f1", DataTypes.STRING()))), + DataTypes.FIELD("f31", DataTypes.INTERVAL(DataTypes.DAY)), + DataTypes.FIELD("f32", DataTypes.INTERVAL(DataTypes.DAY)), + DataTypes.FIELD("f33", DataTypes.INTERVAL(DataTypes.DAY)), + DataTypes.FIELD("f34", DataTypes.INTERVAL(DataTypes.DAY)) ) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala index fd5a5a4343e7f..0f043a307b13a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala @@ -32,6 +32,8 @@ import org.apache.flink.types.Row import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.jupiter.api.{BeforeEach, Test} +import java.time.DayOfWeek + /** Misc tests. */ class MiscITCase extends BatchTestBase { @@ -621,4 +623,94 @@ class MiscITCase extends BatchTestBase { )).isInstanceOf(classOf[ValidationException]) } + + @Test + def testEqualAndNotEqual(): Unit = { + // character string + checkQuery( + Seq((null, 2), ("b", 1)), + "SELECT f1 FROM Table1 WHERE f0 <> 'a'", + Seq(Tuple1(1)) + ) + checkQuery( + Seq(("aa", "aa"), ("aa", "bb"), ("aa", null)), + "SELECT * FROM Table1 WHERE SUBSTR(f0, 2, 1) <> SUBSTR(f1, 2, 1)", + Seq(("aa", "bb")) + ) + checkQuery( + Seq(("aa", "aa"), ("aa", "bb"), ("aa", null)), + "SELECT * FROM Table1 WHERE SUBSTR(f0, 2, 1) = SUBSTR(f1, 2, 1)", + Seq(("aa", "aa")) + ) + + // raw + checkQuery( + Seq( + (DayOfWeek.SUNDAY, DayOfWeek.SUNDAY), + (DayOfWeek.SUNDAY, DayOfWeek.MONDAY), + (DayOfWeek.SUNDAY, null)), + "SELECT * FROM Table1 WHERE f0 = f1", + Seq((DayOfWeek.SUNDAY, DayOfWeek.SUNDAY)) + ) + checkQuery( + Seq( + (DayOfWeek.SUNDAY, DayOfWeek.SUNDAY), + (DayOfWeek.SUNDAY, DayOfWeek.MONDAY), + (DayOfWeek.SUNDAY, null)), + "SELECT * FROM Table1 WHERE f0 <> f1", + Seq((DayOfWeek.SUNDAY, DayOfWeek.MONDAY)) + ) + + // multiset + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT t1.ms = t2.ms FROM " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1 LEFT JOIN " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t2 ON t1.f1 = t2.f1", + Seq(Tuple1("true")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT t1.ms = t2.ms FROM " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1 LEFT JOIN " + + "(SELECT f1, COLLECT(f0) AS ms FROM (SELECT * FROM Table1 LIMIT 2) GROUP BY f1) t2 " + + "ON t1.f1 = t2.f1", + Seq(Tuple1("false")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT t1.ms = NULL FROM (SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1", + Seq(Tuple1("null")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT NULL = t1.ms FROM (SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1", + Seq(Tuple1("null")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT NOT(t1.ms = t2.ms) FROM " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1 LEFT JOIN " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t2 ON t1.f1 = t2.f1", + Seq(Tuple1("false")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT NOT(t1.ms = t2.ms) FROM " + + "(SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1 LEFT JOIN " + + "(SELECT f1, COLLECT(f0) AS ms FROM (SELECT * FROM Table1 LIMIT 2) GROUP BY f1) t2 " + + "ON t1.f1 = t2.f1", + Seq(Tuple1("true")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT NOT(t1.ms = NULL) FROM (SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1", + Seq(Tuple1("null")) + ) + checkQuery( + Seq(("b", 1), ("a", 1), ("b", 1)), + "SELECT NOT(NULL = t1.ms) FROM (SELECT f1, COLLECT(f0) AS ms FROM Table1 GROUP BY f1) t1", + Seq(Tuple1("null")) + ) + } } From fa8ea3f9a443f555de5c5479d725e16befefc957 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 30 Oct 2023 13:45:20 +0100 Subject: [PATCH 080/104] [FLINK-18286] Implement type inference for functions on composite types --- .../functions/BuiltInFunctionDefinitions.java | 18 ++- .../types/inference/InputTypeStrategies.java | 4 + .../strategies/ArrayElementTypeStrategy.java | 49 ++++++++ .../ItemAtIndexArgumentTypeStrategy.java | 97 ++++++++++++++++ .../strategies/ItemAtTypeStrategy.java | 61 ++++++++++ .../strategies/SpecificTypeStrategies.java | 7 +- .../inference/strategies/StrategyUtils.java | 20 ++++ .../ItemAtIndexArgumentTypeStrategyTest.java | 77 +++++++++++++ .../inference/TypeStrategiesTestBase.java | 15 ++- .../ArrayElementTypeStrategyTest.java | 49 ++++++++ .../strategies/ItemAtTypeStrategyTest.java | 42 +++++++ .../PlannerExpressionConverter.scala | 12 -- .../planner/expressions/collection.scala | 105 ------------------ .../planner/expressions/ArrayTypeTest.scala | 10 +- .../validation/ArrayTypeValidationTest.scala | 6 - 15 files changed, 436 insertions(+), 136 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategy.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtIndexArgumentTypeStrategy.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategy.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ItemAtIndexArgumentTypeStrategyTest.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategyTest.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategyTest.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/collection.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index 2d49dc30725e0..4f9953f45c9a6 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -1826,14 +1826,25 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) BuiltInFunctionDefinition.newBuilder() .name("at") .kind(SCALAR) - .outputTypeStrategy(TypeStrategies.MISSING) + .inputTypeStrategy( + sequence( + or( + logical(LogicalTypeRoot.ARRAY), + logical(LogicalTypeRoot.MAP)), + InputTypeStrategies.ITEM_AT_INDEX)) + .outputTypeStrategy(SpecificTypeStrategies.ITEM_AT) .build(); public static final BuiltInFunctionDefinition CARDINALITY = BuiltInFunctionDefinition.newBuilder() .name("cardinality") .kind(SCALAR) - .outputTypeStrategy(TypeStrategies.MISSING) + .inputTypeStrategy( + sequence( + or( + logical(LogicalTypeFamily.COLLECTION), + logical(LogicalTypeRoot.MAP)))) + .outputTypeStrategy(nullableIfArgs(TypeStrategies.explicit(DataTypes.INT()))) .build(); public static final BuiltInFunctionDefinition ARRAY = @@ -1848,7 +1859,8 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) BuiltInFunctionDefinition.newBuilder() .name("element") .kind(SCALAR) - .outputTypeStrategy(TypeStrategies.MISSING) + .inputTypeStrategy(sequence(logical(LogicalTypeRoot.ARRAY))) + .outputTypeStrategy(forceNullable(SpecificTypeStrategies.ARRAY_ELEMENT)) .build(); public static final BuiltInFunctionDefinition MAP = diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java index 7ed5ba771f52f..f5477c508d4fc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java @@ -32,6 +32,7 @@ import org.apache.flink.table.types.inference.strategies.ConstraintArgumentTypeStrategy; import org.apache.flink.table.types.inference.strategies.ExplicitArgumentTypeStrategy; import org.apache.flink.table.types.inference.strategies.FamilyArgumentTypeStrategy; +import org.apache.flink.table.types.inference.strategies.ItemAtIndexArgumentTypeStrategy; import org.apache.flink.table.types.inference.strategies.LiteralArgumentTypeStrategy; import org.apache.flink.table.types.inference.strategies.OrArgumentTypeStrategy; import org.apache.flink.table.types.inference.strategies.OrInputTypeStrategy; @@ -369,6 +370,9 @@ public static InputTypeStrategy arrayFullyComparableElementType() { return new ArrayComparableElementTypeStrategy(StructuredComparison.FULL); } + /** @see ItemAtIndexArgumentTypeStrategy */ + public static final ArgumentTypeStrategy ITEM_AT_INDEX = new ItemAtIndexArgumentTypeStrategy(); + // -------------------------------------------------------------------------------------------- private InputTypeStrategies() { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategy.java new file mode 100644 index 0000000000000..b0515961e9ca9 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategy.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.TypeStrategy; +import org.apache.flink.table.types.logical.LogicalTypeFamily; + +import java.util.Optional; + +/** Returns the element of an {@link LogicalTypeFamily#COLLECTION} type. */ +@Internal +public final class ArrayElementTypeStrategy implements TypeStrategy { + @Override + public Optional inferType(CallContext callContext) { + + DataType arrayType = callContext.getArgumentDataTypes().get(0); + final Optional legacyArrayElement = + StrategyUtils.extractLegacyArrayElement(arrayType); + if (legacyArrayElement.isPresent()) { + return legacyArrayElement; + } + + if (!arrayType.getLogicalType().is(LogicalTypeFamily.COLLECTION)) { + return Optional.empty(); + } + + return Optional.of(((CollectionDataType) arrayType).getElementDataType().nullable()); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtIndexArgumentTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtIndexArgumentTypeStrategy.java new file mode 100644 index 0000000000000..82b360ccc4354 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtIndexArgumentTypeStrategy.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.KeyValueDataType; +import org.apache.flink.table.types.inference.ArgumentTypeStrategy; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.Signature; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; + +import java.util.List; +import java.util.Optional; + +/** + * An {@link ArgumentTypeStrategy} that expects: + * + *

a {@link LogicalTypeFamily#NUMERIC} if the first argument is an {@link LogicalTypeRoot#ARRAY} + * or {@link LogicalTypeRoot#MULTISET} + * + *

the type to be equal to the key type of {@link LogicalTypeRoot#MAP} if the first argument is a + * map. + */ +@Internal +public final class ItemAtIndexArgumentTypeStrategy implements ArgumentTypeStrategy { + @Override + public Optional inferArgumentType( + CallContext callContext, int argumentPos, boolean throwOnFailure) { + final List argumentDataTypes = callContext.getArgumentDataTypes(); + final LogicalType collectionType = argumentDataTypes.get(0).getLogicalType(); + final DataType indexType = argumentDataTypes.get(1); + + if (collectionType.is(LogicalTypeRoot.ARRAY)) { + if (indexType.getLogicalType().is(LogicalTypeFamily.INTEGER_NUMERIC)) { + + if (callContext.isArgumentLiteral(1)) { + Optional literalVal = callContext.getArgumentValue(1, Integer.class); + if (literalVal.isPresent() && literalVal.get() <= 0) { + return callContext.fail( + throwOnFailure, + "The provided index must be a valid SQL index starting from 1, but was '%s'", + literalVal.get()); + } + } + + return Optional.of(indexType); + } else { + return callContext.fail( + throwOnFailure, "Array can be indexed only using an INTEGER NUMERIC type."); + } + } + + if (collectionType.is(LogicalTypeRoot.MAP)) { + MapType mapType = (MapType) collectionType; + if (LogicalTypeCasts.supportsImplicitCast( + indexType.getLogicalType(), mapType.getKeyType())) { + final KeyValueDataType mapDataType = (KeyValueDataType) argumentDataTypes.get(0); + return Optional.of(mapDataType.getKeyDataType()); + } else { + return callContext.fail( + throwOnFailure, + "Expected index for a MAP to be of type: %s", + mapType.getKeyType()); + } + } + + return Optional.empty(); + } + + @Override + public Signature.Argument getExpectedArgument( + FunctionDefinition functionDefinition, int argumentPos) { + return Signature.Argument.of("[ | ]"); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategy.java new file mode 100644 index 0000000000000..dc5a3ddebf8f4 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategy.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.KeyValueDataType; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.TypeStrategy; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.Optional; + +/** + * An output type strategy for {@link BuiltInFunctionDefinitions#AT}. + * + *

Returns either the element of an {@link LogicalTypeFamily#COLLECTION} type or the value of + * {@link LogicalTypeRoot#MAP}. + */ +@Internal +public final class ItemAtTypeStrategy implements TypeStrategy { + @Override + public Optional inferType(CallContext callContext) { + + DataType arrayOrMapType = callContext.getArgumentDataTypes().get(0); + final Optional legacyArrayElement = + StrategyUtils.extractLegacyArrayElement(arrayOrMapType); + + if (legacyArrayElement.isPresent()) { + return legacyArrayElement; + } + + if (arrayOrMapType.getLogicalType().is(LogicalTypeRoot.ARRAY)) { + return Optional.of( + ((CollectionDataType) arrayOrMapType).getElementDataType().nullable()); + } else if (arrayOrMapType instanceof KeyValueDataType) { + return Optional.of(((KeyValueDataType) arrayOrMapType).getValueDataType().nullable()); + } + + return Optional.empty(); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java index 911f0d643e06b..d6fba4409a533 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java @@ -59,12 +59,9 @@ public final class SpecificTypeStrategies { public static final TypeStrategy ARRAY = new ArrayTypeStrategy(); /** Type strategy specific for array element. */ - public static final TypeStrategy ARRAY_ELEMENT = - callContext -> - Optional.of( - ((CollectionDataType) callContext.getArgumentDataTypes().get(0)) - .getElementDataType()); + public static final TypeStrategy ARRAY_ELEMENT = new ArrayElementTypeStrategy(); + public static final TypeStrategy ITEM_AT = new ItemAtTypeStrategy(); /** See {@link GetTypeStrategy}. */ public static final TypeStrategy GET = new GetTypeStrategy(); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java index fe2b3d85ca4fd..a4b302fc5b0dc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java @@ -18,18 +18,22 @@ package org.apache.flink.table.types.inference.strategies; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; +import org.apache.flink.table.types.utils.TypeConversions; import javax.annotation.Nullable; @@ -197,6 +201,22 @@ static boolean isDecimalComputation(LogicalType type1, LogicalType type2) { } } + static Optional extractLegacyArrayElement(DataType arrayType) { + final LogicalType logicalType = arrayType.getLogicalType(); + if (logicalType instanceof LegacyTypeInformationType) { + final TypeInformation typeInformation = + ((LegacyTypeInformationType) logicalType).getTypeInformation(); + if (typeInformation instanceof BasicArrayTypeInfo) { + return Optional.of( + TypeConversions.fromLegacyInfoToDataType( + ((BasicArrayTypeInfo) typeInformation) + .getComponentInfo()) + .nullable()); + } + } + return Optional.empty(); + } + private StrategyUtils() { // no instantiation } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ItemAtIndexArgumentTypeStrategyTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ItemAtIndexArgumentTypeStrategyTest.java new file mode 100644 index 0000000000000..d38ebd011f347 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ItemAtIndexArgumentTypeStrategyTest.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.inference.strategies.ItemAtIndexArgumentTypeStrategy; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; + +import java.util.stream.Stream; + +/** Tests for {@link ItemAtIndexArgumentTypeStrategy}. */ +class ItemAtIndexArgumentTypeStrategyTest extends InputTypeStrategiesTestBase { + + private static final InputTypeStrategy ITEM_AT_INPUT_STRATEGY = + BuiltInFunctionDefinitions.AT + .getTypeInference(new DataTypeFactoryMock()) + .getInputTypeStrategy(); + + @Override + protected Stream testData() { + + return Stream.of( + TestSpec.forStrategy("Validate integer index for an array", ITEM_AT_INPUT_STRATEGY) + .calledWithArgumentTypes( + DataTypes.ARRAY(DataTypes.STRING().notNull()), + DataTypes.SMALLINT().notNull()) + .expectSignature( + "f([ | ], [ | ])") + .expectArgumentTypes( + DataTypes.ARRAY(DataTypes.STRING().notNull()), + DataTypes.SMALLINT().notNull()), + TestSpec.forStrategy( + "Validate not an integer index for an array", + ITEM_AT_INPUT_STRATEGY) + .calledWithArgumentTypes( + DataTypes.ARRAY(DataTypes.STRING().notNull()), DataTypes.STRING()) + .expectErrorMessage( + "Array can be indexed only using an INTEGER NUMERIC type."), + TestSpec.forStrategy("Validate correct map key", ITEM_AT_INPUT_STRATEGY) + .calledWithArgumentTypes( + DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING().notNull()), + DataTypes.SMALLINT()) + .expectSignature( + "f([ | ], [ | ])") + .expectArgumentTypes( + DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING().notNull()), + DataTypes.BIGINT()), + TestSpec.forStrategy("Validate incorrect map key", ITEM_AT_INPUT_STRATEGY) + .calledWithArgumentTypes( + DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING().notNull()), + DataTypes.STRING()) + .expectErrorMessage("Expected index for a MAP to be of type: BIGINT"), + TestSpec.forStrategy("Validate incorrect index", ITEM_AT_INPUT_STRATEGY) + .calledWithArgumentTypes( + DataTypes.ARRAY(DataTypes.BIGINT()), DataTypes.INT().notNull()) + .calledWithLiteralAt(1, 0) + .expectErrorMessage( + "The provided index must be a valid SQL index starting from 1, but was '0'")); + } +} diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTestBase.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTestBase.java index 94a95e186de95..310c1faa7ea75 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTestBase.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTestBase.java @@ -54,8 +54,13 @@ void testTypeStrategy(TestSpec testSpec) { anyCauseMatches( ValidationException.class, testSpec.expectedErrorMessage)); } else if (testSpec.expectedDataType != null) { - assertThat(runTypeInference(testSpec).getOutputDataType()) - .isEqualTo(testSpec.expectedDataType); + if (testSpec.compareConversionClass) { + assertThat(runTypeInference(testSpec).getOutputDataType()) + .isEqualTo(testSpec.expectedDataType); + } else { + assertThat(runTypeInference(testSpec).getOutputDataType().getLogicalType()) + .isEqualTo(testSpec.expectedDataType.getLogicalType()); + } } } @@ -114,6 +119,7 @@ public static class TestSpec { private @Nullable Object literalValue; private boolean isGroupedAggregation; + private boolean compareConversionClass = false; private TestSpec(@Nullable String description, TypeStrategy strategy) { this.description = description; @@ -154,6 +160,11 @@ public TestSpec expectErrorMessage(String expectedErrorMessage) { return this; } + public TestSpec compareConversionClass() { + this.compareConversionClass = true; + return this; + } + @Override public String toString() { return description != null ? description : ""; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategyTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategyTest.java new file mode 100644 index 0000000000000..6f0f518f766a6 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ArrayElementTypeStrategyTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.inference.TypeStrategiesTestBase; + +import java.util.stream.Stream; + +/** Tests for {@link ArrayElementTypeStrategy}. */ +class ArrayElementTypeStrategyTest extends TypeStrategiesTestBase { + + @Override + protected Stream testData() { + return Stream.of( + TestSpec.forStrategy( + "Infer an element of an array type", + SpecificTypeStrategies.ARRAY_ELEMENT) + .inputTypes(DataTypes.ARRAY(DataTypes.BIGINT().notNull())) + .expectDataType(DataTypes.BIGINT().nullable()), + TestSpec.forStrategy( + "Infer an element of a multiset type", + SpecificTypeStrategies.ARRAY_ELEMENT) + .inputTypes(DataTypes.MULTISET(DataTypes.STRING().notNull())) + .expectDataType(DataTypes.STRING().nullable()), + TestSpec.forStrategy( + "Error on non collection type", + SpecificTypeStrategies.ARRAY_ELEMENT) + .inputTypes(DataTypes.BIGINT()) + .expectErrorMessage( + "Could not infer an output type for the given arguments.")); + } +} diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategyTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategyTest.java new file mode 100644 index 0000000000000..a908d98526b04 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/ItemAtTypeStrategyTest.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.inference.TypeStrategiesTestBase; + +import java.util.stream.Stream; + +/** Tests for {@link ItemAtTypeStrategy}. */ +class ItemAtTypeStrategyTest extends TypeStrategiesTestBase { + + @Override + protected Stream testData() { + return Stream.of( + TestSpec.forStrategy("Infer an item at array type", SpecificTypeStrategies.ITEM_AT) + .inputTypes(DataTypes.ARRAY(DataTypes.BIGINT().notNull()), DataTypes.INT()) + .expectDataType(DataTypes.BIGINT().nullable()), + TestSpec.forStrategy("Infer an item at map type", SpecificTypeStrategies.ITEM_AT) + .inputTypes( + DataTypes.MAP( + DataTypes.STRING().notNull(), DataTypes.BIGINT().notNull()), + DataTypes.STRING()) + .expectDataType(DataTypes.BIGINT().nullable())); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 953da5ede4f31..7996be7dd8784 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -139,18 +139,6 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp assert(args.size == 1) Collect(args.head) - case AT => - assert(args.size == 2) - ItemAt(args.head, args.last) - - case CARDINALITY => - assert(args.size == 1) - Cardinality(args.head) - - case ARRAY_ELEMENT => - assert(args.size == 1) - ArrayElement(args.head) - case ORDER_ASC => assert(args.size == 1) Asc(args.head) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/collection.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/collection.scala deleted file mode 100644 index 35bf9c2d721fd..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/collection.scala +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.table.planner.expressions - -import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, BasicTypeInfo, PrimitiveArrayTypeInfo, TypeInformation} -import org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO -import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo} -import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils.{isArray, isMap} -import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} - -case class ArrayElement(array: PlannerExpression) extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = Seq(array) - - override def toString = s"($array).element()" - - override private[flink] def resultType = array.resultType match { - case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo - case bati: BasicArrayTypeInfo[_, _] => bati.getComponentInfo - case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType - } - - override private[flink] def validateInput(): ValidationResult = { - array.resultType match { - case ati: TypeInformation[_] if isArray(ati) => ValidationSuccess - case other @ _ => ValidationFailure(s"Array expected but was '$other'.") - } - } -} - -case class Cardinality(container: PlannerExpression) extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = Seq(container) - - override def toString = s"($container).cardinality()" - - override private[flink] def resultType = BasicTypeInfo.INT_TYPE_INFO - - override private[flink] def validateInput(): ValidationResult = { - container.resultType match { - case mti: TypeInformation[_] if isMap(mti) => ValidationSuccess - case ati: TypeInformation[_] if isArray(ati) => ValidationSuccess - case other @ _ => ValidationFailure(s"Array or map expected but was '$other'.") - } - } -} - -case class ItemAt(container: PlannerExpression, key: PlannerExpression) extends PlannerExpression { - - override private[flink] def children: Seq[PlannerExpression] = Seq(container, key) - - override def toString = s"($container).at($key)" - - override private[flink] def resultType = container.resultType match { - case mti: MapTypeInfo[_, _] => mti.getValueTypeInfo - case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo - case bati: BasicArrayTypeInfo[_, _] => bati.getComponentInfo - case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType - } - - override private[flink] def validateInput(): ValidationResult = { - container.resultType match { - - case ati: TypeInformation[_] if isArray(ati) => - if (key.resultType == INT_TYPE_INFO) { - // check for common user mistake - key match { - case Literal(value: Int, INT_TYPE_INFO) if value < 1 => - ValidationFailure( - s"Array element access needs an index starting at 1 but was $value.") - case _ => ValidationSuccess - } - } else { - ValidationFailure( - s"Array element access needs an integer index but was '${key.resultType}'.") - } - - case mti: MapTypeInfo[_, _] => - if (key.resultType == mti.getKeyTypeInfo) { - ValidationSuccess - } else { - ValidationFailure( - s"Map entry access needs a valid key of type " + - s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.") - } - - case other @ _ => ValidationFailure(s"Array or map expected but was '$other'.") - } - } -} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala index d5465104c1d13..bc4b13615e38a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala @@ -17,10 +17,12 @@ */ package org.apache.flink.table.planner.expressions +import org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches import org.apache.flink.table.api._ import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime} +import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.jupiter.api.Test import java.time.{LocalDateTime => JLocalDateTime} @@ -226,9 +228,11 @@ class ArrayTypeTest extends ArrayTypeTestBase { @Test def testArrayIndexStaticCheckForTable(): Unit = { - testExpectedTableApiException( - 'f2.at(0), - "Array element access needs an index starting at 1 but was 0.") + assertThatThrownBy(() => testTableApi('f2.at(0), "1")) + .satisfies( + anyCauseMatches( + classOf[ValidationException], + "The provided index must be a valid SQL index starting from 1")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala index ce47498ebfeeb..ae71cfbb60468 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala @@ -31,12 +31,6 @@ class ArrayTypeValidationTest extends ArrayTypeTestBase { .isThrownBy(() => testSqlApi("ARRAY['string', 12]", "FAIL")) } - @Test - def testObviousInvalidIndexTableApi(): Unit = { - assertThatExceptionOfType(classOf[ValidationException]) - .isThrownBy(() => testTableApi('f2.at(0), "FAIL")) - } - @Test def testEmptyArraySql(): Unit = { assertThatExceptionOfType(classOf[ValidationException]) From 64992471c5dd52fb4269f6de51c15e27c2a45b55 Mon Sep 17 00:00:00 2001 From: 1996fanrui <1996fanrui@gmail.com> Date: Thu, 26 Oct 2023 14:38:25 +0800 Subject: [PATCH 081/104] [FLINK-33354][runtime][refactor] Refactor ShuffleDescriptorsCache into a generic GroupCache --- .../InputGateDeploymentDescriptor.java | 9 +- .../deployment/TaskDeploymentDescriptor.java | 5 +- .../DefaultShuffleDescriptorsCache.java | 153 ---------------- .../taskexecutor/ShuffleDescriptorsCache.java | 54 ------ .../runtime/taskexecutor/TaskExecutor.java | 8 +- .../taskexecutor/TaskManagerServices.java | 17 +- .../flink/runtime/util/DefaultGroupCache.java | 163 ++++++++++++++++++ .../apache/flink/runtime/util/GroupCache.java | 50 ++++++ .../DefaultExecutionGraphDeploymentTest.java | 5 +- ...oymentWithSmallBlobCacheSizeLimitTest.java | 5 +- .../TaskManagerServicesBuilder.java | 3 +- .../DefaultGroupCacheTest.java} | 45 ++--- .../NoOpGroupCache.java} | 19 +- 13 files changed, 274 insertions(+), 262 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCache.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/ShuffleDescriptorsCache.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/DefaultGroupCache.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/GroupCache.java rename flink-runtime/src/test/java/org/apache/flink/runtime/{taskexecutor/DefaultShuffleDescriptorsCacheTest.java => util/DefaultGroupCacheTest.java} (76%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{taskexecutor/NoOpShuffleDescriptorsCache.java => util/NoOpGroupCache.java} (56%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java index 50ed98ed1dcf7..333a91e0a7320 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; -import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache; +import org.apache.flink.runtime.util.GroupCache; import org.apache.flink.util.CompressedSerializedValue; import org.apache.flink.util.Preconditions; @@ -167,7 +167,7 @@ public ShuffleDescriptor[] getShuffleDescriptors() { public void tryLoadAndDeserializeShuffleDescriptors( @Nullable PermanentBlobService blobService, JobID jobId, - ShuffleDescriptorsCache shuffleDescriptorsCache) + GroupCache shuffleDescriptorsCache) throws IOException { if (inputChannels != null) { return; @@ -190,13 +190,14 @@ private void tryLoadAndDeserializeShuffleDescriptorGroup( @Nullable PermanentBlobService blobService, JobID jobId, MaybeOffloaded serializedShuffleDescriptors, - ShuffleDescriptorsCache shuffleDescriptorsCache) + GroupCache shuffleDescriptorsCache) throws IOException, ClassNotFoundException { if (serializedShuffleDescriptors instanceof Offloaded) { PermanentBlobKey blobKey = ((Offloaded) serializedShuffleDescriptors) .serializedValueKey; - ShuffleDescriptorGroup shuffleDescriptorGroup = shuffleDescriptorsCache.get(blobKey); + ShuffleDescriptorGroup shuffleDescriptorGroup = + shuffleDescriptorsCache.get(jobId, blobKey); if (shuffleDescriptorGroup == null) { Preconditions.checkNotNull(blobService); // NOTE: Do not delete the ShuffleDescriptor BLOBs since it may be needed again diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index bd3b770142c01..5985801a5f2eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -23,10 +23,11 @@ import org.apache.flink.runtime.blob.PermanentBlobService; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache; +import org.apache.flink.runtime.util.GroupCache; import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -242,7 +243,7 @@ public AllocationID getAllocationId() { */ public void loadBigData( @Nullable PermanentBlobService blobService, - ShuffleDescriptorsCache shuffleDescriptorsCache) + GroupCache shuffleDescriptorsCache) throws IOException, ClassNotFoundException { // re-integrate offloaded job info from blob diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCache.java deleted file mode 100644 index 99a97f2370b27..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCache.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.taskexecutor; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; - -import org.apache.flink.shaded.guava31.com.google.common.base.Ticker; -import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; -import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; -import org.apache.flink.shaded.guava31.com.google.common.cache.RemovalNotification; - -import java.time.Duration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** Default implement of {@link ShuffleDescriptorsCache}. Entries will be expired after timeout. */ -public class DefaultShuffleDescriptorsCache implements ShuffleDescriptorsCache { - private final Cache shuffleDescriptorsCache; - private final Map> cachedBlobKeysPerJob; - - private DefaultShuffleDescriptorsCache( - Duration expireTimeout, int cacheSizeLimit, Ticker ticker) { - this.cachedBlobKeysPerJob = new HashMap<>(); - this.shuffleDescriptorsCache = - CacheBuilder.newBuilder() - .concurrencyLevel(1) - .maximumSize(cacheSizeLimit) - .expireAfterAccess(expireTimeout) - .ticker(ticker) - .removalListener(this::onCacheRemoval) - .build(); - } - - @Override - public void clear() { - cachedBlobKeysPerJob.clear(); - shuffleDescriptorsCache.cleanUp(); - } - - @Override - public ShuffleDescriptorGroup get(PermanentBlobKey blobKey) { - ShuffleDescriptorCacheEntry entry = shuffleDescriptorsCache.getIfPresent(blobKey); - return entry == null ? null : entry.getShuffleDescriptorGroup(); - } - - @Override - public void put( - JobID jobId, PermanentBlobKey blobKey, ShuffleDescriptorGroup shuffleDescriptorGroup) { - shuffleDescriptorsCache.put( - blobKey, new ShuffleDescriptorCacheEntry(shuffleDescriptorGroup, jobId)); - cachedBlobKeysPerJob.computeIfAbsent(jobId, ignore -> new HashSet<>()).add(blobKey); - } - - @Override - public void clearCacheForJob(JobID jobId) { - Set removed = cachedBlobKeysPerJob.remove(jobId); - if (removed != null) { - shuffleDescriptorsCache.invalidateAll(removed); - } - } - - /** - * Removal listener that remove the index of serializedShuffleDescriptorsPerJob . - * - * @param removalNotification of removed element. - */ - private void onCacheRemoval( - RemovalNotification - removalNotification) { - PermanentBlobKey blobKey = removalNotification.getKey(); - ShuffleDescriptorCacheEntry entry = removalNotification.getValue(); - if (blobKey != null && entry != null) { - cachedBlobKeysPerJob.computeIfPresent( - entry.getJobId(), - (jobID, permanentBlobKeys) -> { - permanentBlobKeys.remove(blobKey); - if (permanentBlobKeys.isEmpty()) { - return null; - } else { - return permanentBlobKeys; - } - }); - } - } - - private static class ShuffleDescriptorCacheEntry { - private final ShuffleDescriptorGroup shuffleDescriptorGroup; - private final JobID jobId; - - public ShuffleDescriptorCacheEntry( - ShuffleDescriptorGroup shuffleDescriptorGroup, JobID jobId) { - this.shuffleDescriptorGroup = checkNotNull(shuffleDescriptorGroup); - this.jobId = checkNotNull(jobId); - } - - public ShuffleDescriptorGroup getShuffleDescriptorGroup() { - return shuffleDescriptorGroup; - } - - public JobID getJobId() { - return jobId; - } - } - - /** The Factory of {@link DefaultShuffleDescriptorsCache}. */ - public static class Factory { - private static final Duration DEFAULT_CACHE_EXPIRE_TIMEOUT = Duration.ofSeconds(300); - private static final int DEFAULT_CACHE_SIZE_LIMIT = 100; - private static final Ticker DEFAULT_TICKER = Ticker.systemTicker(); - - private final Duration cacheExpireTimeout; - private final int cacheSizeLimit; - private final Ticker ticker; - - public Factory() { - this(DEFAULT_CACHE_EXPIRE_TIMEOUT, DEFAULT_CACHE_SIZE_LIMIT, DEFAULT_TICKER); - } - - @VisibleForTesting - public Factory(Duration cacheExpireTimeout, int cacheSizeLimit, Ticker ticker) { - this.cacheExpireTimeout = cacheExpireTimeout; - this.cacheSizeLimit = cacheSizeLimit; - this.ticker = ticker; - } - - public DefaultShuffleDescriptorsCache create() { - return new DefaultShuffleDescriptorsCache(cacheExpireTimeout, cacheSizeLimit, ticker); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/ShuffleDescriptorsCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/ShuffleDescriptorsCache.java deleted file mode 100644 index a86e6a67722da..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/ShuffleDescriptorsCache.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.taskexecutor; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; - -/** Cache of shuffle descriptors in TaskExecutor. */ -public interface ShuffleDescriptorsCache { - - /** clear all cache. */ - void clear(); - - /** - * Get shuffle descriptor group in cache. - * - * @param blobKey identify the shuffle descriptor group - * @return shuffle descriptor group in cache if exists, otherwise null - */ - ShuffleDescriptorGroup get(PermanentBlobKey blobKey); - - /** - * Put shuffle descriptor group to cache. - * - * @param jobId of job - * @param blobKey identify the shuffle descriptor group - * @param shuffleDescriptorGroup shuffle descriptor group to cache - */ - void put(JobID jobId, PermanentBlobKey blobKey, ShuffleDescriptorGroup shuffleDescriptorGroup); - - /** - * Clear all cache for the Job. - * - * @param jobId of job - */ - void clearCacheForJob(JobID jobId); -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 680514f09a6f1..7146c2befa952 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -25,6 +25,7 @@ import org.apache.flink.management.jmx.JMXService; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.JobPermanentBlobService; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.blob.TaskExecutorBlobService; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; @@ -39,6 +40,7 @@ import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; @@ -131,6 +133,7 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; +import org.apache.flink.runtime.util.GroupCache; import org.apache.flink.runtime.webmonitor.threadinfo.ThreadInfoSamplesRequest; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.CollectionUtil; @@ -297,7 +300,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final ThreadInfoSampleService threadInfoSampleService; - private final ShuffleDescriptorsCache shuffleDescriptorsCache; + private final GroupCache + shuffleDescriptorsCache; public TaskExecutor( RpcService rpcService, @@ -1903,7 +1907,7 @@ private void releaseJobResources(JobID jobId, Exception cause) { changelogStoragesManager.releaseResourcesForJob(jobId); currentSlotOfferPerJob.remove(jobId); channelStateExecutorFactoryManager.releaseResourcesForJob(jobId); - shuffleDescriptorsCache.clearCacheForJob(jobId); + shuffleDescriptorsCache.clearCacheForGroup(jobId); fileMergingManager.releaseMergingSnapshotManagerForJob(jobId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index f4b206421aa34..ae78575410e4c 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -18,11 +18,14 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.blob.PermanentBlobService; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; import org.apache.flink.runtime.entrypoint.WorkingDirectory; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; @@ -48,6 +51,8 @@ import org.apache.flink.runtime.taskexecutor.slot.TimerService; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; +import org.apache.flink.runtime.util.DefaultGroupCache; +import org.apache.flink.runtime.util.GroupCache; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -90,7 +95,8 @@ public class TaskManagerServices { private final LibraryCacheManager libraryCacheManager; private final SlotAllocationSnapshotPersistenceService slotAllocationSnapshotPersistenceService; private final SharedResources sharedResources; - private final ShuffleDescriptorsCache shuffleDescriptorsCache; + private final GroupCache + shuffleDescriptorsCache; TaskManagerServices( UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, @@ -111,7 +117,7 @@ public class TaskManagerServices { LibraryCacheManager libraryCacheManager, SlotAllocationSnapshotPersistenceService slotAllocationSnapshotPersistenceService, SharedResources sharedResources, - ShuffleDescriptorsCache shuffleDescriptorsCache) { + GroupCache shuffleDescriptorsCache) { this.unresolvedTaskManagerLocation = Preconditions.checkNotNull(unresolvedTaskManagerLocation); @@ -208,7 +214,7 @@ public SharedResources getSharedResources() { return sharedResources; } - public ShuffleDescriptorsCache getShuffleDescriptorCache() { + public GroupCache getShuffleDescriptorCache() { return shuffleDescriptorsCache; } @@ -409,8 +415,9 @@ public static TaskManagerServices fromConfiguration( NoOpSlotAllocationSnapshotPersistenceService.INSTANCE; } - final ShuffleDescriptorsCache shuffleDescriptorsCache = - new DefaultShuffleDescriptorsCache.Factory().create(); + final GroupCache shuffleDescriptorsCache = + new DefaultGroupCache.Factory() + .create(); return new TaskManagerServices( unresolvedTaskManagerLocation, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/DefaultGroupCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/DefaultGroupCache.java new file mode 100644 index 0000000000000..e424e0521b69e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/DefaultGroupCache.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.util; + +import org.apache.flink.annotation.VisibleForTesting; + +import org.apache.flink.shaded.guava31.com.google.common.base.Ticker; +import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; +import org.apache.flink.shaded.guava31.com.google.common.cache.RemovalNotification; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.time.Duration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** Default implement of {@link GroupCache}. Entries will be expired after timeout. */ +@NotThreadSafe +public class DefaultGroupCache implements GroupCache { + private final Cache, V> cache; + private final Map>> cachedBlobKeysPerJob; + + private DefaultGroupCache(Duration expireTimeout, int cacheSizeLimit, Ticker ticker) { + this.cachedBlobKeysPerJob = new HashMap<>(); + this.cache = + CacheBuilder.newBuilder() + .concurrencyLevel(1) + .maximumSize(cacheSizeLimit) + .expireAfterAccess(expireTimeout) + .ticker(ticker) + .removalListener(this::onCacheRemoval) + .build(); + } + + @Override + public void clear() { + cachedBlobKeysPerJob.clear(); + cache.cleanUp(); + } + + @Override + public V get(G group, K key) { + return cache.getIfPresent(new CacheKey<>(group, key)); + } + + @Override + public void put(G group, K key, V value) { + CacheKey cacheKey = new CacheKey<>(group, key); + cache.put(cacheKey, value); + cachedBlobKeysPerJob.computeIfAbsent(group, ignore -> new HashSet<>()).add(cacheKey); + } + + @Override + public void clearCacheForGroup(G group) { + Set> removed = cachedBlobKeysPerJob.remove(group); + if (removed != null) { + cache.invalidateAll(removed); + } + } + + /** + * Removal listener that remove the cache key of this group . + * + * @param removalNotification of removed element. + */ + private void onCacheRemoval(RemovalNotification, V> removalNotification) { + CacheKey cacheKey = removalNotification.getKey(); + V value = removalNotification.getValue(); + if (cacheKey != null && value != null) { + cachedBlobKeysPerJob.computeIfPresent( + cacheKey.getGroup(), + (group, keys) -> { + keys.remove(cacheKey); + if (keys.isEmpty()) { + return null; + } else { + return keys; + } + }); + } + } + + private static class CacheKey { + private final G group; + private final K key; + + public CacheKey(G group, K key) { + this.group = group; + this.key = key; + } + + public G getGroup() { + return group; + } + + public K getKey() { + return key; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CacheKey cacheKey = (CacheKey) o; + return Objects.equals(group, cacheKey.group) && Objects.equals(key, cacheKey.key); + } + + @Override + public int hashCode() { + return Objects.hash(group, key); + } + } + + /** The Factory of {@link DefaultGroupCache}. */ + public static class Factory { + private static final Duration DEFAULT_CACHE_EXPIRE_TIMEOUT = Duration.ofSeconds(300); + private static final int DEFAULT_CACHE_SIZE_LIMIT = 100; + private static final Ticker DEFAULT_TICKER = Ticker.systemTicker(); + + private final Duration cacheExpireTimeout; + private final int cacheSizeLimit; + private final Ticker ticker; + + public Factory() { + this(DEFAULT_CACHE_EXPIRE_TIMEOUT, DEFAULT_CACHE_SIZE_LIMIT, DEFAULT_TICKER); + } + + @VisibleForTesting + public Factory(Duration cacheExpireTimeout, int cacheSizeLimit, Ticker ticker) { + this.cacheExpireTimeout = cacheExpireTimeout; + this.cacheSizeLimit = cacheSizeLimit; + this.ticker = ticker; + } + + public DefaultGroupCache create() { + return new DefaultGroupCache<>(cacheExpireTimeout, cacheSizeLimit, ticker); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/GroupCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/GroupCache.java new file mode 100644 index 0000000000000..25757837878be --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/GroupCache.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.util; + +import javax.annotation.Nullable; + +/** + * This {@link GroupCache} can cache group, key and value. The group and key are cache key, each key + * belongs to a certain group. All corresponding keys and values will be cleared if a group is + * cleared. + * + * @param The group. + * @param The key. + * @param The value. + */ +public interface GroupCache { + + /** clear all cache. */ + void clear(); + + /** + * Get value in cache. + * + * @return value in cache if exists, otherwise null + */ + @Nullable + V get(G group, K key); + + /** Put group, key and value to cache. */ + void put(G group, K key, V value); + + /** Clear all caches of the corresponding group. */ + void clearCacheForGroup(G group); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java index fc2df07230376..54d8fc3096402 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java @@ -58,7 +58,6 @@ import org.apache.flink.runtime.scheduler.TestingPhysicalSlotProvider; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; -import org.apache.flink.runtime.taskexecutor.NoOpShuffleDescriptorsCache; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; @@ -66,6 +65,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.runtime.util.NoOpGroupCache; import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.function.FunctionUtils; @@ -179,8 +179,7 @@ void testBuildDeploymentDescriptor() throws Exception { taskManagerGateway.setSubmitConsumer( FunctionUtils.uncheckedConsumer( taskDeploymentDescriptor -> { - taskDeploymentDescriptor.loadBigData( - blobCache, NoOpShuffleDescriptorsCache.INSTANCE); + taskDeploymentDescriptor.loadBigData(blobCache, new NoOpGroupCache<>()); tdd.complete(taskDeploymentDescriptor); })); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java index 6fdc9de9adeb5..1848faa3959bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java @@ -42,8 +42,8 @@ import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; import org.apache.flink.runtime.operators.BatchTask; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; -import org.apache.flink.runtime.taskexecutor.NoOpShuffleDescriptorsCache; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.runtime.util.NoOpGroupCache; import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.function.FunctionUtils; @@ -120,8 +120,7 @@ void testDeployMultipleTasksWithSmallBlobCacheSizeLimit() throws Exception { taskManagerGateway.setSubmitConsumer( FunctionUtils.uncheckedConsumer( taskDeploymentDescriptor -> { - taskDeploymentDescriptor.loadBigData( - blobCache, NoOpShuffleDescriptorsCache.INSTANCE); + taskDeploymentDescriptor.loadBigData(blobCache, new NoOpGroupCache<>()); tdds.offer(taskDeploymentDescriptor); })); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 2bd18f543a1d5..193e56e20c3a9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; +import org.apache.flink.runtime.util.NoOpGroupCache; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; @@ -200,6 +201,6 @@ public TaskManagerServices build() { libraryCacheManager, slotAllocationSnapshotPersistenceService, sharedResources, - NoOpShuffleDescriptorsCache.INSTANCE); + new NoOpGroupCache<>()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DefaultGroupCacheTest.java similarity index 76% rename from flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCacheTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/DefaultGroupCacheTest.java index 84c1acaa4e118..b5a3b717094fc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DefaultGroupCacheTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.taskexecutor; +package org.apache.flink.runtime.util; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.blob.PermanentBlobKey; @@ -35,14 +35,14 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for {@link DefaultShuffleDescriptorsCache}. */ -class DefaultShuffleDescriptorsCacheTest { +/** Tests for {@link DefaultGroupCache}. */ +class DefaultGroupCacheTest { private final Duration expireTimeout = Duration.ofSeconds(10); @Test void testGetEntry() { - DefaultShuffleDescriptorsCache cache = - new DefaultShuffleDescriptorsCache.Factory( + DefaultGroupCache cache = + new DefaultGroupCache.Factory( expireTimeout, Integer.MAX_VALUE, Ticker.systemTicker()) .create(); @@ -56,16 +56,16 @@ void testGetEntry() { PermanentBlobKey blobKey = new PermanentBlobKey(); - assertThat(cache.get(blobKey)).isNull(); + assertThat(cache.get(jobId, blobKey)).isNull(); cache.put(jobId, blobKey, shuffleDescriptorGroup); - assertThat(cache.get(blobKey)).isEqualTo(shuffleDescriptorGroup); + assertThat(cache.get(jobId, blobKey)).isEqualTo(shuffleDescriptorGroup); } @Test void testClearCacheForJob() { - DefaultShuffleDescriptorsCache cache = - new DefaultShuffleDescriptorsCache.Factory( + DefaultGroupCache cache = + new DefaultGroupCache.Factory( expireTimeout, Integer.MAX_VALUE, Ticker.systemTicker()) .create(); @@ -78,19 +78,20 @@ void testClearCacheForJob() { }); PermanentBlobKey blobKey = new PermanentBlobKey(); - assertThat(cache.get(blobKey)).isNull(); + assertThat(cache.get(jobId, blobKey)).isNull(); cache.put(jobId, blobKey, shuffleDescriptorGroup); - assertThat(cache.get(blobKey)).isEqualTo(shuffleDescriptorGroup); + assertThat(cache.get(jobId, blobKey)).isEqualTo(shuffleDescriptorGroup); - cache.clearCacheForJob(jobId); - assertThat(cache.get(blobKey)).isNull(); + cache.clearCacheForGroup(jobId); + assertThat(cache.get(jobId, blobKey)).isNull(); } @Test void testPutWhenOverLimit() { - DefaultShuffleDescriptorsCache cache = - new DefaultShuffleDescriptorsCache.Factory(expireTimeout, 1, Ticker.systemTicker()) + DefaultGroupCache cache = + new DefaultGroupCache.Factory( + expireTimeout, 1, Ticker.systemTicker()) .create(); JobID jobId = new JobID(); @@ -104,7 +105,7 @@ void testPutWhenOverLimit() { PermanentBlobKey blobKey = new PermanentBlobKey(); cache.put(jobId, blobKey, shuffleDescriptorGroup); - assertThat(cache.get(blobKey)).isEqualTo(shuffleDescriptorGroup); + assertThat(cache.get(jobId, blobKey)).isEqualTo(shuffleDescriptorGroup); ShuffleDescriptorGroup otherShuffleDescriptorGroup = new ShuffleDescriptorGroup( @@ -115,15 +116,15 @@ void testPutWhenOverLimit() { PermanentBlobKey otherBlobKey = new PermanentBlobKey(); cache.put(jobId, otherBlobKey, otherShuffleDescriptorGroup); - assertThat(cache.get(blobKey)).isNull(); - assertThat(cache.get(otherBlobKey)).isEqualTo(otherShuffleDescriptorGroup); + assertThat(cache.get(jobId, blobKey)).isNull(); + assertThat(cache.get(jobId, otherBlobKey)).isEqualTo(otherShuffleDescriptorGroup); } @Test void testEntryExpired() { TestingTicker ticker = new TestingTicker(); - DefaultShuffleDescriptorsCache cache = - new DefaultShuffleDescriptorsCache.Factory( + DefaultGroupCache cache = + new DefaultGroupCache.Factory( Duration.ofSeconds(1), Integer.MAX_VALUE, ticker) .create(); @@ -138,10 +139,10 @@ void testEntryExpired() { PermanentBlobKey blobKey = new PermanentBlobKey(); cache.put(jobId, blobKey, shuffleDescriptorGroup); - assertThat(cache.get(blobKey)).isEqualTo(shuffleDescriptorGroup); + assertThat(cache.get(jobId, blobKey)).isEqualTo(shuffleDescriptorGroup); ticker.advance(Duration.ofSeconds(2)); - assertThat(cache.get(blobKey)).isNull(); + assertThat(cache.get(jobId, blobKey)).isNull(); } private static class TestingTicker extends Ticker { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NoOpShuffleDescriptorsCache.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NoOpGroupCache.java similarity index 56% rename from flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NoOpShuffleDescriptorsCache.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/NoOpGroupCache.java index 5c7ad4236e3ea..93a0012bdd3d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NoOpShuffleDescriptorsCache.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NoOpGroupCache.java @@ -16,29 +16,22 @@ * limitations under the License. */ -package org.apache.flink.runtime.taskexecutor; +package org.apache.flink.runtime.util; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup; - -/** Non op implement of {@link ShuffleDescriptorsCache}. */ -public class NoOpShuffleDescriptorsCache implements ShuffleDescriptorsCache { - - public static final NoOpShuffleDescriptorsCache INSTANCE = new NoOpShuffleDescriptorsCache(); +/** Non op implement of {@link GroupCache}. */ +public class NoOpGroupCache implements GroupCache { @Override public void clear() {} @Override - public ShuffleDescriptorGroup get(PermanentBlobKey blobKey) { + public V get(G group, K key) { return null; } @Override - public void put( - JobID jobId, PermanentBlobKey blobKey, ShuffleDescriptorGroup shuffleDescriptorGroup) {} + public void put(G group, K key, V value) {} @Override - public void clearCacheForJob(JobID jobId) {} + public void clearCacheForGroup(G group) {} } From d237ed64476b48c78b494b0fbd6d5888785d0e55 Mon Sep 17 00:00:00 2001 From: 1996fanrui <1996fanrui@gmail.com> Date: Thu, 26 Oct 2023 15:08:38 +0800 Subject: [PATCH 082/104] [FLINK-33354][runtime][refactor] serializedJobInformation and taskInfo are never null --- .../runtime/deployment/TaskDeploymentDescriptor.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 5985801a5f2eb..97596e1ac9fd3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -98,10 +98,10 @@ public Offloaded(PermanentBlobKey serializedValueKey) { } } - /** Serialized job information or null if offloaded. */ + /** Serialized job information if non-offloaded or PermanentBlobKey if offloaded. */ private MaybeOffloaded serializedJobInformation; - /** Serialized task information or null if offloaded. */ + /** Serialized task information if non-offloaded or PermanentBlobKey if offloaded. */ private MaybeOffloaded serializedTaskInformation; /** @@ -283,10 +283,6 @@ public void loadBigData( inputGate.tryLoadAndDeserializeShuffleDescriptors( blobService, jobId, shuffleDescriptorsCache); } - - // make sure that the serialized job and task information fields are filled - Preconditions.checkNotNull(serializedJobInformation); - Preconditions.checkNotNull(serializedTaskInformation); } @Override From b1786ba952bb81e8180cac9c5c95c2fc46e44fb7 Mon Sep 17 00:00:00 2001 From: 1996fanrui <1996fanrui@gmail.com> Date: Thu, 26 Oct 2023 15:35:15 +0800 Subject: [PATCH 083/104] [FLINK-33354][runtime] Cache TaskInformation and JobInformation to avoid deserializing duplicate big objects --- .../deployment/TaskDeploymentDescriptor.java | 95 ++++++++---- .../executiongraph/JobInformation.java | 59 ++++++-- .../executiongraph/TaskInformation.java | 40 +++++ .../runtime/taskexecutor/TaskExecutor.java | 21 ++- .../taskexecutor/TaskManagerServices.java | 23 +++ .../TaskDeploymentDescriptorTest.java | 139 ++++++++++++++---- .../runtime/dispatcher/JobMasterTester.java | 4 +- .../DefaultExecutionGraphDeploymentTest.java | 12 +- ...oymentWithSmallBlobCacheSizeLimitTest.java | 6 +- .../TaskManagerServicesBuilder.java | 2 + 10 files changed, 315 insertions(+), 86 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 97596e1ac9fd3..e92162fb14eac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.util.GroupCache; import org.apache.flink.util.FileUtils; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -99,10 +100,22 @@ public Offloaded(PermanentBlobKey serializedValueKey) { } /** Serialized job information if non-offloaded or PermanentBlobKey if offloaded. */ - private MaybeOffloaded serializedJobInformation; + private final MaybeOffloaded serializedJobInformation; /** Serialized task information if non-offloaded or PermanentBlobKey if offloaded. */ - private MaybeOffloaded serializedTaskInformation; + private final MaybeOffloaded serializedTaskInformation; + + /** + * The job information, it isn't null when serializedJobInformation is offloaded and after + * {@link #loadBigData}. + */ + private transient JobInformation jobInformation; + + /** + * The task information, it isn't null when serializedTaskInformation is offloaded and after + * {@link #loadBigData}. + */ + private transient TaskInformation taskInformation; /** * The ID referencing the job this task belongs to. @@ -152,39 +165,43 @@ public TaskDeploymentDescriptor( } /** - * Return the sub task's serialized job information. + * Return the sub task's job information. * - * @return serialized job information (may throw {@link IllegalStateException} if {@link - * #loadBigData} is not called beforehand). + * @return job information (may throw {@link IllegalStateException} if {@link #loadBigData} is + * not called beforehand). * @throws IllegalStateException If job information is offloaded to BLOB store. */ - public SerializedValue getSerializedJobInformation() { + public JobInformation getJobInformation() throws IOException, ClassNotFoundException { + if (jobInformation != null) { + return jobInformation; + } if (serializedJobInformation instanceof NonOffloaded) { NonOffloaded jobInformation = (NonOffloaded) serializedJobInformation; - return jobInformation.serializedValue; - } else { - throw new IllegalStateException( - "Trying to work with offloaded serialized job information."); + return jobInformation.serializedValue.deserializeValue(getClass().getClassLoader()); } + throw new IllegalStateException( + "Trying to work with offloaded serialized job information."); } /** - * Return the sub task's serialized task information. + * Return the sub task's task information. * - * @return serialized task information (may throw {@link IllegalStateException} if {@link - * #loadBigData} is not called beforehand)). + * @return task information (may throw {@link IllegalStateException} if {@link #loadBigData} is + * not called beforehand)). * @throws IllegalStateException If job information is offloaded to BLOB store. */ - public SerializedValue getSerializedTaskInformation() { + public TaskInformation getTaskInformation() throws IOException, ClassNotFoundException { + if (taskInformation != null) { + return taskInformation; + } if (serializedTaskInformation instanceof NonOffloaded) { NonOffloaded taskInformation = (NonOffloaded) serializedTaskInformation; - return taskInformation.serializedValue; - } else { - throw new IllegalStateException( - "Trying to work with offloaded serialized job information."); + return taskInformation.serializedValue.deserializeValue(getClass().getClassLoader()); } + throw new IllegalStateException( + "Trying to work with offloaded serialized task information."); } /** @@ -243,6 +260,8 @@ public AllocationID getAllocationId() { */ public void loadBigData( @Nullable PermanentBlobService blobService, + GroupCache jobInformationCache, + GroupCache taskInformationCache, GroupCache shuffleDescriptorsCache) throws IOException, ClassNotFoundException { @@ -254,13 +273,19 @@ public void loadBigData( Preconditions.checkNotNull(blobService); - final File dataFile = blobService.getFile(jobId, jobInfoKey); - // NOTE: Do not delete the job info BLOB since it may be needed again during recovery. - // (it is deleted automatically on the BLOB server and cache when the job - // enters a terminal state) - SerializedValue serializedValue = - SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath())); - serializedJobInformation = new NonOffloaded<>(serializedValue); + JobInformation jobInformation = jobInformationCache.get(jobId, jobInfoKey); + if (jobInformation == null) { + final File dataFile = blobService.getFile(jobId, jobInfoKey); + // NOTE: Do not delete the job info BLOB since it may be needed again during + // recovery. (it is deleted automatically on the BLOB server and cache when the job + // enters a terminal state) + jobInformation = + InstantiationUtil.deserializeObject( + FileUtils.readAllBytes(dataFile.toPath()), + getClass().getClassLoader()); + jobInformationCache.put(jobId, jobInfoKey, jobInformation); + } + this.jobInformation = jobInformation.deepCopy(); } // re-integrate offloaded task info from blob @@ -270,13 +295,19 @@ public void loadBigData( Preconditions.checkNotNull(blobService); - final File dataFile = blobService.getFile(jobId, taskInfoKey); - // NOTE: Do not delete the task info BLOB since it may be needed again during recovery. - // (it is deleted automatically on the BLOB server and cache when the job - // enters a terminal state) - SerializedValue serializedValue = - SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath())); - serializedTaskInformation = new NonOffloaded<>(serializedValue); + TaskInformation taskInformation = taskInformationCache.get(jobId, taskInfoKey); + if (taskInformation == null) { + final File dataFile = blobService.getFile(jobId, taskInfoKey); + // NOTE: Do not delete the task info BLOB since it may be needed again during + // recovery. (it is deleted automatically on the BLOB server and cache when the job + // enters a terminal state) + taskInformation = + InstantiationUtil.deserializeObject( + FileUtils.readAllBytes(dataFile.toPath()), + getClass().getClassLoader()); + taskInformationCache.put(jobId, taskInfoKey, taskInformation); + } + this.taskInformation = taskInformation.deepCopy(); } for (InputGateDeploymentDescriptor inputGate : inputGates) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java index f30bf6f32d214..5792caa271153 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java @@ -21,13 +21,18 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableCollection; +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList; + import java.io.Serializable; import java.net.URL; import java.util.Collection; +import java.util.Objects; /** Container class for job information which is stored in the {@link ExecutionGraph}. */ public class JobInformation implements Serializable { @@ -44,13 +49,13 @@ public class JobInformation implements Serializable { private final SerializedValue serializedExecutionConfig; /** Configuration of the job. */ - private final Configuration jobConfiguration; + private final UnmodifiableConfiguration jobConfiguration; /** Blob keys for the required jar files. */ - private final Collection requiredJarFileBlobKeys; + private final ImmutableCollection requiredJarFileBlobKeys; /** URLs specifying the classpath to add to the class loader. */ - private final Collection requiredClasspathURLs; + private final ImmutableCollection requiredClasspathURLs; public JobInformation( JobID jobId, @@ -62,9 +67,12 @@ public JobInformation( this.jobId = Preconditions.checkNotNull(jobId); this.jobName = Preconditions.checkNotNull(jobName); this.serializedExecutionConfig = Preconditions.checkNotNull(serializedExecutionConfig); - this.jobConfiguration = Preconditions.checkNotNull(jobConfiguration); - this.requiredJarFileBlobKeys = Preconditions.checkNotNull(requiredJarFileBlobKeys); - this.requiredClasspathURLs = Preconditions.checkNotNull(requiredClasspathURLs); + this.jobConfiguration = + new UnmodifiableConfiguration(Preconditions.checkNotNull(jobConfiguration)); + this.requiredJarFileBlobKeys = + ImmutableList.copyOf(Preconditions.checkNotNull(requiredJarFileBlobKeys)); + this.requiredClasspathURLs = + ImmutableList.copyOf(Preconditions.checkNotNull(requiredClasspathURLs)); } public JobID getJobId() { @@ -79,18 +87,51 @@ public SerializedValue getSerializedExecutionConfig() { return serializedExecutionConfig; } - public Configuration getJobConfiguration() { + public UnmodifiableConfiguration getJobConfiguration() { return jobConfiguration; } - public Collection getRequiredJarFileBlobKeys() { + public ImmutableCollection getRequiredJarFileBlobKeys() { return requiredJarFileBlobKeys; } - public Collection getRequiredClasspathURLs() { + public ImmutableCollection getRequiredClasspathURLs() { return requiredClasspathURLs; } + // All fields are immutable, so return this directly. + public JobInformation deepCopy() { + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JobInformation that = (JobInformation) o; + return Objects.equals(jobId, that.jobId) + && Objects.equals(jobName, that.jobName) + && Objects.equals(serializedExecutionConfig, that.serializedExecutionConfig) + && Objects.equals(jobConfiguration, that.jobConfiguration) + && Objects.equals(requiredJarFileBlobKeys, that.requiredJarFileBlobKeys) + && Objects.equals(requiredClasspathURLs, that.requiredClasspathURLs); + } + + @Override + public int hashCode() { + return Objects.hash( + jobId, + jobName, + serializedExecutionConfig, + jobConfiguration, + requiredJarFileBlobKeys, + requiredClasspathURLs); + } + // ------------------------------------------------------------------------ @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java index 6a3b1a26e7f0b..e1b59d4bfb12f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java @@ -23,6 +23,7 @@ import org.apache.flink.util.Preconditions; import java.io.Serializable; +import java.util.Objects; /** * Container class for operator/task specific information which are stored at the {@link @@ -88,4 +89,43 @@ public String getInvokableClassName() { public Configuration getTaskConfiguration() { return taskConfiguration; } + + public TaskInformation deepCopy() { + return new TaskInformation( + getJobVertexId(), + getTaskName(), + getNumberOfSubtasks(), + getMaxNumberOfSubtasks(), + getInvokableClassName(), + // Return the new Configuration to avoid shared conf being changed. + new Configuration(getTaskConfiguration())); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskInformation that = (TaskInformation) o; + return numberOfSubtasks == that.numberOfSubtasks + && maxNumberOfSubtasks == that.maxNumberOfSubtasks + && Objects.equals(jobVertexId, that.jobVertexId) + && Objects.equals(taskName, that.taskName) + && Objects.equals(invokableClassName, that.invokableClassName) + && Objects.equals(taskConfiguration, that.taskConfiguration); + } + + @Override + public int hashCode() { + return Objects.hash( + jobVertexId, + taskName, + numberOfSubtasks, + maxNumberOfSubtasks, + invokableClassName, + taskConfiguration); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 7146c2befa952..c3a705e0e0e68 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -300,6 +300,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final ThreadInfoSampleService threadInfoSampleService; + private final GroupCache jobInformationCache; + private final GroupCache taskInformationCache; private final GroupCache shuffleDescriptorsCache; @@ -378,6 +380,8 @@ public TaskExecutor( taskExecutorServices.getSlotAllocationSnapshotPersistenceService(); this.sharedResources = taskExecutorServices.getSharedResources(); + this.jobInformationCache = taskExecutorServices.getJobInformationCache(); + this.taskInformationCache = taskExecutorServices.getTaskInformationCache(); this.shuffleDescriptorsCache = taskExecutorServices.getShuffleDescriptorCache(); } @@ -508,6 +512,8 @@ public CompletableFuture onStop() { changelogStoragesManager.shutdown(); channelStateExecutorFactoryManager.shutdown(); + jobInformationCache.clear(); + taskInformationCache.clear(); shuffleDescriptorsCache.clear(); Preconditions.checkState(jobTable.isEmpty()); @@ -672,7 +678,10 @@ public CompletableFuture submitTask( // re-integrate offloaded data and deserialize shuffle descriptors try { tdd.loadBigData( - taskExecutorBlobService.getPermanentBlobService(), shuffleDescriptorsCache); + taskExecutorBlobService.getPermanentBlobService(), + jobInformationCache, + taskInformationCache, + shuffleDescriptorsCache); } catch (IOException | ClassNotFoundException e) { throw new TaskSubmissionException( "Could not re-integrate offloaded TaskDeploymentDescriptor data.", e); @@ -682,12 +691,8 @@ public CompletableFuture submitTask( final JobInformation jobInformation; final TaskInformation taskInformation; try { - jobInformation = - tdd.getSerializedJobInformation() - .deserializeValue(getClass().getClassLoader()); - taskInformation = - tdd.getSerializedTaskInformation() - .deserializeValue(getClass().getClassLoader()); + jobInformation = tdd.getJobInformation(); + taskInformation = tdd.getTaskInformation(); } catch (IOException | ClassNotFoundException e) { throw new TaskSubmissionException( "Could not deserialize the job or task information.", e); @@ -1907,6 +1912,8 @@ private void releaseJobResources(JobID jobId, Exception cause) { changelogStoragesManager.releaseResourcesForJob(jobId); currentSlotOfferPerJob.remove(jobId); channelStateExecutorFactoryManager.releaseResourcesForJob(jobId); + jobInformationCache.clearCacheForGroup(jobId); + taskInformationCache.clearCacheForGroup(jobId); shuffleDescriptorsCache.clearCacheForGroup(jobId); fileMergingManager.releaseMergingSnapshotManagerForJob(jobId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index ae78575410e4c..586d8291a0f51 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -29,6 +29,8 @@ import org.apache.flink.runtime.entrypoint.WorkingDirectory; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.executiongraph.JobInformation; +import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -95,6 +97,8 @@ public class TaskManagerServices { private final LibraryCacheManager libraryCacheManager; private final SlotAllocationSnapshotPersistenceService slotAllocationSnapshotPersistenceService; private final SharedResources sharedResources; + private final GroupCache jobInformationCache; + private final GroupCache taskInformationCache; private final GroupCache shuffleDescriptorsCache; @@ -117,6 +121,8 @@ public class TaskManagerServices { LibraryCacheManager libraryCacheManager, SlotAllocationSnapshotPersistenceService slotAllocationSnapshotPersistenceService, SharedResources sharedResources, + GroupCache jobInformationCache, + GroupCache taskInformationCache, GroupCache shuffleDescriptorsCache) { this.unresolvedTaskManagerLocation = @@ -139,6 +145,8 @@ public class TaskManagerServices { this.libraryCacheManager = Preconditions.checkNotNull(libraryCacheManager); this.slotAllocationSnapshotPersistenceService = slotAllocationSnapshotPersistenceService; this.sharedResources = Preconditions.checkNotNull(sharedResources); + this.jobInformationCache = jobInformationCache; + this.taskInformationCache = taskInformationCache; this.shuffleDescriptorsCache = Preconditions.checkNotNull(shuffleDescriptorsCache); } @@ -214,6 +222,14 @@ public SharedResources getSharedResources() { return sharedResources; } + public GroupCache getJobInformationCache() { + return jobInformationCache; + } + + public GroupCache getTaskInformationCache() { + return taskInformationCache; + } + public GroupCache getShuffleDescriptorCache() { return shuffleDescriptorsCache; } @@ -415,6 +431,11 @@ public static TaskManagerServices fromConfiguration( NoOpSlotAllocationSnapshotPersistenceService.INSTANCE; } + final GroupCache jobInformationCache = + new DefaultGroupCache.Factory().create(); + final GroupCache taskInformationCache = + new DefaultGroupCache.Factory().create(); + final GroupCache shuffleDescriptorsCache = new DefaultGroupCache.Factory() .create(); @@ -438,6 +459,8 @@ public static TaskManagerServices fromConfiguration( libraryCacheManager, slotAllocationSnapshotPersistenceService, new SharedResources(), + jobInformationCache, + taskInformationCache, shuffleDescriptorsCache); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 6191b19767d20..ac003665584fe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -20,9 +20,13 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -32,14 +36,21 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; +import org.apache.flink.runtime.util.DefaultGroupCache; +import org.apache.flink.runtime.util.GroupCache; +import org.apache.flink.runtime.util.NoOpGroupCache; +import org.apache.flink.testutils.junit.utils.TempDirUtils; +import org.apache.flink.types.Either; import org.apache.flink.util.SerializedValue; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import javax.annotation.Nonnull; import java.io.IOException; import java.net.URL; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; @@ -50,6 +61,8 @@ /** Tests for the {@link TaskDeploymentDescriptor}. */ class TaskDeploymentDescriptorTest { + @TempDir Path temporaryFolder; + private static final JobID jobID = new JobID(); private static final JobVertexID vertexID = new JobVertexID(); private static final ExecutionAttemptID execId = createExecutionAttemptId(vertexID); @@ -73,24 +86,28 @@ class TaskDeploymentDescriptorTest { private final SerializedValue executionConfig = new SerializedValue<>(new ExecutionConfig()); + + private final JobInformation jobInformation = + new JobInformation( + jobID, + jobName, + executionConfig, + jobConfiguration, + requiredJars, + requiredClasspaths); private final SerializedValue serializedJobInformation = - new SerializedValue<>( - new JobInformation( - jobID, - jobName, - executionConfig, - jobConfiguration, - requiredJars, - requiredClasspaths)); + new SerializedValue<>(jobInformation); + + private final TaskInformation taskInformation = + new TaskInformation( + vertexID, + taskName, + currentNumberOfSubtasks, + numberOfKeyGroups, + invokableClass.getName(), + taskConfiguration); private final SerializedValue serializedJobVertexInformation = - new SerializedValue<>( - new TaskInformation( - vertexID, - taskName, - currentNumberOfSubtasks, - numberOfKeyGroups, - invokableClass.getName(), - taskConfiguration)); + new SerializedValue<>(taskInformation); TaskDeploymentDescriptorTest() throws IOException {} @@ -104,19 +121,15 @@ void testSerialization() throws Exception { final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); - assertThat(orig.getSerializedJobInformation()) - .isNotSameAs(copy.getSerializedJobInformation()); - assertThat(orig.getSerializedTaskInformation()) - .isNotSameAs(copy.getSerializedTaskInformation()); + assertThat(orig.getJobInformation()).isNotSameAs(copy.getJobInformation()); + assertThat(orig.getTaskInformation()).isNotSameAs(copy.getTaskInformation()); assertThat(orig.getExecutionAttemptId()).isNotSameAs(copy.getExecutionAttemptId()); assertThat(orig.getTaskRestore()).isNotSameAs(copy.getTaskRestore()); assertThat(orig.getProducedPartitions()).isNotSameAs(copy.getProducedPartitions()); assertThat(orig.getInputGates()).isNotSameAs(copy.getInputGates()); - assertThat(orig.getSerializedJobInformation()) - .isEqualTo(copy.getSerializedJobInformation()); - assertThat(orig.getSerializedTaskInformation()) - .isEqualTo(copy.getSerializedTaskInformation()); + assertThat(orig.getJobInformation()).isEqualTo(copy.getJobInformation()); + assertThat(orig.getTaskInformation()).isEqualTo(copy.getTaskInformation()); assertThat(orig.getExecutionAttemptId()).isEqualTo(copy.getExecutionAttemptId()); assertThat(orig.getAllocationId()).isEqualTo(copy.getAllocationId()); assertThat(orig.getSubtaskIndex()).isEqualTo(copy.getSubtaskIndex()); @@ -130,20 +143,88 @@ void testSerialization() throws Exception { } @Test - void testOffLoadedAndNonOffLoadedPayload() { + void testOffLoadedAndNonOffLoadedPayload() throws IOException, ClassNotFoundException { final TaskDeploymentDescriptor taskDeploymentDescriptor = createTaskDeploymentDescriptor( new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), new TaskDeploymentDescriptor.Offloaded<>(new PermanentBlobKey())); - SerializedValue actualSerializedJobInformation = - taskDeploymentDescriptor.getSerializedJobInformation(); - assertThat(actualSerializedJobInformation).isSameAs(serializedJobInformation); + JobInformation actualJobInformation = taskDeploymentDescriptor.getJobInformation(); + assertThat(actualJobInformation).isEqualTo(jobInformation); - assertThatThrownBy(taskDeploymentDescriptor::getSerializedTaskInformation) + assertThatThrownBy(taskDeploymentDescriptor::getTaskInformation) .isInstanceOf(IllegalStateException.class); } + @Test + void testTaskInformationCache() throws IOException, ClassNotFoundException { + try (BlobServer blobServer = setupBlobServer()) { + // Serialize taskInformation to blobServer and get the permanentBlobKey + Either, PermanentBlobKey> taskInformationOrBlobKey = + BlobWriter.serializeAndTryOffload(taskInformation, jobID, blobServer); + assertThat(taskInformationOrBlobKey.isRight()).isTrue(); + PermanentBlobKey permanentBlobKey = taskInformationOrBlobKey.right(); + + GroupCache taskInformationCache = + new DefaultGroupCache.Factory() + .create(); + // Test for tdd1 + final TaskDeploymentDescriptor tdd1 = + createTaskDeploymentDescriptor( + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.Offloaded<>(permanentBlobKey)); + assertThat(taskInformationCache.get(jobID, permanentBlobKey)).isNull(); + tdd1.loadBigData( + blobServer, + new NoOpGroupCache<>(), + taskInformationCache, + new NoOpGroupCache<>()); + TaskInformation taskInformation1 = tdd1.getTaskInformation(); + assertThat(taskInformation1).isEqualTo(taskInformation); + // The TaskInformation is cached in taskInformationCache, and it's equals to + // taskInformation1. + assertThat(taskInformationCache.get(jobID, permanentBlobKey)) + .isNotNull() + .isEqualTo(taskInformation1); + + // Test for tdd2 + final TaskDeploymentDescriptor tdd2 = + createTaskDeploymentDescriptor( + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.Offloaded<>(permanentBlobKey)); + tdd2.loadBigData( + blobServer, + new NoOpGroupCache<>(), + taskInformationCache, + new NoOpGroupCache<>()); + TaskInformation taskInformation2 = tdd2.getTaskInformation(); + // The TaskInformation2 is equals to taskInformation1 and original taskInformation, but + // they are not same. + assertThat(taskInformation2) + .isNotNull() + .isEqualTo(taskInformation1) + .isNotSameAs(taskInformation1) + .isEqualTo(taskInformation); + // Configuration may be changed by subtask, so the configuration must be not same. + assertThat(taskInformation2.getTaskConfiguration()) + .isNotNull() + .isEqualTo(taskInformation1.getTaskConfiguration()) + .isNotSameAs(taskInformation1.getTaskConfiguration()); + } + } + + private BlobServer setupBlobServer() throws IOException { + + Configuration config = new Configuration(); + // always offload the serialized job and task information + config.setInteger(BlobServerOptions.OFFLOAD_MINSIZE, 0); + BlobServer blobServer = + new BlobServer( + config, TempDirUtils.newFolder(temporaryFolder), new VoidBlobStore()); + blobServer.start(); + return blobServer; + } + @Nonnull private TaskDeploymentDescriptor createTaskDeploymentDescriptor( TaskDeploymentDescriptor.MaybeOffloaded jobInformation, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobMasterTester.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobMasterTester.java index a5508f9698ec7..08783509a9fe1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobMasterTester.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobMasterTester.java @@ -204,9 +204,7 @@ private CompletableFuture getTaskInformation( "Task descriptor for %s not found.", executionAttemptId))); try { - return descriptor - .getSerializedTaskInformation() - .deserializeValue(Thread.currentThread().getContextClassLoader()); + return descriptor.getTaskInformation(); } catch (Exception e) { throw new IllegalStateException( String.format( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java index 54d8fc3096402..d354b49fedde9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java @@ -179,7 +179,11 @@ void testBuildDeploymentDescriptor() throws Exception { taskManagerGateway.setSubmitConsumer( FunctionUtils.uncheckedConsumer( taskDeploymentDescriptor -> { - taskDeploymentDescriptor.loadBigData(blobCache, new NoOpGroupCache<>()); + taskDeploymentDescriptor.loadBigData( + blobCache, + new NoOpGroupCache<>(), + new NoOpGroupCache<>(), + new NoOpGroupCache<>()); tdd.complete(taskDeploymentDescriptor); })); @@ -202,10 +206,8 @@ void testBuildDeploymentDescriptor() throws Exception { TaskDeploymentDescriptor descr = tdd.get(); assertThat(descr).isNotNull(); - JobInformation jobInformation = - descr.getSerializedJobInformation().deserializeValue(getClass().getClassLoader()); - TaskInformation taskInformation = - descr.getSerializedTaskInformation().deserializeValue(getClass().getClassLoader()); + JobInformation jobInformation = descr.getJobInformation(); + TaskInformation taskInformation = descr.getTaskInformation(); assertThat(descr.getJobId()).isEqualTo(jobId); assertThat(jobInformation.getJobId()).isEqualTo(jobId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java index 1848faa3959bb..fae6a754646a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java @@ -120,7 +120,11 @@ void testDeployMultipleTasksWithSmallBlobCacheSizeLimit() throws Exception { taskManagerGateway.setSubmitConsumer( FunctionUtils.uncheckedConsumer( taskDeploymentDescriptor -> { - taskDeploymentDescriptor.loadBigData(blobCache, new NoOpGroupCache<>()); + taskDeploymentDescriptor.loadBigData( + blobCache, + new NoOpGroupCache<>(), + new NoOpGroupCache<>(), + new NoOpGroupCache<>()); tdds.offer(taskDeploymentDescriptor); })); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 193e56e20c3a9..68d9d5a275de7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -201,6 +201,8 @@ public TaskManagerServices build() { libraryCacheManager, slotAllocationSnapshotPersistenceService, sharedResources, + new NoOpGroupCache<>(), + new NoOpGroupCache<>(), new NoOpGroupCache<>()); } } From b759794e33d68b557f4ee2d7287f5fae9a80167e Mon Sep 17 00:00:00 2001 From: Rui Fan <1996fanrui@gmail.com> Date: Fri, 27 Oct 2023 15:34:58 +0800 Subject: [PATCH 084/104] [FLINK-33354][runtime] Using the InputStream instead of byte array to avoid contiguous huge memory usage --- .../flink/runtime/deployment/TaskDeploymentDescriptor.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index e92162fb14eac..5684066735f03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -28,16 +28,17 @@ import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.util.GroupCache; -import org.apache.flink.util.FileUtils; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import javax.annotation.Nullable; +import java.io.BufferedInputStream; import java.io.File; import java.io.IOException; import java.io.Serializable; +import java.nio.file.Files; import java.util.List; /** @@ -281,7 +282,7 @@ public void loadBigData( // enters a terminal state) jobInformation = InstantiationUtil.deserializeObject( - FileUtils.readAllBytes(dataFile.toPath()), + new BufferedInputStream(Files.newInputStream(dataFile.toPath())), getClass().getClassLoader()); jobInformationCache.put(jobId, jobInfoKey, jobInformation); } @@ -303,7 +304,7 @@ public void loadBigData( // enters a terminal state) taskInformation = InstantiationUtil.deserializeObject( - FileUtils.readAllBytes(dataFile.toPath()), + new BufferedInputStream(Files.newInputStream(dataFile.toPath())), getClass().getClassLoader()); taskInformationCache.put(jobId, taskInfoKey, taskInformation); } From c83792b0f3fb6ae18dfabd52a89dc42c074ee73c Mon Sep 17 00:00:00 2001 From: Rui Fan <1996fanrui@gmail.com> Date: Fri, 27 Oct 2023 10:53:15 +0800 Subject: [PATCH 085/104] [FLINK-33317][runtime] Add cleaning mechanism for initial configs to reduce the memory usage Address Piotr's comment Refactor the test to avoid create RegularOperatorChain repeatedly --- .../streaming/api/graph/StreamConfig.java | 31 +++++++++ .../streaming/runtime/tasks/StreamTask.java | 1 + .../streaming/graph/StreamConfigTest.java | 65 +++++++++++++++++++ .../SubtaskCheckpointCoordinatorTest.java | 24 ++++--- .../streaming/util/MockStreamConfig.java | 14 ++++ 5 files changed, 126 insertions(+), 9 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamConfigTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 79a5e904ea98e..91ed4ce3b4797 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -52,6 +52,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -151,6 +152,13 @@ public class StreamConfig implements Serializable { private final transient CompletableFuture serializationFuture = new CompletableFuture<>(); + /** + * In order to release memory during processing data, some keys are removed in {@link + * #clearInitialConfigs()}. Recording these keys here to prevent they are accessed after + * removing. + */ + private final Set removedKeys = new HashSet<>(); + public StreamConfig(Configuration config) { this.config = config; } @@ -386,6 +394,9 @@ public > T getStreamOperator(ClassLoader cl) { public > T getStreamOperatorFactory(ClassLoader cl) { try { + checkState( + !removedKeys.contains(SERIALIZED_UDF), + String.format("%s has been removed.", SERIALIZED_UDF)); return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZED_UDF, cl); } catch (ClassNotFoundException e) { String classLoaderInfo = ClassLoaderUtil.getUserCodeClassLoaderInfo(cl); @@ -584,6 +595,9 @@ public void setTransitiveChainedTaskConfigs(Map chainedTa public Map getTransitiveChainedTaskConfigs(ClassLoader cl) { try { + checkState( + !removedKeys.contains(CHAINED_TASK_CONFIG), + String.format("%s has been removed.", CHAINED_TASK_CONFIG)); Map confs = InstantiationUtil.readObjectFromConfig(this.config, CHAINED_TASK_CONFIG, cl); return confs == null ? new HashMap() : confs; @@ -795,6 +809,23 @@ public boolean isGraphContainingLoops() { return config.getBoolean(GRAPH_CONTAINING_LOOPS, false); } + /** + * In general, we don't clear any configuration. However, the {@link #SERIALIZED_UDF} may be + * very large when operator includes some large objects, the SERIALIZED_UDF is used to create a + * StreamOperator and usually only needs to be called once. {@link #CHAINED_TASK_CONFIG} may be + * large as well due to the StreamConfig of all non-head operators in OperatorChain will be + * serialized and stored in CHAINED_TASK_CONFIG. They can be cleared to reduce the memory after + * StreamTask is initialized. If so, TM will have more memory during running. See FLINK-33315 + * and FLINK-33317 for more information. + */ + public void clearInitialConfigs() { + removedKeys.add(SERIALIZED_UDF); + config.removeKey(SERIALIZED_UDF); + + removedKeys.add(CHAINED_TASK_CONFIG); + config.removeKey(CHAINED_TASK_CONFIG); + } + /** * Requirements of the different inputs of an operator. Each input can have a different * requirement. For all {@link #SORTED} inputs, records are sorted/grouped by key and all diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 39b460337dd02..eef96812fc43b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -743,6 +743,7 @@ void restoreInternal() throws Exception { // task specific initialization init(); + configuration.clearInitialConfigs(); // save the work of reloading state, etc, if the task is already canceled ensureNotCanceled(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamConfigTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamConfigTest.java new file mode 100644 index 0000000000000..d3eb45f1990f9 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamConfigTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.graph; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.util.MockStreamConfig; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link StreamConfig}. */ +class StreamConfigTest { + + @Test + void testClearInitialConfigs() { + int chainedTaskId = 3456; + MockStreamConfig streamConfig = + new MockStreamConfig( + new Configuration(), + 1, + Collections.singletonMap( + chainedTaskId, new MockStreamConfig(new Configuration(), 1))); + + ClassLoader classLoader = getClass().getClassLoader(); + StreamOperatorFactory streamOperatorFactory = + streamConfig.getStreamOperatorFactory(classLoader); + assertThat(streamOperatorFactory).isNotNull(); + assertThat(streamConfig.getStreamOperatorFactoryClass(classLoader)).isNotNull(); + assertThat(streamConfig.getTransitiveChainedTaskConfigs(classLoader)) + .hasSize(1) + .containsKey(chainedTaskId); + + // StreamOperatorFactory and ChainedTaskConfigs should be cleared after clearInitialConfigs, + // but the factory class shouldn't be cleared. + streamConfig.clearInitialConfigs(); + assertThatThrownBy(() -> streamConfig.getStreamOperatorFactory(classLoader)) + .hasCauseInstanceOf(IllegalStateException.class) + .hasRootCauseMessage("serializedUDF has been removed."); + assertThat(streamConfig.getStreamOperatorFactoryClass(classLoader)).isNotNull(); + assertThatThrownBy(() -> streamConfig.getTransitiveChainedTaskConfigs(classLoader)) + .hasCauseInstanceOf(IllegalStateException.class) + .hasRootCauseMessage("chainedTaskConfig_ has been removed."); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java index 5bb1110922bf0..2cba71b4e908f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java @@ -44,6 +44,7 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.DoneFuture; @@ -72,7 +73,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -389,10 +389,15 @@ public void testBroadcastCancelCheckpointMarkerOnAbortingFromCoordinator() throw StreamConfig streamConfig = testHarness.getStreamConfig(); streamConfig.setStreamOperator(new MapOperator()); - testHarness.invoke(); - testHarness.waitForTaskRunning(); - - MockEnvironment mockEnvironment = MockEnvironment.builder().build(); + StreamMockEnvironment mockEnvironment = + new StreamMockEnvironment( + testHarness.jobConfig, + testHarness.taskConfig, + testHarness.executionConfig, + testHarness.memorySize, + new MockInputSplitProvider(), + testHarness.bufferSize, + testHarness.taskStateManager); try (SubtaskCheckpointCoordinator subtaskCheckpointCoordinator = new MockSubtaskCheckpointCoordinatorBuilder() @@ -404,13 +409,14 @@ public void testBroadcastCancelCheckpointMarkerOnAbortingFromCoordinator() throw ResultPartitionWriter resultPartitionWriter = new RecordOrEventCollectingResultPartitionWriter<>( recordOrEvents, stringStreamElementSerializer); - mockEnvironment.addOutputs(Collections.singletonList(resultPartitionWriter)); + mockEnvironment.addOutput(resultPartitionWriter); + + testHarness.invoke(mockEnvironment); + testHarness.waitForTaskRunning(); OneInputStreamTask task = testHarness.getTask(); OperatorChain> operatorChain = - new RegularOperatorChain<>( - task, - StreamTask.createRecordWriterDelegate(streamConfig, mockEnvironment)); + task.operatorChain; long checkpointId = 42L; // notify checkpoint aborted before execution. subtaskCheckpointCoordinator.notifyCheckpointAborted( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java index 3080e55f512c5..ec35ce681e202 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java @@ -30,13 +30,24 @@ import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.tasks.SourceStreamTask; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; +import java.util.Map; /** A dummy stream config implementation for specifying the number of outputs in tests. */ public class MockStreamConfig extends StreamConfig { public MockStreamConfig(Configuration configuration, int numberOfOutputs) { + this(configuration, numberOfOutputs, null); + } + + public MockStreamConfig( + Configuration configuration, + int numberOfOutputs, + @Nullable Map chainedTaskConfigs) { + super(configuration); setChainStart(); @@ -71,6 +82,9 @@ public MockStreamConfig(Configuration configuration, int numberOfOutputs) { } setVertexNonChainedOutputs(streamOutputs); setOperatorNonChainedOutputs(streamOutputs); + if (chainedTaskConfigs != null) { + setAndSerializeTransitiveChainedTaskConfigs(chainedTaskConfigs); + } serializeAllConfigs(); } } From ed07a70e45a753a259287c9fdbdae73e3415cc21 Mon Sep 17 00:00:00 2001 From: Yuepeng Pan Date: Tue, 7 Nov 2023 17:54:47 +0800 Subject: [PATCH 086/104] [FLINK-33448][runtime] Introduce the new configuration 'taskmanager.load-balance.mode' (#23658) --- .../generated/all_taskmanager_section.html | 6 ++ .../generated/cluster_configuration.html | 6 -- .../generated/expert_scheduling_section.html | 12 +-- .../generated/task_manager_configuration.html | 6 ++ .../flink/configuration/ClusterOptions.java | 7 ++ .../configuration/TaskManagerOptions.java | 52 +++++++++++++ .../TaskManagerLoadBalanceModeTest.java | 76 +++++++++++++++++++ .../ResourceManagerRuntimeServices.java | 2 +- .../DefaultResourceAllocationStrategy.java | 5 +- .../slotmanager/SlotManagerConfiguration.java | 21 ++--- .../util/SlotSelectionStrategyUtils.java | 10 ++- .../DeclarativeSlotManagerBuilder.java | 15 ++-- .../DeclarativeSlotManagerTest.java | 3 +- ...DefaultResourceAllocationStrategyTest.java | 19 +++-- ...faultResourceAllocationStrategyITCase.java | 2 +- .../SlotManagerConfigurationBuilder.java | 15 ++-- 16 files changed, 206 insertions(+), 51 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java diff --git a/docs/layouts/shortcodes/generated/all_taskmanager_section.html b/docs/layouts/shortcodes/generated/all_taskmanager_section.html index bde8c250bdf89..256672be9b56f 100644 --- a/docs/layouts/shortcodes/generated/all_taskmanager_section.html +++ b/docs/layouts/shortcodes/generated/all_taskmanager_section.html @@ -74,6 +74,12 @@ Boolean Whether to kill the TaskManager when the task thread throws an OutOfMemoryError. + +

taskmanager.load-balance.mode
+ NONE +

Enum

+ Mode for the load-balance allocation strategy across all available TaskManagers.
  • The SLOTS mode tries to spread out the slots evenly across all available TaskManagers.
  • The NONE mode is the default mode without any specified strategy.


Possible values:
  • "NONE"
  • "SLOTS"
+
taskmanager.memory.min-segment-size
256 bytes diff --git a/docs/layouts/shortcodes/generated/cluster_configuration.html b/docs/layouts/shortcodes/generated/cluster_configuration.html index d734ea405f13e..49b6af5f659a0 100644 --- a/docs/layouts/shortcodes/generated/cluster_configuration.html +++ b/docs/layouts/shortcodes/generated/cluster_configuration.html @@ -8,12 +8,6 @@ - -
cluster.evenly-spread-out-slots
- false - Boolean - Enable the slot spread out allocation strategy. This strategy tries to spread out the slots evenly across all available TaskExecutors. -
cluster.intercept-user-system-exit
DISABLED diff --git a/docs/layouts/shortcodes/generated/expert_scheduling_section.html b/docs/layouts/shortcodes/generated/expert_scheduling_section.html index 2c07a95372fd7..96fc305609914 100644 --- a/docs/layouts/shortcodes/generated/expert_scheduling_section.html +++ b/docs/layouts/shortcodes/generated/expert_scheduling_section.html @@ -8,12 +8,6 @@ - -
cluster.evenly-spread-out-slots
- false - Boolean - Enable the slot spread out allocation strategy. This strategy tries to spread out the slots evenly across all available TaskExecutors. -
execution.batch.adaptive.auto-parallelism.avg-data-volume-per-task
16 mb @@ -188,5 +182,11 @@ Double The finished execution ratio threshold to calculate the slow tasks detection baseline. Given that the parallelism is N and the ratio is R, define T as the median of the first N*R finished tasks' execution time. The baseline will be T*M, where M is the multiplier of the baseline. Note that the execution time will be weighted with the task's input bytes to ensure the accuracy of the detection if data skew occurs. + +
taskmanager.load-balance.mode
+ NONE +

Enum

+ Mode for the load-balance allocation strategy across all available TaskManagers.
  • The SLOTS mode tries to spread out the slots evenly across all available TaskManagers.
  • The NONE mode is the default mode without any specified strategy.


Possible values:
  • "NONE"
  • "SLOTS"
+ diff --git a/docs/layouts/shortcodes/generated/task_manager_configuration.html b/docs/layouts/shortcodes/generated/task_manager_configuration.html index 6f2c12d5df3f2..61e01e96bba2e 100644 --- a/docs/layouts/shortcodes/generated/task_manager_configuration.html +++ b/docs/layouts/shortcodes/generated/task_manager_configuration.html @@ -56,6 +56,12 @@ Boolean Whether to kill the TaskManager when the task thread throws an OutOfMemoryError. + +
taskmanager.load-balance.mode
+ NONE +

Enum

+ Mode for the load-balance allocation strategy across all available TaskManagers.
  • The SLOTS mode tries to spread out the slots evenly across all available TaskManagers.
  • The NONE mode is the default mode without any specified strategy.


Possible values:
  • "NONE"
  • "SLOTS"
+
taskmanager.network.bind-policy
"ip" diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java index 58150beb4b7f5..c0b9ca8459ef1 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java @@ -84,6 +84,13 @@ public class ClusterOptions { + "By default it will use 4 * the number of CPU cores (hardware contexts) that the cluster process has access to. " + "Increasing the pool size allows to run more IO operations concurrently."); + /** + * @deprecated Please use {@link TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE} instead. + * Note: The 'taskmanager.load-balance.mode: SLOTS' is equal to + * 'cluster.evenly-spread-out-slots: true'. The 'taskmanager.load-balance.mode: NONE' is + * equal to 'cluster.evenly-spread-out-slots: false'. + */ + @Deprecated @Documentation.Section(Documentation.Sections.EXPERT_SCHEDULING) public static final ConfigOption EVENLY_SPREAD_OUT_SLOTS_STRATEGY = ConfigOptions.key("cluster.evenly-spread-out-slots") diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index 8f4a836d4f338..0e4bc7f7d15f3 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -25,9 +25,12 @@ import org.apache.flink.configuration.description.Description; import org.apache.flink.util.TimeUtils; +import javax.annotation.Nonnull; + import java.time.Duration; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static org.apache.flink.configuration.ConfigOptions.key; import static org.apache.flink.configuration.description.TextElement.code; @@ -708,6 +711,55 @@ public class TaskManagerOptions { "Time we wait for the timers in milliseconds to finish all pending timer threads" + " when the stream task is cancelled."); + @Documentation.Section({ + Documentation.Sections.EXPERT_SCHEDULING, + Documentation.Sections.ALL_TASK_MANAGER + }) + public static final ConfigOption TASK_MANAGER_LOAD_BALANCE_MODE = + ConfigOptions.key("taskmanager.load-balance.mode") + .enumType(TaskManagerLoadBalanceMode.class) + .defaultValue(TaskManagerLoadBalanceMode.NONE) + .withDescription( + Description.builder() + .text( + "Mode for the load-balance allocation strategy across all available %s.", + code("TaskManagers")) + .list( + text( + "The %s mode tries to spread out the slots evenly across all available %s.", + code(TaskManagerLoadBalanceMode.SLOTS.name()), + code("TaskManagers")), + text( + "The %s mode is the default mode without any specified strategy.", + code(TaskManagerLoadBalanceMode.NONE.name()))) + .build()); + + /** Type of {@link TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE}. */ + public enum TaskManagerLoadBalanceMode { + NONE, + SLOTS; + + /** + * The method is mainly to load the {@link + * TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE} from {@link Configuration}, which is + * compatible with {@link ClusterOptions#EVENLY_SPREAD_OUT_SLOTS_STRATEGY}. + */ + public static TaskManagerLoadBalanceMode loadFromConfiguration( + @Nonnull Configuration configuration) { + Optional taskManagerLoadBalanceModeOptional = + configuration.getOptional(TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE); + if (taskManagerLoadBalanceModeOptional.isPresent()) { + return taskManagerLoadBalanceModeOptional.get(); + } + boolean evenlySpreadOutSlots = + configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + + return evenlySpreadOutSlots + ? TaskManagerLoadBalanceMode.SLOTS + : TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE.defaultValue(); + } + } + // ------------------------------------------------------------------------ /** Not intended to be instantiated. */ diff --git a/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java b/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java new file mode 100644 index 0000000000000..4ffae33b0808b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.configuration; + +import org.junit.jupiter.api.Test; + +import static org.apache.flink.configuration.ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY; +import static org.apache.flink.configuration.TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link TaskManagerLoadBalanceMode}. */ +class TaskManagerLoadBalanceModeTest { + + @Test + void testReadTaskManagerLoadBalanceMode() { + // Check for non-set 'taskmanager.load-balance.mode' and + // 'cluster.evenly-spread-out-slots: false' + Configuration conf1 = new Configuration(); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf1)) + .isEqualTo(TASK_MANAGER_LOAD_BALANCE_MODE.defaultValue()); + + // Check for non-set 'taskmanager.load-balance.mode' and + // 'cluster.evenly-spread-out-slots: true' + Configuration conf2 = new Configuration(); + conf2.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf2)) + .isEqualTo(TaskManagerLoadBalanceMode.SLOTS); + + // Check for setting manually 'taskmanager.load-balance.mode: NONE' and + // 'cluster.evenly-spread-out-slots: false' + Configuration conf3 = new Configuration(); + conf3.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.NONE); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf3)) + .isEqualTo(TaskManagerLoadBalanceMode.NONE); + + // Check for setting manually 'taskmanager.load-balance.mode: NONE' and + // 'cluster.evenly-spread-out-slots: true' + Configuration conf4 = new Configuration(); + conf4.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.NONE); + conf4.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf4)) + .isEqualTo(TaskManagerLoadBalanceMode.NONE); + + // Check for setting manually 'taskmanager.load-balance.mode: SLOTS' and + // 'cluster.evenly-spread-out-slots: false' + Configuration conf5 = new Configuration(); + conf5.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.SLOTS); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf5)) + .isEqualTo(TaskManagerLoadBalanceMode.SLOTS); + + // Check for setting manually 'taskmanager.load-balance.mode: SLOTS' and + // 'cluster.evenly-spread-out-slots: true' + Configuration conf6 = new Configuration(); + conf6.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.SLOTS); + conf6.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true); + assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf6)) + .isEqualTo(TaskManagerLoadBalanceMode.SLOTS); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java index 1a18364c2077a..1368748fb9c6f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java @@ -90,7 +90,7 @@ private static SlotManager createSlotManager( SlotManagerUtils.generateTaskManagerTotalResourceProfile( slotManagerConfiguration.getDefaultWorkerResourceSpec()), slotManagerConfiguration.getNumSlotsPerWorker(), - slotManagerConfiguration.isEvenlySpreadOutSlots(), + slotManagerConfiguration.getTaskManagerLoadBalanceMode(), slotManagerConfiguration.getTaskManagerTimeout(), slotManagerConfiguration.getRedundantTaskManagerNum(), slotManagerConfiguration.getMinTotalCpu(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java index 9f12c46972ce5..62f31480b4abc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java @@ -40,6 +40,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile; /** @@ -84,7 +85,7 @@ public class DefaultResourceAllocationStrategy implements ResourceAllocationStra public DefaultResourceAllocationStrategy( ResourceProfile totalResourceProfile, int numSlotsPerWorker, - boolean evenlySpreadOutSlots, + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode, Time taskManagerTimeout, int redundantTaskManagerNum, CPUResource minTotalCPU, @@ -95,7 +96,7 @@ public DefaultResourceAllocationStrategy( SlotManagerUtils.generateDefaultSlotResourceProfile( totalResourceProfile, numSlotsPerWorker); this.availableResourceMatchingStrategy = - evenlySpreadOutSlots + taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS ? LeastUtilizationResourceMatchingStrategy.INSTANCE : AnyMatchingResourceMatchingStrategy.INSTANCE; this.taskManagerTimeout = taskManagerTimeout; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java index 330a65b783470..a41b0c16ef528 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.resources.CPUResource; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.AkkaOptions; -import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ResourceManagerOptions; @@ -33,6 +32,8 @@ import java.math.RoundingMode; import java.time.Duration; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; + /** Configuration for the {@link SlotManager}. */ public class SlotManagerConfiguration { private final Time taskManagerRequestTimeout; @@ -41,7 +42,7 @@ public class SlotManagerConfiguration { private final Duration declareNeededResourceDelay; private final boolean waitResultConsumedBeforeRelease; private final SlotMatchingStrategy slotMatchingStrategy; - private final boolean evenlySpreadOutSlots; + private final TaskManagerLoadBalanceMode taskManagerLoadBalanceMode; private final WorkerResourceSpec defaultWorkerResourceSpec; private final int numSlotsPerWorker; private final int minSlotNum; @@ -59,7 +60,7 @@ public SlotManagerConfiguration( Duration declareNeededResourceDelay, boolean waitResultConsumedBeforeRelease, SlotMatchingStrategy slotMatchingStrategy, - boolean evenlySpreadOutSlots, + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode, WorkerResourceSpec defaultWorkerResourceSpec, int numSlotsPerWorker, int minSlotNum, @@ -76,7 +77,7 @@ public SlotManagerConfiguration( this.declareNeededResourceDelay = Preconditions.checkNotNull(declareNeededResourceDelay); this.waitResultConsumedBeforeRelease = waitResultConsumedBeforeRelease; this.slotMatchingStrategy = Preconditions.checkNotNull(slotMatchingStrategy); - this.evenlySpreadOutSlots = evenlySpreadOutSlots; + this.taskManagerLoadBalanceMode = taskManagerLoadBalanceMode; this.defaultWorkerResourceSpec = Preconditions.checkNotNull(defaultWorkerResourceSpec); Preconditions.checkState(numSlotsPerWorker > 0); this.numSlotsPerWorker = numSlotsPerWorker; @@ -199,8 +200,8 @@ public SlotMatchingStrategy getSlotMatchingStrategy() { return slotMatchingStrategy; } - public boolean isEvenlySpreadOutSlots() { - return evenlySpreadOutSlots; + public TaskManagerLoadBalanceMode getTaskManagerLoadBalanceMode() { + return taskManagerLoadBalanceMode; } public WorkerResourceSpec getDefaultWorkerResourceSpec() { @@ -260,10 +261,10 @@ public static SlotManagerConfiguration fromConfiguration( configuration.getBoolean( ResourceManagerOptions.TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMED); - boolean evenlySpreadOutSlots = - configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode = + TaskManagerLoadBalanceMode.loadFromConfiguration(configuration); final SlotMatchingStrategy slotMatchingStrategy = - evenlySpreadOutSlots + taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS ? LeastUtilizationSlotMatchingStrategy.INSTANCE : AnyMatchingSlotMatchingStrategy.INSTANCE; @@ -282,7 +283,7 @@ public static SlotManagerConfiguration fromConfiguration( declareNeededResourceDelay, waitResultConsumedBeforeRelease, slotMatchingStrategy, - evenlySpreadOutSlots, + taskManagerLoadBalanceMode, defaultWorkerResourceSpec, numSlotsPerWorker, minSlotNum, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtils.java index 3eb7a9f20f3ad..e4693fa353b1c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtils.java @@ -20,8 +20,8 @@ package org.apache.flink.runtime.util; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.jobgraph.JobType; import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; import org.apache.flink.runtime.jobmaster.slotpool.PreviousAllocationSlotSelectionStrategy; @@ -30,6 +30,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; + /** Utility class for selecting {@link SlotSelectionStrategy}. */ public class SlotSelectionStrategyUtils { @@ -37,13 +39,13 @@ public class SlotSelectionStrategyUtils { public static SlotSelectionStrategy selectSlotSelectionStrategy( final JobType jobType, final Configuration configuration) { - final boolean evenlySpreadOutSlots = - configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode = + TaskManagerOptions.TaskManagerLoadBalanceMode.loadFromConfiguration(configuration); final SlotSelectionStrategy locationPreferenceSlotSelectionStrategy; locationPreferenceSlotSelectionStrategy = - evenlySpreadOutSlots + taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS ? LocationPreferenceSlotSelectionStrategy.createEvenlySpreadOut() : LocationPreferenceSlotSelectionStrategy.createDefault(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java index 57492af56f175..aaa240a2b6d66 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java @@ -34,9 +34,11 @@ import java.time.Duration; import java.util.concurrent.Executor; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; + /** Builder for {@link DeclarativeSlotManager}. */ public class DeclarativeSlotManagerBuilder { - private boolean evenlySpreadOutSlots; + private TaskManagerLoadBalanceMode taskManagerLoadBalanceMode; private final ScheduledExecutor scheduledExecutor; private Time taskManagerRequestTimeout; private Time taskManagerTimeout; @@ -53,7 +55,7 @@ public class DeclarativeSlotManagerBuilder { private Duration declareNeededResourceDelay; private DeclarativeSlotManagerBuilder(ScheduledExecutor scheduledExecutor) { - this.evenlySpreadOutSlots = false; + this.taskManagerLoadBalanceMode = TaskManagerLoadBalanceMode.NONE; this.scheduledExecutor = scheduledExecutor; this.taskManagerRequestTimeout = TestingUtils.infiniteTime(); this.taskManagerTimeout = TestingUtils.infiniteTime(); @@ -93,8 +95,9 @@ public DeclarativeSlotManagerBuilder setWaitResultConsumedBeforeRelease( return this; } - public DeclarativeSlotManagerBuilder setEvenlySpreadOutSlots(boolean evenlySpreadOutSlots) { - this.evenlySpreadOutSlots = evenlySpreadOutSlots; + public DeclarativeSlotManagerBuilder setTaskManagerLoadBalanceMode( + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode) { + this.taskManagerLoadBalanceMode = taskManagerLoadBalanceMode; return this; } @@ -159,10 +162,10 @@ public DeclarativeSlotManager build() { requirementCheckDelay, declareNeededResourceDelay, waitResultConsumedBeforeRelease, - evenlySpreadOutSlots + taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS ? LeastUtilizationSlotMatchingStrategy.INSTANCE : AnyMatchingSlotMatchingStrategy.INSTANCE, - evenlySpreadOutSlots, + taskManagerLoadBalanceMode, defaultWorkerResourceSpec, numSlotsPerWorker, minSlotNum, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java index 339509125eb11..5bbb678b34396 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java @@ -81,6 +81,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link DeclarativeSlotManager}. */ @@ -1121,7 +1122,7 @@ private TaskExecutorConnection createTaskExecutorConnection( void testSpreadOutSlotAllocationStrategy() throws Exception { try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder() - .setEvenlySpreadOutSlots(true) + .setTaskManagerLoadBalanceMode(TaskManagerLoadBalanceMode.SLOTS) .buildAndStartWithDirectExec()) { final List> requestSlotFutures = new ArrayList<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategyTest.java index be46dfe6fd073..b4e61a89e8eda 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategyTest.java @@ -36,6 +36,7 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link DefaultResourceAllocationStrategy}. */ @@ -44,9 +45,10 @@ class DefaultResourceAllocationStrategyTest { ResourceProfile.fromResources(1, 100); private static final int NUM_OF_SLOTS = 5; private static final DefaultResourceAllocationStrategy ANY_MATCHING_STRATEGY = - createStrategy(false); + createStrategy(TaskManagerLoadBalanceMode.NONE); - private static final DefaultResourceAllocationStrategy EVENLY_STRATEGY = createStrategy(true); + private static final DefaultResourceAllocationStrategy EVENLY_STRATEGY = + createStrategy(TaskManagerLoadBalanceMode.SLOTS); @Test void testFulfillRequirementWithRegisteredResources() { @@ -694,20 +696,21 @@ void testMinRequiredResourceLimitInFulfillRequirements( .hasSize(pendingTaskManagersToAllocate); } - private static DefaultResourceAllocationStrategy createStrategy(boolean evenlySpreadOutSlots) { - return createStrategy(evenlySpreadOutSlots, 0); + private static DefaultResourceAllocationStrategy createStrategy( + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode) { + return createStrategy(taskManagerLoadBalanceMode, 0); } private static DefaultResourceAllocationStrategy createStrategy(int redundantTaskManagerNum) { - return createStrategy(false, redundantTaskManagerNum); + return createStrategy(TaskManagerLoadBalanceMode.NONE, redundantTaskManagerNum); } private static DefaultResourceAllocationStrategy createStrategy( - boolean evenlySpreadOutSlots, int redundantTaskManagerNum) { + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode, int redundantTaskManagerNum) { return new DefaultResourceAllocationStrategy( DEFAULT_SLOT_RESOURCE.multiply(NUM_OF_SLOTS), NUM_OF_SLOTS, - evenlySpreadOutSlots, + taskManagerLoadBalanceMode, Time.milliseconds(0), redundantTaskManagerNum, new CPUResource(0.0), @@ -719,7 +722,7 @@ private static DefaultResourceAllocationStrategy createStrategy( return new DefaultResourceAllocationStrategy( DEFAULT_SLOT_RESOURCE.multiply(NUM_OF_SLOTS), NUM_OF_SLOTS, - false, + TaskManagerLoadBalanceMode.NONE, Time.milliseconds(0), 0, minRequiredCPU, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerDefaultResourceAllocationStrategyITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerDefaultResourceAllocationStrategyITCase.java index f4e91a6ac70a6..d0eaeb5020da3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerDefaultResourceAllocationStrategyITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerDefaultResourceAllocationStrategyITCase.java @@ -55,7 +55,7 @@ protected Optional getResourceAllocationStrategy( new DefaultResourceAllocationStrategy( DEFAULT_TOTAL_RESOURCE_PROFILE, DEFAULT_NUM_SLOTS_PER_WORKER, - slotManagerConfiguration.isEvenlySpreadOutSlots(), + slotManagerConfiguration.getTaskManagerLoadBalanceMode(), slotManagerConfiguration.getTaskManagerTimeout(), slotManagerConfiguration.getRedundantTaskManagerNum(), slotManagerConfiguration.getMinTotalCpu(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfigurationBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfigurationBuilder.java index d5d70e164ebd8..260592d44dcf3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfigurationBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfigurationBuilder.java @@ -27,6 +27,8 @@ import java.time.Duration; +import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode; + /** Builder for {@link SlotManagerConfiguration}. */ public class SlotManagerConfigurationBuilder { private Time taskManagerRequestTimeout; @@ -43,7 +45,7 @@ public class SlotManagerConfigurationBuilder { private MemorySize minTotalMem; private MemorySize maxTotalMem; private int redundantTaskManagerNum; - private boolean evenlySpreadOutSlots; + private TaskManagerLoadBalanceMode taskManagerLoadBalanceMode; private SlotManagerConfigurationBuilder() { this.taskManagerRequestTimeout = TestingUtils.infiniteTime(); @@ -62,7 +64,7 @@ private SlotManagerConfigurationBuilder() { this.maxTotalMem = MemorySize.MAX_VALUE; this.redundantTaskManagerNum = ResourceManagerOptions.REDUNDANT_TASK_MANAGER_NUM.defaultValue(); - this.evenlySpreadOutSlots = false; + this.taskManagerLoadBalanceMode = TaskManagerLoadBalanceMode.NONE; } public static SlotManagerConfigurationBuilder newBuilder() { @@ -141,8 +143,9 @@ public SlotManagerConfigurationBuilder setRedundantTaskManagerNum(int redundantT return this; } - public SlotManagerConfigurationBuilder setEvenlySpreadOutSlots(boolean evenlySpreadOutSlots) { - this.evenlySpreadOutSlots = evenlySpreadOutSlots; + public SlotManagerConfigurationBuilder setTaskManagerLoadBalanceMode( + TaskManagerLoadBalanceMode taskManagerLoadBalanceMode) { + this.taskManagerLoadBalanceMode = taskManagerLoadBalanceMode; return this; } @@ -153,10 +156,10 @@ public SlotManagerConfiguration build() { requirementCheckDelay, declareNeededResourceDelay, waitResultConsumedBeforeRelease, - evenlySpreadOutSlots + taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS ? LeastUtilizationSlotMatchingStrategy.INSTANCE : AnyMatchingSlotMatchingStrategy.INSTANCE, - evenlySpreadOutSlots, + taskManagerLoadBalanceMode, defaultWorkerResourceSpec, numSlotsPerWorker, minSlotNum, From 008e1916e8bbeb18c1d06c74e2797da5a439cd47 Mon Sep 17 00:00:00 2001 From: Yu Chen Date: Tue, 7 Nov 2023 14:01:29 +0800 Subject: [PATCH 087/104] [FLINK-33474][runtime-web] fix undefined error of show plan in job submit page --- .../web-dashboard/src/app/pages/submit/submit.component.ts | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts index cfa74457a5455..8e14bcf82c1d4 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts @@ -78,7 +78,7 @@ export class SubmitComponent implements OnInit, OnDestroy { public validateForm: UntypedFormGroup; public planVisible = false; - @ViewChild(DagreComponent, { static: true }) private readonly dagreComponent: DagreComponent; + @ViewChild(DagreComponent) private readonly dagreComponent: DagreComponent; private readonly destroy$ = new Subject(); @@ -177,6 +177,7 @@ export class SubmitComponent implements OnInit, OnDestroy { ) .subscribe(data => { this.planVisible = true; + this.cdr.detectChanges(); this.dagreComponent.flush(data.nodes, data.links, true); }); } From e0240c782073f879a1a1e3fe22b45a38ee499c45 Mon Sep 17 00:00:00 2001 From: Zakelly Date: Mon, 6 Nov 2023 11:11:46 +0800 Subject: [PATCH 088/104] [FLINK-33060][state] Fix the javadoc of ListState interfaces about not allowing null value --- .../flink/api/common/state/AppendingState.java | 3 ++- .../apache/flink/api/common/state/ListState.java | 14 ++++++++++---- .../runtime/state/PartitionableListState.java | 10 ++++++++-- .../runtime/state/internal/InternalListState.java | 14 ++++++++++---- .../sorted/state/BatchExecutionKeyListState.java | 2 ++ 5 files changed, 32 insertions(+), 11 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java index d020c0b7dcf26..b081f86466c25 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java @@ -58,7 +58,8 @@ public interface AppendingState extends State { * of values. The next time {@link #get()} is called (for the same state partition) the returned * state will represent the updated list. * - *

If null is passed in, the state value will remain unchanged. + *

If null is passed in, the behaviour is undefined (implementation related). + * TODO: An unified behaviour across all sub-classes. * * @param value The new value for the state. * @throws Exception Thrown if the system cannot access the state. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java index 7508054f5c021..3ea5b16fb7f96 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java @@ -48,10 +48,13 @@ public interface ListState extends MergingState> { * given list of values. The next time {@link #get()} is called (for the same state partition) * the returned state will represent the updated list. * - *

If null or an empty list is passed in, the state value will be null. + *

If an empty list is passed in, the state value will be null. + * + *

Null value passed in or any null value in list is not allowed. * * @param values The new values for the state. - * @throws Exception The method may forward exception thrown internally (by I/O or functions). + * @throws Exception The method may forward exception thrown internally (by I/O or functions, or + * sanity check for null value). */ void update(List values) throws Exception; @@ -60,10 +63,13 @@ public interface ListState extends MergingState> { * existing list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. * - *

If null or an empty list is passed in, the state value remains unchanged. + *

If an empty list is passed in, the state value remains unchanged. + * + *

Null value passed in or any null value in list is not allowed. * * @param values The new values to be added to the state. - * @throws Exception The method may forward exception thrown internally (by I/O or functions). + * @throws Exception The method may forward exception thrown internally (by I/O or functions, or + * sanity check for null value). */ void addAll(List values) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java index 7cb363b91c332..35d6c78e7eff5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java @@ -29,6 +29,8 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Implementation of operator list state. * @@ -129,8 +131,12 @@ public void update(List values) { @Override public void addAll(List values) { - if (values != null && !values.isEmpty()) { - internalList.addAll(values); + Preconditions.checkNotNull(values, "List of values to add cannot be null."); + if (!values.isEmpty()) { + for (S value : values) { + checkNotNull(value, "Any value to add to a list cannot be null."); + add(value); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java index bfb75a00bb2f9..48096efa24eab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java @@ -39,10 +39,13 @@ public interface InternalListState * given list of values. The next time {@link #get()} is called (for the same state partition) * the returned state will represent the updated list. * - *

If `null` or an empty list is passed in, the state value will be null + *

If an empty list is passed in, the state value will be null + * + *

Null value passed in or any null value in list is not allowed. * * @param values The new values for the state. - * @throws Exception The method may forward exception thrown internally (by I/O or functions). + * @throws Exception The method may forward exception thrown internally (by I/O or functions, or + * sanity check for null value). */ void update(List values) throws Exception; @@ -51,10 +54,13 @@ public interface InternalListState * existing list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. * - *

If `null` or an empty list is passed in, the state value remains unchanged + *

If an empty list is passed in, the state value remains unchanged + * + *

Null value passed in or any null value in list is not allowed. * * @param values The new values to be added to the state. - * @throws Exception The method may forward exception thrown internally (by I/O or functions). + * @throws Exception The method may forward exception thrown internally (by I/O or functions, or + * sanity check for null value). */ void addAll(List values) throws Exception; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyListState.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyListState.java index 2fecbc2c50706..cdc17d68a2c85 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyListState.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionKeyListState.java @@ -47,6 +47,7 @@ public void update(List values) { checkNotNull(values); clear(); for (T value : values) { + checkNotNull(value); add(value); } } @@ -57,6 +58,7 @@ public void addAll(List values) { return; } for (T value : values) { + checkNotNull(value); add(value); } } From 5056b0d5dfec25ef90cd0f0e860816ade90d3869 Mon Sep 17 00:00:00 2001 From: Zakelly Date: Wed, 8 Nov 2023 12:43:26 +0800 Subject: [PATCH 089/104] fixup! [FLINK-33060][state] Fix the javadoc of ListState interfaces about not allowing null value --- .../org/apache/flink/api/common/state/AppendingState.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java index b081f86466c25..1f21427e99648 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java @@ -58,8 +58,8 @@ public interface AppendingState extends State { * of values. The next time {@link #get()} is called (for the same state partition) the returned * state will represent the updated list. * - *

If null is passed in, the behaviour is undefined (implementation related). - * TODO: An unified behaviour across all sub-classes. + *

If null is passed in, the behaviour is undefined (implementation related). TODO: An + * unified behaviour across all sub-classes. * * @param value The new value for the state. * @throws Exception Thrown if the system cannot access the state. From 648b6bac95232c8498b392ac01e5089777553c77 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 2 Nov 2023 16:17:51 +0100 Subject: [PATCH 090/104] [FLINK-33439] Implement type inference for IN function --- .../functions/BuiltInFunctionDefinitions.java | 3 +- .../strategies/ComparableTypeStrategy.java | 124 +--------------- .../SpecificInputTypeStrategies.java | 3 + .../strategies/SubQueryInputTypeStrategy.java | 119 +++++++++++++++ .../logical/utils/LogicalTypeChecks.java | 136 ++++++++++++++++++ .../SubQueryInputTypeStrategyTest.java | 128 +++++++++++++++++ .../PlannerExpressionConverter.scala | 4 - .../table/planner/expressions/subquery.scala | 78 ---------- .../ScalarFunctionsValidationTest.scala | 9 -- .../ScalarOperatorsValidationTest.scala | 6 - 10 files changed, 392 insertions(+), 218 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategy.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategyTest.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/subquery.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index 4f9953f45c9a6..b8012922df2e1 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -2242,7 +2242,8 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) BuiltInFunctionDefinition.newBuilder() .name("in") .kind(SCALAR) - .outputTypeStrategy(TypeStrategies.MISSING) + .inputTypeStrategy(SpecificInputTypeStrategies.IN) + .outputTypeStrategy(nullableIfArgs(explicit(DataTypes.BOOLEAN()))) .build(); public static final BuiltInFunctionDefinition CAST = diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ComparableTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ComparableTypeStrategy.java index 88aa6877c3e15..cb62543cce5a3 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ComparableTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ComparableTypeStrategy.java @@ -26,14 +26,9 @@ import org.apache.flink.table.types.inference.ConstantArgumentCount; import org.apache.flink.table.types.inference.InputTypeStrategy; import org.apache.flink.table.types.inference.Signature; -import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.LogicalTypeRoot; -import org.apache.flink.table.types.logical.RawType; -import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.table.types.logical.StructuredType.StructuredComparison; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.util.Preconditions; import java.util.Collections; @@ -49,7 +44,7 @@ * with itself (e.g. for aggregations). * *

For the rules which types are comparable with which types see {@link - * #areComparable(LogicalType, LogicalType)}. + * LogicalTypeChecks#areComparable(LogicalType, LogicalType, StructuredComparison)}. */ @Internal public final class ComparableTypeStrategy implements InputTypeStrategy { @@ -78,7 +73,7 @@ public Optional> inferInputTypes( final List argumentDataTypes = callContext.getArgumentDataTypes(); if (argumentDataTypes.size() == 1) { final LogicalType argType = argumentDataTypes.get(0).getLogicalType(); - if (!areComparable(argType, argType)) { + if (!LogicalTypeChecks.areComparable(argType, argType, requiredComparison)) { return callContext.fail( throwOnFailure, "Type '%s' should support %s comparison with itself.", @@ -90,7 +85,7 @@ public Optional> inferInputTypes( final LogicalType firstType = argumentDataTypes.get(i).getLogicalType(); final LogicalType secondType = argumentDataTypes.get(i + 1).getLogicalType(); - if (!areComparable(firstType, secondType)) { + if (!LogicalTypeChecks.areComparable(firstType, secondType, requiredComparison)) { return callContext.fail( throwOnFailure, "All types in a comparison should support %s comparison with each other. " @@ -111,120 +106,9 @@ private String comparisonToString() { : "both 'EQUALS' and 'ORDER'"; } - private boolean areComparable(LogicalType firstType, LogicalType secondType) { - return areComparableWithNormalizedNullability(firstType.copy(true), secondType.copy(true)); - } - - private boolean areComparableWithNormalizedNullability( - LogicalType firstType, LogicalType secondType) { - // A hack to support legacy types. To be removed when we drop the legacy types. - if (firstType instanceof LegacyTypeInformationType - || secondType instanceof LegacyTypeInformationType) { - return true; - } - - // everything is comparable with null, it should return null in that case - if (firstType.is(LogicalTypeRoot.NULL) || secondType.is(LogicalTypeRoot.NULL)) { - return true; - } - - if (firstType.getTypeRoot() == secondType.getTypeRoot()) { - return areTypesOfSameRootComparable(firstType, secondType); - } - - if (firstType.is(LogicalTypeFamily.NUMERIC) && secondType.is(LogicalTypeFamily.NUMERIC)) { - return true; - } - - // DATE + ALL TIMESTAMPS - if (firstType.is(LogicalTypeFamily.DATETIME) && secondType.is(LogicalTypeFamily.DATETIME)) { - return true; - } - - // VARCHAR + CHAR (we do not compare collations here) - if (firstType.is(LogicalTypeFamily.CHARACTER_STRING) - && secondType.is(LogicalTypeFamily.CHARACTER_STRING)) { - return true; - } - - // VARBINARY + BINARY - if (firstType.is(LogicalTypeFamily.BINARY_STRING) - && secondType.is(LogicalTypeFamily.BINARY_STRING)) { - return true; - } - - return false; - } - - private boolean areTypesOfSameRootComparable(LogicalType firstType, LogicalType secondType) { - switch (firstType.getTypeRoot()) { - case ARRAY: - case MULTISET: - case MAP: - case ROW: - return areConstructedTypesComparable(firstType, secondType); - case DISTINCT_TYPE: - return areDistinctTypesComparable(firstType, secondType); - case STRUCTURED_TYPE: - return areStructuredTypesComparable(firstType, secondType); - case RAW: - return areRawTypesComparable(firstType, secondType); - default: - return true; - } - } - - private boolean areRawTypesComparable(LogicalType firstType, LogicalType secondType) { - return firstType.equals(secondType) - && Comparable.class.isAssignableFrom( - ((RawType) firstType).getOriginatingClass()); - } - - private boolean areDistinctTypesComparable(LogicalType firstType, LogicalType secondType) { - DistinctType firstDistinctType = (DistinctType) firstType; - DistinctType secondDistinctType = (DistinctType) secondType; - return firstType.equals(secondType) - && areComparable( - firstDistinctType.getSourceType(), secondDistinctType.getSourceType()); - } - - private boolean areStructuredTypesComparable(LogicalType firstType, LogicalType secondType) { - return firstType.equals(secondType) && hasRequiredComparison((StructuredType) firstType); - } - - private boolean areConstructedTypesComparable(LogicalType firstType, LogicalType secondType) { - List firstChildren = firstType.getChildren(); - List secondChildren = secondType.getChildren(); - - if (firstChildren.size() != secondChildren.size()) { - return false; - } - - for (int i = 0; i < firstChildren.size(); i++) { - if (!areComparable(firstChildren.get(i), secondChildren.get(i))) { - return false; - } - } - - return true; - } - @Override public List getExpectedSignatures(FunctionDefinition definition) { return Collections.singletonList( Signature.of(Signature.Argument.ofGroupVarying("COMPARABLE"))); } - - private Boolean hasRequiredComparison(StructuredType structuredType) { - switch (requiredComparison) { - case EQUALS: - return structuredType.getComparison().isEquality(); - case FULL: - return structuredType.getComparison().isComparison(); - case NONE: - default: - // this is not important, required comparison will never be NONE - return true; - } - } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java index b004ce20e9dda..e0a07150d7869 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java @@ -122,6 +122,9 @@ public final class SpecificInputTypeStrategies { public static final InputTypeStrategy TWO_EQUALS_COMPARABLE = comparable(ConstantArgumentCount.of(2), StructuredType.StructuredComparison.EQUALS); + /** Type strategy specific for {@link BuiltInFunctionDefinitions#IN}. */ + public static final InputTypeStrategy IN = new SubQueryInputTypeStrategy(); + private SpecificInputTypeStrategies() { // no instantiation } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategy.java new file mode 100644 index 0000000000000..96ff9450d528a --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategy.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.types.inference.strategies; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.inference.ArgumentCount; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.ConstantArgumentCount; +import org.apache.flink.table.types.inference.InputTypeStrategy; +import org.apache.flink.table.types.inference.Signature; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.logical.utils.LogicalTypeMerging; +import org.apache.flink.table.types.utils.TypeConversions; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** {@link InputTypeStrategy} for {@link BuiltInFunctionDefinitions#IN}. */ +@Internal +public class SubQueryInputTypeStrategy implements InputTypeStrategy { + @Override + public ArgumentCount getArgumentCount() { + return ConstantArgumentCount.from(2); + } + + @Override + public Optional> inferInputTypes( + CallContext callContext, boolean throwOnFailure) { + final LogicalType rightType; + final DataType leftType = callContext.getArgumentDataTypes().get(0); + if (callContext.getArgumentDataTypes().size() > 2) { + final Optional commonType = + LogicalTypeMerging.findCommonType( + callContext.getArgumentDataTypes().stream() + .map(DataType::getLogicalType) + .collect(Collectors.toList())); + if (!commonType.isPresent()) { + return callContext.fail( + throwOnFailure, "Could not find a common type of the sublist."); + } + rightType = commonType.get(); + } else { + rightType = callContext.getArgumentDataTypes().get(1).getLogicalType(); + } + + // check if the types are comparable, if the types are not comparable, check if it is not + // a sub-query case like SELECT a IN (SELECT b FROM table1). We check if the result of the + // rightType is of a ROW type with a single column, and if that column is comparable with + // left type + if (!LogicalTypeChecks.areComparable( + leftType.getLogicalType(), + rightType, + StructuredType.StructuredComparison.EQUALS) + && !isComparableWithSubQuery(leftType.getLogicalType(), rightType)) { + return callContext.fail( + throwOnFailure, + "Types on the right side of IN operator (%s) are not comparable with %s.", + rightType, + leftType.getLogicalType()); + } + + return Optional.of( + Stream.concat( + Stream.of(leftType), + IntStream.range(1, callContext.getArgumentDataTypes().size()) + .mapToObj( + i -> + TypeConversions.fromLogicalToDataType( + rightType))) + .collect(Collectors.toList())); + } + + private static boolean isComparableWithSubQuery(LogicalType left, LogicalType right) { + if (right.is(LogicalTypeRoot.ROW) && right.getChildren().size() == 1) { + final RowType rowType = (RowType) right; + return LogicalTypeChecks.areComparable( + left, rowType.getTypeAt(0), StructuredType.StructuredComparison.EQUALS); + } + return false; + } + + @Override + public List getExpectedSignatures(FunctionDefinition definition) { + return Arrays.asList( + Signature.of( + Signature.Argument.ofGroup("COMPARABLE"), + Signature.Argument.ofGroupVarying("COMPARABLE")), + Signature.of( + Signature.Argument.ofGroup("COMPARABLE"), + Signature.Argument.ofGroup("SUBQUERY"))); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java index cb2fd4c966894..d6e31ea92da9c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java @@ -32,11 +32,14 @@ import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RawType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.table.types.logical.StructuredType.StructuredAttribute; +import org.apache.flink.table.types.logical.StructuredType.StructuredComparison; import org.apache.flink.table.types.logical.TimeType; import org.apache.flink.table.types.logical.TimestampKind; import org.apache.flink.table.types.logical.TimestampType; @@ -242,6 +245,139 @@ public static boolean hasWellDefinedString(LogicalType logicalType) { } } + public static boolean areComparable( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + return areComparableWithNormalizedNullability( + firstType.copy(true), secondType.copy(true), requiredComparison); + } + + private static boolean areComparableWithNormalizedNullability( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + // A hack to support legacy types. To be removed when we drop the legacy types. + if (firstType instanceof LegacyTypeInformationType + || secondType instanceof LegacyTypeInformationType) { + return true; + } + + // everything is comparable with null, it should return null in that case + if (firstType.is(LogicalTypeRoot.NULL) || secondType.is(LogicalTypeRoot.NULL)) { + return true; + } + + if (firstType.getTypeRoot() == secondType.getTypeRoot()) { + return areTypesOfSameRootComparable(firstType, secondType, requiredComparison); + } + + if (firstType.is(LogicalTypeFamily.NUMERIC) && secondType.is(LogicalTypeFamily.NUMERIC)) { + return true; + } + + // DATE + ALL TIMESTAMPS + if (firstType.is(LogicalTypeFamily.DATETIME) && secondType.is(LogicalTypeFamily.DATETIME)) { + return true; + } + + // VARCHAR + CHAR (we do not compare collations here) + if (firstType.is(LogicalTypeFamily.CHARACTER_STRING) + && secondType.is(LogicalTypeFamily.CHARACTER_STRING)) { + return true; + } + + // VARBINARY + BINARY + if (firstType.is(LogicalTypeFamily.BINARY_STRING) + && secondType.is(LogicalTypeFamily.BINARY_STRING)) { + return true; + } + + return false; + } + + private static boolean areTypesOfSameRootComparable( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + switch (firstType.getTypeRoot()) { + case ARRAY: + case MULTISET: + case MAP: + case ROW: + return areConstructedTypesComparable(firstType, secondType, requiredComparison); + case DISTINCT_TYPE: + return areDistinctTypesComparable(firstType, secondType, requiredComparison); + case STRUCTURED_TYPE: + return areStructuredTypesComparable(firstType, secondType, requiredComparison); + case RAW: + return areRawTypesComparable(firstType, secondType); + default: + return true; + } + } + + private static boolean areRawTypesComparable(LogicalType firstType, LogicalType secondType) { + return firstType.equals(secondType) + && Comparable.class.isAssignableFrom( + ((RawType) firstType).getOriginatingClass()); + } + + private static boolean areDistinctTypesComparable( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + DistinctType firstDistinctType = (DistinctType) firstType; + DistinctType secondDistinctType = (DistinctType) secondType; + return firstType.equals(secondType) + && areComparable( + firstDistinctType.getSourceType(), + secondDistinctType.getSourceType(), + requiredComparison); + } + + private static boolean areStructuredTypesComparable( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + return firstType.equals(secondType) + && hasRequiredComparison((StructuredType) firstType, requiredComparison); + } + + private static boolean areConstructedTypesComparable( + LogicalType firstType, + LogicalType secondType, + StructuredComparison requiredComparison) { + List firstChildren = firstType.getChildren(); + List secondChildren = secondType.getChildren(); + + if (firstChildren.size() != secondChildren.size()) { + return false; + } + + for (int i = 0; i < firstChildren.size(); i++) { + if (!areComparable(firstChildren.get(i), secondChildren.get(i), requiredComparison)) { + return false; + } + } + + return true; + } + + private static Boolean hasRequiredComparison( + StructuredType structuredType, StructuredComparison requiredComparison) { + switch (requiredComparison) { + case EQUALS: + return structuredType.getComparison().isEquality(); + case FULL: + return structuredType.getComparison().isComparison(); + case NONE: + default: + // this is not important, required comparison will never be NONE + return true; + } + } + private LogicalTypeChecks() { // no instantiation } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategyTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategyTest.java new file mode 100644 index 0000000000000..82ce646860b30 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/SubQueryInputTypeStrategyTest.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.table.types.inference.strategies; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.inference.InputTypeStrategiesTestBase; + +import java.util.stream.Stream; + +/** Tests for {@link SubQueryInputTypeStrategy}. */ +class SubQueryInputTypeStrategyTest extends InputTypeStrategiesTestBase { + + @Override + protected Stream testData() { + return Stream.of( + TestSpec.forStrategy("IN a set", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.SMALLINT(), + DataTypes.INT()) + .expectArgumentTypes( + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT()), + TestSpec.forStrategy("IN a set, binary", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.BYTES(), + DataTypes.BYTES(), + DataTypes.BYTES(), + DataTypes.BYTES()) + .expectArgumentTypes( + DataTypes.BYTES(), + DataTypes.BYTES(), + DataTypes.BYTES(), + DataTypes.BYTES()), + TestSpec.forStrategy("IN a set, string", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING()) + .expectArgumentTypes( + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING()), + TestSpec.forStrategy( + "IN a set, multiset(timestamp)", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP())) + .expectArgumentTypes( + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP()), + DataTypes.MULTISET(DataTypes.TIMESTAMP())), + TestSpec.forStrategy("IN a set, arrays", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.ARRAY(DataTypes.INT()), + DataTypes.ARRAY(DataTypes.SMALLINT())) + .expectArgumentTypes( + DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.ARRAY(DataTypes.BIGINT())), + TestSpec.forStrategy("IN a set of ROWs", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())), + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()))) + .expectArgumentTypes( + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())), + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()))), + TestSpec.forStrategy("IN a subquery", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.INT(), + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.BIGINT()))) + .expectArgumentTypes( + DataTypes.INT(), + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.BIGINT()))), + TestSpec.forStrategy("IN a set not comparable", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes(DataTypes.INT(), DataTypes.STRING()) + .expectErrorMessage( + "Types on the right side of IN operator (STRING) are not comparable with INT."), + TestSpec.forStrategy("IN a subquery not comparable", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.INT(), + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.STRING()))) + .expectErrorMessage( + "Types on the right side of IN operator (ROW<`f0` STRING>) are not comparable with INT"), + TestSpec.forStrategy("IN a subquery of ROWs", SpecificInputTypeStrategies.IN) + .calledWithArgumentTypes( + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())), + DataTypes.ROW( + DataTypes.FIELD( + "f0", + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.INT()))))) + .expectArgumentTypes( + DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())), + DataTypes.ROW( + DataTypes.FIELD( + "f0", + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.INT())))))); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 7996be7dd8784..cf95c97cf0db7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -127,10 +127,6 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp case fd: FunctionDefinition => fd match { - case IN => - assert(args.size > 1) - In(args.head, args.drop(1)) - case DISTINCT => assert(args.size == 1) DistinctAgg(args.head) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/subquery.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/subquery.scala deleted file mode 100644 index a5595024e8d14..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/subquery.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.table.planner.expressions - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.operations.QueryOperation -import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils._ -import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} -import org.apache.flink.table.types.utils.TypeConversions -import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo - -case class In(expression: PlannerExpression, elements: Seq[PlannerExpression]) - extends PlannerExpression { - - override def toString = s"$expression.in(${elements.mkString(", ")})" - - override private[flink] def children: Seq[PlannerExpression] = expression +: elements.distinct - - override private[flink] def validateInput(): ValidationResult = { - // check if this is a sub-query expression or an element list - elements.head match { - - case TableReference(name, tableOperation: QueryOperation) => - if (elements.length != 1) { - return ValidationFailure("IN operator supports only one table reference.") - } - val resolvedSchema = tableOperation.getResolvedSchema - if (resolvedSchema.getColumnCount > 1) { - return ValidationFailure( - s"The sub-query table '$name' must not have more than one column.") - } - ( - expression.resultType, - fromDataTypeToLegacyInfo(resolvedSchema.getColumnDataTypes.get(0))) match { - case (lType, rType) if lType == rType => ValidationSuccess - case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess - case (lType, rType) if isArray(lType) && lType.getTypeClass == rType.getTypeClass => - ValidationSuccess - case (lType, rType) => - ValidationFailure(s"IN operator on incompatible types: $lType and $rType.") - } - - case _ => - val types = children.tail.map(_.resultType) - if (types.distinct.length != 1) { - return ValidationFailure( - s"Types on the right side of the IN operator must be the same, " + - s"got ${types.mkString(", ")}.") - } - (children.head.resultType, children.last.resultType) match { - case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess - case (lType, rType) if lType == rType => ValidationSuccess - case (lType, rType) if isArray(lType) && lType.getTypeClass == rType.getTypeClass => - ValidationSuccess - case (lType, rType) => - ValidationFailure(s"IN operator on incompatible types: $lType and $rType.") - } - } - } - - override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO -} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala index 1ce90d8e9e989..4a604ae733c41 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala @@ -123,15 +123,6 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase { // Sub-query functions // ---------------------------------------------------------------------------------------------- - @Test - def testInValidationExceptionMoreThanOneTypes(): Unit = { - assertThatExceptionOfType(classOf[ValidationException]) - .isThrownBy(() => testTableApi('f2.in('f3, 'f8), "TRUE")) - - assertThatExceptionOfType(classOf[ValidationException]) - .isThrownBy(() => testTableApi('f2.in('f3, 'f4, 4), "FALSE")) - } - @Test def scalaInValidationExceptionDifferentOperandsTest(): Unit = { assertThatExceptionOfType(classOf[ValidationException]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala index f319924c035dd..189591c691daf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala @@ -47,12 +47,6 @@ class ScalarOperatorsValidationTest extends ScalarOperatorsTestBase { // Sub-query functions // ---------------------------------------------------------------------------------------------- - @Test - def testInMoreThanOneTypes(): Unit = { - assertThatExceptionOfType(classOf[ValidationException]) - .isThrownBy(() => testTableApi('f2.in('f3, 'f4, 4), "FAIL")) - } - @Test def testInDifferentOperands(): Unit = { assertThatExceptionOfType(classOf[ValidationException]) From 92951a05127f1e0e2ab0ea04ae022659fc5276ab Mon Sep 17 00:00:00 2001 From: pvary Date: Wed, 8 Nov 2023 17:55:43 +0100 Subject: [PATCH 091/104] [FLINK-33295] Separate SinkV2 and SinkV1Adapter tests Co-authored-by: Peter Vary --- .../base/sink/writer/TestSinkInitContext.java | 4 +- .../file/sink/writer/FileWriterTest.java | 8 +- .../groups/InternalSinkWriterMetricGroup.java | 16 +- .../metrics/groups/MetricsGroupTestUtils.java | 47 ++ .../api/datastream/DataStreamSinkTest.java | 8 +- .../api/functions/PrintSinkTest.java | 5 +- ...inkTransformationTranslatorITCaseBase.java | 225 +++++++++ ...SinkV1TransformationTranslatorITCase.java} | 190 +------- .../SinkV2TransformationTranslatorITCase.java | 100 ++++ ...st.java => CommitterOperatorTestBase.java} | 122 ++--- .../runtime/operators/sink/SinkTestUtil.java | 4 +- .../sink/SinkV2CommitterOperatorTest.java | 75 +++ .../sink/SinkV2SinkWriterOperatorTest.java | 149 ++++++ ...t.java => SinkWriterOperatorTestBase.java} | 212 ++++----- .../runtime/operators/sink/TestSink.java | 50 +- .../runtime/operators/sink/TestSinkV2.java | 434 ++++++++++++++++++ .../WithAdapterCommitterOperatorTest.java | 83 ++++ .../WithAdapterSinkWriterOperatorTest.java | 132 ++++++ .../test/streaming/runtime/SinkV2ITCase.java | 138 ++++++ .../runtime/SinkV2MetricsITCase.java | 183 ++++++++ 20 files changed, 1725 insertions(+), 460 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricsGroupTestUtils.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCaseBase.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/{SinkTransformationTranslatorITCase.java => SinkV1TransformationTranslatorITCase.java} (52%) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV2TransformationTranslatorITCase.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/{CommitterOperatorTest.java => CommitterOperatorTestBase.java} (77%) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2CommitterOperatorTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2SinkWriterOperatorTest.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/{SinkWriterOperatorTest.java => SinkWriterOperatorTestBase.java} (77%) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSinkV2.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterCommitterOperatorTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterSinkWriterOperatorTest.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2ITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2MetricsITCase.java diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java index 601d4f9d427b1..1f70b03413caa 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java @@ -28,7 +28,7 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; -import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.MetricsGroupTestUtils; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; @@ -51,7 +51,7 @@ public class TestSinkInitContext implements Sink.InitContext { private final OperatorIOMetricGroup operatorIOMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); private final SinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock( + MetricsGroupTestUtils.mockWriterMetricGroup( metricListener.getMetricGroup(), operatorIOMetricGroup); private final MailboxExecutor mailboxExecutor; diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java index 0149bf3e6da4d..cd0dda1d9781d 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java @@ -31,7 +31,7 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; -import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.MetricsGroupTestUtils; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner; import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig; @@ -292,7 +292,7 @@ void testNumberRecordsOutCounter(@TempDir java.nio.file.Path tempDir) final OperatorIOMetricGroup operatorIOMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); final SinkWriterMetricGroup sinkWriterMetricGroup = - InternalSinkWriterMetricGroup.mock( + MetricsGroupTestUtils.mockWriterMetricGroup( metricListener.getMetricGroup(), operatorIOMetricGroup); Counter recordsCounter = sinkWriterMetricGroup.getIOMetricGroup().getNumRecordsOutCounter(); @@ -471,7 +471,7 @@ private FileWriter createWriter( basePath, rollingPolicy, outputFileConfig, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + MetricsGroupTestUtils.mockWriterMetricGroup(metricListener.getMetricGroup())); } private FileWriter createWriter( @@ -484,7 +484,7 @@ private FileWriter createWriter( throws IOException { return new FileWriter<>( basePath, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + MetricsGroupTestUtils.mockWriterMetricGroup(metricListener.getMetricGroup()), bucketAssigner, new DefaultFileWriterBucketFactory<>(), new RowWiseBucketWriter<>( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java index 81aa8d78ce3e3..27d0c72ed5c7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java @@ -26,7 +26,6 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.metrics.MetricNames; /** Special {@link org.apache.flink.metrics.MetricGroup} representing an Operator. */ @@ -40,7 +39,8 @@ public class InternalSinkWriterMetricGroup extends ProxyMetricGroup private final Counter numBytesWritten; private final OperatorIOMetricGroup operatorIOMetricGroup; - private InternalSinkWriterMetricGroup( + @VisibleForTesting + InternalSinkWriterMetricGroup( MetricGroup parentMetricGroup, OperatorIOMetricGroup operatorIOMetricGroup) { super(parentMetricGroup); numRecordsOutErrors = parentMetricGroup.counter(MetricNames.NUM_RECORDS_OUT_ERRORS); @@ -61,18 +61,6 @@ public static InternalSinkWriterMetricGroup wrap(OperatorMetricGroup operatorMet operatorMetricGroup, operatorMetricGroup.getIOMetricGroup()); } - @VisibleForTesting - public static InternalSinkWriterMetricGroup mock(MetricGroup metricGroup) { - return new InternalSinkWriterMetricGroup( - metricGroup, UnregisteredMetricsGroup.createOperatorIOMetricGroup()); - } - - @VisibleForTesting - public static InternalSinkWriterMetricGroup mock( - MetricGroup metricGroup, OperatorIOMetricGroup operatorIOMetricGroup) { - return new InternalSinkWriterMetricGroup(metricGroup, operatorIOMetricGroup); - } - @Override public OperatorIOMetricGroup getIOMetricGroup() { return operatorIOMetricGroup; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricsGroupTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricsGroupTestUtils.java new file mode 100644 index 0000000000000..fae09a39e9b23 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricsGroupTestUtils.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.metrics.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; + +/** Util class to create metric groups for SinkV2 tests. */ +public class MetricsGroupTestUtils { + + @VisibleForTesting + public static InternalSinkWriterMetricGroup mockWriterMetricGroup() { + return new InternalSinkWriterMetricGroup( + new UnregisteredMetricsGroup(), + UnregisteredMetricsGroup.createOperatorIOMetricGroup()); + } + + @VisibleForTesting + public static InternalSinkWriterMetricGroup mockWriterMetricGroup(MetricGroup metricGroup) { + return new InternalSinkWriterMetricGroup( + metricGroup, UnregisteredMetricsGroup.createOperatorIOMetricGroup()); + } + + @VisibleForTesting + public static InternalSinkWriterMetricGroup mockWriterMetricGroup( + MetricGroup metricGroup, OperatorIOMetricGroup operatorIOMetricGroup) { + return new InternalSinkWriterMetricGroup(metricGroup, operatorIOMetricGroup); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/DataStreamSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/DataStreamSinkTest.java index fb2e9d4ccebf1..401ed55215c1b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/DataStreamSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/DataStreamSinkTest.java @@ -20,7 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.transformations.SinkTransformation; -import org.apache.flink.streaming.runtime.operators.sink.TestSink; +import org.apache.flink.streaming.runtime.operators.sink.TestSinkV2; import org.junit.Test; @@ -33,13 +33,15 @@ public class DataStreamSinkTest { public void testGettingTransformationWithNewSinkAPI() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final Transformation transformation = - env.fromElements(1, 2).sinkTo(TestSink.newBuilder().build()).getTransformation(); + env.fromElements(1, 2) + .sinkTo(TestSinkV2.newBuilder().build()) + .getTransformation(); assertTrue(transformation instanceof SinkTransformation); } @Test(expected = UnsupportedOperationException.class) public void throwExceptionWhenSetUidWithNewSinkAPI() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.fromElements(1, 2).sinkTo(TestSink.newBuilder().build()).setUidHash("Test"); + env.fromElements(1, 2).sinkTo(TestSinkV2.newBuilder().build()).setUidHash("Test"); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java index e8760d6d5de99..352118db5d473 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java @@ -26,8 +26,7 @@ import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.MetricsGroupTestUtils; import org.apache.flink.streaming.api.functions.sink.PrintSink; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.util.FlinkRuntimeException; @@ -200,7 +199,7 @@ public int getAttemptNumber() { @Override public SinkWriterMetricGroup metricGroup() { - return InternalSinkWriterMetricGroup.mock(new UnregisteredMetricsGroup()); + return MetricsGroupTestUtils.mockWriterMetricGroup(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCaseBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCaseBase.java new file mode 100644 index 0000000000000..3c93b178b518e --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCaseBase.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.graph; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperatorFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.function.Predicate; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Tests for {@link org.apache.flink.streaming.api.transformations.SinkTransformation}. + * + *

ATTENTION: This test is extremely brittle. Do NOT remove, add or re-order test cases. + */ +@RunWith(Parameterized.class) +public abstract class SinkTransformationTranslatorITCaseBase extends TestLogger { + + @Parameterized.Parameters(name = "Execution Mode: {0}") + public static Collection data() { + return Arrays.asList(RuntimeExecutionMode.STREAMING, RuntimeExecutionMode.BATCH); + } + + @Parameterized.Parameter() public RuntimeExecutionMode runtimeExecutionMode; + + static final String NAME = "FileSink"; + static final String SLOT_SHARE_GROUP = "FileGroup"; + static final String UID = "FileUid"; + static final int PARALLELISM = 2; + + abstract SinkT simpleSink(); + + abstract SinkT sinkWithCommitter(); + + abstract DataStreamSink sinkTo(DataStream stream, SinkT sink); + + @Test + public void generateWriterTopology() { + final StreamGraph streamGraph = buildGraph(simpleSink(), runtimeExecutionMode); + + final StreamNode sourceNode = findNodeName(streamGraph, node -> node.contains("Source")); + final StreamNode writerNode = findWriter(streamGraph); + + assertThat(streamGraph.getStreamNodes().size(), equalTo(2)); + + validateTopology( + sourceNode, + IntSerializer.class, + writerNode, + SinkWriterOperatorFactory.class, + PARALLELISM, + -1); + } + + @Test + public void generateWriterCommitterTopology() { + + final StreamGraph streamGraph = buildGraph(sinkWithCommitter(), runtimeExecutionMode); + + final StreamNode sourceNode = findNodeName(streamGraph, node -> node.contains("Source")); + final StreamNode writerNode = findWriter(streamGraph); + + validateTopology( + sourceNode, + IntSerializer.class, + writerNode, + SinkWriterOperatorFactory.class, + PARALLELISM, + -1); + + final StreamNode committerNode = + findNodeName(streamGraph, name -> name.contains("Committer")); + + assertThat(streamGraph.getStreamNodes().size(), equalTo(3)); + + validateTopology( + writerNode, + SimpleVersionedSerializerTypeSerializerProxy.class, + committerNode, + CommitterOperatorFactory.class, + PARALLELISM, + -1); + } + + StreamNode findWriter(StreamGraph streamGraph) { + return findNodeName( + streamGraph, name -> name.contains("Writer") && !name.contains("Committer")); + } + + StreamNode findCommitter(StreamGraph streamGraph) { + return findNodeName( + streamGraph, + name -> name.contains("Committer") && !name.contains("Global Committer")); + } + + StreamNode findGlobalCommitter(StreamGraph streamGraph) { + return findNodeName(streamGraph, name -> name.contains("Global Committer")); + } + + @Test(expected = IllegalStateException.class) + public void throwExceptionWithoutSettingUid() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Configuration config = new Configuration(); + config.set(ExecutionOptions.RUNTIME_MODE, runtimeExecutionMode); + env.configure(config, getClass().getClassLoader()); + // disable auto generating uid + env.getConfig().disableAutoGeneratedUIDs(); + sinkTo(env.fromElements(1, 2), simpleSink()); + env.getStreamGraph(); + } + + @Test + public void disableOperatorChain() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final DataStreamSource src = env.fromElements(1, 2); + final DataStreamSink dataStreamSink = sinkTo(src, sinkWithCommitter()).name(NAME); + dataStreamSink.disableChaining(); + + final StreamGraph streamGraph = env.getStreamGraph(); + final StreamNode writer = findWriter(streamGraph); + final StreamNode committer = findCommitter(streamGraph); + + assertThat(writer.getOperatorFactory().getChainingStrategy(), is(ChainingStrategy.NEVER)); + assertThat( + committer.getOperatorFactory().getChainingStrategy(), is(ChainingStrategy.NEVER)); + } + + void validateTopology( + StreamNode src, + Class srcOutTypeInfo, + StreamNode dest, + Class operatorFactoryClass, + int expectedParallelism, + int expectedMaxParallelism) { + + // verify src node + final StreamEdge srcOutEdge = src.getOutEdges().get(0); + assertThat(srcOutEdge.getTargetId(), equalTo(dest.getId())); + assertThat(src.getTypeSerializerOut(), instanceOf(srcOutTypeInfo)); + + // verify dest node input + final StreamEdge destInputEdge = dest.getInEdges().get(0); + assertThat(destInputEdge.getSourceId(), equalTo(src.getId())); + assertThat(dest.getTypeSerializersIn()[0], instanceOf(srcOutTypeInfo)); + + // make sure 2 sink operators have different names/uid + assertThat(dest.getOperatorName(), not(equalTo(src.getOperatorName()))); + assertThat(dest.getTransformationUID(), not(equalTo(src.getTransformationUID()))); + + assertThat(dest.getOperatorFactory(), instanceOf(operatorFactoryClass)); + assertThat(dest.getParallelism(), equalTo(expectedParallelism)); + assertThat(dest.getMaxParallelism(), equalTo(expectedMaxParallelism)); + assertThat(dest.getOperatorFactory().getChainingStrategy(), is(ChainingStrategy.ALWAYS)); + assertThat(dest.getSlotSharingGroup(), equalTo(SLOT_SHARE_GROUP)); + } + + StreamGraph buildGraph(SinkT sink, RuntimeExecutionMode runtimeExecutionMode) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Configuration config = new Configuration(); + config.set(ExecutionOptions.RUNTIME_MODE, runtimeExecutionMode); + env.configure(config, getClass().getClassLoader()); + final DataStreamSource src = env.fromElements(1, 2); + final DataStreamSink dataStreamSink = sinkTo(src.rebalance(), sink); + setSinkProperty(dataStreamSink); + // Trigger the plan generation but do not clear the transformations + env.getExecutionPlan(); + return env.getStreamGraph(); + } + + private void setSinkProperty(DataStreamSink dataStreamSink) { + dataStreamSink.name(NAME); + dataStreamSink.uid(UID); + dataStreamSink.setParallelism(SinkTransformationTranslatorITCaseBase.PARALLELISM); + dataStreamSink.slotSharingGroup(SLOT_SHARE_GROUP); + } + + StreamNode findNodeName(StreamGraph streamGraph, Predicate predicate) { + return streamGraph.getStreamNodes().stream() + .filter(node -> predicate.test(node.getOperatorName())) + .findFirst() + .orElseThrow(() -> new IllegalStateException("Can not find the node")); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV1TransformationTranslatorITCase.java similarity index 52% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCase.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV1TransformationTranslatorITCase.java index 4d0b2323c7824..1b5bd22142f24 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV1TransformationTranslatorITCase.java @@ -20,33 +20,23 @@ import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; import org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperatorFactory; import org.apache.flink.streaming.runtime.operators.sink.TestSink; -import org.apache.flink.util.TestLogger; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; -import java.util.Collection; -import java.util.function.Predicate; - import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -56,69 +46,22 @@ *

ATTENTION: This test is extremely brittle. Do NOT remove, add or re-order test cases. */ @RunWith(Parameterized.class) -public class SinkTransformationTranslatorITCase extends TestLogger { +public class SinkV1TransformationTranslatorITCase + extends SinkTransformationTranslatorITCaseBase> { - @Parameterized.Parameters(name = "Execution Mode: {0}") - public static Collection data() { - return Arrays.asList(RuntimeExecutionMode.STREAMING, RuntimeExecutionMode.BATCH); + @Override + Sink simpleSink() { + return TestSink.newBuilder().build(); } - @Parameterized.Parameter() public RuntimeExecutionMode runtimeExecutionMode; - - static final String NAME = "FileSink"; - static final String SLOT_SHARE_GROUP = "FileGroup"; - static final String UID = "FileUid"; - static final int PARALLELISM = 2; - - @Test - public void generateWriterTopology() { - final StreamGraph streamGraph = - buildGraph(TestSink.newBuilder().build(), runtimeExecutionMode); - - final StreamNode sourceNode = findNodeName(streamGraph, node -> node.contains("Source")); - final StreamNode writerNode = findWriter(streamGraph); - - assertThat(streamGraph.getStreamNodes().size(), equalTo(2)); - - validateTopology( - sourceNode, - IntSerializer.class, - writerNode, - SinkWriterOperatorFactory.class, - PARALLELISM, - -1); + @Override + Sink sinkWithCommitter() { + return TestSink.newBuilder().setDefaultCommitter().build(); } - @Test - public void generateWriterCommitterTopology() { - - final StreamGraph streamGraph = - buildGraph( - TestSink.newBuilder().setDefaultCommitter().build(), runtimeExecutionMode); - - final StreamNode sourceNode = findNodeName(streamGraph, node -> node.contains("Source")); - final StreamNode writerNode = findWriter(streamGraph); - - validateTopology( - sourceNode, - IntSerializer.class, - writerNode, - SinkWriterOperatorFactory.class, - PARALLELISM, - -1); - - final StreamNode committerNode = - findNodeName(streamGraph, name -> name.contains("Committer")); - - assertThat(streamGraph.getStreamNodes().size(), equalTo(3)); - - validateTopology( - writerNode, - SimpleVersionedSerializerTypeSerializerProxy.class, - committerNode, - CommitterOperatorFactory.class, - PARALLELISM, - -1); + @Override + DataStreamSink sinkTo(DataStream stream, Sink sink) { + return stream.sinkTo(sink); } @Test @@ -219,56 +162,6 @@ public void generateWriterGlobalCommitterTopology() { 1); } - private StreamNode findWriter(StreamGraph streamGraph) { - return findNodeName( - streamGraph, name -> name.contains("Writer") && !name.contains("Committer")); - } - - private StreamNode findCommitter(StreamGraph streamGraph) { - return findNodeName(streamGraph, name -> name.contains("Committer")); - } - - private StreamNode findGlobalCommitter(StreamGraph streamGraph) { - return findNodeName(streamGraph, name -> name.contains("Global Committer")); - } - - @Test(expected = IllegalStateException.class) - public void throwExceptionWithoutSettingUid() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - final Configuration config = new Configuration(); - config.set(ExecutionOptions.RUNTIME_MODE, runtimeExecutionMode); - env.configure(config, getClass().getClassLoader()); - // disable auto generating uid - env.getConfig().disableAutoGeneratedUIDs(); - env.fromElements(1, 2).sinkTo(TestSink.newBuilder().build()); - env.getStreamGraph(); - } - - @Test - public void disableOperatorChain() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - final DataStreamSource src = env.fromElements(1, 2); - final DataStreamSink dataStreamSink = - src.sinkTo( - TestSink.newBuilder() - .setDefaultCommitter() - .setDefaultGlobalCommitter() - .build()) - .name(NAME); - dataStreamSink.disableChaining(); - - final StreamGraph streamGraph = env.getStreamGraph(); - final StreamNode writer = findWriter(streamGraph); - final StreamNode globalCommitter = findCommitter(streamGraph); - - assertThat(writer.getOperatorFactory().getChainingStrategy(), is(ChainingStrategy.NEVER)); - assertThat( - globalCommitter.getOperatorFactory().getChainingStrategy(), - is(ChainingStrategy.NEVER)); - } - @Test public void testSettingOperatorUidHash() { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -319,61 +212,4 @@ public void testSettingOperatorUids() { findGlobalCommitter(streamGraph).getTransformationUID(), String.format("Sink %s Global Committer", sinkUid)); } - - private void validateTopology( - StreamNode src, - Class srcOutTypeInfo, - StreamNode dest, - Class operatorFactoryClass, - int expectedParallelism, - int expectedMaxParallelism) { - - // verify src node - final StreamEdge srcOutEdge = src.getOutEdges().get(0); - assertThat(srcOutEdge.getTargetId(), equalTo(dest.getId())); - assertThat(src.getTypeSerializerOut(), instanceOf(srcOutTypeInfo)); - - // verify dest node input - final StreamEdge destInputEdge = dest.getInEdges().get(0); - assertThat(destInputEdge.getSourceId(), equalTo(src.getId())); - assertThat(dest.getTypeSerializersIn()[0], instanceOf(srcOutTypeInfo)); - - // make sure 2 sink operators have different names/uid - assertThat(dest.getOperatorName(), not(equalTo(src.getOperatorName()))); - assertThat(dest.getTransformationUID(), not(equalTo(src.getTransformationUID()))); - - assertThat(dest.getOperatorFactory(), instanceOf(operatorFactoryClass)); - assertThat(dest.getParallelism(), equalTo(expectedParallelism)); - assertThat(dest.getMaxParallelism(), equalTo(expectedMaxParallelism)); - assertThat(dest.getOperatorFactory().getChainingStrategy(), is(ChainingStrategy.ALWAYS)); - assertThat(dest.getSlotSharingGroup(), equalTo(SLOT_SHARE_GROUP)); - } - - private StreamGraph buildGraph(TestSink sink, RuntimeExecutionMode runtimeExecutionMode) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - final Configuration config = new Configuration(); - config.set(ExecutionOptions.RUNTIME_MODE, runtimeExecutionMode); - env.configure(config, getClass().getClassLoader()); - final DataStreamSource src = env.fromElements(1, 2); - final DataStreamSink dataStreamSink = src.rebalance().sinkTo(sink); - setSinkProperty(dataStreamSink); - // Trigger the plan generation but do not clear the transformations - env.getExecutionPlan(); - return env.getStreamGraph(); - } - - private void setSinkProperty(DataStreamSink dataStreamSink) { - dataStreamSink.name(NAME); - dataStreamSink.uid(UID); - dataStreamSink.setParallelism(SinkTransformationTranslatorITCase.PARALLELISM); - dataStreamSink.slotSharingGroup(SLOT_SHARE_GROUP); - } - - private StreamNode findNodeName(StreamGraph streamGraph, Predicate predicate) { - return streamGraph.getStreamNodes().stream() - .filter(node -> predicate.test(node.getOperatorName())) - .findFirst() - .orElseThrow(() -> new IllegalStateException("Can not find the node")); - } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV2TransformationTranslatorITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV2TransformationTranslatorITCase.java new file mode 100644 index 0000000000000..612cb9d780816 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkV2TransformationTranslatorITCase.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.graph; + +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.runtime.operators.sink.TestSinkV2; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link org.apache.flink.streaming.api.transformations.SinkTransformation}. + * + *

ATTENTION: This test is extremely brittle. Do NOT remove, add or re-order test cases. + */ +@RunWith(Parameterized.class) +public class SinkV2TransformationTranslatorITCase + extends SinkTransformationTranslatorITCaseBase> { + + @Override + Sink simpleSink() { + return TestSinkV2.newBuilder().build(); + } + + @Override + Sink sinkWithCommitter() { + return TestSinkV2.newBuilder().setDefaultCommitter().build(); + } + + @Override + DataStreamSink sinkTo(DataStream stream, Sink sink) { + return stream.sinkTo(sink); + } + + @Test + public void testSettingOperatorUidHash() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + final String writerHash = "f6b178ce445dc3ffaa06bad27a51fead"; + final String committerHash = "68ac8ae79eae4e3135a54f9689c4aa10"; + final CustomSinkOperatorUidHashes operatorsUidHashes = + CustomSinkOperatorUidHashes.builder() + .setWriterUidHash(writerHash) + .setCommitterUidHash(committerHash) + .build(); + src.sinkTo( + TestSinkV2.newBuilder().setDefaultCommitter().build(), + operatorsUidHashes) + .name(NAME); + + final StreamGraph streamGraph = env.getStreamGraph(); + + assertEquals(findWriter(streamGraph).getUserHash(), writerHash); + assertEquals(findCommitter(streamGraph).getUserHash(), committerHash); + } + + /** + * When ever you need to change something in this test case please think about possible state + * upgrade problems introduced by your changes. + */ + @Test + public void testSettingOperatorUids() { + final String sinkUid = "f6b178ce445dc3ffaa06bad27a51fead"; + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + src.sinkTo(TestSinkV2.newBuilder().setDefaultCommitter().build()) + .name(NAME) + .uid(sinkUid); + + final StreamGraph streamGraph = env.getStreamGraph(); + assertEquals(findWriter(streamGraph).getTransformationUID(), sinkUid); + assertEquals( + findCommitter(streamGraph).getTransformationUID(), + String.format("Sink Committer: %s", sinkUid)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTestBase.java similarity index 77% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTestBase.java index 7ae860c54e0c4..fe3625ec8ce17 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorTestBase.java @@ -18,8 +18,6 @@ package org.apache.flink.streaming.runtime.operators.sink; -import org.apache.flink.api.connector.sink.Committer; -import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; @@ -34,45 +32,31 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Collections; import java.util.List; +import java.util.function.IntSupplier; import static org.apache.flink.streaming.runtime.operators.sink.SinkTestUtil.fromOutput; import static org.apache.flink.streaming.runtime.operators.sink.SinkTestUtil.toCommittableSummary; import static org.apache.flink.streaming.runtime.operators.sink.SinkTestUtil.toCommittableWithLinage; import static org.assertj.core.api.Assertions.assertThat; -class CommitterOperatorTest { +abstract class CommitterOperatorTestBase { @ParameterizedTest @ValueSource(booleans = {true, false}) void testEmitCommittables(boolean withPostCommitTopology) throws Exception { - final ForwardingCommitter committer = new ForwardingCommitter(); - - Sink sink; + SinkAndCounters sinkAndCounters; if (withPostCommitTopology) { // Insert global committer to simulate post commit topology - sink = - TestSink.newBuilder() - .setCommitter(committer) - .setDefaultGlobalCommitter() - .setCommittableSerializer(TestSink.StringCommittableSerializer.INSTANCE) - .build() - .asV2(); + sinkAndCounters = sinkWithPostCommit(); } else { - sink = - TestSink.newBuilder() - .setCommitter(committer) - .setCommittableSerializer(TestSink.StringCommittableSerializer.INSTANCE) - .build() - .asV2(); + sinkAndCounters = sinkWithoutPostCommit(); } final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> testHarness = new OneInputStreamOperatorTestHarness<>( - new CommitterOperatorFactory<>( - (TwoPhaseCommittingSink) sink, false, true)); + new CommitterOperatorFactory<>(sinkAndCounters.sink, false, true)); testHarness.open(); final CommittableSummary committableSummary = @@ -85,7 +69,7 @@ void testEmitCommittables(boolean withPostCommitTopology) throws Exception { // Trigger commit testHarness.notifyOfCompletedCheckpoint(1); - assertThat(committer.getSuccessfulCommits()).isEqualTo(1); + assertThat(sinkAndCounters.commitCounter.getAsInt()).isEqualTo(1); if (withPostCommitTopology) { final List output = fromOutput(testHarness.getOutput()); SinkV2Assertions.assertThat(toCommittableSummary(output.get(0))) @@ -102,10 +86,10 @@ void testEmitCommittables(boolean withPostCommitTopology) throws Exception { @Test void testWaitForCommittablesOfLatestCheckpointBeforeCommitting() throws Exception { - final ForwardingCommitter committer = new ForwardingCommitter(); + SinkAndCounters sinkAndCounters = sinkWithPostCommit(); final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> - testHarness = createTestHarness(committer, false, true); + testHarness = createTestHarness(sinkAndCounters.sink, false, true); testHarness.open(); testHarness.setProcessingTime(0); @@ -119,7 +103,7 @@ void testWaitForCommittablesOfLatestCheckpointBeforeCommitting() throws Exceptio testHarness.notifyOfCompletedCheckpoint(1); assertThat(testHarness.getOutput()).isEmpty(); - assertThat(committer.getSuccessfulCommits()).isEqualTo(0); + assertThat(sinkAndCounters.commitCounter.getAsInt()).isZero(); final CommittableWithLineage second = new CommittableWithLineage<>("2", 1L, 1); testHarness.processElement(new StreamRecord<>(second)); @@ -129,7 +113,7 @@ void testWaitForCommittablesOfLatestCheckpointBeforeCommitting() throws Exceptio final List output = fromOutput(testHarness.getOutput()); assertThat(output).hasSize(3); - assertThat(committer.getSuccessfulCommits()).isEqualTo(2); + assertThat(sinkAndCounters.commitCounter.getAsInt()).isEqualTo(2); SinkV2Assertions.assertThat(toCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) .hasOverallCommittables(committableSummary.getNumberOfCommittables()) @@ -143,10 +127,11 @@ void testWaitForCommittablesOfLatestCheckpointBeforeCommitting() throws Exceptio @Test void testImmediatelyCommitLateCommittables() throws Exception { - final ForwardingCommitter committer = new ForwardingCommitter(); + SinkAndCounters sinkAndCounters = sinkWithPostCommit(); + final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> - testHarness = createTestHarness(committer, false, true); + testHarness = createTestHarness(sinkAndCounters.sink, false, true); testHarness.open(); final CommittableSummary committableSummary = @@ -166,7 +151,7 @@ void testImmediatelyCommitLateCommittables() throws Exception { final List output = fromOutput(testHarness.getOutput()); assertThat(output).hasSize(2); - assertThat(committer.getSuccessfulCommits()).isEqualTo(1); + assertThat(sinkAndCounters.commitCounter.getAsInt()).isEqualTo(1); SinkV2Assertions.assertThat(toCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) .hasOverallCommittables(committableSummary.getNumberOfCommittables()) @@ -179,10 +164,10 @@ void testImmediatelyCommitLateCommittables() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testEmitAllCommittablesOnEndOfInput(boolean isBatchMode) throws Exception { - final ForwardingCommitter committer = new ForwardingCommitter(); + SinkAndCounters sinkAndCounters = sinkWithPostCommit(); final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> - testHarness = createTestHarness(committer, isBatchMode, !isBatchMode); + testHarness = createTestHarness(sinkAndCounters.sink, isBatchMode, !isBatchMode); testHarness.open(); final CommittableSummary committableSummary = @@ -199,7 +184,7 @@ void testEmitAllCommittablesOnEndOfInput(boolean isBatchMode) throws Exception { testHarness.endInput(); if (!isBatchMode) { - assertThat(testHarness.getOutput()).hasSize(0); + assertThat(testHarness.getOutput()).isEmpty(); // notify final checkpoint complete testHarness.notifyOfCompletedCheckpoint(1); } @@ -227,7 +212,7 @@ void testStateRestore() throws Exception { CommittableMessage, CommittableMessage> testHarness = createTestHarness( - new TestSink.RetryOnceCommitter(), + sinkWithPostCommitWithRetry().sink, false, true, 1, @@ -262,15 +247,15 @@ void testStateRestore() throws Exception { assertThat(testHarness.getOutput()).isEmpty(); testHarness.close(); - final ForwardingCommitter committer = new ForwardingCommitter(); - // create new testHarness but with different parallelism level and subtaskId that original // one. // we will make sure that new subtaskId was used during committable recovery. + SinkAndCounters sinkAndCounters = sinkWithPostCommit(); final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> restored = - createTestHarness(committer, false, true, 10, 10, subtaskIdAfterRecovery); + createTestHarness( + sinkAndCounters.sink, false, true, 10, 10, subtaskIdAfterRecovery); restored.initializeState(snapshot); restored.open(); @@ -278,7 +263,7 @@ void testStateRestore() throws Exception { // Previous committables are immediately committed if possible final List output = fromOutput(restored.getOutput()); assertThat(output).hasSize(3); - assertThat(committer.getSuccessfulCommits()).isEqualTo(2); + assertThat(sinkAndCounters.commitCounter.getAsInt()).isEqualTo(2); SinkV2Assertions.assertThat(toCommittableSummary(output.get(0))) .hasCheckpointId(checkpointId) .hasFailedCommittables(0) @@ -300,22 +285,14 @@ void testStateRestore() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testHandleEndInputInStreamingMode(boolean isCheckpointingEnabled) throws Exception { - final Sink sink = - TestSink.newBuilder() - .setDefaultCommitter() - .setDefaultGlobalCommitter() - .setCommittableSerializer(TestSink.StringCommittableSerializer.INSTANCE) - .build() - .asV2(); + final SinkAndCounters sinkAndCounters = sinkWithPostCommit(); final OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> testHarness = new OneInputStreamOperatorTestHarness<>( new CommitterOperatorFactory<>( - (TwoPhaseCommittingSink) sink, - false, - isCheckpointingEnabled)); + sinkAndCounters.sink, false, isCheckpointingEnabled)); testHarness.open(); final CommittableSummary committableSummary = @@ -363,28 +340,18 @@ CommittableWithLineage copyCommittableWithDifferentOrigin( private OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> createTestHarness( - Committer committer, + TwoPhaseCommittingSink sink, boolean isBatchMode, boolean isCheckpointingEnabled) throws Exception { return new OneInputStreamOperatorTestHarness<>( - new CommitterOperatorFactory<>( - (TwoPhaseCommittingSink) - TestSink.newBuilder() - .setCommitter(committer) - .setDefaultGlobalCommitter() - .setCommittableSerializer( - TestSink.StringCommittableSerializer.INSTANCE) - .build() - .asV2(), - isBatchMode, - isCheckpointingEnabled)); + new CommitterOperatorFactory<>(sink, isBatchMode, isCheckpointingEnabled)); } private OneInputStreamOperatorTestHarness< CommittableMessage, CommittableMessage> createTestHarness( - Committer committer, + TwoPhaseCommittingSink sink, boolean isBatchMode, boolean isCheckpointingEnabled, int maxParallelism, @@ -392,36 +359,25 @@ CommittableWithLineage copyCommittableWithDifferentOrigin( int subtaskId) throws Exception { return new OneInputStreamOperatorTestHarness<>( - new CommitterOperatorFactory<>( - (TwoPhaseCommittingSink) - TestSink.newBuilder() - .setCommitter(committer) - .setDefaultGlobalCommitter() - .setCommittableSerializer( - TestSink.StringCommittableSerializer.INSTANCE) - .build() - .asV2(), - isBatchMode, - isCheckpointingEnabled), + new CommitterOperatorFactory<>(sink, isBatchMode, isCheckpointingEnabled), maxParallelism, parallelism, subtaskId); } - private static class ForwardingCommitter extends TestSink.DefaultCommitter { - private int successfulCommits = 0; + abstract SinkAndCounters sinkWithPostCommit(); - @Override - public List commit(List committables) { - successfulCommits += committables.size(); - return Collections.emptyList(); - } + abstract SinkAndCounters sinkWithPostCommitWithRetry(); + + abstract SinkAndCounters sinkWithoutPostCommit(); - @Override - public void close() throws Exception {} + static class SinkAndCounters { + TwoPhaseCommittingSink sink; + IntSupplier commitCounter; - public int getSuccessfulCommits() { - return successfulCommits; + public SinkAndCounters(TwoPhaseCommittingSink sink, IntSupplier commitCounter) { + this.sink = sink; + this.commitCounter = commitCounter; } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkTestUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkTestUtil.java index c309835cc7591..b457b5a0b7825 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkTestUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkTestUtil.java @@ -52,7 +52,7 @@ static List toBytes(Collection elements) { static byte[] toBytes(String obj) { try { return SimpleVersionedSerialization.writeVersionAndSerialize( - TestSink.StringCommittableSerializer.INSTANCE, obj); + TestSinkV2.StringSerializer.INSTANCE, obj); } catch (IOException e) { throw new IllegalStateException(e); } @@ -83,7 +83,7 @@ static String fromRecord(StreamRecord obj) { static String fromBytes(byte[] obj) { try { return SimpleVersionedSerialization.readVersionAndDeSerialize( - TestSink.StringCommittableSerializer.INSTANCE, obj); + TestSinkV2.StringSerializer.INSTANCE, obj); } catch (IOException e) { throw new IllegalStateException(e); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2CommitterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2CommitterOperatorTest.java new file mode 100644 index 0000000000000..ed8e53ff342ce --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2CommitterOperatorTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.operators.sink; + +import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; + +import java.util.Collection; + +class SinkV2CommitterOperatorTest extends CommitterOperatorTestBase { + @Override + SinkAndCounters sinkWithPostCommit() { + ForwardingCommitter committer = new ForwardingCommitter(); + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSinkV2.newBuilder() + .setCommitter(committer) + .setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE) + .setWithPostCommitTopology(true) + .build(), + () -> committer.successfulCommits); + } + + @Override + SinkAndCounters sinkWithPostCommitWithRetry() { + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSinkV2.newBuilder() + .setCommitter(new TestSinkV2.RetryOnceCommitter()) + .setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE) + .setWithPostCommitTopology(true) + .build(), + () -> 0); + } + + @Override + SinkAndCounters sinkWithoutPostCommit() { + ForwardingCommitter committer = new ForwardingCommitter(); + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSinkV2.newBuilder() + .setCommitter(committer) + .setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE) + .setWithPostCommitTopology(false) + .build(), + () -> committer.successfulCommits); + } + + private static class ForwardingCommitter extends TestSinkV2.DefaultCommitter { + private int successfulCommits = 0; + + @Override + public void commit(Collection> committables) { + successfulCommits += committables.size(); + } + + @Override + public void close() throws Exception {} + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2SinkWriterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2SinkWriterOperatorTest.java new file mode 100644 index 0000000000000..5bb0135d69b4a --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkV2SinkWriterOperatorTest.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.streaming.runtime.operators.sink; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.java.tuple.Tuple3; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +class SinkV2SinkWriterOperatorTest extends SinkWriterOperatorTestBase { + + @Override + SinkAndSuppliers sinkWithoutCommitter() { + TestSinkV2.DefaultSinkWriter sinkWriter = new TestSinkV2.DefaultSinkWriter<>(); + return new SinkAndSuppliers( + TestSinkV2.newBuilder().setWriter(sinkWriter).build(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + TestSinkV2.StringSerializer::new); + } + + @Override + SinkAndSuppliers sinkWithCommitter() { + TestSinkV2.DefaultSinkWriter sinkWriter = + new TestSinkV2.DefaultCommittingSinkWriter<>(); + return new SinkAndSuppliers( + TestSinkV2.newBuilder() + .setWriter(sinkWriter) + .setDefaultCommitter() + .build(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + TestSinkV2.StringSerializer::new); + } + + @Override + SinkAndSuppliers sinkWithTimeBasedWriter() { + TestSinkV2.DefaultSinkWriter sinkWriter = new TimeBasedBufferingSinkWriter(); + return new SinkAndSuppliers( + TestSinkV2.newBuilder() + .setWriter(sinkWriter) + .setDefaultCommitter() + .build(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + TestSinkV2.StringSerializer::new); + } + + @Override + SinkAndSuppliers sinkWithSnapshottingWriter(boolean withState, String stateName) { + SnapshottingBufferingSinkWriter sinkWriter = new SnapshottingBufferingSinkWriter(); + TestSinkV2.Builder builder = + TestSinkV2.newBuilder() + .setWriter(sinkWriter) + .setDefaultCommitter() + .setWithPostCommitTopology(true); + if (withState) { + builder.setWriterState(true); + } + if (stateName != null) { + builder.setCompatibleStateNames(stateName); + } + return new SinkAndSuppliers( + builder.build(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> sinkWriter.lastCheckpointId, + () -> new TestSinkV2.StringSerializer()); + } + + private static class TimeBasedBufferingSinkWriter + extends TestSinkV2.DefaultCommittingSinkWriter + implements ProcessingTimeService.ProcessingTimeCallback { + + private final List cachedCommittables = new ArrayList<>(); + private ProcessingTimeService processingTimeService; + + @Override + public void write(Integer element, Context context) { + cachedCommittables.add( + Tuple3.of(element, context.timestamp(), context.currentWatermark()).toString()); + } + + @Override + public void onProcessingTime(long time) { + elements.addAll(cachedCommittables); + cachedCommittables.clear(); + this.processingTimeService.registerTimer(time + 1000, this); + } + + @Override + public void init(org.apache.flink.api.connector.sink2.Sink.InitContext context) { + this.processingTimeService = context.getProcessingTimeService(); + this.processingTimeService.registerTimer(1000, this); + } + } + + private static class SnapshottingBufferingSinkWriter + extends TestSinkV2.DefaultStatefulSinkWriter { + public static final int NOT_SNAPSHOTTED = -1; + long lastCheckpointId = NOT_SNAPSHOTTED; + boolean endOfInput = false; + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + this.endOfInput = endOfInput; + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + lastCheckpointId = checkpointId; + return super.snapshotState(checkpointId); + } + + @Override + public Collection prepareCommit() { + if (!endOfInput) { + return ImmutableList.of(); + } + List result = elements; + elements = new ArrayList<>(); + return result; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTestBase.java similarity index 77% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTestBase.java index 86cc85c3155c7..464b08c8f67b0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperatorTestBase.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; -import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.core.io.SimpleVersionedSerialization; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -62,29 +62,31 @@ import java.util.List; import java.util.Queue; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.LongSupplier; +import java.util.function.Supplier; import static org.apache.flink.streaming.runtime.operators.sink.SinkTestUtil.fromOutput; import static org.assertj.core.api.Assertions.assertThat; -class SinkWriterOperatorTest { +abstract class SinkWriterOperatorTestBase { @Test void testNotEmitCommittablesWithoutCommitter() throws Exception { - final TestSink.DefaultSinkWriter sinkWriter = new TestSink.DefaultSinkWriter<>(); + SinkAndSuppliers sinkAndSuppliers = sinkWithoutCommitter(); final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder().setWriter(sinkWriter).build().asV2())); + new SinkWriterOperatorFactory<>(sinkAndSuppliers.sink)); testHarness.open(); testHarness.processElement(1, 1); assertThat(testHarness.getOutput()).isEmpty(); - assertThat(sinkWriter.elements).containsOnly("(1,1," + Long.MIN_VALUE + ")"); + assertThat(sinkAndSuppliers.elementSupplier.get()) + .containsOnly("(1,1," + Long.MIN_VALUE + ")"); testHarness.prepareSnapshotPreBarrier(1); assertThat(testHarness.getOutput()).isEmpty(); // Elements are flushed - assertThat(sinkWriter.elements).isEmpty(); + assertThat(sinkAndSuppliers.elementSupplier.get()).isEmpty(); testHarness.close(); } @@ -92,11 +94,10 @@ void testNotEmitCommittablesWithoutCommitter() throws Exception { void testWatermarkPropagatedToSinkWriter() throws Exception { final long initialTime = 0; - final TestSink.DefaultSinkWriter writer = new TestSink.DefaultSinkWriter<>(); + SinkAndSuppliers sinkAndSuppliers = sinkWithoutCommitter(); final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder().setWriter(writer).build().asV2())); + new SinkWriterOperatorFactory<>(sinkAndSuppliers.sink)); testHarness.open(); testHarness.processWatermark(initialTime); @@ -104,7 +105,7 @@ void testWatermarkPropagatedToSinkWriter() throws Exception { assertThat(testHarness.getOutput()) .containsExactly(new Watermark(initialTime), new Watermark(initialTime + 1)); - assertThat(writer.watermarks) + assertThat(sinkAndSuppliers.watermarkSupplier.get()) .containsExactly( new org.apache.flink.api.common.eventtime.Watermark(initialTime), new org.apache.flink.api.common.eventtime.Watermark(initialTime + 1)); @@ -117,12 +118,7 @@ public void testTimeBasedBufferingSinkWriter() throws Exception { final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder() - .setDefaultCommitter() - .setWriter(new TimeBasedBufferingSinkWriter()) - .build() - .asV2())); + new SinkWriterOperatorFactory<>(sinkWithTimeBasedWriter().sink)); testHarness.open(); @@ -149,8 +145,7 @@ public void testTimeBasedBufferingSinkWriter() throws Exception { void testEmitOnFlushWithCommitter() throws Exception { final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder().setDefaultCommitter().build().asV2())); + new SinkWriterOperatorFactory<>(sinkWithCommitter().sink)); testHarness.open(); assertThat(testHarness.getOutput()).isEmpty(); @@ -168,8 +163,7 @@ void testEmitOnFlushWithCommitter() throws Exception { @Test void testEmitOnEndOfInputInBatchMode() throws Exception { final SinkWriterOperatorFactory writerOperatorFactory = - new SinkWriterOperatorFactory<>( - TestSink.newBuilder().setDefaultCommitter().build().asV2()); + new SinkWriterOperatorFactory<>(sinkWithCommitter().sink); final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>(writerOperatorFactory); @@ -187,10 +181,9 @@ void testStateRestore(boolean stateful) throws Exception { final long initialTime = 0; - final SnapshottingBufferingSinkWriter snapshottingWriter = - new SnapshottingBufferingSinkWriter(); + final SinkAndSuppliers sinkAndSuppliers = sinkWithSnapshottingWriter(stateful, null); final OneInputStreamOperatorTestHarness> testHarness = - createTestHarnessWithBufferingSinkWriter(snapshottingWriter, stateful); + createTestHarnessWithBufferingSinkWriter(sinkAndSuppliers.sink); testHarness.open(); @@ -204,15 +197,14 @@ void testStateRestore(boolean stateful) throws Exception { // we see the watermark and the committable summary, so the committables must be stored in // state assertThat(testHarness.getOutput()).hasSize(2).contains(new Watermark(initialTime)); - assertThat(snapshottingWriter.lastCheckpointId) - .isEqualTo(stateful ? 1L : SnapshottingBufferingSinkWriter.NOT_SNAPSHOTTED); + assertThat(sinkAndSuppliers.lastCheckpointSupplier.getAsLong()) + .isEqualTo(stateful ? 1L : -1L); testHarness.close(); + final SinkAndSuppliers restoredSink = sinkWithSnapshottingWriter(stateful, null); final OneInputStreamOperatorTestHarness> - restoredTestHarness = - createTestHarnessWithBufferingSinkWriter( - new SnapshottingBufferingSinkWriter(), stateful); + restoredTestHarness = createTestHarnessWithBufferingSinkWriter(restoredSink.sink); restoredTestHarness.initializeState(snapshot); restoredTestHarness.open(); @@ -221,6 +213,7 @@ void testStateRestore(boolean stateful) throws Exception { restoredTestHarness.endInput(); final long checkpointId = 2; restoredTestHarness.prepareSnapshotPreBarrier(checkpointId); + restoredTestHarness.notifyOfCompletedCheckpoint(checkpointId); if (stateful) { assertBasicOutput(restoredTestHarness.getOutput(), 2, Long.MAX_VALUE); @@ -246,16 +239,20 @@ public void testLoadPreviousSinkState(boolean stateful) throws Exception { "bit", "mention", "thick", "stick", "stir", "easy", "sleep", "forth", "cost", "prompt"); + SinkAndSuppliers sinkAndSuppliers = + sinkWithSnapshottingWriter(stateful, DummySinkOperator.DUMMY_SINK_STATE_NAME); final OneInputStreamOperatorTestHarness previousSink = new OneInputStreamOperatorTestHarness<>( - new DummySinkOperator(), StringSerializer.INSTANCE); + new DummySinkOperator(sinkAndSuppliers.serializerSupplier.get()), + StringSerializer.INSTANCE); OperatorSubtaskState previousSinkState = TestHarnessUtil.buildSubtaskState(previousSink, previousSinkInputs); // 2. Load previous sink state and verify the output final OneInputStreamOperatorTestHarness> - compatibleWriterOperator = createCompatibleStateTestHarness(stateful); + compatibleWriterOperator = + createTestHarnessWithBufferingSinkWriter(sinkAndSuppliers.sink); final List expectedOutput1 = stateful ? new ArrayList<>(previousSinkInputs) : new ArrayList<>(); @@ -280,8 +277,11 @@ public void testLoadPreviousSinkState(boolean stateful) throws Exception { assertEmitted(expectedOutput1, compatibleWriterOperator.getOutput()); // 3. Restore the sink without previous sink's state + SinkAndSuppliers sinkAndSuppliers2 = + sinkWithSnapshottingWriter(stateful, DummySinkOperator.DUMMY_SINK_STATE_NAME); final OneInputStreamOperatorTestHarness> - restoredSinkOperator = createCompatibleStateTestHarness(stateful); + restoredSinkOperator = + createTestHarnessWithBufferingSinkWriter(sinkAndSuppliers2.sink); final List expectedOutput2 = Arrays.asList( Tuple3.of(2, 2, Long.MIN_VALUE).toString(), @@ -306,22 +306,18 @@ public void testLoadPreviousSinkState(boolean stateful) throws Exception { void testRestoreCommitterState() throws Exception { final List committables = Arrays.asList("state1", "state2"); + SinkAndSuppliers sinkAndSuppliers = sinkWithCommitter(); final OneInputStreamOperatorTestHarness committer = new OneInputStreamOperatorTestHarness<>( - new TestCommitterOperator(), StringSerializer.INSTANCE); + new TestCommitterOperator(sinkAndSuppliers.serializerSupplier.get()), + StringSerializer.INSTANCE); final OperatorSubtaskState committerState = TestHarnessUtil.buildSubtaskState(committer, committables); - final TestSink.DefaultSinkWriter sinkWriter = new TestSink.DefaultSinkWriter<>(); final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder() - .setDefaultCommitter() - .setWriter(sinkWriter) - .build() - .asV2())); + new SinkWriterOperatorFactory<>(sinkAndSuppliers.sink)); testHarness.initializeState(committerState); @@ -361,21 +357,16 @@ void testRestoreCommitterState() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testHandleEndInputInStreamingMode(boolean isCheckpointingEnabled) throws Exception { - final TestSink.DefaultSinkWriter sinkWriter = new TestSink.DefaultSinkWriter<>(); + SinkAndSuppliers sinkAndSuppliers = sinkWithCommitter(); final OneInputStreamOperatorTestHarness> testHarness = new OneInputStreamOperatorTestHarness<>( - new SinkWriterOperatorFactory<>( - TestSink.newBuilder() - .setWriter(sinkWriter) - .setDefaultCommitter() - .build() - .asV2())); + new SinkWriterOperatorFactory<>(sinkAndSuppliers.sink)); testHarness.open(); testHarness.processElement(1, 1); assertThat(testHarness.getOutput()).isEmpty(); final String record = "(1,1," + Long.MIN_VALUE + ")"; - assertThat(sinkWriter.elements).containsOnly(record); + assertThat(sinkAndSuppliers.elementSupplier.get()).containsOnly(record); testHarness.endInput(); @@ -384,7 +375,7 @@ void testHandleEndInputInStreamingMode(boolean isCheckpointingEnabled) throws Ex } assertEmitted(Collections.singletonList(record), testHarness.getOutput()); - assertThat(sinkWriter.elements).isEmpty(); + assertThat(sinkAndSuppliers.elementSupplier.get()).isEmpty(); testHarness.close(); } @@ -472,33 +463,9 @@ private static void assertEmitted(List records, Queue output) { } private static OneInputStreamOperatorTestHarness> - createTestHarnessWithBufferingSinkWriter( - SnapshottingBufferingSinkWriter snapshottingWriter, boolean stateful) - throws Exception { - final TestSink.Builder builder = - TestSink.newBuilder().setDefaultCommitter().setWriter(snapshottingWriter); - if (stateful) { - builder.withWriterState(); - } + createTestHarnessWithBufferingSinkWriter(Sink sink) throws Exception { final SinkWriterOperatorFactory writerOperatorFactory = - new SinkWriterOperatorFactory<>(builder.build().asV2()); - return new OneInputStreamOperatorTestHarness<>(writerOperatorFactory); - } - - private static OneInputStreamOperatorTestHarness> - createCompatibleStateTestHarness(boolean stateful) throws Exception { - final SnapshottingBufferingSinkWriter snapshottingWriter = - new SnapshottingBufferingSinkWriter(); - final TestSink.Builder builder = - TestSink.newBuilder() - .setDefaultCommitter() - .setCompatibleStateNames(DummySinkOperator.DUMMY_SINK_STATE_NAME) - .setWriter(snapshottingWriter); - if (stateful) { - builder.withWriterState(); - } - final SinkWriterOperatorFactory writerOperatorFactory = - new SinkWriterOperatorFactory<>(builder.build().asV2()); + new SinkWriterOperatorFactory<>(sink); return new OneInputStreamOperatorTestHarness<>(writerOperatorFactory); } @@ -527,30 +494,6 @@ private static void assertBasicOutput( } } - private static class TimeBasedBufferingSinkWriter extends TestSink.DefaultSinkWriter - implements Sink.ProcessingTimeService.ProcessingTimeCallback { - - private final List cachedCommittables = new ArrayList<>(); - - @Override - public void write(Integer element, Context context) { - cachedCommittables.add( - Tuple3.of(element, context.timestamp(), context.currentWatermark()).toString()); - } - - void setProcessingTimerService(Sink.ProcessingTimeService processingTimerService) { - super.setProcessingTimerService(processingTimerService); - this.processingTimerService.registerProcessingTimer(1000, this); - } - - @Override - public void onProcessingTime(long time) { - elements.addAll(cachedCommittables); - cachedCommittables.clear(); - this.processingTimerService.registerProcessingTimer(time + 1000, this); - } - } - private static class TestCommitterOperator extends AbstractStreamOperator implements OneInputStreamOperator { @@ -559,6 +502,11 @@ private static class TestCommitterOperator extends AbstractStreamOperator> committerState; private final List buffer = new ArrayList<>(); + private final SimpleVersionedSerializer serializer; + + public TestCommitterOperator(SimpleVersionedSerializer serializer) { + this.serializer = serializer; + } @Override public void initializeState(StateInitializationContext context) throws Exception { @@ -567,8 +515,7 @@ public void initializeState(StateInitializationContext context) throws Exception new SimpleVersionedListState<>( context.getOperatorStateStore() .getListState(STREAMING_COMMITTER_RAW_STATES_DESC), - new TestingCommittableSerializer( - TestSink.StringCommittableSerializer.INSTANCE)); + new TestingCommittableSerializer(serializer)); } @Override @@ -592,13 +539,18 @@ private static class DummySinkOperator extends AbstractStreamOperator new ListStateDescriptor<>( DUMMY_SINK_STATE_NAME, BytePrimitiveArraySerializer.INSTANCE); ListState sinkState; + private final SimpleVersionedSerializer serializer; + + public DummySinkOperator(SimpleVersionedSerializer serializer) { + this.serializer = serializer; + } public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); sinkState = new SimpleVersionedListState<>( context.getOperatorStateStore().getListState(SINK_STATE_DESC), - TestSink.StringCommittableSerializer.INSTANCE); + serializer); } @Override @@ -607,33 +559,6 @@ public void processElement(StreamRecord element) throws Exception { } } - private static class SnapshottingBufferingSinkWriter - extends TestSink.DefaultSinkWriter { - public static final int NOT_SNAPSHOTTED = -1; - long lastCheckpointId = NOT_SNAPSHOTTED; - - @Override - public List snapshotState(long checkpointId) { - lastCheckpointId = checkpointId; - return elements; - } - - @Override - void restoredFrom(List states) { - this.elements = new ArrayList<>(states); - } - - @Override - public List prepareCommit(boolean flush) { - if (!flush) { - return Collections.emptyList(); - } - List result = elements; - elements = new ArrayList<>(); - return result; - } - } - private static class TestingCommittableSerializer extends SinkV1WriterCommittableSerializer { @@ -654,4 +579,33 @@ public byte[] serialize(List obj) throws IOException { return out.getCopyOfBuffer(); } } + + abstract SinkAndSuppliers sinkWithoutCommitter(); + + abstract SinkAndSuppliers sinkWithTimeBasedWriter(); + + abstract SinkAndSuppliers sinkWithSnapshottingWriter(boolean withState, String stateName); + + abstract SinkAndSuppliers sinkWithCommitter(); + + static class SinkAndSuppliers { + org.apache.flink.api.connector.sink2.Sink sink; + Supplier> elementSupplier; + Supplier> watermarkSupplier; + LongSupplier lastCheckpointSupplier; + Supplier> serializerSupplier; + + public SinkAndSuppliers( + org.apache.flink.api.connector.sink2.Sink sink, + Supplier> elementSupplier, + Supplier> watermarkSupplier, + LongSupplier lastCheckpointSupplier, + Supplier> serializerSupplier) { + this.sink = sink; + this.elementSupplier = elementSupplier; + this.watermarkSupplier = watermarkSupplier; + this.lastCheckpointSupplier = lastCheckpointSupplier; + this.serializerSupplier = serializerSupplier; + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSink.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSink.java index ce00362f6a058..742e4438b5d36 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSink.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSink.java @@ -49,7 +49,13 @@ import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertNotNull; -/** A {@link Sink TestSink} for all the sink related tests. */ +/** + * A {@link Sink TestSink} for all the sink related tests. Use only for tests where {@link + * SinkV1Adapter} should be tested. + * + * @deprecated Use {@link TestSinkV2} instead. + */ +@Deprecated public class TestSink implements Sink { public static final String END_OF_INPUT_STR = "end of input"; @@ -181,11 +187,6 @@ public Builder setDefaultCommitter(Supplier> queueSupplier) { return this; } - public Builder setGlobalCommitter(GlobalCommitter globalCommitter) { - this.globalCommitter = globalCommitter; - return this; - } - public Builder setGlobalCommittableSerializer( SimpleVersionedSerializer globalCommittableSerializer) { this.globalCommittableSerializer = globalCommittableSerializer; @@ -363,10 +364,6 @@ static class DefaultGlobalCommitter extends DefaultCommitter private final String committedSuccessData; - DefaultGlobalCommitter() { - this(""); - } - DefaultGlobalCommitter(String committedSuccessData) { this.committedSuccessData = committedSuccessData; } @@ -397,39 +394,6 @@ public void endOfInput() { } } - /** A {@link GlobalCommitter} that always re-commits global committables it received. */ - static class RetryOnceGlobalCommitter extends DefaultGlobalCommitter { - - private final Set seen = new LinkedHashSet<>(); - - @Override - public List filterRecoveredCommittables(List globalCommittables) { - return globalCommittables; - } - - @Override - public String combine(List committables) { - return String.join("|", committables); - } - - @Override - public void endOfInput() {} - - @Override - public List commit(List committables) { - committables.forEach( - c -> { - if (seen.remove(c)) { - checkNotNull(committedData); - committedData.add(c); - } else { - seen.add(c); - } - }); - return new ArrayList<>(seen); - } - } - /** * We introduce this {@link StringCommittableSerializer} is because that all the fields of * {@link TestSink} should be serializable. diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSinkV2.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSinkV2.java new file mode 100644 index 0000000000000..bb89bf0fe7cc0 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSinkV2.java @@ -0,0 +1,434 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.streaming.runtime.operators.sink; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.SimpleVersionedStringSerializer; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.Supplier; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertNotNull; + +/** A {@link org.apache.flink.api.connector.sink2.Sink} for all the sink related tests. */ +public class TestSinkV2 implements Sink { + + private final DefaultSinkWriter writer; + + private TestSinkV2(DefaultSinkWriter writer) { + this.writer = writer; + } + + @Override + public SinkWriter createWriter(InitContext context) { + writer.init(context); + return writer; + } + + DefaultSinkWriter getWriter() { + return writer; + } + + public static Builder newBuilder() { + return new Builder<>(); + } + + /** A builder class for {@link TestSinkV2}. */ + public static class Builder { + private DefaultSinkWriter writer = null; + private DefaultCommitter committer; + private SimpleVersionedSerializer committableSerializer; + private boolean withPostCommitTopology = false; + private boolean withWriterState = false; + private String compatibleStateNames; + + public Builder setWriter(DefaultSinkWriter writer) { + this.writer = checkNotNull(writer); + return this; + } + + public Builder setCommitter(DefaultCommitter committer) { + this.committer = committer; + return this; + } + + public Builder setCommittableSerializer( + SimpleVersionedSerializer committableSerializer) { + this.committableSerializer = committableSerializer; + return this; + } + + public Builder setDefaultCommitter() { + this.committer = new DefaultCommitter(); + this.committableSerializer = StringSerializer.INSTANCE; + return this; + } + + public Builder setDefaultCommitter( + Supplier>> queueSupplier) { + this.committer = new DefaultCommitter(queueSupplier); + this.committableSerializer = StringSerializer.INSTANCE; + return this; + } + + public Builder setWithPostCommitTopology(boolean withPostCommitTopology) { + this.withPostCommitTopology = withPostCommitTopology; + return this; + } + + public Builder setWriterState(boolean withWriterState) { + this.withWriterState = withWriterState; + return this; + } + + public Builder setCompatibleStateNames(String compatibleStateNames) { + this.compatibleStateNames = compatibleStateNames; + return this; + } + + public TestSinkV2 build() { + if (committer == null) { + if (writer == null) { + writer = new DefaultSinkWriter<>(); + } + // SinkV2 with a simple writer + return new TestSinkV2<>(writer); + } else { + if (writer == null) { + writer = new DefaultCommittingSinkWriter<>(); + } + if (!withPostCommitTopology) { + // TwoPhaseCommittingSink with a stateless writer and a committer + return new TestSinkV2TwoPhaseCommittingSink<>( + writer, committableSerializer, committer); + } else { + if (withWriterState) { + // TwoPhaseCommittingSink with a stateful writer and a committer and post + // commit topology + Preconditions.checkArgument( + writer instanceof DefaultStatefulSinkWriter, + "Please provide a DefaultStatefulSinkWriter instance"); + return new TestStatefulSinkV2( + (DefaultStatefulSinkWriter) writer, + committableSerializer, + committer, + compatibleStateNames); + } else { + // TwoPhaseCommittingSink with a stateless writer and a committer and post + // commit topology + Preconditions.checkArgument( + writer instanceof DefaultCommittingSinkWriter, + "Please provide a DefaultCommittingSinkWriter instance"); + return new TestSinkV2WithPostCommitTopology<>( + (DefaultCommittingSinkWriter) writer, + committableSerializer, + committer); + } + } + } + } + } + + private static class TestSinkV2TwoPhaseCommittingSink extends TestSinkV2 + implements TwoPhaseCommittingSink { + private final DefaultCommitter committer; + private final SimpleVersionedSerializer committableSerializer; + + public TestSinkV2TwoPhaseCommittingSink( + DefaultSinkWriter writer, + SimpleVersionedSerializer committableSerializer, + DefaultCommitter committer) { + super(writer); + this.committer = committer; + this.committableSerializer = committableSerializer; + } + + @Override + public Committer createCommitter() { + committer.init(); + return committer; + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return committableSerializer; + } + + @Override + public PrecommittingSinkWriter createWriter(InitContext context) { + return (PrecommittingSinkWriter) super.createWriter(context); + } + } + + // -------------------------------------- Sink With PostCommitTopology ------------------------- + + private static class TestSinkV2WithPostCommitTopology + extends TestSinkV2TwoPhaseCommittingSink + implements WithPostCommitTopology { + public TestSinkV2WithPostCommitTopology( + DefaultSinkWriter writer, + SimpleVersionedSerializer committableSerializer, + DefaultCommitter committer) { + super(writer, committableSerializer, committer); + } + + @Override + public void addPostCommitTopology(DataStream> committables) { + // We do not need to do anything for tests + } + } + + private static class TestStatefulSinkV2 extends TestSinkV2WithPostCommitTopology + implements StatefulSink, StatefulSink.WithCompatibleState { + private String compatibleState; + + public TestStatefulSinkV2( + DefaultStatefulSinkWriter writer, + SimpleVersionedSerializer committableSerializer, + DefaultCommitter committer, + String compatibleState) { + super(writer, committableSerializer, committer); + this.compatibleState = compatibleState; + } + + @Override + public DefaultStatefulSinkWriter createWriter(InitContext context) { + return (DefaultStatefulSinkWriter) super.createWriter(context); + } + + @Override + public StatefulSinkWriter restoreWriter( + InitContext context, Collection recoveredState) { + DefaultStatefulSinkWriter statefulWriter = + (DefaultStatefulSinkWriter) getWriter(); + + statefulWriter.restore(recoveredState); + return statefulWriter; + } + + @Override + public SimpleVersionedSerializer getWriterStateSerializer() { + return new StringSerializer(); + } + + @Override + public Collection getCompatibleWriterStateNames() { + return compatibleState == null ? ImmutableSet.of() : ImmutableSet.of(compatibleState); + } + } + + // -------------------------------------- Sink Writer ------------------------------------------ + + /** Base class for out testing {@link SinkWriter}. */ + public static class DefaultSinkWriter implements SinkWriter, Serializable { + + protected List elements; + + protected List watermarks; + + protected DefaultSinkWriter() { + this.elements = new ArrayList<>(); + this.watermarks = new ArrayList<>(); + } + + @Override + public void write(InputT element, Context context) { + elements.add( + Tuple3.of(element, context.timestamp(), context.currentWatermark()).toString()); + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + elements = new ArrayList<>(); + } + + @Override + public void writeWatermark(Watermark watermark) { + watermarks.add(watermark); + } + + @Override + public void close() throws Exception { + // noting to do here + } + + public void init(InitContext context) { + // context is not used in default case + } + } + + /** Base class for out testing {@link TwoPhaseCommittingSink.PrecommittingSinkWriter}. */ + protected static class DefaultCommittingSinkWriter extends DefaultSinkWriter + implements TwoPhaseCommittingSink.PrecommittingSinkWriter, + Serializable { + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // We empty the elements on prepareCommit + } + + @Override + public Collection prepareCommit() { + List result = elements; + elements = new ArrayList<>(); + return result; + } + } + + /** + * Base class for out testing {@link StatefulSink.StatefulSinkWriter}. Extends the {@link + * DefaultCommittingSinkWriter} for simplicity. + */ + protected static class DefaultStatefulSinkWriter + extends DefaultCommittingSinkWriter + implements StatefulSink.StatefulSinkWriter { + + @Override + public List snapshotState(long checkpointId) throws IOException { + return elements; + } + + protected void restore(Collection recoveredState) { + this.elements = new ArrayList<>(recoveredState); + } + } + + // -------------------------------------- Sink Committer --------------------------------------- + + /** Base class for testing {@link Committer}. */ + static class DefaultCommitter implements Committer, Serializable { + + @Nullable protected Queue> committedData; + + private boolean isClosed; + + @Nullable private final Supplier>> queueSupplier; + + public DefaultCommitter() { + this.committedData = new ConcurrentLinkedQueue<>(); + this.isClosed = false; + this.queueSupplier = null; + } + + public DefaultCommitter(@Nullable Supplier>> queueSupplier) { + this.queueSupplier = queueSupplier; + this.isClosed = false; + this.committedData = null; + } + + public List> getCommittedData() { + if (committedData != null) { + return new ArrayList<>(committedData); + } else { + return Collections.emptyList(); + } + } + + @Override + public void commit(Collection> committables) { + if (committedData == null) { + assertNotNull(queueSupplier); + committedData = queueSupplier.get(); + } + committedData.addAll(committables); + } + + public void close() throws Exception { + isClosed = true; + } + + public boolean isClosed() { + return isClosed; + } + + public void init() { + // context is not used for this implementation + } + } + + /** A {@link Committer} that always re-commits the committables data it received. */ + static class RetryOnceCommitter extends DefaultCommitter { + + private final Set> seen = new LinkedHashSet<>(); + + @Override + public void commit(Collection> committables) { + committables.forEach( + c -> { + if (seen.remove(c)) { + checkNotNull(committedData); + committedData.add(c); + } else { + seen.add(c); + c.retryLater(); + } + }); + } + } + + /** + * We introduce this {@link StringSerializer} is because that all the fields of {@link + * TestSinkV2} should be serializable. + */ + public static class StringSerializer + implements SimpleVersionedSerializer, Serializable { + + public static final StringSerializer INSTANCE = new StringSerializer(); + + @Override + public int getVersion() { + return SimpleVersionedStringSerializer.INSTANCE.getVersion(); + } + + @Override + public byte[] serialize(String obj) { + return SimpleVersionedStringSerializer.INSTANCE.serialize(obj); + } + + @Override + public String deserialize(int version, byte[] serialized) throws IOException { + return SimpleVersionedStringSerializer.INSTANCE.deserialize(version, serialized); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterCommitterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterCommitterOperatorTest.java new file mode 100644 index 0000000000000..c516db8746743 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterCommitterOperatorTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.operators.sink; + +import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; + +import java.util.Collections; +import java.util.List; + +class WithAdapterCommitterOperatorTest extends CommitterOperatorTestBase { + + @Override + SinkAndCounters sinkWithPostCommit() { + ForwardingCommitter committer = new ForwardingCommitter(); + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSink.newBuilder() + .setCommitter(committer) + .setDefaultGlobalCommitter() + .setCommittableSerializer( + TestSink.StringCommittableSerializer.INSTANCE) + .build() + .asV2(), + () -> committer.successfulCommits); + } + + @Override + SinkAndCounters sinkWithPostCommitWithRetry() { + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSink.newBuilder() + .setCommitter(new TestSink.RetryOnceCommitter()) + .setDefaultGlobalCommitter() + .setCommittableSerializer( + TestSink.StringCommittableSerializer.INSTANCE) + .build() + .asV2(), + () -> 0); + } + + @Override + SinkAndCounters sinkWithoutPostCommit() { + ForwardingCommitter committer = new ForwardingCommitter(); + return new SinkAndCounters( + (TwoPhaseCommittingSink) + TestSink.newBuilder() + .setCommitter(committer) + .setCommittableSerializer( + TestSink.StringCommittableSerializer.INSTANCE) + .build() + .asV2(), + () -> committer.successfulCommits); + } + + private static class ForwardingCommitter extends TestSink.DefaultCommitter { + private int successfulCommits = 0; + + @Override + public List commit(List committables) { + successfulCommits += committables.size(); + return Collections.emptyList(); + } + + @Override + public void close() throws Exception {} + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterSinkWriterOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterSinkWriterOperatorTest.java new file mode 100644 index 0000000000000..5af5ac5a679a4 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/WithAdapterSinkWriterOperatorTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.operators.sink; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.java.tuple.Tuple3; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +class WithAdapterSinkWriterOperatorTest extends SinkWriterOperatorTestBase { + + @Override + SinkAndSuppliers sinkWithoutCommitter() { + TestSink.DefaultSinkWriter sinkWriter = new TestSink.DefaultSinkWriter<>(); + return new SinkAndSuppliers( + TestSink.newBuilder().setWriter(sinkWriter).build().asV2(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + () -> new TestSink.StringCommittableSerializer()); + } + + @Override + SinkAndSuppliers sinkWithCommitter() { + TestSink.DefaultSinkWriter sinkWriter = new TestSink.DefaultSinkWriter<>(); + return new SinkAndSuppliers( + TestSink.newBuilder().setWriter(sinkWriter).setDefaultCommitter().build().asV2(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + () -> new TestSink.StringCommittableSerializer()); + } + + @Override + SinkAndSuppliers sinkWithTimeBasedWriter() { + TestSink.DefaultSinkWriter sinkWriter = new TimeBasedBufferingSinkWriter(); + return new SinkAndSuppliers( + TestSink.newBuilder().setWriter(sinkWriter).setDefaultCommitter().build().asV2(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> -1, + () -> new TestSink.StringCommittableSerializer()); + } + + @Override + SinkAndSuppliers sinkWithSnapshottingWriter(boolean withState, String stateName) { + SnapshottingBufferingSinkWriter sinkWriter = new SnapshottingBufferingSinkWriter(); + TestSink.Builder builder = + TestSink.newBuilder().setWriter(sinkWriter).setDefaultCommitter(); + if (withState) { + builder.withWriterState(); + } + if (stateName != null) { + builder.setCompatibleStateNames(stateName); + } + return new SinkAndSuppliers( + builder.build().asV2(), + () -> sinkWriter.elements, + () -> sinkWriter.watermarks, + () -> sinkWriter.lastCheckpointId, + () -> new TestSink.StringCommittableSerializer()); + } + + private static class TimeBasedBufferingSinkWriter extends TestSink.DefaultSinkWriter + implements Sink.ProcessingTimeService.ProcessingTimeCallback { + + private final List cachedCommittables = new ArrayList<>(); + + @Override + public void write(Integer element, Context context) { + cachedCommittables.add( + Tuple3.of(element, context.timestamp(), context.currentWatermark()).toString()); + } + + void setProcessingTimerService(Sink.ProcessingTimeService processingTimerService) { + super.setProcessingTimerService(processingTimerService); + this.processingTimerService.registerProcessingTimer(1000, this); + } + + @Override + public void onProcessingTime(long time) { + elements.addAll(cachedCommittables); + cachedCommittables.clear(); + this.processingTimerService.registerProcessingTimer(time + 1000, this); + } + } + + private static class SnapshottingBufferingSinkWriter + extends TestSink.DefaultSinkWriter { + public static final int NOT_SNAPSHOTTED = -1; + long lastCheckpointId = NOT_SNAPSHOTTED; + + @Override + public List snapshotState(long checkpointId) { + lastCheckpointId = checkpointId; + return elements; + } + + @Override + void restoredFrom(List states) { + this.elements = new ArrayList<>(states); + } + + @Override + public List prepareCommit(boolean flush) { + if (!flush) { + return Collections.emptyList(); + } + List result = elements; + elements = new ArrayList<>(); + return result; + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2ITCase.java new file mode 100644 index 0000000000000..70d9400474883 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2ITCase.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.test.streaming.runtime; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.typeinfo.IntegerTypeInfo; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.runtime.operators.sink.TestSinkV2; +import org.apache.flink.streaming.util.FiniteTestSource; +import org.apache.flink.test.util.AbstractTestBase; + +import org.junit.Before; +import org.junit.Test; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.BooleanSupplier; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +/** + * Integration test for {@link org.apache.flink.api.connector.sink.Sink} run time implementation. + */ +public class SinkV2ITCase extends AbstractTestBase { + static final List SOURCE_DATA = + Arrays.asList( + 895, 127, 148, 161, 148, 662, 822, 491, 275, 122, 850, 630, 682, 765, 434, 970, + 714, 795, 288, 422); + + // source send data two times + static final int STREAMING_SOURCE_SEND_ELEMENTS_NUM = SOURCE_DATA.size() * 2; + + static final List EXPECTED_COMMITTED_DATA_IN_STREAMING_MODE = + SOURCE_DATA.stream() + // source send data two times + .flatMap( + x -> + Collections.nCopies( + 2, Tuple3.of(x, null, Long.MIN_VALUE).toString()) + .stream()) + .collect(Collectors.toList()); + + static final List EXPECTED_COMMITTED_DATA_IN_BATCH_MODE = + SOURCE_DATA.stream() + .map(x -> Tuple3.of(x, null, Long.MIN_VALUE).toString()) + .collect(Collectors.toList()); + + static final Queue> COMMIT_QUEUE = + new ConcurrentLinkedQueue<>(); + + static final BooleanSupplier COMMIT_QUEUE_RECEIVE_ALL_DATA = + (BooleanSupplier & Serializable) + () -> COMMIT_QUEUE.size() == STREAMING_SOURCE_SEND_ELEMENTS_NUM; + + @Before + public void init() { + COMMIT_QUEUE.clear(); + } + + @Test + public void writerAndCommitterExecuteInStreamingMode() throws Exception { + final StreamExecutionEnvironment env = buildStreamEnv(); + final FiniteTestSource source = + new FiniteTestSource<>(COMMIT_QUEUE_RECEIVE_ALL_DATA, SOURCE_DATA); + + env.addSource(source, IntegerTypeInfo.INT_TYPE_INFO) + .sinkTo( + TestSinkV2.newBuilder() + .setDefaultCommitter( + (Supplier>> + & Serializable) + () -> COMMIT_QUEUE) + .build()); + env.execute(); + assertThat( + COMMIT_QUEUE.stream() + .map(Committer.CommitRequest::getCommittable) + .collect(Collectors.toList()), + containsInAnyOrder(EXPECTED_COMMITTED_DATA_IN_STREAMING_MODE.toArray())); + } + + @Test + public void writerAndCommitterExecuteInBatchMode() throws Exception { + final StreamExecutionEnvironment env = buildBatchEnv(); + + env.fromCollection(SOURCE_DATA) + .sinkTo( + TestSinkV2.newBuilder() + .setDefaultCommitter( + (Supplier>> + & Serializable) + () -> COMMIT_QUEUE) + .build()); + env.execute(); + assertThat( + COMMIT_QUEUE.stream() + .map(Committer.CommitRequest::getCommittable) + .collect(Collectors.toList()), + containsInAnyOrder(EXPECTED_COMMITTED_DATA_IN_BATCH_MODE.toArray())); + } + + private StreamExecutionEnvironment buildStreamEnv() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + env.enableCheckpointing(100); + return env; + } + + private StreamExecutionEnvironment buildBatchEnv() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + return env; + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2MetricsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2MetricsITCase.java new file mode 100644 index 0000000000000..76fcc7b66cc38 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkV2MetricsITCase.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.streaming.runtime; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.runtime.operators.sink.TestSinkV2; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.testutils.junit.SharedReference; +import org.apache.flink.util.TestLogger; + +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CyclicBarrier; +import java.util.stream.LongStream; + +import static org.apache.flink.metrics.testutils.MetricAssertions.assertThatCounter; +import static org.apache.flink.metrics.testutils.MetricAssertions.assertThatGauge; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; + +/** Tests whether all provided metrics of a {@link Sink} are of the expected values (FLIP-33). */ +public class SinkV2MetricsITCase extends TestLogger { + + private static final String TEST_SINK_NAME = "MetricTestSink"; + // please refer to SinkTransformationTranslator#WRITER_NAME + private static final String DEFAULT_WRITER_NAME = "Writer"; + private static final int DEFAULT_PARALLELISM = 4; + + @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .build()); + + @Test + public void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + int numSplits = Math.max(1, env.getParallelism() - 2); + + int numRecordsPerSplit = 10; + + // make sure all parallel instances have processed the same amount of records before + // validating metrics + SharedReference beforeBarrier = + sharedObjects.add(new CyclicBarrier(numSplits + 1)); + SharedReference afterBarrier = + sharedObjects.add(new CyclicBarrier(numSplits + 1)); + int stopAtRecord1 = 4; + int stopAtRecord2 = numRecordsPerSplit - 1; + + env.fromSequence(0, numSplits - 1) + .flatMap( + (split, collector) -> + LongStream.range(0, numRecordsPerSplit).forEach(collector::collect)) + .returns(BasicTypeInfo.LONG_TYPE_INFO) + .map( + i -> { + if (i % numRecordsPerSplit == stopAtRecord1 + || i % numRecordsPerSplit == stopAtRecord2) { + beforeBarrier.get().await(); + afterBarrier.get().await(); + } + return i; + }) + .sinkTo(TestSinkV2.newBuilder().setWriter(new MetricWriter()).build()) + .name(TEST_SINK_NAME); + JobClient jobClient = env.executeAsync(); + final JobID jobId = jobClient.getJobID(); + + beforeBarrier.get().await(); + assertSinkMetrics(jobId, stopAtRecord1, env.getParallelism(), numSplits); + afterBarrier.get().await(); + + beforeBarrier.get().await(); + assertSinkMetrics(jobId, stopAtRecord2, env.getParallelism(), numSplits); + afterBarrier.get().await(); + + jobClient.getJobExecutionResult().get(); + } + + @SuppressWarnings("checkstyle:WhitespaceAfter") + private void assertSinkMetrics( + JobID jobId, long processedRecordsPerSubtask, int parallelism, int numSplits) { + List groups = + reporter.findOperatorMetricGroups( + jobId, TEST_SINK_NAME + ": " + DEFAULT_WRITER_NAME); + assertThat(groups, hasSize(parallelism)); + + int subtaskWithMetrics = 0; + for (OperatorMetricGroup group : groups) { + Map metrics = reporter.getMetricsByGroup(group); + // There are only 2 splits assigned; so two groups will not update metrics. + if (group.getIOMetricGroup().getNumRecordsOutCounter().getCount() == 0) { + continue; + } + subtaskWithMetrics++; + + // SinkWriterMetricGroup metrics + assertThatCounter(metrics.get(MetricNames.IO_NUM_RECORDS_OUT)) + .isEqualTo(processedRecordsPerSubtask); + assertThatCounter(metrics.get(MetricNames.IO_NUM_BYTES_OUT)) + .isEqualTo(processedRecordsPerSubtask * MetricWriter.RECORD_SIZE_IN_BYTES); + // MetricWriter is just incrementing errors every even record + assertThatCounter(metrics.get(MetricNames.NUM_RECORDS_OUT_ERRORS)) + .isEqualTo((processedRecordsPerSubtask + 1) / 2); + + // Test "send" metric series has the same value as "out" metric series. + assertThatCounter(metrics.get(MetricNames.NUM_RECORDS_SEND)) + .isEqualTo(processedRecordsPerSubtask); + assertThatCounter(metrics.get(MetricNames.NUM_BYTES_SEND)) + .isEqualTo(processedRecordsPerSubtask * MetricWriter.RECORD_SIZE_IN_BYTES); + assertThatCounter(metrics.get(MetricNames.NUM_RECORDS_SEND_ERRORS)) + .isEqualTo((processedRecordsPerSubtask + 1) / 2); + + // check if the latest send time is fetched + assertThatGauge(metrics.get(MetricNames.CURRENT_SEND_TIME)) + .isEqualTo((processedRecordsPerSubtask - 1) * MetricWriter.BASE_SEND_TIME); + } + assertThat(subtaskWithMetrics, equalTo(numSplits)); + } + + private static class MetricWriter extends TestSinkV2.DefaultSinkWriter { + static final long BASE_SEND_TIME = 100; + static final long RECORD_SIZE_IN_BYTES = 10; + private SinkWriterMetricGroup metricGroup; + private long sendTime; + + @Override + public void init(Sink.InitContext context) { + this.metricGroup = context.metricGroup(); + metricGroup.setCurrentSendTimeGauge(() -> sendTime); + } + + @Override + public void write(Long element, Context context) { + super.write(element, context); + sendTime = element * BASE_SEND_TIME; + metricGroup.getIOMetricGroup().getNumRecordsOutCounter().inc(); + if (element % 2 == 0) { + metricGroup.getNumRecordsOutErrorsCounter().inc(); + } + metricGroup.getIOMetricGroup().getNumBytesOutCounter().inc(RECORD_SIZE_IN_BYTES); + } + } +} From eb4ae5d4e7d517300e98e632de95249dbdd22192 Mon Sep 17 00:00:00 2001 From: Matthias Schwalbe <45448573+RainerMatthiasS@users.noreply.github.com> Date: Fri, 18 Aug 2023 12:59:59 +0200 Subject: [PATCH 092/104] [FLINK-26585][state-processor-api] replaced implementation of MultiStateKeyIterator with Stream-free implementation(#23239) Avoids a known flaw in Stream.flatMap, see https://bugs.openjdk.org/browse/JDK-8267359 --- .../api/input/MultiStateKeyIterator.java | 53 +++-- .../api/input/MultiStateKeyIteratorTest.java | 214 ++++++++++++++++++ 2 files changed, 245 insertions(+), 22 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java index ed219e69db592..d4ffbce264e7b 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java @@ -31,6 +31,8 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; +import java.util.NoSuchElementException; +import java.util.stream.Stream; /** * An iterator for reading all keys in a state backend across multiple partitioned states. @@ -46,7 +48,10 @@ public final class MultiStateKeyIterator implements CloseableIterator { private final KeyedStateBackend backend; - private final Iterator internal; + /** Avoids using Stream#flatMap due to a known flaw, see FLINK-26585 for more details. */ + private final Iterator> outerIter; + + private Iterator innerIter; private final CloseableRegistry registry; @@ -54,39 +59,43 @@ public final class MultiStateKeyIterator implements CloseableIterator { public MultiStateKeyIterator( List> descriptors, KeyedStateBackend backend) { + this.descriptors = Preconditions.checkNotNull(descriptors); this.backend = Preconditions.checkNotNull(backend); + outerIter = this.descriptors.iterator(); + innerIter = null; + this.registry = new CloseableRegistry(); - this.internal = - descriptors.stream() - .map( - descriptor -> - backend.getKeys( - descriptor.getName(), VoidNamespace.INSTANCE)) - .peek( - stream -> { - try { - registry.registerCloseable(stream::close); - } catch (IOException e) { - throw new RuntimeException( - "Failed to read keys from configured StateBackend", - e); - } - }) - .flatMap(stream -> stream) - .iterator(); } @Override public boolean hasNext() { - return internal.hasNext(); + while (innerIter == null || !innerIter.hasNext()) { + if (!outerIter.hasNext()) { + return false; + } + + StateDescriptor descriptor = outerIter.next(); + Stream stream = backend.getKeys(descriptor.getName(), VoidNamespace.INSTANCE); + innerIter = stream.iterator(); + try { + registry.registerCloseable(stream::close); + } catch (IOException e) { + throw new RuntimeException("Failed to read keys from configured StateBackend", e); + } + } + return true; } @Override public K next() { - currentKey = internal.next(); - return currentKey; + if (!this.hasNext()) { + throw new NoSuchElementException(); + } else { + currentKey = this.innerIter.next(); + return currentKey; + } } /** Removes the current key from ALL known states in the state backend. */ diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java index 33fcf2cea816d..e891c5583ed69 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java @@ -18,28 +18,57 @@ package org.apache.flink.state.api.input; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.SavepointResources; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateSerializerProvider; +import org.apache.flink.runtime.state.StateSnapshotTransformer; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.heap.InternalKeyContext; +import org.apache.flink.runtime.state.heap.InternalKeyContextImpl; +import org.apache.flink.runtime.state.metrics.LatencyTrackingStateConfig; import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.runtime.state.ttl.mock.MockRestoreOperation; import org.apache.flink.runtime.state.ttl.mock.MockStateBackend; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nonnull; + import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.RunnableFuture; +import java.util.stream.IntStream; +import java.util.stream.Stream; /** Test for the multi-state key iterator. */ public class MultiStateKeyIteratorTest { @@ -68,6 +97,37 @@ private static AbstractKeyedStateBackend createKeyedStateBackend() { new CloseableRegistry()); } + private static CountingKeysKeyedStateBackend createCountingKeysKeyedStateBackend( + Integer numKeys) { + Environment env = new DummyEnvironment(); + TypeSerializer keySerializer = IntSerializer.INSTANCE; + int numberOfKeyGroups = 129; + KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 128); + TaskKvStateRegistry kvStateRegistry = null; + TtlTimeProvider ttlTimeProvider = TtlTimeProvider.DEFAULT; + @Nonnull Collection stateHandles = Collections.emptyList(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + + Map>> stateValues = new HashMap<>(); + MockRestoreOperation restoreOperation = + new MockRestoreOperation<>(stateHandles, stateValues); + restoreOperation.restore(); + + StateSerializerProvider keySerializerProvider = + StateSerializerProvider.fromNewRegisteredSerializer(keySerializer); + + return new CountingKeysKeyedStateBackend( + numKeys, + kvStateRegistry, + keySerializerProvider.currentSchemaSerializer(), + env.getUserCodeClassLoader().asClassLoader(), + env.getExecutionConfig(), + ttlTimeProvider, + LatencyTrackingStateConfig.disabled(), + cancelStreamRegistry, + new InternalKeyContextImpl<>(keyGroupRange, numberOfKeyGroups)); + } + private static void setKey( AbstractKeyedStateBackend backend, ValueStateDescriptor descriptor, @@ -79,6 +139,17 @@ private static void setKey( .update(0); } + private static void clearKey( + AbstractKeyedStateBackend backend, + ValueStateDescriptor descriptor, + int key) + throws Exception { + backend.setCurrentKey(key); + backend.getPartitionedState( + VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, descriptor) + .clear(); + } + @Test public void testIteratorPullsKeyFromAllDescriptors() throws Exception { AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend(); @@ -99,6 +170,36 @@ public void testIteratorPullsKeyFromAllDescriptors() throws Exception { Assert.assertEquals("Unexpected keys found", Arrays.asList(1, 2), keys); } + @Test + public void testIteratorSkipsEmptyDescriptors() throws Exception { + AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend(); + + List> threeDescriptors = new ArrayList<>(3); + threeDescriptors.add(new ValueStateDescriptor<>("state-1", Types.INT)); + threeDescriptors.add(new ValueStateDescriptor<>("state-2", Types.INT)); + threeDescriptors.add(new ValueStateDescriptor<>("state-3", Types.INT)); + + setKey(keyedStateBackend, threeDescriptors.get(0), 1); + + // initializes descriptor 1, but empties it immediately after + setKey(keyedStateBackend, threeDescriptors.get(1), 12); + clearKey(keyedStateBackend, threeDescriptors.get(1), 12); + + setKey(keyedStateBackend, threeDescriptors.get(2), 2); + + MultiStateKeyIterator iterator = + new MultiStateKeyIterator<>(threeDescriptors, keyedStateBackend); + + List keys = new ArrayList<>(); + + while (iterator.hasNext()) { + keys.add(iterator.next()); + } + + Assert.assertEquals("Unexpected number of keys", 2, keys.size()); + Assert.assertEquals("Unexpected keys found", Arrays.asList(1, 2), keys); + } + @Test public void testIteratorRemovesFromAllDescriptors() throws Exception { AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend(); @@ -125,4 +226,117 @@ public void testIteratorRemovesFromAllDescriptors() throws Exception { .count()); } } + + /** Test for lazy enumeration of inner iterators. */ + @Test + public void testIteratorPullsSingleKeyFromAllDescriptors() throws AssertionError { + CountingKeysKeyedStateBackend keyedStateBackend = + createCountingKeysKeyedStateBackend(100_000_000); + MultiStateKeyIterator testedIterator = + new MultiStateKeyIterator<>(descriptors, keyedStateBackend); + + testedIterator.hasNext(); + + Assert.assertEquals( + "Unexpected number of keys enumerated", + 1, + keyedStateBackend.numberOfKeysEnumerated); + } + + /** + * Mockup {@link AbstractKeyedStateBackend} that counts how many keys are enumerated. + * + *

Generates a configured number of integer keys, only method actually implemented is {@link + * CountingKeysKeyedStateBackend#getKeys(java.lang.String, java.lang.Object)} + */ + static class CountingKeysKeyedStateBackend extends AbstractKeyedStateBackend { + int numberOfKeysGenerated; + public long numberOfKeysEnumerated; + + public CountingKeysKeyedStateBackend( + int numberOfKeysGenerated, + TaskKvStateRegistry kvStateRegistry, + TypeSerializer keySerializer, + ClassLoader userCodeClassLoader, + ExecutionConfig executionConfig, + TtlTimeProvider ttlTimeProvider, + LatencyTrackingStateConfig latencyTrackingStateConfig, + CloseableRegistry cancelStreamRegistry, + InternalKeyContext keyContext) { + super( + kvStateRegistry, + keySerializer, + userCodeClassLoader, + executionConfig, + ttlTimeProvider, + latencyTrackingStateConfig, + cancelStreamRegistry, + keyContext); + this.numberOfKeysGenerated = numberOfKeysGenerated; + numberOfKeysEnumerated = 0; + } + + @Override + public Stream getKeys(String state, N namespace) { + return IntStream.range(0, this.numberOfKeysGenerated) + .boxed() + .peek(i -> numberOfKeysEnumerated++); + } + + @Override + public int numKeyValueStateEntries() { + return numberOfKeysGenerated; + } + + @Nonnull + @Override + public & Keyed> + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer) { + throw new UnsupportedOperationException( + "Operations other than getKeys() are not supported on this testing StateBackend."); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + @Nonnull + @Override + public RunnableFuture> snapshot( + long checkpointId, + long timestamp, + @Nonnull CheckpointStreamFactory streamFactory, + @Nonnull CheckpointOptions checkpointOptions) + throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Operations other than getKeys() are not supported on this testing StateBackend."); + } + + @Nonnull + @Override + public IS createOrUpdateInternalState( + @Nonnull TypeSerializer namespaceSerializer, + @Nonnull StateDescriptor stateDesc, + @Nonnull + StateSnapshotTransformer.StateSnapshotTransformFactory + snapshotTransformFactory) + throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Operations other than getKeys() are not supported on this testing StateBackend."); + } + + @Override + public Stream> getKeysAndNamespaces(String state) { + throw new UnsupportedOperationException( + "Operations other than getKeys() are not supported on this testing StateBackend."); + } + + @Nonnull + @Override + public SavepointResources savepoint() throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Operations other than getKeys() are not supported on this testing StateBackend."); + } + } } From 55162dcc5cca6db6aeedddb30d80dd9f9b8d5202 Mon Sep 17 00:00:00 2001 From: Zhanghao Chen Date: Sat, 4 Nov 2023 21:00:25 +0800 Subject: [PATCH 093/104] [FLINK-33262][table-api] Extend source provider interfaces with the new parallelism provider interface Close apache/flink#23663 --- .../source/DataStreamScanProvider.java | 4 +++- .../source/SourceFunctionProvider.java | 21 ++++++++++++++++++- .../table/connector/ParallelismProvider.java | 12 +++++------ .../connector/source/InputFormatProvider.java | 21 ++++++++++++++++++- .../connector/source/SourceProvider.java | 18 +++++++++++++++- .../flink/table/factories/FactoryUtil.java | 9 ++++++++ .../TransformationScanProvider.java | 4 +++- 7 files changed, 78 insertions(+), 11 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/DataStreamScanProvider.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/DataStreamScanProvider.java index 7fc4687363f28..213e3806327a1 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/DataStreamScanProvider.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/DataStreamScanProvider.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.connector.ParallelismProvider; import org.apache.flink.table.connector.ProviderContext; import org.apache.flink.table.data.RowData; @@ -35,7 +36,8 @@ * or {@link InputFormatProvider}. */ @PublicEvolving -public interface DataStreamScanProvider extends ScanTableSource.ScanRuntimeProvider { +public interface DataStreamScanProvider + extends ScanTableSource.ScanRuntimeProvider, ParallelismProvider { /** * Creates a scan Java {@link DataStream} from a {@link StreamExecutionEnvironment}. diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/SourceFunctionProvider.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/SourceFunctionProvider.java index ff7238d8a2f7c..e5c35525e162d 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/SourceFunctionProvider.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/source/SourceFunctionProvider.java @@ -20,8 +20,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.connector.ParallelismProvider; import org.apache.flink.table.data.RowData; +import javax.annotation.Nullable; + +import java.util.Optional; + /** * Provider of a {@link SourceFunction} instance as a runtime implementation for {@link * ScanTableSource}. @@ -32,10 +37,19 @@ */ @Deprecated @PublicEvolving -public interface SourceFunctionProvider extends ScanTableSource.ScanRuntimeProvider { +public interface SourceFunctionProvider + extends ScanTableSource.ScanRuntimeProvider, ParallelismProvider { /** Helper method for creating a static provider. */ static SourceFunctionProvider of(SourceFunction sourceFunction, boolean isBounded) { + return of(sourceFunction, isBounded, null); + } + + /** Helper method for creating a Source provider with a provided source parallelism. */ + static SourceFunctionProvider of( + SourceFunction sourceFunction, + boolean isBounded, + @Nullable Integer sourceParallelism) { return new SourceFunctionProvider() { @Override public SourceFunction createSourceFunction() { @@ -46,6 +60,11 @@ public SourceFunction createSourceFunction() { public boolean isBounded() { return isBounded; } + + @Override + public Optional getParallelism() { + return Optional.ofNullable(sourceParallelism); + } }; } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/ParallelismProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/ParallelismProvider.java index f9c4684383a49..27e4047a016d6 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/ParallelismProvider.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/ParallelismProvider.java @@ -20,14 +20,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider; +import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider; import java.util.Optional; /** * Parallelism provider for other connector providers. It allows to express a custom parallelism for - * the connector runtime implementation. Otherwise the parallelism is determined by the planner. - * - *

Note: Currently, this interface only works with {@link SinkRuntimeProvider}. + * the connector runtime implementation. Otherwise, the parallelism is determined by the planner. */ @PublicEvolving public interface ParallelismProvider { @@ -38,9 +37,10 @@ public interface ParallelismProvider { *

The parallelism denotes how many parallel instances of a source or sink will be spawned * during the execution. * - *

Enforcing a different parallelism for sinks might mess up the changelog if the input is - * not {@link ChangelogMode#insertOnly()}. Therefore, a primary key is required by which the - * input will be shuffled before records enter the {@link SinkRuntimeProvider} implementation. + *

Enforcing a different parallelism for sources/sinks might mess up the changelog if the + * output/input is not {@link ChangelogMode#insertOnly()}. Therefore, a primary key is required + * by which the output/input will be shuffled after/before records leave/enter the {@link + * ScanRuntimeProvider}/{@link SinkRuntimeProvider} implementation. * * @return empty if the connector does not provide a custom parallelism, then the planner will * decide the number of parallel instances by itself. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/InputFormatProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/InputFormatProvider.java index bbf20e1b1930b..a9775becc33c9 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/InputFormatProvider.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/InputFormatProvider.java @@ -20,18 +20,32 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.table.connector.ParallelismProvider; import org.apache.flink.table.data.RowData; +import javax.annotation.Nullable; + +import java.util.Optional; + /** * Provider of an {@link InputFormat} instance as a runtime implementation for {@link * ScanTableSource}. */ @PublicEvolving -public interface InputFormatProvider extends ScanTableSource.ScanRuntimeProvider { +public interface InputFormatProvider + extends ScanTableSource.ScanRuntimeProvider, ParallelismProvider { /** Helper method for creating a static provider. */ static InputFormatProvider of(InputFormat inputFormat) { + return of(inputFormat, null); + } + + /** Helper method for creating a static provider with a provided source parallelism. */ + static InputFormatProvider of( + InputFormat inputFormat, @Nullable Integer sourceParallelism) { + return new InputFormatProvider() { + @Override public InputFormat createInputFormat() { return inputFormat; @@ -41,6 +55,11 @@ static InputFormatProvider of(InputFormat inputFormat) { public boolean isBounded() { return true; } + + @Override + public Optional getParallelism() { + return Optional.ofNullable(sourceParallelism); + } }; } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/SourceProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/SourceProvider.java index f0e85f88624d1..2d9f5143626cd 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/SourceProvider.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/SourceProvider.java @@ -21,8 +21,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; +import org.apache.flink.table.connector.ParallelismProvider; import org.apache.flink.table.data.RowData; +import javax.annotation.Nullable; + +import java.util.Optional; + /** * Provider of a {@link Source} instance as a runtime implementation for {@link ScanTableSource}. * @@ -30,11 +35,17 @@ * advanced connector developers. */ @PublicEvolving -public interface SourceProvider extends ScanTableSource.ScanRuntimeProvider { +public interface SourceProvider extends ScanTableSource.ScanRuntimeProvider, ParallelismProvider { /** Helper method for creating a static provider. */ static SourceProvider of(Source source) { + return of(source, null); + } + + /** Helper method for creating a Source provider with a provided source parallelism. */ + static SourceProvider of(Source source, @Nullable Integer sourceParallelism) { return new SourceProvider() { + @Override public Source createSource() { return source; @@ -44,6 +55,11 @@ static SourceProvider of(Source source) { public boolean isBounded() { return Boundedness.BOUNDED.equals(source.getBoundedness()); } + + @Override + public Optional getParallelism() { + return Optional.ofNullable(sourceParallelism); + } }; } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java index 8f103be0c7e25..d8d6d7e90008d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java @@ -120,6 +120,15 @@ public final class FactoryUtil { .defaultValues("rest") .withDescription("Specify the endpoints that are used."); + public static final ConfigOption SOURCE_PARALLELISM = + ConfigOptions.key("scan.parallelism") + .intType() + .noDefaultValue() + .withDescription( + "Defines a custom parallelism for the source. " + + "By default, if this option is not defined, the planner will derive the parallelism " + + "for each statement individually by also considering the global configuration."); + public static final ConfigOption WATERMARK_EMIT_STRATEGY = ConfigOptions.key("scan.watermark.emit.strategy") .enumType(WatermarkEmitStrategy.class) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationScanProvider.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationScanProvider.java index e6642bc2ab112..46e739ec54d89 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationScanProvider.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationScanProvider.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.dag.Transformation; import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.connector.ParallelismProvider; import org.apache.flink.table.connector.ProviderContext; import org.apache.flink.table.connector.source.InputFormatProvider; import org.apache.flink.table.connector.source.ScanTableSource; @@ -37,7 +38,8 @@ * SourceFunctionProvider}, or {@link SourceProvider}. */ @Internal -public interface TransformationScanProvider extends ScanTableSource.ScanRuntimeProvider { +public interface TransformationScanProvider + extends ScanTableSource.ScanRuntimeProvider, ParallelismProvider { /** * Creates a {@link Transformation} instance. From f77dbdfd0c7d510ae32ce932c7b84ed968496130 Mon Sep 17 00:00:00 2001 From: Zhanghao Chen Date: Thu, 9 Nov 2023 05:19:07 -0600 Subject: [PATCH 094/104] [FLINK-33147] Introduce endpoint field in REST API and deprecate host field (#23465) --- .../generated/rest_v1_dispatcher.html | 27 +++++++++++ docs/static/generated/rest_v1_dispatcher.yml | 14 ++++++ .../src/test/resources/rest_api_v1.snapshot | 27 +++++++++++ .../handler/job/JobExceptionsHandler.java | 14 ++---- .../job/JobVertexTaskManagersHandler.java | 27 ++++++----- .../job/SubtasksAllAccumulatorsHandler.java | 6 ++- .../handler/job/SubtasksTimesHandler.java | 5 +- .../rest/messages/JobExceptionsInfo.java | 21 ++++++++- .../JobExceptionsInfoWithHistory.java | 47 ++++++++++++++++--- .../messages/JobVertexTaskManagersInfo.java | 10 +++- .../rest/messages/SubtasksTimesInfo.java | 11 ++++- .../SubtaskExecutionAttemptDetailsInfo.java | 22 +++++++-- .../job/SubtasksAllAccumulatorsInfo.java | 11 ++++- .../ExceptionHistoryEntry.java | 4 ++ .../taskmanager/TaskManagerLocation.java | 9 ++++ ...btaskCurrentAttemptDetailsHandlerTest.java | 1 + ...askExecutionAttemptDetailsHandlerTest.java | 1 + .../AggregatedTaskDetailsInfoTest.java | 1 + ...obExceptionsInfoWithHistoryNoRootTest.java | 2 + .../messages/JobVertexDetailsInfoTest.java | 4 ++ .../JobVertexTaskManagersInfoTest.java | 1 + .../rest/messages/SubtasksTimesInfoTest.java | 9 ++-- ...ubtaskExecutionAttemptDetailsInfoTest.java | 1 + .../job/SubtasksAllAccumulatorsInfoTest.java | 2 +- 24 files changed, 230 insertions(+), 47 deletions(-) diff --git a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html index ae68e8c83838d..a96b18f5d5134 100644 --- a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html +++ b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html @@ -2524,6 +2524,9 @@ "type" : "object", "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobExceptionsInfo:ExecutionExceptionInfo", "properties" : { + "endpoint" : { + "type" : "string" + }, "exception" : { "type" : "string" }, @@ -2558,6 +2561,9 @@ "type" : "object", "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobExceptionsInfoWithHistory:ExceptionInfo", "properties" : { + "endpoint" : { + "type" : "string" + }, "exceptionName" : { "type" : "string" }, @@ -2585,6 +2591,9 @@ } } }, + "endpoint" : { + "type" : "string" + }, "exceptionName" : { "type" : "string" }, @@ -3635,6 +3644,9 @@ "end-time" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, @@ -4079,6 +4091,9 @@ "attempt" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, @@ -4221,6 +4236,9 @@ "end-time" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, @@ -4351,6 +4369,9 @@ "end-time" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, @@ -4621,6 +4642,9 @@ "duration" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, @@ -4729,6 +4753,9 @@ "end-time" : { "type" : "integer" }, + "endpoint" : { + "type" : "string" + }, "host" : { "type" : "string" }, diff --git a/docs/static/generated/rest_v1_dispatcher.yml b/docs/static/generated/rest_v1_dispatcher.yml index 4c5bdd89d2918..62d298241c112 100644 --- a/docs/static/generated/rest_v1_dispatcher.yml +++ b/docs/static/generated/rest_v1_dispatcher.yml @@ -2022,6 +2022,8 @@ components: ExceptionInfo: type: object properties: + endpoint: + type: string exceptionName: type: string failureLabels: @@ -2058,6 +2060,8 @@ components: ExecutionExceptionInfo: type: object properties: + endpoint: + type: string exception: type: string location: @@ -2593,6 +2597,8 @@ components: end-time: type: integer format: int64 + endpoint: + type: string host: type: string metrics: @@ -2782,6 +2788,8 @@ components: type: array items: $ref: '#/components/schemas/ExceptionInfo' + endpoint: + type: string exceptionName: type: string failureLabels: @@ -2892,6 +2900,8 @@ components: attempt: type: integer format: int32 + endpoint: + type: string host: type: string subtask: @@ -2969,6 +2979,8 @@ components: end-time: type: integer format: int64 + endpoint: + type: string host: type: string metrics: @@ -2998,6 +3010,8 @@ components: duration: type: integer format: int64 + endpoint: + type: string host: type: string subtask: diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 62f8166917c1f..d6ba3eb2f5cd1 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -1971,6 +1971,9 @@ "location" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "timestamp" : { "type" : "integer" }, @@ -2014,6 +2017,9 @@ "location" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "taskManagerId" : { "type" : "string" }, @@ -2044,6 +2050,9 @@ "location" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "taskManagerId" : { "type" : "string" } @@ -2598,6 +2607,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "start-time" : { "type" : "integer" }, @@ -2941,6 +2953,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "user-accumulators" : { "type" : "array", "items" : { @@ -3033,6 +3048,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "start-time" : { "type" : "integer" }, @@ -3142,6 +3160,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "start-time" : { "type" : "integer" }, @@ -3338,6 +3359,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "duration" : { "type" : "integer" }, @@ -3393,6 +3417,9 @@ "host" : { "type" : "string" }, + "endpoint" : { + "type" : "string" + }, "status" : { "type" : "string", "enum" : [ "CREATED", "SCHEDULED", "DEPLOYING", "RUNNING", "FINISHED", "CANCELING", "CANCELED", "FAILED", "RECONCILING", "INITIALIZING" ] diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java index 5ece82a267173..55c7875e85cf4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java @@ -242,6 +242,7 @@ private static JobExceptionsInfoWithHistory.RootExceptionInfo createRootExceptio historyEntry.getFailureLabels(), historyEntry.getFailingTaskName(), toString(historyEntry.getTaskManagerLocation()), + toString(historyEntry.getTaskManagerLocation()), toTaskManagerId(historyEntry.getTaskManagerLocation()), concurrentExceptions); } @@ -257,6 +258,7 @@ private static JobExceptionsInfoWithHistory.ExceptionInfo createExceptionInfo( exceptionHistoryEntry.getFailureLabels(), exceptionHistoryEntry.getFailingTaskName(), toString(exceptionHistoryEntry.getTaskManagerLocation()), + toString(exceptionHistoryEntry.getTaskManagerLocation()), toTaskManagerId(exceptionHistoryEntry.getTaskManagerLocation())); } @@ -270,9 +272,7 @@ private static void assertLocalExceptionInfo(ExceptionHistoryEntry exceptionHist static String toString(@Nullable TaskManagerLocation location) { // '(unassigned)' being the default value is added to support backward-compatibility for the // deprecated fields - return location != null - ? taskManagerLocationToString(location.getFQDNHostname(), location.dataPort()) - : "(unassigned)"; + return location != null ? location.getEndpoint() : "(unassigned)"; } @VisibleForTesting @@ -285,9 +285,7 @@ static String toTaskManagerId(@Nullable TaskManagerLocation location) { @VisibleForTesting @Nullable static String toString(@Nullable ExceptionHistoryEntry.ArchivedTaskManagerLocation location) { - return location != null - ? taskManagerLocationToString(location.getFQDNHostname(), location.getPort()) - : null; + return location != null ? location.getEndpoint() : null; } @VisibleForTesting @@ -295,8 +293,4 @@ static String toTaskManagerId( @Nullable ExceptionHistoryEntry.ArchivedTaskManagerLocation location) { return location != null ? String.format("%s", location.getResourceID()) : null; } - - private static String taskManagerLocationToString(String fqdnHostname, int port) { - return String.format("%s:%d", fqdnHostname, port); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java index 0ed6565ac2452..948bdcb1de8f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java @@ -132,8 +132,7 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo( JobID jobID, @Nullable MetricFetcher metricFetcher) { // Build a map that groups task executions by TaskManager - Map taskManagerId2Host = new HashMap<>(); - Map> taskManagerExecutions = new HashMap<>(); + Map> taskManagerExecutions = new HashMap<>(); Set representativeExecutions = new HashSet<>(); for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) { AccessExecution representativeAttempt = vertex.getCurrentExecutionAttempt(); @@ -141,16 +140,9 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo( for (AccessExecution execution : vertex.getCurrentExecutions()) { TaskManagerLocation location = execution.getAssignedResourceLocation(); - String taskManagerHost = - location == null - ? "(unassigned)" - : location.getHostname() + ':' + location.dataPort(); - String taskmanagerId = - location == null ? "(unassigned)" : location.getResourceID().toString(); - taskManagerId2Host.put(taskmanagerId, taskManagerHost); List executions = taskManagerExecutions.computeIfAbsent( - taskmanagerId, ignored -> new ArrayList<>()); + location, ignored -> new ArrayList<>()); executions.add(execution); } } @@ -158,9 +150,17 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo( final long now = System.currentTimeMillis(); List taskManagersInfoList = new ArrayList<>(4); - for (Map.Entry> entry : taskManagerExecutions.entrySet()) { - String taskmanagerId = entry.getKey(); - String host = taskManagerId2Host.get(taskmanagerId); + for (Map.Entry> entry : + taskManagerExecutions.entrySet()) { + TaskManagerLocation location = entry.getKey(); + // Port information is included in the host field for backward-compatibility + String host = + location == null + ? "(unassigned)" + : location.getHostname() + ':' + location.dataPort(); + String endpoint = location == null ? "(unassigned)" : location.getEndpoint(); + String taskmanagerId = + location == null ? "(unassigned)" : location.getResourceID().toString(); List executions = entry.getValue(); List ioMetricsInfos = new ArrayList<>(); @@ -266,6 +266,7 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo( taskManagersInfoList.add( new JobVertexTaskManagersInfo.TaskManagersInfo( host, + endpoint, jobVertexState, startTime, endTime, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java index a47ad6a46be44..4db45a2f3e418 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java @@ -78,7 +78,8 @@ protected SubtasksAllAccumulatorsInfo handleRequest( for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) { for (AccessExecution execution : vertex.getCurrentExecutions()) { TaskManagerLocation location = execution.getAssignedResourceLocation(); - String locationString = location == null ? "(unassigned)" : location.getHostname(); + String host = location == null ? "(unassigned)" : location.getHostname(); + String endpoint = location == null ? "(unassigned)" : location.getEndpoint(); StringifiedAccumulatorResult[] accs = execution.getUserAccumulatorsStringified(); List userAccumulators = new ArrayList<>(accs.length); @@ -91,7 +92,8 @@ protected SubtasksAllAccumulatorsInfo handleRequest( new SubtasksAllAccumulatorsInfo.SubtaskAccumulatorsInfo( execution.getParallelSubtaskIndex(), execution.getAttemptNumber(), - locationString, + host, + endpoint, userAccumulators)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java index f097bbfbb4530..34e3fed0673ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java @@ -113,7 +113,8 @@ private static SubtasksTimesInfo createSubtaskTimesInfo(AccessExecutionJobVertex long duration = start >= 0 ? end - start : -1L; TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); - String locationString = location == null ? "(unassigned)" : location.getHostname(); + String host = location == null ? "(unassigned)" : location.getHostname(); + String endpoint = location == null ? "(unassigned)" : location.getEndpoint(); Map timestampMap = CollectionUtil.newHashMapWithExpectedSize(ExecutionState.values().length); @@ -123,7 +124,7 @@ private static SubtasksTimesInfo createSubtaskTimesInfo(AccessExecutionJobVertex subtasks.add( new SubtasksTimesInfo.SubtaskTimeInfo( - num++, locationString, duration, timestampMap)); + num++, host, endpoint, duration, timestampMap)); } return new SubtasksTimesInfo(id, name, now, subtasks); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java index 2bd3fd46bfc39..0acc64d4b47dc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java @@ -144,7 +144,8 @@ public boolean isTruncated() { public static final class ExecutionExceptionInfo { public static final String FIELD_NAME_EXCEPTION = "exception"; public static final String FIELD_NAME_TASK = "task"; - public static final String FIELD_NAME_LOCATION = "location"; + @Deprecated public static final String FIELD_NAME_LOCATION = "location"; + public static final String FIELD_NAME_ENDPOINT = "endpoint"; public static final String FIELD_NAME_TIMESTAMP = "timestamp"; public static final String FIELD_NAME_TASK_MANAGER_ID = "taskManagerId"; @@ -157,22 +158,36 @@ public static final class ExecutionExceptionInfo { @JsonProperty(FIELD_NAME_LOCATION) private final String location; + @JsonProperty(FIELD_NAME_ENDPOINT) + private final String endpoint; + @JsonProperty(FIELD_NAME_TIMESTAMP) private final long timestamp; @JsonProperty(FIELD_NAME_TASK_MANAGER_ID) private final String taskManagerId; + public ExecutionExceptionInfo( + String exception, + String task, + String endpoint, + long timestamp, + String taskManagerId) { + this(exception, task, endpoint, endpoint, timestamp, taskManagerId); + } + @JsonCreator public ExecutionExceptionInfo( @JsonProperty(FIELD_NAME_EXCEPTION) String exception, @JsonProperty(FIELD_NAME_TASK) String task, @JsonProperty(FIELD_NAME_LOCATION) String location, + @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint, @JsonProperty(FIELD_NAME_TIMESTAMP) long timestamp, @JsonProperty(FIELD_NAME_TASK_MANAGER_ID) String taskManagerId) { this.exception = Preconditions.checkNotNull(exception); this.task = Preconditions.checkNotNull(task); this.location = Preconditions.checkNotNull(location); + this.endpoint = Preconditions.checkNotNull(endpoint); this.timestamp = timestamp; this.taskManagerId = taskManagerId; } @@ -191,12 +206,13 @@ public boolean equals(Object o) { && Objects.equals(exception, that.exception) && Objects.equals(task, that.task) && Objects.equals(location, that.location) + && Objects.equals(endpoint, that.endpoint) && Objects.equals(taskManagerId, that.taskManagerId); } @Override public int hashCode() { - return Objects.hash(timestamp, exception, task, location, taskManagerId); + return Objects.hash(timestamp, exception, task, location, endpoint, taskManagerId); } @Override @@ -205,6 +221,7 @@ public String toString() { .add("exception='" + exception + "'") .add("task='" + task + "'") .add("location='" + location + "'") + .add("endpoint='" + endpoint + "'") .add("timestamp=" + timestamp) .add("taskManagerId=" + taskManagerId) .toString(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java index dd6b264386e9b..cb15825cc6bd6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java @@ -176,7 +176,8 @@ public static class ExceptionInfo { public static final String FIELD_NAME_EXCEPTION_STACKTRACE = "stacktrace"; public static final String FIELD_NAME_EXCEPTION_TIMESTAMP = "timestamp"; public static final String FIELD_NAME_TASK_NAME = "taskName"; - public static final String FIELD_NAME_LOCATION = "location"; + @Deprecated public static final String FIELD_NAME_LOCATION = "location"; + public static final String FIELD_NAME_ENDPOINT = "endpoint"; public static final String FIELD_NAME_TASK_MANAGER_ID = "taskManagerId"; public static final String FIELD_NAME_FAILURE_LABELS = "failureLabels"; @@ -194,11 +195,18 @@ public static class ExceptionInfo { @Nullable private final String taskName; + /** @deprecated Use {@link ExceptionInfo#endpoint} instead. */ + @Deprecated @JsonInclude(NON_NULL) @JsonProperty(FIELD_NAME_LOCATION) @Nullable private final String location; + @JsonInclude(NON_NULL) + @JsonProperty(FIELD_NAME_ENDPOINT) + @Nullable + private final String endpoint; + @JsonInclude(NON_NULL) @JsonProperty(FIELD_NAME_TASK_MANAGER_ID) @Nullable @@ -208,7 +216,15 @@ public static class ExceptionInfo { private final Map failureLabels; public ExceptionInfo(String exceptionName, String stacktrace, long timestamp) { - this(exceptionName, stacktrace, timestamp, Collections.emptyMap(), null, null, null); + this( + exceptionName, + stacktrace, + timestamp, + Collections.emptyMap(), + null, + null, + null, + null); } @JsonCreator @@ -219,6 +235,7 @@ public ExceptionInfo( @JsonProperty(FIELD_NAME_FAILURE_LABELS) Map failureLabels, @JsonProperty(FIELD_NAME_TASK_NAME) @Nullable String taskName, @JsonProperty(FIELD_NAME_LOCATION) @Nullable String location, + @JsonProperty(FIELD_NAME_ENDPOINT) @Nullable String endpoint, @JsonProperty(FIELD_NAME_TASK_MANAGER_ID) @Nullable String taskManagerId) { this.exceptionName = checkNotNull(exceptionName); this.stacktrace = checkNotNull(stacktrace); @@ -226,6 +243,7 @@ public ExceptionInfo( this.failureLabels = checkNotNull(failureLabels); this.taskName = taskName; this.location = location; + this.endpoint = endpoint; this.taskManagerId = taskManagerId; } @@ -250,12 +268,19 @@ public String getTaskName() { return taskName; } + @Deprecated @JsonIgnore @Nullable public String getLocation() { return location; } + @JsonIgnore + @Nullable + public String getEndpoint() { + return endpoint; + } + @JsonIgnore @Nullable public String getTaskManagerId() { @@ -283,13 +308,20 @@ public boolean equals(Object o) { && Objects.equals(timestamp, that.timestamp) && Objects.equals(failureLabels, that.failureLabels) && Objects.equals(taskName, that.taskName) - && Objects.equals(location, that.location); + && Objects.equals(location, that.location) + && Objects.equals(location, that.endpoint); } @Override public int hashCode() { return Objects.hash( - exceptionName, stacktrace, timestamp, failureLabels, taskName, location); + exceptionName, + stacktrace, + timestamp, + failureLabels, + taskName, + location, + endpoint); } @Override @@ -300,7 +332,7 @@ public String toString() { .add("timestamp=" + timestamp) .add("failureLabels=" + failureLabels) .add("taskName='" + taskName + "'") - .add("location='" + location + "'") + .add("endpoint='" + endpoint + "'") .toString(); } } @@ -330,6 +362,7 @@ public RootExceptionInfo( null, null, null, + null, concurrentExceptions); } @@ -341,6 +374,7 @@ public RootExceptionInfo( @JsonProperty(FIELD_NAME_FAILURE_LABELS) Map failureLabels, @JsonProperty(FIELD_NAME_TASK_NAME) @Nullable String taskName, @JsonProperty(FIELD_NAME_LOCATION) @Nullable String location, + @JsonProperty(FIELD_NAME_ENDPOINT) @Nullable String endpoint, @JsonProperty(FIELD_NAME_TASK_MANAGER_ID) @Nullable String taskManagerId, @JsonProperty(FIELD_NAME_CONCURRENT_EXCEPTIONS) Collection concurrentExceptions) { @@ -351,6 +385,7 @@ public RootExceptionInfo( failureLabels, taskName, location, + endpoint, taskManagerId); this.concurrentExceptions = concurrentExceptions; } @@ -386,7 +421,7 @@ public String toString() { .add("stacktrace='" + getStacktrace() + "'") .add("timestamp=" + getTimestamp()) .add("taskName='" + getTaskName() + "'") - .add("location='" + getLocation() + "'") + .add("endpoint='" + getEndpoint() + "'") .add("concurrentExceptions=" + getConcurrentExceptions()) .toString(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java index de6a388292779..c0b2e34dfe2b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java @@ -100,7 +100,8 @@ public int hashCode() { /** Detailed information about task managers. */ @Schema(name = "JobVertexTaskManagerInfo") public static class TaskManagersInfo { - public static final String TASK_MANAGERS_FIELD_HOST = "host"; + @Deprecated public static final String TASK_MANAGERS_FIELD_HOST = "host"; + public static final String TASK_MANAGERS_FIELD_ENDPOINT = "endpoint"; public static final String TASK_MANAGERS_FIELD_STATUS = "status"; public static final String TASK_MANAGERS_FIELD_START_TIME = "start-time"; public static final String TASK_MANAGERS_FIELD_END_TIME = "end-time"; @@ -113,6 +114,9 @@ public static class TaskManagersInfo { @JsonProperty(TASK_MANAGERS_FIELD_HOST) private final String host; + @JsonProperty(TASK_MANAGERS_FIELD_ENDPOINT) + private final String endpoint; + @JsonProperty(TASK_MANAGERS_FIELD_STATUS) private final ExecutionState status; @@ -140,6 +144,7 @@ public static class TaskManagersInfo { @JsonCreator public TaskManagersInfo( @JsonProperty(TASK_MANAGERS_FIELD_HOST) String host, + @JsonProperty(TASK_MANAGERS_FIELD_ENDPOINT) String endpoint, @JsonProperty(TASK_MANAGERS_FIELD_STATUS) ExecutionState status, @JsonProperty(TASK_MANAGERS_FIELD_START_TIME) long startTime, @JsonProperty(TASK_MANAGERS_FIELD_END_TIME) long endTime, @@ -151,6 +156,7 @@ public TaskManagersInfo( @JsonProperty(TASK_MANAGERS_FIELD_AGGREGATED) AggregatedTaskDetailsInfo aggregated) { this.host = checkNotNull(host); + this.endpoint = checkNotNull(endpoint); this.status = checkNotNull(status); this.startTime = startTime; this.endTime = endTime; @@ -171,6 +177,7 @@ public boolean equals(Object o) { } TaskManagersInfo that = (TaskManagersInfo) o; return Objects.equals(host, that.host) + && Objects.equals(endpoint, that.endpoint) && Objects.equals(status, that.status) && startTime == that.startTime && endTime == that.endTime @@ -185,6 +192,7 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash( host, + endpoint, status, startTime, endTime, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java index 7da481f2ad9f8..67cf50778e8cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java @@ -92,7 +92,8 @@ public int hashCode() { public static final class SubtaskTimeInfo { public static final String FIELD_NAME_SUBTASK = "subtask"; - public static final String FIELD_NAME_HOST = "host"; + @Deprecated public static final String FIELD_NAME_HOST = "host"; + public static final String FIELD_NAME_ENDPOINT = "endpoint"; public static final String FIELD_NAME_DURATION = "duration"; public static final String FIELD_NAME_TIMESTAMPS = "timestamps"; @@ -102,6 +103,9 @@ public static final class SubtaskTimeInfo { @JsonProperty(FIELD_NAME_HOST) private final String host; + @JsonProperty(FIELD_NAME_ENDPOINT) + private final String endpoint; + @JsonProperty(FIELD_NAME_DURATION) private final long duration; @@ -111,10 +115,12 @@ public static final class SubtaskTimeInfo { public SubtaskTimeInfo( @JsonProperty(FIELD_NAME_SUBTASK) int subtask, @JsonProperty(FIELD_NAME_HOST) String host, + @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint, @JsonProperty(FIELD_NAME_DURATION) long duration, @JsonProperty(FIELD_NAME_TIMESTAMPS) Map timestamps) { this.subtask = subtask; this.host = checkNotNull(host); + this.endpoint = checkNotNull(endpoint); this.duration = duration; this.timestamps = checkNotNull(timestamps); } @@ -132,13 +138,14 @@ public boolean equals(Object o) { SubtaskTimeInfo that = (SubtaskTimeInfo) o; return subtask == that.subtask && Objects.equals(host, that.host) + && Objects.equals(endpoint, that.endpoint) && duration == that.duration && Objects.equals(timestamps, that.timestamps); } @Override public int hashCode() { - return Objects.hash(subtask, host, duration, timestamps); + return Objects.hash(subtask, host, endpoint, duration, timestamps); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java index 1a8463ab5597c..df0fed46d0b94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java @@ -54,7 +54,9 @@ public class SubtaskExecutionAttemptDetailsInfo implements ResponseBody { public static final String FIELD_NAME_ATTEMPT = "attempt"; - public static final String FIELD_NAME_HOST = "host"; + @Deprecated public static final String FIELD_NAME_HOST = "host"; + + public static final String FIELD_NAME_ENDPOINT = "endpoint"; public static final String FIELD_NAME_START_TIME = "start-time"; @@ -84,6 +86,9 @@ public class SubtaskExecutionAttemptDetailsInfo implements ResponseBody { @JsonProperty(FIELD_NAME_HOST) private final String host; + @JsonProperty(FIELD_NAME_ENDPOINT) + private final String endpoint; + @JsonProperty(FIELD_NAME_START_TIME) private final long startTime; @@ -118,6 +123,7 @@ public SubtaskExecutionAttemptDetailsInfo( @JsonProperty(FIELD_NAME_STATUS) ExecutionState status, @JsonProperty(FIELD_NAME_ATTEMPT) int attempt, @JsonProperty(FIELD_NAME_HOST) String host, + @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint, @JsonProperty(FIELD_NAME_START_TIME) long startTime, @JsonProperty(FIELD_NAME_END_TIME) long endTime, @JsonProperty(FIELD_NAME_DURATION) long duration, @@ -131,6 +137,7 @@ public SubtaskExecutionAttemptDetailsInfo( this.status = Preconditions.checkNotNull(status); this.attempt = attempt; this.host = Preconditions.checkNotNull(host); + this.endpoint = Preconditions.checkNotNull(endpoint); this.startTime = startTime; this.startTimeCompatible = startTime; this.endTime = endTime; @@ -153,10 +160,15 @@ public int getAttempt() { return attempt; } + @Deprecated public String getHost() { return host; } + public String getEndpoint() { + return endpoint; + } + public long getStartTime() { return startTime; } @@ -203,7 +215,8 @@ public static SubtaskExecutionAttemptDetailsInfo create( final long now = System.currentTimeMillis(); final TaskManagerLocation location = execution.getAssignedResourceLocation(); - final String locationString = location == null ? "(unassigned)" : location.getHostname(); + final String host = location == null ? "(unassigned)" : location.getHostname(); + final String endpoint = location == null ? "(unassigned)" : location.getEndpoint(); String taskmanagerId = location == null ? "(unassigned)" : location.getResourceID().toString(); @@ -235,7 +248,8 @@ public static SubtaskExecutionAttemptDetailsInfo create( execution.getParallelSubtaskIndex(), status, execution.getAttemptNumber(), - locationString, + host, + endpoint, startTime, endTime, duration, @@ -260,6 +274,7 @@ public boolean equals(Object o) { && status == that.status && attempt == that.attempt && Objects.equals(host, that.host) + && Objects.equals(endpoint, that.endpoint) && startTime == that.startTime && startTimeCompatible == that.startTimeCompatible && endTime == that.endTime @@ -277,6 +292,7 @@ public int hashCode() { status, attempt, host, + endpoint, startTime, startTimeCompatible, endTime, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java index e245a19388b53..8035ff2a35512 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java @@ -90,7 +90,8 @@ public int hashCode() { public static class SubtaskAccumulatorsInfo { public static final String FIELD_NAME_SUBTASK_INDEX = "subtask"; public static final String FIELD_NAME_ATTEMPT_NUM = "attempt"; - public static final String FIELD_NAME_HOST = "host"; + @Deprecated public static final String FIELD_NAME_HOST = "host"; + public static final String FIELD_NAME_ENDPOINT = "endpoint"; public static final String FIELD_NAME_USER_ACCUMULATORS = "user-accumulators"; @JsonProperty(FIELD_NAME_SUBTASK_INDEX) @@ -102,6 +103,9 @@ public static class SubtaskAccumulatorsInfo { @JsonProperty(FIELD_NAME_HOST) private final String host; + @JsonProperty(FIELD_NAME_ENDPOINT) + private final String endpoint; + @JsonProperty(FIELD_NAME_USER_ACCUMULATORS) private final Collection userAccumulators; @@ -110,12 +114,14 @@ public SubtaskAccumulatorsInfo( @JsonProperty(FIELD_NAME_SUBTASK_INDEX) int subtaskIndex, @JsonProperty(FIELD_NAME_ATTEMPT_NUM) int attemptNum, @JsonProperty(FIELD_NAME_HOST) String host, + @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint, @JsonProperty(FIELD_NAME_USER_ACCUMULATORS) Collection userAccumulators) { this.subtaskIndex = subtaskIndex; this.attemptNum = attemptNum; this.host = Preconditions.checkNotNull(host); + this.endpoint = Preconditions.checkNotNull(endpoint); this.userAccumulators = Preconditions.checkNotNull(userAccumulators); } @@ -131,12 +137,13 @@ public boolean equals(Object o) { return subtaskIndex == that.subtaskIndex && attemptNum == that.attemptNum && Objects.equals(host, that.host) + && Objects.equals(endpoint, that.endpoint) && Objects.equals(userAccumulators, that.userAccumulators); } @Override public int hashCode() { - return Objects.hash(subtaskIndex, attemptNum, host, userAccumulators); + return Objects.hash(subtaskIndex, attemptNum, host, endpoint, userAccumulators); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java index b0ab00e26ac29..6435ff81762e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java @@ -237,6 +237,10 @@ public String getFQDNHostname() { return fqdnHostname; } + public String getEndpoint() { + return String.format("%s:%d", fqdnHostname, port); + } + @Override public String toString() { return new StringJoiner( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index 3690324854846..3e5f87883415b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -224,6 +224,15 @@ public String getNodeId() { return nodeId; } + /** + * Gets the endpoint of the TaskManager in the format of "$HOST:$PORT". + * + * @return The endpoint of the TaskManager. + */ + public String getEndpoint() { + return String.format("%s:%d", getFQDNHostname(), dataPort); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java index be4938ab96faa..63e53d889158a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java @@ -186,6 +186,7 @@ void testHandleRequest() throws Exception { expectedState, attempt, assignedResourceLocation.getHostname(), + assignedResourceLocation.getEndpoint(), deployingTs, finishedTs, finishedTs - deployingTs, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java index ba828377f4409..661e9e3982cf8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java @@ -189,6 +189,7 @@ void testHandleRequest() throws Exception { expectedState, attempt, "(unassigned)", + "(unassigned)", -1L, 0L, -1L, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/AggregatedTaskDetailsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/AggregatedTaskDetailsInfoTest.java index cc916be116adc..32c42d6a0b6fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/AggregatedTaskDetailsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/AggregatedTaskDetailsInfoTest.java @@ -74,6 +74,7 @@ protected AggregatedTaskDetailsInfo getTestResponseInstance() throws Exception { ExecutionState.values()[ random.nextInt(ExecutionState.values().length)], Math.abs(random.nextInt()), + "localhost", "localhost:" + random.nextInt(65536), Math.abs(random.nextLong()), Math.abs(random.nextLong()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistoryNoRootTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistoryNoRootTest.java index eb35780142cfb..3bb7d1be0b9c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistoryNoRootTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistoryNoRootTest.java @@ -77,6 +77,7 @@ protected JobExceptionsInfoWithHistory getTestResponseInstance() throws Exceptio Collections.emptyMap(), "task name #2", "location #2", + "location #2", "taskManagerId #2"))), new JobExceptionsInfoWithHistory.RootExceptionInfo( "local task failure #1", @@ -85,6 +86,7 @@ protected JobExceptionsInfoWithHistory getTestResponseInstance() throws Exceptio Collections.emptyMap(), "task name", "location", + "location", "taskManagerId", Collections.emptyList())), false)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java index 155a31621ce58..bcce56ffccb17 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java @@ -70,6 +70,7 @@ protected JobVertexDetailsInfo getTestResponseInstance() throws Exception { ExecutionState.CREATED, random.nextInt(), "local1", + "local1:123", System.currentTimeMillis(), System.currentTimeMillis(), 1L, @@ -83,6 +84,7 @@ protected JobVertexDetailsInfo getTestResponseInstance() throws Exception { ExecutionState.RUNNING, random.nextInt(), "local2", + "local2:123", System.currentTimeMillis(), System.currentTimeMillis(), 1L, @@ -95,6 +97,7 @@ protected JobVertexDetailsInfo getTestResponseInstance() throws Exception { ExecutionState.FAILED, random.nextInt(), "local2", + "local2:123", System.currentTimeMillis(), System.currentTimeMillis(), 1L, @@ -108,6 +111,7 @@ protected JobVertexDetailsInfo getTestResponseInstance() throws Exception { ExecutionState.FINISHED, random.nextInt(), "local3", + "local3:123", System.currentTimeMillis(), System.currentTimeMillis(), 1L, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java index 16a9fb243d20a..9f739e1af19e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java @@ -70,6 +70,7 @@ protected JobVertexTaskManagersInfo getTestResponseInstance() throws Exception { taskManagersInfoList.add( new TaskManagersInfo( "host1", + "host1:123", ExecutionState.CANCELING, 1L, 2L, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfoTest.java index a411527bca4ec..24c9805438bc0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfoTest.java @@ -45,19 +45,22 @@ protected SubtasksTimesInfo getTestResponseInstance() throws Exception { subTimeMap1.put(ExecutionState.RUNNING, 1L); subTimeMap1.put(ExecutionState.FAILED, 2L); subTimeMap1.put(ExecutionState.CANCELED, 3L); - subtasks.add(new SubtasksTimesInfo.SubtaskTimeInfo(0, "local1", 1L, subTimeMap1)); + subtasks.add( + new SubtasksTimesInfo.SubtaskTimeInfo(0, "local1", "local1:123", 1L, subTimeMap1)); Map subTimeMap2 = new HashMap<>(); subTimeMap2.put(ExecutionState.RUNNING, 4L); subTimeMap2.put(ExecutionState.FAILED, 5L); subTimeMap2.put(ExecutionState.CANCELED, 6L); - subtasks.add(new SubtasksTimesInfo.SubtaskTimeInfo(1, "local2", 2L, subTimeMap2)); + subtasks.add( + new SubtasksTimesInfo.SubtaskTimeInfo(1, "local2", "local2:123", 2L, subTimeMap2)); Map subTimeMap3 = new HashMap<>(); subTimeMap3.put(ExecutionState.SCHEDULED, 1L); subTimeMap3.put(ExecutionState.FAILED, 2L); subTimeMap3.put(ExecutionState.CANCELING, 3L); - subtasks.add(new SubtasksTimesInfo.SubtaskTimeInfo(2, "local3", 3L, subTimeMap3)); + subtasks.add( + new SubtasksTimesInfo.SubtaskTimeInfo(2, "local3", "local3:123", 3L, subTimeMap3)); return new SubtasksTimesInfo("testId", "testName", System.currentTimeMillis(), subtasks); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfoTest.java index 36a92b54cb545..90fe77b9155a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfoTest.java @@ -68,6 +68,7 @@ protected SubtaskExecutionAttemptDetailsInfo getTestResponseInstance() throws Ex Math.abs(random.nextInt()), ExecutionState.values()[random.nextInt(ExecutionState.values().length)], Math.abs(random.nextInt()), + "localhost", "localhost:" + random.nextInt(65536), Math.abs(random.nextLong()), Math.abs(random.nextLong()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfoTest.java index 651a34c61946e..5e3e0a9376cc8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfoTest.java @@ -48,7 +48,7 @@ protected SubtasksAllAccumulatorsInfo getTestResponseInstance() throws Exception for (int i = 0; i < 3; ++i) { subtaskAccumulatorsInfos.add( new SubtasksAllAccumulatorsInfo.SubtaskAccumulatorsInfo( - i, i, "host-" + String.valueOf(i), userAccumulators)); + i, i, "host-" + i, "host-" + i + ":123", userAccumulators)); } return new SubtasksAllAccumulatorsInfo(new JobVertexID(), 4, subtaskAccumulatorsInfos); } From fa1036c73e3bcd66b57d835c7859572ca4b2250d Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 11 Oct 2023 11:50:31 +0200 Subject: [PATCH 095/104] [hotfix] Remove Kafka documentation for SQL/Table API, since this is now externalized --- .../content.zh/docs/connectors/table/kafka.md | 630 ---------------- docs/content/docs/connectors/table/kafka.md | 688 ------------------ 2 files changed, 1318 deletions(-) delete mode 100644 docs/content.zh/docs/connectors/table/kafka.md delete mode 100644 docs/content/docs/connectors/table/kafka.md diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md deleted file mode 100644 index 635c5e14dabe1..0000000000000 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ /dev/null @@ -1,630 +0,0 @@ ---- -title: Kafka -weight: 3 -type: docs -aliases: - - /zh/dev/table/connectors/kafka.html ---- - - -# Apache Kafka SQL 连接器 - -{{< label "Scan Source: Bounded" >}} -{{< label "Scan Source: Unbounded" >}} -{{< label "Sink: Streaming Append Mode" >}} - -Kafka 连接器提供从 Kafka topic 中消费和写入数据的能力。 - -依赖 ------------- - -{{< sql_download_table "kafka" >}} - -Kafka 连接器目前并不包含在 Flink 的二进制发行版中,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Kafka 连接器。 - -如何创建 Kafka 表 ----------------- - -以下示例展示了如何创建 Kafka 表: - -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'format' = 'csv' -) -``` - -可用的元数据 ------------------- - -以下的连接器元数据可以在表定义中通过元数据列的形式获取。 - -`R/W` 列定义了一个元数据是可读的(`R`)还是可写的(`W`)。 -只读列必须声明为 `VIRTUAL` 以在 `INSERT INTO` 操作中排除它们。 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
数据类型描述R/W
topicSTRING NOT NULLKafka 记录的 Topic 名。R
partitionINT NOT NULLKafka 记录的 partition ID。R
headersMAP NOT NULL二进制 Map 类型的 Kafka 记录头(Header)。R/W
leader-epochINT NULLKafka 记录的 Leader epoch(如果可用)。R
offsetBIGINT NOT NULLKafka 记录在 partition 中的 offset。R
timestampTIMESTAMP_LTZ(3) NOT NULLKafka 记录的时间戳。R/W
timestamp-typeSTRING NOT NULLKafka 记录的时间戳类型。可能的类型有 "NoTimestampType", - "CreateTime"(会在写入元数据时设置),或 "LogAppendTime"。R
- -以下扩展的 `CREATE TABLE` 示例展示了使用这些元数据字段的语法: - -```sql -CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `partition` BIGINT METADATA VIRTUAL, - `offset` BIGINT METADATA VIRTUAL, - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'format' = 'csv' -); -``` - -**格式元信息** - -连接器可以读出消息格式的元数据。格式元数据的配置键以 `'value.'` 作为前缀。 - -以下示例展示了如何获取 Kafka 和 Debezium 的元数据字段: - -```sql -CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format - `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format - `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector - `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'value.format' = 'debezium-json' -); -``` - -连接器参数 ----------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
参数是否必选默认值数据类型描述
connector
必选(无)String指定使用的连接器,Kafka 连接器使用 'kafka'
topic
required for sink(无)String当表用作 source 时读取数据的 topic 名。亦支持用分号间隔的 topic 列表,如 'topic-1;topic-2'。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。当表被用作 sink 时,该配置表示写入的 topic 名。注意 sink 表不支持 topic 列表。
topic-pattern
可选(无)String匹配读取 topic 名称的正则表达式。在作业开始运行时,所有匹配该正则表达式的 topic 都将被 Kafka consumer 订阅。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。
properties.bootstrap.servers
必选(无)String逗号分隔的 Kafka broker 列表。
properties.group.id
对 source 可选,不适用于 sink(无)StringKafka source 的消费组 id。如果未指定消费组 ID,则会使用自动生成的 "KafkaSource-{tableIdentifier}" 作为消费组 ID。
properties.*
可选(无)String - 可以设置和传递任意 Kafka 的配置项。后缀名必须匹配在 Kafka 配置文档 中定义的配置键。Flink 将移除 "properties." 配置键前缀并将变换后的配置键和值传入底层的 Kafka 客户端。例如,你可以通过 'properties.allow.auto.create.topics' = 'false' 来禁用 topic 的自动创建。但是某些配置项不支持进行配置,因为 Flink 会覆盖这些配置,例如 'key.deserializer''value.deserializer'。 -
format
必选(无)String用来序列化或反序列化 Kafka 消息的格式。 - 请参阅 }}">格式 页面以获取更多关于格式的细节和相关配置项。 - 注意:该配置项和 'value.format' 二者必需其一。 -
key.format
可选(无)String用来序列化和反序列化 Kafka 消息键(Key)的格式。 - 请参阅 }}">格式 页面以获取更多关于格式的细节和相关配置项。 - 注意:如果定义了键格式,则配置项 'key.fields' 也是必需的。 - 否则 Kafka 记录将使用空值作为键。 -
key.fields
可选[]List<String>表结构中用来配置消息键(Key)格式数据类型的字段列表。默认情况下该列表为空,因此消息键没有定义。 - 列表格式为 'field1;field2'。 -
key.fields-prefix
可选(无)String为所有消息键(Key)格式字段指定自定义前缀,以避免与消息体(Value)格式字段重名。默认情况下前缀为空。 - 如果定义了前缀,表结构和配置项 'key.fields' 都需要使用带前缀的名称。 - 当构建消息键格式字段时,前缀会被移除,消息键格式将会使用无前缀的名称。 - 请注意该配置项要求必须将 'value.fields-include' 配置为 'EXCEPT_KEY'。 -
value.format
必选(无)String序列化和反序列化 Kafka 消息体时使用的格式。 - 请参阅 }}">格式 页面以获取更多关于格式的细节和相关配置项。 - 注意:该配置项和 'format' 二者必需其一。 -
value.fields-include
可选ALL

枚举类型

可选值:[ALL, EXCEPT_KEY]
定义消息体(Value)格式如何处理消息键(Key)字段的策略。 - 默认情况下,表结构中 'ALL' 即所有的字段都会包含在消息体格式中,即消息键字段在消息键和消息体格式中都会出现。 -
scan.startup.mode
可选group-offsetsEnumKafka consumer 的启动模式。有效值为:'earliest-offset''latest-offset''group-offsets''timestamp''specific-offsets'。 - 请参阅下方 起始消费位点 以获取更多细节。
scan.startup.specific-offsets
可选(无)String在使用 'specific-offsets' 启动模式时为每个 partition 指定 offset,例如 'partition:0,offset:42;partition:1,offset:300'。 -
scan.startup.timestamp-millis
可选(无)Long在使用 'timestamp' 启动模式时指定启动的时间戳(单位毫秒)。
scan.bounded.mode
optionalunboundedEnumBounded mode for Kafka consumer, valid values are 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. - See the following Bounded Ending Position for more details.
scan.bounded.specific-offsets
optionalyes(none)StringSpecify offsets for each partition in case of 'specific-offsets' bounded mode, e.g. 'partition:0,offset:42;partition:1,offset:300'. If an offset - for a partition is not provided it will not consume from that partition.. -
scan.bounded.timestamp-millis
optionalyes(none)LongEnd at the specified epoch timestamp (milliseconds) used in case of 'timestamp' bounded mode.
scan.topic-partition-discovery.interval
可选(无)DurationConsumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。
sink.partitioner
可选'default'StringFlink partition 到 Kafka partition 的分区映射关系,可选值有: -
    -
  • default:使用 Kafka 默认的分区器对消息进行分区。
  • -
  • fixed:每个 Flink partition 最终对应最多一个 Kafka partition。
  • -
  • round-robin:Flink partition 按轮循(round-robin)的模式对应到 Kafka partition。只有当未指定消息的消息键时生效。
  • -
  • 自定义 FlinkKafkaPartitioner 的子类:例如 'org.mycompany.MyPartitioner'
  • -
- 请参阅下方 Sink 分区 以获取更多细节。 -
sink.semantic
可选at-least-onceString定义 Kafka sink 的语义。有效值为 'at-least-once''exactly-once''none'。请参阅 一致性保证 以获取更多细节。
sink.parallelism
可选(无)Integer定义 Kafka sink 算子的并行度。默认情况下,并行度由框架定义为与上游串联的算子相同。
- -特性 ----------------- - -### 消息键(Key)与消息体(Value)的格式 - -Kafka 消息的消息键和消息体部分都可以使用某种 [格式]({{< ref "docs/connectors/table/formats/overview" >}}) 来序列化或反序列化成二进制数据。 - -**消息体格式** - -由于 Kafka 消息中消息键是可选的,以下语句将使用消息体格式读取和写入消息,但不使用消息键格式。 -`'format'` 选项与 `'value.format'` 意义相同。 -所有的格式配置使用格式识别符作为前缀。 - -```sql -CREATE TABLE KafkaTable ( - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'format' = 'json', - 'json.ignore-parse-errors' = 'true' -) -``` - -消息体格式将配置为以下的数据类型: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> -``` - -**消息键和消息体格式** - -以下示例展示了如何配置和使用消息键和消息体格式。 -格式配置使用 `'key'` 或 `'value'` 加上格式识别符作为前缀。 - -```sql -CREATE TABLE KafkaTable ( - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'key.format' = 'json', - 'key.json.ignore-parse-errors' = 'true', - 'key.fields' = 'user_id;item_id', - - 'value.format' = 'json', - 'value.json.fail-on-missing-field' = 'false', - 'value.fields-include' = 'ALL' -) -``` - -消息键格式包含了在 `'key.fields'` 中列出的字段(使用 `';'` 分隔)和字段顺序。 -因此将配置为以下的数据类型: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT> -``` - -由于消息体格式配置为 `'value.fields-include' = 'ALL'`,所以消息键字段也会出现在消息体格式的数据类型中: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> -``` - -**重名的格式字段** - -如果消息键字段和消息体字段重名,连接器无法根据表结构信息将这些列区分开。 -`'key.fields-prefix'` 配置项可以在表结构中为消息键字段指定一个唯一名称,并在配置消息键格式的时候保留原名。 - -以下示例展示了在消息键和消息体中同时包含 `version` 字段的情况: - -```sql -CREATE TABLE KafkaTable ( - `k_version` INT, - `k_user_id` BIGINT, - `k_item_id` BIGINT, - `version` INT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'key.format' = 'json', - 'key.fields-prefix' = 'k_', - 'key.fields' = 'k_version;k_user_id;k_item_id', - - 'value.format' = 'json', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` - -消息体格式必须配置为 `'EXCEPT_KEY'` 模式。格式将被配置为以下的数据类型: - -```text -消息键格式: -ROW<`version` INT, `user_id` BIGINT, `item_id` BIGINT> - -消息体格式: -ROW<`version` INT, `behavior` STRING> -``` - -### Topic 和 Partition 的探测 - -`topic` 和 `topic-pattern` 配置项决定了 source 消费的 topic 或 topic 的匹配规则。`topic` 配置项可接受使用分号间隔的 topic 列表,例如 `topic-1;topic-2`。 -`topic-pattern` 配置项使用正则表达式来探测匹配的 topic。例如 `topic-pattern` 设置为 `test-topic-[0-9]`,则在作业启动时,所有匹配该正则表达式的 topic(以 `test-topic-` 开头,以一位数字结尾)都将被 consumer 订阅。 - -为允许 consumer 在作业启动之后探测到动态创建的 topic,请将 `scan.topic-partition-discovery.interval` 配置为一个非负值。这将使 consumer 能够探测匹配名称规则的 topic 中新的 partition。 - -请参阅 [Kafka DataStream 连接器文档]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-consumer-topic-和分区发现) 以获取更多关于 topic 和 partition 探测的信息。 - -注意 topic 列表和 topic 匹配规则只适用于 source。对于 sink 端,Flink 目前只支持单一 topic。 - -### 起始消费位点 - -`scan.startup.mode` 配置项决定了 Kafka consumer 的启动模式。有效值为: - -* `group-offsets`:从 Zookeeper/Kafka 中某个指定的消费组已提交的偏移量开始。 -* `earliest-offset`:从可能的最早偏移量开始。 -* `latest-offset`:从最末尾偏移量开始。 -* `timestamp`:从用户为每个 partition 指定的时间戳开始。 -* `specific-offsets`:从用户为每个 partition 指定的偏移量开始。 - -默认值 `group-offsets` 表示从 Zookeeper/Kafka 中最近一次已提交的偏移量开始消费。 - -如果使用了 `timestamp`,必须使用另外一个配置项 `scan.startup.timestamp-millis` 来指定一个从格林尼治标准时间 1970 年 1 月 1 日 00:00:00.000 开始计算的毫秒单位时间戳作为起始时间。 - -如果使用了 `specific-offsets`,必须使用另外一个配置项 `scan.startup.specific-offsets` 来为每个 partition 指定起始偏移量, -例如,选项值 `partition:0,offset:42;partition:1,offset:300` 表示 partition `0` 从偏移量 `42` 开始,partition `1` 从偏移量 `300` 开始。 - -### Bounded Ending Position - -The config option `scan.bounded.mode` specifies the bounded mode for Kafka consumer. The valid enumerations are: -

    -
  • `group-offsets`: bounded by committed offsets in ZooKeeper / Kafka brokers of a specific consumer group. This is evaluated at the start of consumption from a given partition.
  • -
  • `latest-offset`: bounded by latest offsets. This is evaluated at the start of consumption from a given partition.
  • -
  • `timestamp`: bounded by a user-supplied timestamp.
  • -
  • `specific-offsets`: bounded by user-supplied specific offsets for each partition.
  • -
- -If config option value `scan.bounded.mode` is not set the default is an unbounded table. - -If `timestamp` is specified, another config option `scan.bounded.timestamp-millis` is required to specify a specific bounded timestamp in milliseconds since January 1, 1970 00:00:00.000 GMT. - -If `specific-offsets` is specified, another config option `scan.bounded.specific-offsets` is required to specify specific bounded offsets for each partition, -e.g. an option value `partition:0,offset:42;partition:1,offset:300` indicates offset `42` for partition `0` and offset `300` for partition `1`. If an offset for a partition is not provided it will not consume from that partition. - -### CDC 变更日志(Changelog) Source - -Flink 原生支持使用 Kafka 作为 CDC 变更日志(changelog) source。如果 Kafka topic 中的消息是通过变更数据捕获(CDC)工具从其他数据库捕获的变更事件,则你可以使用 CDC 格式将消息解析为 Flink SQL 系统中的插入(INSERT)、更新(UPDATE)、删除(DELETE)消息。 - -在许多情况下,变更日志(changelog) source 都是非常有用的功能,例如将数据库中的增量数据同步到其他系统,审核日志,数据库的物化视图,时态表关联数据库表的更改历史等。 - -Flink 提供了几种 CDC 格式: - -* [debezium]({{< ref "docs/connectors/table/formats/debezium.md" >}}) -* [canal]({{< ref "docs/connectors/table/formats/canal.md" >}}) -* [maxwell]({{< ref "docs/connectors/table/formats/maxwell.md" >}}) - -### Sink 分区 - -配置项 `sink.partitioner` 指定了从 Flink 分区到 Kafka 分区的映射关系。 -默认情况下,Flink 使用 [Kafka 默认分区器](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java) 来对消息分区。默认分区器对没有消息键的消息使用 [粘性分区策略(sticky partition strategy)](https://www.confluent.io/blog/apache-kafka-producer-improvements-sticky-partitioner/) 进行分区,对含有消息键的消息使用 murmur2 哈希算法计算分区。 - -为了控制数据行到分区的路由,也可以提供一个自定义的 sink 分区器。'fixed' 分区器会将同一个 Flink 分区中的消息写入同一个 Kafka 分区,从而减少网络连接的开销。 - -### 一致性保证 - -默认情况下,如果查询在 [启用 checkpoint]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}#enabling-and-configuring-checkpointing) 模式下执行时,Kafka sink 按照至少一次(at-lease-once)语义保证将数据写入到 Kafka topic 中。 - -当 Flink checkpoint 启用时,`kafka` 连接器可以提供精确一次(exactly-once)的语义保证。 - -除了启用 Flink checkpoint,还可以通过传入对应的 `sink.semantic` 选项来选择三种不同的运行模式: - - * `none`:Flink 不保证任何语义。已经写出的记录可能会丢失或重复。 - * `at-least-once` (默认设置):保证没有记录会丢失(但可能会重复)。 - * `exactly-once`:使用 Kafka 事务提供精确一次(exactly-once)语义。当使用事务向 Kafka 写入数据时,请将所有从 Kafka 中消费记录的应用中的 `isolation.level` 配置项设置成实际所需的值(`read_committed` 或 `read_uncommitted`,后者为默认值)。 - -请参阅 [Kafka 文档]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-和容错) 以获取更多关于语义保证的信息。 - -### Source 按分区 Watermark - -Flink 对于 Kafka 支持发送按分区的 watermark。Watermark 在 Kafka consumer 中生成。 -按分区 watermark 的合并方式和在流 shuffle 时合并 Watermark 的方式一致。 -Source 输出的 watermark 由读取的分区中最小的 watermark 决定。 -如果 topic 中的某些分区闲置,watermark 生成器将不会向前推进。 -你可以在表配置中设置 [`'table.exec.source.idle-timeout'`]({{< ref "docs/dev/table/config" >}}#table-exec-source-idle-timeout) 选项来避免上述问题。 - -请参阅 [Kafka watermark 策略]({{< ref "docs/dev/datastream/event-time/generating_watermarks" >}}#watermark-策略和-kafka-连接器) 以获取更多细节。 - -### 安全 -要启用加密和认证相关的安全配置,只需将安全配置加上 "properties." 前缀配置在 Kafka 表上即可。下面的代码片段展示了如何配置 Kafka 表以使用 -PLAIN 作为 SASL 机制并提供 JAAS 配置: -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - ... - 'properties.security.protocol' = 'SASL_PLAINTEXT', - 'properties.sasl.mechanism' = 'PLAIN', - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' -) -``` -另一个更复杂的例子,使用 SASL_SSL 作为安全协议并使用 SCRAM-SHA-256 作为 SASL 机制: -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - ... - 'properties.security.protocol' = 'SASL_SSL', - /* SSL 配置 */ - /* 配置服务端提供的 truststore (CA 证书) 的路径 */ - 'properties.ssl.truststore.location' = '/path/to/kafka.client.truststore.jks', - 'properties.ssl.truststore.password' = 'test1234', - /* 如果要求客户端认证,则需要配置 keystore (私钥) 的路径 */ - 'properties.ssl.keystore.location' = '/path/to/kafka.client.keystore.jks', - 'properties.ssl.keystore.password' = 'test1234', - /* SASL 配置 */ - /* 将 SASL 机制配置为 as SCRAM-SHA-256 */ - 'properties.sasl.mechanism' = 'SCRAM-SHA-256', - /* 配置 JAAS */ - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' -) -``` - -如果在作业 JAR 中 Kafka 客户端依赖的类路径被重置了(relocate class),登录模块(login module)的类路径可能会不同,因此请根据登录模块在 -JAR 中实际的类路径来改写以上配置。例如在 SQL client JAR 中,Kafka client 依赖被重置在了 `org.apache.flink.kafka.shaded.org.apache.kafka` 路径下, -因此 plain 登录模块的类路径应写为 `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule`。 - -关于安全配置的详细描述,请参阅 Apache Kafka 文档中的"安全"一节。 - -数据类型映射 ----------------- - -Kafka 将消息键值以二进制进行存储,因此 Kafka 并不存在 schema 或数据类型。Kafka 消息使用格式配置进行序列化和反序列化,例如 csv,json,avro。 -因此,数据类型映射取决于使用的格式。请参阅 [格式]({{< ref "docs/connectors/table/formats/overview" >}}) 页面以获取更多细节。 - -{{< top >}} diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md deleted file mode 100644 index 1bdf3ca8c566e..0000000000000 --- a/docs/content/docs/connectors/table/kafka.md +++ /dev/null @@ -1,688 +0,0 @@ - ---- -title: Kafka -weight: 3 -type: docs -aliases: - - /dev/table/connectors/kafka.html ---- - - -# Apache Kafka SQL Connector - -{{< label "Scan Source: Bounded" >}} -{{< label "Scan Source: Unbounded" >}} -{{< label "Sink: Streaming Append Mode" >}} - -The Kafka connector allows for reading data from and writing data into Kafka topics. - -Dependencies ------------- - -{{< sql_download_table "kafka" >}} - -The Kafka connector is not part of the binary distribution. -See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). - -How to create a Kafka table ----------------- - -The example below shows how to create a Kafka table: - -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'format' = 'csv' -) -``` - -Available Metadata ------------------- - -The following connector metadata can be accessed as metadata columns in a table definition. - -The `R/W` column defines whether a metadata field is readable (`R`) and/or writable (`W`). -Read-only columns must be declared `VIRTUAL` to exclude them during an `INSERT INTO` operation. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
KeyData TypeDescriptionR/W
topicSTRING NOT NULLTopic name of the Kafka record.R
partitionINT NOT NULLPartition ID of the Kafka record.R
headersMAP NOT NULLHeaders of the Kafka record as a map of raw bytes.R/W
leader-epochINT NULLLeader epoch of the Kafka record if available.R
offsetBIGINT NOT NULLOffset of the Kafka record in the partition.R
timestampTIMESTAMP_LTZ(3) NOT NULLTimestamp of the Kafka record.R/W
timestamp-typeSTRING NOT NULLTimestamp type of the Kafka record. Either "NoTimestampType", - "CreateTime" (also set when writing metadata), or "LogAppendTime".R
- -The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata fields: - -```sql -CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `partition` BIGINT METADATA VIRTUAL, - `offset` BIGINT METADATA VIRTUAL, - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'format' = 'csv' -); -``` - -**Format Metadata** - -The connector is able to expose metadata of the value format for reading. Format metadata keys -are prefixed with `'value.'`. - -The following example shows how to access both Kafka and Debezium metadata fields: - -```sql -CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format - `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format - `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector - `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'value.format' = 'debezium-json' -); -``` - -Connector Options ----------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
OptionRequiredForwardedDefaultTypeDescription
connector
requiredno(none)StringSpecify what connector to use, for Kafka use 'kafka'.
topic
required for sinkyes(none)StringTopic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified for sources. When the table is used as sink, the topic name is the topic to write data to. Note topic list is not supported for sinks.
topic-pattern
optionalyes(none)StringThe regular expression for a pattern of topic names to read from. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. Note, only one of "topic-pattern" and "topic" can be specified for sources.
properties.bootstrap.servers
requiredyes(none)StringComma separated list of Kafka brokers.
properties.group.id
optional for source, not applicable for sinkyes(none)StringThe id of the consumer group for Kafka source. If group ID is not specified, an automatically generated id "KafkaSource-{tableIdentifier}" will be used.
properties.*
optionalno(none)String - This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. -
format
requiredno(none)StringThe format used to deserialize and serialize the value part of Kafka messages. - Please refer to the }}">formats page for - more details and more format options. - Note: Either this option or the 'value.format' option are required. -
key.format
optionalno(none)StringThe format used to deserialize and serialize the key part of Kafka messages. - Please refer to the }}">formats page - for more details and more format options. Note: If a key format is defined, the 'key.fields' - option is required as well. Otherwise the Kafka records will have an empty key. -
key.fields
optionalno[]List<String>Defines an explicit list of physical columns from the table schema that configure the data - type for the key format. By default, this list is empty and thus a key is undefined. - The list should look like 'field1;field2'. -
key.fields-prefix
optionalno(none)StringDefines a custom prefix for all fields of the key format to avoid name clashes with fields - of the value format. By default, the prefix is empty. If a custom prefix is defined, both the - table schema and 'key.fields' will work with prefixed names. When constructing the - data type of the key format, the prefix will be removed and the non-prefixed names will be used - within the key format. Please note that this option requires that 'value.fields-include' - must be set to 'EXCEPT_KEY'. -
value.format
requiredno(none)StringThe format used to deserialize and serialize the value part of Kafka messages. - Please refer to the }}">formats page - for more details and more format options. - Note: Either this option or the 'format' option are required. -
value.fields-include
optionalnoALL

Enum

Possible values: [ALL, EXCEPT_KEY]
Defines a strategy how to deal with key columns in the data type of the value format. By - default, 'ALL' physical columns of the table schema will be included in the value - format which means that key columns appear in the data type for both the key and value format. -
scan.startup.mode
optionalyesgroup-offsetsEnumStartup mode for Kafka consumer, valid values are 'earliest-offset', 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. - See the following Start Reading Position for more details.
scan.startup.specific-offsets
optionalyes(none)StringSpecify offsets for each partition in case of 'specific-offsets' startup mode, e.g. 'partition:0,offset:42;partition:1,offset:300'. -
scan.startup.timestamp-millis
optionalyes(none)LongStart from the specified epoch timestamp (milliseconds) used in case of 'timestamp' startup mode.
scan.bounded.mode
optionalunboundedEnumBounded mode for Kafka consumer, valid values are 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. - See the following Bounded Ending Position for more details.
scan.bounded.specific-offsets
optionalyes(none)StringSpecify offsets for each partition in case of 'specific-offsets' bounded mode, e.g. 'partition:0,offset:42;partition:1,offset:300'. If an offset - for a partition is not provided it will not consume from that partition.. -
scan.bounded.timestamp-millis
optionalyes(none)LongEnd at the specified epoch timestamp (milliseconds) used in case of 'timestamp' bounded mode.
scan.topic-partition-discovery.interval
optionalyes(none)DurationInterval for consumer to discover dynamically created Kafka topics and partitions periodically.
sink.partitioner
optionalyes'default'StringOutput partitioning from Flink's partitions into Kafka's partitions. Valid values are -
    -
  • default: use the kafka default partitioner to partition records.
  • -
  • fixed: each Flink partition ends up in at most one Kafka partition.
  • -
  • round-robin: a Flink partition is distributed to Kafka partitions sticky round-robin. It only works when record's keys are not specified.
  • -
  • Custom FlinkKafkaPartitioner subclass: e.g. 'org.mycompany.MyPartitioner'.
  • -
- See the following Sink Partitioning for more details. -
sink.semantic
optionalnoat-least-onceStringDeprecated: Please use sink.delivery-guarantee.
sink.delivery-guarantee
optionalnoat-least-onceStringDefines the delivery semantic for the Kafka sink. Valid enumerationns are 'at-least-once', 'exactly-once' and 'none'. See Consistency guarantees for more details.
sink.transactional-id-prefix
optionalyes(none)StringIf the delivery guarantee is configured as 'exactly-once' this value must be set and is used a prefix for the identifier of all opened Kafka transactions.
sink.parallelism
optionalno(none)IntegerDefines the parallelism of the Kafka sink operator. By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator.
- -Features ----------------- - -### Key and Value Formats - -Both the key and value part of a Kafka record can be serialized to and deserialized from raw bytes using -one of the given [formats]({{< ref "docs/connectors/table/formats/overview" >}}). - -**Value Format** - -Since a key is optional in Kafka records, the following statement reads and writes records with a configured -value format but without a key format. The `'format'` option is a synonym for `'value.format'`. All format -options are prefixed with the format identifier. - -```sql -CREATE TABLE KafkaTable ( - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'format' = 'json', - 'json.ignore-parse-errors' = 'true' -) -``` - -The value format will be configured with the following data type: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> -``` - -**Key and Value Format** - -The following example shows how to specify and configure key and value formats. The format options are -prefixed with either the `'key'` or `'value'` plus format identifier. - -```sql -CREATE TABLE KafkaTable ( - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'key.format' = 'json', - 'key.json.ignore-parse-errors' = 'true', - 'key.fields' = 'user_id;item_id', - - 'value.format' = 'json', - 'value.json.fail-on-missing-field' = 'false', - 'value.fields-include' = 'ALL' -) -``` - -The key format includes the fields listed in `'key.fields'` (using `';'` as the delimiter) in the same -order. Thus, it will be configured with the following data type: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT> -``` - -Since the value format is configured with `'value.fields-include' = 'ALL'`, key fields will also end up in -the value format's data type: - -```text -ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> -``` - -**Overlapping Format Fields** - -The connector cannot split the table's columns into key and value fields based on schema information -if both key and value formats contain fields of the same name. The `'key.fields-prefix'` option allows -to give key columns a unique name in the table schema while keeping the original names when configuring -the key format. - -The following example shows a key and value format that both contain a `version` field: - -```sql -CREATE TABLE KafkaTable ( - `k_version` INT, - `k_user_id` BIGINT, - `k_item_id` BIGINT, - `version` INT, - `behavior` STRING -) WITH ( - 'connector' = 'kafka', - ... - - 'key.format' = 'json', - 'key.fields-prefix' = 'k_', - 'key.fields' = 'k_version;k_user_id;k_item_id', - - 'value.format' = 'json', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` - -The value format must be configured in `'EXCEPT_KEY'` mode. The formats will be configured with -the following data types: - -```text -key format: -ROW<`version` INT, `user_id` BIGINT, `item_id` BIGINT> - -value format: -ROW<`version` INT, `behavior` STRING> -``` - -### Topic and Partition Discovery - -The config option `topic` and `topic-pattern` specifies the topics or topic pattern to consume for source. The config option `topic` can accept topic list using semicolon separator like 'topic-1;topic-2'. -The config option `topic-pattern` will use regular expression to discover the matched topic. For example, if the `topic-pattern` is `test-topic-[0-9]`, then all topics with names that match the specified regular expression (starting with `test-topic-` and ending with a single digit)) will be subscribed by the consumer when the job starts running. - -To allow the consumer to discover dynamically created topics after the job started running, set a non-negative value for `scan.topic-partition-discovery.interval`. This allows the consumer to discover partitions of new topics with names that also match the specified pattern. - -Please refer to [Kafka DataStream Connector documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-consumers-topic-and-partition-discovery) for more about topic and partition discovery. - -Note that topic list and topic pattern only work in sources. In sinks, Flink currently only supports a single topic. - -### Start Reading Position - -The config option `scan.startup.mode` specifies the startup mode for Kafka consumer. The valid enumerations are: - -* `group-offsets`: start from committed offsets in ZK / Kafka brokers of a specific consumer group. -* `earliest-offset`: start from the earliest offset possible. -* `latest-offset`: start from the latest offset. -* `timestamp`: start from user-supplied timestamp for each partition. -* `specific-offsets`: start from user-supplied specific offsets for each partition. - -The default option value is `group-offsets` which indicates to consume from last committed offsets in ZK / Kafka brokers. - -If `timestamp` is specified, another config option `scan.startup.timestamp-millis` is required to specify a specific startup timestamp in milliseconds since January 1, 1970 00:00:00.000 GMT. - -If `specific-offsets` is specified, another config option `scan.startup.specific-offsets` is required to specify specific startup offsets for each partition, -e.g. an option value `partition:0,offset:42;partition:1,offset:300` indicates offset `42` for partition `0` and offset `300` for partition `1`. - -### Bounded Ending Position - -The config option `scan.bounded.mode` specifies the bounded mode for Kafka consumer. The valid enumerations are: -
    -
  • `group-offsets`: bounded by committed offsets in ZooKeeper / Kafka brokers of a specific consumer group. This is evaluated at the start of consumption from a given partition.
  • -
  • `latest-offset`: bounded by latest offsets. This is evaluated at the start of consumption from a given partition.
  • -
  • `timestamp`: bounded by a user-supplied timestamp.
  • -
  • `specific-offsets`: bounded by user-supplied specific offsets for each partition.
  • -
- -If config option value `scan.bounded.mode` is not set the default is an unbounded table. - -If `timestamp` is specified, another config option `scan.bounded.timestamp-millis` is required to specify a specific bounded timestamp in milliseconds since January 1, 1970 00:00:00.000 GMT. - -If `specific-offsets` is specified, another config option `scan.bounded.specific-offsets` is required to specify specific bounded offsets for each partition, -e.g. an option value `partition:0,offset:42;partition:1,offset:300` indicates offset `42` for partition `0` and offset `300` for partition `1`. If an offset for a partition is not provided it will not consume from that partition. - -### CDC Changelog Source - -Flink natively supports Kafka as a CDC changelog source. If messages in a Kafka topic are change event captured from other databases using a CDC tool, you can use the corresponding Flink CDC format to interpret the messages as INSERT/UPDATE/DELETE statements into a Flink SQL table. - -The changelog source is a very useful feature in many cases, such as synchronizing incremental data from databases to other systems, auditing logs, materialized views on databases, temporal join changing history of a database table and so on. - -Flink provides several CDC formats: - -* [debezium]({{< ref "docs/connectors/table/formats/debezium.md" >}}) -* [canal]({{< ref "docs/connectors/table/formats/canal.md" >}}) -* [maxwell]({{< ref "docs/connectors/table/formats/maxwell.md" >}}) - -### Sink Partitioning - -The config option `sink.partitioner` specifies output partitioning from Flink's partitions into Kafka's partitions. -By default, Flink uses the [Kafka default partitioner](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java) to partition records. It uses the [sticky partition strategy](https://www.confluent.io/blog/apache-kafka-producer-improvements-sticky-partitioner/) for records with null keys and uses a murmur2 hash to compute the partition for a record with the key defined. - -In order to control the routing of rows into partitions, a custom sink partitioner can be provided. The 'fixed' partitioner will write the records in the same Flink partition into the same Kafka partition, which could reduce the cost of the network connections. - -### Consistency guarantees - -By default, a Kafka sink ingests data with at-least-once guarantees into a Kafka topic if the query is executed with [checkpointing enabled]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}#enabling-and-configuring-checkpointing). - -With Flink's checkpointing enabled, the `kafka` connector can provide exactly-once delivery guarantees. - -Besides enabling Flink's checkpointing, you can also choose three different modes of operating chosen by passing appropriate `sink.delivery-guarantee` option: - - * `none`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. - * `at-least-once` (default setting): This guarantees that no records will be lost (although they can be duplicated). - * `exactly-once`: Kafka transactions will be used to provide exactly-once semantic. Whenever you write - to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_committed` - or `read_uncommitted` - the latter one is the default value) for any application consuming records - from Kafka. - -Please refer to [Kafka documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-and-fault-tolerance) for more caveats about delivery guarantees. - -### Source Per-Partition Watermarks - -Flink supports to emit per-partition watermarks for Kafka. Watermarks are generated inside the Kafka -consumer. The per-partition watermarks are merged in the same way as watermarks are merged during streaming -shuffles. The output watermark of the source is determined by the minimum watermark among the partitions -it reads. If some partitions in the topics are idle, the watermark generator will not advance. You can -alleviate this problem by setting the [`'table.exec.source.idle-timeout'`]({{< ref "docs/dev/table/config" >}}#table-exec-source-idle-timeout) -option in the table configuration. - -Please refer to [Kafka watermark strategies]({{< ref "docs/dev/datastream/event-time/generating_watermarks" >}}#watermark-strategies-and-the-kafka-connector) -for more details. - -### Security -In order to enable security configurations including encryption and authentication, you just need to setup security -configurations with "properties." prefix in table options. The code snippet below shows configuring Kafka table to -use PLAIN as SASL mechanism and provide JAAS configuration: -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - ... - 'properties.security.protocol' = 'SASL_PLAINTEXT', - 'properties.sasl.mechanism' = 'PLAIN', - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' -) -``` -For a more complex example, use SASL_SSL as the security protocol and use SCRAM-SHA-256 as SASL mechanism: -```sql -CREATE TABLE KafkaTable ( - `user_id` BIGINT, - `item_id` BIGINT, - `behavior` STRING, - `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' -) WITH ( - 'connector' = 'kafka', - ... - 'properties.security.protocol' = 'SASL_SSL', - /* SSL configurations */ - /* Configure the path of truststore (CA) provided by the server */ - 'properties.ssl.truststore.location' = '/path/to/kafka.client.truststore.jks', - 'properties.ssl.truststore.password' = 'test1234', - /* Configure the path of keystore (private key) if client authentication is required */ - 'properties.ssl.keystore.location' = '/path/to/kafka.client.keystore.jks', - 'properties.ssl.keystore.password' = 'test1234', - /* SASL configurations */ - /* Set SASL mechanism as SCRAM-SHA-256 */ - 'properties.sasl.mechanism' = 'SCRAM-SHA-256', - /* Set JAAS configurations */ - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' -) -``` - -Please note that the class path of the login module in `sasl.jaas.config` might be different if you relocate Kafka -client dependencies, so you may need to rewrite it with the actual class path of the module in the JAR. -For example if you are using SQL client JAR, which has relocate Kafka client dependencies to `org.apache.flink.kafka.shaded.org.apache.kafka`, -the path of plain login module should be `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule` instead. - -For detailed explanations of security configurations, please refer to -the "Security" section in Apache Kafka documentation. - -Data Type Mapping ----------------- - -Kafka stores message keys and values as bytes, so Kafka doesn't have schema or data types. The Kafka messages are deserialized and serialized by formats, e.g. csv, json, avro. -Thus, the data type mapping is determined by specific formats. Please refer to [Formats]({{< ref "docs/connectors/table/formats/overview" >}}) pages for more details. - -{{< top >}} From 7542b56f2abb860f42a83c4687f6e38bb82b78c6 Mon Sep 17 00:00:00 2001 From: James Hughes Date: Thu, 9 Nov 2023 10:49:15 -0500 Subject: [PATCH 096/104] [FLINK-33469] Implement restore tests for Limit node (#23675) --- .../nodes/exec/stream/LimitJsonPlanTest.java | 66 --------------- .../nodes/exec/stream/LimitRestoreTest.java | 38 +++++++++ .../nodes/exec/stream/LimitTestPrograms.java | 60 +++++++++++++ .../stream/jsonplan/LimitJsonPlanITCase.java | 50 ----------- .../limit/plan/limit.json} | 79 ++++++++---------- .../limit/savepoint/_metadata | Bin 0 -> 11422 bytes 6 files changed, 134 insertions(+), 159 deletions(-) delete mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitTestPrograms.java delete mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out => restore-tests/stream-exec-limit_1/limit/plan/limit.json} (70%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/savepoint/_metadata diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest.java deleted file mode 100644 index 062edcaff15d7..0000000000000 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.plan.nodes.exec.stream; - -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.planner.utils.StreamTableTestUtil; -import org.apache.flink.table.planner.utils.TableTestBase; - -import org.junit.Before; -import org.junit.Test; - -/** Test json serialization for sort limit. */ -public class LimitJsonPlanTest extends TableTestBase { - - private StreamTableTestUtil util; - private TableEnvironment tEnv; - - @Before - public void setup() { - util = streamTestUtil(TableConfig.getDefault()); - tEnv = util.getTableEnv(); - - String srcTableDdl = - "CREATE TABLE MyTable (\n" - + " a bigint,\n" - + " b int not null,\n" - + " c varchar,\n" - + " d timestamp(3)\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - tEnv.executeSql(srcTableDdl); - } - - @Test - public void testLimit() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a bigint,\n" - + " b bigint\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - String sql = "insert into MySink SELECT a, a from MyTable limit 10"; - util.verifyJsonPlan(sql); - } -} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitRestoreTest.java new file mode 100644 index 0000000000000..9d6180064ffea --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitRestoreTest.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.planner.plan.nodes.exec.testutils.RestoreTestBase; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Arrays; +import java.util.List; + +/** Restore tests for {@link StreamExecLimit}. */ +public class LimitRestoreTest extends RestoreTestBase { + + public LimitRestoreTest() { + super(StreamExecLimit.class); + } + + @Override + public List programs() { + return Arrays.asList(LimitTestPrograms.LIMIT); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitTestPrograms.java new file mode 100644 index 0000000000000..0fc8a44695f3d --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitTestPrograms.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +/** + * {@link TableTestProgram} definitions for testing {@link + * org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit}. + */ +public class LimitTestPrograms { + + static final Row[] DATA1 = + new Row[] { + Row.of(2, "a", 6), + Row.of(4, "b", 8), + Row.of(6, "c", 10), + Row.of(1, "a", 5), + Row.of(3, "b", 7), + Row.of(5, "c", 9) + }; + + static final Row[] DATA2 = new Row[] {Row.of(8, "d", 3), Row.of(7, "e", 2)}; + static final TableTestProgram LIMIT = + TableTestProgram.of("limit", "validates limit node") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema("a INT", "b VARCHAR", "c INT") + .producedBeforeRestore(DATA1) + .producedAfterRestore(DATA2) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a INT", "b VARCHAR", "c BIGINT") + .consumedBeforeRestore( + "+I[2, a, 6]", "+I[4, b, 8]", "+I[6, c, 10]") + .consumedAfterRestore(new String[] {}) + .build()) + .runSql("INSERT INTO sink_t SELECT * from source_t LIMIT 3") + .build(); +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java deleted file mode 100644 index 1de8f66bd56e7..0000000000000 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/LimitJsonPlanITCase.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.runtime.stream.jsonplan; - -import org.apache.flink.table.planner.factories.TestValuesTableFactory; -import org.apache.flink.table.planner.runtime.utils.TestData; -import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; -import org.apache.flink.table.planner.utils.JsonPlanTestBase; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.ExecutionException; - -/** Test for limit JsonPlan ser/de. */ -class LimitJsonPlanITCase extends JsonPlanTestBase { - @Test - void testLimit() throws ExecutionException, InterruptedException, IOException { - createTestValuesSourceTable( - "MyTable", - JavaScalaConversionUtil.toJava(TestData.data1()), - "a int", - "b varchar", - "c int"); - createTestNonInsertOnlyValuesSinkTable("`result`", "a int", "b varchar", "c bigint"); - String sql = "insert into `result` select * from MyTable limit 3"; - compileSqlAndExecutePlan(sql).await(); - - List expected = Arrays.asList("+I[2, a, 6]", "+I[4, b, 8]", "+I[6, c, 10]"); - assertResult(expected, TestValuesTableFactory.getResultsAsStrings("result")); - } -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/plan/limit.json similarity index 70% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/plan/limit.json index ebce53e07ccf3..15cb677b4e450 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/plan/limit.json @@ -1,50 +1,35 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { "id" : 1, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`source_t`", "resolvedTable" : { "schema" : { "columns" : [ { "name" : "a", - "dataType" : "BIGINT" + "dataType" : "INT" }, { "name" : "b", - "dataType" : "INT NOT NULL" - }, { - "name" : "c", "dataType" : "VARCHAR(2147483647)" }, { - "name" : "d", - "dataType" : "TIMESTAMP(3)" + "name" : "c", + "dataType" : "INT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } + "partitionKeys" : [ ] } }, "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 0 ] ], - "producedType" : "ROW<`a` BIGINT> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`a` BIGINT> NOT NULL" - }, { "type" : "LimitPushDown", - "limit" : 10 + "limit" : 3 } ] }, - "outputType" : "ROW<`a` BIGINT>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a], metadata=[], limit=[10]]], fields=[a])", + "outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, limit=[3]]], fields=[a, b, c])", "inputProperties" : [ ] }, { "id" : 2, @@ -56,7 +41,7 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT>", + "outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` INT>", "description" : "Exchange(distribution=[single])" }, { "id" : 3, @@ -67,7 +52,7 @@ "rankRange" : { "type" : "Constant", "start" : 1, - "end" : 10 + "end" : 3 }, "rankStrategy" : { "type" : "AppendFast" @@ -85,8 +70,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT>", - "description" : "Limit(offset=[0], fetch=[10])", + "outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` INT>", + "description" : "Limit(offset=[0], fetch=[3])", "rankType" : "ROW_NUMBER", "partition" : { "fields" : [ ] @@ -101,10 +86,20 @@ "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "INT" }, { "kind" : "INPUT_REF", - "inputIndex" : 0, + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + } ], "type" : "BIGINT" } ], "condition" : null, @@ -115,8 +110,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `a0` BIGINT>", - "description" : "Calc(select=[a, a AS a0])" + "outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>", + "description" : "Calc(select=[a, b, CAST(c AS BIGINT) AS c])" }, { "id" : 5, "type" : "stream-exec-sink_1", @@ -129,24 +124,22 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { "name" : "a", - "dataType" : "BIGINT" + "dataType" : "INT" }, { "name" : "b", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "c", "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } + "partitionKeys" : [ ] } } }, @@ -158,8 +151,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `a0` BIGINT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, a0])" + "outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c])" } ], "edges" : [ { "source" : 1, @@ -190,4 +183,4 @@ }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-limit_1/limit/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c29a96bc807de74139e833d67edc5270eda4b14c GIT binary patch literal 11422 zcmeGiU5pz?d2Ci8_%xwaR98SjpLn_KVt zcE^{yNQ6XH8>C94mEZxP(5T8wMUYSth!+qq1jNq+^nr&W6(JB3q7o3Kwg@=BncY3F z&%V^x=1vs6l0CaK^L_KpH{X0e^G)`1^&moM2!8DG*##Op__a3<{m6Xw!D~O1e!B99 z*G8Q--tKmC`npB=8hY-v@psR~UcUX-^Y8vzb1hw^(MTu}l9HjAluRa4sbnG| zT6mQ-4OOSXa9EO(v7i*ip>RYFN9B|v)~X2-i%ZF%T*IMYRFP`IRE@-f@pvtkP?7{o zi8w;T?*N(m7^!|#Hq5$!n^;~VLQPZkWr5) zv8G-i=2TZrNyklV$)JEaz|f1Ha7z30R`ZL7R8hV&#QDPh?`BKE4(M`HTq=c znao_JR8E&OmGU!%3<{i17ogdXMi;}4_XE{z1H+yG*NV?Br#ZLR_G|7)!}n^6$={H>q16vHN1^Yitf$k%b6!K zMTdL+v0wq4vrE|QWPvY>_Gv_~)207cRxYHAv!~LFGsxR4P${-`}Oje0b2OI|9@9qgQvwIVl z;BP;Bb06==3aey-Jjq_z18xC-uw7i{E~@=LX%}sGzUuVxVBGBE!pTmbVF!3)!+AmI zm+xGC`eUE}?)QIv<;3H+UcDWL2Ii%5PnG|%59Stv^uzvJ4w{6TSf)@~Ese50=x)OC z=hz2Aq(Vpip5M2v#Edo*3E~$ zdHLmvANg|VaE-OOO%_F?Ip*z&EI<;Ob3P5e*rRvE?gHPsdVB2a{%0G0Kw`K4K$gZluVorgP|^z!YR7{tcU3N>Nz4)pn7Oo)fI{+dyvb<6(NXx5_&+- z!%YX3JLtG)&>U<+qMX9k5*vQ4*>~sx__sLDIkb<{Zmpn;$oJppJ)^w{L;}V6r+|l# z@o0s~uP|$98Y~$sr8RX%Zy1VNQ+ad+a{bSdN5kLw^Or{XE*R3F#^v3K_eRi0JU)Xi z{p_=EALD3zGDQz~R=C~GUf%oLnA4_-*V+DByZoz%zxrc*Er1~W8M5FMVyy{M8#vEY zbpYlQ>y~bQoI$yaI~#}{WiXjw2N);Wj&=m!=SKk$e1%xDsWxq)KI)Na+YeD5|B#2uU-Ls0d7J=P9+U}QcHLApX{^nO6KKSL|zQB+*Ibp|q z4Cy3)7Xh^g#Fl~Uf4PyoF#YYBHpmnQd5i_4n_!-!fSjS%r(gK}i$A}5{7oR|(=It| z9Ek%w&Jmng^B$TFj&R7}2&Y4Hd5*2xO6$XW8sl-}%|$`EL!y29XQXCxggEnOPr1E_XYHGKgGwU^<9g+%y;WYzL9c zAac387Yhc-XN{d-yx)A^Ao*)zhV!%9?6CPT?!Qb|a` zs8o#xlgSu-)Nvk$(%p~{B%Q@2!+g(MM9SjE29h?_F1n2GHewa{lFM%@f>sMaif8*(~@A2ft zFwoe*{IVTIvF@CkY?wqOYea^WkD-g3h)!;Tx#BlRZef#dp$FZrIQ4T2@4#$2KcCOergM;Kbyj$C6DJS=?AUTE dwudNg0 Date: Tue, 12 Sep 2023 14:39:33 +0800 Subject: [PATCH 097/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/planner/utils (TableTestBase) --- .../utils/InternalConfigOptionsTest.java | 28 ++--- .../table/planner/utils/TableTestBase.scala | 100 ++++++++++-------- 2 files changed, 71 insertions(+), 57 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/InternalConfigOptionsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/InternalConfigOptionsTest.java index 8d1c182d3eaf2..bb30acf1228a5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/InternalConfigOptionsTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/InternalConfigOptionsTest.java @@ -24,12 +24,14 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraph; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.calcite.rel.RelNode; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -40,21 +42,21 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link InternalConfigOptions}. */ -@RunWith(Parameterized.class) -public class InternalConfigOptionsTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class InternalConfigOptionsTest extends TableTestBase { private TableEnvironment tEnv; private PlannerBase planner; - @Parameterized.Parameters(name = "plannerMode = {0}") - public static Collection parameters() { + @Parameters(name = "plannerMode = {0}") + private static Collection parameters() { return Arrays.asList("STREAMING", "BATCH"); } - @Parameterized.Parameter public String plannerMode; + @Parameter private String plannerMode; - @Before - public void setUp() { + @BeforeEach + void setUp() { if (plannerMode.equals("STREAMING")) { StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -66,8 +68,8 @@ public void setUp() { } } - @Test - public void testTranslateExecNodeGraphWithInternalTemporalConf() { + @TestTemplate + void testTranslateExecNodeGraphWithInternalTemporalConf() { Table table = tEnv.sqlQuery("SELECT LOCALTIME, LOCALTIMESTAMP, CURRENT_TIME, CURRENT_TIMESTAMP"); RelNode relNode = planner.optimize(TableTestUtil.toRelNode(table)); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 7fc23d8281f6f..d8920bc62c06c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.configuration.BatchExecutionOptions -import org.apache.flink.core.testutils.FlinkMatchers.containsMessage import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode import org.apache.flink.streaming.api.{environment, TimeCharacteristic} @@ -67,6 +66,7 @@ import org.apache.flink.table.sources.{StreamTableSource, TableSource} import org.apache.flink.table.types.logical.LogicalType import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.typeutils.FieldInfoUtils +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension import org.apache.flink.types.Row import org.apache.flink.util.{FlinkUserCodeClassLoaders, MutableURLClassLoader} import org.apache.flink.util.jackson.JacksonMapperFactory @@ -79,35 +79,27 @@ import org.apache.calcite.avatica.util.TimeUnit import org.apache.calcite.rel.RelNode import org.apache.calcite.sql.{SqlExplainLevel, SqlIntervalQualifier} import org.apache.calcite.sql.parser.SqlParserPos -import org.junit.Assert.{assertEquals, assertThat, assertTrue, fail} -import org.junit.Rule -import org.junit.rules.{ExpectedException, TemporaryFolder, TestName} +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType, fail} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.extension.{BeforeEachCallback, ExtendWith, ExtensionContext, RegisterExtension} +import org.junit.jupiter.api.io.TempDir +import org.junit.platform.commons.support.AnnotationSupport import java.io.{File, IOException} import java.net.URL -import java.nio.file.{Files, Paths} +import java.nio.file.{Files, Path, Paths} import java.time.Duration import java.util.Collections /** Test base for testing Table API / SQL plans. */ abstract class TableTestBase { - // used for accurate exception information checking. - val expectedException: ExpectedException = ExpectedException.none() - // used for get test case method name + @RegisterExtension val testName: TestName = new TestName - val _tempFolder = new TemporaryFolder - - @Rule - def tempFolder: TemporaryFolder = _tempFolder - - @Rule - def thrown: ExpectedException = expectedException - - @Rule - def name: TestName = testName + @TempDir + var tempFolder: Path = _ def streamTestUtil(tableConfig: TableConfig = TableConfig.getDefault): StreamTableTestUtil = StreamTableTestUtil(this, tableConfig = tableConfig) @@ -127,9 +119,36 @@ abstract class TableTestBase { val expectedString = FlinkRelOptUtil.toString(TableTestUtil.toRelNode(expected)) val actualString = FlinkRelOptUtil.toString(TableTestUtil.toRelNode(actual)) assertEquals( - "Logical plans do not match", LogicalPlanFormatUtils.formatTempTableId(expectedString), - LogicalPlanFormatUtils.formatTempTableId(actualString)) + LogicalPlanFormatUtils.formatTempTableId(actualString), + "Logical plans do not match") + } +} + +class TestName extends BeforeEachCallback { + + private val bracketsRegex = """\[.*\]""".r + + private var methodName: String = _ + + def getMethodName: String = methodName + + override def beforeEach(context: ExtensionContext): Unit = { + if (hasParameterizedTestExtension(context)) { + val displayName = context.getDisplayName match { + case bracketsRegex(_*) => context.getDisplayName + case _ => s"[${context.getDisplayName}]" + } + methodName = s"${context.getTestMethod.get().getName}$displayName" + } else { + methodName = context.getTestMethod.get().getName + } + } + + private def hasParameterizedTestExtension(context: ExtensionContext): Boolean = { + Option(AnnotationSupport.findAnnotation(context.getTestClass, classOf[ExtendWith]).orElse(null)) + .map(_.value) + .exists(_.contains(classOf[ParameterizedTestExtension])) } } @@ -622,7 +641,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val optimizedPlan = getOptimizedRelPlan(Array(optimizedRel), Array.empty, withRowType = false) val result = notExpected.forall(!optimizedPlan.contains(_)) val message = s"\nactual plan:\n$optimizedPlan\nnot expected:\n${notExpected.mkString(", ")}" - assertTrue(message, result) + assertTrue(result, message) } /** @@ -752,14 +771,9 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) sql: String, message: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { - verifyExplain(sql) - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e: Throwable => - assertTrue(clazz.isAssignableFrom(e.getClass)) - assertThat(e, containsMessage(message)) - } + assertThatExceptionOfType(clazz) + .isThrownBy(() => verifyExplain(sql)) + .withMessageContaining(message) } /** @@ -841,22 +855,20 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) fail(s"$testMethodFileName regenerated.") } else { val expected = String.join("\n", Files.readAllLines(path)) - assertEquals( - TableTestUtil.replaceExecNodeId(TableTestUtil.getPrettyJson(expected)), - TableTestUtil.replaceExecNodeId(TableTestUtil.getPrettyJson(jsonPlanWithoutFlinkVersion)) - ) + assertThat( + TableTestUtil.replaceExecNodeId(TableTestUtil.getPrettyJson(jsonPlanWithoutFlinkVersion))) + .isEqualTo(TableTestUtil.replaceExecNodeId(TableTestUtil.getPrettyJson(expected))) // check json serde round trip as well val expectedWithFlinkVersion = JsonTestUtils.writeToString( JsonTestUtils .setFlinkVersion(JsonTestUtils.readFromString(expected), FlinkVersion.current())) - assertEquals( - TableTestUtil.replaceExecNodeId(TableTestUtil.getFormattedJson(expectedWithFlinkVersion)), + assertThat( TableTestUtil.replaceExecNodeId( - TableTestUtil.getFormattedJson( - getPlanner - .loadPlan(PlanReference.fromJsonString(expectedWithFlinkVersion)) - .asJsonString())) - ) + TableTestUtil.getFormattedJson(getPlanner + .loadPlan(PlanReference.fromJsonString(expectedWithFlinkVersion)) + .asJsonString()))) + .isEqualTo( + TableTestUtil.replaceExecNodeId(TableTestUtil.getFormattedJson(expectedWithFlinkVersion))) } } @@ -1127,16 +1139,16 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def assertEqualsOrExpand(tag: String, actual: String, expand: Boolean = true): Unit = { val expected = s"$${$tag}" if (!expand) { - diffRepository.assertEquals(test.name.getMethodName, tag, expected, actual) + diffRepository.assertEquals(test.testName.getMethodName, tag, expected, actual) return } - val expanded = diffRepository.expand(test.name.getMethodName, tag, expected) + val expanded = diffRepository.expand(test.testName.getMethodName, tag, expected) if (expanded != null && !expanded.equals(expected)) { // expected does exist, check result - diffRepository.assertEquals(test.name.getMethodName, tag, expected, actual) + diffRepository.assertEquals(test.testName.getMethodName, tag, expected, actual) } else { // expected does not exist, update - diffRepository.expand(test.name.getMethodName, tag, actual) + diffRepository.expand(test.testName.getMethodName, tag, actual) } } } From bf6afc3d52eeb0ec31d483a7cb9e746570116fb3 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 14:46:35 +0800 Subject: [PATCH 098/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/api (TableTestBase) --- .../MatchRecognizeValidationTest.java | 188 +++++++++--------- .../flink/table/api/batch/ExplainTest.scala | 32 +-- .../flink/table/api/stream/ExplainTest.scala | 32 +-- .../validation/OverWindowValidationTest.scala | 13 +- 4 files changed, 135 insertions(+), 130 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/batch/sql/validation/MatchRecognizeValidationTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/batch/sql/validation/MatchRecognizeValidationTest.java index cb40481db8d46..c0ca9fbf975c7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/batch/sql/validation/MatchRecognizeValidationTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/batch/sql/validation/MatchRecognizeValidationTest.java @@ -26,40 +26,40 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.calcite.sql.SqlMatchRecognize; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + /** Validation test for {@link SqlMatchRecognize}. */ -@RunWith(Parameterized.class) -public class MatchRecognizeValidationTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class MatchRecognizeValidationTest extends TableTestBase { private static final String STREAM = "stream"; private static final String BATCH = "batch"; - @Parameterized.Parameter public String mode; + @Parameter private String mode; - @Parameterized.Parameters(name = "mode = {0}") - public static Collection parameters() { + @Parameters(name = "mode = {0}") + private static Collection parameters() { return Arrays.asList(STREAM, BATCH); } - @Rule public ExpectedException expectedException = ExpectedException.none(); - private TableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = STREAM.equals(mode) ? streamTestUtil(TableConfig.getDefault()) @@ -86,32 +86,31 @@ public void setup() { + ")"); } - @After - public void after() { + @AfterEach + void after() { util.getTableEnv().executeSql("DROP TABLE Ticker"); util.getTableEnv().executeSql("DROP TABLE MyTable"); } /** Function 'MATCH_ROWTIME()' can only be used in MATCH_RECOGNIZE. */ - @Test(expected = ValidationException.class) - public void testMatchRowTimeInSelect() { + @TestTemplate + void testMatchRowTimeInSelect() { String sql = "SELECT MATCH_ROWTIME() FROM MyTable"; - util.verifyExplain(sql); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> util.verifyExplain(sql)); } /** Function 'MATCH_PROCTIME()' can only be used in MATCH_RECOGNIZE. */ - @Test(expected = ValidationException.class) - public void testMatchProcTimeInSelect() { + @TestTemplate + void testMatchProcTimeInSelect() { String sql = "SELECT MATCH_PROCTIME() FROM MyTable"; - util.verifyExplain(sql); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> util.verifyExplain(sql)); } - @Test - public void testSortProcessingTimeDesc() { + @TestTemplate + void testSortProcessingTimeDesc() { if (STREAM.equals(mode)) { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "Primary sort order of a streaming table must be ascending on time."); String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -123,16 +122,16 @@ public void testSortProcessingTimeDesc() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "Primary sort order of a streaming table must be ascending on time."); } } - @Test - public void testSortProcessingTimeSecondaryField() { + @TestTemplate + void testSortProcessingTimeSecondaryField() { if (STREAM.equals(mode)) { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "You must specify either rowtime or proctime for order by as the first one."); String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -144,16 +143,16 @@ public void testSortProcessingTimeSecondaryField() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "You must specify either rowtime or proctime for order by as the first one."); } } - @Test - public void testSortNoOrder() { + @TestTemplate + void testSortNoOrder() { if (STREAM.equals(mode)) { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "You must specify either rowtime or proctime for order by."); String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -164,16 +163,16 @@ public void testSortNoOrder() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "You must specify either rowtime or proctime for order by."); } } - @Test - public void testUpdatesInUpstreamOperatorNotSupported() { + @TestTemplate + void testUpdatesInUpstreamOperatorNotSupported() { if (STREAM.equals(mode)) { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "Match Recognize doesn't support consuming update changes which is produced by node GroupAggregate("); String sqlQuery = "SELECT *\n" + "FROM (SELECT DISTINCT * FROM Ticker)\n" @@ -186,14 +185,15 @@ public void testUpdatesInUpstreamOperatorNotSupported() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "Match Recognize doesn't support consuming update changes which is produced by node GroupAggregate("); } } - @Test - public void testAggregatesOnMultiplePatternVariablesNotSupported() { - expectedException.expect(ValidationException.class); - expectedException.expectMessage("SQL validation failed."); + @TestTemplate + void testAggregatesOnMultiplePatternVariablesNotSupported() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -205,13 +205,13 @@ public void testAggregatesOnMultiplePatternVariablesNotSupported() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining("SQL validation failed."); } - @Test - public void testAggregatesOnMultiplePatternVariablesNotSupportedInUDAGs() { - expectedException.expect(ValidationException.class); - expectedException.expectMessage("Aggregation must be applied to a single pattern variable"); + @TestTemplate + void testAggregatesOnMultiplePatternVariablesNotSupportedInUDAGs() { util.addTemporarySystemFunction("weightedAvg", new WeightedAvg()); String sqlQuery = "SELECT *\n" @@ -224,13 +224,13 @@ public void testAggregatesOnMultiplePatternVariablesNotSupportedInUDAGs() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining("Aggregation must be applied to a single pattern variable"); } - @Test - public void testValidatingAmbiguousColumns() { - expectedException.expect(ValidationException.class); - expectedException.expectMessage("Columns ambiguously defined: {symbol, price}"); + @TestTemplate + void testValidatingAmbiguousColumns() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -244,7 +244,9 @@ public void testValidatingAmbiguousColumns() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining("Columns ambiguously defined: {symbol, price}"); } // *************************************************************************************** @@ -253,11 +255,8 @@ public void testValidatingAmbiguousColumns() { // *************************************************************************************** /** Python Function can not be used in MATCH_RECOGNIZE for now. */ - @Test - public void testMatchPythonFunction() { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "Python Function can not be used in MATCH_RECOGNIZE for now."); + @TestTemplate + void testMatchPythonFunction() { util.addTemporarySystemFunction("pyFunc", new PythonScalarFunction("pyFunc")); String sql = "SELECT T.aa as ta\n" @@ -272,13 +271,14 @@ public void testMatchPythonFunction() { + " A AS a = 1,\n" + " B AS b = 'b'\n" + ") AS T"; - util.verifyExplain(sql); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sql)) + .withMessageContaining( + "Python Function can not be used in MATCH_RECOGNIZE for now."); } - @Test - public void testAllRowsPerMatch() { - expectedException.expect(TableException.class); - expectedException.expectMessage("All rows per match mode is not supported yet."); + @TestTemplate + void testAllRowsPerMatch() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -291,15 +291,13 @@ public void testAllRowsPerMatch() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining("All rows per match mode is not supported yet."); } - @Test - public void testGreedyQuantifierAtTheEndIsNotSupported() { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "Greedy quantifiers are not allowed as the last element of a " - + "Pattern yet. Finish your pattern with either a simple variable or reluctant quantifier."); + @TestTemplate + void testGreedyQuantifierAtTheEndIsNotSupported() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -311,15 +309,15 @@ public void testGreedyQuantifierAtTheEndIsNotSupported() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "Greedy quantifiers are not allowed as the last element of a " + + "Pattern yet. Finish your pattern with either a simple variable or reluctant quantifier."); } - @Test - public void testPatternsProducingEmptyMatchesAreNotSupported() { - expectedException.expect(TableException.class); - expectedException.expectMessage( - "Patterns that can produce empty matches are not supported. " - + "There must be at least one non-optional state."); + @TestTemplate + void testPatternsProducingEmptyMatchesAreNotSupported() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -331,13 +329,15 @@ public void testPatternsProducingEmptyMatchesAreNotSupported() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining( + "Patterns that can produce empty matches are not supported. " + + "There must be at least one non-optional state."); } - @Test - public void testDistinctAggregationsNotSupported() { - expectedException.expect(ValidationException.class); - expectedException.expectMessage("SQL validation failed."); + @TestTemplate + void testDistinctAggregationsNotSupported() { String sqlQuery = "SELECT *\n" + "FROM Ticker\n" @@ -349,6 +349,8 @@ public void testDistinctAggregationsNotSupported() { + " DEFINE\n" + " A AS A.symbol = 'a'\n" + ") AS T"; - tEnv.executeSql(sqlQuery); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tEnv.executeSql(sqlQuery)) + .withMessageContaining("SQL validation failed."); } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala index e6ed0bce82f65..715bee95477df 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala @@ -23,12 +23,12 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class ExplainTest(extended: Boolean) extends TableTestBase { private val extraDetails = if (extended) { @@ -46,33 +46,33 @@ class ExplainTest(extended: Boolean) extends TableTestBase { val LONG = new BigIntType() val INT = new IntType() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, Int.box(4)) } - @Test + @TestTemplate def testExplainWithTableSourceScan(): Unit = { util.verifyExplain("SELECT * FROM MyTable", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithDataStreamScan(): Unit = { util.verifyExplain("SELECT * FROM MyTable1", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithFilter(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithAgg(): Unit = { util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithJoin(): Unit = { // TODO support other join operators when them are supported util.tableEnv.getConfig @@ -80,24 +80,24 @@ class ExplainTest(extended: Boolean) extends TableTestBase { util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithUnion(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithSort(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithSingleSink(): Unit = { val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10") val sink = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING)) util.verifyExplainInsert(table, sink, "sink", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithMultiSinks(): Unit = { val stmtSet = util.tableEnv.createStatementSet() val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a") @@ -116,7 +116,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase { util.verifyExplain(stmtSet, extraDetails: _*) } - @Test + @TestTemplate def testExplainMultipleInput(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") @@ -133,7 +133,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase { } object ExplainTest { - @Parameterized.Parameters(name = "extended={0}") + @Parameters(name = "extended={0}") def parameters(): java.util.Collection[Boolean] = { java.util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala index 39d2326e332a8..8fce44b2dac87 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala @@ -23,15 +23,15 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.sql.Timestamp import java.time.Duration -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class ExplainTest(extended: Boolean) extends TableTestBase { private val extraDetails = if (extended) { @@ -49,55 +49,55 @@ class ExplainTest(extended: Boolean) extends TableTestBase { val LONG = new BigIntType() val INT = new IntType() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, Int.box(4)) } - @Test + @TestTemplate def testExplainTableSourceScan(): Unit = { util.verifyExplain("SELECT * FROM MyTable", extraDetails: _*) } - @Test + @TestTemplate def testExplainDataStreamScan(): Unit = { util.verifyExplain("SELECT * FROM MyTable1", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithFilter(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithAgg(): Unit = { util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithJoin(): Unit = { util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithUnion(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithSort(): Unit = { util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithSingleSink(): Unit = { val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10") val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING)) util.verifyExplainInsert(table, appendSink, "appendSink", extraDetails: _*) } - @Test + @TestTemplate def testExplainWithMultiSinks(): Unit = { val stmtSet = util.tableEnv.createStatementSet() val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a") @@ -120,7 +120,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase { util.verifyExplain(stmtSet, extraDetails: _*) } - @Test + @TestTemplate def testMiniBatchIntervalInfer(): Unit = { val stmtSet = util.tableEnv.createStatementSet() // Test emit latency propagate among RelNodeBlocks @@ -176,7 +176,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase { } object ExplainTest { - @Parameterized.Parameters(name = "extended={0}") + @Parameters(name = "extended={0}") def parameters(): java.util.Collection[Boolean] = { java.util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/OverWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/OverWindowValidationTest.scala index 60a7e24789116..e171ab8c56b36 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/OverWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/OverWindowValidationTest.scala @@ -24,7 +24,8 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions. import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.types.Row -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class OverWindowValidationTest extends TableTestBase { @@ -32,7 +33,7 @@ class OverWindowValidationTest extends TableTestBase { streamUtil.addDataStream[(Int, String, Long)]("T1", 'a, 'b, 'c, 'proctime.proctime) /** All aggregates must be computed on the same window. */ - @Test(expected = classOf[TableException]) + @Test def testMultiWindow(): Unit = { val sqlQuery = "SELECT " + @@ -41,16 +42,18 @@ class OverWindowValidationTest extends TableTestBase { "sum(a) OVER (PARTITION BY b ORDER BY proctime RANGE UNBOUNDED preceding) " + "from T1" - streamUtil.tableEnv.sqlQuery(sqlQuery).toAppendStream[Row] + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => streamUtil.tableEnv.sqlQuery(sqlQuery).toAppendStream[Row]) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation(): Unit = { streamUtil.addFunction("overAgg", new OverAgg0) val sqlQuery = "SELECT overAgg(c, a) FROM MyTable" - streamUtil.tableEnv.sqlQuery(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => streamUtil.tableEnv.sqlQuery(sqlQuery)) } } From 4cd8285274a0b48b2fbce57f56872e2b1f1befa2 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 14:49:09 +0800 Subject: [PATCH 099/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-connectors/flink-connector-hive (TableTestBase) --- .../hive/HiveTemporalJoinITCase.java | 49 +++++++++++-------- 1 file changed, 29 insertions(+), 20 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java index 3f1685fb0aef7..0b238aff180a3 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java @@ -31,13 +31,15 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.types.Row; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.sql.Timestamp; import java.util.Arrays; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + /** * Test Temporal join of hive tables. * @@ -45,13 +47,13 @@ * 3.1.1. To run this test, please use mvn command: mvn test -Phive-3.1.1 * -Dtest=org.apache.flink.connectors.hive.HiveTemporalJoinITCase */ -public class HiveTemporalJoinITCase extends TableTestBase { +class HiveTemporalJoinITCase extends TableTestBase { private static TableEnvironment tableEnv; private static HiveCatalog hiveCatalog; - @BeforeClass - public static void setup() { + @BeforeAll + static void setup() { if (!HiveVersionTestUtil.HIVE_310_OR_LATER) { return; } @@ -96,7 +98,7 @@ public static void setup() { } @Test - public void testProcTimeTemporalJoinHiveTable() throws Exception { + void testProcTimeTemporalJoinHiveTable() throws Exception { if (!HiveVersionTestUtil.HIVE_310_OR_LATER) { return; } @@ -104,16 +106,22 @@ public void testProcTimeTemporalJoinHiveTable() throws Exception { tableEnv.executeSql("insert into build values (1,'a',10),(2,'a',21),(2,'b',22),(3,'c',33)") .await(); - expectedException().expect(TableException.class); - expectedException().expectMessage("Processing-time temporal join is not supported yet."); tableEnv.executeSql( "select p.x, p.y, b.z from " + " default_catalog.default_database.probe as p " + " join build for system_time as of p.p as b on p.x=b.x and p.y=b.y"); + assertThatThrownBy( + () -> + tableEnv.executeSql( + "select p.x, p.y, b.z from " + + " default_catalog.default_database.probe as p " + + " join build for system_time as of p.p as b on p.x=b.x and p.y=b.y")) + .hasMessageContaining("Processing-time temporal join is not supported yet.") + .isInstanceOf(TableException.class); } @Test - public void testRowTimeTemporalJoinHiveTable() throws Exception { + void testRowTimeTemporalJoinHiveTable() throws Exception { if (!HiveVersionTestUtil.HIVE_310_OR_LATER) { return; } @@ -122,19 +130,20 @@ public void testRowTimeTemporalJoinHiveTable() throws Exception { .await(); // Streaming hive table does not support defines watermark - expectedException().expect(ValidationException.class); - expectedException() - .expectMessage( + assertThatThrownBy( + () -> + tableEnv.executeSql( + "select p.x, p.y, b.z from " + + " default_catalog.default_database.probe as p " + + " join build for system_time as of p.rowtime as b on p.x=b.x and p.y=b.y")) + .hasMessageContaining( "Event-Time Temporal Table Join requires both primary key" - + " and row time attribute in versioned table, but no row time attribute can be found."); - tableEnv.executeSql( - "select p.x, p.y, b.z from " - + " default_catalog.default_database.probe as p " - + " join build for system_time as of p.rowtime as b on p.x=b.x and p.y=b.y"); + + " and row time attribute in versioned table, but no row time attribute can be found.") + .isInstanceOf(ValidationException.class); } - @AfterClass - public static void tearDown() { + @AfterAll + static void tearDown() { if (!HiveVersionTestUtil.HIVE_310_OR_LATER) { return; } From 6564bccbcb219a1e1eb641e2b54c51842cb56762 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 14:55:19 +0800 Subject: [PATCH 100/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/connector (TableTestBase) --- .../file/table/FileSystemTableSourceTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java index 3b3b71a7e10b4..358f5ed5776c0 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java @@ -23,16 +23,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test for {@link FileSystemTableSource}. */ -public class FileSystemTableSourceTest extends TableTestBase { +class FileSystemTableSourceTest extends TableTestBase { private StreamTableTestUtil util; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); TableEnvironment tEnv = util.getTableEnv(); @@ -71,12 +71,12 @@ public void setup() { } @Test - public void testFilterPushDown() { + void testFilterPushDown() { util.verifyRelPlanInsert("insert into MySink select * from MyTable where a > 10"); } @Test - public void testMetadataReading() { + void testMetadataReading() { util.verifyRelPlanInsert( "insert into MySink(a, b, c) select a, b, filemeta from MyTableWithMeta"); } From f44b6764cbd7708748b27b2ab49cf81980819806 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 15:39:31 +0800 Subject: [PATCH 101/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/planner/alias (TableTestBase) --- ...nHintWithCapitalizeJoinHintShuttleTest.java | 14 +++++++------- ...nHintWithInvalidPropagationShuttleTest.java | 12 ++++++------ ...tWithInvalidPropagationShuttleTestBase.java | 8 ++++---- ...nHintWithInvalidPropagationShuttleTest.java | 18 +++++++++--------- 4 files changed, 26 insertions(+), 26 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithCapitalizeJoinHintShuttleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithCapitalizeJoinHintShuttleTest.java index 210cc122c7d23..86b0157502ad6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithCapitalizeJoinHintShuttleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithCapitalizeJoinHintShuttleTest.java @@ -31,13 +31,13 @@ import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Collections; /** Tests clearing lookup join hint case-insensitive in stream. */ -public class ClearJoinHintWithCapitalizeJoinHintShuttleTest +class ClearJoinHintWithCapitalizeJoinHintShuttleTest extends ClearJoinHintWithInvalidPropagationShuttleTestBase { @Override @@ -50,8 +50,8 @@ boolean isBatchMode() { return false; } - @Before - public void before() throws Exception { + @BeforeEach + void before() throws Exception { super.before(); util.tableEnv() @@ -72,7 +72,7 @@ public void before() throws Exception { } @Test - public void testClearCaseInsensitiveLookupHint() { + void testClearCaseInsensitiveLookupHint() { // SELECT /*+ lookUp('table'='lookup', 'retry-predicate'='lookup_miss', // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 'max-attempts'='10', // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 'time-out'='300 s') @@ -130,7 +130,7 @@ public void testClearCaseInsensitiveLookupHint() { } @Override - public void verifyRelPlan(RelNode node) { + protected void verifyRelPlan(RelNode node) { String plan = buildRelPlanWithQueryBlockAlias(node); util.assertEqualsOrExpand("beforePropagatingHints", plan, true); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java index 6801c602539d2..d4ebef36706cd 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java @@ -26,10 +26,10 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.hint.RelHint; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** Tests clearing join hint with invalid propagation in batch. */ -public class ClearJoinHintWithInvalidPropagationShuttleTest +class ClearJoinHintWithInvalidPropagationShuttleTest extends ClearJoinHintWithInvalidPropagationShuttleTestBase { @Override TableTestUtil getTableTestUtil() { @@ -42,7 +42,7 @@ boolean isBatchMode() { } @Test - public void testNoNeedToClearJoinHint() { + void testNoNeedToClearJoinHint() { // SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a RelHint joinHintInView = RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build(); @@ -60,7 +60,7 @@ public void testNoNeedToClearJoinHint() { } @Test - public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints() { + void testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints() { // SELECT /*+ BROADCAST(t3)*/t4.a FROM ( // SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a // ) t4 JOIN t3 ON t4.a = t3.a @@ -91,7 +91,7 @@ public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints() } @Test - public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints() { + void testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints() { // SELECT /*+ BROADCAST(t3)*/t4.a FROM ( // SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a // ) t4 JOIN t3 ON t4.a = t3.a @@ -119,7 +119,7 @@ public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints } @Test - public void testClearJoinHintWithoutPropagatingToView() { + void testClearJoinHintWithoutPropagatingToView() { // SELECT /*+ BROADCAST(t1)*/t4.a FROM ( // SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a // ) t4 JOIN t3 ON t4.a = t3.a diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTestBase.java index 105f4e42a38d3..745becc26ff81 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTestBase.java @@ -34,12 +34,12 @@ import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.sql.SqlExplainLevel; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import java.util.Collections; /** A base class for testing clearing join hint with invalid propagation. */ -public abstract class ClearJoinHintWithInvalidPropagationShuttleTestBase extends TableTestBase { +abstract class ClearJoinHintWithInvalidPropagationShuttleTestBase extends TableTestBase { protected final TableTestUtil util = getTableTestUtil(); @@ -67,8 +67,8 @@ public abstract class ClearJoinHintWithInvalidPropagationShuttleTestBase extends .build(); protected final FlinkRelBuilder builder = plannerMocks.getPlannerContext().createRelBuilder(); - @Before - public void before() throws Exception { + @BeforeEach + void before() throws Exception { util.tableEnv().registerCatalog("testCatalog", catalog); util.tableEnv().executeSql("use catalog testCatalog"); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupJoinHintWithInvalidPropagationShuttleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupJoinHintWithInvalidPropagationShuttleTest.java index 13c8f7e6eb54a..0e89a4aa2b285 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupJoinHintWithInvalidPropagationShuttleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupJoinHintWithInvalidPropagationShuttleTest.java @@ -38,13 +38,13 @@ import org.apache.calcite.sql.fun.SqlCollectionTableOperator; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlModality; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Collections; /** Tests clearing lookup join hint with invalid propagation in stream. */ -public class ClearLookupJoinHintWithInvalidPropagationShuttleTest +class ClearLookupJoinHintWithInvalidPropagationShuttleTest extends ClearJoinHintWithInvalidPropagationShuttleTestBase { @Override TableTestUtil getTableTestUtil() { @@ -56,8 +56,8 @@ boolean isBatchMode() { return false; } - @Before - public void before() throws Exception { + @BeforeEach + void before() throws Exception { super.before(); util.tableEnv() @@ -84,7 +84,7 @@ public void before() throws Exception { } @Test - public void testNoNeedToClearLookupHint() { + void testNoNeedToClearLookupHint() { // SELECT /*+ LOOKUP('table'='lookup', 'retry-predicate'='lookup_miss', // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 'max-attempts'='10', // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 'time-out'='300 s') @@ -129,7 +129,7 @@ public void testNoNeedToClearLookupHint() { } @Test - public void testClearLookupHintWithInvalidPropagationToSubQuery() { + void testClearLookupHintWithInvalidPropagationToSubQuery() { // SELECT /*+ LOOKUP('table'='src', 'retry-predicate'='lookup_miss', // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 'max-attempts'='10', // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 'time-out'='300 s') @@ -187,7 +187,7 @@ public void testClearLookupHintWithInvalidPropagationToSubQuery() { } @Test - public void testNoNeedToClearLookupHintWhileJoinWithUnnest() { + void testNoNeedToClearLookupHintWhileJoinWithUnnest() { // SELECT /*+ LOOKUP('table'='d', 'retry-predicate'='lookup_miss', // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 'max-attempts'='10', // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 'time-out'='300 s') @@ -224,7 +224,7 @@ public void testNoNeedToClearLookupHintWhileJoinWithUnnest() { } @Test - public void testNoNeedToClearLookupHintWhileJoinWithUDTF() { + void testNoNeedToClearLookupHintWhileJoinWithUDTF() { // SELECT /*+ LOOKUP('table'='d', 'retry-predicate'='lookup_miss', // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 'max-attempts'='10', // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 'time-out'='300 s') From b9923dee7890b402b499303915b7f92cf613eb87 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 15:40:26 +0800 Subject: [PATCH 102/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/planner/analyze (TableTestBase) --- .../analyze/GroupAggregationAnalyzerTest.java | 56 ++++++++++--------- .../NonDeterministicUpdateAnalyzerTest.java | 24 ++++---- 2 files changed, 41 insertions(+), 39 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.java index 6dc02bbc75839..1c48e72300028 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.java @@ -28,11 +28,13 @@ import org.apache.flink.table.planner.utils.PlanKind; import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; @@ -41,21 +43,21 @@ import static scala.runtime.BoxedUnit.UNIT; /** Test for {@link GroupAggregationAnalyzer}. */ -@RunWith(Parameterized.class) -public class GroupAggregationAnalyzerTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class GroupAggregationAnalyzerTest extends TableTestBase { private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); - @Parameterized.Parameter public boolean isMiniBatchEnabled; + @Parameter private boolean isMiniBatchEnabled; - @Parameterized.Parameter(1) - public AggregatePhaseStrategy strategy; + @Parameter(1) + private AggregatePhaseStrategy strategy; - @Parameterized.Parameter(2) - public long miniBatchLatency; + @Parameter(2) + private long miniBatchLatency; - @Parameterized.Parameter(3) - public long miniBatchSize; + @Parameter(3) + private long miniBatchSize; private final String query = "SELECT\n" @@ -66,8 +68,8 @@ public class GroupAggregationAnalyzerTest extends TableTestBase { + " MAX(c) FILTER (WHERE a > 1) AS max_c\n" + "FROM MyTable"; - @Before - public void before() { + @BeforeEach + void before() { util.getTableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled) @@ -99,8 +101,8 @@ public void before() { + " 'sink-insert-only' = 'false')"); } - @Test - public void testSelect() { + @TestTemplate + void testSelect() { util.doVerifyPlan( query, new ExplainDetail[] {ExplainDetail.PLAN_ADVICE}, @@ -109,8 +111,8 @@ public void testSelect() { false); } - @Test - public void testInsertInto() { + @TestTemplate + void testInsertInto() { util.doVerifyPlanInsert( String.format("INSERT INTO MySink\n%s", query), new ExplainDetail[] {ExplainDetail.PLAN_ADVICE}, @@ -118,8 +120,8 @@ public void testInsertInto() { new Enumeration.Value[] {PlanKind.OPT_REL_WITH_ADVICE()}); } - @Test - public void testStatementSet() { + @TestTemplate + void testStatementSet() { StatementSet stmtSet = util.getTableEnv().createStatementSet(); util.getTableEnv().executeSql("CREATE TABLE MySink2 LIKE MySink"); util.getTableEnv() @@ -151,8 +153,8 @@ public void testStatementSet() { false); } - @Test - public void testSubplanReuse() { + @TestTemplate + void testSubplanReuse() { util.doVerifyPlan( "WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM MyTable GROUP BY c)\n" + "SELECT * FROM r r1, r r2 WHERE r1.a = CAST(r2.b AS BIGINT) AND r2.a > 1", @@ -162,8 +164,8 @@ public void testSubplanReuse() { false); } - @Test - public void testUserDefinedAggCalls() { + @TestTemplate + void testUserDefinedAggCalls() { StatementSet stmtSet = util.getTableEnv().createStatementSet(); util.addTemporarySystemFunction( "weightedAvg", JavaUserDefinedAggFunctions.WeightedAvgWithMerge.class); @@ -199,9 +201,9 @@ public void testUserDefinedAggCalls() { false); } - @Parameterized.Parameters( + @Parameters( name = "isMiniBatchEnabled={0}, strategy={1}, miniBatchLatency={2}, miniBatchSize={3}") - public static Object[][] data() { + private static Object[][] data() { return new Object[][] { new Object[] {true, AggregatePhaseStrategy.ONE_PHASE, 10L, 5L}, new Object[] {true, AggregatePhaseStrategy.AUTO, 10L, 5L}, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java index 7ee9cc1270069..5c2ac6316f161 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java @@ -26,20 +26,20 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import scala.Enumeration; import static scala.runtime.BoxedUnit.UNIT; /** Test for {@link NonDeterministicUpdateAnalyzer}. */ -public class NonDeterministicUpdateAnalyzerTest extends TableTestBase { +class NonDeterministicUpdateAnalyzerTest extends TableTestBase { private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); - @Before - public void before() { + @BeforeEach + void before() { util.getTableEnv() .executeSql( "create temporary table cdc (\n" @@ -104,7 +104,7 @@ public void before() { } @Test - public void testCdcWithMetaRenameSinkWithCompositePk() { + void testCdcWithMetaRenameSinkWithCompositePk() { // from NonDeterministicDagTest#testCdcWithMetaRenameSinkWithCompositePk util.getTableEnv() .executeSql( @@ -142,7 +142,7 @@ public void testCdcWithMetaRenameSinkWithCompositePk() { } @Test - public void testSourceWithComputedColumnSinkWithPk() { + void testSourceWithComputedColumnSinkWithPk() { // from NonDeterministicDagTest#testSourceWithComputedColumnSinkWithPk util.getTableEnv() .executeSql( @@ -168,7 +168,7 @@ public void testSourceWithComputedColumnSinkWithPk() { } @Test - public void testCdcJoinDimWithPkNonDeterministicLocalCondition() { + void testCdcJoinDimWithPkNonDeterministicLocalCondition() { // from NonDeterministicDagTest#testCdcJoinDimWithPkNonDeterministicLocalCondition util.doVerifyPlanInsert( "insert into sink_without_pk\n" @@ -183,7 +183,7 @@ public void testCdcJoinDimWithPkNonDeterministicLocalCondition() { } @Test - public void testCdcWithMetaSinkWithPk() { + void testCdcWithMetaSinkWithPk() { // from NonDeterministicDagTest#testCdcWithMetaSinkWithPk util.doVerifyPlanInsert( "insert into sink_with_pk\n" + "select a, metadata_3, c\n" + "from cdc_with_meta", @@ -193,7 +193,7 @@ public void testCdcWithMetaSinkWithPk() { } @Test - public void testGroupByNonDeterministicFuncWithCdcSource() { + void testGroupByNonDeterministicFuncWithCdcSource() { // from NonDeterministicDagTest#testGroupByNonDeterministicFuncWithCdcSource util.doVerifyPlanInsert( "insert into sink_with_pk\n" @@ -209,7 +209,7 @@ public void testGroupByNonDeterministicFuncWithCdcSource() { } @Test - public void testMultiSinkOnJoinedView() { + void testMultiSinkOnJoinedView() { // from NonDeterministicDagTest#testMultiSinkOnJoinedView util.getTableEnv() .executeSql( @@ -293,7 +293,7 @@ public void testMultiSinkOnJoinedView() { } @Test - public void testCdcJoinDimWithPkOutputNoPkSinkWithoutPk() { + void testCdcJoinDimWithPkOutputNoPkSinkWithoutPk() { // from NonDeterministicDagTest#testCdcJoinDimWithPkOutputNoPkSinkWithoutPk util.doVerifyPlanInsert( "insert into sink_without_pk\n" From 1fbe223a5d7b9232d58f471ae620306b9dad58ea Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 15:42:18 +0800 Subject: [PATCH 103/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/planner/catalog,expressions,runtime (TableTestBase) --- .../planner/catalog/JavaCatalogTableTest.java | 34 ++++++++++--------- .../runtime/batch/ParallelismSettingTest.java | 20 +++++------ 2 files changed, 28 insertions(+), 26 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.java index 1bec0f771eb2d..fbb5d8e89f559 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.java @@ -30,10 +30,12 @@ import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -49,14 +51,14 @@ /** * Tests for resolving types of computed columns (including time attributes) of tables from catalog. */ -@RunWith(Parameterized.class) -public class JavaCatalogTableTest extends TableTestBase { - @Parameterized.Parameters(name = "streamingMode = {0}") - public static Collection parameters() { +@ExtendWith(ParameterizedTestExtension.class) +class JavaCatalogTableTest extends TableTestBase { + @Parameters(name = "streamingMode = {0}") + private static Collection parameters() { return Arrays.asList(true, false); } - @Parameterized.Parameter public boolean isStreamingMode; + @Parameter private boolean isStreamingMode; private TableTestUtil getTestUtil() { if (isStreamingMode) { @@ -66,8 +68,8 @@ private TableTestUtil getTestUtil() { } } - @Test - public void testResolvingSchemaOfCustomCatalogTableSql() throws Exception { + @TestTemplate + void testResolvingSchemaOfCustomCatalogTableSql() throws Exception { TableTestUtil testUtil = getTestUtil(); TableEnvironment tableEnvironment = testUtil.getTableEnv(); GenericInMemoryCatalog genericInMemoryCatalog = new GenericInMemoryCatalog("in-memory"); @@ -83,8 +85,8 @@ public void testResolvingSchemaOfCustomCatalogTableSql() throws Exception { "SELECT COUNT(*) FROM testTable2 GROUP BY TUMBLE(rowtime, INTERVAL '10' MINUTE)"); } - @Test - public void testResolvingSchemaOfCustomCatalogTableTableApi() throws Exception { + @TestTemplate + void testResolvingSchemaOfCustomCatalogTableTableApi() throws Exception { TableTestUtil testUtil = getTestUtil(); TableEnvironment tableEnvironment = testUtil.getTableEnv(); GenericInMemoryCatalog genericInMemoryCatalog = new GenericInMemoryCatalog("in-memory"); @@ -103,8 +105,8 @@ public void testResolvingSchemaOfCustomCatalogTableTableApi() throws Exception { testUtil.verifyExecPlan(table); } - @Test - public void testResolvingProctimeOfCustomTableSql() throws Exception { + @TestTemplate + void testResolvingProctimeOfCustomTableSql() throws Exception { if (!isStreamingMode) { // proctime not supported in batch return; @@ -123,8 +125,8 @@ public void testResolvingProctimeOfCustomTableSql() throws Exception { + "GROUP BY TUMBLE(proctime, INTERVAL '10' MINUTE)"); } - @Test - public void testResolvingProctimeOfCustomTableTableApi() throws Exception { + @TestTemplate + void testResolvingProctimeOfCustomTableTableApi() throws Exception { if (!isStreamingMode) { // proctime not supported in batch return; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/ParallelismSettingTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/ParallelismSettingTest.java index e5115a4d3e858..844a0fbec333f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/ParallelismSettingTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/ParallelismSettingTest.java @@ -25,8 +25,8 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.List; @@ -38,8 +38,8 @@ public class ParallelismSettingTest extends TableTestBase { private BatchTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.getTableEnv() .getConfig() @@ -58,7 +58,7 @@ public void before() { } @Test - public void testParallelismSettingAfterSingletonShuffleRemove() { + void testParallelismSettingAfterSingletonShuffleRemove() { util.getTableEnv() .executeSql( "CREATE TABLE MySink (\n" @@ -88,7 +88,7 @@ public void testParallelismSettingAfterSingletonShuffleRemove() { } @Test - public void testSortQuery() { + void testSortQuery() { util.getTableEnv() .executeSql( "CREATE TABLE MySink (\n" @@ -121,7 +121,7 @@ public void testSortQuery() { } @Test - public void testLimitQuery() { + void testLimitQuery() { util.getTableEnv() .executeSql( "CREATE TABLE MySink (\n" @@ -155,7 +155,7 @@ public void testLimitQuery() { } @Test - public void testSortLimitQuery() { + void testSortLimitQuery() { util.getTableEnv() .executeSql( "CREATE TABLE MySink (\n" @@ -190,7 +190,7 @@ public void testSortLimitQuery() { } @Test - public void testRankQuery() { + void testRankQuery() { util.getTableEnv() .executeSql( "CREATE TABLE MySink (\n" @@ -241,7 +241,7 @@ public void testRankQuery() { } @Test - public void testJoinQuery() { + void testJoinQuery() { util.tableEnv() .executeSql( "CREATE TABLE MyTable2 (\n" From cdb759b0ecda97bb04912553c7453710a07d499d Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Tue, 12 Sep 2023 15:43:23 +0800 Subject: [PATCH 104/104] [FLINK-33023][table-planner][JUnit5 Migration] Module: flink-table-planner/planner/plan (TableTestBase) --- .../planner/plan/batch/sql/CalcMergeTest.java | 2 +- .../plan/batch/sql/DynamicFilteringTest.java | 47 +- .../batch/sql/DynamicFunctionPlanTest.java | 2 +- .../batch/sql/ForwardHashExchangeTest.java | 52 +- .../plan/batch/sql/MatchRecognizeTest.java | 18 +- .../plan/batch/sql/RowLevelDeleteTest.java | 43 +- .../plan/batch/sql/RowLevelUpdateTest.java | 66 +- .../plan/batch/sql/TimeTravelTest.java | 30 +- .../plan/common/CalcMergeTestBase.java | 26 +- .../common/DynamicFunctionPlanTestBase.java | 16 +- .../hints/batch/BroadcastJoinHintTest.java | 2 +- .../plan/hints/batch/JoinHintTestBase.java | 208 +-- .../hints/batch/NestLoopJoinHintTest.java | 2 +- .../hints/batch/ShuffleHashJoinHintTest.java | 2 +- .../hints/batch/ShuffleMergeJoinHintTest.java | 2 +- .../exec/UnsupportedNodesInPlanTest.java | 6 +- .../exec/operator/BatchOperatorNameTest.java | 52 +- .../exec/operator/OperatorNameTestBase.java | 44 +- .../exec/operator/StreamOperatorNameTest.java | 86 +- ...ultipleInputNodeCreationProcessorTest.java | 15 +- .../nodes/exec/stream/CalcJsonPlanTest.java | 20 +- .../stream/ChangelogSourceJsonPlanTest.java | 14 +- .../exec/stream/CorrelateJsonPlanTest.java | 24 +- .../stream/DeduplicationJsonPlanTest.java | 12 +- .../nodes/exec/stream/ExpandJsonPlanTest.java | 12 +- .../stream/GroupAggregateJsonPlanTest.java | 40 +- .../GroupWindowAggregateJsonPlanTest.java | 22 +- .../IncrementalAggregateJsonPlanTest.java | 14 +- .../exec/stream/IntervalJoinJsonPlanTest.java | 14 +- .../nodes/exec/stream/JoinJsonPlanTest.java | 18 +- .../exec/stream/LookupJoinJsonPlanTest.java | 34 +- .../stream/MatchRecognizeJsonPlanTest.java | 20 +- .../stream/OverAggregateJsonPlanTest.java | 24 +- .../exec/stream/PythonCalcJsonPlanTest.java | 14 +- .../stream/PythonCorrelateJsonPlanTest.java | 14 +- .../PythonGroupAggregateJsonPlanTest.java | 12 +- ...ythonGroupWindowAggregateJsonPlanTest.java | 22 +- .../PythonOverAggregateJsonPlanTest.java | 20 +- .../nodes/exec/stream/RankJsonPlanTest.java | 12 +- .../nodes/exec/stream/SortJsonPlanTest.java | 12 +- .../exec/stream/SortLimitJsonPlanTest.java | 12 +- .../exec/stream/TableSinkJsonPlanTest.java | 20 +- .../exec/stream/TableSourceJsonPlanTest.java | 24 +- .../exec/stream/TemporalJoinJsonPlanTest.java | 14 +- .../exec/stream/TemporalSortJsonPlanTest.java | 14 +- .../nodes/exec/stream/UnionJsonPlanTest.java | 12 +- .../nodes/exec/stream/ValuesJsonPlanTest.java | 12 +- .../stream/WatermarkAssignerJsonPlanTest.java | 12 +- .../stream/WindowAggregateJsonPlanTest.java | 30 +- .../exec/stream/WindowJoinJsonPlanTest.java | 12 +- .../WindowTableFunctionJsonPlanTest.java | 20 +- .../ClearQueryBlockAliasResolverTest.java | 2 +- .../planner/plan/optimize/ScanReuseTest.java | 125 +- .../DynamicPartitionPruningProgramTest.java | 91 +- .../FlinkRuntimeFilterProgramTest.java | 60 +- .../logical/FlinkFilterJoinRuleTest.java | 116 +- .../ProjectSnapshotTransposeRuleTest.java | 40 +- ...ectWatermarkAssignerTransposeRuleTest.java | 32 +- ...shFilterInCalcIntoTableSourceRuleTest.java | 17 +- ...PushFilterIntoTableSourceScanRuleTest.java | 25 +- ...FilterIntoTableSourceScanRuleTestBase.java | 44 +- .../PushLimitIntoTableSourceScanRuleTest.java | 6 +- ...hPartitionIntoTableSourceScanRuleTest.java | 13 +- ...ushProjectIntoTableSourceScanRuleTest.java | 44 +- ...hWatermarkIntoTableSourceScanRuleTest.java | 40 +- ...eUnreachableCoalesceArgumentsRuleTest.java | 24 +- .../WrapJsonAggFunctionArgumentsRuleTest.java | 63 +- ...shLocalAggIntoTableSourceScanRuleTest.java | 44 +- ...ushCalcPastChangelogNormalizeRuleTest.java | 22 +- .../plan/stream/sql/CalcMergeTest.java | 2 +- .../stream/sql/DynamicFunctionPlanTest.java | 2 +- .../plan/stream/sql/InsertIntoValuesTest.java | 6 +- .../stream/sql/LegacyTableFactoryTest.java | 10 +- .../planner/plan/stream/sql/NullTypeTest.java | 79 +- .../planner/plan/stream/table/ValuesTest.java | 90 +- .../plan/common/ViewsExpandingTest.xml | 20 +- .../planner/plan/hint/OptionsHintTest.xml | 20 +- .../planner/plan/batch/sql/CalcTest.scala | 10 +- .../batch/sql/CompactManagedTableTest.scala | 6 +- .../plan/batch/sql/DagOptimizationTest.scala | 2 +- .../plan/batch/sql/DeadlockBreakupTest.scala | 4 +- .../plan/batch/sql/LegacyLimitTest.scala | 15 +- .../plan/batch/sql/LegacySinkTest.scala | 3 +- .../batch/sql/LegacyTableSourceTest.scala | 13 +- .../planner/plan/batch/sql/LimitTest.scala | 3 + .../batch/sql/MultipleInputCreationTest.scala | 45 +- .../batch/sql/OperatorFusionCodegenTest.scala | 18 +- .../batch/sql/PartitionableSinkTest.scala | 17 +- .../batch/sql/PartitionableSourceTest.scala | 26 +- .../planner/plan/batch/sql/RankTest.scala | 30 +- .../plan/batch/sql/RemoveCollationTest.scala | 4 +- .../plan/batch/sql/RemoveShuffleTest.scala | 4 +- .../plan/batch/sql/SetOperatorsTest.scala | 27 +- .../plan/batch/sql/SortLimitTest.scala | 2 +- .../planner/plan/batch/sql/SortTest.scala | 2 +- .../plan/batch/sql/SubplanReuseTest.scala | 4 +- .../plan/batch/sql/TableScanTest.scala | 43 +- .../plan/batch/sql/TableSinkTest.scala | 8 +- .../plan/batch/sql/TableSourceTest.scala | 4 +- .../planner/plan/batch/sql/UnionTest.scala | 4 +- .../planner/plan/batch/sql/ValuesTest.scala | 2 +- .../batch/sql/WindowTableFunctionTest.scala | 38 +- .../batch/sql/agg/AggregateTestBase.scala | 48 +- .../plan/batch/sql/agg/GroupWindowTest.scala | 153 +- .../plan/batch/sql/agg/GroupingSetsTest.scala | 6 +- .../batch/sql/agg/HashAggregateTest.scala | 34 +- .../batch/sql/agg/OverAggregateTest.scala | 30 +- .../batch/sql/agg/SortAggregateTest.scala | 14 +- .../sql/join/BroadcastHashJoinTest.scala | 95 +- .../join/BroadcastHashSemiAntiJoinTest.scala | 125 +- .../plan/batch/sql/join/JoinReorderTest.scala | 6 +- .../plan/batch/sql/join/JoinTestBase.scala | 18 +- .../plan/batch/sql/join/LookupJoinTest.scala | 108 +- .../batch/sql/join/NestedLoopJoinTest.scala | 4 +- .../sql/join/NestedLoopSemiAntiJoinTest.scala | 4 +- .../batch/sql/join/SemiAntiJoinTest.scala | 2 +- .../batch/sql/join/SemiAntiJoinTestBase.scala | 3 +- .../batch/sql/join/ShuffledHashJoinTest.scala | 71 +- .../join/ShuffledHashSemiAntiJoinTest.scala | 113 +- .../batch/sql/join/SingleRowJoinTest.scala | 2 +- .../batch/sql/join/SortMergeJoinTest.scala | 71 +- .../sql/join/SortMergeSemiAntiJoinTest.scala | 113 +- .../sql/join/TemporalFunctionJoinTest.scala | 30 +- .../batch/sql/join/TemporalJoinTest.scala | 25 +- .../plan/batch/table/AggregateTest.scala | 2 +- .../planner/plan/batch/table/CalcTest.scala | 2 +- .../batch/table/ColumnFunctionsTest.scala | 2 +- .../plan/batch/table/CorrelateTest.scala | 2 +- .../plan/batch/table/GroupWindowTest.scala | 18 +- .../planner/plan/batch/table/JoinTest.scala | 18 +- .../batch/table/PythonAggregateTest.scala | 8 +- .../plan/batch/table/PythonCalcTest.scala | 4 +- .../PythonGroupWindowAggregateTest.scala | 18 +- .../table/PythonOverWindowAggregateTest.scala | 8 +- .../plan/batch/table/SetOperatorsTest.scala | 2 +- .../table/TemporalTableFunctionJoinTest.scala | 18 +- .../validation/AggregateValidationTest.scala | 142 +- .../table/validation/CalcValidationTest.scala | 72 +- .../validation/CorrelateValidationTest.scala | 18 +- .../GroupWindowValidationTest.scala | 142 +- .../table/validation/JoinValidationTest.scala | 79 +- .../validation/OverWindowValidationTest.scala | 21 +- .../SetOperatorsValidationTest.scala | 38 +- .../table/validation/SortValidationTest.scala | 18 +- .../AggregateReduceGroupingTestBase.scala | 4 +- .../common/DistinctAggregateTestBase.scala | 10 +- .../plan/common/JoinReorderTestBase.scala | 58 +- .../plan/common/PartialInsertTest.scala | 78 +- .../plan/common/TableFactoryTest.scala | 19 +- .../planner/plan/common/UnnestTestBase.scala | 2 +- .../plan/common/ViewsExpandingTest.scala | 57 +- .../planner/plan/hint/OptionsHintTest.scala | 57 +- .../AggregateReduceGroupingRuleTest.scala | 4 +- .../CalcPruneAggregateCallRuleTest.scala | 2 + ...CalcPythonCorrelateTransposeRuleTest.scala | 4 +- .../logical/CalcRankTransposeRuleTest.scala | 4 +- .../logical/ConvertToNotInOrInRuleTest.scala | 4 +- .../logical/CorrelateSortToRankRuleTest.scala | 4 +- .../DecomposeGroupingSetsRuleTest.scala | 8 +- .../ExpressionReductionRulesTest.scala | 2 +- ...gateExpandDistinctAggregatesRuleTest.scala | 4 +- .../FlinkAggregateJoinTransposeRuleTest.scala | 4 +- .../FlinkAggregateRemoveRuleTest.scala | 4 +- .../logical/FlinkCalcMergeRuleTest.scala | 4 +- .../FlinkJoinPushExpressionsRuleTest.scala | 4 +- .../FlinkJoinToMultiJoinRuleTest.scala | 4 +- .../logical/FlinkLimit0RemoveRuleTest.scala | 4 +- ...kLogicalRankRuleForConstantRangeTest.scala | 2 +- .../FlinkLogicalRankRuleForRangeEndTest.scala | 9 +- .../logical/FlinkPruneEmptyRulesTest.scala | 4 +- ...kSemiAntiJoinFilterTransposeRuleTest.scala | 4 +- ...inkSemiAntiJoinJoinTransposeRuleTest.scala | 4 +- ...SemiAntiJoinProjectTransposeRuleTest.scala | 4 +- ...oinConditionEqualityTransferRuleTest.scala | 4 +- .../JoinConditionTypeCoerceRuleTest.scala | 4 +- ...DependentConditionDerivationRuleTest.scala | 4 +- .../JoinDeriveNullFilterRuleTest.scala | 4 +- ...elateToJoinFromTemporalTableRuleTest.scala | 55 +- .../rules/logical/LogicalUnnestRuleTest.scala | 4 +- .../ProjectPruneAggregateCallRuleTest.scala | 2 + ...ProjectSemiAntiJoinTransposeRuleTest.scala | 4 +- ...WindowTableFunctionTransposeRuleTest.scala | 4 +- .../PruneAggregateCallRuleTestBase.scala | 4 +- ...terIntoLegacyTableSourceScanRuleTest.scala | 4 +- ...mitIntoLegacyTableSourceScanRuleTest.scala | 20 +- ...ionIntoLegacyTableSourceScanRuleTest.scala | 48 +- ...ectIntoLegacyTableSourceScanRuleTest.scala | 4 +- .../logical/PythonCalcSplitRuleTest.scala | 4 +- .../PythonCorrelateSplitRuleTest.scala | 4 +- .../logical/PythonMapMergeRuleTest.scala | 4 +- .../RankNumberColumnRemoveRuleTest.scala | 4 +- .../RemoveSingleAggregateRuleTest.scala | 4 +- ...ReplaceIntersectWithSemiJoinRuleTest.scala | 4 +- .../ReplaceMinusWithAntiJoinRuleTest.scala | 4 +- .../logical/RewriteIntersectAllRuleTest.scala | 4 +- .../logical/RewriteMinusAllRuleTest.scala | 4 +- .../RewriteMultiJoinConditionRuleTest.scala | 4 +- .../SimplifyFilterConditionRuleTest.scala | 4 +- .../SimplifyJoinConditionRuleTest.scala | 4 +- .../logical/SplitAggregateRuleTest.scala | 2 +- ...PythonConditionFromCorrelateRuleTest.scala | 4 +- ...SplitPythonConditionFromJoinRuleTest.scala | 4 +- ...oralJoinRewriteWithUniqueKeyRuleTest.scala | 64 +- .../logical/WindowGroupReorderRuleTest.scala | 4 +- .../FlinkRewriteSubQueryRuleTest.scala | 4 +- .../subquery/SubQueryAntiJoinTest.scala | 36 +- .../subquery/SubQuerySemiJoinTest.scala | 164 +- ...ueryCorrelateVariablesValidationTest.scala | 23 +- .../batch/EnforceLocalAggRuleTestBase.scala | 4 +- .../batch/EnforceLocalHashAggRuleTest.scala | 4 +- .../batch/EnforceLocalSortAggRuleTest.scala | 4 +- .../RemoveRedundantLocalHashAggRuleTest.scala | 4 +- .../RemoveRedundantLocalRankRuleTest.scala | 4 +- .../RemoveRedundantLocalSortAggRuleTest.scala | 4 +- .../stream/ChangelogModeInferenceTest.scala | 4 +- ...WindowTableFunctionTransposeRuleTest.scala | 4 +- ...rChangelogNormalizeTransposeRuleTest.scala | 4 +- .../planner/plan/stream/sql/CalcTest.scala | 10 +- .../plan/stream/sql/DagOptimizationTest.scala | 2 +- .../plan/stream/sql/DeduplicateTest.scala | 4 +- .../stream/sql/FilterableSourceTest.scala | 4 +- .../plan/stream/sql/LegacySinkTest.scala | 18 +- .../stream/sql/LegacyTableSourceTest.scala | 4 +- .../planner/plan/stream/sql/LimitTest.scala | 15 +- .../plan/stream/sql/LimitableSourceTest.scala | 4 +- .../plan/stream/sql/MatchRecognizeTest.scala | 52 +- .../sql/MiniBatchIntervalInferTest.scala | 4 +- .../stream/sql/ModifiedMonotonicityTest.scala | 6 +- .../stream/sql/NonDeterministicDagTest.scala | 1384 ++++++++++------- .../stream/sql/PartitionableSinkTest.scala | 17 +- .../stream/sql/PartitionableSourceTest.scala | 26 +- .../planner/plan/stream/sql/RankTest.scala | 51 +- .../sql/RelTimeIndicatorConverterTest.scala | 2 +- .../plan/stream/sql/SetOperatorsTest.scala | 27 +- .../plan/stream/sql/SortLimitTest.scala | 2 +- .../planner/plan/stream/sql/SortTest.scala | 2 +- .../plan/stream/sql/SourceWatermarkTest.scala | 4 +- .../plan/stream/sql/SubplanReuseTest.scala | 4 +- .../plan/stream/sql/TableScanTest.scala | 84 +- .../plan/stream/sql/TableSinkTest.scala | 72 +- .../plan/stream/sql/TableSourceTest.scala | 16 +- .../planner/plan/stream/sql/UnionTest.scala | 4 +- .../planner/plan/stream/sql/UnnestTest.scala | 2 +- .../planner/plan/stream/sql/ValuesTest.scala | 2 +- .../stream/sql/WindowDeduplicateTest.scala | 15 +- .../plan/stream/sql/WindowRankTest.scala | 61 +- .../stream/sql/WindowTableFunctionTest.scala | 20 +- .../plan/stream/sql/agg/AggregateTest.scala | 24 +- .../sql/agg/DistinctAggregateTest.scala | 46 +- .../plan/stream/sql/agg/GroupWindowTest.scala | 63 +- .../stream/sql/agg/GroupingSetsTest.scala | 13 +- .../sql/agg/IncrementalAggregateTest.scala | 12 +- .../stream/sql/agg/OverAggregateTest.scala | 31 +- .../sql/agg/TwoStageAggregateTest.scala | 4 +- .../stream/sql/agg/WindowAggregateTest.scala | 211 ++- .../stream/sql/join/IntervalJoinTest.scala | 31 +- .../stream/sql/join/JoinReorderTest.scala | 6 +- .../plan/stream/sql/join/JoinTest.scala | 2 +- .../plan/stream/sql/join/LookupJoinTest.scala | 155 +- .../stream/sql/join/SemiAntiJoinTest.scala | 2 +- .../sql/join/TemporalFunctionJoinTest.scala | 18 +- .../stream/sql/join/TemporalJoinTest.scala | 29 +- .../plan/stream/sql/join/WindowJoinTest.scala | 21 +- .../plan/stream/table/AggregateTest.scala | 2 +- .../planner/plan/stream/table/CalcTest.scala | 2 +- .../stream/table/ColumnFunctionsTest.scala | 2 +- .../plan/stream/table/CorrelateTest.scala | 2 +- .../table/GroupWindowTableAggregateTest.scala | 2 +- .../plan/stream/table/GroupWindowTest.scala | 2 +- .../planner/plan/stream/table/JoinTest.scala | 2 +- .../stream/table/LegacyTableSourceTest.scala | 2 +- .../plan/stream/table/OverAggregateTest.scala | 2 +- .../stream/table/PythonAggregateTest.scala | 32 +- .../plan/stream/table/PythonCalcTest.scala | 4 +- .../PythonGroupWindowAggregateTest.scala | 14 +- .../table/PythonOverWindowAggregateTest.scala | 2 +- .../table/PythonTableAggregateTest.scala | 2 +- .../plan/stream/table/SetOperatorsTest.scala | 2 +- .../stream/table/TableAggregateTest.scala | 2 +- .../plan/stream/table/TableSourceTest.scala | 20 +- .../table/TemporalTableFunctionJoinTest.scala | 27 +- .../stream/table/TwoStageAggregateTest.scala | 4 +- .../validation/AggregateValidationTest.scala | 114 +- .../table/validation/CalcValidationTest.scala | 162 +- .../validation/CorrelateValidationTest.scala | 99 +- ...upWindowTableAggregateValidationTest.scala | 44 +- .../GroupWindowValidationTest.scala | 301 ++-- .../LegacyTableSinkValidationTest.scala | 67 +- .../validation/OverWindowValidationTest.scala | 173 ++- .../SetOperatorsValidationTest.scala | 38 +- .../TableAggregateValidationTest.scala | 137 +- .../TemporalTableJoinValidationTest.scala | 46 +- 292 files changed, 4915 insertions(+), 4260 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/CalcMergeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/CalcMergeTest.java index 4a0e196f1cbbe..e9d25da76b351 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/CalcMergeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/CalcMergeTest.java @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.utils.TableTestUtil; /** Plan test for calc merge. */ -public class CalcMergeTest extends CalcMergeTestBase { +class CalcMergeTest extends CalcMergeTestBase { @Override protected boolean isBatchMode() { return true; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.java index 682c19fbf2b7b..7d8fde5746465 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.java @@ -26,23 +26,24 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; import java.util.Collections; /** Plan test for dynamic filtering. */ -@RunWith(Parameterized.class) -public class DynamicFilteringTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class DynamicFilteringTest extends TableTestBase { // Notes that the here name is used to load the correct plan. - @Parameterized.Parameters(name = "mode = {0}") - public static Collection data() { + @Parameters(name = "mode = {0}") + private static Collection data() { return Arrays.asList( new Object[][] { {BatchShuffleMode.ALL_EXCHANGES_BLOCKING}, @@ -52,14 +53,14 @@ public static Collection data() { private final BatchShuffleMode batchShuffleMode; - public DynamicFilteringTest(BatchShuffleMode batchShuffleMode) { + DynamicFilteringTest(BatchShuffleMode batchShuffleMode) { this.batchShuffleMode = batchShuffleMode; } private BatchTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.tableEnv() .getConfig() @@ -117,8 +118,8 @@ public void before() { + ")"); } - @Test - public void testLegacySource() { + @TestTemplate + void testLegacySource() { util.tableEnv() .executeSql( "CREATE TABLE legacy_source (\n" @@ -142,8 +143,8 @@ public void testLegacySource() { Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN))); } - @Test - public void testSimpleDynamicFiltering() { + @TestTemplate + void testSimpleDynamicFiltering() { // the execution plan contains 'Placeholder-Filter' operator util.verifyExplain( "SELECT * FROM fact1, dim WHERE p1 = p AND x > 10", @@ -151,8 +152,8 @@ public void testSimpleDynamicFiltering() { Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN))); } - @Test - public void testDynamicFilteringWithMultipleInput() { + @TestTemplate + void testDynamicFilteringWithMultipleInput() { // the execution plan does not contain 'Placeholder-Filter' operator util.verifyExplain( "SELECT * FROM fact1, dim, fact2 WHERE p1 = p and p1 = p2 AND x > 10", @@ -160,8 +161,8 @@ public void testDynamicFilteringWithMultipleInput() { Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN))); } - @Test - public void testDuplicateFactTables() { + @TestTemplate + void testDuplicateFactTables() { // the fact tables can not be reused util.verifyExplain( "SELECT * FROM (SELECT * FROM fact1, dim WHERE p1 = p AND x > 10) t1 JOIN fact1 t2 ON t1.y = t2.b1", @@ -169,8 +170,8 @@ public void testDuplicateFactTables() { Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN))); } - @Test - public void testReuseDimSide() { + @TestTemplate + void testReuseDimSide() { // dynamic filtering collector will be reused for both fact tables util.verifyExplain( "SELECT * FROM fact1, dim WHERE p1 = p AND x > 10 " @@ -180,8 +181,8 @@ public void testReuseDimSide() { Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN))); } - @Test - public void testDynamicFilteringWithStaticPartitionPruning() { + @TestTemplate + void testDynamicFilteringWithStaticPartitionPruning() { util.verifyExplain( "SELECT * FROM fact1, dim WHERE p1 = p AND x > 10 and p1 > 1", JavaScalaConversionUtil.toScala( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFunctionPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFunctionPlanTest.java index a765f94c881f9..2b2e64b417188 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFunctionPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/DynamicFunctionPlanTest.java @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.utils.TableTestUtil; /** Plan test for queries contain dynamic functions in batch. */ -public class DynamicFunctionPlanTest extends DynamicFunctionPlanTestBase { +class DynamicFunctionPlanTest extends DynamicFunctionPlanTestBase { @Override protected TableTestUtil getTableTestUtil() { return batchTestUtil(TableConfig.getDefault()); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java index 33fe021ac3e2f..eaf2202a11fb1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java @@ -23,16 +23,16 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Tests for ForwardHashExchangeProcessor. */ -public class ForwardHashExchangeTest extends TableTestBase { +class ForwardHashExchangeTest extends TableTestBase { private BatchTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.tableEnv() @@ -77,13 +77,13 @@ public void before() { } @Test - public void testRankWithHashShuffle() { + void testRankWithHashShuffle() { util.verifyExecPlan( "SELECT * FROM (SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM T) WHERE rk <= 10"); } @Test - public void testSortAggregateWithHashShuffle() { + void testSortAggregateWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -91,7 +91,7 @@ public void testSortAggregateWithHashShuffle() { } @Test - public void testOverAggOnHashAggWithHashShuffle() { + void testOverAggOnHashAggWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); @@ -106,7 +106,7 @@ public void testOverAggOnHashAggWithHashShuffle() { } @Test - public void testOverAggOnHashAggWithGlobalShuffle() { + void testOverAggOnHashAggWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); @@ -114,7 +114,7 @@ public void testOverAggOnHashAggWithGlobalShuffle() { } @Test - public void testOverAggOnSortAggWithHashShuffle() { + void testOverAggOnSortAggWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -129,7 +129,7 @@ public void testOverAggOnSortAggWithHashShuffle() { } @Test - public void testOverAggOnSortAggWithGlobalShuffle() { + void testOverAggOnSortAggWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -137,7 +137,7 @@ public void testOverAggOnSortAggWithGlobalShuffle() { } @Test - public void testHashAggOnHashJoinWithHashShuffle() { + void testHashAggOnHashJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( @@ -149,7 +149,7 @@ public void testHashAggOnHashJoinWithHashShuffle() { } @Test - public void testOnePhaseSortAggOnSortMergeJoinWithHashShuffle() { + void testOnePhaseSortAggOnSortMergeJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( @@ -164,7 +164,7 @@ public void testOnePhaseSortAggOnSortMergeJoinWithHashShuffle() { } @Test - public void testTwoPhaseSortAggOnSortMergeJoinWithHashShuffle() { + void testTwoPhaseSortAggOnSortMergeJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( @@ -179,7 +179,7 @@ public void testTwoPhaseSortAggOnSortMergeJoinWithHashShuffle() { } @Test - public void testAutoPhaseSortAggOnSortMergeJoinWithHashShuffle() { + void testAutoPhaseSortAggOnSortMergeJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( @@ -194,7 +194,7 @@ public void testAutoPhaseSortAggOnSortMergeJoinWithHashShuffle() { } @Test - public void testHashAggOnNestedLoopJoinWithGlobalShuffle() { + void testHashAggOnNestedLoopJoinWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); @@ -208,7 +208,7 @@ public void testHashAggOnNestedLoopJoinWithGlobalShuffle() { } @Test - public void testSortAggOnNestedLoopJoinWithGlobalShuffle() { + void testSortAggOnNestedLoopJoinWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -222,7 +222,7 @@ public void testSortAggOnNestedLoopJoinWithGlobalShuffle() { } @Test - public void testRankOnHashAggWithHashShuffle() { + void testRankOnHashAggWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); @@ -235,7 +235,7 @@ public void testRankOnHashAggWithHashShuffle() { } @Test - public void testRankOnHashAggWithGlobalShuffle() { + void testRankOnHashAggWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); @@ -248,7 +248,7 @@ public void testRankOnHashAggWithGlobalShuffle() { } @Test - public void testRankOnOnePhaseSortAggWithHashShuffle() { + void testRankOnOnePhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -264,7 +264,7 @@ public void testRankOnOnePhaseSortAggWithHashShuffle() { } @Test - public void testRankOnTwoPhaseSortAggWithHashShuffle() { + void testRankOnTwoPhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -280,7 +280,7 @@ public void testRankOnTwoPhaseSortAggWithHashShuffle() { } @Test - public void testRankOnOnePhaseSortAggWithGlobalShuffle() { + void testRankOnOnePhaseSortAggWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -296,7 +296,7 @@ public void testRankOnOnePhaseSortAggWithGlobalShuffle() { } @Test - public void testRankOnTwoPhaseSortAggWithGlobalShuffle() { + void testRankOnTwoPhaseSortAggWithGlobalShuffle() { util.tableEnv() .getConfig() .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); @@ -312,7 +312,7 @@ public void testRankOnTwoPhaseSortAggWithGlobalShuffle() { } @Test - public void testHashJoinWithMultipleInputDisabled() { + void testHashJoinWithMultipleInputDisabled() { util.tableEnv() .getConfig() .set( @@ -328,7 +328,7 @@ public void testHashJoinWithMultipleInputDisabled() { } @Test - public void testSortJoinWithMultipleInputDisabled() { + void testSortJoinWithMultipleInputDisabled() { util.tableEnv() .getConfig() .set( @@ -344,7 +344,7 @@ public void testSortJoinWithMultipleInputDisabled() { } @Test - public void testMultipleInputs() { + void testMultipleInputs() { util.getTableEnv() .getConfig() .set(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, false) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/MatchRecognizeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/MatchRecognizeTest.java index eca0bb0cfcfb2..b5e42f7819e61 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/MatchRecognizeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/MatchRecognizeTest.java @@ -22,17 +22,17 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.calcite.sql.SqlMatchRecognize; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Tests for {@link SqlMatchRecognize}. */ -public class MatchRecognizeTest extends TableTestBase { +class MatchRecognizeTest extends TableTestBase { private BatchTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.getTableEnv() .executeSql( @@ -47,13 +47,13 @@ public void before() { + ")"); } - @After - public void after() { + @AfterEach + void after() { util.getTableEnv().executeSql("DROP TABLE Ticker"); } @Test - public void testCascadeMatch() { + void testCascadeMatch() { String sqlQuery = "SELECT *\n" + "FROM (\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.java index 7bc295ce6fd03..874de69f3a8c8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.java @@ -25,11 +25,12 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -38,8 +39,8 @@ import scala.collection.Seq; /** Test for row-level delete. */ -@RunWith(Parameterized.class) -public class RowLevelDeleteTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class RowLevelDeleteTest extends TableTestBase { private final SupportsRowLevelDelete.RowLevelDeleteMode deleteMode; private final Seq explainDetails = @@ -48,19 +49,19 @@ public class RowLevelDeleteTest extends TableTestBase { private BatchTableTestUtil util; - @Parameterized.Parameters(name = "deleteMode = {0}") - public static Collection data() { + @Parameters(name = "deleteMode = {0}") + private static Collection data() { return Arrays.asList( SupportsRowLevelDelete.RowLevelDeleteMode.DELETED_ROWS, SupportsRowLevelDelete.RowLevelDeleteMode.REMAINING_ROWS); } - public RowLevelDeleteTest(SupportsRowLevelDelete.RowLevelDeleteMode deleteMode) { + RowLevelDeleteTest(SupportsRowLevelDelete.RowLevelDeleteMode deleteMode) { this.deleteMode = deleteMode; } - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.tableEnv() .getConfig() @@ -68,27 +69,27 @@ public void before() { .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 12); } - @Test - public void testDeleteWithoutFilter() { + @TestTemplate + void testDeleteWithoutFilter() { createTableForDelete(); util.verifyExplainInsert("DELETE FROM t", explainDetails); } - @Test - public void testDeleteWithFilter() { + @TestTemplate + void testDeleteWithFilter() { createTableForDelete(); util.verifyExplainInsert("DELETE FROM t where a = 1 and b = '123'", explainDetails); } - @Test - public void testDeleteWithSubQuery() { + @TestTemplate + void testDeleteWithSubQuery() { createTableForDelete(); util.verifyExplainInsert( "DELETE FROM t where b = '123' and a = (select count(*) from t)", explainDetails); } - @Test - public void testDeleteWithCustomColumns() { + @TestTemplate + void testDeleteWithCustomColumns() { util.tableEnv() .executeSql( String.format( @@ -102,8 +103,8 @@ public void testDeleteWithCustomColumns() { util.verifyExplainInsert("DELETE FROM t where b = '123'", explainDetails); } - @Test - public void testDeleteWithMetaColumns() { + @TestTemplate + void testDeleteWithMetaColumns() { util.tableEnv() .executeSql( String.format( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.java index 22a09cd3159ce..57f22238d0540 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.java @@ -26,11 +26,12 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -38,9 +39,11 @@ import scala.collection.Seq; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + /** Test for row-level update. */ -@RunWith(Parameterized.class) -public class RowLevelUpdateTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class RowLevelUpdateTest extends TableTestBase { private final Seq explainDetails = JavaScalaConversionUtil.toScala( @@ -49,19 +52,19 @@ public class RowLevelUpdateTest extends TableTestBase { private BatchTableTestUtil util; - @Parameterized.Parameters(name = "updateMode = {0}") - public static Collection data() { + @Parameters(name = "updateMode = {0}") + private static Collection data() { return Arrays.asList( SupportsRowLevelUpdate.RowLevelUpdateMode.UPDATED_ROWS, SupportsRowLevelUpdate.RowLevelUpdateMode.ALL_ROWS); } - public RowLevelUpdateTest(SupportsRowLevelUpdate.RowLevelUpdateMode updateMode) { + RowLevelUpdateTest(SupportsRowLevelUpdate.RowLevelUpdateMode updateMode) { this.updateMode = updateMode; } - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.tableEnv() .getConfig() @@ -69,21 +72,21 @@ public void before() { .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 12); } - @Test - public void testUpdateWithoutFilter() { + @TestTemplate + void testUpdateWithoutFilter() { createTableForUpdate(); util.verifyExplainInsert("UPDATE t SET b = 'n1', a = char_length(b) * a ", explainDetails); } - @Test - public void testUpdateWithFilter() { + @TestTemplate + void testUpdateWithFilter() { createTableForUpdate(); util.verifyExplainInsert( "UPDATE t SET b = 'v2' WHERE a = 123 AND b = 'v1'", explainDetails); } - @Test - public void testUpdateWithSubQuery() { + @TestTemplate + void testUpdateWithSubQuery() { createTableForUpdate(); util.tableEnv() .executeSql( @@ -95,8 +98,8 @@ public void testUpdateWithSubQuery() { "UPDATE t SET b = 'v2' WHERE a = (SELECT count(*) FROM t1)", explainDetails); } - @Test - public void testUpdateAllColsWithOnlyRequireUpdatedCols() { + @TestTemplate + void testUpdateAllColsWithOnlyRequireUpdatedCols() { util.tableEnv() .executeSql( String.format( @@ -110,8 +113,8 @@ public void testUpdateAllColsWithOnlyRequireUpdatedCols() { "UPDATE t SET b = 'v2', a = 123, c = c + 1 WHERE c > 123", explainDetails); } - @Test - public void testUpdatePartColsWithOnlyRequireUpdatedCols() { + @TestTemplate + void testUpdatePartColsWithOnlyRequireUpdatedCols() { util.tableEnv() .executeSql( String.format( @@ -125,8 +128,8 @@ public void testUpdatePartColsWithOnlyRequireUpdatedCols() { "UPDATE t SET b = 'v2', a = 123, c = c + 1 WHERE c > 123", explainDetails); } - @Test - public void testUpdateWithCustomColumns() { + @TestTemplate + void testUpdateWithCustomColumns() { util.tableEnv() .executeSql( String.format( @@ -140,8 +143,8 @@ public void testUpdateWithCustomColumns() { util.verifyExplainInsert("UPDATE t SET b = 'v2' WHERE b = '123'", explainDetails); } - @Test - public void testUpdateWithMetaColumns() { + @TestTemplate + void testUpdateWithMetaColumns() { util.tableEnv() .executeSql( String.format( @@ -155,8 +158,8 @@ public void testUpdateWithMetaColumns() { util.verifyExplainInsert("UPDATE t SET b = 'v2' WHERE b = '123'", explainDetails); } - @Test(expected = SqlParserException.class) - public void testUpdateWithCompositeType() { + @TestTemplate + void testUpdateWithCompositeType() { util.tableEnv() .executeSql( String.format( @@ -169,8 +172,13 @@ public void testUpdateWithCompositeType() { + "'update-mode' = '%s'" + ") ", updateMode)); - util.verifyExplainInsert( - "UPDATE t SET b.b1 = 'v2', c.c1 = 1000 WHERE b = '123'", explainDetails); + + assertThatExceptionOfType(SqlParserException.class) + .isThrownBy( + () -> + util.verifyExplainInsert( + "UPDATE t SET b.b1 = 'v2', c.c1 = 1000 WHERE b = '123'", + explainDetails)); } private void createTableForUpdate() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/TimeTravelTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/TimeTravelTest.java index 92b0a16bbe6ef..89acd62c5d672 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/TimeTravelTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/TimeTravelTest.java @@ -28,20 +28,20 @@ import org.apache.flink.table.planner.utils.DateTimeTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.time.ZoneId; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Plan tests for time travel. */ -public class TimeTravelTest extends TableTestBase { +class TimeTravelTest extends TableTestBase { private BatchTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); String catalogName = "TimeTravelCatalog"; TestTimeTravelCatalog catalog = @@ -53,25 +53,25 @@ public void before() { } @Test - public void testTimeTravel() { + void testTimeTravel() { util.verifyExecPlan( "SELECT * FROM t1 FOR SYSTEM_TIME AS OF TIMESTAMP '2023-01-01 02:00:00'"); } @Test - public void testTimeTravelWithAsExpression() { + void testTimeTravelWithAsExpression() { util.verifyExecPlan( "SELECT * FROM t1 FOR SYSTEM_TIME AS OF TIMESTAMP '2023-01-01 02:00:00' AS t2"); } @Test - public void testTimeTravelWithSimpleExpression() { + void testTimeTravelWithSimpleExpression() { util.verifyExecPlan( "SELECT * FROM t1 FOR SYSTEM_TIME AS OF TIMESTAMP '2023-01-01 00:00:00'+INTERVAL '60' DAY"); } @Test - public void testTimeTravelWithDifferentTimezone() { + void testTimeTravelWithDifferentTimezone() { util.tableEnv().getConfig().setLocalTimeZone(ZoneId.of("Asia/Shanghai")); util.verifyExecPlan( @@ -85,7 +85,7 @@ public void testTimeTravelWithDifferentTimezone() { } @Test - public void testTimeTravelOneTableMultiTimes() { + void testTimeTravelOneTableMultiTimes() { util.verifyExecPlan( "SELECT\n" + " f1\n" @@ -99,7 +99,7 @@ public void testTimeTravelOneTableMultiTimes() { } @Test - public void testTimeTravelWithLookupJoin() { + void testTimeTravelWithLookupJoin() { util.verifyExecPlan( "SELECT\n" + " l.f2,\n" @@ -116,13 +116,13 @@ public void testTimeTravelWithLookupJoin() { } @Test - public void testTimeTravelWithHints() { + void testTimeTravelWithHints() { util.verifyExecPlan( "SELECT * FROM t1 /*+ options('bounded'='true') */ FOR SYSTEM_TIME AS OF TIMESTAMP '2023-01-01 02:00:00' AS t2"); } @Test - public void testTimeTravelWithUnsupportedExpression() { + void testTimeTravelWithUnsupportedExpression() { assertThatThrownBy( () -> util.tableEnv() @@ -149,7 +149,7 @@ public void testTimeTravelWithUnsupportedExpression() { } @Test - public void testTimeTravelWithIdentifierSnapshot() { + void testTimeTravelWithIdentifierSnapshot() { util.tableEnv() .executeSql( "CREATE TABLE\n" @@ -173,7 +173,7 @@ public void testTimeTravelWithIdentifierSnapshot() { } @Test - public void testTimeTravelWithView() { + void testTimeTravelWithView() { util.tableEnv().executeSql("CREATE VIEW tb_view AS SELECT * FROM t1"); assertThatThrownBy( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/CalcMergeTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/CalcMergeTestBase.java index 1f8cf1caad262..728ef1edfdb91 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/CalcMergeTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/CalcMergeTestBase.java @@ -22,8 +22,8 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Base plan test for calc merge, the difference between FlinkCalcMergeRuleTest is this test @@ -37,8 +37,8 @@ public abstract class CalcMergeTestBase extends TableTestBase { protected abstract TableTestUtil getTableTestUtil(); - @Before - public void setup() { + @BeforeEach + void setup() { util = getTableTestUtil(); util.tableEnv() @@ -57,59 +57,59 @@ public void setup() { } @Test - public void testCalcMergeWithSameDigest() { + void testCalcMergeWithSameDigest() { util.verifyExecPlan("SELECT a, b FROM (SELECT * FROM MyTable WHERE a = b) t WHERE b = a"); } @Test - public void testCalcMergeWithNonDeterministicExpr1() { + void testCalcMergeWithNonDeterministicExpr1() { util.verifyExecPlan( "SELECT a, a1 FROM (SELECT a, random_udf(a) AS a1 FROM MyTable) t WHERE a1 > 10"); } @Test - public void testCalcMergeWithNonDeterministicExpr2() { + void testCalcMergeWithNonDeterministicExpr2() { util.verifyExecPlan( "SELECT random_udf(a1) as a2 FROM (SELECT random_udf(a) as" + " a1, b FROM MyTable) t WHERE b > 10"); } @Test - public void testCalcMergeWithTopMultiNonDeterministicExpr() { + void testCalcMergeWithTopMultiNonDeterministicExpr() { util.verifyExecPlan( "SELECT random_udf(a1) as a2, random_udf(a1) as a3 FROM" + " (SELECT random_udf(a) as a1, b FROM MyTable) t WHERE b > 10"); } @Test - public void testCalcMergeTopFilterHasNonDeterministicExpr() { + void testCalcMergeTopFilterHasNonDeterministicExpr() { util.verifyExecPlan( "SELECT a, c FROM" + " (SELECT a, random_udf(b) as b1, c FROM MyTable) t WHERE b1 > 10"); } @Test - public void testCalcMergeWithBottomMultiNonDeterministicExpr() { + void testCalcMergeWithBottomMultiNonDeterministicExpr() { util.verifyExecPlan( "SELECT a1, b2 FROM" + " (SELECT random_udf(a) as a1, random_udf(b) as b2, c FROM MyTable) t WHERE c > 10"); } @Test - public void testCalcMergeWithBottomMultiNonDeterministicInConditionExpr() { + void testCalcMergeWithBottomMultiNonDeterministicInConditionExpr() { util.verifyExecPlan( "SELECT c FROM" + " (SELECT random_udf(a) as a1, random_udf(b) as b2, c FROM MyTable) t WHERE a1 > b2"); } @Test - public void testCalcMergeWithoutInnerNonDeterministicExpr() { + void testCalcMergeWithoutInnerNonDeterministicExpr() { util.verifyExecPlan( "SELECT a, c FROM (SELECT a, random_udf(a) as a1, c FROM MyTable) t WHERE c > 10"); } @Test - public void testCalcMergeWithNonDeterministicNestedExpr() { + void testCalcMergeWithNonDeterministicNestedExpr() { util.verifyExecPlan( "SELECT a, a1 FROM (SELECT a, substr(cast(random_udf(a) as varchar), 1, 2) AS a1 FROM MyTable) t WHERE a1 > '10'"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/DynamicFunctionPlanTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/DynamicFunctionPlanTestBase.java index 22140d384a67e..78b414954c239 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/DynamicFunctionPlanTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/common/DynamicFunctionPlanTestBase.java @@ -21,8 +21,8 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Plan test for queries contain dynamic functions. */ public abstract class DynamicFunctionPlanTestBase extends TableTestBase { @@ -33,8 +33,8 @@ public abstract class DynamicFunctionPlanTestBase extends TableTestBase { protected abstract TableTestUtil getTableTestUtil(); - @Before - public void setup() { + @BeforeEach + void setup() { util = getTableTestUtil(); util.tableEnv() @@ -56,7 +56,7 @@ public void setup() { } @Test - public void testAggregateReduceConstants() { + void testAggregateReduceConstants() { util.verifyExecPlan( "SELECT\n" + " cat, gmt_date, SUM(cnt), count(*)\n" @@ -66,7 +66,7 @@ public void testAggregateReduceConstants() { } @Test - public void testAggregateReduceConstants2() { + void testAggregateReduceConstants2() { // current RelMdPredicates only look at columns that are projected without any function // applied, so 'SUBSTR(CAST(LOCALTIME AS VARCHAR), 1, 2)' will never be inferred as constant util.verifyExecPlan( @@ -78,7 +78,7 @@ public void testAggregateReduceConstants2() { } @Test - public void testAggregateReduceConstants3() { + void testAggregateReduceConstants3() { util.verifyExecPlan( "SELECT\n" + " gmt_date, ts, cat, SUBSTR(CAST(ts AS VARCHAR), 1, 2), SUM(cnt)\n" @@ -89,7 +89,7 @@ public void testAggregateReduceConstants3() { } @Test - public void testCalcMerge() { + void testCalcMerge() { util.verifyExecPlan( "SELECT * FROM ( \n" + " SELECT *, SUBSTR(CAST(LOCALTIME AS VARCHAR), 1, 2) hh\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.java index 1b93904339a42..23fd75908aea1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.java @@ -21,7 +21,7 @@ import org.apache.flink.table.planner.hint.JoinStrategy; /** Test for Broadcast join hint. */ -public class BroadcastJoinHintTest extends JoinHintTestBase { +class BroadcastJoinHintTest extends JoinHintTestBase { @Override protected String getTestSingleJoinHint() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java index 829581d482322..eada3bb388173 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java @@ -36,14 +36,15 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.sql.SqlExplainLevel; import org.apache.logging.log4j.util.Strings; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.stream.Collectors; import scala.Enumeration; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static scala.runtime.BoxedUnit.UNIT; /** @@ -62,8 +63,8 @@ public abstract class JoinHintTestBase extends TableTestBase { .map(JoinStrategy::getJoinHintName) .collect(Collectors.toList()); - @Before - public void before() { + @BeforeEach + void before() { util = batchTestUtil(TableConfig.getDefault()); util.tableEnv() .executeSql( @@ -130,21 +131,21 @@ protected List getOtherJoinHints() { } @Test - public void testSimpleJoinHintWithLeftSideAsBuildSide() { + void testSimpleJoinHintWithLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testSimpleJoinHintWithRightSideAsBuildSide() { + void testSimpleJoinHintWithRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide1() { + void testJoinHintWithMultiJoinAndFirstSideAsBuildSide1() { // the T1 will be the build side in first join String sql = "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3"; @@ -153,7 +154,7 @@ public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide1() { } @Test - public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide2() { + void testJoinHintWithMultiJoinAndFirstSideAsBuildSide2() { String sql = "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; @@ -161,7 +162,7 @@ public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide2() { } @Test - public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1() { + void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1() { String sql = "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3"; @@ -169,7 +170,7 @@ public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1() { } @Test - public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2() { + void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2() { String sql = "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; @@ -177,7 +178,7 @@ public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2() { } @Test - public void testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides() { + void testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides() { String sql = "select /*+ %s(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; @@ -185,95 +186,90 @@ public void testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides() { } @Test - public void testJoinHintWithUnknownTable() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - String.format( - "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", - "T99", getTestSingleJoinHint())); + void testJoinHintWithUnknownTable() { String sql = "select /*+ %s(T99) */* from T1 join T2 on T1.a1 = T2.a2"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + assertThatThrownBy(() -> verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", + "T99", getTestSingleJoinHint()); } @Test - public void testJoinHintWithUnknownTableNameMixedWithValidTableNames1() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - String.format( - "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", - "T99", getTestSingleJoinHint())); + void testJoinHintWithUnknownTableNameMixedWithValidTableNames1() { String sql = "select /*+ %s(T1, T99) */* from T1 join T2 on T1.a1 = T2.a2"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + assertThatThrownBy(() -> verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", + "T99", getTestSingleJoinHint()); } @Test - public void testJoinHintWithUnknownTableNameMixedWithValidTableNames2() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - String.format( - "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", - "T99", getTestSingleJoinHint())); + void testJoinHintWithUnknownTableNameMixedWithValidTableNames2() { String sql = "select /*+ %s(T1, T99, T2) */* from T1 join T2 on T1.a1 = T2.a2"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + assertThatThrownBy(() -> verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", + "T99", getTestSingleJoinHint()); } @Test - public void testJoinHintWithMultiUnknownTableNamesMixedWithValidTableNames() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - String.format( - "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", - "T98, T99", getTestSingleJoinHint())); + void testJoinHintWithMultiUnknownTableNamesMixedWithValidTableNames() { String sql = "select /*+ %s(T1, T99, T98) */* from T1 join T2 on T1.a1 = T2.a2"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + assertThatThrownBy(() -> verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", + "T98, T99", getTestSingleJoinHint()); } @Test - public void testJoinHintWithView() { + void testJoinHintWithView() { String sql = "select /*+ %s(V4) */* from T1 join V4 on T1.a1 = V4.a4"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithUnknownView() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - String.format( - "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", - "V99", getTestSingleJoinHint())); + void testJoinHintWithUnknownView() { String sql = "select /*+ %s(V99) */* from T1 join V4 on T1.a1 = V4.a4"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + assertThatThrownBy(() -> verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The options of following hints cannot match the name of input tables or views: \n`%s` in `%s`", + "V99", getTestSingleJoinHint()); } @Test - public void testJoinHintWithEquiPred() { + void testJoinHintWithEquiPred() { String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithEquiPredAndFilter() { + void testJoinHintWithEquiPredAndFilter() { String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithEquiAndLocalPred() { + void testJoinHintWithEquiAndLocalPred() { String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithEquiAndNonEquiPred() { + void testJoinHintWithEquiAndNonEquiPred() { String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2"; @@ -281,56 +277,56 @@ public void testJoinHintWithEquiAndNonEquiPred() { } @Test - public void testJoinHintWithoutJoinPred() { + void testJoinHintWithoutJoinPred() { String sql = "select /*+ %s(T1) */* from T1, T2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithNonEquiPred() { + void testJoinHintWithNonEquiPred() { String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 > T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithLeftJoinAndLeftSideAsBuildSide() { + void testJoinHintWithLeftJoinAndLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 left join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithLeftJoinAndRightSideAsBuildSide() { + void testJoinHintWithLeftJoinAndRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 left join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithRightJoinAndLeftSideAsBuildSide() { + void testJoinHintWithRightJoinAndLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 right join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithRightJoinAndRightSideAsBuildSide() { + void testJoinHintWithRightJoinAndRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 right join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithFullJoinAndLeftSideAsBuildSide() { + void testJoinHintWithFullJoinAndLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 full join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithFullJoinAndRightSideAsBuildSide() { + void testJoinHintWithFullJoinAndRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 full join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); @@ -339,7 +335,7 @@ public void testJoinHintWithFullJoinAndRightSideAsBuildSide() { // TODO currently join hint is not supported on SEMI join, it will use default join strategy by // planner @Test - public void testJoinHintWithSemiJoinAndLeftSideAsBuildSide() { + void testJoinHintWithSemiJoinAndLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 where a1 in (select a2 from T2)"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); @@ -348,7 +344,7 @@ public void testJoinHintWithSemiJoinAndLeftSideAsBuildSide() { // TODO currently join hint is not supported on SEMI join, it will use default join strategy by // planner @Test - public void testJoinHintWithSemiJoinAndRightSideAsBuildSide() { + void testJoinHintWithSemiJoinAndRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 where a1 in (select a2 from T2)"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); @@ -357,7 +353,7 @@ public void testJoinHintWithSemiJoinAndRightSideAsBuildSide() { // TODO currently join hint is not supported on ANTI join, it will use default join strategy by // planner @Test - public void testJoinHintWithAntiJoinAndLeftSideAsBuildSide() { + void testJoinHintWithAntiJoinAndLeftSideAsBuildSide() { String sql = "select /*+ %s(T1) */* from T1 where a1 not in (select a2 from T2)"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); @@ -366,14 +362,14 @@ public void testJoinHintWithAntiJoinAndLeftSideAsBuildSide() { // TODO currently join hint is not supported on ANTI join, it will use default join strategy by // planner @Test - public void testJoinHintWithAntiJoinAndRightSideAsBuildSide() { + void testJoinHintWithAntiJoinAndRightSideAsBuildSide() { String sql = "select /*+ %s(T2) */* from T1 where a1 not in (select a2 from T2)"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithMultiArgsAndLeftSideFirst() { + void testJoinHintWithMultiArgsAndLeftSideFirst() { // the first arg will be chosen as the build side String sql = "select /*+ %s(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2"; @@ -381,7 +377,7 @@ public void testJoinHintWithMultiArgsAndLeftSideFirst() { } @Test - public void testJoinHintWithMultiArgsAndRightSideFirst() { + void testJoinHintWithMultiArgsAndRightSideFirst() { // the first arg will be chosen as the build side String sql = "select /*+ %s(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2"; @@ -389,7 +385,7 @@ public void testJoinHintWithMultiArgsAndRightSideFirst() { } @Test - public void testMultiJoinHints() { + void testMultiJoinHints() { // the first join hint will be chosen String sql = "select /*+ %s(T1), %s */* from T1 join T2 on T1.a1 = T2.a2"; @@ -404,7 +400,7 @@ public void testMultiJoinHints() { } @Test - public void testMultiJoinHintsWithTheFirstOneIsInvalid() { + void testMultiJoinHintsWithTheFirstOneIsInvalid() { // the first join hint is invalid because it is not equi join except NEST_LOOP String sql = "select /*+ %s(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2"; @@ -412,7 +408,7 @@ public void testMultiJoinHintsWithTheFirstOneIsInvalid() { } @Test - public void testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin() { + void testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin() { // the join in V2 will use the planner's default join strategy, // and the join between T1 and V5 will use the tested join hint String sql = "select /*+ %s(T1)*/T1.* from T1 join V5 on T1.a1 = V5.a1"; @@ -421,7 +417,7 @@ public void testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin } @Test - public void testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin() { + void testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin() { // the join in V2 will use the planner's default join strategy, // and the join between T1 and V5 will use the tested join hint String sql = "select /*+ %s(T1)*/* from V5"; @@ -430,7 +426,7 @@ public void testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin() { } @Test - public void testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter() { + void testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter() { // the join in V2 will use the planner's default join strategy, // and the join between T1 and V2 will use the tested join hint util.tableEnv() @@ -443,7 +439,7 @@ public void testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter() { } @Test - public void testJoinHintWithSimpleSumInSelectList() { + void testJoinHintWithSimpleSumInSelectList() { String sql = "select /*+ %s(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on T1.b1 = T2.b2 group by T1.b1"; @@ -451,7 +447,7 @@ public void testJoinHintWithSimpleSumInSelectList() { } @Test - public void testJoinHintWithCastInSelectList() { + void testJoinHintWithCastInSelectList() { String sql = "select /*+ %s(T1)*/T1.b1, cast(T1.a1 as int) from T1 join T2 on T1.b1 = T2.b2"; @@ -459,7 +455,7 @@ public void testJoinHintWithCastInSelectList() { } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin() { // the join in sub-query will use the planner's default join strategy, // and the join outside will use the tested join hint String sql = @@ -469,14 +465,14 @@ public void testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuter } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin() { String sql = "select /*+ %s(T1)*/* from (select T1.* from T1 join T2 on T1.a1 = T2.a2)"; verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter() { String sql = "select /*+ %s(T1)*/* from (select T1.* from T1 join T2 on T1.a1 = T2.a2 where T1.b1 = 'abc')"; @@ -484,7 +480,7 @@ public void testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilte } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock() { String sql = "select /*+ %s(T1)*/T4.a1, (select count(*) from T1 join T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 where T1.b1 = 'abc') T4"; @@ -492,7 +488,7 @@ public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlo } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList() { String sql = "select /*+ %s(T1)*/T4.a1, (select count(*) from T1 join ((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) T4"; @@ -500,7 +496,7 @@ public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinI } @Test - public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom() { + void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom() { String sql = "select /*+ %s(T1)*/T4.a1 from (select T1.* from T1 join ((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) T4"; @@ -508,7 +504,7 @@ public void testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinI } @Test - public void testJoinHintWithTableAlias() { + void testJoinHintWithTableAlias() { // the join in sub-query will use the planner's default join strategy, // and the join between T1 and alias V2 will use the tested join hint String sql = @@ -518,7 +514,7 @@ public void testJoinHintWithTableAlias() { } @Test - public void testJoinHintsWithMultiSameJoinHintsAndSingleArg() { + void testJoinHintsWithMultiSameJoinHintsAndSingleArg() { // the first join hint will be chosen and T1 will be chosen as the build side String sql = "select /*+ %s(T1), %s(T2) */* from T1 join T2 on T1.a1 = T2.a2"; @@ -526,7 +522,7 @@ public void testJoinHintsWithMultiSameJoinHintsAndSingleArg() { } @Test - public void testJoinHintsWithDuplicatedArgs() { + void testJoinHintsWithDuplicatedArgs() { // T1 will be chosen as the build side String sql = "select /*+ %s(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2"; @@ -534,7 +530,7 @@ public void testJoinHintsWithDuplicatedArgs() { } @Test - public void testJoinHintsWithMultiSameJoinHintsAndMultiArgs() { + void testJoinHintsWithMultiSameJoinHintsAndMultiArgs() { // the first join hint will be chosen and T1 will be chosen as the build side String sql = "select /*+ %s(T1, T2), %s(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2"; @@ -542,16 +538,22 @@ public void testJoinHintsWithMultiSameJoinHintsAndMultiArgs() { } @Test - public void testJoinHintsWithMultiHintsThrowException() { - thrown().expect(SqlParserException.class); - thrown().expectMessage("SQL parse failed."); + void testJoinHintsWithMultiHintsThrowException() { String sql = "select /*+ %s(T1) */ /*+ %s(T2) */ * from T1 join T2 on T1.a1 = T2.a2"; - verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + assertThatThrownBy( + () -> + verifyRelPlanByCustom( + String.format( + sql, + getTestSingleJoinHint(), + getTestSingleJoinHint()))) + .isInstanceOf(SqlParserException.class) + .hasMessageContaining("SQL parse failed."); } @Test - public void testJoinHintWithDisabledOperator() { + void testJoinHintWithDisabledOperator() { util.tableEnv() .getConfig() .set( @@ -564,7 +566,7 @@ public void testJoinHintWithDisabledOperator() { } @Test - public void testJoinHintsWithUnion() { + void testJoinHintsWithUnion() { // there are two query blocks and join hints are independent String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ %s(T3) */* from T3 join T1 on T3.a3 = T1.a1"; @@ -573,7 +575,7 @@ public void testJoinHintsWithUnion() { } @Test - public void testJoinHintsWithFilter() { + void testJoinHintsWithFilter() { // there are two query blocks and join hints are independent String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5"; @@ -581,7 +583,7 @@ public void testJoinHintsWithFilter() { } @Test - public void testJoinHintsWithCalc() { + void testJoinHintsWithCalc() { // there are two query blocks and join hints are independent String sql = "select /*+ %s(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2"; @@ -589,7 +591,7 @@ public void testJoinHintsWithCalc() { } @Test - public void testJoinHintInView() { + void testJoinHintInView() { // the build side in view is left util.tableEnv() .executeSql( @@ -604,7 +606,7 @@ public void testJoinHintInView() { } @Test - public void testJoinHintInMultiLevelView() { + void testJoinHintInMultiLevelView() { // the inside view keeps multi alias // the build side in this view is left util.tableEnv() @@ -627,7 +629,7 @@ public void testJoinHintInMultiLevelView() { } @Test - public void testJoinHintsOnSameViewWithoutReusingView() { + void testJoinHintsOnSameViewWithoutReusingView() { // the build side in this view is left util.tableEnv() .executeSql( @@ -671,7 +673,7 @@ public void testJoinHintsOnSameViewWithoutReusingView() { } @Test - public void testJoinHintsOnSameViewWithReusingView() { + void testJoinHintsOnSameViewWithReusingView() { util.tableEnv() .getConfig() .set( @@ -722,7 +724,7 @@ public void testJoinHintsOnSameViewWithReusingView() { } @Test - public void testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints() { + void testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints() { util.tableEnv() .getConfig() .set( @@ -781,7 +783,7 @@ public void testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints() } @Test - public void testJoinHintWithSubStringViewName1() { + void testJoinHintWithSubStringViewName1() { util.tableEnv() .executeSql( String.format( @@ -802,7 +804,7 @@ public void testJoinHintWithSubStringViewName1() { } @Test - public void testJoinHintWithSubStringViewName2() { + void testJoinHintWithSubStringViewName2() { util.tableEnv() .executeSql( String.format( @@ -823,14 +825,14 @@ public void testJoinHintWithSubStringViewName2() { } @Test - public void testJoinHintWithoutCaseSensitive() { + void testJoinHintWithoutCaseSensitive() { String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2"; verifyRelPlanByCustom(String.format(sql, buildCaseSensitiveStr(getTestSingleJoinHint()))); } @Test - public void testJoinHintWithJoinHintInSubQuery() { + void testJoinHintWithJoinHintInSubQuery() { String sql = "select * from T1 WHERE a1 IN (select /*+ %s(T2) */ a2 from T2 join T3 on T2.a2 = T3.a3)"; @@ -838,7 +840,7 @@ public void testJoinHintWithJoinHintInSubQuery() { } @Test - public void testJoinHintWithJoinHintInCorrelateAndWithFilter() { + void testJoinHintWithJoinHintInCorrelateAndWithFilter() { String sql = "select * from T1 WHERE a1 IN (select /*+ %s(T2) */ a2 from T2 join T3 on T2.a2 = T3.a3 where T1.a1 = T2.a2)"; @@ -846,7 +848,7 @@ public void testJoinHintWithJoinHintInCorrelateAndWithFilter() { } @Test - public void testJoinHintWithJoinHintInCorrelateAndWithProject() { + void testJoinHintWithJoinHintInCorrelateAndWithProject() { String sql = "select * from T1 WHERE a1 IN (select /*+ %s(T2) */ a2 + T1.a1 from T2 join T3 on T2.a2 = T3.a3)"; @@ -854,7 +856,7 @@ public void testJoinHintWithJoinHintInCorrelateAndWithProject() { } @Test - public void testJoinHintWithJoinHintInCorrelateAndWithAgg() { + void testJoinHintWithJoinHintInCorrelateAndWithAgg() { String sql = "select * from T1 WHERE a1 IN (select /*+ %s(T2) */ count(T2.a2) from T2 join T1 on T2.a2 = T1.a1 group by T1.a1)"; @@ -862,7 +864,7 @@ public void testJoinHintWithJoinHintInCorrelateAndWithAgg() { } @Test - public void testJoinHintWithJoinHintInCorrelateAndWithSortLimit() { + void testJoinHintWithJoinHintInCorrelateAndWithSortLimit() { String sql = "select * from T1 WHERE a1 IN (select /*+ %s(T2) */ T2.a2 from T2 join T1 on T2.a2 = T1.a1 order by T1.a1 limit 10)"; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.java index 1daf888052e02..2e70b6abe5fce 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.java @@ -21,7 +21,7 @@ import org.apache.flink.table.planner.hint.JoinStrategy; /** Test for Nest Loop join hint. */ -public class NestLoopJoinHintTest extends JoinHintTestBase { +class NestLoopJoinHintTest extends JoinHintTestBase { @Override protected String getTestSingleJoinHint() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.java index cc9f2be89e212..42e512a7866e9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.java @@ -21,7 +21,7 @@ import org.apache.flink.table.planner.hint.JoinStrategy; /** Test for Shuffle Hash join hint. */ -public class ShuffleHashJoinHintTest extends JoinHintTestBase { +class ShuffleHashJoinHintTest extends JoinHintTestBase { @Override protected String getTestSingleJoinHint() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.java index deaa6c97b881a..a28be93a79eaf 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.java @@ -21,7 +21,7 @@ import org.apache.flink.table.planner.hint.JoinStrategy; /** Test for Shuffle Merge join hint. */ -public class ShuffleMergeJoinHintTest extends JoinHintTestBase { +class ShuffleMergeJoinHintTest extends JoinHintTestBase { @Override protected String getTestSingleJoinHint() { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/UnsupportedNodesInPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/UnsupportedNodesInPlanTest.java index 6f1a0adce9de5..39540efb0efea 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/UnsupportedNodesInPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/UnsupportedNodesInPlanTest.java @@ -23,15 +23,15 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for deserialising invalid {@link org.apache.flink.table.api.CompiledPlan}. */ -public class UnsupportedNodesInPlanTest extends TableTestBase { +class UnsupportedNodesInPlanTest extends TableTestBase { @Test - public void testInvalidType() { + void testInvalidType() { final TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); assertThatThrownBy( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java index 80c8baa66e0d2..bd0bf014d922b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java @@ -30,20 +30,20 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; import java.util.Optional; /** Tests for verifying name and description of batch sql operator. */ -public class BatchOperatorNameTest extends OperatorNameTestBase { +class BatchOperatorNameTest extends OperatorNameTestBase { @Override protected TableTestUtil getTableTestUtil() { return batchTestUtil(TableConfig.getDefault()); } - @Test - public void testBoundedStreamScan() { + @TestTemplate + void testBoundedStreamScan() { final DataStream dataStream = util.getStreamEnv().fromElements(1, 2, 3, 4, 5); TableTestUtil.createTemporaryView( tEnv, @@ -56,23 +56,23 @@ public void testBoundedStreamScan() { } /** Verify Expand, HashAggregate. */ - @Test - public void testHashAggregate() { + @TestTemplate + void testHashAggregate() { createTestSource(); verifyQuery("SELECT a, " + "count(distinct b) as b " + "FROM MyTable GROUP BY a"); } /** Verify Sort, SortAggregate. */ - @Test - public void testSortAggregate() { + @TestTemplate + void testSortAggregate() { tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); createTestSource(); verifyQuery("SELECT a, " + "count(distinct b) as b " + "FROM MyTable GROUP BY a"); } /** Verify SortWindowAggregate. */ - @Test - public void testSortWindowAggregate() { + @TestTemplate + void testSortWindowAggregate() { createSourceWithTimeAttribute(); verifyQuery( "SELECT\n" @@ -84,14 +84,14 @@ public void testSortWindowAggregate() { } /** Verify HashJoin. */ - @Test - public void testHashJoin() { + @TestTemplate + void testHashJoin() { testJoinInternal(); } /** Verify NestedLoopJoin. */ - @Test - public void testNestedLoopJoin() { + @TestTemplate + void testNestedLoopJoin() { tEnv.getConfig() .set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, @@ -100,8 +100,8 @@ public void testNestedLoopJoin() { } /** Verify SortMergeJoin. */ - @Test - public void testSortMergeJoin() { + @TestTemplate + void testSortMergeJoin() { tEnv.getConfig() .set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, @@ -110,8 +110,8 @@ public void testSortMergeJoin() { } /** Verify MultiInput. */ - @Test - public void testMultiInput() { + @TestTemplate + void testMultiInput() { createTestSource("A"); createTestSource("B"); createTestSource("C"); @@ -119,21 +119,21 @@ public void testMultiInput() { } /** Verify Limit. */ - @Test - public void testLimit() { + @TestTemplate + void testLimit() { createTestSource(); verifyQuery("select * from MyTable limit 10"); } /** Verify SortLimit. */ - @Test - public void testSortLimit() { + @TestTemplate + void testSortLimit() { createTestSource(); verifyQuery("select * from MyTable order by a limit 10"); } - @Test - public void testLegacySourceSink() { + @TestTemplate + void testLegacySourceSink() { TableSchema schema = TestLegacyFilterableTableSource.defaultSchema(); TestLegacyFilterableTableSource.createTemporaryTable( tEnv, @@ -153,8 +153,8 @@ public void testLegacySourceSink() { verifyInsert("insert into MySink select * from MySource"); } - @Test - public void testMatch() { + @TestTemplate + void testMatch() { createSourceWithTimeAttribute(); String sql = "SELECT T.aid, T.bid, T.cid\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java index 82c3fbfa0558a..991d42c88b0f3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java @@ -25,30 +25,32 @@ import org.apache.flink.table.planner.utils.TableFunc1; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; import java.util.List; /** Base class for verifying name and description of SQL operators. */ -@RunWith(Parameterized.class) -public abstract class OperatorNameTestBase extends TableTestBase { - @Parameterized.Parameter public boolean isNameSimplifyEnabled; +@ExtendWith(ParameterizedTestExtension.class) +abstract class OperatorNameTestBase extends TableTestBase { + @Parameter private boolean isNameSimplifyEnabled; protected TableTestUtil util; protected TableEnvironment tEnv; - @Parameterized.Parameters(name = "isNameSimplifyEnabled={0}") - public static List testData() { + @Parameters(name = "isNameSimplifyEnabled={0}") + private static List testData() { return Arrays.asList(true, false); } - @Before - public void setup() { + @BeforeEach + void setup() { util = getTableTestUtil(); util.getStreamEnv().setParallelism(2); tEnv = util.getTableEnv(); @@ -73,16 +75,16 @@ protected void verifyInsert(String statement) { } /** Verify Correlate and Calc. */ - @Test - public void testCorrelate() { + @TestTemplate + void testCorrelate() { createTestSource(); util.addTemporarySystemFunction("func1", new TableFunc1()); verifyQuery("SELECT s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)"); } /** Verify LookUpJoin. */ - @Test - public void testLookupJoin() { + @TestTemplate + void testLookupJoin() { createSourceWithTimeAttribute(); String srcTableB = "CREATE TABLE LookupTable (\n" @@ -99,8 +101,8 @@ public void testLookupJoin() { } /** Verify GroupWindowAggregate. */ - @Test - public void testGroupWindowAggregate() { + @TestTemplate + void testGroupWindowAggregate() { createSourceWithTimeAttribute(); verifyQuery( "SELECT\n" @@ -112,8 +114,8 @@ public void testGroupWindowAggregate() { } /** Verify OverAggregate. */ - @Test - public void testOverAggregate() { + @TestTemplate + void testOverAggregate() { createSourceWithTimeAttribute(); String sql = "SELECT b,\n" @@ -124,8 +126,8 @@ public void testOverAggregate() { } /** Verify Rank. */ - @Test - public void testRank() { + @TestTemplate + void testRank() { createTestSource(); String sql = "SELECT a, row_num\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java index c49e5ab31bea2..6adc3f69f92a8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java @@ -32,8 +32,8 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.types.Row; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; import java.util.Optional; @@ -41,7 +41,7 @@ import static org.apache.flink.table.api.Expressions.call; /** Tests for verifying name and description of stream sql operator name. */ -public class StreamOperatorNameTest extends OperatorNameTestBase { +class StreamOperatorNameTest extends OperatorNameTestBase { private StreamTableTestUtil util; @@ -50,15 +50,15 @@ protected TableTestUtil getTableTestUtil() { return streamTestUtil(TableConfig.getDefault()); } - @Before - public void setup() { + @BeforeEach + void setup() { super.setup(); util = (StreamTableTestUtil) super.util; } /** Verify DropUpdateBefore. */ - @Test - public void testDropUpdateBefore() throws Exception { + @TestTemplate + void testDropUpdateBefore() { util.getStreamEnv().setParallelism(2); @@ -92,8 +92,8 @@ public void testDropUpdateBefore() throws Exception { } /** Verify ChangelogNormalize and SinkMaterialize. */ - @Test - public void testChangelogNormalize() throws Exception { + @TestTemplate + void testChangelogNormalize() throws Exception { util.getStreamEnv().setParallelism(2); @@ -127,8 +127,8 @@ public void testChangelogNormalize() throws Exception { } /** Verify Deduplicate. */ - @Test - public void testDeduplicate() { + @TestTemplate + void testDeduplicate() { createSourceWithTimeAttribute(); verifyQuery( "SELECT a, b, c FROM " @@ -142,8 +142,8 @@ public void testDeduplicate() { * Verify Expand, MiniBatchAssigner, LocalGroupAggregate, GlobalGroupAggregate, * IncrementalAggregate. */ - @Test - public void testIncrementalAggregate() { + @TestTemplate + void testIncrementalAggregate() { util.enableMiniBatch(); tEnv.getConfig() .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true); @@ -152,14 +152,14 @@ public void testIncrementalAggregate() { } /** Verify GroupAggregate. */ - @Test - public void testGroupAggregate() { + @TestTemplate + void testGroupAggregate() { testGroupAggregateInternal(); } /** Verify RowConversion, TableGroupAggregate. */ - @Test - public void testTableGroupAggregate() { + @TestTemplate + void testTableGroupAggregate() { final DataStream dataStream = util.getStreamEnv().fromElements(1, 2, 3, 4, 5); TableTestUtil.createTemporaryView( tEnv, @@ -178,8 +178,8 @@ public void testTableGroupAggregate() { } /** Verify IntervalJoin. */ - @Test - public void testIntervalJoin() { + @TestTemplate + void testIntervalJoin() { createSourceWithTimeAttribute("A"); createSourceWithTimeAttribute("B"); verifyQuery( @@ -189,8 +189,8 @@ public void testIntervalJoin() { } /** Verify IntervalJoin. */ - @Test - public void testIntervalJoinNegativeWindow() { + @TestTemplate + void testIntervalJoinNegativeWindow() { createSourceWithTimeAttribute("A"); createSourceWithTimeAttribute("B"); verifyQuery( @@ -200,13 +200,13 @@ public void testIntervalJoinNegativeWindow() { } /** Verify Join. */ - @Test - public void testJoin() { + @TestTemplate + void testJoin() { testJoinInternal(); } - @Test - public void testMatch() { + @TestTemplate + void testMatch() { createSourceWithTimeAttribute(); String sql = "SELECT T.aid, T.bid, T.cid\n" @@ -225,8 +225,8 @@ public void testMatch() { verifyQuery(sql); } - @Test - public void testTemporalJoin() { + @TestTemplate + void testTemporalJoin() { tEnv.executeSql( "CREATE TABLE Orders (\n" + " amount INT,\n" @@ -257,21 +257,21 @@ public void testTemporalJoin() { + "WHERE o.currency = r.currency "); } - @Test - public void testTemporalSortOnProcTime() { + @TestTemplate + void testTemporalSortOnProcTime() { createSourceWithTimeAttribute(); verifyQuery("SELECT a FROM MyTable order by proctime, c"); } - @Test - public void testTemporalSortOnEventTime() { + @TestTemplate + void testTemporalSortOnEventTime() { createSourceWithTimeAttribute(); verifyQuery("SELECT a FROM MyTable order by rowtime, c"); } /** Verify WindowJoin, WindowRank, WindowAggregate, WindowDeduplicate. */ - @Test - public void testWindowAggregate() { + @TestTemplate + void testWindowAggregate() { tEnv.getConfig() .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); createSourceWithTimeAttribute(); @@ -288,8 +288,8 @@ public void testWindowAggregate() { } /** Verify LocalWindowAggregate, GlobalWindowAggregate. */ - @Test - public void testLocalGlobalWindowAggregate() { + @TestTemplate + void testLocalGlobalWindowAggregate() { createSourceWithTimeAttribute(); verifyQuery( "SELECT\n" @@ -304,8 +304,8 @@ public void testLocalGlobalWindowAggregate() { } /** Verify WindowJoin. */ - @Test - public void testWindowJoin() { + @TestTemplate + void testWindowJoin() { createSourceWithTimeAttribute("MyTable"); createSourceWithTimeAttribute("MyTable2"); verifyQuery( @@ -342,8 +342,8 @@ public void testWindowJoin() { } /** Verify WindowTableFunction and WindowRank. */ - @Test - public void testWindowRank() { + @TestTemplate + void testWindowRank() { createSourceWithTimeAttribute(); verifyQuery( "select\n" @@ -361,8 +361,8 @@ public void testWindowRank() { } /** Verify WindowDeduplicate. */ - @Test - public void testWindowDeduplicate() { + @TestTemplate + void testWindowDeduplicate() { createSourceWithTimeAttribute(); verifyQuery( "select\n" @@ -380,8 +380,8 @@ public void testWindowDeduplicate() { } /** Verify LegacySource and LegacySink. */ - @Test - public void testLegacySourceSink() { + @TestTemplate + void testLegacySourceSink() { TableSchema schema = TestLegacyFilterableTableSource.defaultSchema(); TestLegacyFilterableTableSource.createTemporaryTable( tEnv, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java index 213982425cef3..31b7ebf7273ce 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java @@ -32,9 +32,10 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.FileUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -42,13 +43,13 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link MultipleInputNodeCreationProcessor}. */ -public class MultipleInputNodeCreationProcessorTest extends TableTestBase { +class MultipleInputNodeCreationProcessorTest extends TableTestBase { private final BatchTableTestUtil batchUtil = batchTestUtil(TableConfig.getDefault()); private final StreamTableTestUtil streamUtil = streamTestUtil(TableConfig.getDefault()); @Test - public void testIsChainableDataStreamSource() { + void testIsChainableDataStreamSource() { createChainableStream(batchUtil); assertChainableSource("chainableStream", batchUtil, true); createChainableStream(streamUtil); @@ -56,7 +57,7 @@ public void testIsChainableDataStreamSource() { } @Test - public void testNonChainableDataStreamSource() { + void testNonChainableDataStreamSource() { createNonChainableStream(batchUtil); assertChainableSource("nonChainableStream", batchUtil, false); createNonChainableStream(streamUtil); @@ -64,7 +65,7 @@ public void testNonChainableDataStreamSource() { } @Test - public void testIsChainableTableSource() throws IOException { + void testIsChainableTableSource() throws IOException { createTestFileSource(batchUtil.tableEnv(), "fileSource1", "Source"); assertChainableSource("fileSource1", batchUtil, true); createTestFileSource(streamUtil.tableEnv(), "fileSource1", "Source"); @@ -77,7 +78,7 @@ public void testIsChainableTableSource() throws IOException { } @Test - public void testNonChainableTableSource() throws IOException { + void testNonChainableTableSource() throws IOException { createTestValueSource(batchUtil.tableEnv(), "valueSource1", "DataStream"); assertChainableSource("valueSource1", batchUtil, false); createTestValueSource(streamUtil.tableEnv(), "valueSource1", "DataStream"); @@ -135,7 +136,7 @@ private void createNonChainableStream(TableTestUtil util) { private void createTestFileSource(TableEnvironment tEnv, String name, String runtimeSource) throws IOException { - File file = tempFolder().newFile(); + File file = TempDirUtils.newFile(tempFolder()); file.delete(); file.createNewFile(); FileUtils.writeFileUtf8(file, "1\n2\n3\n"); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest.java index 0eea911954f62..7509fc282b383 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest.java @@ -28,17 +28,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for calc. */ -public class CalcJsonPlanTest extends TableTestBase { +class CalcJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -55,7 +55,7 @@ public void setup() { } @Test - public void testSimpleProject() { + void testSimpleProject() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -68,7 +68,7 @@ public void testSimpleProject() { } @Test - public void testSimpleFilter() { + void testSimpleFilter() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -83,7 +83,7 @@ public void testSimpleFilter() { } @Test - public void testComplexCalc() { + void testComplexCalc() { tEnv.createTemporaryFunction("udf1", new JavaFunc0()); tEnv.createTemporaryFunction("udf2", JavaFunc1.class); tEnv.createTemporarySystemFunction("udf3", new JavaFunc2()); @@ -116,7 +116,7 @@ public void testComplexCalc() { } @Test - public void testSarg() { + void testSarg() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint\n" @@ -130,7 +130,7 @@ public void testSarg() { } @Test - public void testProjectPushDown() { + void testProjectPushDown() { // ensure PartitionPushDownSpec was added to exec plan String sinkTableDdl = "CREATE TABLE MySink (\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest.java index 1ec1567d134b5..0e35bc11bca3e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest.java @@ -24,24 +24,24 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for changelog source, including upsert source. */ -public class ChangelogSourceJsonPlanTest extends TableTestBase { +class ChangelogSourceJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true); } @Test - public void testChangelogSource() { + void testChangelogSource() { String srcTableDdl = "CREATE TABLE MyTable (\n" + " a bigint,\n" @@ -68,7 +68,7 @@ public void testChangelogSource() { } @Test - public void testUpsertSource() { + void testUpsertSource() { String srcTableDdl = "CREATE TABLE MyTable (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest.java index 8d1ebe80727a1..d7672a03e03a5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest.java @@ -24,17 +24,17 @@ import org.apache.flink.table.planner.utils.TableFunc1; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for correlate. */ -public class CorrelateJsonPlanTest extends TableTestBase { +class CorrelateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -51,7 +51,7 @@ public void setup() { } @Test - public void testCrossJoin() { + void testCrossJoin() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" @@ -68,8 +68,8 @@ public void testCrossJoin() { } @Test - @Ignore("the case is ignored because of FLINK-21870") - public void testRegisterByClass() { + @Disabled("the case is ignored because of FLINK-21870") + void testRegisterByClass() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" @@ -86,7 +86,7 @@ public void testRegisterByClass() { } @Test - public void testCrossJoinOverrideParameters() { + void testCrossJoinOverrideParameters() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" @@ -103,7 +103,7 @@ public void testCrossJoinOverrideParameters() { } @Test - public void testLeftOuterJoinWithLiteralTrue() { + void testLeftOuterJoinWithLiteralTrue() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" @@ -120,7 +120,7 @@ public void testLeftOuterJoinWithLiteralTrue() { } @Test - public void testJoinWithFilter() { + void testJoinWithFilter() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest.java index 10543540f5880..cccd49cffc049 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest.java @@ -23,23 +23,23 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for deduplicate. */ -public class DeduplicationJsonPlanTest extends TableTestBase { +class DeduplicationJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); } @Test - public void testDeduplication() { + void testDeduplication() { String srcTableDdl = "CREATE TABLE srcValuesTable (\n" + " order_id bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest.java index b55cf03fb9f47..9d568e6417e9d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest.java @@ -26,17 +26,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for expand. */ -public class ExpandJsonPlanTest extends TableTestBase { +class ExpandJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); tEnv.getConfig() @@ -58,7 +58,7 @@ public void setup() { } @Test - public void testExpand() { + void testExpand() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest.java index 27b7296488853..e655a3527ea4f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest.java @@ -27,32 +27,34 @@ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg; import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.Arrays; import java.util.List; /** Test json serialization/deserialization for group aggregate. */ -@RunWith(Parameterized.class) -public class GroupAggregateJsonPlanTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class GroupAggregateJsonPlanTest extends TableTestBase { - @Parameterized.Parameter public boolean isMiniBatchEnabled; + @Parameter private boolean isMiniBatchEnabled; private StreamTableTestUtil util; private TableEnvironment tEnv; - @Parameterized.Parameters(name = "isMiniBatchEnabled={0}") - public static List testData() { + @Parameters(name = "isMiniBatchEnabled={0}") + private static List testData() { return Arrays.asList(true, false); } - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); if (isMiniBatchEnabled) { @@ -78,8 +80,8 @@ public void setup() { tEnv.executeSql(srcTableDdl); } - @Test - public void testSimpleAggCallsWithGroupBy() { + @TestTemplate + void testSimpleAggCallsWithGroupBy() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b bigint,\n" @@ -99,8 +101,8 @@ public void testSimpleAggCallsWithGroupBy() { + "from MyTable group by b"); } - @Test - public void testSimpleAggWithoutGroupBy() { + @TestTemplate + void testSimpleAggWithoutGroupBy() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " avg_a double,\n" @@ -123,8 +125,8 @@ public void testSimpleAggWithoutGroupBy() { + "from MyTable"); } - @Test - public void testDistinctAggCalls() { + @TestTemplate + void testDistinctAggCalls() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " d bigint,\n" @@ -150,8 +152,8 @@ public void testDistinctAggCalls() { + "from MyTable group by d"); } - @Test - public void testUserDefinedAggCalls() { + @TestTemplate + void testUserDefinedAggCalls() { tEnv.createTemporaryFunction("my_sum1", new VarSum1AggFunction()); tEnv.createFunction("my_avg", WeightedAvg.class); tEnv.createTemporarySystemFunction("my_sum2", VarSum2AggFunction.class); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest.java index 12313ff234e1d..d7341829a7375 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest.java @@ -24,16 +24,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for group window aggregate. */ -public class GroupWindowAggregateJsonPlanTest extends TableTestBase { +class GroupWindowAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -51,7 +51,7 @@ public void setup() { } @Test - public void testEventTimeTumbleWindow() { + void testEventTimeTumbleWindow() { tEnv.createFunction( "concat_distinct_agg", JavaUserDefinedAggFunctions.ConcatDistinctAggFunction.class); String sinkTableDdl = @@ -80,7 +80,7 @@ public void testEventTimeTumbleWindow() { } @Test - public void testProcTimeTumbleWindow() { + void testProcTimeTumbleWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -99,7 +99,7 @@ public void testProcTimeTumbleWindow() { } @Test - public void testEventTimeHopWindow() { + void testEventTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -118,7 +118,7 @@ public void testEventTimeHopWindow() { } @Test - public void testProcTimeHopWindow() { + void testProcTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -135,7 +135,7 @@ public void testProcTimeHopWindow() { } @Test - public void testEventTimeSessionWindow() { + void testEventTimeSessionWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -154,7 +154,7 @@ public void testEventTimeSessionWindow() { } @Test - public void testProcTimeSessionWindow() { + void testProcTimeSessionWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest.java index cc210a6dff4e5..26dcc04f30346 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest.java @@ -27,19 +27,19 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.time.Duration; /** Test json serialization/deserialization for incremental aggregate. */ -public class IncrementalAggregateJsonPlanTest extends TableTestBase { +class IncrementalAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); tEnv.getConfig() @@ -67,7 +67,7 @@ public void setup() { } @Test - public void testIncrementalAggregate() { + void testIncrementalAggregate() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -84,7 +84,7 @@ public void testIncrementalAggregate() { } @Test - public void testIncrementalAggregateWithSumCountDistinctAndRetraction() { + void testIncrementalAggregateWithSumCountDistinctAndRetraction() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest.java index d5325f133fa33..71a6f52ea0b0d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for IntervalJoin. */ -public class IntervalJoinJsonPlanTest extends TableTestBase { +class IntervalJoinJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -64,7 +64,7 @@ public void setup() { } @Test - public void testProcessingTimeInnerJoinWithOnClause() { + void testProcessingTimeInnerJoinWithOnClause() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" @@ -81,7 +81,7 @@ public void testProcessingTimeInnerJoinWithOnClause() { } @Test - public void testRowTimeInnerJoinWithOnClause() { + void testRowTimeInnerJoinWithOnClause() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest.java index c0154949333e2..dd2771ea7cf8c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for join. */ -public class JoinJsonPlanTest extends TableTestBase { +class JoinJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -76,7 +76,7 @@ public void setup() { } @Test - public void testInnerJoin() { + void testInnerJoin() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a1 int,\n" @@ -89,7 +89,7 @@ public void testInnerJoin() { } @Test - public void testInnerJoinWithEqualPk() { + void testInnerJoinWithEqualPk() { String query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"; String query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"; String query = @@ -107,7 +107,7 @@ public void testInnerJoinWithEqualPk() { } @Test - public void testInnerJoinWithPk() { + void testInnerJoinWithPk() { String query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"; String query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"; String query = @@ -128,7 +128,7 @@ public void testInnerJoinWithPk() { } @Test - public void testLeftJoinNonEqui() { + void testLeftJoinNonEqui() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a1 int,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java index 1cdfca1c3db28..6aa3a3a75c7f1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java @@ -29,8 +29,8 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.types.Row; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -40,13 +40,13 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test json serialization/deserialization for LookupJoin. */ -public class LookupJoinJsonPlanTest extends TableTestBase { +class LookupJoinJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -97,7 +97,7 @@ public void setup() { } @Test - public void testJoinTemporalTable() { + void testJoinTemporalTable() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" @@ -118,7 +118,7 @@ public void testJoinTemporalTable() { } @Test - public void testJoinTemporalTableWithProjectionPushDown() { + void testJoinTemporalTableWithProjectionPushDown() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" @@ -140,7 +140,7 @@ public void testJoinTemporalTableWithProjectionPushDown() { } @Test - public void testLegacyTableSourceException() { + void testLegacyTableSourceException() { TableSchema tableSchema = TableSchema.builder() .field("id", Types.INT) @@ -182,7 +182,7 @@ public void testLegacyTableSourceException() { } @Test - public void testAggAndLeftJoinWithTryResolveMode() { + void testAggAndLeftJoinWithTryResolveMode() { tEnv.getConfig() .set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, @@ -197,7 +197,7 @@ public void testAggAndLeftJoinWithTryResolveMode() { } @Test - public void testJoinTemporalTableWithAsyncHint() { + void testJoinTemporalTableWithAsyncHint() { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " @@ -207,7 +207,7 @@ public void testJoinTemporalTableWithAsyncHint() { } @Test - public void testJoinTemporalTableWithAsyncHint2() { + void testJoinTemporalTableWithAsyncHint2() { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " @@ -217,7 +217,7 @@ public void testJoinTemporalTableWithAsyncHint2() { } @Test - public void testJoinTemporalTableWithRetryHint() { + void testJoinTemporalTableWithRetryHint() { util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " + "/*+ LOOKUP('table'='D', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " @@ -226,7 +226,7 @@ public void testJoinTemporalTableWithRetryHint() { } @Test - public void testJoinTemporalTableWithAsyncRetryHint() { + void testJoinTemporalTableWithAsyncRetryHint() { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " @@ -236,7 +236,7 @@ public void testJoinTemporalTableWithAsyncRetryHint() { } @Test - public void testJoinTemporalTableWithAsyncRetryHint2() { + void testJoinTemporalTableWithAsyncRetryHint2() { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " @@ -246,7 +246,7 @@ public void testJoinTemporalTableWithAsyncRetryHint2() { } @Test - public void testLeftJoinTemporalTableWithPreFilter() { + void testLeftJoinTemporalTableWithPreFilter() { util.verifyJsonPlan( "INSERT INTO MySink1 SELECT * " + "FROM MyTable AS T LEFT JOIN LookupTable " @@ -254,7 +254,7 @@ public void testLeftJoinTemporalTableWithPreFilter() { } @Test - public void testLeftJoinTemporalTableWithPostFilter() { + void testLeftJoinTemporalTableWithPostFilter() { util.verifyJsonPlan( "INSERT INTO MySink1 SELECT * " + "FROM MyTable AS T LEFT JOIN LookupTable " @@ -263,7 +263,7 @@ public void testLeftJoinTemporalTableWithPostFilter() { } @Test - public void testLeftJoinTemporalTableWithMultiJoinConditions() { + void testLeftJoinTemporalTableWithMultiJoinConditions() { util.verifyJsonPlan( "INSERT INTO MySink1 SELECT * " + "FROM MyTable AS T LEFT JOIN LookupTable " diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java index e0ef65b447beb..c31f300ca4e5d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java @@ -23,22 +23,22 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for match recognize. */ -public class MatchRecognizeJsonPlanTest extends TableTestBase { +class MatchRecognizeJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); } @Test - public void testMatch() { + void testMatch() { String srcTableDdl = "CREATE TABLE MyTable (\n" + " id bigint,\n" @@ -78,22 +78,22 @@ public void testMatch() { } @Test - public void testSkipToLast() { + void testSkipToLast() { doTestAfterMatch("AFTER MATCH SKIP TO LAST B"); } @Test - public void testSkipToFirst() { + void testSkipToFirst() { doTestAfterMatch("AFTER MATCH SKIP TO FIRST B"); } @Test - public void testSkipPastLastRow() { + void testSkipPastLastRow() { doTestAfterMatch("AFTER MATCH SKIP PAST LAST ROW"); } @Test - public void testSkipToNextRow() { + void testSkipToNextRow() { doTestAfterMatch("AFTER MATCH SKIP TO NEXT ROW"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest.java index 6607dca9a3443..c01fa4121768e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest.java @@ -23,16 +23,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for over aggregate. */ -public class OverAggregateJsonPlanTest extends TableTestBase { +class OverAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); String srcTableDdl = @@ -50,7 +50,7 @@ public void setup() { } @Test - public void testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver() { + void testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a varchar,\n" @@ -78,7 +78,7 @@ public void testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver() { } @Test - public void testProctimeBoundedDistinctPartitionedRowOver() { + void testProctimeBoundedDistinctPartitionedRowOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -100,7 +100,7 @@ public void testProctimeBoundedDistinctPartitionedRowOver() { } @Test - public void testProcTimeBoundedPartitionedRangeOver() { + void testProcTimeBoundedPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -119,7 +119,7 @@ public void testProcTimeBoundedPartitionedRangeOver() { } @Test - public void testProcTimeBoundedNonPartitionedRangeOver() { + void testProcTimeBoundedNonPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -138,7 +138,7 @@ public void testProcTimeBoundedNonPartitionedRangeOver() { } @Test - public void testProcTimeUnboundedPartitionedRangeOver() { + void testProcTimeUnboundedPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -158,7 +158,7 @@ public void testProcTimeUnboundedPartitionedRangeOver() { } @Test - public void testRowTimeBoundedPartitionedRowsOver() { + void testRowTimeBoundedPartitionedRowsOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -177,7 +177,7 @@ public void testRowTimeBoundedPartitionedRowsOver() { } @Test - public void testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime() { + void testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest.java index 07a367d6d24d0..dbe3c840cb075 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCalcJsonPlanTest.java @@ -25,17 +25,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for calc. */ -public class PythonCalcJsonPlanTest extends TableTestBase { +class PythonCalcJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -52,7 +52,7 @@ public void setup() { } @Test - public void testPythonCalc() { + void testPythonCalc() { tEnv.createTemporaryFunction("pyFunc", new PythonScalarFunction("pyFunc")); String sinkTableDdl = "CREATE TABLE MySink (\n" @@ -66,7 +66,7 @@ public void testPythonCalc() { } @Test - public void testPythonFunctionInWhereClause() { + void testPythonFunctionInWhereClause() { tEnv.createTemporaryFunction("pyFunc", new BooleanPythonScalarFunction("pyFunc")); String sinkTableDdl = "CREATE TABLE MySink (\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest.java index 456df0dd0ed1d..c2bc2009b4601 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonCorrelateJsonPlanTest.java @@ -25,16 +25,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for correlate. */ -public class PythonCorrelateJsonPlanTest extends TableTestBase { +class PythonCorrelateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { TableConfig tableConfig = TableConfig.getDefault(); util = streamTestUtil(tableConfig); tEnv = util.getTableEnv(); @@ -54,7 +54,7 @@ public void setup() { } @Test - public void testPythonTableFunction() { + void testPythonTableFunction() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" @@ -71,7 +71,7 @@ public void testPythonTableFunction() { } @Test - public void testJoinWithFilter() { + void testJoinWithFilter() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest.java index 5b50a2597cb3e..8f2e72d491604 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupAggregateJsonPlanTest.java @@ -24,17 +24,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for group aggregate. */ -public class PythonGroupAggregateJsonPlanTest extends TableTestBase { +class PythonGroupAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -52,7 +52,7 @@ public void setup() { } @Test - public void tesPythonAggCallsWithGroupBy() { + void tesPythonAggCallsWithGroupBy() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest.java index afcd4a86fe1a6..0b1058b15299a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest.java @@ -24,16 +24,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for group window aggregate. */ -public class PythonGroupWindowAggregateJsonPlanTest extends TableTestBase { +class PythonGroupWindowAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -52,7 +52,7 @@ public void setup() { } @Test - public void testEventTimeTumbleWindow() { + void testEventTimeTumbleWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -73,7 +73,7 @@ public void testEventTimeTumbleWindow() { } @Test - public void testProcTimeTumbleWindow() { + void testProcTimeTumbleWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -92,7 +92,7 @@ public void testProcTimeTumbleWindow() { } @Test - public void testEventTimeHopWindow() { + void testEventTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -109,7 +109,7 @@ public void testEventTimeHopWindow() { } @Test - public void testProcTimeHopWindow() { + void testProcTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -126,7 +126,7 @@ public void testProcTimeHopWindow() { } @Test - public void testEventTimeSessionWindow() { + void testEventTimeSessionWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -143,7 +143,7 @@ public void testEventTimeSessionWindow() { } @Test - public void testProcTimeSessionWindow() { + void testProcTimeSessionWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest.java index ec5f4e3f96768..cf91be90074e7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest.java @@ -24,16 +24,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for over aggregate. */ -public class PythonOverAggregateJsonPlanTest extends TableTestBase { +class PythonOverAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); String srcTableDdl = @@ -52,7 +52,7 @@ public void setup() { } @Test - public void testProcTimeBoundedPartitionedRangeOver() { + void testProcTimeBoundedPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -71,7 +71,7 @@ public void testProcTimeBoundedPartitionedRangeOver() { } @Test - public void testProcTimeBoundedNonPartitionedRangeOver() { + void testProcTimeBoundedNonPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -90,7 +90,7 @@ public void testProcTimeBoundedNonPartitionedRangeOver() { } @Test - public void testProcTimeUnboundedPartitionedRangeOver() { + void testProcTimeUnboundedPartitionedRangeOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -108,7 +108,7 @@ public void testProcTimeUnboundedPartitionedRangeOver() { } @Test - public void testRowTimeBoundedPartitionedRowsOver() { + void testRowTimeBoundedPartitionedRowsOver() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -127,7 +127,7 @@ public void testRowTimeBoundedPartitionedRowsOver() { } @Test - public void testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime() { + void testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest.java index ab3e1411b7ddb..ba4bdd7c6f1aa 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for rank. */ -public class RankJsonPlanTest extends TableTestBase { +class RankJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -50,7 +50,7 @@ public void setup() { } @Test - public void testRank() { + void testRank() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest.java index a23264cc2b753..91c62f9fe3364 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for sort limit. */ -public class SortJsonPlanTest extends TableTestBase { +class SortJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -50,7 +50,7 @@ public void setup() { } @Test - public void testSort() { + void testSort() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest.java index 089f4d676f446..374b309696211 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization for sort limit. */ -public class SortLimitJsonPlanTest extends TableTestBase { +class SortLimitJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -50,7 +50,7 @@ public void setup() { } @Test - public void testSortLimit() { + void testSortLimit() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest.java index 1536d3e499a29..3c321b5c59ae3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest.java @@ -24,17 +24,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for table sink. */ -public class TableSinkJsonPlanTest extends TableTestBase { +class TableSinkJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -50,7 +50,7 @@ public void setup() { } @Test - public void testOverwrite() { + void testOverwrite() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -65,7 +65,7 @@ public void testOverwrite() { } @Test - public void testPartitioning() { + void testPartitioning() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -80,7 +80,7 @@ public void testPartitioning() { } @Test - public void testWritingMetadata() { + void testWritingMetadata() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" @@ -94,7 +94,7 @@ public void testWritingMetadata() { } @Test - public void testCdcWithNonDeterministicFuncSinkWithDifferentPk() { + void testCdcWithNonDeterministicFuncSinkWithDifferentPk() { tEnv.createTemporaryFunction( "ndFunc", new JavaUserDefinedScalarFunctions.NonDeterministicUdf()); @@ -129,7 +129,7 @@ public void testCdcWithNonDeterministicFuncSinkWithDifferentPk() { } @Test - public void testPartialInsert() { + void testPartialInsert() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest.java index 390e0901ecafc..fef94f021e06e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest.java @@ -24,17 +24,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for table source. */ -public class TableSourceJsonPlanTest extends TableTestBase { +class TableSourceJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -60,7 +60,7 @@ public void setup() { } @Test - public void testProjectPushDown() { + void testProjectPushDown() { String sinkTableDdl = "CREATE TABLE sink (\n" + " a bigint,\n" @@ -73,7 +73,7 @@ public void testProjectPushDown() { } @Test - public void testReadingMetadata() { + void testReadingMetadata() { String srcTableDdl = "CREATE TABLE MyTable2 (\n" + " a bigint,\n" @@ -98,7 +98,7 @@ public void testReadingMetadata() { } @Test - public void testFilterPushDown() { + void testFilterPushDown() { String srcTableDdl = "CREATE TABLE src (\n" + " a bigint,\n" @@ -113,12 +113,12 @@ public void testFilterPushDown() { } @Test - public void testLimitPushDown() { + void testLimitPushDown() { util.verifyJsonPlan("insert into MySink select * from MyTable limit 3"); } @Test - public void testPartitionPushDown() { + void testPartitionPushDown() { String srcTableDdl = "CREATE TABLE PartitionTable (\n" + " a bigint,\n" @@ -134,7 +134,7 @@ public void testPartitionPushDown() { } @Test - public void testWatermarkPushDown() { + void testWatermarkPushDown() { String srcTableDdl = "CREATE TABLE WatermarkTable (\n" + " a bigint,\n" @@ -160,7 +160,7 @@ public void testWatermarkPushDown() { } @Test - public void testReuseSourceWithoutProjectionPushDown() { + void testReuseSourceWithoutProjectionPushDown() { tEnv.executeSql( "CREATE TEMPORARY TABLE src (\n" + " x varchar,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest.java index 5d51c8757a44a..da3e6eaebae1d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest.java @@ -24,19 +24,19 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.apache.flink.table.api.Expressions.$; /** Test json serialization/deserialization for TemporalJoin. */ -public class TemporalJoinJsonPlanTest extends TableTestBase { +class TemporalJoinJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -66,7 +66,7 @@ public void setup() { } @Test - public void testJoinTemporalFunction() { + void testJoinTemporalFunction() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int\n" @@ -83,7 +83,7 @@ public void testJoinTemporalFunction() { } @Test - public void testTemporalTableJoin() { + void testTemporalTableJoin() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a int\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest.java index 3ba476dc5c591..8ebd6a47e5971 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest.java @@ -23,16 +23,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for temporal sort. */ -public class TemporalSortJsonPlanTest extends TableTestBase { +class TemporalSortJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); TableEnvironment tEnv = util.getTableEnv(); @@ -57,12 +57,12 @@ public void setup() { } @Test - public void testSortProcessingTime() { + void testSortProcessingTime() { util.verifyJsonPlan("insert into MySink SELECT a FROM MyTable order by proctime, c"); } @Test - public void testSortRowTime() { + void testSortRowTime() { util.verifyJsonPlan("insert into MySink SELECT a FROM MyTable order by rowtime, c"); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest.java index 7bad10a85d664..bbb1b041c2314 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest.java @@ -23,16 +23,16 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for union. */ -public class UnionJsonPlanTest extends TableTestBase { +class UnionJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -58,7 +58,7 @@ public void setup() { } @Test - public void testUnion() { + void testUnion() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest.java index b42a3a24ff464..23eae28bc66cc 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for values. */ -public class ValuesJsonPlanTest extends TableTestBase { +class ValuesJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); String sinkTableDdl = @@ -49,7 +49,7 @@ public void setup() { } @Test - public void testValues() { + void testValues() { util.verifyJsonPlan( "INSERT INTO MySink SELECT * FROM (VALUES (1, 2, 'Hi'), (3, 4, 'Hello'))"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest.java index 8552862427080..a8690eec56f2a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest.java @@ -23,23 +23,23 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for watermark assigner. */ -public class WatermarkAssignerJsonPlanTest extends TableTestBase { +class WatermarkAssignerJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); } @Test - public void testWatermarkAssigner() { + void testWatermarkAssigner() { String srcTableDdl = "CREATE TABLE WatermarkTable (\n" + " a bigint,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest.java index 7f6672f3fd0fa..c6a2662c813cd 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest.java @@ -25,17 +25,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for window aggregate. */ -public class WindowAggregateJsonPlanTest extends TableTestBase { +class WindowAggregateJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -53,7 +53,7 @@ public void setup() { } @Test - public void testEventTimeTumbleWindow() { + void testEventTimeTumbleWindow() { tEnv.createFunction("concat_distinct_agg", ConcatDistinctAggFunction.class); String sinkTableDdl = "CREATE TABLE MySink (\n" @@ -82,7 +82,7 @@ public void testEventTimeTumbleWindow() { } @Test - public void testEventTimeTumbleWindowWithOffset() { + void testEventTimeTumbleWindowWithOffset() { tEnv.createFunction("concat_distinct_agg", ConcatDistinctAggFunction.class); String sinkTableDdl = "CREATE TABLE MySink (\n" @@ -115,7 +115,7 @@ public void testEventTimeTumbleWindowWithOffset() { } @Test - public void testProcTimeTumbleWindow() { + void testProcTimeTumbleWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -135,7 +135,7 @@ public void testProcTimeTumbleWindow() { } @Test - public void testEventTimeHopWindow() { + void testEventTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -155,7 +155,7 @@ public void testEventTimeHopWindow() { } @Test - public void testEventTimeHopWindowWithOffset() { + void testEventTimeHopWindowWithOffset() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -180,7 +180,7 @@ public void testEventTimeHopWindowWithOffset() { } @Test - public void testProcTimeHopWindow() { + void testProcTimeHopWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -198,7 +198,7 @@ public void testProcTimeHopWindow() { } @Test - public void testEventTimeCumulateWindow() { + void testEventTimeCumulateWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -224,7 +224,7 @@ public void testEventTimeCumulateWindow() { } @Test - public void testEventTimeCumulateWindowWithOffset() { + void testEventTimeCumulateWindowWithOffset() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -251,7 +251,7 @@ public void testEventTimeCumulateWindowWithOffset() { } @Test - public void testProcTimeCumulateWindow() { + void testProcTimeCumulateWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " b BIGINT,\n" @@ -273,7 +273,7 @@ public void testProcTimeCumulateWindow() { } @Test - public void testDistinctSplitEnabled() { + void testDistinctSplitEnabled() { tEnv.getConfig() .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true); String sinkTableDdl = diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest.java index 58a36f01df5ba..5d69beb7acea1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for window join. */ -public class WindowJoinJsonPlanTest extends TableTestBase { +class WindowJoinJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -63,7 +63,7 @@ public void setup() { } @Test - public void testEventTimeTumbleWindow() { + void testEventTimeTumbleWindow() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " l_a INT,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest.java index 87da0b2e2e200..2b9ec7ba1e475 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest.java @@ -23,17 +23,17 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test json serialization/deserialization for window table function. */ -public class WindowTableFunctionJsonPlanTest extends TableTestBase { +class WindowTableFunctionJsonPlanTest extends TableTestBase { private StreamTableTestUtil util; private TableEnvironment tEnv; - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); tEnv = util.getTableEnv(); @@ -63,7 +63,7 @@ public void setup() { } @Test - public void testIndividualWindowTVF() { + void testIndividualWindowTVF() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " window_start TIMESTAMP(3),\n" @@ -85,7 +85,7 @@ public void testIndividualWindowTVF() { } @Test - public void testIndividualWindowTVFProcessingTime() { + void testIndividualWindowTVFProcessingTime() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " window_start TIMESTAMP(3),\n" @@ -107,7 +107,7 @@ public void testIndividualWindowTVFProcessingTime() { } @Test - public void testFollowedByWindowJoin() { + void testFollowedByWindowJoin() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " window_start TIMESTAMP(3) NOT NULL,\n" @@ -155,7 +155,7 @@ public void testFollowedByWindowJoin() { } @Test - public void testFollowedByWindowRank() { + void testFollowedByWindowRank() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " window_start TIMESTAMP(3),\n" @@ -182,7 +182,7 @@ public void testFollowedByWindowRank() { } @Test - public void testFollowedByWindowDeduplicate() { + void testFollowedByWindowDeduplicate() { String sinkTableDdl = "CREATE TABLE MySink (\n" + " window_start TIMESTAMP(3),\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java index 3fd2b8131e19f..873604a3b0b9f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java @@ -33,7 +33,7 @@ import java.util.stream.Collectors; /** A test class for {@link ClearQueryBlockAliasResolver}. */ -public class ClearQueryBlockAliasResolverTest extends JoinHintTestBase { +class ClearQueryBlockAliasResolverTest extends JoinHintTestBase { // use any join hint for test @Override diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ScanReuseTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ScanReuseTest.java index 7016e15d15ec7..82b9d851ac89d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ScanReuseTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ScanReuseTest.java @@ -21,37 +21,38 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assumptions.assumeThat; /** Test push project into source with sub plan reuse. */ -@RunWith(Parameterized.class) -public class ScanReuseTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class ScanReuseTest extends TableTestBase { private final boolean isStreaming; private final TableTestUtil util; - public ScanReuseTest(boolean isStreaming) { + ScanReuseTest(boolean isStreaming) { this.isStreaming = isStreaming; TableConfig config = TableConfig.getDefault(); this.util = isStreaming ? streamTestUtil(config) : batchTestUtil(config); } - @Parameterized.Parameters(name = "isStreaming: {0}") - public static Collection parameters() { + @Parameters(name = "isStreaming: {0}") + private static Collection parameters() { return Arrays.asList(true, false); } - @Before - public void before() { + @BeforeEach + void before() { String table = isStreaming ? "CREATE TABLE MyTable (\n" @@ -89,29 +90,29 @@ public void before() { util.tableEnv().executeSql(table); } - @Test - public void testProject() { + @TestTemplate + void testProject() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM MyTable T1, MyTable T2 WHERE T1.a = T2.a"; util.verifyExecPlan(sqlQuery); } - @Test - public void testProject1() { + @TestTemplate + void testProject1() { // One side projection String sqlQuery = "SELECT T1.a, T1.b, T1.c, T2.c FROM MyTable T1, MyTable T2 WHERE T1.a = T2.a"; util.verifyExecPlan(sqlQuery); } - @Test - public void testProject2() { + @TestTemplate + void testProject2() { // Two side projection String sqlQuery = "SELECT T1.a, T1.b, T2.c FROM MyTable T1, MyTable T2 WHERE T1.a = T2.a"; util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectNested1() { + @TestTemplate + void testProjectNested1() { String sqlQuery = "SELECT T1.a, T1.i, T2.j FROM" + " (SELECT a, nested.i as i FROM MyTable) T1," @@ -119,8 +120,8 @@ public void testProjectNested1() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectNested2() { + @TestTemplate + void testProjectNested2() { String sqlQuery = "SELECT T1.a, T1.i, T2.i FROM" + " (SELECT a, nested.i as i FROM MyTable) T1," @@ -128,8 +129,8 @@ public void testProjectNested2() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectNestedWithWholeField() { + @TestTemplate + void testProjectNestedWithWholeField() { String sqlQuery = "SELECT * FROM" + " (SELECT a, nested.i FROM MyTable) T1," @@ -137,16 +138,16 @@ public void testProjectNestedWithWholeField() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithExpr() { + @TestTemplate + void testProjectWithExpr() { String sqlQuery = "SELECT T1.a, T1.b, T2.c FROM" + " (SELECT a, b + 1 as b FROM MyTable) T1, MyTable T2 WHERE T1.a = T2.a"; util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithFilter() { + @TestTemplate + void testProjectWithFilter() { String sqlQuery = "SELECT T1.a, T1.b, T2.c FROM" + " (SELECT * FROM MyTable WHERE b = 2) T1," @@ -154,8 +155,8 @@ public void testProjectWithFilter() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithMeta1() { + @TestTemplate + void testProjectWithMeta1() { // One side meta String sqlQuery = "SELECT T1.a, T1.b, T1.metadata_1, T1.metadata_2, T2.c, T2.metadata_2" @@ -163,8 +164,8 @@ public void testProjectWithMeta1() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithMeta2() { + @TestTemplate + void testProjectWithMeta2() { // One side meta String sqlQuery = "SELECT T1.a, T1.b, T1.metadata_1, T2.c, T2.metadata_2" @@ -172,8 +173,8 @@ public void testProjectWithMeta2() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithMeta3() { + @TestTemplate + void testProjectWithMeta3() { // meta projection String sqlQuery = "SELECT T1.a, T1.b, T1.metadata_1, T2.c, T2.metadata_1" @@ -181,16 +182,16 @@ public void testProjectWithMeta3() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithMetaAndCompute() { + @TestTemplate + void testProjectWithMetaAndCompute() { String sqlQuery = "SELECT T1.a, T1.b, T1.metadata_1, T1.compute_metadata, T2.c, T2.metadata_2" + " FROM MyTable T1, MyTable T2 WHERE T1.a = T2.a"; util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithHints() { + @TestTemplate + void testProjectWithHints() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " MyTable /*+ OPTIONS('source.num-element-to-skip'='1') */ T1," @@ -199,8 +200,8 @@ public void testProjectWithHints() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectReuseWithHints() { + @TestTemplate + void testProjectReuseWithHints() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " MyTable /*+ OPTIONS('source.num-element-to-skip'='1') */ T1," @@ -209,8 +210,8 @@ public void testProjectReuseWithHints() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithDifferentHints() { + @TestTemplate + void testProjectWithDifferentHints() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " MyTable /*+ OPTIONS('source.num-element-to-skip'='1') */ T1," @@ -219,8 +220,8 @@ public void testProjectWithDifferentHints() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithFilterPushDown() { + @TestTemplate + void testProjectWithFilterPushDown() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " (SELECT * FROM" @@ -231,8 +232,8 @@ public void testProjectWithFilterPushDown() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectReuseWithFilterPushDown() { + @TestTemplate + void testProjectReuseWithFilterPushDown() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " (SELECT * FROM" @@ -243,8 +244,8 @@ public void testProjectReuseWithFilterPushDown() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectReuseWithWatermark() { + @TestTemplate + void testProjectReuseWithWatermark() { if (isStreaming) { String ddl = "CREATE TABLE W_T (\n" @@ -271,8 +272,8 @@ public void testProjectReuseWithWatermark() { } } - @Test - public void testProjectWithLimitPushDown() { + @TestTemplate + void testProjectWithLimitPushDown() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " (SELECT * FROM" @@ -283,8 +284,8 @@ public void testProjectWithLimitPushDown() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectReuseWithLimitPushDown() { + @TestTemplate + void testProjectReuseWithLimitPushDown() { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" + " (SELECT * FROM" @@ -295,8 +296,8 @@ public void testProjectReuseWithLimitPushDown() { util.verifyExecPlan(sqlQuery); } - @Test - public void testProjectWithPartitionPushDown() { + @TestTemplate + void testProjectWithPartitionPushDown() { if (!isStreaming) { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" @@ -311,8 +312,8 @@ public void testProjectWithPartitionPushDown() { } } - @Test - public void testProjectReuseWithPartitionPushDown() { + @TestTemplate + void testProjectReuseWithPartitionPushDown() { if (!isStreaming) { String sqlQuery = "SELECT T1.a, T1.c, T2.c FROM" @@ -327,9 +328,9 @@ public void testProjectReuseWithPartitionPushDown() { } } - @Test - public void testReuseWithReadMetadataAndWatermarkPushDown1() { - assumeTrue(isStreaming); + @TestTemplate + void testReuseWithReadMetadataAndWatermarkPushDown1() { + assumeThat(isStreaming).isTrue(); String ddl = "CREATE TABLE MyTable1 (\n" + " metadata_0 int METADATA VIRTUAL,\n" @@ -364,9 +365,9 @@ public void testReuseWithReadMetadataAndWatermarkPushDown1() { util.verifyExecPlan(sqlQuery); } - @Test - public void testReuseWithReadMetadataAndWatermarkPushDown2() { - assumeTrue(isStreaming); + @TestTemplate + void testReuseWithReadMetadataAndWatermarkPushDown2() { + assumeThat(isStreaming).isTrue(); String ddl = "CREATE TABLE MyTable1 (\n" + " metadata_0 int METADATA VIRTUAL,\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java index dd776667a155d..9eae51af4120f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java @@ -27,20 +27,20 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Tests for rules that extend {@link FlinkDynamicPartitionPruningProgram} to create {@link * org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalDynamicFilteringTableSourceScan}. */ -public class DynamicPartitionPruningProgramTest extends TableTestBase { +class DynamicPartitionPruningProgramTest extends TableTestBase { private final BatchTableTestUtil util = batchTestUtil(TableConfig.getDefault()); private final TestValuesCatalog catalog = new TestValuesCatalog("testCatalog", "test_database", true); - @Before - public void setup() { + @BeforeEach + void setup() { catalog.open(); util.tableEnv().registerCatalog("testCatalog", catalog); util.tableEnv().useCatalog("testCatalog"); @@ -82,7 +82,7 @@ public void setup() { } @Test - public void testDimTableFilteringFieldsNotInJoinKey() { + void testDimTableFilteringFieldsNotInJoinKey() { // fact_part.id not in dynamic-filtering-fields, so dynamic partition pruning will not // succeed. String query = @@ -91,7 +91,7 @@ public void testDimTableFilteringFieldsNotInJoinKey() { } @Test - public void testDimTableWithoutFilter() { + void testDimTableWithoutFilter() { // If dim side without filters, dynamic partition pruning will not succeed. String query = "Select * from dim, fact_part where fact_part.fact_date_sk = dim.dim_date_sk" @@ -100,7 +100,7 @@ public void testDimTableWithoutFilter() { } @Test - public void testDimTableWithUnsuitableFilter() { + void testDimTableWithUnsuitableFilter() { // For filters in dim table side, they need to filter enough partitions. Like NOT NULL will // not succeed for dynamic partition pruning. String query = @@ -109,7 +109,7 @@ public void testDimTableWithUnsuitableFilter() { } @Test - public void testFactTableIsNotPartitionTable() { + void testFactTableIsNotPartitionTable() { // non-partition fact table. Dynamic partition pruning will not succeed if fact side is not // partition table. util.tableEnv() @@ -134,7 +134,7 @@ public void testFactTableIsNotPartitionTable() { } @Test - public void testFactTableIsLegacySource() { + void testFactTableIsLegacySource() { util.tableEnv() .executeSql( "CREATE TABLE legacy_source (\n" @@ -158,7 +158,7 @@ public void testFactTableIsLegacySource() { } @Test - public void testDimTableWithFilterPushDown() { + void testDimTableWithFilterPushDown() { // Even though have filter push down, dynamic partition pruning will succeed. String query = "Select * from fact_part join (Select * from dim) t1" @@ -167,7 +167,7 @@ public void testDimTableWithFilterPushDown() { } @Test - public void testJoinKeyIsDynamicFilterFieldNotPartitionKey() { + void testJoinKeyIsDynamicFilterFieldNotPartitionKey() { // Not only partition key, but also dynamic filtering field in join key will succeed in // dynamic partition pruning. String query = @@ -176,7 +176,7 @@ public void testJoinKeyIsDynamicFilterFieldNotPartitionKey() { } @Test - public void testDynamicFilteringFactInRightRule() throws TableNotExistException { + void testDynamicFilteringFactInRightRule() throws TableNotExistException { // Base rule. CatalogTableStatistics tableStatistics = new CatalogTableStatistics(1, 1, 1, 1); catalog.alterTableStatistics( @@ -187,7 +187,7 @@ public void testDynamicFilteringFactInRightRule() throws TableNotExistException } @Test - public void testDynamicFilteringFactInLeftRule() throws TableNotExistException { + void testDynamicFilteringFactInLeftRule() throws TableNotExistException { // Base rule. CatalogTableStatistics tableStatistics = new CatalogTableStatistics(1, 1, 1, 1); catalog.alterTableStatistics( @@ -198,7 +198,7 @@ public void testDynamicFilteringFactInLeftRule() throws TableNotExistException { } @Test - public void testDynamicFilteringFactInRightWithExchangeRule() { + void testDynamicFilteringFactInRightWithExchangeRule() { // Base rule. String query = "Select * from dim, fact_part where fact_part.fact_date_sk = dim.dim_date_sk and dim.price < 500"; @@ -206,7 +206,7 @@ public void testDynamicFilteringFactInRightWithExchangeRule() { } @Test - public void testDynamicFilteringFactInLeftWithExchangeRule() { + void testDynamicFilteringFactInLeftWithExchangeRule() { // Base rule. String query = "Select * from fact_part, dim where fact_part.fact_date_sk = dim.dim_date_sk and dim.price < 500"; @@ -214,7 +214,7 @@ public void testDynamicFilteringFactInLeftWithExchangeRule() { } @Test - public void testDynamicFilteringFactInRightWithCalcRule() throws TableNotExistException { + void testDynamicFilteringFactInRightWithCalcRule() throws TableNotExistException { // Base rule. CatalogTableStatistics tableStatistics = new CatalogTableStatistics(1, 1, 1, 1); catalog.alterTableStatistics( @@ -226,7 +226,7 @@ public void testDynamicFilteringFactInRightWithCalcRule() throws TableNotExistEx } @Test - public void testDynamicFilteringFactInLeftWithCalcRule() throws TableNotExistException { + void testDynamicFilteringFactInLeftWithCalcRule() throws TableNotExistException { // Base rule. CatalogTableStatistics tableStatistics = new CatalogTableStatistics(1, 1, 1, 1); catalog.alterTableStatistics( @@ -238,7 +238,7 @@ public void testDynamicFilteringFactInLeftWithCalcRule() throws TableNotExistExc } @Test - public void testDynamicFilteringFactInRightWithExchangeAndCalcRule() { + void testDynamicFilteringFactInRightWithExchangeAndCalcRule() { // Base rule. String query = "Select * from dim, fact_part where fact_part.fact_date_sk = dim.dim_date_sk" @@ -247,7 +247,7 @@ public void testDynamicFilteringFactInRightWithExchangeAndCalcRule() { } @Test - public void testDynamicFilteringFactInLeftWithExchangeAndCalcRule() { + void testDynamicFilteringFactInLeftWithExchangeAndCalcRule() { // Base rule. String query = "Select * from fact_part, dim where fact_part.fact_date_sk = dim.dim_date_sk" @@ -256,7 +256,7 @@ public void testDynamicFilteringFactInLeftWithExchangeAndCalcRule() { } @Test - public void testComplexCalcInFactSide() { + void testComplexCalcInFactSide() { // Although the partition key is converted, Dynamic Partition pruning can be successfully // applied. String query = @@ -266,7 +266,7 @@ public void testComplexCalcInFactSide() { } @Test - public void testPartitionKeysIsComputeColumnsInFactSide() { + void testPartitionKeysIsComputeColumnsInFactSide() { // Dynamic filtering will not succeed for this query. String query = "Select * from dim join (select fact_date_sk + 1 as fact_date_sk1, price + 1 as price1 from fact_part) t1" @@ -275,7 +275,7 @@ public void testPartitionKeysIsComputeColumnsInFactSide() { } @Test - public void testPartitionKeysOrderIsChangedInFactSide() { + void testPartitionKeysOrderIsChangedInFactSide() { // Dynamic filtering will succeed for this query. String query = "Select * from dim join (select fact_date_sk, id, name, amount, price from fact_part) t1" @@ -284,7 +284,7 @@ public void testPartitionKeysOrderIsChangedInFactSide() { } @Test - public void testPartitionKeysNameIsChangedInFactSide() { + void testPartitionKeysNameIsChangedInFactSide() { // Dynamic filtering will succeed for this query. String query = "Select * from dim join (select id, name, amount, price, fact_date_sk as fact_date_sk1 from fact_part) t1" @@ -293,8 +293,7 @@ public void testPartitionKeysNameIsChangedInFactSide() { } @Test - public void testDynamicFilteringFieldIsComputeColumnsInFactSide() - throws TableNotExistException { + void testDynamicFilteringFieldIsComputeColumnsInFactSide() throws TableNotExistException { CatalogTableStatistics tableStatistics = new CatalogTableStatistics(1, 1, 1, 1); catalog.alterTableStatistics( new ObjectPath("test_database", "dim"), tableStatistics, false); @@ -306,7 +305,7 @@ public void testDynamicFilteringFieldIsComputeColumnsInFactSide() } @Test - public void testLeftOuterJoinWithFactInLeft() { + void testLeftOuterJoinWithFactInLeft() { // left outer join with fact in left will not succeed. Because if fact in left, filtering // condition is useless. String query = @@ -316,7 +315,7 @@ public void testLeftOuterJoinWithFactInLeft() { } @Test - public void testLeftOutJoinWithFactInRight() { + void testLeftOutJoinWithFactInRight() { // left outer join with fact in right will succeed. String query = "Select * from dim left outer join fact_part on fact_part.fact_date_sk = dim.dim_date_sk" @@ -325,7 +324,7 @@ public void testLeftOutJoinWithFactInRight() { } @Test - public void testSemiJoin() { + void testSemiJoin() { // Now dynamic partition pruning support semi join, this query will succeed. String query = "Select * from fact_part where fact_part.fact_date_sk in" @@ -334,7 +333,7 @@ public void testSemiJoin() { } @Test - public void testFullOuterJoin() { + void testFullOuterJoin() { // Now dynamic partition pruning don't support full outer join. String query = "Select * from fact_part full outer join" @@ -343,7 +342,7 @@ public void testFullOuterJoin() { } @Test - public void testAntiJoin() { + void testAntiJoin() { // Now dynamic partition prune don't support anti join. String query = "Select * from fact_part where not exists" @@ -352,7 +351,7 @@ public void testAntiJoin() { } @Test - public void testMultiJoin() { + void testMultiJoin() { // Another table. util.tableEnv() .executeSql( @@ -372,7 +371,7 @@ public void testMultiJoin() { } @Test - public void testComplexDimSideWithJoinInDimSide() { + void testComplexDimSideWithJoinInDimSide() { // TODO, Dpp will not success with complex dim side. util.tableEnv() .executeSql( @@ -405,7 +404,7 @@ public void testComplexDimSideWithJoinInDimSide() { } @Test - public void testComplexDimSideWithAggInDimSide() { + void testComplexDimSideWithAggInDimSide() { // Dim side contains agg will not succeed in this version, it will improve later. util.tableEnv() .executeSql( @@ -427,7 +426,7 @@ public void testComplexDimSideWithAggInDimSide() { } @Test - public void testDppWithoutJoinReorder() { + void testDppWithoutJoinReorder() { // Dpp will success String ddl = "CREATE TABLE test_database.item (\n" @@ -453,7 +452,7 @@ public void testDppWithoutJoinReorder() { } @Test - public void testDppWithSubQuery() { + void testDppWithSubQuery() { // Dpp will success String ddl = "CREATE TABLE test_database.item (\n" @@ -478,7 +477,7 @@ public void testDppWithSubQuery() { } @Test - public void testDppWithUnionInFactSide() { + void testDppWithUnionInFactSide() { // Dpp will success. String ddl = "CREATE TABLE test_database.item (\n" @@ -501,7 +500,7 @@ public void testDppWithUnionInFactSide() { } @Test - public void testDppWithAggInFactSideAndJoinKeyInGrouping() { + void testDppWithAggInFactSideAndJoinKeyInGrouping() { // Dpp will success String ddl = "CREATE TABLE test_database.item (\n" @@ -522,7 +521,7 @@ public void testDppWithAggInFactSideAndJoinKeyInGrouping() { } @Test - public void testDppWithAggInFactSideAndJoinKeyInGroupFunction() { + void testDppWithAggInFactSideAndJoinKeyInGroupFunction() { // Dpp will not success because join key in group function. String ddl = "CREATE TABLE test_database.item (\n" @@ -544,7 +543,7 @@ public void testDppWithAggInFactSideAndJoinKeyInGroupFunction() { } @Test - public void testDppWithAggInFactSideWithAggPushDownEnable() { + void testDppWithAggInFactSideWithAggPushDownEnable() { // Dpp will not success while fact side source support agg push down and source agg push // down enabled is true. String ddl = @@ -566,7 +565,7 @@ public void testDppWithAggInFactSideWithAggPushDownEnable() { } @Test - public void testDppWithAggInFactSideWithAggPushDownDisable() { + void testDppWithAggInFactSideWithAggPushDownDisable() { // Dpp will success while fact side source support agg push down but source agg push down // enabled is false. TableConfig tableConfig = util.tableEnv().getConfig(); @@ -593,7 +592,7 @@ public void testDppWithAggInFactSideWithAggPushDownDisable() { } @Test - public void testDPPWithFactSideJoinKeyChanged() { + void testDPPWithFactSideJoinKeyChanged() { // If partition keys changed in fact side. DPP factor will not success. String ddl = "CREATE TABLE test_database.item (\n" @@ -615,7 +614,7 @@ public void testDPPWithFactSideJoinKeyChanged() { } @Test - public void testDPPWithDimSideJoinKeyChanged() { + void testDPPWithDimSideJoinKeyChanged() { // Although partition keys changed in dim side. DPP will success. String ddl = "CREATE TABLE test_database.item (\n" @@ -637,7 +636,7 @@ public void testDPPWithDimSideJoinKeyChanged() { } @Test - public void testDPPWithJoinKeysNotIncludePartitionKeys() { + void testDPPWithJoinKeysNotIncludePartitionKeys() { // If join keys of partition table join with dim table not include partition keys, dpp will // not success. String ddl = @@ -661,7 +660,7 @@ public void testDPPWithJoinKeysNotIncludePartitionKeys() { } @Test - public void testDppFactSideCannotReuseWithSameCommonSource() { + void testDppFactSideCannotReuseWithSameCommonSource() { String query = "SELECT * FROM(\n" + " Select fact_part.id, fact_part.price, fact_part.amount from fact_part join (Select * from dim) t1" @@ -671,7 +670,7 @@ public void testDppFactSideCannotReuseWithSameCommonSource() { } @Test - public void testDimSideReuseAfterProjectionPushdown() { + void testDimSideReuseAfterProjectionPushdown() { util.tableEnv() .executeSql( "CREATE TABLE fact_part2 (\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/FlinkRuntimeFilterProgramTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/FlinkRuntimeFilterProgramTest.java index 217ca5577c307..8d01b4cace0ad 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/FlinkRuntimeFilterProgramTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/FlinkRuntimeFilterProgramTest.java @@ -31,8 +31,8 @@ import org.apache.flink.table.planner.utils.BatchTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; @@ -48,8 +48,8 @@ public class FlinkRuntimeFilterProgramTest extends TableTestBase { private final TestValuesCatalog catalog = new TestValuesCatalog("testCatalog", "test_database", true); - @Before - public void setup() { + @BeforeEach + void setup() { catalog.open(); util.tableEnv().registerCatalog("testCatalog", catalog); util.tableEnv().useCatalog("testCatalog"); @@ -91,7 +91,7 @@ public void setup() { } @Test - public void testSimpleInnerJoin() throws Exception { + void testSimpleInnerJoin() throws Exception { // runtime filter will succeed setupSuitableTableStatistics(); String query = "select * from fact, dim where fact.amount = dim.amount and dim.price < 500"; @@ -99,7 +99,7 @@ public void testSimpleInnerJoin() throws Exception { } @Test - public void testSemiJoin() throws Exception { + void testSemiJoin() throws Exception { // runtime filter will succeed setupSuitableTableStatistics(); util.getTableEnv() @@ -111,7 +111,7 @@ public void testSemiJoin() throws Exception { } @Test - public void testLeftOuterJoinWithLeftBuild() throws Exception { + void testLeftOuterJoinWithLeftBuild() throws Exception { // runtime filter will succeed setupSuitableTableStatistics(); String query = @@ -120,7 +120,7 @@ public void testLeftOuterJoinWithLeftBuild() throws Exception { } @Test - public void testLeftOuterJoinWithRightBuild() throws Exception { + void testLeftOuterJoinWithRightBuild() throws Exception { // runtime filter will not succeed setupSuitableTableStatistics(); String query = @@ -129,7 +129,7 @@ public void testLeftOuterJoinWithRightBuild() throws Exception { } @Test - public void testFullOuterJoin() throws Exception { + void testFullOuterJoin() throws Exception { // runtime filter will not succeed setupSuitableTableStatistics(); String query = @@ -138,7 +138,7 @@ public void testFullOuterJoin() throws Exception { } @Test - public void testAntiJoin() throws Exception { + void testAntiJoin() throws Exception { // runtime filter will not succeed setupSuitableTableStatistics(); String query = @@ -147,7 +147,7 @@ public void testAntiJoin() throws Exception { } @Test - public void testNestedLoopJoin() throws Exception { + void testNestedLoopJoin() throws Exception { // runtime filter will not succeed setupTableRowCount("dim", 1L); setupTableRowCount("fact", SUITABLE_FACT_ROW_COUNT); @@ -156,7 +156,7 @@ public void testNestedLoopJoin() throws Exception { } @Test - public void testProbeSideIsTooSmall() throws Exception { + void testProbeSideIsTooSmall() throws Exception { // runtime filter will not succeed setupTableRowCount("dim", SUITABLE_DIM_ROW_COUNT); // fact is 7.5 GB < 10 GB @@ -166,7 +166,7 @@ public void testProbeSideIsTooSmall() throws Exception { } @Test - public void testBuildSideIsTooLarge() throws Exception { + void testBuildSideIsTooLarge() throws Exception { // runtime filter will not succeed // dim is 48 MB > 6MB setupTableRowCount("dim", 1024L * 1024L); @@ -176,7 +176,7 @@ public void testBuildSideIsTooLarge() throws Exception { } @Test - public void testFilterRatioIsTooSmall() throws Exception { + void testFilterRatioIsTooSmall() throws Exception { // runtime filter will not succeed setupSuitableTableStatistics(); setupTableColumnNdv("dim", "amount", 768L); @@ -186,7 +186,7 @@ public void testFilterRatioIsTooSmall() throws Exception { } @Test - public void testBuildSideIsJoinWithoutExchange() throws Exception { + void testBuildSideIsJoinWithoutExchange() throws Exception { // runtime filter will succeed setupSuitableTableStatistics(); util.tableEnv() @@ -209,7 +209,7 @@ public void testBuildSideIsJoinWithoutExchange() throws Exception { } @Test - public void testBuildSideIsJoinWithTwoAggInputs() throws Exception { + void testBuildSideIsJoinWithTwoAggInputs() throws Exception { // runtime filter will succeed setupSuitableTableStatistics(); util.tableEnv() @@ -241,7 +241,7 @@ public void testBuildSideIsJoinWithTwoAggInputs() throws Exception { } @Test - public void testBuildSideIsLeftJoinWithoutExchange() throws Exception { + void testBuildSideIsLeftJoinWithoutExchange() throws Exception { // runtime filter will not succeed, because the original build side is left join(without // exchange), so we can only push builder to it's left input, but the left input is too // large to as builder. @@ -267,7 +267,7 @@ public void testBuildSideIsLeftJoinWithoutExchange() throws Exception { } @Test - public void testBuildSideIsAggWithoutExchange() throws Exception { + void testBuildSideIsAggWithoutExchange() throws Exception { // runtime filter will succeed // The following two config are used to let the build side is a direct Agg (without // Exchange) @@ -288,7 +288,7 @@ public void testBuildSideIsAggWithoutExchange() throws Exception { } @Test - public void testBuildSideIsCalcWithoutExchange() throws Exception { + void testBuildSideIsCalcWithoutExchange() throws Exception { // runtime filter will succeed // The following two config are used to let the build side is a direct Calc (without // Exchange) @@ -308,7 +308,7 @@ public void testBuildSideIsCalcWithoutExchange() throws Exception { } @Test - public void testCannotInjectMoreThanOneRuntimeFilterInSamePlace() throws Exception { + void testCannotInjectMoreThanOneRuntimeFilterInSamePlace() throws Exception { setupSuitableTableStatistics(); util.tableEnv() .executeSql( @@ -330,7 +330,7 @@ public void testCannotInjectMoreThanOneRuntimeFilterInSamePlace() throws Excepti } @Test - public void testPushDownProbeSideWithCalc() throws Exception { + void testPushDownProbeSideWithCalc() throws Exception { setupSuitableTableStatistics(); String query = "select * from dim, fact where dim.amount = fact.amount and dim.price < 500 and fact.price > 600"; @@ -338,7 +338,7 @@ public void testPushDownProbeSideWithCalc() throws Exception { } @Test - public void testCannotPushDownProbeSideWithCalc() throws Exception { + void testCannotPushDownProbeSideWithCalc() throws Exception { setupSuitableTableStatistics(); String query = "select * from dim inner join (select fact_date_sk, RAND(10) as random from fact) " @@ -347,7 +347,7 @@ public void testCannotPushDownProbeSideWithCalc() throws Exception { } @Test - public void testPushDownProbeSideToAllInputsOfJoin() throws Exception { + void testPushDownProbeSideToAllInputsOfJoin() throws Exception { setupSuitableTableStatistics(); util.tableEnv() .executeSql( @@ -369,7 +369,7 @@ public void testPushDownProbeSideToAllInputsOfJoin() throws Exception { } @Test - public void testPushDownProbeSideToOneInputOfJoin() throws Exception { + void testPushDownProbeSideToOneInputOfJoin() throws Exception { setupSuitableTableStatistics(); util.tableEnv() .executeSql( @@ -391,7 +391,7 @@ public void testPushDownProbeSideToOneInputOfJoin() throws Exception { } @Test - public void testCannotPushDownProbeSideWithJoin() throws Exception { + void testCannotPushDownProbeSideWithJoin() throws Exception { setupSuitableTableStatistics(); util.tableEnv() .executeSql( @@ -413,7 +413,7 @@ public void testCannotPushDownProbeSideWithJoin() throws Exception { } @Test - public void testPushDownProbeSideWithAgg() throws Exception { + void testPushDownProbeSideWithAgg() throws Exception { setupTableRowCount("dim", SUITABLE_DIM_ROW_COUNT); setupTableRowCount("fact", 1024L * SUITABLE_FACT_ROW_COUNT); util.getTableEnv() @@ -427,7 +427,7 @@ public void testPushDownProbeSideWithAgg() throws Exception { } @Test - public void testCannotPushDownProbeSideWithAgg() throws Exception { + void testCannotPushDownProbeSideWithAgg() throws Exception { setupTableRowCount("dim", SUITABLE_DIM_ROW_COUNT); setupTableRowCount("fact", 1024L * SUITABLE_FACT_ROW_COUNT); util.getTableEnv() @@ -441,7 +441,7 @@ public void testCannotPushDownProbeSideWithAgg() throws Exception { } @Test - public void testPushDownProbeSideWithUnion() throws Exception { + void testPushDownProbeSideWithUnion() throws Exception { // probe side will be pushed down to union. setupSuitableTableStatistics(); String query = @@ -453,7 +453,7 @@ public void testPushDownProbeSideWithUnion() throws Exception { } @Test - public void testDoesNotApplyRuntimeFilterAndDPPOnSameKey() throws Exception { + void testDoesNotApplyRuntimeFilterAndDPPOnSameKey() throws Exception { // runtime filter will not success, because already applied DPP on the key setupTableRowCount("dim", SUITABLE_DIM_ROW_COUNT); createPartitionedFactTable(SUITABLE_FACT_ROW_COUNT); @@ -463,7 +463,7 @@ public void testDoesNotApplyRuntimeFilterAndDPPOnSameKey() throws Exception { } @Test - public void testProbeSideIsTableSourceWithoutExchange() throws Exception { + void testProbeSideIsTableSourceWithoutExchange() throws Exception { // runtime filter will not succeed, because probe side is a direct table source setupSuitableTableStatistics(); String query = "select * from fact, dim where fact.amount = dim.amount and dim.price = 500"; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java index 4ec5f93f6cca9..7eb96ba0193c5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java @@ -31,16 +31,16 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test for {@link FlinkFilterJoinRule}. */ -public class FlinkFilterJoinRuleTest extends TableTestBase { +class FlinkFilterJoinRuleTest extends TableTestBase { private BatchTableTestUtil util; - @Before - public void setup() { + @BeforeEach + void setup() { util = batchTestUtil(TableConfig.getDefault()); util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE()); CalciteConfig calciteConfig = @@ -102,256 +102,256 @@ public void setup() { } @Test - public void testFilterPushDownLeftSemi1() { + void testFilterPushDownLeftSemi1() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT)) T WHERE T.b > 2"); } @Test - public void testFilterPushDownLeftSemi2() { + void testFilterPushDownLeftSemi2() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT)) T WHERE T.b > 2"); } @Test - public void testFilterPushDownLeftSemi3() { + void testFilterPushDownLeftSemi3() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2"); } @Test - public void testJoinConditionPushDownLeftSemi1() { + void testJoinConditionPushDownLeftSemi1() { util.verifyRelPlan("SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT WHERE b > 2)"); } @Test - public void testJoinConditionPushDownLeftSemi2() { + void testJoinConditionPushDownLeftSemi2() { util.verifyRelPlan("SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE b > 2)"); } @Test - public void testJoinConditionPushDownLeftSemi3() { + void testJoinConditionPushDownLeftSemi3() { util.verifyRelPlan( "SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)"); } @Test - public void testFilterPushDownLeftAnti1() { + void testFilterPushDownLeftAnti1() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE c < 3)) T WHERE T.b > 2"); } @Test - public void testFilterPushDownLeftAnti2() { + void testFilterPushDownLeftAnti2() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT where c > 10)) T WHERE T.b > 2"); } @Test - public void testFilterPushDownLeftAnti3() { + void testFilterPushDownLeftAnti3() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND c < 3)) T WHERE T.b > 2"); } @Test - public void testFilterPushDownLeftAnti4() { + void testFilterPushDownLeftAnti4() { util.verifyRelPlan( "SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2"); } @Test - public void testJoinConditionPushDownLeftAnti1() { + void testJoinConditionPushDownLeftAnti1() { util.verifyRelPlan("SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b > 2)"); } @Test - public void testJoinConditionPushDownLeftAnti2() { + void testJoinConditionPushDownLeftAnti2() { util.verifyRelPlan( "SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE b > 2)"); } @Test - public void testJoinConditionPushDownLeftAnti3() { + void testJoinConditionPushDownLeftAnti3() { util.verifyRelPlan( "SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND b > 1)"); } @Test - public void testJoinConditionPushDownLeftAnti4() { + void testJoinConditionPushDownLeftAnti4() { util.verifyRelPlan( "SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)"); } @Test - public void testInnerJoinWithAllFilterFromBothSide() { + void testInnerJoinWithAllFilterFromBothSide() { // can not be pushed down util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = a2 + 2"); } @Test - public void testInnerJoinWithAllFilterInONClause() { + void testInnerJoinWithAllFilterInONClause() { util.verifyRelPlan( "SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND a1 = 2 AND b2 > 10"); } @Test - public void testInnerJoinWithSomeFiltersFromLeftSide() { + void testInnerJoinWithSomeFiltersFromLeftSide() { util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = 2"); } @Test - public void testInnerJoinWithSomeFiltersFromRightSide() { + void testInnerJoinWithSomeFiltersFromRightSide() { util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a2 = 2"); } @Test - public void testInnerJoinWithSomeFiltersFromLeftRightSide() { + void testInnerJoinWithSomeFiltersFromLeftRightSide() { util.verifyRelPlan( "SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL"); } @Test - public void testInnerJoinWithAllFiltersFromWhere() { + void testInnerJoinWithAllFiltersFromWhere() { util.verifyRelPlan( "SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' "); } @Test - public void testInnerJoinWithNullFilter() { + void testInnerJoinWithNullFilter() { util.verifyRelPlan( "SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 IS NULL"); } @Test - public void testInnerJoinWithNullFilter2() { + void testInnerJoinWithNullFilter2() { util.verifyRelPlan( "SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 IS NULL AND a1 < 10"); } @Test - public void testInnerJoinWithFilter1() { + void testInnerJoinWithFilter1() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 < 1"); } @Test - public void testInnerJoinWithFilter2() { + void testInnerJoinWithFilter2() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 <> 1"); } @Test - public void testInnerJoinWithFilter3() { + void testInnerJoinWithFilter3() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 > 1"); } @Test - public void testInnerJoinWithFilter4() { + void testInnerJoinWithFilter4() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 >= 1"); } @Test - public void testInnerJoinWithFilter5() { + void testInnerJoinWithFilter5() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 <= 1"); } @Test - public void testInnerJoinWithFilter6() { + void testInnerJoinWithFilter6() { util.verifyRelPlan("SELECT * FROM MyTable1 INNER JOIN MyTable2 ON a1 = a2 WHERE a2 = null"); } @Test - public void testLeftJoinWithSomeFiltersFromLeftSide() { + void testLeftJoinWithSomeFiltersFromLeftSide() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a1 = 2"); } @Test - public void testLeftJoinWithAllFilterInONClause() { + void testLeftJoinWithAllFilterInONClause() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND a2 = 2"); } @Test - public void testLeftJoinWithSomeFiltersFromLeftRightSide() { + void testLeftJoinWithSomeFiltersFromLeftRightSide() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL"); } @Test - public void testLeftJoinWithAllFiltersFromWhere() { + void testLeftJoinWithAllFiltersFromWhere() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' "); } @Test - public void testLeftJoinWithNullFilterInRightSide() { + void testLeftJoinWithNullFilterInRightSide() { // Even if there is a filter 'a2 IS NULL', the 'a1 IS NULL' cannot be generated for left // join and this filter cannot be pushed down to both MyTable1 and MyTable2. util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 IS NULL"); } @Test - public void testLeftJoinWithNullFilterInRightSide2() { + void testLeftJoinWithNullFilterInRightSide2() { // 'a2 IS NULL' cannot infer that 'a1 IS NULL'. util.verifyRelPlan( "SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 IS NULL AND a1 < 10"); } @Test - public void testLeftJoinWithFilter1() { + void testLeftJoinWithFilter1() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 < 1"); } @Test - public void testLeftJoinWithFilter2() { + void testLeftJoinWithFilter2() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 <> 1"); } @Test - public void testLeftJoinWithFilter3() { + void testLeftJoinWithFilter3() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 > 1"); } @Test - public void testLeftJoinWithFilter4() { + void testLeftJoinWithFilter4() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 >= 1"); } @Test - public void testLeftJoinWithFilter5() { + void testLeftJoinWithFilter5() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 <= 1"); } @Test - public void testLeftJoinWithFilter6() { + void testLeftJoinWithFilter6() { util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a2 = null"); } @Test - public void testRightJoinWithAllFilterInONClause() { + void testRightJoinWithAllFilterInONClause() { util.verifyRelPlan("SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND a1 = 2"); } @Test - public void testRightJoinWithSomeFiltersFromRightSide() { + void testRightJoinWithSomeFiltersFromRightSide() { util.verifyRelPlan("SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 WHERE a2 = 2"); } @Test - public void testRightJoinWithSomeFiltersFromLeftRightSide() { + void testRightJoinWithSomeFiltersFromLeftRightSide() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL"); } @Test - public void testRightJoinWithAllFiltersFromWhere() { + void testRightJoinWithAllFiltersFromWhere() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' "); } @Test - public void testRightJoinWithNullFilterInLeftSide() { + void testRightJoinWithNullFilterInLeftSide() { // Even if there is a filter 'a1 IS NULL', the 'a2 IS NULL' cannot be generated for right // join and this filter cannot be pushed down to both MyTable1 and MyTable2. util.verifyRelPlan( @@ -359,7 +359,7 @@ public void testRightJoinWithNullFilterInLeftSide() { } @Test - public void testRightJoinWithNullFilterInRightSide2() { + void testRightJoinWithNullFilterInRightSide2() { // 'a1 IS NULL' cannot infer that 'a2 IS NULL'. However, 'a2 < 10' can infer that 'a1 < 10', // and both of them can be pushed down. util.verifyRelPlan( @@ -367,47 +367,47 @@ public void testRightJoinWithNullFilterInRightSide2() { } @Test - public void testFullJoinWithAllFilterInONClause() { + void testFullJoinWithAllFilterInONClause() { util.verifyRelPlan("SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND a1 = 2"); } @Test - public void testFullJoinWithSomeFiltersFromLeftSide() { + void testFullJoinWithSomeFiltersFromLeftSide() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a1 = 2"); } @Test - public void testFullJoinWithSomeFiltersFromRightSide() { + void testFullJoinWithSomeFiltersFromRightSide() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a2 = 2"); } @Test - public void testFullJoinWithSomeFiltersFromLeftRightSide() { + void testFullJoinWithSomeFiltersFromLeftRightSide() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL"); } @Test - public void testFullJoinWithAllFiltersFromWhere() { + void testFullJoinWithAllFiltersFromWhere() { // will be converted to inner join util.verifyRelPlan( "SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' "); } @Test - public void testSemiJoin() { + void testSemiJoin() { // TODO can not be pushed down now, support it later util.verifyRelPlan( "SELECT * FROM MyTable1 WHERE (a1, b1, c1) IN (SELECT a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL"); } @Test - public void testAntiJoin() { + void testAntiJoin() { // can not be pushed down util.verifyRelPlan( "SELECT * FROM MyTable1 WHERE (a1, b1, c1) NOT IN (select a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL"); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRuleTest.java index 7850bda83bd1e..7d0c3374e63ba 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRuleTest.java @@ -26,33 +26,35 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; /** Test rule {@link ProjectSnapshotTransposeRule}. */ -@RunWith(Parameterized.class) -public class ProjectSnapshotTransposeRuleTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class ProjectSnapshotTransposeRuleTest extends TableTestBase { private static final String STREAM = "stream"; private static final String BATCH = "batch"; - @Parameterized.Parameter public String mode; + @Parameter private String mode; - @Parameterized.Parameters(name = "mode = {0}") - public static Collection parameters() { + @Parameters(name = "mode = {0}") + private static Collection parameters() { return Arrays.asList(STREAM, BATCH); } private TableTestUtil util; - @Before - public void setup() { + @BeforeEach + void setup() { boolean isStreaming = STREAM.equals(mode); if (isStreaming) { util = streamTestUtil(TableConfig.getDefault()); @@ -90,8 +92,8 @@ public void setup() { tEnv.executeSql(lookup); } - @Test - public void testJoinTemporalTableWithProjectionPushDown() { + @TestTemplate + void testJoinTemporalTableWithProjectionPushDown() { String sql = "SELECT T.*, D.id\n" + "FROM MyTable AS T\n" @@ -101,8 +103,8 @@ public void testJoinTemporalTableWithProjectionPushDown() { util.verifyRelPlan(sql); } - @Test - public void testJoinTemporalTableNotProjectable() { + @TestTemplate + void testJoinTemporalTableNotProjectable() { String sql = "SELECT T.*, D.*\n" + "FROM MyTable AS T\n" @@ -112,8 +114,8 @@ public void testJoinTemporalTableNotProjectable() { util.verifyRelPlan(sql); } - @Test - public void testJoinTemporalTableWithReorderedProject() { + @TestTemplate + void testJoinTemporalTableWithReorderedProject() { String sql = "SELECT T.*, D.age, D.name, D.id\n" + "FROM MyTable AS T\n" @@ -123,8 +125,8 @@ public void testJoinTemporalTableWithReorderedProject() { util.verifyRelPlan(sql); } - @Test - public void testJoinTemporalTableWithProjectAndFilter() { + @TestTemplate + void testJoinTemporalTableWithProjectAndFilter() { String sql = "SELECT T.*, D.id\n" + "FROM MyTable AS T\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWatermarkAssignerTransposeRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWatermarkAssignerTransposeRuleTest.java index 4da8871104e4d..d8968d95ed4ed 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWatermarkAssignerTransposeRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWatermarkAssignerTransposeRuleTest.java @@ -29,15 +29,15 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.tools.RuleSets; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test for {@link ProjectWatermarkAssignerTransposeRule}. */ -public class ProjectWatermarkAssignerTransposeRuleTest extends TableTestBase { +class ProjectWatermarkAssignerTransposeRuleTest extends TableTestBase { private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); - @Before - public void setup() { + @BeforeEach + void setup() { FlinkChainedProgram program = new FlinkChainedProgram<>(); program.addLast( @@ -106,57 +106,57 @@ public void setup() { } @Test - public void simpleTranspose() { + void simpleTranspose() { util.verifyRelPlan("SELECT a, c FROM SimpleTable"); } @Test - public void transposeWithReorder() { + void transposeWithReorder() { util.verifyRelPlan("SELECT b, a FROM SimpleTable"); } @Test - public void transposeWithNestedField() { + void transposeWithNestedField() { util.verifyRelPlan("SELECT b, d.d1, d.d2 FROM SimpleTable"); } @Test - public void complicatedTranspose() { + void complicatedTranspose() { util.verifyRelPlan("SELECT d.d1, d.d2 + b FROM SimpleTable"); } @Test - public void transposeExcludeRowTime() { + void transposeExcludeRowTime() { util.verifyRelPlan("SELECT SECOND(c) FROM SimpleTable"); } @Test - public void transposeWithIncludeComputedRowTime() { + void transposeWithIncludeComputedRowTime() { util.verifyRelPlan("SELECT a, b, d FROM VirtualTable"); } @Test - public void transposeWithExcludeComputedRowTime() { + void transposeWithExcludeComputedRowTime() { util.verifyRelPlan("SELECT a, b FROM VirtualTable"); } @Test - public void transposeWithExcludeComputedRowTime2() { + void transposeWithExcludeComputedRowTime2() { util.verifyRelPlan("SELECT a, b, SECOND(d) FROM VirtualTable"); } @Test - public void transposeWithExcludeComputedRowTime3() { + void transposeWithExcludeComputedRowTime3() { util.verifyRelPlan("SELECT a, SECOND(d) FROM NestedTable"); } @Test - public void transposeWithDuplicateColumns() { + void transposeWithDuplicateColumns() { util.verifyRelPlan("SELECT a, b, b as e FROM VirtualTable"); } @Test - public void transposeWithWatermarkWithMultipleInput() { + void transposeWithWatermarkWithMultipleInput() { util.verifyRelPlan("SELECT a FROM UdfTable"); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceRuleTest.java index 9875443137623..1d929b751aea5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceRuleTest.java @@ -34,15 +34,14 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test for {@link PushFilterInCalcIntoTableSourceRuleTest}. */ -public class PushFilterInCalcIntoTableSourceRuleTest - extends PushFilterIntoTableSourceScanRuleTestBase { +class PushFilterInCalcIntoTableSourceRuleTest extends PushFilterIntoTableSourceScanRuleTestBase { - @Before - public void setup() { + @BeforeEach + void setup() { util = streamTestUtil(TableConfig.getDefault()); FlinkChainedProgram program = new FlinkChainedProgram<>(); @@ -112,12 +111,12 @@ public void setup() { } @Test - public void testFailureToPushFilterIntoSourceWithoutWatermarkPushdown() { + void testFailureToPushFilterIntoSourceWithoutWatermarkPushdown() { util.verifyRelPlan("SELECT * FROM WithWatermark WHERE LOWER(name) = 'foo'"); } @Test - public void testLowerUpperPushdown() { + void testLowerUpperPushdown() { String ddl = "CREATE TABLE MTable (\n" + " a STRING,\n" @@ -132,7 +131,7 @@ public void testLowerUpperPushdown() { } @Test - public void testWithInterval() { + void testWithInterval() { String ddl = "CREATE TABLE MTable (\n" + "a TIMESTAMP(3),\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.java index 47e03c3beb409..f4969a6fe887b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.java @@ -30,15 +30,14 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** Test for {@link PushFilterIntoTableSourceScanRule}. */ -public class PushFilterIntoTableSourceScanRuleTest - extends PushFilterIntoTableSourceScanRuleTestBase { +class PushFilterIntoTableSourceScanRuleTest extends PushFilterIntoTableSourceScanRuleTestBase { - @Before - public void setup() { + @BeforeEach + void setup() { util = batchTestUtil(TableConfig.getDefault()); ((BatchTableTestUtil) util).buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE()); CalciteConfig calciteConfig = @@ -122,7 +121,7 @@ public void setup() { } @Test - public void testLowerUpperPushdown() { + void testLowerUpperPushdown() { String ddl = "CREATE TABLE MTable (\n" + " a STRING,\n" @@ -137,7 +136,7 @@ public void testLowerUpperPushdown() { } @Test - public void testWithInterval() { + void testWithInterval() { String ddl = "CREATE TABLE MTable (\n" + "a TIMESTAMP(3),\n" @@ -153,31 +152,31 @@ public void testWithInterval() { } @Test - public void testBasicNestedFilter() { + void testBasicNestedFilter() { util.verifyRelPlan("SELECT * FROM NestedTable WHERE deepNested.nested1.`value` > 2"); } @Test - public void testNestedFilterWithDotInTheName() { + void testNestedFilterWithDotInTheName() { util.verifyRelPlan( "SELECT id FROM NestedTable WHERE `deepNestedWith.`.nested.`.value` > 5"); } @Test - public void testNestedFilterWithBacktickInTheName() { + void testNestedFilterWithBacktickInTheName() { util.verifyRelPlan( "SELECT id FROM NestedTable WHERE `deepNestedWith.`.nested.```name` = 'foo'"); } @Test - public void testNestedFilterOnMapKey() { + void testNestedFilterOnMapKey() { util.verifyRelPlan( "SELECT * FROM NestedItemTable WHERE" + " `Result`.`Mid`.data_map['item'].`value` = 3"); } @Test - public void testNestedFilterOnArrayField() { + void testNestedFilterOnArrayField() { util.verifyRelPlan( "SELECT * FROM NestedItemTable WHERE `Result`.`Mid`.data_arr[2].`value` = 3"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTestBase.java index 80629167fd1ee..435474926b65b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTestBase.java @@ -22,106 +22,106 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** Test base for testing rule which pushes filter into table source. */ -public abstract class PushFilterIntoTableSourceScanRuleTestBase extends TableTestBase { +abstract class PushFilterIntoTableSourceScanRuleTestBase extends TableTestBase { protected TableTestUtil util; @Test - public void testCanPushDown() { + void testCanPushDown() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2"); } @Test - public void testCanPushDownWithCastConstant() { + void testCanPushDownWithCastConstant() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > cast(1.1 as int)"); } @Test - public void testCanPushDownWithVirtualColumn() { + void testCanPushDownWithVirtualColumn() { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE amount > 2"); } @Test - public void testCannotPushDown() { + void testCannotPushDown() { // TestFilterableTableSource only accept predicates with `amount` util.verifyRelPlan("SELECT * FROM MyTable WHERE price > 10"); } @Test - public void testCannotPushDownWithCastConstant() { + void testCannotPushDownWithCastConstant() { // TestFilterableTableSource only accept predicates with `amount` util.verifyRelPlan("SELECT * FROM MyTable WHERE price > cast(10.1 as int)"); } @Test - public void testCannotPushDownWithVirtualColumn() { + void testCannotPushDownWithVirtualColumn() { // TestFilterableTableSource only accept predicates with `amount` util.verifyRelPlan("SELECT * FROM VirtualTable WHERE price > 10"); } @Test - public void testPartialPushDown() { + void testPartialPushDown() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2 AND price > 10"); } @Test - public void testPartialPushDownWithVirtualColumn() { + void testPartialPushDownWithVirtualColumn() { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE amount > 2 AND price > 10"); } @Test - public void testFullyPushDown() { + void testFullyPushDown() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2 AND amount < 10"); } @Test - public void testFullyPushDownWithVirtualColumn() { + void testFullyPushDownWithVirtualColumn() { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE amount > 2 AND amount < 10"); } @Test - public void testPartialPushDown2() { + void testPartialPushDown2() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2 OR price > 10"); } @Test - public void testPartialPushDown2WithVirtualColumn() { + void testPartialPushDown2WithVirtualColumn() { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE amount > 2 OR price > 10"); } @Test - public void testCannotPushDown3() { + void testCannotPushDown3() { util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2 OR amount < 10"); } @Test - public void testCannotPushDown3WithVirtualColumn() { + void testCannotPushDown3WithVirtualColumn() { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE amount > 2 OR amount < 10"); } @Test - public void testUnconvertedExpression() { + void testUnconvertedExpression() { util.verifyRelPlan( "SELECT * FROM MyTable WHERE\n" + " amount > 2 AND id < 100 AND CAST(amount AS BIGINT) > 10"); } @Test - public void testWithUdf() { + void testWithUdf() { util.addFunction("myUdf", Func1$.MODULE$); util.verifyRelPlan("SELECT * FROM MyTable WHERE amount > 2 AND myUdf(amount) < 32"); } @Test - public void testLowerUpperPushdown() { + void testLowerUpperPushdown() { util.verifyRelPlan("SELECT * FROM MTable WHERE LOWER(a) = 'foo' AND UPPER(b) = 'bar'"); } @Test - public void testWithInterval() { + void testWithInterval() { util.verifyRelPlan( "SELECT * FROM MTable\n" + "WHERE TIMESTAMPADD(HOUR, 5, a) >= b\n" @@ -130,7 +130,7 @@ public void testWithInterval() { } @Test - public void testCannotPushDownIn() { + void testCannotPushDownIn() { // this test is to avoid filter push down rules throwing exceptions // when dealing with IN expressions, this is because Filter in calcite // requires its condition to be "flat" @@ -138,7 +138,7 @@ public void testCannotPushDownIn() { } @Test - public void testWithNullLiteral() { + void testWithNullLiteral() { util.verifyRelPlan( "WITH MyView AS (SELECT CASE\n" + " WHEN amount > 0 THEN name\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRuleTest.java index 3fb88c1cbfe6f..671457a670221 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRuleTest.java @@ -30,10 +30,12 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; +import org.junit.jupiter.api.BeforeEach; /** Test for {@link PushLimitIntoTableSourceScanRule}. */ -public class PushLimitIntoTableSourceScanRuleTest - extends PushLimitIntoLegacyTableSourceScanRuleTest { +class PushLimitIntoTableSourceScanRuleTest extends PushLimitIntoLegacyTableSourceScanRuleTest { + + @BeforeEach @Override public void setup() { util().buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE()); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.java index 355374f3d8d5c..613f6c75adf84 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.java @@ -33,7 +33,8 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; import java.util.Arrays; import java.util.HashMap; @@ -41,13 +42,13 @@ import java.util.Map; /** Test for {@link PushPartitionIntoTableSourceScanRule}. */ -public class PushPartitionIntoTableSourceScanRuleTest +class PushPartitionIntoTableSourceScanRuleTest extends PushPartitionIntoLegacyTableSourceScanRuleTest { - public PushPartitionIntoTableSourceScanRuleTest( - boolean sourceFetchPartitions, boolean useFilter) { + PushPartitionIntoTableSourceScanRuleTest(boolean sourceFetchPartitions, boolean useFilter) { super(sourceFetchPartitions, useFilter); } + @BeforeEach @Override public void setup() throws Exception { util().buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE()); @@ -147,8 +148,8 @@ public void setup() throws Exception { } } - @Test - public void testUnconvertedExpression() { + @TestTemplate + void testUnconvertedExpression() { String sql = "select * from MyTable where trim(part1) = 'A' and part2 > 1"; util().verifyRelPlan(sql); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java index 9d16e67c7c513..1349cf2cf1c2c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java @@ -31,14 +31,15 @@ import org.apache.flink.table.planner.plan.optimize.program.HEP_RULES_EXECUTION_TYPE; import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.types.DataType; -import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.testutils.junit.SharedObjectsExtension; import org.apache.flink.testutils.junit.SharedReference; import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import java.util.ArrayList; import java.util.Arrays; @@ -51,11 +52,12 @@ import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link PushProjectIntoTableSourceScanRule}. */ -public class PushProjectIntoTableSourceScanRuleTest - extends PushProjectIntoLegacyTableSourceScanRuleTest { +class PushProjectIntoTableSourceScanRuleTest extends PushProjectIntoLegacyTableSourceScanRuleTest { - @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + @RegisterExtension + private final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); + @BeforeEach @Override public void setup() { util().buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE()); @@ -177,7 +179,7 @@ public void setup() { } @Test - public void testProjectWithMapType() { + void testProjectWithMapType() { String sqlQuery = "SELECT id, testMap['e']\n" + "FROM NestedTable"; util().verifyRelPlan(sqlQuery); } @@ -196,7 +198,7 @@ public void testNestedProject() { } @Test - public void testComplicatedNestedProject() { + void testComplicatedNestedProject() { String sqlQuery = "SELECT id," + " deepNested.nested1.name AS nestedName,\n" @@ -206,14 +208,14 @@ public void testComplicatedNestedProject() { } @Test - public void testProjectWithDuplicateMetadataKey() { + void testProjectWithDuplicateMetadataKey() { String sqlQuery = "SELECT id, metadata_3, metadata_1 FROM MetadataTable"; util().verifyRelPlan(sqlQuery); } @Test - public void testNestProjectWithMetadata() { + void testNestProjectWithMetadata() { String sqlQuery = "SELECT id," + " deepNested.nested1 AS nested1,\n" @@ -224,7 +226,7 @@ public void testNestProjectWithMetadata() { } @Test - public void testNestProjectWithUpsertSource() { + void testNestProjectWithUpsertSource() { String sqlQuery = "SELECT id," + " deepNested.nested1 AS nested1,\n" @@ -235,7 +237,7 @@ public void testNestProjectWithUpsertSource() { } @Test - public void testNestedProjectFieldAccessWithITEM() { + void testNestedProjectFieldAccessWithITEM() { util().verifyRelPlan( "SELECT " + "`Result`.`Mid`.data_arr[ID].`value`, " @@ -244,7 +246,7 @@ public void testNestedProjectFieldAccessWithITEM() { } @Test - public void testNestedProjectFieldAccessWithITEMWithConstantIndex() { + void testNestedProjectFieldAccessWithITEMWithConstantIndex() { util().verifyRelPlan( "SELECT " + "`Result`.`Mid`.data_arr[2].`value`, " @@ -253,7 +255,7 @@ public void testNestedProjectFieldAccessWithITEMWithConstantIndex() { } @Test - public void testNestedProjectFieldAccessWithITEMContainsTopLevelAccess() { + void testNestedProjectFieldAccessWithITEMContainsTopLevelAccess() { util().verifyRelPlan( "SELECT " + "`Result`.`Mid`.data_arr[2].`value`, " @@ -264,7 +266,7 @@ public void testNestedProjectFieldAccessWithITEMContainsTopLevelAccess() { } @Test - public void testProjectFieldAccessWithITEM() { + void testProjectFieldAccessWithITEM() { util().verifyRelPlan( "SELECT " + "`Result`.data_arr[ID].`value`, " @@ -276,7 +278,7 @@ public void testProjectFieldAccessWithITEM() { } @Test - public void testMetadataProjectionWithoutProjectionPushDownWhenSupported() { + void testMetadataProjectionWithoutProjectionPushDownWhenSupported() { final SharedReference> appliedKeys = sharedObjects.add(new ArrayList<>()); final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() @@ -290,7 +292,7 @@ public void testMetadataProjectionWithoutProjectionPushDownWhenSupported() { } @Test - public void testMetadataProjectionWithoutProjectionPushDownWhenNotSupported() { + void testMetadataProjectionWithoutProjectionPushDownWhenNotSupported() { final SharedReference> appliedKeys = sharedObjects.add(new ArrayList<>()); final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() @@ -304,7 +306,7 @@ public void testMetadataProjectionWithoutProjectionPushDownWhenNotSupported() { } @Test - public void testMetadataProjectionWithoutProjectionPushDownWhenSupportedAndNoneSelected() { + void testMetadataProjectionWithoutProjectionPushDownWhenSupportedAndNoneSelected() { final SharedReference> appliedKeys = sharedObjects.add(new ArrayList<>()); final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() @@ -320,7 +322,7 @@ public void testMetadataProjectionWithoutProjectionPushDownWhenSupportedAndNoneS } @Test - public void testMetadataProjectionWithoutProjectionPushDownWhenNotSupportedAndNoneSelected() { + void testMetadataProjectionWithoutProjectionPushDownWhenNotSupportedAndNoneSelected() { final SharedReference> appliedKeys = sharedObjects.add(new ArrayList<>()); final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() @@ -334,7 +336,7 @@ public void testMetadataProjectionWithoutProjectionPushDownWhenNotSupportedAndNo } @Test - public void testProjectionIncludingOnlyMetadata() { + void testProjectionIncludingOnlyMetadata() { replaceProgramWithProjectMergeRule(); final AtomicReference appliedProjectionDataType = new AtomicReference<>(null); @@ -374,7 +376,7 @@ private void replaceProgramWithProjectMergeRule() { } @Test - public void testProjectionWithMetadataAndPhysicalFields() { + void testProjectionWithMetadataAndPhysicalFields() { replaceProgramWithProjectMergeRule(); final AtomicReference appliedProjectionDataType = new AtomicReference<>(null); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.java index 47a21ad0ae97f..6fc848f56b45a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.java @@ -37,8 +37,8 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.tools.RuleSets; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.time.Duration; @@ -48,10 +48,10 @@ * Test rule {@link PushWatermarkIntoTableSourceScanAcrossCalcRule} and {@link * PushWatermarkIntoTableSourceScanRule}. */ -public class PushWatermarkIntoTableSourceScanRuleTest extends TableTestBase { +class PushWatermarkIntoTableSourceScanRuleTest extends TableTestBase { private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); - @Before + @BeforeEach public void setup() { FlinkChainedProgram program = new FlinkChainedProgram<>(); program.addLast( @@ -80,7 +80,7 @@ public void setup() { } @Test - public void testSimpleWatermark() { + void testSimpleWatermark() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -98,7 +98,7 @@ public void testSimpleWatermark() { } @Test - public void testWatermarkOnComputedColumn() { + void testWatermarkOnComputedColumn() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -117,7 +117,7 @@ public void testWatermarkOnComputedColumn() { } @Test - public void testWatermarkOnComputedColumnWithQuery() { + void testWatermarkOnComputedColumnWithQuery() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -137,7 +137,7 @@ public void testWatermarkOnComputedColumnWithQuery() { } @Test - public void testWatermarkOnComputedColumnWithMultipleInputs() { + void testWatermarkOnComputedColumnWithMultipleInputs() { String ddl = "CREATE TABLE MyTable(" + " a STRING,\n" @@ -155,7 +155,7 @@ public void testWatermarkOnComputedColumnWithMultipleInputs() { } @Test - public void testWatermarkOnRow() { + void testWatermarkOnRow() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -174,7 +174,7 @@ public void testWatermarkOnRow() { } @Test - public void testWatermarkOnNestedRow() { + void testWatermarkOnNestedRow() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -193,7 +193,7 @@ public void testWatermarkOnNestedRow() { } @Test - public void testWatermarkWithMultiInputUdf() { + void testWatermarkWithMultiInputUdf() { JavaFunc5.closeCalled = false; JavaFunc5.openCalled = false; util.addTemporarySystemFunction("func", new JavaFunc5()); @@ -216,7 +216,7 @@ public void testWatermarkWithMultiInputUdf() { } @Test - public void testWatermarkOnMetadata() { + void testWatermarkOnMetadata() { String ddl = "CREATE TABLE MyTable(" + " `a` INT,\n" @@ -237,7 +237,7 @@ public void testWatermarkOnMetadata() { } @Test - public void testWatermarkWithIdleSource() { + void testWatermarkWithIdleSource() { util.tableEnv().getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(1000)); String ddl = "CREATE TABLE MyTable(" @@ -256,7 +256,7 @@ public void testWatermarkWithIdleSource() { } @Test - public void testWatermarkWithPythonFunctionInComputedColumn() { + void testWatermarkWithPythonFunctionInComputedColumn() { util.tableEnv() .createTemporaryFunction( "parse_ts", @@ -277,7 +277,7 @@ public void testWatermarkWithPythonFunctionInComputedColumn() { } @Test - public void testWatermarkEmitStrategyWithOptions() { + void testWatermarkEmitStrategyWithOptions() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -296,7 +296,7 @@ public void testWatermarkEmitStrategyWithOptions() { } @Test - public void testWatermarkEmitStrategyWithHint() { + void testWatermarkEmitStrategyWithHint() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -316,7 +316,7 @@ public void testWatermarkEmitStrategyWithHint() { } @Test - public void testWatermarkAlignmentWithOptions() { + void testWatermarkAlignmentWithOptions() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -336,7 +336,7 @@ public void testWatermarkAlignmentWithOptions() { } @Test - public void testWatermarkAlignmentWithHint() { + void testWatermarkAlignmentWithHint() { String ddl = "CREATE TABLE MyTable(" + " a INT,\n" @@ -357,7 +357,7 @@ public void testWatermarkAlignmentWithHint() { } @Test - public void testIdleSourceWithOptions() { + void testIdleSourceWithOptions() { util.tableEnv().getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(1000)); String ddl = "CREATE TABLE MyTable(" @@ -377,7 +377,7 @@ public void testIdleSourceWithOptions() { } @Test - public void testIdleSourceWithHint() { + void testIdleSourceWithHint() { util.tableEnv().getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(1000)); String ddl = "CREATE TABLE MyTable(" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/RemoveUnreachableCoalesceArgumentsRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/RemoveUnreachableCoalesceArgumentsRuleTest.java index 799ac38014c4e..3610b20d6f3a5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/RemoveUnreachableCoalesceArgumentsRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/RemoveUnreachableCoalesceArgumentsRuleTest.java @@ -25,18 +25,18 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.apache.flink.table.api.DataTypes.STRING; /** Test rule {@link RemoveUnreachableCoalesceArgumentsRule}. */ -public class RemoveUnreachableCoalesceArgumentsRuleTest extends TableTestBase { +class RemoveUnreachableCoalesceArgumentsRuleTest extends TableTestBase { private StreamTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = streamTestUtil(TableConfig.getDefault()); final TableDescriptor sourceDescriptor = @@ -54,38 +54,38 @@ public void before() { } @Test - public void testOnlyLastNonNull() { + void testOnlyLastNonNull() { util.verifyRelPlan("SELECT COALESCE(f0, f1) FROM T"); } @Test - public void testAllNullable() { + void testAllNullable() { util.verifyRelPlan("SELECT COALESCE(f0, f2) FROM T"); } @Test - public void testDropLastConstant() { + void testDropLastConstant() { util.verifyRelPlan("SELECT COALESCE(f0, f1, '-') FROM T"); } @Test - public void testDropCoalesce() { + void testDropCoalesce() { util.verifyRelPlan("SELECT COALESCE(f1, '-') FROM T"); } @Test - public void testFilterCoalesce() { + void testFilterCoalesce() { util.verifyRelPlan("SELECT * FROM T WHERE COALESCE(f0, f1, '-') = 'abc'"); } @Test - public void testJoinCoalesce() { + void testJoinCoalesce() { util.verifyRelPlan( "SELECT * FROM T t1 LEFT JOIN T t2 ON COALESCE(t1.f0, '-', t1.f2) = t2.f0"); } @Test - public void testMultipleCoalesces() { + void testMultipleCoalesces() { util.verifyRelPlan( "SELECT COALESCE(1),\n" + "COALESCE(1, 2),\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/WrapJsonAggFunctionArgumentsRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/WrapJsonAggFunctionArgumentsRuleTest.java index ba4d497fd9e5c..e512d3b26d545 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/WrapJsonAggFunctionArgumentsRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/WrapJsonAggFunctionArgumentsRuleTest.java @@ -21,33 +21,34 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; /** Tests for {@link WrapJsonAggFunctionArgumentsRule}. */ -@RunWith(Parameterized.class) -public class WrapJsonAggFunctionArgumentsRuleTest extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class WrapJsonAggFunctionArgumentsRuleTest extends TableTestBase { private final boolean batchMode; private TableTestUtil util; - @Parameterized.Parameters(name = "batchMode = {0}") - public static Collection data() { + @Parameters(name = "batchMode = {0}") + private static Collection data() { return Arrays.asList(true, false); } - public WrapJsonAggFunctionArgumentsRuleTest(boolean batchMode) { + WrapJsonAggFunctionArgumentsRuleTest(boolean batchMode) { this.batchMode = batchMode; } - @Before - public void setup() { + @BeforeEach + void setup() { if (batchMode) { util = batchTestUtil(TableConfig.getDefault()); } else { @@ -79,55 +80,55 @@ public void setup() { + "'\n)"); } - @Test - public void testJsonObjectAgg() { + @TestTemplate + void testJsonObjectAgg() { util.verifyRelPlan("SELECT JSON_OBJECTAGG(f1 VALUE f1) FROM T"); } - @Test - public void testJsonObjectAggInGroupWindow() { + @TestTemplate + void testJsonObjectAggInGroupWindow() { util.verifyRelPlan("SELECT f0, JSON_OBJECTAGG(f1 VALUE f0) FROM T GROUP BY f0"); } - @Test - public void testJsonArrayAgg() { + @TestTemplate + void testJsonArrayAgg() { util.verifyRelPlan("SELECT JSON_ARRAYAGG(f0) FROM T"); } - @Test - public void testJsonArrayAggInGroupWindow() { + @TestTemplate + void testJsonArrayAggInGroupWindow() { util.verifyRelPlan("SELECT f0, JSON_ARRAYAGG(f0) FROM T GROUP BY f0"); } - @Test - public void testJsonObjectAggWithOtherAggs() { + @TestTemplate + void testJsonObjectAggWithOtherAggs() { util.verifyRelPlan("SELECT COUNT(*), JSON_OBJECTAGG(f1 VALUE f1) FROM T"); } - @Test - public void testGroupJsonObjectAggWithOtherAggs() { + @TestTemplate + void testGroupJsonObjectAggWithOtherAggs() { util.verifyRelPlan( "SELECT f0, COUNT(*), JSON_OBJECTAGG(f1 VALUE f0), SUM(f2) FROM T GROUP BY f0"); } - @Test - public void testJsonArrayAggWithOtherAggs() { + @TestTemplate + void testJsonArrayAggWithOtherAggs() { util.verifyRelPlan("SELECT COUNT(*), JSON_ARRAYAGG(f0) FROM T"); } - @Test - public void testGroupJsonArrayAggInWithOtherAggs() { + @TestTemplate + void testGroupJsonArrayAggInWithOtherAggs() { util.verifyRelPlan("SELECT f0, COUNT(*), JSON_ARRAYAGG(f0), SUM(f2) FROM T GROUP BY f0"); } - @Test - public void testJsonArrayAggAndJsonObjectAggWithOtherAggs() { + @TestTemplate + void testJsonArrayAggAndJsonObjectAggWithOtherAggs() { util.verifyRelPlan( "SELECT MAX(f0), JSON_OBJECTAGG(f1 VALUE f0), JSON_ARRAYAGG(f1), JSON_ARRAYAGG(f0) FROM T"); } - @Test - public void testGroupJsonArrayAggAndJsonObjectAggWithOtherAggs() { + @TestTemplate + void testGroupJsonArrayAggAndJsonObjectAggWithOtherAggs() { util.verifyRelPlan( "SELECT f0, JSON_OBJECTAGG(f1 VALUE f2), JSON_ARRAYAGG(f1), JSON_ARRAYAGG(f2)," + " SUM(f2) FROM T GROUP BY f0"); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java index 6f20e6ef0bba6..01558291d809b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java @@ -26,18 +26,18 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.apache.flink.table.runtime.functions.aggregate.CollectAggFunction; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Test for rules that extend {@link PushLocalAggIntoScanRuleBase} to push down local aggregates * into table source. */ -public class PushLocalAggIntoTableSourceScanRuleTest extends TableTestBase { +class PushLocalAggIntoTableSourceScanRuleTest extends TableTestBase { protected BatchTableTestUtil util = batchTestUtil(TableConfig.getDefault()); - @Before - public void setup() { + @BeforeEach + void setup() { TableConfig tableConfig = util.tableEnv().getConfig(); tableConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED, true); @@ -109,7 +109,7 @@ public void setup() { } @Test - public void testCanPushDownLocalHashAggWithGroup() { + void testCanPushDownLocalHashAggWithGroup() { util.verifyRelPlan( "SELECT\n" + " sum(amount),\n" @@ -120,7 +120,7 @@ public void testCanPushDownLocalHashAggWithGroup() { } @Test - public void testDisablePushDownLocalAgg() { + void testDisablePushDownLocalAgg() { // disable push down local agg util.getTableEnv() .getConfig() @@ -145,7 +145,7 @@ public void testDisablePushDownLocalAgg() { } @Test - public void testCanPushDownLocalHashAggWithoutGroup() { + void testCanPushDownLocalHashAggWithoutGroup() { util.verifyRelPlan( "SELECT\n" + " min(id),\n" @@ -157,7 +157,7 @@ public void testCanPushDownLocalHashAggWithoutGroup() { } @Test - public void testCanPushDownLocalSortAggWithoutSort() { + void testCanPushDownLocalSortAggWithoutSort() { // enable sort agg util.getTableEnv() .getConfig() @@ -179,7 +179,7 @@ public void testCanPushDownLocalSortAggWithoutSort() { } @Test - public void testCanPushDownLocalSortAggWithSort() { + void testCanPushDownLocalSortAggWithSort() { // enable sort agg util.getTableEnv() .getConfig() @@ -200,7 +200,7 @@ public void testCanPushDownLocalSortAggWithSort() { } @Test - public void testCanPushDownLocalAggAfterFilterPushDown() { + void testCanPushDownLocalAggAfterFilterPushDown() { util.verifyRelPlan( "SELECT\n" @@ -213,7 +213,7 @@ public void testCanPushDownLocalAggAfterFilterPushDown() { } @Test - public void testCanPushDownLocalAggWithMetadata() { + void testCanPushDownLocalAggWithMetadata() { util.verifyRelPlan( "SELECT\n" + " sum(amount),\n" @@ -226,7 +226,7 @@ public void testCanPushDownLocalAggWithMetadata() { } @Test - public void testCanPushDownLocalAggWithPartition() { + void testCanPushDownLocalAggWithPartition() { util.verifyRelPlan( "SELECT\n" + " sum(amount),\n" @@ -238,7 +238,7 @@ public void testCanPushDownLocalAggWithPartition() { } @Test - public void testCanPushDownLocalAggWithoutProjectionPushDown() { + void testCanPushDownLocalAggWithoutProjectionPushDown() { util.verifyRelPlan( "SELECT\n" + " sum(amount),\n" @@ -250,7 +250,7 @@ public void testCanPushDownLocalAggWithoutProjectionPushDown() { } @Test - public void testCanPushDownLocalAggWithAuxGrouping() { + void testCanPushDownLocalAggWithAuxGrouping() { // enable two-phase aggregate, otherwise there is no local aggregate util.getTableEnv() .getConfig() @@ -264,7 +264,7 @@ public void testCanPushDownLocalAggWithAuxGrouping() { } @Test - public void testCannotPushDownLocalAggAfterLimitPushDown() { + void testCannotPushDownLocalAggAfterLimitPushDown() { util.verifyRelPlan( "SELECT\n" @@ -281,7 +281,7 @@ public void testCannotPushDownLocalAggAfterLimitPushDown() { } @Test - public void testCannotPushDownLocalAggWithUDAF() { + void testCannotPushDownLocalAggWithUDAF() { // add udf util.addTemporarySystemFunction( "udaf_collect", new CollectAggFunction<>(DataTypes.BIGINT().getLogicalType())); @@ -296,7 +296,7 @@ public void testCannotPushDownLocalAggWithUDAF() { } @Test - public void testCannotPushDownLocalAggWithUnsupportedDataTypes() { + void testCannotPushDownLocalAggWithUnsupportedDataTypes() { util.verifyRelPlan( "SELECT\n" + " max(name),\n" @@ -306,7 +306,7 @@ public void testCannotPushDownLocalAggWithUnsupportedDataTypes() { } @Test - public void testCannotPushDownWithColumnExpression() { + void testCannotPushDownWithColumnExpression() { util.verifyRelPlan( "SELECT\n" + " min(amount + price),\n" @@ -319,7 +319,7 @@ public void testCannotPushDownWithColumnExpression() { } @Test - public void testCannotPushDownWithUnsupportedAggFunction() { + void testCannotPushDownWithUnsupportedAggFunction() { util.verifyRelPlan( "SELECT\n" + " min(id),\n" @@ -332,7 +332,7 @@ public void testCannotPushDownWithUnsupportedAggFunction() { } @Test - public void testCannotPushDownWithWindowAggFunction() { + void testCannotPushDownWithWindowAggFunction() { util.verifyRelPlan( "SELECT\n" + " id,\n" @@ -343,7 +343,7 @@ public void testCannotPushDownWithWindowAggFunction() { } @Test - public void testCannotPushDownWithArgFilter() { + void testCannotPushDownWithArgFilter() { util.verifyRelPlan( "SELECT\n" + " min(id),\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushCalcPastChangelogNormalizeRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushCalcPastChangelogNormalizeRuleTest.java index 4e0b2009e7d3d..b27dda1ded20b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushCalcPastChangelogNormalizeRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushCalcPastChangelogNormalizeRuleTest.java @@ -26,25 +26,25 @@ import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.apache.flink.table.api.DataTypes.BIGINT; import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.STRING; /** Tests for {@link PushCalcPastChangelogNormalizeRule}. */ -public class PushCalcPastChangelogNormalizeRuleTest extends TableTestBase { +class PushCalcPastChangelogNormalizeRuleTest extends TableTestBase { private StreamTableTestUtil util; - @Before - public void before() { + @BeforeEach + void before() { util = streamTestUtil(TableConfig.getDefault()); } @Test - public void testWithSinglePrimaryKeyFilter() { + void testWithSinglePrimaryKeyFilter() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( @@ -61,7 +61,7 @@ public void testWithSinglePrimaryKeyFilter() { } @Test - public void testWithMultipleFilters() { + void testWithMultipleFilters() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( @@ -82,7 +82,7 @@ public void testWithMultipleFilters() { } @Test - public void testWithMultiplePrimaryKeyColumns() { + void testWithMultiplePrimaryKeyColumns() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( @@ -100,7 +100,7 @@ public void testWithMultiplePrimaryKeyColumns() { } @Test - public void testOnlyProjection() { + void testOnlyProjection() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( @@ -118,7 +118,7 @@ public void testOnlyProjection() { } @Test - public void testFilterAndProjection() { + void testFilterAndProjection() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( @@ -141,7 +141,7 @@ public void testFilterAndProjection() { } @Test - public void testPartialPrimaryKeyFilterAndProjection() { + void testPartialPrimaryKeyFilterAndProjection() { final TableDescriptor sourceDescriptor = TableFactoryHarness.newBuilder() .schema( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/CalcMergeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/CalcMergeTest.java index 9da719f03b19f..74efb225979d3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/CalcMergeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/CalcMergeTest.java @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.utils.TableTestUtil; /** Plan test for calc merge. */ -public class CalcMergeTest extends CalcMergeTestBase { +class CalcMergeTest extends CalcMergeTestBase { @Override protected boolean isBatchMode() { return false; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/DynamicFunctionPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/DynamicFunctionPlanTest.java index adb4523e78dd4..2a00d2f033bb5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/DynamicFunctionPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/DynamicFunctionPlanTest.java @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.utils.TableTestUtil; /** Plan test for queries contain dynamic functions in streaming. */ -public class DynamicFunctionPlanTest extends DynamicFunctionPlanTestBase { +class DynamicFunctionPlanTest extends DynamicFunctionPlanTestBase { @Override protected TableTestUtil getTableTestUtil() { return streamTestUtil(TableConfig.getDefault()); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/InsertIntoValuesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/InsertIntoValuesTest.java index 34c5286a3c399..eb032a1c0efdd 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/InsertIntoValuesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/InsertIntoValuesTest.java @@ -21,14 +21,14 @@ import org.apache.flink.table.planner.utils.JavaStreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** Plan test for INSERT INTO. */ -public class InsertIntoValuesTest extends TableTestBase { +class InsertIntoValuesTest extends TableTestBase { private final JavaStreamTableTestUtil util = javaStreamTestUtil(); @Test - public void testTypeInferenceWithNestedTypes() { + void testTypeInferenceWithNestedTypes() { util.tableEnv() .executeSql( "CREATE TABLE t1 (" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java index 2984e74086dd1..73d849826cb22 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java @@ -22,25 +22,25 @@ import org.apache.flink.table.planner.utils.JavaStreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** Tests for usages of {@link LegacyTableFactory}. */ -public class LegacyTableFactoryTest extends TableTestBase { +class LegacyTableFactoryTest extends TableTestBase { private final JavaStreamTableTestUtil util; - public LegacyTableFactoryTest() { + LegacyTableFactoryTest() { util = javaStreamTestUtil(); util.tableEnv().executeSql("CREATE TABLE T (a INT) WITH ('type'='legacy')"); } @Test - public void testSelect() { + void testSelect() { util.verifyExecPlan("SELECT * FROM T"); } @Test - public void testInsert() { + void testInsert() { util.verifyExecPlanInsert("INSERT INTO T VALUES (1)"); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/NullTypeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/NullTypeTest.java index 9aa303dcadba1..61f161bb2b26b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/NullTypeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/NullTypeTest.java @@ -25,71 +25,82 @@ import org.apache.flink.table.planner.utils.JavaStreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for usages of {@link DataTypes#NULL()}. */ -public class NullTypeTest extends TableTestBase { +class NullTypeTest extends TableTestBase { private final JavaStreamTableTestUtil util = javaStreamTestUtil(); @Test - public void testValues() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Illegal use of 'NULL'"); - util.verifyExecPlan("SELECT * FROM (VALUES (1, NULL), (2, NULL)) AS T(a, b)"); + void testValues() { + assertThatThrownBy( + () -> + util.verifyExecPlan( + "SELECT * FROM (VALUES (1, NULL), (2, NULL)) AS T(a, b)")) + .hasMessageContaining("Illegal use of 'NULL'") + .isInstanceOf(ValidationException.class); } @Test - public void testValuesWithoutTypeCoercion() { + void testValuesWithoutTypeCoercion() { // should work if we enable type coercion, works already in Table API - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Illegal use of 'NULL'"); - util.verifyExecPlan("SELECT * FROM (VALUES (1, NULL), (2, 1)) AS T(a, b)"); + assertThatThrownBy( + () -> + util.verifyExecPlan( + "SELECT * FROM (VALUES (1, NULL), (2, 1)) AS T(a, b)")) + .hasMessageContaining("Illegal use of 'NULL'") + .isInstanceOf(ValidationException.class); } @Test - public void testSetOperationWithoutTypeCoercion() { + void testSetOperationWithoutTypeCoercion() { // we might want to support type coercion here - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Parameters must be of the same type"); - util.verifyExecPlan("SELECT ARRAY[1,2] IN (ARRAY[1], ARRAY[1,2], ARRAY[NULL, NULL, NULL])"); + assertThatThrownBy( + () -> + util.verifyExecPlan( + "SELECT ARRAY[1,2] IN (ARRAY[1], ARRAY[1,2], ARRAY[NULL, NULL, NULL])")) + .hasMessageContaining("Parameters must be of the same type") + .isInstanceOf(ValidationException.class); } @Test - public void testBuiltInFunction() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Illegal use of 'NULL'"); - util.verifyExecPlan("SELECT ABS(NULL)"); + void testBuiltInFunction() { + assertThatThrownBy(() -> util.verifyExecPlan("SELECT ABS(NULL)")) + .hasMessageContaining("Illegal use of 'NULL'") + .isInstanceOf(ValidationException.class); } @Test - public void testArrayConstructor() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Parameters must be of the same type"); - util.verifyExecPlan("SELECT ARRAY[NULL]"); + void testArrayConstructor() { + assertThatThrownBy(() -> util.verifyExecPlan("SELECT ARRAY[NULL]")) + .hasMessageContaining("Parameters must be of the same type") + .isInstanceOf(ValidationException.class); } @Test - public void testMapConstructor() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("Parameters must be of the same type"); - util.verifyExecPlan("SELECT MAP[NULL, NULL]"); + void testMapConstructor() { + assertThatThrownBy(() -> util.verifyExecPlan("SELECT MAP[NULL, NULL]")) + .hasMessageContaining("Parameters must be of the same type") + .isInstanceOf(ValidationException.class); } @Test - public void testFunctionReturningNull() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("SQL validation failed. Invalid function call"); + void testFunctionReturningNull() { util.addTemporarySystemFunction("NullTypeFunction", NullTypeFunction.class); - util.verifyExecPlan("SELECT NullTypeFunction(12)"); + assertThatThrownBy(() -> util.verifyExecPlan("SELECT NullTypeFunction(12)")) + .hasMessageContaining("SQL validation failed. Invalid function call") + .isInstanceOf(ValidationException.class); } @Test - public void testNestedNull() { - expectedException().expect(ValidationException.class); - expectedException().expectMessage("SQL validation failed. Invalid function call"); + void testNestedNull() { util.addTemporarySystemFunction("NestedNullTypeFunction", NestedNullTypeFunction.class); - util.verifyExecPlan("SELECT NestedNullTypeFunction(12)"); + assertThatThrownBy(() -> util.verifyExecPlan("SELECT NestedNullTypeFunction(12)")) + .hasMessageContaining("SQL validation failed. Invalid function call") + .isInstanceOf(ValidationException.class); } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/table/ValuesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/table/ValuesTest.java index 7c4e1a8d276f8..667683089a6c5 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/table/ValuesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/table/ValuesTest.java @@ -26,7 +26,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.time.LocalTime; import java.util.Arrays; @@ -38,12 +38,13 @@ import static org.apache.flink.table.api.Expressions.nullOf; import static org.apache.flink.table.api.Expressions.pi; import static org.apache.flink.table.api.Expressions.row; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link org.apache.flink.table.api.TableEnvironment#fromValues}. */ -public class ValuesTest extends TableTestBase { +class ValuesTest extends TableTestBase { @Test - public void testValuesAllEqualTypes() { + void testValuesAllEqualTypes() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -56,14 +57,14 @@ public void testValuesAllEqualTypes() { } @Test - public void testValuesFromLiterals() { + void testValuesFromLiterals() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv().fromValues(1, 3.1f, 99L, null); util.verifyExecPlan(t); } @Test - public void testValuesFromRowExpression() { + void testValuesFromRowExpression() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -84,7 +85,7 @@ public void testValuesFromRowExpression() { } @Test - public void testValuesFromRowObject() { + void testValuesFromRowObject() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -98,7 +99,7 @@ public void testValuesFromRowObject() { } @Test - public void testValuesFromMixedObjectsAndExpressions() { + void testValuesFromMixedObjectsAndExpressions() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -112,7 +113,7 @@ public void testValuesFromMixedObjectsAndExpressions() { } @Test - public void testValuesFromRowObjectInCollection() { + void testValuesFromRowObjectInCollection() { JavaStreamTableTestUtil util = javaStreamTestUtil(); List data = Arrays.asList( @@ -132,7 +133,7 @@ public void testValuesFromRowObjectInCollection() { } @Test - public void testValuesFromNestedRowObject() { + void testValuesFromNestedRowObject() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -143,7 +144,7 @@ public void testValuesFromNestedRowObject() { } @Test - public void testValuesOverrideSchema() { + void testValuesOverrideSchema() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -157,7 +158,7 @@ public void testValuesOverrideSchema() { } @Test - public void testValuesOverrideNullability() { + void testValuesOverrideNullability() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -171,7 +172,7 @@ public void testValuesOverrideNullability() { } @Test - public void testValuesWithComplexNesting() { + void testValuesWithComplexNesting() { JavaStreamTableTestUtil util = javaStreamTestUtil(); Table t = util.getTableEnv() @@ -210,39 +211,54 @@ public void testValuesWithComplexNesting() { } @Test - public void testNoCommonType() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - "Types in fromValues(...) must have a common super type. Could not" - + " find a common type for all rows at column 1."); + void testNoCommonType() { JavaStreamTableTestUtil util = javaStreamTestUtil(); - util.getTableEnv() - .fromValues(row("ABC", 1L), row("ABC", lit(LocalTime.of(0, 0, 0))), row("ABC", 2)); + + assertThatThrownBy( + () -> + util.getTableEnv() + .fromValues( + row("ABC", 1L), + row("ABC", lit(LocalTime.of(0, 0, 0))), + row("ABC", 2))) + .hasMessageContaining( + "Types in fromValues(...) must have a common super type. Could not" + + " find a common type for all rows at column 1.") + .isInstanceOf(ValidationException.class); } @Test - public void testCannotCast() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - "Could not cast the value of the 0 column: [ 4 ] of a row: [ 4 ]" - + " to the requested type: BINARY(3)"); + void testCannotCast() { JavaStreamTableTestUtil util = javaStreamTestUtil(); - util.getTableEnv() - .fromValues(DataTypes.ROW(DataTypes.FIELD("f1", DataTypes.BINARY(3))), row(4)); + + assertThatThrownBy( + () -> + util.getTableEnv() + .fromValues( + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.BINARY(3))), + row(4))) + .hasMessageContaining( + "Could not cast the value of the 0 column: [ 4 ] of a row: [ 4 ]" + + " to the requested type: BINARY(3)") + .isInstanceOf(ValidationException.class); } @Test - public void testWrongRowTypeLength() { - thrown().expect(ValidationException.class); - thrown().expectMessage( - "All rows in a fromValues(...) clause must have the same fields number. Row [4] has a different" - + " length than the expected size: 2."); + void testWrongRowTypeLength() { JavaStreamTableTestUtil util = javaStreamTestUtil(); - util.getTableEnv() - .fromValues( - DataTypes.ROW( - DataTypes.FIELD("f1", DataTypes.BINARY(3)), - DataTypes.FIELD("f2", DataTypes.STRING())), - row(4)); + + assertThatThrownBy( + () -> + util.getTableEnv() + .fromValues( + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.BINARY(3)), + DataTypes.FIELD("f2", DataTypes.STRING())), + row(4))) + .hasMessageContaining( + "All rows in a fromValues(...) clause must have the same fields number. Row [4] has a different" + + " length than the expected size: 2.") + .isInstanceOf(ValidationException.class); } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.xml index d8bf60b49529f..ad37a1cdc1a82 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.xml @@ -16,7 +16,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + @@ -34,7 +34,7 @@ BoundedStreamScan(table=[[default_catalog, default_database, t1]], fields=[a, b, ]]> - + @@ -52,7 +52,7 @@ DataStreamScan(table=[[default_catalog, default_database, t1]], fields=[a, b, c] ]]> - + @@ -75,7 +75,7 @@ Calc(select=[f0, name AS f1]) ]]> - + @@ -94,7 +94,7 @@ BoundedStreamScan(table=[[default_catalog, default_database, t1]], fields=[a, b, ]]> - + @@ -113,7 +113,7 @@ DataStreamScan(table=[[default_catalog, default_database, t1]], fields=[a, b, c] ]]> - + - + - + @@ -159,7 +159,7 @@ Calc(select=[CAST(a AS INTEGER) AS a, b, CAST(EXPR$2 AS INTEGER) AS c]) ]]> - + @@ -182,7 +182,7 @@ Calc(select=[f0, name AS f1]) ]]> - + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hint/OptionsHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hint/OptionsHintTest.xml index 568039e1c24d8..c128d22e19a0c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hint/OptionsHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hint/OptionsHintTest.xml @@ -16,7 +16,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + @@ -34,7 +34,7 @@ Calc(select=[a, b, (a + 1) AS c]) ]]> - + @@ -52,7 +52,7 @@ Calc(select=[a, b, (a + 1) AS c]) ]]> - + - + - + @@ -129,7 +129,7 @@ Calc(select=[a, b, (a + 1) AS c]) ]]> - + - + - + - + @@ -232,7 +232,7 @@ Calc(select=[a, b, (a + 1) AS c]) ]]> - + util.tableEnv.sqlQuery("SELECT a, foo FROM MyTable")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CompactManagedTableTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CompactManagedTableTest.scala index c43c6380bed56..b68d9b84129e1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CompactManagedTableTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CompactManagedTableTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.catalog.{CatalogPartitionSpec, ObjectIdentifier} import org.apache.flink.table.factories.TestManagedTableFactory import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{After, Before, Test} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import java.util import java.util.Collections @@ -34,7 +34,7 @@ class CompactManagedTableTest extends TableTestBase { ObjectIdentifier.of("default_catalog", "default_database", "ManagedTable") private val testUtil = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { val tableRef = new AtomicReference[util.Map[String, String]] TestManagedTableFactory.MANAGED_TABLES.put(tableIdentifier, tableRef) @@ -60,7 +60,7 @@ class CompactManagedTableTest extends TableTestBase { TestManagedTableFactory.MANAGED_TABLE_FILE_ENTRIES.put(tableIdentifier, fileRef) } - @After + @AfterEach def after(): Unit = { val ddl = "DROP TABLE ManagedTable" testUtil.tableEnv.executeSql(ddl) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala index 8f4a4dd55d998..9a18da17f6f28 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala @@ -25,7 +25,7 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase} import org.apache.flink.table.types.logical._ -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala index cd6abc57dcbec..20868e298da9c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala @@ -24,13 +24,13 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class DeadlockBreakupTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_PIPELINED) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyLimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyLimitTest.scala index 7d968732dad67..9eb463351112a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyLimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyLimitTest.scala @@ -21,13 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api.{SqlParserException, _} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} class LegacyLimitTest extends TableTestBase { protected val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val ddl = @@ -54,9 +55,10 @@ class LegacyLimitTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM MyTable LIMIT 0") } - @Test(expected = classOf[SqlParserException]) + @Test def testNegativeLimitWithoutOffset(): Unit = { - util.verifyExecPlan("SELECT * FROM MyTable LIMIT -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyExecPlan("SELECT * FROM MyTable LIMIT -1")) } @Test @@ -79,9 +81,10 @@ class LegacyLimitTest extends TableTestBase { util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 0 OFFSET 10") } - @Test(expected = classOf[SqlParserException]) + @Test def testLimitWithNegativeOffset(): Unit = { - util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 10 OFFSET -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 10 OFFSET -1")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala index a1a8b9bf05beb..873b10a090973 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala @@ -19,13 +19,12 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType} -import org.junit.Test +import org.junit.jupiter.api.Test class LegacySinkTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyTableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyTableSourceTest.scala index e7350e2fb1bd2..a4da620ec0ebf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyTableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacyTableSourceTest.scala @@ -20,14 +20,14 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{DataTypes, TableSchema, Types, ValidationException} -import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.utils._ import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter import org.apache.flink.types.Row -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class LegacyTableSourceTest extends TableTestBase { @@ -37,7 +37,7 @@ class LegacyTableSourceTest extends TableTestBase { .fields(Array("a", "b", "c"), Array(DataTypes.INT(), DataTypes.BIGINT(), DataTypes.STRING())) .build() - @Before + @BeforeEach def setup(): Unit = { util.tableEnv .asInstanceOf[TableEnvironmentInternal] @@ -69,9 +69,10 @@ class LegacyTableSourceTest extends TableTestBase { @Test def testUnboundedStreamTableSource(): Unit = { TestTableSource.createTemporaryTable(util.tableEnv, isBounded = false, tableSchema, "MyTable") - thrown.expect(classOf[ValidationException]) - thrown.expectMessage("Only bounded StreamTableSource can be used in batch mode.") - util.verifyExecPlan("SELECT * FROM MyTable") + + assertThatThrownBy(() => util.verifyExecPlan("SELECT * FROM MyTable")) + .hasMessageContaining("Only bounded StreamTableSource can be used in batch mode.") + .isInstanceOf[ValidationException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala index d7762eacd4063..86fe1c7c96522 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala @@ -21,6 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown +import org.junit.jupiter.api.BeforeEach + /** * The plan of following unit test in LimitTest.xml is a bit diffirent from LegacyLimitTest.xml. * Because the TestValuesTableSource has implemented [[SupportsProjectionPushDown]] while the @@ -30,6 +32,7 @@ import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushD */ class LimitTest extends LegacyLimitTest { + @BeforeEach override def setup(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val ddl = diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala index 43a2944e1fef9..11a8f5c5ef514 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala @@ -27,19 +27,18 @@ import org.apache.flink.configuration.JobManagerOptions.SchedulerType import org.apache.flink.table.api._ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.runners.Parameterized.Parameters +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: SchedulerType) extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTableSource[(Int, Long, String, Int)]("x", 'a, 'b, 'c, 'nx) util.addTableSource[(Int, Long, String, Int)]("y", 'd, 'e, 'f, 'ny) @@ -49,7 +48,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.tableConfig.set(JobManagerOptions.SCHEDULER, schedulerType) } - @Test + @TestTemplate def testBasicMultipleInput(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") @@ -64,7 +63,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testManyMultipleInputs(): Unit = { // y z t y t // | | | | | @@ -102,7 +101,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinWithAggAsProbe(): Unit = { util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, @@ -119,7 +118,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testKeepMultipleInputWithOneMemberForChainableSource(): Unit = { createChainableTableSource() util.tableEnv.getConfig @@ -128,7 +127,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAvoidIncludingUnionFromInputSide(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") @@ -141,7 +140,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testIncludeUnionForChainableSource(): Unit = { createChainableTableSource() util.tableEnv.getConfig @@ -155,7 +154,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAvoidIncludingCalcAfterNonChainableSource(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") @@ -169,7 +168,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testIncludeCalcForChainableSource(): Unit = { createChainableTableSource() util.tableEnv.getConfig @@ -184,7 +183,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAvoidIncludingSingleton(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,HashAgg") @@ -202,7 +201,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testNoPriorityConstraint(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") @@ -215,7 +214,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testRelatedInputs(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") @@ -233,7 +232,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testRelatedInputsWithAgg(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg") @@ -251,7 +250,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testRemoveRedundantUnion(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) @@ -273,7 +272,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testRemoveOneInputOperatorFromRoot(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) @@ -292,7 +291,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testCleanUpMultipleInputWithOneMember(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") @@ -308,7 +307,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testKeepUsefulUnion(): Unit = { createChainableTableSource() util.tableEnv.getConfig @@ -328,7 +327,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode, schedulerType: Sc util.verifyExecPlan(sql) } - @Test + @TestTemplate def testDeadlockCausedByExchangeInAncestor(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/OperatorFusionCodegenTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/OperatorFusionCodegenTest.scala index 19e1a0b2066f1..02859f707f8b0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/OperatorFusionCodegenTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/OperatorFusionCodegenTest.scala @@ -23,21 +23,21 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.plan.stats.TableStats import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} import com.google.common.collect.ImmutableSet -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util /** Tests for operator fusion codegen. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class OperatorFusionCodegenTest(fusionCodegenEnabled: Boolean) extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource( "T1", @@ -75,21 +75,21 @@ class OperatorFusionCodegenTest(fusionCodegenEnabled: Boolean) extends TableTest Boolean.box(fusionCodegenEnabled)) } - @Test + @TestTemplate def testHashAggAsMutltipleInputRoot(): Unit = { util.verifyExecPlan( "SELECT a1, b1, a2, b2, COUNT(c1) FROM " + "(SELECT * FROM T1, T2 WHERE a1 = b2) t GROUP BY a1, b1, a2, b2") } - @Test + @TestTemplate def testLocalHashAggAsMutltipleInputRoot(): Unit = { util.verifyExecPlan( "SELECT a2, b2, a3, b3, COUNT(c2), AVG(d3) FROM " + "(SELECT * FROM T2, T3 WHERE b2 = a3) t GROUP BY a2, b2, a3, b3") } - @Test + @TestTemplate def testCalcAsMutltipleInputRoot(): Unit = { util.verifyExecPlan( "SELECT a1, b1, a2, b2, a3, b3, COUNT(c1) FROM " + @@ -98,7 +98,7 @@ class OperatorFusionCodegenTest(fusionCodegenEnabled: Boolean) extends TableTest } object OperatorFusionCodegenTest { - @Parameterized.Parameters(name = "fusionCodegenEnabled={0}") + @Parameters(name = "fusionCodegenEnabled={0}") def parameters(): util.Collection[Boolean] = { util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala index 2d239cb0543bc..80ddcb839ec07 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class PartitionableSinkTest extends TableTestBase { @@ -66,14 +67,20 @@ class PartitionableSinkTest extends TableTestBase { util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable") } - @Test(expected = classOf[ValidationException]) + @Test def testWrongStatic(): Unit = { - util.verifyExecPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExecPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")) } - @Test(expected = classOf[ValidationException]) + @Test def testWrongFields(): Unit = { - util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSourceTest.scala index 007648c9e32dc..d2d4935e63b83 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSourceTest.scala @@ -21,22 +21,22 @@ import org.apache.flink.table.catalog.{CatalogPartitionImpl, CatalogPartitionSpe import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.factories.TestValuesCatalog import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class PartitionableSourceTest(val sourceFetchPartitions: Boolean, val useCatalogFilter: Boolean) extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val partitionableTable = """ @@ -112,50 +112,50 @@ class PartitionableSourceTest(val sourceFetchPartitions: Boolean, val useCatalog } } - @Test + @TestTemplate def testSimplePartitionFieldPredicate1(): Unit = { util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE part1 = 'A'") } - @Test + @TestTemplate def testPartialPartitionFieldPredicatePushDown(): Unit = { util.verifyExecPlan( "SELECT * FROM PartitionableTable WHERE (id > 2 OR part1 = 'A') AND part2 > 1") } - @Test + @TestTemplate def testWithUdfAndVirtualColumn(): Unit = { util.addFunction("MyUdf", Func1) util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE id > 2 AND MyUdf(part2) < 3") } - @Test + @TestTemplate def testUnconvertedExpression(): Unit = { util.verifyExecPlan("select * from PartitionableTable where trim(part1) = 'A' and part2 > 1") } - @Test + @TestTemplate def testPushDownPartitionAndFiltersContainPartitionKeys(): Unit = { util.verifyExecPlan( "select * from PartitionableAndFilterableTable " + "where part1 = 'A' and part2 > 1 and id > 1") } - @Test + @TestTemplate def testPushDownPartitionAndFiltersContainPartitionKeysWithSingleProjection(): Unit = { util.verifyExecPlan( "select name from PartitionableAndFilterableTable " + "where part1 = 'A' and part2 > 1 and id > 1") } - @Test + @TestTemplate def testPushDownNonExistentPartition(): Unit = { util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE part2 = 4") } } object PartitionableSourceTest { - @Parameterized.Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") + @Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") def parameters(): util.Collection[Array[Any]] = { Seq[Array[Any]]( Array(true, false), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RankTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RankTest.scala index bc18eebfe0b01..49ff3aaa033bd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RankTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RankTest.scala @@ -21,8 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.assertj.core.api.Assertions.assertThatThrownBy -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test class RankTest extends TableTestBase { @@ -59,7 +59,7 @@ class RankTest extends TableTestBase { "please re-check the over window statement.") } - @Test(expected = classOf[RuntimeException]) + @Test def testRowNumberWithMultiGroups(): Unit = { val sqlQuery = """ @@ -67,19 +67,22 @@ class RankTest extends TableTestBase { | ROW_NUMBER() over (partition by b) as b | FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[ValidationException]) + @Test def testRankWithoutOrderBy(): Unit = { val sqlQuery = """ |SELECT RANK() over (partition by a) FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[ValidationException]) + @Test def testRankWithMultiGroups(): Unit = { val sqlQuery = """ @@ -87,19 +90,21 @@ class RankTest extends TableTestBase { | RANK() over (partition by b) as b | FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[ValidationException]) + @Test def testDenseRankWithoutOrderBy(): Unit = { val sqlQuery = """ |SELECT dense_rank() over (partition by a) FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[ValidationException]) + @Test def testDenseRankWithMultiGroups(): Unit = { val sqlQuery = """ @@ -107,7 +112,8 @@ class RankTest extends TableTestBase { | DENSE_RANK() over (partition by b) as b | FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala index 1e803359eec05..1abdd4e80074b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala @@ -29,13 +29,13 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunction import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase} import com.google.common.collect.ImmutableSet -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class RemoveCollationTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource( "x", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala index 66839e280d941..c65f341a29a77 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.table.planner.plan.rules.physical.batch.{BatchPhysicalJo import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class RemoveShuffleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource( "x", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala index eaa5998d73c5e..943e6c766e01b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala @@ -25,13 +25,14 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} class SetOperatorsTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig.set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") util.addTableSource[(Int, Long, String)]("T1", 'a, 'b, 'c) @@ -39,16 +40,19 @@ class SetOperatorsTest extends TableTestBase { util.addTableSource[(Int, Long, Int, String, Long)]("T3", 'a, 'b, 'd, 'c, 'e) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentColumnSize(): Unit = { // must fail. Union inputs have different column size. - util.verifyExecPlan("SELECT * FROM T1 UNION ALL SELECT * FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT * FROM T1 UNION ALL SELECT * FROM T3")) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentFieldTypes(): Unit = { // must fail. Union inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 UNION ALL SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.verifyExecPlan("SELECT a, b, c FROM T1 UNION ALL SELECT d, c, e FROM T3")) } @Test @@ -56,10 +60,12 @@ class SetOperatorsTest extends TableTestBase { util.verifyExecPlan("SELECT c FROM T1 INTERSECT ALL SELECT f FROM T2") } - @Test(expected = classOf[ValidationException]) + @Test def testIntersectDifferentFieldTypes(): Unit = { // must fail. Intersect inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 INTERSECT SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.verifyExecPlan("SELECT a, b, c FROM T1 INTERSECT SELECT d, c, e FROM T3")) } @Test @@ -67,10 +73,11 @@ class SetOperatorsTest extends TableTestBase { util.verifyExecPlan("SELECT c FROM T1 EXCEPT ALL SELECT f FROM T2") } - @Test(expected = classOf[ValidationException]) + @Test def testMinusDifferentFieldTypes(): Unit = { // must fail. Minus inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 EXCEPT SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT a, b, c FROM T1 EXCEPT SELECT d, c, e FROM T3")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala index e3027e4e83ea4..924417803693c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SortLimitTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala index 9722d139d963e..32149d42f9c29 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SortTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala index 4bda75a5b3fec..5718002822f01 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala @@ -28,13 +28,13 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunction import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.runtime.functions.aggregate.FirstValueAggFunction -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class SubplanReuseTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala index 932f9153583fd..2b2e658cb3ac4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala @@ -18,18 +18,18 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.scala._ -import org.apache.flink.table.api import org.apache.flink.table.api._ import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class TableScanTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.registerFunction("my_udf", Func0) @@ -96,11 +96,12 @@ class TableScanTest extends TableTestBase { | 'bounded' = 'false' |) """.stripMargin) - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Querying an unbounded table 'default_catalog.default_database.src' in batch mode is not " + - "allowed. The table source is unbounded.") - util.verifyExecPlan("SELECT * FROM src WHERE a > 1") + + assertThatThrownBy(() => util.verifyExecPlan("SELECT * FROM src WHERE a > 1")) + .hasMessageContaining( + "Querying an unbounded table 'default_catalog.default_database.src' in batch mode is not " + + "allowed. The table source is unbounded.") + .isInstanceOf[ValidationException] } @Test @@ -116,12 +117,13 @@ class TableScanTest extends TableTestBase { | 'changelog-mode' = 'I,UA,UB' |) """.stripMargin) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Querying a table in batch mode is currently only possible for INSERT-only table sources. " + - "But the source for table 'default_catalog.default_database.src' produces other changelog " + - "messages than just INSERT.") - util.verifyExecPlan("SELECT * FROM src WHERE a > 1") + + assertThatThrownBy(() => util.verifyExecPlan("SELECT * FROM src WHERE a > 1")) + .hasMessageContaining( + "Querying a table in batch mode is currently only possible for INSERT-only table sources. " + + "But the source for table 'default_catalog.default_database.src' produces other changelog " + + "messages than just INSERT.") + .isInstanceOf[TableException] } @Test @@ -138,12 +140,13 @@ class TableScanTest extends TableTestBase { | 'changelog-mode' = 'UA,D' |) """.stripMargin) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Querying a table in batch mode is currently only possible for INSERT-only table sources. " + - "But the source for table 'default_catalog.default_database.src' produces other changelog " + - "messages than just INSERT.") - util.verifyExecPlan("SELECT * FROM src WHERE a > 1") + + assertThatThrownBy(() => util.verifyExecPlan("SELECT * FROM src WHERE a > 1")) + .hasMessageContaining( + "Querying a table in batch mode is currently only possible for INSERT-only table sources. " + + "But the source for table 'default_catalog.default_database.src' produces other changelog " + + "messages than just INSERT.") + .isInstanceOf[TableException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala index 49a8190cb282c..3a950be62bfd5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala @@ -19,15 +19,11 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.apache.flink.table.api.config.TableConfigOptions -import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder -import org.apache.flink.table.planner.runtime.utils.BatchAbstractTestBase -import org.apache.flink.table.planner.runtime.utils.TestData.smallData3 -import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} +import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType} -import org.junit.{Assert, Test} +import org.junit.jupiter.api.Test class TableSinkTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala index 65ccd9b75c244..418258d0a331e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala @@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.utils._ -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class TableSourceTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val ddl = s""" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnionTest.scala index 1646da14c530a..f3b7bce440982 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnionTest.scala @@ -21,14 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} // TODO add more union case after aggregation and join supported class UnionTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable1", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.scala index f8e57bb005ae3..6bcaedd89bc94 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class ValuesTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala index d7565eedb92d2..b8c67d9f63d9f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} import java.sql.Timestamp @@ -29,7 +30,7 @@ class WindowTableFunctionTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTableSource[(Timestamp, Long, Int, String)]("MyTable", 'ts, 'a, 'b, 'c) util.addTableSource[(Int, Long, String, Int, Timestamp)]("MyTable1", 'a, 'b, 'c, 'd, 'ts) @@ -51,12 +52,12 @@ class WindowTableFunctionTest extends TableTestBase { |SELECT * |FROM TABLE(TUMBLE(TABLE MyTable1, DESCRIPTOR(b), INTERVAL '15' MINUTE)) |""".stripMargin - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "The window function TUMBLE(TABLE table_name, DESCRIPTOR(timecol), datetime interval" - + "[, datetime interval]) requires the timecol to be TIMESTAMP or TIMESTAMP_LTZ, " - + "but is BIGINT.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining( + "The window function TUMBLE(TABLE table_name, DESCRIPTOR(timecol), datetime interval" + + "[, datetime interval]) requires the timecol to be TIMESTAMP or TIMESTAMP_LTZ, " + + "but is BIGINT.") + .isInstanceOf[ValidationException] } @Test @@ -76,9 +77,10 @@ class WindowTableFunctionTest extends TableTestBase { |SELECT * |FROM TABLE(TUMBLE(TABLE MyTable2, DESCRIPTOR(c), INTERVAL '15' MINUTE)) |""".stripMargin - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Processing time Window TableFunction is not supported yet.") - util.verifyExplain(sql) + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TableFunction is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -98,9 +100,10 @@ class WindowTableFunctionTest extends TableTestBase { |SELECT * |FROM TABLE(HOP(TABLE MyTable2, DESCRIPTOR(c), INTERVAL '1' HOUR, INTERVAL '2' HOUR)) |""".stripMargin - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Processing time Window TableFunction is not supported yet.") - util.verifyExplain(sql) + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TableFunction is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -122,9 +125,10 @@ class WindowTableFunctionTest extends TableTestBase { |FROM TABLE( | CUMULATE(TABLE MyTable2, DESCRIPTOR(c), INTERVAL '10' MINUTE, INTERVAL '1' HOUR)) |""".stripMargin - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Processing time Window TableFunction is not supported yet.") - util.verifyExplain(sql) + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TableFunction is not supported yet.") + .isInstanceOf[TableException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala index a4d431d35f318..3b3d092d94411 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala @@ -24,7 +24,8 @@ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{Va import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} import org.apache.flink.table.runtime.typeutils.DecimalDataTypeInfo -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.TestTemplate abstract class AggregateTestBase extends TableTestBase { @@ -63,7 +64,7 @@ abstract class AggregateTestBase extends TableTestBase { ) util.addTableSource[(Int, Long, String)]("MyTable1", 'a, 'b, 'c) - @Test + @TestTemplate def testAvg(): Unit = { util.verifyRelPlanWithType(""" |SELECT AVG(`byte`), @@ -78,7 +79,7 @@ abstract class AggregateTestBase extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testSum(): Unit = { util.verifyRelPlanWithType(""" |SELECT SUM(`byte`), @@ -93,7 +94,7 @@ abstract class AggregateTestBase extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testCount(): Unit = { util.verifyRelPlanWithType(""" |SELECT COUNT(`byte`), @@ -113,12 +114,12 @@ abstract class AggregateTestBase extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testCountStart(): Unit = { util.verifyRelPlanWithType("SELECT COUNT(*) FROM MyTable") } - @Test + @TestTemplate def testCountStartWithProjectPushDown(): Unit = { // the test values table source supports projection push down by default util.tableEnv.executeSql(""" @@ -133,15 +134,16 @@ abstract class AggregateTestBase extends TableTestBase { util.verifyRelPlanWithType("SELECT COUNT(*) FROM src") } - @Test + @TestTemplate def testCannotCountOnMultiFields(): Unit = { val sql = "SELECT b, COUNT(a, c) FROM MyTable1 GROUP BY b" - thrown.expect(classOf[TableException]) - thrown.expectMessage("We now only support the count of one field") - util.verifyExecPlan(sql) + + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining("We now only support the count of one field") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testMinWithFixLengthType(): Unit = { util.verifyRelPlanWithType(""" |SELECT MIN(`byte`), @@ -160,12 +162,12 @@ abstract class AggregateTestBase extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testMinWithVariableLengthType(): Unit = { util.verifyRelPlanWithType("SELECT MIN(`string`) FROM MyTable") } - @Test + @TestTemplate def testMaxWithFixLengthType(): Unit = { util.verifyRelPlanWithType(""" |SELECT MAX(`byte`), @@ -184,55 +186,55 @@ abstract class AggregateTestBase extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testMaxWithVariableLengthType(): Unit = { util.verifyRelPlanWithType("SELECT MAX(`string`) FROM MyTable") } - @Test + @TestTemplate def testAggregateWithoutFunction(): Unit = { util.verifyExecPlan("SELECT a, b FROM MyTable1 GROUP BY a, b") } - @Test + @TestTemplate def testAggregateWithoutGroupBy(): Unit = { util.verifyExecPlan("SELECT AVG(a), SUM(b), COUNT(c) FROM MyTable1") } - @Test + @TestTemplate def testAggregateWithFilter(): Unit = { util.verifyExecPlan("SELECT AVG(a), SUM(b), COUNT(c) FROM MyTable1 WHERE a = 1") } - @Test + @TestTemplate def testAggregateWithFilterOnNestedFields(): Unit = { util.addTableSource[(Int, Long, (Int, Long))]("MyTable2", 'a, 'b, 'c) util.verifyExecPlan("SELECT AVG(a), SUM(b), COUNT(c), SUM(c._1) FROM MyTable2 WHERE a = 1") } - @Test + @TestTemplate def testGroupAggregate(): Unit = { util.verifyExecPlan("SELECT a, SUM(b), COUNT(c) FROM MyTable1 GROUP BY a") } - @Test + @TestTemplate def testGroupAggregateWithFilter(): Unit = { util.verifyExecPlan("SELECT a, SUM(b), count(c) FROM MyTable1 WHERE a = 1 GROUP BY a") } - @Test + @TestTemplate def testAggNotSupportMerge(): Unit = { util.addFunction("var_sum", new VarSum2AggFunction) util.verifyExecPlan("SELECT b, var_sum(a) FROM MyTable1 GROUP BY b") } - @Test + @TestTemplate def testPojoAccumulator(): Unit = { util.addFunction("var_sum", new VarSum1AggFunction) util.verifyExecPlan("SELECT b, var_sum(a) FROM MyTable1 GROUP BY b") } - @Test + @TestTemplate def testGroupByWithConstantKey(): Unit = { val sql = """ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala index d0f8720a1ff67..c6dcac1a54bcb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala @@ -22,22 +22,23 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, CountAggFunction, TableTestBase} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.sql.Timestamp import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) @@ -56,59 +57,61 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase |""".stripMargin) } - @Test + @TestTemplate def testHopWindowNoOffset(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable2 " + "GROUP BY HOP(ts, INTERVAL '1' HOUR, INTERVAL '2' HOUR, TIME '10:00:00')" - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("HOP window with alignment is not supported yet.") - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("HOP window with alignment is not supported yet.") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testSessionWindowNoOffset(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable2 " + "GROUP BY SESSION(ts, INTERVAL '2' HOUR, TIME '10:00:00')" - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("SESSION window with alignment is not supported yet.") - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("SESSION window with alignment is not supported yet.") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testVariableWindowSize(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Only constant window descriptors are supported") - util.verifyExecPlan( - "SELECT COUNT(*) FROM MyTable2 GROUP BY TUMBLE(ts, b * INTERVAL '1' MINUTE)") + assertThatThrownBy( + () => + util.verifyExecPlan( + "SELECT COUNT(*) FROM MyTable2 GROUP BY TUMBLE(ts, b * INTERVAL '1' MINUTE)")) + .hasMessageContaining("Only constant window descriptors are supported") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testTumbleWindowWithInvalidUdAggArgs(): Unit = { val weightedAvg = new WeightedAvgWithMerge util.addFunction("weightedAvg", weightedAvg) val sql = "SELECT weightedAvg(c, a) AS wAvg FROM MyTable2 " + "GROUP BY TUMBLE(ts, INTERVAL '4' MINUTE)" - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "SQL validation failed. " + + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining("SQL validation failed. " + "Given parameters of function 'weightedAvg' do not match any signature.") - util.verifyExecPlan(sql) + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testWindowProctime(): Unit = { val sqlQuery = "SELECT TUMBLE_PROCTIME(ts, INTERVAL '4' MINUTE) FROM MyTable2 " + "GROUP BY TUMBLE(ts, INTERVAL '4' MINUTE), c" - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("PROCTIME window property is not supported in batch queries.") - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("PROCTIME window property is not supported in batch queries.") + .isInstanceOf[ValidationException] } - @Test(expected = classOf[AssertionError]) + @TestTemplate def testWindowAggWithGroupSets(): Unit = { // TODO supports group sets // currently, the optimized plan is not collect, and an exception will be thrown in code-gen @@ -119,48 +122,50 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase |FROM MyTable1 | GROUP BY rollup(TUMBLE(ts, INTERVAL '15' MINUTE), b) """.stripMargin - util.verifyRelPlanNotExpected(sql, "TUMBLE(ts") + + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sql, "TUMBLE(ts")) } - @Test + @TestTemplate def testNoGroupingTumblingWindow(): Unit = { val sqlQuery = "SELECT AVG(c), SUM(a) FROM MyTable GROUP BY TUMBLE(b, INTERVAL '3' SECOND)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTumblingWindowSortAgg1(): Unit = { val sqlQuery = "SELECT MAX(c) FROM MyTable1 GROUP BY a, TUMBLE(ts, INTERVAL '3' SECOND)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTumblingWindowSortAgg2(): Unit = { val sqlQuery = "SELECT AVG(c), countFun(a) FROM MyTable " + "GROUP BY a, d, TUMBLE(b, INTERVAL '3' SECOND)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTumblingWindowHashAgg1(): Unit = { val sqlQuery = "SELECT COUNT(c) FROM MyTable1 GROUP BY a, TUMBLE(ts, INTERVAL '3' SECOND)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTumblingWindowHashAgg2(): Unit = { val sql = "SELECT AVG(c), COUNT(a) FROM MyTable GROUP BY a, d, TUMBLE(b, INTERVAL '3' SECOND)" util.verifyExecPlan(sql) } - @Test + @TestTemplate def testNonPartitionedTumblingWindow(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable2 GROUP BY TUMBLE(ts, INTERVAL '2' HOUR)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testPartitionedTumblingWindow(): Unit = { val sqlQuery = """ @@ -176,7 +181,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTumblingWindowWithUdAgg(): Unit = { util.addFunction("weightedAvg", new WeightedAvgWithMerge) val sql = "SELECT weightedAvg(b, a) AS wAvg FROM MyTable2 " + @@ -184,16 +189,16 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sql) } - @Test + @TestTemplate def testTumblingWindowWithProctime(): Unit = { val sql = "select sum(a), max(b) from MyTable3 group by TUMBLE(c, INTERVAL '1' SECOND)" - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Window can not be defined over a proctime attribute column for batch mode") - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining( + "Window can not be defined over a proctime attribute column for batch mode") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testNoGroupingSlidingWindow(): Unit = { val sqlQuery = """ @@ -206,42 +211,42 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowSortAgg1(): Unit = { val sqlQuery = "SELECT MAX(c) FROM MyTable1 " + "GROUP BY a, HOP(ts, INTERVAL '3' SECOND, INTERVAL '1' HOUR)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowSortAgg2(): Unit = { val sqlQuery = "SELECT MAX(c) FROM MyTable1 " + "GROUP BY b, HOP(ts, INTERVAL '0.111' SECOND(1,3), INTERVAL '1' SECOND)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowSortAgg3(): Unit = { val sqlQuery = "SELECT countFun(c) FROM MyTable " + " GROUP BY a, d, HOP(b, INTERVAL '3' SECOND, INTERVAL '1' HOUR)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowSortAggWithPaneOptimization(): Unit = { val sqlQuery = "SELECT COUNT(c) FROM MyTable1 " + "GROUP BY a, HOP(ts, INTERVAL '3' SECOND, INTERVAL '1' HOUR)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowHashAgg(): Unit = { val sqlQuery = "SELECT count(c) FROM MyTable1 " + "GROUP BY b, HOP(ts, INTERVAL '3' SECOND, INTERVAL '1' HOUR)" util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testNonPartitionedSlidingWindow(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " + @@ -251,7 +256,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testPartitionedSlidingWindow(): Unit = { val sqlQuery = "SELECT " + @@ -267,7 +272,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSlidingWindowWithProctime(): Unit = { val sql = s""" @@ -275,22 +280,22 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase |from MyTable3 |group by HOP(c, INTERVAL '1' SECOND, INTERVAL '1' MINUTE) |""".stripMargin - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Window can not be defined over a proctime attribute column for batch mode") - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining("Window can not be defined over a proctime attribute column for " + + "batch mode") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate // TODO session window is not supported now def testNonPartitionedSessionWindow(): Unit = { val sqlQuery = "SELECT COUNT(*) AS cnt FROM MyTable2 GROUP BY SESSION(ts, INTERVAL '30' MINUTE)" - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Cannot generate a valid execution plan for the given query") - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } - @Test + @TestTemplate // TODO session window is not supported now def testPartitionedSessionWindow(): Unit = { val sqlQuery = @@ -304,12 +309,12 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase |FROM MyTable2 | GROUP BY SESSION(ts, INTERVAL '12' HOUR), c, d """.stripMargin - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Cannot generate a valid execution plan for the given query") - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testSessionWindowWithProctime(): Unit = { val sql = s""" @@ -317,13 +322,13 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase |from MyTable3 |group by SESSION(c, INTERVAL '1' MINUTE) |""".stripMargin - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Window can not be defined over a proctime attribute column for batch mode") - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining( + "Window can not be defined over a proctime attribute column for batch mode") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testWindowEndOnly(): Unit = { val sqlQuery = "SELECT TUMBLE_END(ts, INTERVAL '4' MINUTE) FROM MyTable2 " + @@ -331,7 +336,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testExpressionOnWindowHavingFunction(): Unit = { val sql = """ @@ -346,7 +351,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sql) } - @Test + @TestTemplate def testDecomposableAggFunctions(): Unit = { val sql = """ @@ -363,7 +368,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase } // TODO: fix the plan regression when FLINK-19668 is fixed. - @Test + @TestTemplate def testReturnTypeInferenceForWindowAgg() = { val sql = @@ -385,7 +390,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase util.verifyExecPlan(sql) } - @Test + @TestTemplate def testWindowAggregateWithDifferentWindows(): Unit = { // This test ensures that the LogicalWindowAggregate node' digest contains the window specs. // This allows the planner to make the distinction between similar aggregations using different @@ -415,7 +420,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase object GroupWindowTest { - @Parameterized.Parameters(name = "aggStrategy={0}") + @Parameters(name = "aggStrategy={0}") def parameters(): util.Collection[AggregatePhaseStrategy] = { Seq[AggregatePhaseStrategy]( AggregatePhaseStrategy.AUTO, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.scala index 010c648433d26..55cd258d578d5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.scala @@ -22,8 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.Test import java.sql.Date @@ -501,7 +501,7 @@ class GroupingSetsTest extends TableTestBase { val table2 = util.tableEnv.sqlQuery(sql2) val optimized1 = util.getPlanner.optimize(TableTestUtil.toRelNode(table1)) val optimized2 = util.getPlanner.optimize(TableTestUtil.toRelNode(table2)) - assertEquals(FlinkRelOptUtil.toString(optimized1), FlinkRelOptUtil.toString(optimized2)) + assertThat(FlinkRelOptUtil.toString(optimized2)).isEqualTo(FlinkRelOptUtil.toString(optimized1)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala index 185b53bcadab2..e53e8845ac791 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala @@ -21,19 +21,20 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.plan.utils.OperatorType import org.apache.flink.table.planner.utils.AggregatePhaseStrategy +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.Before -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTestBase { - @Before + @BeforeEach def before(): Unit = { // disable sort agg util.tableEnv.getConfig @@ -42,28 +43,31 @@ class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) } + @TestTemplate override def testMinWithVariableLengthType(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMinWithVariableLengthType() + assertThatThrownBy(() => super.testMinWithVariableLengthType()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } + @TestTemplate override def testMaxWithVariableLengthType(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMaxWithVariableLengthType() + assertThatThrownBy(() => super.testMaxWithVariableLengthType()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } + @TestTemplate override def testPojoAccumulator(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testPojoAccumulator() + assertThatThrownBy(() => super.testPojoAccumulator()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } object HashAggregateTest { - @Parameterized.Parameters(name = "aggStrategy={0}") + @Parameters(name = "aggStrategy={0}") def parameters(): util.Collection[AggregatePhaseStrategy] = { Seq[AggregatePhaseStrategy]( AggregatePhaseStrategy.AUTO, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.scala index d0753274fc0c8..e95d40c8c8483 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.OverAgg0 import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -216,7 +217,7 @@ class OverAggregateTest extends TableTestBase { util.verifyExecPlan(sqlQuery) } - @Test(expected = classOf[ValidationException]) + @Test def testOverWindowRangeProhibitType(): Unit = { val sqlQuery = """ @@ -224,7 +225,8 @@ class OverAggregateTest extends TableTestBase { | COUNT(*) OVER (PARTITION BY c ORDER BY c RANGE BETWEEN -1 PRECEDING AND 10 FOLLOWING) |FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } @Test @@ -253,7 +255,7 @@ class OverAggregateTest extends TableTestBase { util.verifyExecPlan(sqlQuery) } - @Test(expected = classOf[ValidationException]) + @Test def testRowsWindowWithNegative(): Unit = { val sqlQuery = """ @@ -261,7 +263,8 @@ class OverAggregateTest extends TableTestBase { | COUNT(*) OVER (PARTITION BY c ORDER BY a ROWS BETWEEN -1 PRECEDING AND 10 FOLLOWING) |FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } @Test @@ -320,7 +323,7 @@ class OverAggregateTest extends TableTestBase { util.verifyExecPlan(sqlQuery) } - @Test(expected = classOf[RuntimeException]) + @Test def testDistinct(): Unit = { val sqlQuery = """ @@ -328,22 +331,27 @@ class OverAggregateTest extends TableTestBase { | OVER (PARTITION BY c ORDER BY a RANGE BETWEEN -1 FOLLOWING AND 10 FOLLOWING) |FROM MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation(): Unit = { util.addFunction("overAgg", new OverAgg0) - util.verifyExecPlan("SELECT overAgg(b, a) FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT overAgg(b, a) FROM MyTable")) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation2(): Unit = { util.addTableSource[(Int, Long, String, Timestamp)]("T", 'a, 'b, 'c, 'ts) util.addFunction("overAgg", new OverAgg0) - util.verifyExecPlan("SELECT overAgg(b, a) FROM T GROUP BY TUMBLE(ts, INTERVAL '2' HOUR)") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExecPlan("SELECT overAgg(b, a) FROM T GROUP BY TUMBLE(ts, INTERVAL '2' HOUR)")) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala index c1fb17d3493f9..86dfdd18b8dab 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala @@ -20,19 +20,19 @@ package org.apache.flink.table.planner.plan.batch.sql.agg import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.plan.utils.OperatorType import org.apache.flink.table.planner.utils.AggregatePhaseStrategy +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTestBase { - @Before + @BeforeEach def before(): Unit = { // disable hash agg util.tableEnv.getConfig @@ -41,7 +41,7 @@ class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) } - @Test + @TestTemplate def testApproximateCountDistinct(): Unit = { val sql = """ @@ -66,7 +66,7 @@ class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe object SortAggregateTest { - @Parameterized.Parameters(name = "aggStrategy={0}") + @Parameters(name = "aggStrategy={0}") def parameters(): util.Collection[AggregatePhaseStrategy] = { Seq[AggregatePhaseStrategy]( AggregatePhaseStrategy.AUTO, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala index bab7c14257fb8..0f3c494ee5e6a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class BroadcastHashJoinTest extends JoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue)) @@ -35,106 +36,106 @@ class BroadcastHashJoinTest extends JoinTestBase { @Test override def testInnerJoinWithoutJoinPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInnerJoinWithoutJoinPred() + assertThatThrownBy(() => super.testInnerJoinWithoutJoinPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinNoEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinNoEquiPred() + assertThatThrownBy(() => super.testLeftOuterJoinNoEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnTrue() + assertThatThrownBy(() => super.testLeftOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnFalse() + assertThatThrownBy(() => super.testLeftOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnTrue() + assertThatThrownBy(() => super.testRightOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnFalse() + assertThatThrownBy(() => super.testRightOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testRightOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithEquiPred() + assertThatThrownBy(() => super.testFullOuterJoinWithEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithEquiAndLocalPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithEquiAndLocalPred() + assertThatThrownBy(() => super.testFullOuterJoinWithEquiAndLocalPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithEquiAndNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithEquiAndNonEquiPred() + assertThatThrownBy(() => super.testFullOuterJoinWithEquiAndNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testFullOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnFalse() + assertThatThrownBy(() => super.testFullOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterWithUsing(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterWithUsing() + assertThatThrownBy(() => super.testFullOuterWithUsing()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnTrue() + assertThatThrownBy(() => super.testFullOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testCrossJoin(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testCrossJoin() + assertThatThrownBy(() => super.testCrossJoin()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala index 8eef862af55c1..bae55c7b394b1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue)) @@ -39,141 +40,141 @@ class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase { // because NestedLoopJoin(non-singleRowJoin) is disabled. @Test override def testNotInWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInNotInExistsNotExists(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInNotInExistsNotExists() + assertThatThrownBy(() => super.testInNotInExistsNotExists()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithUncorrelated_JoinInSubQuery(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithUncorrelated_JoinInSubQuery() + assertThatThrownBy(() => super.testExistsWithUncorrelated_JoinInSubQuery()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiExistsWithCorrelate1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiExistsWithCorrelate1() + assertThatThrownBy(() => super.testMultiExistsWithCorrelate1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_MultiFields(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_MultiFields() + assertThatThrownBy(() => super.testNotInWithUncorrelated_MultiFields()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiNotInWithCorrelated(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiNotInWithCorrelated() + assertThatThrownBy(() => super.testMultiNotInWithCorrelated()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithCorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithCorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithCorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithUncorrelated_JoinInSubQuery(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithUncorrelated_JoinInSubQuery() + assertThatThrownBy(() => super.testInWithUncorrelated_JoinInSubQuery()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.scala index 8b00f7112ebff..2649202ed7d3b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.scala @@ -19,12 +19,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.planner.plan.common.JoinReorderTestBase import org.apache.flink.table.planner.utils.TableTestUtil +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.extension.ExtendWith /** The plan test for join reorder in batch mode. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class JoinReorderTest(isBushyJoinReorder: Boolean) extends JoinReorderTestBase(isBushyJoinReorder) { override protected def getTableTestUtil: TableTestUtil = batchTestUtil() } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala index 4f412e21d2580..f4c0301d13a7b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test abstract class JoinTestBase extends TableTestBase { @@ -29,9 +30,10 @@ abstract class JoinTestBase extends TableTestBase { util.addTableSource[(Int, Long, String)]("MyTable1", 'a, 'b, 'c) util.addTableSource[(Int, Long, Int, String, Long)]("MyTable2", 'd, 'e, 'f, 'g, 'h) - @Test(expected = classOf[ValidationException]) + @Test def testJoinNonExistingKey(): Unit = { - util.verifyExecPlan("SELECT c, g FROM MyTable1, MyTable2 WHERE foo = e") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT c, g FROM MyTable1, MyTable2 WHERE foo = e")) } @Test @@ -60,16 +62,18 @@ abstract class JoinTestBase extends TableTestBase { util.verifyExecPlan("SELECT d, e, f FROM MyTable1 LEFT JOIN MyTable2 ON a = d where d = null") } - @Test(expected = classOf[TableException]) + @Test def testJoinNonMatchingKeyTypes(): Unit = { // INTEGER and VARCHAR(65536) does not have common type now - util.verifyExecPlan("SELECT c, g FROM MyTable1, MyTable2 WHERE a = g") + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan("SELECT c, g FROM MyTable1, MyTable2 WHERE a = g")) } - @Test(expected = classOf[ValidationException]) + @Test def testJoinWithAmbiguousFields(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable0", 'a0, 'b0, 'c) - util.verifyExecPlan("SELECT a, c FROM MyTable1, MyTable0 WHERE a = a0") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT c FROM MyTable1, MyTable0 WHERE a = a0")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala index 080f3654f2563..7121e02d723bb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala @@ -24,24 +24,26 @@ import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.plan.stream.sql.join.TestTemporalTable import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} import _root_.java.lang.{Boolean => JBoolean} import _root_.java.util.{Collection => JCollection} import _root_.scala.collection.JavaConversions._ -import org.junit.{Assume, Before, Test} -import org.junit.Assert.{assertTrue, fail} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType, assertThatThrownBy, catchThrowable} +import org.assertj.core.api.Assumptions.assumeThat +import org.assertj.core.api.ThrowableAssert.ThrowingCallable +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith /** * The physical plans for legacy [[org.apache.flink.table.sources.LookupableTableSource]] and new * [[org.apache.flink.table.connector.source.LookupTableSource]] should be identical. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { private val testUtil = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { testUtil.addDataStream[(Int, String, Long)]("T0", 'a, 'b, 'c) testUtil.addDataStream[(Int, String, Long, Double)]("T1", 'a, 'b, 'c, 'd) @@ -76,7 +78,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { } } - @Test + @TestTemplate def testJoinInvalidJoinTemporalTable(): Unit = { // must follow a period specification expectExceptionThrown( @@ -113,7 +115,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { ) } - @Test + @TestTemplate def testNotDistinctFromInJoinCondition(): Unit = { // does not support join condition contains `IS NOT DISTINCT` @@ -135,13 +137,8 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { ) } - @Test + @TestTemplate def testPythonUDFInJoinCondition(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Only inner join condition with equality predicates supports the " + - "Python UDF taking the inputs from the left table and the right table at the same time, " + - "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") testUtil.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) val sql = """ @@ -149,10 +146,15 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { |LEFT OUTER JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D |ON T.a = D.id AND D.age = 10 AND pyFunc(D.age, T.a) > 100 """.stripMargin - testUtil.verifyExecPlan(sql) + + assertThatThrownBy(() => testUtil.verifyExecPlan(sql)) + .hasMessageContaining("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testLogicalPlan(): Unit = { val sql1 = """ @@ -181,37 +183,37 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyRelPlan(sql) } - @Test + @TestTemplate def testLogicalPlanWithImplicitTypeCast(): Unit = { val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig) programs.remove(FlinkBatchProgram.PHYSICAL) testUtil.replaceBatchProgram(programs) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "implicit type conversion between VARCHAR(2147483647) and INTEGER " + + assertThatThrownBy( + () => + testUtil.verifyRelPlan( + "SELECT * FROM MyTable AS T JOIN LookupTable " + + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id")) + .hasMessageContaining("implicit type conversion between VARCHAR(2147483647) and INTEGER " + "is not supported on join's condition now") - - testUtil.verifyRelPlan( - "SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testJoinTemporalTable(): Unit = { val sql = "SELECT * FROM MyTable AS T JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id" testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testLeftJoinTemporalTable(): Unit = { val sql = "SELECT * FROM MyTable AS T LEFT JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id" testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithNestedQuery(): Unit = { val sql = "SELECT * FROM " + "(SELECT a, b, proctime FROM MyTable WHERE c > 1000) AS T " + @@ -220,7 +222,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithProjectionPushDown(): Unit = { val sql = """ @@ -232,7 +234,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithFilterPushDown(): Unit = { val sql = """ @@ -244,7 +246,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testAvoidAggregatePushDown(): Unit = { val sql1 = """ @@ -270,12 +272,8 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithTrueCondition(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Temporal table join requires an equality condition on fields of " + - "table [default_catalog.default_database.LookupTable]") val sql = """ |SELECT * FROM MyTable AS T @@ -283,13 +281,17 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { |ON true |WHERE T.c > 1000 """.stripMargin - testUtil.verifyExplain(sql) + + assertThatThrownBy(() => testUtil.verifyExplain(sql)) + .hasMessageContaining("Temporal table join requires an equality condition on fields of " + + "table [default_catalog.default_database.LookupTable]") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testJoinTemporalTableWithComputedColumn(): Unit = { // Computed column do not support in legacyTableSource. - Assume.assumeFalse(legacyTableSource) + assumeThat(legacyTableSource).isFalse val sql = """ |SELECT @@ -301,10 +303,10 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithComputedColumnAndPushDown(): Unit = { // Computed column do not support in legacyTableSource. - Assume.assumeFalse(legacyTableSource) + assumeThat(legacyTableSource).isFalse val sql = """ |SELECT @@ -316,7 +318,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { testUtil.verifyExecPlan(sql) } - @Test + @TestTemplate def testReusing(): Unit = { testUtil.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) @@ -357,26 +359,20 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { sql: String, keywords: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { - testUtil.verifyExplain(sql) - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e if e.getClass == clazz => - if (keywords != null) { - assertTrue( - s"The actual exception message \n${e.getMessage}\n" + - s"doesn't contain expected keyword \n$keywords\n", - e.getMessage.contains(keywords)) - } - case e: Throwable => - e.printStackTrace() - fail(s"Expected throw ${clazz.getSimpleName}, but is $e.") + val callable: ThrowingCallable = () => testUtil.verifyExplain(sql) + if (keywords != null) { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) + .withMessageContaining(keywords) + } else { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) } } } object LookupJoinTest { - @Parameterized.Parameters(name = "LegacyTableSource={0}") + @Parameters(name = "LegacyTableSource={0}") def parameters(): JCollection[Array[Object]] = { Seq[Array[AnyRef]](Array(JBoolean.TRUE), Array(JBoolean.FALSE)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala index f725d350d0d1a..b4d9a7a1c1b63 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala @@ -19,11 +19,11 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class NestedLoopJoinTest extends JoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala index 57e27dce442b8..7901d6a51a454 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala @@ -19,11 +19,11 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.junit.Before +import org.junit.jupiter.api.BeforeEach class NestedLoopSemiAntiJoinTest extends SemiAntiJoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.scala index 745b300dbae24..0705c00ca6862 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.planner.plan.batch.sql.join -import org.junit.Test +import org.junit.jupiter.api.Test /** Test SEMI/ANTI Join, the join operators are chose based on cost. */ class SemiAntiJoinTest extends SemiAntiJoinTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTestBase.scala index 77a5d3762cc3c..d9afccaddcf52 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTestBase.scala @@ -19,11 +19,10 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test abstract class SemiAntiJoinTestBase extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala index b76f098897c27..a9c4fa1189018 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class ShuffledHashJoinTest extends JoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig.getConfiguration .setLong(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, 1L) @@ -35,78 +36,78 @@ class ShuffledHashJoinTest extends JoinTestBase { @Test override def testInnerJoinWithoutJoinPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInnerJoinWithoutJoinPred() + assertThatThrownBy(() => super.testInnerJoinWithoutJoinPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinNoEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinNoEquiPred() + assertThatThrownBy(() => super.testLeftOuterJoinNoEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnTrue() + assertThatThrownBy(() => super.testLeftOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnFalse() + assertThatThrownBy(() => super.testLeftOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnTrue() + assertThatThrownBy(() => super.testRightOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnFalse() + assertThatThrownBy(() => super.testRightOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testRightOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testFullOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnFalse() + assertThatThrownBy(() => super.testFullOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnTrue() + assertThatThrownBy(() => super.testFullOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testCrossJoin(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testCrossJoin() + assertThatThrownBy(() => super.testCrossJoin()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala index 0d37c2ee03179..e5f3eaeb5a84e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, @@ -37,128 +38,128 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase { // because NestedLoopJoin(non-singleRowJoin) is disabled. @Test override def testNotInWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInNotInExistsNotExists(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInNotInExistsNotExists() + assertThatThrownBy(() => super.testInNotInExistsNotExists()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiExistsWithCorrelate1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiExistsWithCorrelate1() + assertThatThrownBy(() => super.testMultiExistsWithCorrelate1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_MultiFields(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_MultiFields() + assertThatThrownBy(() => super.testNotInWithUncorrelated_MultiFields()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiNotInWithCorrelated(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiNotInWithCorrelated() + assertThatThrownBy(() => super.testMultiNotInWithCorrelated()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithCorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithCorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithCorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.scala index 5a5597fd6665e..19aeecbe0550f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SingleRowJoinTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala index f8dc8d4ad9255..2d7cf24fab4d0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class SortMergeJoinTest extends JoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin") @@ -32,79 +33,79 @@ class SortMergeJoinTest extends JoinTestBase { @Test override def testInnerJoinWithoutJoinPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInnerJoinWithoutJoinPred() + assertThatThrownBy(() => super.testInnerJoinWithoutJoinPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinNoEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinNoEquiPred() + assertThatThrownBy(() => super.testLeftOuterJoinNoEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnTrue() + assertThatThrownBy(() => super.testLeftOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testLeftOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testLeftOuterJoinOnFalse() + assertThatThrownBy(() => super.testLeftOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnTrue() + assertThatThrownBy(() => super.testRightOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinOnFalse() + assertThatThrownBy(() => super.testRightOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testRightOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testRightOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testRightOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinWithNonEquiPred(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinWithNonEquiPred() + assertThatThrownBy(() => super.testFullOuterJoinWithNonEquiPred()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnFalse(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnFalse() + assertThatThrownBy(() => super.testFullOuterJoinOnFalse()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testFullOuterJoinOnTrue(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testFullOuterJoinOnTrue() + assertThatThrownBy(() => super.testFullOuterJoinOnTrue()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testCrossJoin(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testCrossJoin() + assertThatThrownBy(() => super.testCrossJoin()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala index 5399af2f9a6b9..b72bda06091b6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala @@ -20,11 +20,12 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase { - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin") @@ -36,127 +37,127 @@ class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase { // because NestedLoopJoin(non-singleRowJoin) is disabled. @Test override def testNotInWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotInWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInNotInExistsNotExists(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInNotInExistsNotExists() + assertThatThrownBy(() => super.testInNotInExistsNotExists()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testNotExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition1() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testExistsWithCorrelated_NonEquiCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_NonEquiCondition2() + assertThatThrownBy(() => super.testExistsWithCorrelated_NonEquiCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiExistsWithCorrelate1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiExistsWithCorrelate1() + assertThatThrownBy(() => super.testMultiExistsWithCorrelate1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_MultiFields(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_MultiFields() + assertThatThrownBy(() => super.testNotInWithUncorrelated_MultiFields()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testMultiNotInWithCorrelated(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testMultiNotInWithCorrelated() + assertThatThrownBy(() => super.testMultiNotInWithCorrelated()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testInWithCorrelated_ComplexCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithCorrelated_ComplexCondition3() + assertThatThrownBy(() => super.testInWithCorrelated_ComplexCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition1(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition1() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition1()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition2(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition2() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition2()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test override def testNotInWithUncorrelated_SimpleCondition3(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testNotInWithUncorrelated_SimpleCondition3() + assertThatThrownBy(() => super.testNotInWithUncorrelated_SimpleCondition3()) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalFunctionJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalFunctionJoinTest.scala index 911f8a523c1ed..e52041d5430d8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalFunctionJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalFunctionJoinTest.scala @@ -21,8 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.hamcrest.Matchers.containsString -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -42,16 +42,15 @@ class TemporalFunctionJoinTest extends TableTestBase { @Test def testSimpleJoin(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Cannot generate a valid execution plan for the given query") - val sqlQuery = "SELECT " + "o_amount * rate as rate " + "FROM Orders AS o, " + "LATERAL TABLE (Rates(o_rowtime)) AS r " + "WHERE currency = o_currency" - util.verifyExplain(sqlQuery) + assertThatThrownBy(() => util.verifyExplain(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } /** @@ -59,7 +58,7 @@ class TemporalFunctionJoinTest extends TableTestBase { * complex OR join condition and there are some columns that are not being used (are being * pruned). */ - @Test(expected = classOf[TableException]) + @Test def testComplexJoin(): Unit = { val util = batchTestUtil() util.addDataStream[(String, Int)]("Table3", 't3_comment, 't3_secondary_key) @@ -92,30 +91,29 @@ class TemporalFunctionJoinTest extends TableTestBase { "Table3 " + "WHERE t3_secondary_key = secondary_key" - util.verifyExplain(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExplain(sqlQuery)) } @Test def testUncorrelatedJoin(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage(containsString("Cannot generate a valid execution plan")) - val sqlQuery = "SELECT " + "o_amount * rate as rate " + "FROM Orders AS o, " + "LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123')) AS r " + "WHERE currency = o_currency" - util.verifyExplain(sqlQuery) + assertThatThrownBy(() => util.verifyExplain(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan") + .isInstanceOf[TableException] } @Test def testTemporalTableFunctionScan(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage(containsString("Cannot generate a valid execution plan")) - val sqlQuery = "SELECT * FROM LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123'))"; - util.verifyExplain(sqlQuery) + assertThatThrownBy(() => util.verifyExplain(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalJoinTest.scala index a8f46a959b9ba..89292aea47a17 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalJoinTest.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.batch.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} /** * Test temporal join in batch mode. @@ -31,7 +32,7 @@ class TemporalJoinTest extends TableTestBase { val util: BatchTableTestUtil = batchTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTable(""" |CREATE TABLE Orders ( @@ -104,7 +105,7 @@ class TemporalJoinTest extends TableTestBase { "GROUP BY currency ") } - @Test(expected = classOf[TableException]) + @Test def testSimpleJoin(): Unit = { val sqlQuery = "SELECT " + "o_amount * rate as rate " + @@ -112,10 +113,11 @@ class TemporalJoinTest extends TableTestBase { "RatesHistoryWithPK FOR SYSTEM_TIME AS OF o.o_rowtime as r " + "on o.o_currency = r.currency" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testSimpleRowtimeVersionedViewJoin(): Unit = { val sqlQuery = "SELECT " + "o_amount * rate as rate " + @@ -124,10 +126,11 @@ class TemporalJoinTest extends TableTestBase { "FOR SYSTEM_TIME AS OF o.o_rowtime as r1 " + "on o.o_currency = r1.currency" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testSimpleProctimeVersionedViewJoin(): Unit = { val sqlQuery = "SELECT " + "o_amount * rate as rate " + @@ -136,10 +139,11 @@ class TemporalJoinTest extends TableTestBase { "FOR SYSTEM_TIME AS OF o.o_proctime as r1 " + "on o.o_currency = r1.currency" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testSimpleViewProcTimeJoin(): Unit = { val sqlQuery = "SELECT " + @@ -149,6 +153,7 @@ class TemporalJoinTest extends TableTestBase { "FOR SYSTEM_TIME AS OF o.o_proctime as r1 " + "on o.o_currency = r1.currency" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala index 496e7cff50616..91ad63060b94e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test /** Test for testing aggregate plans. */ class AggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CalcTest.scala index 92294643de20a..0ce5bc82917db 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CalcTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.plan.batch.table.CalcTest.{giveMeCaseClass, MyHashCode, TestCaseClass, WC} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class CalcTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.scala index 8edd75ea627c3..bfc45f93d0cf1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test /** Tests for column functions. */ class ColumnFunctionsTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala index 3b84724b4cc4c..da8f7a5e9f7c3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.planner.utils.{MockPythonTableFunction, TableFunc0 import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.Test +import org.junit.jupiter.api.Test class CorrelateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.scala index 72d749b879bda..4c5bd55eeb747 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -32,7 +33,7 @@ class GroupWindowTest extends TableTestBase { // Common test // =============================================================================================== - @Test(expected = classOf[TableException]) + @Test def testEventTimeTumblingGroupWindowOverCount(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) @@ -41,7 +42,8 @@ class GroupWindowTest extends TableTestBase { .window(Tumble.over(2.rows).on('long).as('w)) .groupBy('w, 'string) .select('string, 'int.count) - util.verifyExecPlan(windowedTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(windowedTable)) } @Test @@ -82,7 +84,7 @@ class GroupWindowTest extends TableTestBase { util.verifyExecPlan(windowedTable) } - @Test(expected = classOf[TableException]) + @Test def testAllEventTimeTumblingGroupWindowOverCount(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) @@ -92,7 +94,8 @@ class GroupWindowTest extends TableTestBase { .groupBy('w) .select('int.count) - util.verifyExecPlan(windowedTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(windowedTable)) } @Test @@ -138,7 +141,7 @@ class GroupWindowTest extends TableTestBase { util.verifyExecPlan(windowedTable) } - @Test(expected = classOf[TableException]) + @Test def testEventTimeSlidingGroupWindowOverCount(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) @@ -148,7 +151,8 @@ class GroupWindowTest extends TableTestBase { .groupBy('w, 'string) .select('string, 'int.count) - util.verifyExecPlan(windowedTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(windowedTable)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala index ff73c8aaa4342..be5228bcaf243 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.plan.batch.table.JoinTest.Merger import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class JoinTest extends TableTestBase { @@ -140,31 +141,34 @@ class JoinTest extends TableTestBase { util.verifyExecPlan(results) } - @Test(expected = classOf[ValidationException]) + @Test def testFullJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - util.verifyExecPlan(ds2.fullOuterJoin(ds1, 'b < 'd).select('c, 'g)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(ds2.fullOuterJoin(ds1, 'b < 'd).select('c, 'g))) } - @Test(expected = classOf[ValidationException]) + @Test def testLeftJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - util.verifyExecPlan(ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g))) } - @Test(expected = classOf[ValidationException]) + @Test def testRightJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - util.verifyExecPlan(ds2.rightOuterJoin(ds1, 'b < 'd).select('c, 'g)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(ds2.rightOuterJoin(ds1, 'b < 'd).select('c, 'g))) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.scala index 8c50e8ea99365..f09e7604294a7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.PandasAggregateFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class PythonAggregateTest extends TableTestBase { @@ -50,7 +51,7 @@ class PythonAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testMixedUsePandasAggAndJavaAgg(): Unit = { val util = batchTestUtil() val sourceTable = util.addTableSource[(Int, Long, Int)]("MyTable", 'a, 'b, 'c) @@ -60,6 +61,7 @@ class PythonAggregateTest extends TableTestBase { .groupBy('b) .select('b, func('a, 'c), 'a.count()) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala index f1bd371e35db2..9cf2d1d2151ee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala @@ -22,12 +22,12 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class PythonCalcTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) util.addFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.scala index 47ad3ee5b6415..d27d74922d9ff 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.{PandasAggregateFunction, TestPythonAggregateFunction} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class PythonGroupWindowAggregateTest extends TableTestBase { @@ -41,7 +42,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testPandasEventTimeTumblingGroupWindowOverCount(): Unit = { val util = batchTestUtil() val sourceTable = @@ -53,7 +54,8 @@ class PythonGroupWindowAggregateTest extends TableTestBase { .groupBy('w, 'b) .select('b, func('a, 'c)) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } @Test @@ -71,7 +73,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testPandasEventTimeSlidingGroupWindowOverCount(): Unit = { val util = batchTestUtil() val sourceTable = @@ -83,7 +85,8 @@ class PythonGroupWindowAggregateTest extends TableTestBase { .groupBy('w, 'b) .select('b, func('a, 'c)) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } @Test @@ -101,7 +104,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testGeneralEventTimeTumblingGroupWindowOverTime(): Unit = { val util = batchTestUtil() val sourceTable = @@ -113,6 +116,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { .groupBy('w, 'b) .select('b, 'w.start, 'w.end, func('a, 'c)) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.scala index 5232dcce5cbc5..a337ae7a2a6b2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.{PandasAggregateFunction, TestPythonAggregateFunction} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class PythonOverWindowAggregateTest extends TableTestBase { @@ -64,7 +65,7 @@ class PythonOverWindowAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testGeneralRangeOverWindowAggregate(): Unit = { val util = batchTestUtil() val sourceTable = @@ -80,6 +81,7 @@ class PythonOverWindowAggregateTest extends TableTestBase { .as('w)) .select('b, func('a, 'c).over('w)) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala index 4caceecf87a3d..8efbce6e873e9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/TemporalTableFunctionJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/TemporalTableFunctionJoinTest.scala index c799402b93d5b..5d7d914e61a70 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/TemporalTableFunctionJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/TemporalTableFunctionJoinTest.scala @@ -21,8 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.hamcrest.Matchers.containsString -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -41,23 +41,19 @@ class TemporalTableFunctionJoinTest extends TableTestBase { @Test def testSimpleJoin(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Cannot generate a valid execution plan for the given query") - val result = orders .as("o_amount", "o_currency", "o_rowtime") .joinLateral(rates('o_rowtime), 'currency === 'o_currency) .select($"o_amount" * $"rate") .as("rate") - util.verifyExecPlan(result) + assertThatThrownBy(() => util.verifyExecPlan(result)) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[TableException] } @Test def testUncorrelatedJoin(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage(containsString("Cannot generate a valid execution plan")) - val result = orders .as("o_amount", "o_currency", "o_rowtime") .joinLateral( @@ -65,7 +61,9 @@ class TemporalTableFunctionJoinTest extends TableTestBase { 'o_currency === 'currency) .select($"o_amount" * $"rate") - util.verifyExecPlan(result) + assertThatThrownBy(() => util.verifyExecPlan(result)) + .hasMessageContaining("Cannot generate a valid execution plan") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/AggregateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/AggregateValidationTest.scala index 928d078eb50ad..6fb62a24756e5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/AggregateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/AggregateValidationTest.scala @@ -22,59 +22,66 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class AggregateValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testNonWorkingAggregationDataTypes(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(String, Int)]("Table2") // Must fail. Field '_1 is not a numeric type. - t.select('_1.sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select('_1.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testNoNestedAggregations(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(String, Int)]("Table2") // Must fail. Sum aggregation can not be chained. - t.select('_2.sum.sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select('_2.sum.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testGroupingOnNonExistentField(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // must fail. '_foo not a valid field - t.groupBy('_foo).select('a.avg) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.groupBy('_foo).select('a.avg)) } - @Test(expected = classOf[ValidationException]) + @Test def testGroupingInvalidSelection(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - t.groupBy('a, 'b) - // must fail. 'c is not a grouping key or aggregation - .select('c) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy('a, 'b) + // must fail. 'c is not a grouping key or aggregation + .select('c)) } - @Test(expected = classOf[ValidationException]) + @Test def testAggregationOnNonExistingField(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // Must fail. Field 'foo does not exist. - t.select('foo.avg) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select('foo.avg)) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testInvalidUdAggArgs() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -82,116 +89,124 @@ class AggregateValidationTest extends TableTestBase { val myWeightedAvg = new WeightedAvgWithMergeAndReset // must fail. UDAGG does not accept String type - t.select(myWeightedAvg('c, 'a)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select(myWeightedAvg('c, 'a))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testGroupingInvalidUdAggArgs() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val myWeightedAvg = new WeightedAvgWithMergeAndReset - t.groupBy('b) - // must fail. UDAGG does not accept String type - .select(myWeightedAvg('c, 'a)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy('b) + // must fail. UDAGG does not accept String type + .select(myWeightedAvg('c, 'a))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testGroupingNestedUdAgg() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val myWeightedAvg = new WeightedAvgWithMergeAndReset - t.groupBy('c) - // must fail. UDAGG does not accept String type - .select(myWeightedAvg(myWeightedAvg('b, 'a), 'a)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy('c) + // must fail. UDAGG does not accept String type + .select(myWeightedAvg(myWeightedAvg('b, 'a), 'a))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testAggregationOnNonExistingFieldJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - t.select($"foo".avg) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select($"foo".avg)) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testNonWorkingAggregationDataTypesJava() { val util = batchTestUtil() val t = util.addTableSource[(Long, String)]("Table2", 'b, 'c) // Must fail. Cannot compute SUM aggregate on String field. - t.select($"c".sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select($"c".sum)) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testNoNestedAggregationsJava() { val util = batchTestUtil() val t = util.addTableSource[(Long, String)]("Table2", 'b, 'c) // Must fail. Aggregation on aggregation not allowed. - t.select($"b".sum.sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select($"b".sum.sum)) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testNoDeeplyNestedAggregationsJava() { val util = batchTestUtil() val t = util.addTableSource[(Long, String)]("Table2", 'b, 'c) // Must fail. Aggregation on aggregation not allowed. - t.select(($"b".sum + 1).sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select(($"b".sum + 1).sum)) } - @Test(expected = classOf[ValidationException]) + @Test @throws[Exception] def testGroupingOnNonExistentFieldJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // must fail. Field foo is not in input - t.groupBy($"foo") - .select($"a".avg) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.groupBy($"foo").select($"a".avg)) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testGroupingInvalidSelectionJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - t.groupBy($"a", $"b") - // must fail. Field c is not a grouping key or aggregation - .select($"c") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy($"a", $"b") + // must fail. Field c is not a grouping key or aggregation + .select($"c")) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testUnknownUdAggJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // must fail. unknown is not known - t.select(call("unknown", $"c")) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select(call("unknown", $"c"))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testGroupingUnknownUdAggJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - t.groupBy($"a", $"b") - // must fail. unknown is not known - .select(call("unknown", $"c")) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy($"a", $"b") + // must fail. unknown is not known + .select(call("unknown", $"c"))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testInvalidUdAggArgsJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -200,11 +215,11 @@ class AggregateValidationTest extends TableTestBase { util.addFunction("myWeightedAvg", myWeightedAvg) // must fail. UDAGG does not accept String type - t.select(call("myWeightedAvg", $"c", $"a")) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select(call("myWeightedAvg", $"c", $"a"))) } - @Test(expected = classOf[ValidationException]) - @throws[Exception] + @Test def testGroupingInvalidUdAggArgsJava() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -212,8 +227,11 @@ class AggregateValidationTest extends TableTestBase { val myWeightedAvg = new WeightedAvgWithMergeAndReset util.addFunction("myWeightedAvg", myWeightedAvg) - t.groupBy($"b") - // must fail. UDAGG does not accept String type - .select(call("myWeightedAvg", $"c", $"a")) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + t.groupBy($"b") + // must fail. UDAGG does not accept String type + .select(call("myWeightedAvg", $"c", $"a"))) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CalcValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CalcValidationTest.scala index 42aff351917a6..c81918456ff58 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CalcValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CalcValidationTest.scala @@ -21,88 +21,86 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ -import org.junit.Assert._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class CalcValidationTest extends TableTestBase { @Test def testSelectInvalidFieldFields(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Cannot resolve field [foo], input field list:[a, b, c].") val util = batchTestUtil() - util - .addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - // must fail. Field 'foo does not exist - .select('a, 'foo) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) + // must fail. Field 'foo does not exist + .select('a, 'foo)) + .withMessageContaining("Cannot resolve field [foo], input field list:[a, b, c].") } - @Test(expected = classOf[ValidationException]) + @Test def testFilterInvalidFieldName(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // must fail. Field 'foo does not exist - t.filter('foo === 2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.filter('foo === 2)) } - @Test(expected = classOf[ValidationException]) + @Test def testSelectInvalidField() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // Must fail. Field foo does not exist - t.select($"a" + 1, $"foo" + 2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select($"a" + 1, $"foo" + 2)) } - @Test(expected = classOf[ValidationException]) + @Test def testSelectAmbiguousFieldNames() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // Must fail. Field foo does not exist - t.select(($"a" + 1).as("foo"), ($"b" + 2).as("foo")) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select(($"a" + 1).as("foo"), ($"b" + 2).as("foo"))) } - @Test(expected = classOf[ValidationException]) + @Test def testFilterInvalidField() { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) // Must fail. Field foo does not exist. - t.filter($"foo" === 17) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.filter($"foo" === 17)) } @Test def testAliasStarException(): Unit = { val util = batchTestUtil() - try { - util.addTableSource[(Int, Long, String)]("Table1", '*, 'b, 'c) - fail("TableException expected") - } catch { - case _: ValidationException => // ignore - } - - try { - util.addTableSource[(Int, Long, String)]("Table3").as("*", "b", "c") - fail("ValidationException expected") - } catch { - case _: ValidationException => // ignore - } - try { - util.addTableSource[(Int, Long, String)]("Table4", 'a, 'b, 'c).select('*, 'b) - fail("ValidationException expected") - } catch { - case _: ValidationException => // ignore - } + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.addTableSource[(Int, Long, String)]("Table1", '*, 'b, 'c)) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.addTableSource[(Int, Long, String)]("Table3").as("*", "b", "c")) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.addTableSource[(Int, Long, String)]("Table4", 'a, 'b, 'c).select('*, 'b)) } - @Test(expected = classOf[ValidationException]) + @Test def testDuplicateFlattening(): Unit = { val util = batchTestUtil() val table = util.addTableSource[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c) - table.select('a.flatten(), 'a.flatten()) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => table.select('a.flatten(), 'a.flatten())) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala index 20605062b1e9e..ae28906e82345 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class CorrelateValidationTest extends TableTestBase { @@ -29,14 +30,19 @@ class CorrelateValidationTest extends TableTestBase { * Due to the improper translation of TableFunction left outer join (see CALCITE-2004), the join * predicate can only be empty or literal true (the restriction should be removed in FLINK-7865). */ - @Test(expected = classOf[ValidationException]) + @Test def testLeftOuterJoinWithPredicates(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - val result = table - .leftOuterJoinLateral(func('c).as('s), 'c === 's) - .select('c, 's) - util.verifyExecPlan(result) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .leftOuterJoinLateral(func('c).as('s), 'c === 's) + .select('c, 's) + util.verifyExecPlan(result) + }) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/GroupWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/GroupWindowValidationTest.scala index b0bed858099ad..45f1704e0822d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/GroupWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/GroupWindowValidationTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api.{Session, Slide, Tumble, ValidationException, import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class GroupWindowValidationTest extends TableTestBase { @@ -30,141 +31,170 @@ class GroupWindowValidationTest extends TableTestBase { // Common test // =============================================================================================== - @Test(expected = classOf[ValidationException]) + @Test def testGroupByWithoutWindowAlias(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) - table - .window(Tumble.over(5.milli).on('long).as('w)) - .groupBy('string) - .select('string, 'int.count) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Tumble.over(5.milli).on('long).as('w)) + .groupBy('string) + .select('string, 'int.count)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidRowTimeRef(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) - table - .window(Tumble.over(5.milli).on('long).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) - .window(Slide.over(5.milli).every(1.milli).on('int).as('w2)) // 'Int does not exist in input. - .groupBy('w2) - .select('string) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Tumble.over(5.milli).on('long).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count) + .window( + Slide.over(5.milli).every(1.milli).on('int).as('w2) + ) // 'Int does not exist in input. + .groupBy('w2) + .select('string)) } // =============================================================================================== // Tumbling Windows // =============================================================================================== - @Test(expected = classOf[ValidationException]) + @Test def testInvalidProcessingTimeDefinition(): Unit = { val util = batchTestUtil() // proctime is not allowed - util.addTableSource[(Int, String)]('proctime, 'int, 'string) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.addTableSource[(Int, String)]('proctime, 'int, 'string)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidProcessingTimeDefinition2(): Unit = { val util = batchTestUtil() // proctime is not allowed - util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'proctime) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'proctime)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidEventTimeDefinition(): Unit = { val util = batchTestUtil() // definition must not extend schema - util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'rowtime) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'rowtime)) } - @Test(expected = classOf[ValidationException]) + @Test def testTumblingGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Tumble.over(2.minutes).on('rowtime).as('w)) - .groupBy('w, 'long) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Tumble.over(2.minutes).on('rowtime).as('w)) + .groupBy('w, 'long) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } - @Test(expected = classOf[ValidationException]) + @Test def testAllTumblingGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Tumble.over(2.minutes).on('rowtime).as('w)) - .groupBy('w) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Tumble.over(2.minutes).on('rowtime).as('w)) + .groupBy('w) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } // =============================================================================================== // Sliding Windows // =============================================================================================== - @Test(expected = classOf[ValidationException]) + @Test def testSlidingGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Slide.over(2.minutes).every(1.minute).on('rowtime).as('w)) - .groupBy('w, 'long) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Slide.over(2.minutes).every(1.minute).on('rowtime).as('w)) + .groupBy('w, 'long) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } - @Test(expected = classOf[ValidationException]) + @Test def testAllSlidingGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Slide.over(2.minutes).every(1.minute).on('long).as('w)) - .groupBy('w) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Slide.over(2.minutes).every(1.minute).on('long).as('w)) + .groupBy('w) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } - @Test(expected = classOf[ValidationException]) + @Test def testSessionGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Session.withGap(2.minutes).on('rowtime).as('w)) - .groupBy('w, 'long) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Session.withGap(2.minutes).on('rowtime).as('w)) + .groupBy('w, 'long) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } - @Test(expected = classOf[ValidationException]) + @Test def testAllSessionGroupWindowWithInvalidUdAggArgs(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val myWeightedAvg = new WeightedAvgWithMerge - table - .window(Session.withGap(2.minutes).on('rowtime).as('w)) - .groupBy('w) - // invalid function arguments - .select(myWeightedAvg('int, 'string)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Session.withGap(2.minutes).on('rowtime).as('w)) + .groupBy('w) + // invalid function arguments + .select(myWeightedAvg('int, 'string))) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala index 2012af5b04edb..860f88ec69755 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala @@ -24,78 +24,90 @@ import org.apache.flink.table.planner.runtime.utils.CollectionBatchExecTable import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class JoinValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testJoinNonExistingKey(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - ds1 - .join(ds2) - // must fail. Field 'foo does not exist - .where('foo === 'e) - .select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + ds1 + .join(ds2) + // must fail. Field 'foo does not exist + .where('foo === 'e) + .select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testJoinWithNonMatchingKeyTypes(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - ds1 - .join(ds2) - // must fail. Field 'a is Int, and 'g is String - .where('a === 'g) - .select('c, 'g) - + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + ds1 + .join(ds2) + // must fail. Field 'a is Int, and 'g is String + .where('a === 'g) + .select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testJoinWithAmbiguousFields(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'c) - ds1 - .join(ds2) - // must fail. Both inputs share the same field 'c - .where('a === 'd) - .select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + ds1 + .join(ds2) + // must fail. Both inputs share the same field 'c + .where('a === 'd) + .select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testLeftJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testRightJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - ds2.rightOuterJoin(ds1, 'b < 'd).select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds2.rightOuterJoin(ds1, 'b < 'd).select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testFullJoinNoEquiJoinPredicate(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h) - ds2.fullOuterJoin(ds1, 'b < 'd).select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds2.fullOuterJoin(ds1, 'b < 'd).select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testJoinTablesFromDifferentEnvs(): Unit = { val settings = EnvironmentSettings.newInstance().inBatchMode().build() val tEnv1 = TableEnvironmentImpl.create(settings) @@ -104,10 +116,11 @@ class JoinValidationTest extends TableTestBase { val ds2 = CollectionBatchExecTable.get5TupleDataSet(tEnv2, "d, e, f, g, c") // Must fail. Tables are bound to different TableEnvironments. - ds1.join(ds2).where('b === 'e).select('c, 'g) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.join(ds2).where('b === 'e).select('c, 'g)) } - @Test(expected = classOf[ValidationException]) + @Test def testJoinTablesFromDifferentEnvsJava() { val settings = EnvironmentSettings.newInstance().inBatchMode().build() val tEnv1 = TableEnvironmentImpl.create(settings) @@ -115,16 +128,18 @@ class JoinValidationTest extends TableTestBase { val ds1 = CollectionBatchExecTable.getSmall3TupleDataSet(tEnv1, "a, b, c") val ds2 = CollectionBatchExecTable.get5TupleDataSet(tEnv2, "d, e, f, g, c") // Must fail. Tables are bound to different TableEnvironments. - ds1.join(ds2).where($"a" === $"d").select($"g".count) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.join(ds2).where($"a" === $"d").select($"g".count)) } - @Test(expected = classOf[TableException]) + @Test def testOuterJoinWithPythonFunctionInCondition(): Unit = { val util = batchTestUtil() val left = util.addTableSource[(Int, Long, String)]("left", 'a, 'b, 'c) val right = util.addTableSource[(Int, Long, String)]("right", 'd, 'e, 'f) val pyFunc = new PythonScalarFunction("pyFunc") val result = left.leftOuterJoin(right, 'a === 'd && pyFunc('a, 'd) === 'a + 'd) - util.verifyExecPlan(result) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(result)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/OverWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/OverWindowValidationTest.scala index 40735f950de9b..060c58b1bf9bf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/OverWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/OverWindowValidationTest.scala @@ -22,29 +22,34 @@ import org.apache.flink.table.api.{Tumble, ValidationException, _} import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.OverAgg0 import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class OverWindowValidationTest extends TableTestBase { /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation(): Unit = { val util = batchTestUtil() val t = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val overAgg = new OverAgg0 - t.select('c.count, overAgg('b, 'a)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => t.select('c.count, overAgg('b, 'a))) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation2(): Unit = { val util = batchTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) val overAgg = new OverAgg0 - table - .window(Tumble.over(5.milli).on('long).as('w)) - .groupBy('string, 'w) - .select(overAgg('long, 'int)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Tumble.over(5.milli).on('long).as('w)) + .groupBy('string, 'w) + .select(overAgg('long, 'int))) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SetOperatorsValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SetOperatorsValidationTest.scala index 444a63f97290a..acff005d7af34 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SetOperatorsValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SetOperatorsValidationTest.scala @@ -23,21 +23,23 @@ import org.apache.flink.table.api.internal.TableEnvironmentImpl import org.apache.flink.table.planner.runtime.utils.CollectionBatchExecTable import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class SetOperatorsValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentColumnSize(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) val ds2 = util.addTableSource[(Int, Long, Int, String, Long)]("Table5", 'a, 'b, 'd, 'c, 'e) // must fail. Union inputs have different column size. - ds1.unionAll(ds2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.unionAll(ds2)) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentFieldTypes(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -46,10 +48,11 @@ class SetOperatorsValidationTest extends TableTestBase { .select('a, 'b, 'c) // must fail. Union inputs have different field types. - ds1.unionAll(ds2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.unionAll(ds2)) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionTablesFromDifferentEnvs(): Unit = { val settings = EnvironmentSettings.newInstance().inBatchMode().build() val tEnv1 = TableEnvironmentImpl.create(settings) @@ -59,10 +62,11 @@ class SetOperatorsValidationTest extends TableTestBase { val ds2 = CollectionBatchExecTable.getSmall3TupleDataSet(tEnv2) // Must fail. Tables are bound to different TableEnvironments. - ds1.unionAll(ds2).select('c) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.unionAll(ds2).select('c)) } - @Test(expected = classOf[ValidationException]) + @Test def testMinusDifferentFieldTypes(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -71,10 +75,11 @@ class SetOperatorsValidationTest extends TableTestBase { .select('a, 'b, 'c) // must fail. Minus inputs have different field types. - ds1.minus(ds2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.minus(ds2)) } - @Test(expected = classOf[ValidationException]) + @Test def testMinusAllTablesFromDifferentEnvs(): Unit = { val settings = EnvironmentSettings.newInstance().inBatchMode().build() val tEnv1 = TableEnvironmentImpl.create(settings) @@ -84,10 +89,11 @@ class SetOperatorsValidationTest extends TableTestBase { val ds2 = CollectionBatchExecTable.getSmall3TupleDataSet(tEnv2) // Must fail. Tables are bound to different TableEnvironments. - ds1.minusAll(ds2).select('c) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.minusAll(ds2).select('c)) } - @Test(expected = classOf[ValidationException]) + @Test def testIntersectWithDifferentFieldTypes(): Unit = { val util = batchTestUtil() val ds1 = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) @@ -96,10 +102,11 @@ class SetOperatorsValidationTest extends TableTestBase { .select('a, 'b, 'c) // must fail. Intersect inputs have different field types. - ds1.intersect(ds2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.intersect(ds2)) } - @Test(expected = classOf[ValidationException]) + @Test def testIntersectTablesFromDifferentEnvs(): Unit = { val settings = EnvironmentSettings.newInstance().inBatchMode().build() val tEnv1 = TableEnvironmentImpl.create(settings) @@ -109,6 +116,7 @@ class SetOperatorsValidationTest extends TableTestBase { val ds2 = CollectionBatchExecTable.getSmall3TupleDataSet(tEnv2) // Must fail. Tables are bound to different TableEnvironments. - ds1.intersect(ds2).select('c) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.intersect(ds2).select('c)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SortValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SortValidationTest.scala index cfe82e50b89b5..823a1758c663e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SortValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SortValidationTest.scala @@ -21,31 +21,35 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit._ +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class SortValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testFetchBeforeOffset(): Unit = { val util = batchTestUtil() val ds = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - ds.orderBy('a.asc).fetch(5).offset(10) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds.orderBy('a.asc).fetch(5).offset(10)) } - @Test(expected = classOf[ValidationException]) + @Test def testOffsetBeforeOffset(): Unit = { val util = batchTestUtil() val ds = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - ds.orderBy('a.asc).offset(10).offset(5) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds.orderBy('a.asc).offset(10).offset(5)) } - @Test(expected = classOf[ValidationException]) + @Test def testNegativeFetch(): Unit = { val util = batchTestUtil() val ds = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - ds.orderBy('a.asc).offset(-1) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds.orderBy('a.asc).offset(-1)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/AggregateReduceGroupingTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/AggregateReduceGroupingTestBase.scala index 16542dde1cceb..6b5a2619b6ba7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/AggregateReduceGroupingTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/AggregateReduceGroupingTestBase.scala @@ -27,12 +27,12 @@ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} import com.google.common.collect.ImmutableSet import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} abstract class AggregateReduceGroupingTestBase(withExecPlan: Boolean) extends TableTestBase { protected val util: BatchTableTestUtil = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource( "T1", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala index fc761ba9d1ba0..cba75862fad26 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala @@ -22,12 +22,13 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} abstract class DistinctAggregateTestBase(withExecPlan: Boolean) extends TableTestBase { protected val util: BatchTableTestUtil = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, Int)]("MyTable", 'a, 'b, 'c) util.addTableSource[(Int, Long, String, String, String)]("MyTable2", 'a, 'b, 'c, 'd, 'e) @@ -192,7 +193,7 @@ abstract class DistinctAggregateTestBase(withExecPlan: Boolean) extends TableTes verifyPlan(sqlQuery) } - @Test(expected = classOf[RuntimeException]) + @Test def testTooManyDistinctAggOnDifferentColumn(): Unit = { // max group count must be less than 64 val fieldNames = (0 until 64).map(i => s"f$i").toArray @@ -203,7 +204,8 @@ abstract class DistinctAggregateTestBase(withExecPlan: Boolean) extends TableTes val maxList = fieldNames.map(f => s"MAX($f)").mkString(", ") val sqlQuery = s"SELECT $distinctList, $maxList FROM MyTable64" - verifyPlan(sqlQuery) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => verifyPlan(sqlQuery)) } private def verifyPlan(sqlQuery: String): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala index ee563d138b837..02aae7ee59278 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala @@ -24,9 +24,9 @@ import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} import org.apache.flink.table.planner.plan.rules.logical.JoinDeriveNullFilterRule import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters -import org.junit.{Before, Test} -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} import java.util @@ -44,7 +44,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes protected def getTableTestUtil: TableTestUtil - @Before + @BeforeEach def setup(): Unit = { val types = Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING) @@ -145,7 +145,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes } - @Test + @TestTemplate def testStarJoinCondition1(): Unit = { val sql = s""" @@ -155,7 +155,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testStarJoinCondition2(): Unit = { val sql = s""" @@ -165,7 +165,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testBushyJoinCondition1(): Unit = { val sql = s""" @@ -175,7 +175,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testBushyJoinCondition2(): Unit = { val sql = s""" @@ -185,7 +185,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testWithoutColumnStats(): Unit = { val sql = s""" @@ -195,7 +195,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testJoinWithProject(): Unit = { val sql = s""" @@ -209,7 +209,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testJoinWithFilter(): Unit = { val sql = s""" @@ -222,7 +222,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testAllInnerJoin(): Unit = { val sql = s""" @@ -236,7 +236,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerAndLeftOuterJoin(): Unit = { val sql = s""" @@ -250,7 +250,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerAndRightOuterJoin(): Unit = { val sql = s""" @@ -264,7 +264,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerAndFullOuterJoin(): Unit = { val sql = s""" @@ -277,7 +277,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testAllLeftOuterJoin(): Unit = { val sql = s""" @@ -292,7 +292,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testAllRightOuterJoin(): Unit = { val sql = s""" @@ -306,7 +306,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testAllFullOuterJoin(): Unit = { val sql = s""" @@ -320,7 +320,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinLeftOuterJoinInnerJoinLeftOuterJoin(): Unit = { val sql = s""" @@ -334,7 +334,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testLeftOuterJoinInnerJoinLeftOuterJoinInnerJoin(): Unit = { val sql = s""" @@ -348,7 +348,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinRightOuterJoinInnerJoinRightOuterJoin(): Unit = { val sql = s""" @@ -362,7 +362,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testRightOuterJoinInnerJoinRightOuterJoinInnerJoin(): Unit = { val sql = s""" @@ -376,7 +376,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinSemiJoin(): Unit = { val sql = s""" @@ -390,7 +390,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinAntiJoin(): Unit = { val sql = s""" @@ -404,14 +404,14 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCrossJoin(): Unit = { val sql = "SELECT * FROM T1, T2, T3, T4, T5" // All table can reorder. util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinCrossJoin(): Unit = { val sql = s""" @@ -422,7 +422,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinLeftOuterJoinCrossJoin(): Unit = { val sql = s""" @@ -433,7 +433,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testInnerJoinWithBushyTypeJoinCondition(): Unit = { // This case is to test whether can build a bushy join tree. // If variable isBushyJoinReorder is true, it can be built to @@ -450,7 +450,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes util.verifyRelPlan(sql) } - @Test + @TestTemplate def testDeriveNullFilterAfterJoinReorder(): Unit = { val types = Array[TypeInformation[_]](Types.INT, Types.LONG) val builderA = ColumnStats.Builder @@ -524,7 +524,7 @@ abstract class JoinReorderTestBase(isBushyJoinReorder: Boolean) extends TableTes } object JoinReorderTestBase { - @Parameterized.Parameters(name = "isBushyJoinReorder={0}") + @Parameters(name = "isBushyJoinReorder={0}") def parameters(): util.Collection[Boolean] = { util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/PartialInsertTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/PartialInsertTest.scala index 35405ebd00807..2c91b5b421fc2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/PartialInsertTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/PartialInsertTest.scala @@ -20,12 +20,13 @@ package org.apache.flink.table.planner.plan.common import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.Test -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.TestTemplate +import org.junit.jupiter.api.extension.ExtendWith -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class PartialInsertTest(isBatch: Boolean) extends TableTestBase { private val util = if (isBatch) batchTestUtil() else streamTestUtil() @@ -103,36 +104,36 @@ class PartialInsertTest(isBatch: Boolean) extends TableTestBase { | 'writable-metadata' = 'f:BIGINT, g:INT' |)""".stripMargin) - @Test + @TestTemplate def testPartialInsertWithComplexReorder(): Unit = { util.verifyRelPlanInsert( "INSERT INTO sink (b,e,a,g,f,c,d) " + "SELECT b,e,a,456,123,c,d FROM MyTable GROUP BY a,b,c,d,e") } - @Test + @TestTemplate def testPartialInsertWithComplexReorderAndComputedColumn(): Unit = { util.verifyRelPlanInsert( "INSERT INTO partitioned_sink (e,a,g,f,c,d) " + "SELECT e,a,456,123,c,d FROM MyTable GROUP BY a,b,c,d,e") } - @Test + @TestTemplate def testPartialInsertWithUnion(): Unit = { testPartialInsertWithSetOperator("UNION") } - @Test + @TestTemplate def testPartialInsertWithUnionAll(): Unit = { testPartialInsertWithSetOperator("UNION ALL") } - @Test + @TestTemplate def testPartialInsertWithIntersectAll(): Unit = { testPartialInsertWithSetOperator("INTERSECT ALL") } - @Test + @TestTemplate def testPartialInsertWithExceptAll(): Unit = { testPartialInsertWithSetOperator("EXCEPT ALL") } @@ -145,7 +146,7 @@ class PartialInsertTest(isBatch: Boolean) extends TableTestBase { "SELECT e,a,789,456,c,d FROM MyTable GROUP BY a,b,c,d,e ") } - @Test + @TestTemplate def testPartialInsertWithUnionAllNested(): Unit = { util.verifyRelPlanInsert( "INSERT INTO partitioned_sink (e,a,g,f,c,d) " + @@ -156,14 +157,14 @@ class PartialInsertTest(isBatch: Boolean) extends TableTestBase { "SELECT e,a,123,456,c,d FROM MyTable GROUP BY a,b,c,d,e ") } - @Test + @TestTemplate def testPartialInsertWithOrderBy(): Unit = { util.verifyRelPlanInsert( "INSERT INTO partitioned_sink (e,a,g,f,c,d) " + "SELECT e,a,456,123,c,d FROM MyTable ORDER BY a,e,c,d") } - @Test + @TestTemplate def testPartialInsertWithPersistedMetadata(): Unit = { util.verifyRelPlanInsert( "INSERT INTO metadata_sink (a,b,c,d,e,f) " + @@ -171,46 +172,53 @@ class PartialInsertTest(isBatch: Boolean) extends TableTestBase { ) } - @Test + @TestTemplate def testPartialInsertWithVirtualMetaDataColumn(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "SQL validation failed. At line 1, column 38: Unknown target column 'g'") - util.verifyRelPlanInsert( - "INSERT INTO metadata_sink (a,b,c,d,e,g) " + - "SELECT a,b,c,d,e,123 FROM MyTable" - ) + assertThatThrownBy( + () => + util.verifyRelPlanInsert( + "INSERT INTO metadata_sink (a,b,c,d,e,g) " + + "SELECT a,b,c,d,e,123 FROM MyTable" + )) + .hasMessageContaining( + "SQL validation failed. At line 1, column 38: Unknown target column 'g'") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testPartialInsertWithComputedColumn(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "SQL validation failed. At line 1, column 38: Unknown target column 'h'") - util.verifyRelPlanInsert( - "INSERT INTO metadata_sink (a,b,c,d,e,h) " + - "SELECT a,b,c,d,e,123 FROM MyTable" - ) + assertThatThrownBy( + () => + util.verifyRelPlanInsert( + "INSERT INTO metadata_sink (a,b,c,d,e,h) " + + "SELECT a,b,c,d,e,123 FROM MyTable" + )) + .hasMessageContaining( + "SQL validation failed. At line 1, column 38: Unknown target column 'h'") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testPartialInsertWithGroupBy(): Unit = { util.verifyExplainInsert( "INSERT INTO partitioned_sink (e,a,d) " + "SELECT e,a,d FROM MyTable GROUP BY a,b,c,d,e") } - @Test(expected = classOf[ValidationException]) + @TestTemplate def testPartialInsertCompositeType(): Unit = { // TODO this should be updated after FLINK-31301 fixed - util.verifyExplainInsert( - "INSERT INTO complex_type_sink (a,b.b1,c.c2,f) " + - "SELECT a,b.b1,c.c2,f FROM complex_type_src") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExplainInsert( + "INSERT INTO complex_type_sink (a,b.b1,c.c2,f) " + + "SELECT a,b.b1,c.c2,f FROM complex_type_src")) } } object PartialInsertTest { - @Parameterized.Parameters(name = "isBatch: {0}") + @Parameters(name = "isBatch: {0}") def parameters(): java.util.Collection[Boolean] = { java.util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala index 8a15558ff3dd6..9d5003c08da4b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala @@ -22,25 +22,26 @@ import org.apache.flink.table.factories.TableFactory import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory import org.apache.flink.table.planner.plan.utils.TestContextTableFactory import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Assert, Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util.Optional -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class TableFactoryTest(isBatch: Boolean) extends TableTestBase { private val util = if (isBatch) batchTestUtil() else streamTestUtil() - @Before + @BeforeEach def before(): Unit = { // we should clean the data to avoid serialization exception due to dirty data TestCollectionTableFactory.reset() } - @Test + @TestTemplate def testTableSourceSinkFactory(): Unit = { val factory = new TestContextTableFactory( ObjectIdentifier.of("cat", "default", "t1"), @@ -87,13 +88,13 @@ class TableFactoryTest(isBatch: Boolean) extends TableTestBase { util.tableEnv.executeSql(sinkDDL) util.tableEnv.explainSql(query) - Assert.assertTrue(factory.hasInvokedSource) - Assert.assertTrue(factory.hasInvokedSink) + assertThat(factory.hasInvokedSource).isTrue + assertThat(factory.hasInvokedSink).isTrue } } object TableFactoryTest { - @Parameterized.Parameters(name = "isBatch: {0}") + @Parameters(name = "isBatch: {0}") def parameters(): java.util.Collection[Boolean] = { java.util.Arrays.asList(true, false) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala index c119dca088429..00e87af032e8c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala index 6420991531960..0e5e449d7ea5e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala @@ -23,20 +23,18 @@ import org.apache.flink.table.catalog._ import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.plan.common.ViewsExpandingTest.PrimitiveScalarFunction import org.apache.flink.table.planner.utils.{TableFunc0, TableTestBase, TableTestUtil, TableTestUtilBase} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.hamcrest.CoreMatchers.is -import org.junit.Assert.assertThat -import org.junit.Test -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.runners.Parameterized.Parameters +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.TestTemplate +import org.junit.jupiter.api.extension.ExtendWith import java.util -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends TableTestBase { - @Test + @TestTemplate def testMixedSqlTableViewExpanding(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -56,7 +54,7 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends tableUtil.verifyExecPlan("select * from view4") } - @Test + @TestTemplate def testTableApiExpanding(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -69,7 +67,7 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends tableUtil.verifyExecPlan(query) } - @Test + @TestTemplate def testSqlExpanding(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -92,7 +90,7 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends tableUtil.verifyExecPlan(query) } - @Test + @TestTemplate def testViewExpandingWithMismatchRowType(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -117,7 +115,7 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends tableUtil.verifyExecPlan("select * from view1") } - @Test + @TestTemplate def testViewExpandingWithLateralTableFunction(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -135,7 +133,7 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends tableUtil.verifyExecPlan("select * from tmp_view") } - @Test + @TestTemplate def testViewExpandingWithBuiltinFunction(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -152,11 +150,11 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends .get() .getTable(objectID.toObjectPath) assertThat( - view.asInstanceOf[CatalogView].getExpandedQuery, - is("SELECT `CONCAT`('a', 'bc', 'def')")) + view.asInstanceOf[CatalogView].getExpandedQuery + ).isEqualTo("SELECT `CONCAT`('a', 'bc', 'def')") } - @Test + @TestTemplate def testViewExpandingWithUDF(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -174,11 +172,11 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends .get() .getTable(objectID.toObjectPath) assertThat( - view.asInstanceOf[CatalogView].getExpandedQuery, - is("SELECT `default_catalog`.`default_database`.`func`(1, 2, 'abc')")) + view.asInstanceOf[CatalogView].getExpandedQuery + ).isEqualTo("SELECT `default_catalog`.`default_database`.`func`(1, 2, 'abc')") } - @Test + @TestTemplate def testExpandQueryWithSystemAlias(): Unit = { val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv @@ -202,15 +200,14 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends .get() .getTable(objectID.toObjectPath) assertThat( - view.asInstanceOf[CatalogView].getExpandedQuery, - is( - "SELECT *\n" - + "FROM (SELECT `source`.`f0`, " - + "ROW_NUMBER() " - + "OVER (PARTITION BY `source`.`f0` ORDER BY `source`.`f0` DESC) AS `rowNum`\n" - + "FROM `default_catalog`.`default_database`.`source`)\n" - + "WHERE `rowNum` = 1") - ) + view.asInstanceOf[CatalogView].getExpandedQuery + ).isEqualTo( + "SELECT *\n" + + "FROM (SELECT `source`.`f0`, " + + "ROW_NUMBER() " + + "OVER (PARTITION BY `source`.`f0` ORDER BY `source`.`f0` DESC) AS `rowNum`\n" + + "FROM `default_catalog`.`default_database`.`source`)\n" + + "WHERE `rowNum` = 1") } private def createSqlView(originTable: String): CatalogView = { @@ -233,8 +230,8 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends object ViewsExpandingTest { @Parameters - def parameters(): Array[TableTestBase => TableTestUtilBase] = { - Array(_.batchTestUtil(), _.streamTestUtil()) + def parameters(): util.Collection[TableTestBase => TableTestUtilBase] = { + util.Arrays.asList(_.batchTestUtil(), _.streamTestUtil()) } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala index f66034fec7284..aad9d7b9a47de 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala @@ -25,23 +25,22 @@ import org.apache.flink.table.planner.JHashMap import org.apache.flink.table.planner.plan.hint.OptionsHintTest.{IS_BOUNDED, Param} import org.apache.flink.table.planner.plan.nodes.calcite.LogicalLegacySink import org.apache.flink.table.planner.utils.{OptionsTableSink, TableTestBase, TableTestUtil} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.assertj.core.api.Assertions.assertThatThrownBy -import org.hamcrest.Matchers._ -import org.junit.{Before, Test} -import org.junit.Assert.{assertEquals, assertThat} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized -import org.junit.runners.Parameterized.Parameters +import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith + +import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class OptionsHintTest(param: Param) extends TableTestBase { private val util = param.utilSupplier.apply(this) private val is_bounded = param.isBounded - @Before + @BeforeEach def before(): Unit = { util.addTable(s""" |create table t1( @@ -70,18 +69,19 @@ class OptionsHintTest(param: Param) extends TableTestBase { """.stripMargin) } - @Test + @TestTemplate def testOptionsWithGlobalConfDisabled(): Unit = { util.tableEnv.getConfig .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, Boolean.box(false)) - expectedException.expect(isA(classOf[ValidationException])) - expectedException.expectMessage( - s"OPTIONS hint is allowed only when " + + assertThatThrownBy( + () => util.verifyExecPlan("select * from t1/*+ OPTIONS(connector='COLLECTION', k2='#v2') */")) + .hasMessageContaining(s"OPTIONS hint is allowed only when " + s"${TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED.key} is set to true") - util.verifyExecPlan("select * from t1/*+ OPTIONS(connector='COLLECTION', k2='#v2') */") + .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testInsertWithDynamicOptions(): Unit = { val sql = s""" @@ -92,23 +92,24 @@ class OptionsHintTest(param: Param) extends TableTestBase { stmtSet.addInsertSql(sql) val testStmtSet = stmtSet.asInstanceOf[StatementSetImpl[_]] val relNodes = testStmtSet.getOperations.map(util.getPlanner.translateToRel) - assertThat(relNodes.length, is(1)) + assertThat(relNodes.length).isOne assert(relNodes.head.isInstanceOf[LogicalLegacySink]) val sink = relNodes.head.asInstanceOf[LogicalLegacySink] - assertEquals("{k1=#v1, k2=v2, k5=v5}", sink.sink.asInstanceOf[OptionsTableSink].props.toString) + assertThat(sink.sink.asInstanceOf[OptionsTableSink].props.toString) + .isEqualTo("{k1=#v1, k2=v2, k5=v5}") } - @Test + @TestTemplate def testAppendOptions(): Unit = { util.verifyExecPlan("select * from t1/*+ OPTIONS(k5='v5', 'a.b.c'='fakeVal') */") } - @Test + @TestTemplate def testOverrideOptions(): Unit = { util.verifyExecPlan("select * from t1/*+ OPTIONS(k1='#v1', k2='#v2') */") } - @Test + @TestTemplate def testJoinWithAppendedOptions(): Unit = { val sql = s""" @@ -121,7 +122,7 @@ class OptionsHintTest(param: Param) extends TableTestBase { util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinWithOverriddenOptions(): Unit = { val sql = s""" @@ -134,7 +135,7 @@ class OptionsHintTest(param: Param) extends TableTestBase { util.verifyExecPlan(sql) } - @Test + @TestTemplate def testOptionsHintOnTableApiView(): Unit = { val view1 = util.tableEnv.sqlQuery("select * from t1 join t2 on t1.a = t2.d") util.tableEnv.createTemporaryView("view1", view1) @@ -146,7 +147,7 @@ class OptionsHintTest(param: Param) extends TableTestBase { .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testOptionsHintOnSQLView(): Unit = { // Equivalent SQL: // select * from t1 join t2 on t1.a = t2.d @@ -182,7 +183,7 @@ class OptionsHintTest(param: Param) extends TableTestBase { .isInstanceOf[ValidationException] } - @Test + @TestTemplate def testOptionsHintInsideView(): Unit = { util.tableEnv.executeSql( "create view v1 as select * from t1 /*+ OPTIONS(k1='#v111', k4='#v444')*/") @@ -199,8 +200,10 @@ object OptionsHintTest { override def toString: String = s"$IS_BOUNDED=$isBounded" } - @Parameters(name = "{index}: {0}") - def parameters(): Array[Param] = { - Array(Param(_.batchTestUtil(), isBounded = true), Param(_.streamTestUtil(), isBounded = false)) + @Parameters(name = "{0}") + def parameters(): util.Collection[Param] = { + util.Arrays.asList( + Param(_.batchTestUtil(), isBounded = true), + Param(_.streamTestUtil(), isBounded = false)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.scala index e122196f95f8b..28ca3b7ecee32 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.scala @@ -21,12 +21,12 @@ import org.apache.flink.table.planner.plan.common.AggregateReduceGroupingTestBas import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.calcite.tools.RuleSets -import org.junit.Before +import org.junit.jupiter.api.BeforeEach /** Test for [[AggregateReduceGroupingRule]]. */ class AggregateReduceGroupingRuleTest extends AggregateReduceGroupingTestBase(false) { - @Before + @BeforeEach override def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.LOGICAL_REWRITE) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala index 2bb626855395c..08ab09fe3d920 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala @@ -22,10 +22,12 @@ import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets +import org.junit.jupiter.api.BeforeEach /** Test for [[PruneAggregateCallRule]]#CALC_ON_AGGREGATE. */ class CalcPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase { + @BeforeEach override def setup(): Unit = { super.setup() util.buildBatchProgram(FlinkBatchProgram.LOGICAL) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala index 45b3d773f4514..ba5c2e075ea76 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{MockPythonTableFunction, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class CalcPythonCorrelateTransposeRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[StreamOptimizeContext]() // query decorrelation diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.scala index b499d13f47c9d..1b36fd04333f4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.scala @@ -22,13 +22,13 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program.FlinkStreamProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[CalcRankTransposeRule]]. */ class CalcRankTransposeRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildStreamProgram(FlinkStreamProgram.PHYSICAL) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.scala index ffdf8f95ef124..83f022d536352 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.scala @@ -24,13 +24,13 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[ConvertToNotInOrInRule]]. */ class ConvertToNotInOrInRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRuleTest.scala index e8836883f5738..e47f6090fc6cb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRuleTest.scala @@ -22,13 +22,13 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[CalcRankTransposeRule]]. */ class CorrelateSortToRankRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[StreamOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala index 2391abdc0a3a7..4b011eb459f62 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test /** Test for [[DecomposeGroupingSetsRule]]. */ class DecomposeGroupingSetsRuleTest extends TableTestBase { @@ -126,7 +127,7 @@ class DecomposeGroupingSetsRuleTest extends TableTestBase { util.verifyRelPlan(sqlQuery) } - @Test(expected = classOf[RuntimeException]) + @Test def testTooManyGroupingFields(): Unit = { // max group count must be less than 64 val fieldNames = (0 until 64).map(i => s"f$i").toArray @@ -136,6 +137,7 @@ class DecomposeGroupingSetsRuleTest extends TableTestBase { val fields = fieldNames.mkString(",") val sqlQuery = s"SELECT $fields FROM MyTable64 GROUP BY GROUPING SETS ($fields)" - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala index 166fb7902136c..943b6e4c7fafe 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} import org.apache.flink.table.planner.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test /** Test for [[org.apache.flink.table.planner.codegen.ExpressionReducer]]. */ class ExpressionReductionRulesTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala index adc279f768e70..7dac86ae882e9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala @@ -20,12 +20,12 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.table.planner.plan.common.DistinctAggregateTestBase import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkAggregateExpandDistinctAggregatesRule]]. */ class FlinkAggregateExpandDistinctAggregatesRuleTest extends DistinctAggregateTestBase(false) { - @Before + @BeforeEach override def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.PHYSICAL) super.setup() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala index cae28bd043672..46606382e9d63 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala @@ -28,13 +28,13 @@ import com.google.common.collect.ImmutableSet import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkAggregateJoinTransposeRule]]. */ class FlinkAggregateJoinTransposeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val program = new FlinkChainedProgram[BatchOptimizeContext]() program.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala index 127f6805911df..bf245d686dcdb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala @@ -31,13 +31,13 @@ import com.google.common.collect.ImmutableSet import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkAggregateRemoveRule]]. */ class FlinkAggregateRemoveRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala index fab16c1895f6b..4b5165e4ab1d4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala @@ -29,13 +29,13 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkCalcMergeRule]]. */ class FlinkCalcMergeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala index 675a15bbe5a4e..8ae98ea755714 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[FlinkJoinPushExpressionsRule]]. */ class FlinkJoinPushExpressionsRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala index fcde31f624777..5b3d6aaddfbcb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala @@ -25,13 +25,13 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[org.apache.flink.table.planner.plan.rules.logical.FlinkJoinToMultiJoinRule]]. */ class FlinkJoinToMultiJoinRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala index a48e5e82abf94..cb65eb99bd667 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkLimit0RemoveRule]]. */ class FlinkLimit0RemoveRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala index 9b9464153cb12..e12d306e602ce 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test /** Test for [[FlinkLogicalRankRuleForConstantRange]]. */ class FlinkLogicalRankRuleForConstantRangeTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala index d937ebae9939a..e06a78ba2246d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program.FlinkStreamProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test /** Test for [[FlinkLogicalRankRuleForRangeEnd]]. */ class FlinkLogicalRankRuleForRangeEndTest extends TableTestBase { @@ -77,9 +78,9 @@ class FlinkLogicalRankRuleForRangeEndTest extends TableTestBase { | SELECT a, b, RANK() OVER (PARTITION BY b ORDER BY a, c) rk FROM MyTable) t |WHERE rk > 2 """.stripMargin - thrown.expectMessage("Rank end is not specified.") - thrown.expect(classOf[TableException]) - util.verifyRelPlan(sqlQuery) + assertThatThrownBy(() => util.verifyRelPlan(sqlQuery)) + .hasMessageContaining("Rank end is not specified.") + .isInstanceOf[TableException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala index 0d5926d644b7c..1285d0600eb0b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala @@ -25,14 +25,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.{CoreRules, PruneEmptyRules} import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkPruneEmptyRules]]. */ class FlinkPruneEmptyRulesTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala index 79d6fa8fd1514..7aa04829f42a6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkSemiAntiJoinFilterTransposeRule]]. */ class FlinkSemiAntiJoinFilterTransposeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala index ab809d14ad8a1..1d82048b2ec93 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkSemiAntiJoinJoinTransposeRule]]. */ class FlinkSemiAntiJoinJoinTransposeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala index c4063d863a914..5fdfd81603798 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[FlinkSemiAntiJoinProjectTransposeRule]]. */ class FlinkSemiAntiJoinProjectTransposeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala index 91749b845d0c3..edcabf8454fa1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[JoinConditionEqualityTransferRule]]. */ class JoinConditionEqualityTransferRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala index 2d85026f5c22b..d22d325be637a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** * Test for [[JoinConditionTypeCoerceRule]]. Now only semi-join rewrite will lost the type @@ -33,7 +33,7 @@ import org.junit.{Before, Test} class JoinConditionTypeCoerceRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala index 5fc010132a231..649121c0c75f5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala @@ -25,14 +25,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[JoinDependentConditionDerivationRule]]. */ class JoinDependentConditionDerivationRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala index e05122b291858..802b107fd4c8e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} import scala.collection.JavaConversions._ @@ -35,7 +35,7 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.scala index c5f4498d77b3b..f7885029d50be 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.scala @@ -25,14 +25,15 @@ import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase, import org.apache.calcite.plan.RelOptRule import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[LogicalCorrelateToJoinFromTemporalTableRule]]. */ class LogicalCorrelateToJoinFromTemporalTableRuleTest extends TableTestBase { protected val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTable(""" |CREATE TABLE T1 ( @@ -116,45 +117,51 @@ class LogicalCorrelateToJoinFromTemporalTableRuleTest extends TableTestBase { @Test def testProcTimeTemporalJoinOnTrue(): Unit = { setUpCurrentRule(LogicalCorrelateToJoinFromTemporalTableRule.WITHOUT_FILTER) - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Currently the join key in " + + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 JOIN T3 FOR SYSTEM_TIME AS OF T1.rowtime AS dimTable " + + "ON TRUE")) + .hasMessageContaining("Currently the join key in " + "Temporal Table Join can not be empty.") - util.verifyRelPlan( - "SELECT * FROM T1 JOIN T3 FOR SYSTEM_TIME AS OF T1.rowtime AS dimTable " + - "ON TRUE") + .isInstanceOf[ValidationException] } @Test def testRowTimeTemporalJoinOnTrue(): Unit = { setUpCurrentRule(LogicalCorrelateToJoinFromTemporalTableRule.WITHOUT_FILTER) - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Currently the join key in " + + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 JOIN T3 FOR SYSTEM_TIME AS OF T1.proctime AS dimTable " + + "ON TRUE")) + .hasMessageContaining("Currently the join key in " + "Temporal Table Join can not be empty.") - util.verifyRelPlan( - "SELECT * FROM T1 JOIN T3 FOR SYSTEM_TIME AS OF T1.proctime AS dimTable " + - "ON TRUE") + .isInstanceOf[ValidationException] } @Test def testRightTemporalJoin(): Unit = { setUpCurrentRule(LogicalCorrelateToJoinFromTemporalTableRule.WITH_FILTER) - expectedException.expect(classOf[AssertionError]) - expectedException.expectMessage("Correlate has invalid join type RIGHT") - util.verifyRelPlan( - "SELECT * FROM T1 RIGHT JOIN T3 FOR SYSTEM_TIME AS OF T1.rowtime AS dimTable " + - "ON T1.id = dimTable.id AND dimTable.rate > 10") + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 RIGHT JOIN T3 FOR SYSTEM_TIME AS OF T1.proctime AS dimTable " + + "ON T1.id = dimTable.id AND dimTable.rate > 10")) + .hasMessageContaining("Correlate has invalid join type RIGHT") + .isInstanceOf[AssertionError] } @Test def testFullTemporalJoin(): Unit = { setUpCurrentRule(LogicalCorrelateToJoinFromTemporalTableRule.WITH_FILTER) - expectedException.expect(classOf[AssertionError]) - expectedException.expectMessage("Correlate has invalid join type FULL") - util.verifyRelPlan( - "SELECT * FROM T1 FULL JOIN T3 FOR SYSTEM_TIME AS OF T1.rowtime AS dimTable " + - "ON T1.id = dimTable.id AND dimTable.rate > 10") + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 FULL JOIN T3 FOR SYSTEM_TIME AS OF T1.proctime AS dimTable " + + "ON T1.id = dimTable.id AND dimTable.rate > 10")) + .hasMessageContaining("Correlate has invalid join type FULL") + .isInstanceOf[AssertionError] } def setUpCurrentRule(rule: RelOptRule): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.scala index 8a791494fbacf..41ccd810e23d4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.scala @@ -23,14 +23,14 @@ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestUtil} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.Before +import org.junit.jupiter.api.BeforeEach /** Test for [[LogicalUnnestRule]]. */ class LogicalUnnestRuleTest extends UnnestTestBase(false) { override protected def getTableTestUtil: TableTestUtil = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala index 8ee8b4f80050e..02c255ced9578 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala @@ -23,10 +23,12 @@ import org.apache.flink.table.planner.utils.TableConfigUtils import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets +import org.junit.jupiter.api.BeforeEach /** Test for [[PruneAggregateCallRule]]#PROJECT_ON_AGGREGATE. */ class ProjectPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase { + @BeforeEach override def setup(): Unit = { super.setup() util.buildBatchProgram(FlinkBatchProgram.LOGICAL) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala index 403a3985729ae..2ec64e94bc612 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala @@ -25,14 +25,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[ProjectSemiAntiJoinTransposeRule]]. */ class ProjectSemiAntiJoinTransposeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRuleTest.scala index 0887d5dace2ed..4dbdac2789f29 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRuleTest.scala @@ -23,13 +23,13 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[ProjectWindowTableFunctionTransposeRule]]. */ class ProjectWindowTableFunctionTransposeRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[StreamOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala index d93251215f27f..9ceaec78c00cf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala @@ -24,13 +24,13 @@ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} import com.google.common.collect.ImmutableSet -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Base test class for [[PruneAggregateCallRule]]. */ abstract class PruneAggregateCallRuleTestBase extends TableTestBase { protected val util: BatchTableTestUtil = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource( "T1", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala index b2c2a4da7f0e8..d418dbeeffbee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.types.Row import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[PushFilterIntoLegacyTableSourceScanRule]]. */ class PushFilterIntoLegacyTableSourceScanRuleTest extends PushFilterIntoTableSourceScanRuleTestBase { - @Before + @BeforeEach def setup(): Unit = { util = batchTestUtil() util.asInstanceOf[BatchTableTestUtil].buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.scala index eccaf446e0390..4f30db4c81386 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.scala @@ -25,13 +25,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[PushLimitIntoLegacyTableSourceScanRule]]. */ class PushLimitIntoLegacyTableSourceScanRuleTest extends TableTestBase { protected val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) @@ -64,19 +65,22 @@ class PushLimitIntoLegacyTableSourceScanRuleTest extends TableTestBase { util.tableEnv.executeSql(ddl) } - @Test(expected = classOf[SqlParserException]) + @Test def testLimitWithNegativeOffset(): Unit = { - util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT 10 OFFSET -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT 10 OFFSET -1")) } - @Test(expected = classOf[SqlParserException]) + @Test def testNegativeLimitWithoutOffset(): Unit = { - util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT -1")) } - @Test(expected = classOf[SqlParserException]) + @Test def testMysqlLimit(): Unit = { - util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT 1, 10") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyRelPlan("SELECT a, c FROM LimitTable LIMIT 1, 10")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRuleTest.scala index 206ae9a30f580..03b3fa9c7462e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRuleTest.scala @@ -23,20 +23,20 @@ import org.apache.flink.table.expressions.utils.ResolvedExpressionMock import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE} import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableConfigUtils, TableTestBase, TestPartitionableSourceFactory} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util import scala.collection.JavaConversions._ /** Test for [[PushPartitionIntoLegacyTableSourceScanRule]]. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class PushPartitionIntoLegacyTableSourceScanRuleTest( val sourceFetchPartitions: Boolean, val useCatalogFilter: Boolean) @@ -44,7 +44,7 @@ class PushPartitionIntoLegacyTableSourceScanRuleTest( protected val util: BatchTableTestUtil = batchTestUtil() @throws(classOf[Exception]) - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) @@ -93,93 +93,93 @@ class PushPartitionIntoLegacyTableSourceScanRuleTest( isBounded = true) } - @Test + @TestTemplate def testNoPartitionFieldPredicate(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE id > 2") } - @Test + @TestTemplate def testNoPartitionFieldPredicateWithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE id > 2") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate1(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE part1 = 'A'") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate1WithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE part1 = 'A'") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate2(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE part2 > 1") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate2WithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE part2 > 1") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate3(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE part1 = 'A' AND part2 > 1") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate3WithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE part1 = 'A' AND part2 > 1") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate4(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE part1 = 'A' OR part2 > 1") } - @Test + @TestTemplate def testOnlyPartitionFieldPredicate4WithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE part1 = 'A' OR part2 > 1") } - @Test + @TestTemplate def testPartitionFieldPredicateAndOtherPredicate(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE id > 2 AND part1 = 'A'") } - @Test + @TestTemplate def testPartitionFieldPredicateAndOtherPredicateWithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE id > 2 AND part1 = 'A'") } - @Test + @TestTemplate def testPartitionFieldPredicateOrOtherPredicate(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE id > 2 OR part1 = 'A'") } - @Test + @TestTemplate def testPartitionFieldPredicateOrOtherPredicateWithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE id > 2 OR part1 = 'A'") } - @Test + @TestTemplate def testPartialPartitionFieldPredicatePushDown(): Unit = { util.verifyRelPlan("SELECT * FROM MyTable WHERE (id > 2 OR part1 = 'A') AND part2 > 1") } - @Test + @TestTemplate def testPartialPartitionFieldPredicatePushDownWithVirtualColumn(): Unit = { util.verifyRelPlan("SELECT * FROM VirtualTable WHERE (id > 2 OR part1 = 'A') AND part2 > 1") } - @Test + @TestTemplate def testWithUdf(): Unit = { util.addFunction("MyUdf", Func1) util.verifyRelPlan("SELECT * FROM MyTable WHERE id > 2 AND MyUdf(part2) < 3") } - @Test + @TestTemplate def testWithUdfAndVirtualColumn(): Unit = { util.addFunction("MyUdf", Func1) util.verifyRelPlan("SELECT * FROM VirtualTable WHERE id > 2 AND MyUdf(part2) < 3") @@ -187,7 +187,7 @@ class PushPartitionIntoLegacyTableSourceScanRuleTest( } object PushPartitionIntoLegacyTableSourceScanRuleTest { - @Parameterized.Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") + @Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") def parameters(): util.Collection[Array[Any]] = { Seq[Array[Any]]( Array(true, false), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala index 2a46f5c0facc6..4fe843c71e9ef 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala @@ -27,14 +27,14 @@ import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableConfigUtil import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[PushProjectIntoLegacyTableSourceScanRule]]. */ class PushProjectIntoLegacyTableSourceScanRuleTest extends TableTestBase { protected val util: BatchTableTestUtil = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala index b7b0ea6403001..567bbd34d0c6c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala @@ -26,14 +26,14 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[PythonCalcSplitRule]]. */ class PythonCalcSplitRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala index 3dc9cebaa8457..82af9cf2eadce 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala @@ -26,12 +26,12 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{MockPythonTableFunction, TableFunc1, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class PythonCorrelateSplitRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[StreamOptimizeContext]() // query decorrelation diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.scala index cf53137957f8b..fbee5a25a05d5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[PythonMapMergeRule]]. */ class PythonMapMergeRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala index 2dd5c73a163f0..5334a51dc8849 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala @@ -22,13 +22,13 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program.FlinkStreamProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RankNumberColumnRemoveRule]]. */ class RankNumberColumnRemoveRuleTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildStreamProgram(FlinkStreamProgram.PHYSICAL) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.scala index 06ac8f798ad0a..93af994c6c97b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.scala @@ -21,14 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for RemoveSingleAggregateRule. */ class RemoveSingleAggregateRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Int)]("foo", 'x, 'y) util.addTableSource[(Int, String)]("bar", 'i, 's) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala index 89e6a0f141636..4804949b5c066 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[ReplaceIntersectWithSemiJoinRule]]. */ class ReplaceIntersectWithSemiJoinRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala index 39b2a0b4a64bc..d1cec8e88de5c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[ReplaceMinusWithAntiJoinRule]]. */ class ReplaceMinusWithAntiJoinRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.scala index 9398164923873..6af0c912f9376 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RewriteIntersectAllRule]]. */ class RewriteIntersectAllRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.scala index bd8e22f02064a..aa2a3eb04d9c3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RewriteMinusAllRule]]. */ class RewriteMinusAllRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala index 776e4d0d1ea64..00cd68a7e6521 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RewriteMultiJoinConditionRule]]. */ class RewriteMultiJoinConditionRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val program = new FlinkChainedProgram[BatchOptimizeContext]() program.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.scala index a7799c565197b..fae4bb6d70517 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[SimplifyFilterConditionRule]]. */ class SimplifyFilterConditionRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.scala index e731597fb626b..0b39809683576 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[SimplifyJoinConditionRule]]. */ class SimplifyJoinConditionRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala index 656c03895dae5..be22d7b46a467 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.plan.optimize.program.FlinkStreamProgram import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test /** IncrementalAggregateTest Test for [[SplitAggregateRule]]. */ class SplitAggregateRuleTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.scala index 3d67ed3e724b6..fd67aeac49753 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.scala @@ -26,14 +26,14 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{TableFunc2, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[SplitPythonConditionFromCorrelateRule]]. */ class SplitPythonConditionFromCorrelateRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() // query decorrelation diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala index 6f446b166b6a7..a2aa865016dba 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala @@ -26,14 +26,14 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[SplitPythonConditionFromJoinRule]]. */ class SplitPythonConditionFromJoinRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { val programs = new FlinkChainedProgram[BatchOptimizeContext]() programs.addLast( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRuleTest.scala index 68ede4fb31637..a4313aceedd67 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRuleTest.scala @@ -24,13 +24,14 @@ import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[TemporalJoinRewriteWithUniqueKeyRule]]. */ class TemporalJoinRewriteWithUniqueKeyRuleTest extends TableTestBase { protected val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildStreamProgram(PHYSICAL) val chainedProgram = util.getStreamProgram() @@ -116,13 +117,14 @@ class TemporalJoinRewriteWithUniqueKeyRuleTest extends TableTestBase { @Test def testPrimaryKeyInTemporalJoinOnTrue(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Currently the join key in " + + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 JOIN T2 FOR SYSTEM_TIME AS OF T1.rowtime AS T " + + "ON TRUE")) + .hasMessageContaining("Currently the join key in " + "Temporal Table Join can not be empty.") - util.verifyRelPlan( - "SELECT * FROM T1 JOIN T2 FOR SYSTEM_TIME AS OF T1.rowtime AS T " + - "ON TRUE") + .isInstanceOf[ValidationException] } @Test @@ -139,15 +141,17 @@ class TemporalJoinRewriteWithUniqueKeyRuleTest extends TableTestBase { |) """.stripMargin) - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Temporal Table Join requires primary key in versioned table," + - " but no primary key can be found. The physical plan is:\nFlinkLogicalJoin(" + - "condition=[AND(=($0, $4), __INITIAL_TEMPORAL_JOIN_CONDITION($3, $6," + - " __TEMPORAL_JOIN_LEFT_KEY($0), __TEMPORAL_JOIN_RIGHT_KEY($4)))], joinType=[left])") - util.verifyRelPlan( - "SELECT * FROM T1 LEFT JOIN noPkTable FOR SYSTEM_TIME AS OF " + - "T1.rowtime AS T ON T1.id = T.id") + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 LEFT JOIN noPkTable FOR SYSTEM_TIME AS OF " + + "T1.rowtime AS T ON T1.id = T.id")) + .hasMessageContaining( + "Temporal Table Join requires primary key in versioned table," + + " but no primary key can be found. The physical plan is:\nFlinkLogicalJoin(" + + "condition=[AND(=($0, $4), __INITIAL_TEMPORAL_JOIN_CONDITION($3, $6," + + " __TEMPORAL_JOIN_LEFT_KEY($0), __TEMPORAL_JOIN_RIGHT_KEY($4)))], joinType=[left])") + .isInstanceOf[ValidationException] } @Test @@ -160,25 +164,27 @@ class TemporalJoinRewriteWithUniqueKeyRuleTest extends TableTestBase { " ) T " + " WHERE rowNum = 2") - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Temporal Table Join requires primary key in versioned table," + + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 JOIN noPkView FOR SYSTEM_TIME AS OF " + + "T1.rowtime AS T ON T1.id = T.id")) + .hasMessageContaining("Temporal Table Join requires primary key in versioned table," + " but no primary key can be found. The physical plan is:\n" + "FlinkLogicalJoin(condition=[AND(=($0, $4), __INITIAL_TEMPORAL_JOIN_CONDITION(" + "$3, $6, __TEMPORAL_JOIN_LEFT_KEY($0), __TEMPORAL_JOIN_RIGHT_KEY($4)))], joinType=[inner])") - util.verifyRelPlan( - "SELECT * FROM T1 JOIN noPkView FOR SYSTEM_TIME AS OF " + - "T1.rowtime AS T ON T1.id = T.id") + .isInstanceOf[ValidationException] } @Test def testInferredPrimaryKeyInTemporalJoinOnTrue(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Currently the join key in " + + assertThatThrownBy( + () => + util.verifyRelPlan( + "SELECT * FROM T1 JOIN DeduplicatedView FOR SYSTEM_TIME AS OF " + + "T1.rowtime AS T ON TRUE")) + .hasMessageContaining("Currently the join key in " + "Temporal Table Join can not be empty.") - util.verifyRelPlan( - "SELECT * FROM T1 JOIN DeduplicatedView FOR SYSTEM_TIME AS OF " + - "T1.rowtime AS T ON TRUE") + .isInstanceOf[ValidationException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.scala index e4b4d81824091..36840e1861054 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.scala @@ -22,13 +22,13 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[WindowGroupReorderRule]]. */ class WindowGroupReorderRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildBatchProgram(FlinkBatchProgram.LOGICAL) util.addTableSource[(Int, Int, String)]("MyTable", 'a, 'b, 'c) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala index 82a464820d21e..a18f74148f731 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala @@ -20,12 +20,12 @@ package org.apache.flink.table.planner.plan.rules.logical.subquery import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[org.apache.flink.table.planner.plan.rules.logical.FlinkRewriteSubQueryRule]]. */ class FlinkRewriteSubQueryRuleTest extends SubQueryTestBase { - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala index b72f3760feb67..03509beb0b330 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test /** * Tests for [[org.apache.flink.table.planner.plan.rules.logical.FlinkSubQueryRemoveRule]], this @@ -210,14 +211,14 @@ class SubQueryAntiJoinTest extends SubQueryTestBase { def testNotInWithUncorrelatedOnWhere_Case7(): Unit = { util.addTableSource[(Int)]("t1", 'i) - // TODO some bugs in SubQueryRemoveRule - thrown.expect(classOf[RuntimeException]) - // TODO Calcite does not support project with correlated expressions. val sqlQuery = "SELECT b FROM l WHERE " + "(CASE WHEN a NOT IN (SELECT i FROM t1 WHERE l.a = t1.i) THEN 1 ELSE 2 END) " + "NOT IN (SELECT d FROM r)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]") + + // TODO some bugs in SubQueryRemoveRule + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]")) } @Test @@ -616,13 +617,14 @@ class SubQueryAntiJoinTest extends SubQueryTestBase { util.verifyRelPlanNotExpected(sqlQuery4, "joinType=[anti]") } - @Test(expected = classOf[AssertionError]) + @Test def testNotExistsWithCorrelatedOnWhere_UnsupportedCondition2(): Unit = { // TODO Calcite decorrelateRel error val sqlQuery = "SELECT * FROM l WHERE NOT EXISTS " + " (SELECT * FROM (SELECT * FROM r WHERE r.d = l.a AND r.e > 100) s " + "LEFT JOIN t ON s.f = t.k AND l.b = t.j)" - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } @Test @@ -727,17 +729,17 @@ class SubQueryAntiJoinTest extends SubQueryTestBase { def testNotInNotExists3(): Unit = { util.addTableSource[(Int, Long, String)]("t2", 'l, 'm, 'n) - // TODO some bugs in SubQueryRemoveRule - // the result RelNode (LogicalJoin(condition=[=($1, $11)], joinType=[left])) - // after SubQueryRemoveRule is unexpected - thrown.expect(classOf[AssertionError]) - // TODO Calcite does not support project with correlated expressions. val sqlQuery = "SELECT c FROM l WHERE (" + " (CASE WHEN NOT EXISTS (SELECT * FROM t WHERE l.a = t.i) THEN 1 ELSE 2 END), " + " (CASE WHEN b NOT IN (SELECT m FROM t2) THEN 3 ELSE 4 END)) " + " NOT IN (SELECT d, e FROM r)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]") + + // TODO some bugs in SubQueryRemoveRule + // the result RelNode (LogicalJoin(condition=[=($1, $11)], joinType=[left])) + // after SubQueryRemoveRule is unexpected + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]")) } @Test @@ -756,16 +758,16 @@ class SubQueryAntiJoinTest extends SubQueryTestBase { def testInNotInExistsNotExists2(): Unit = { util.addTableSource[(Int, Long, String)]("t2", 'l, 'm, 'n) - // TODO some bugs in SubQueryRemoveRule - thrown.expect(classOf[RuntimeException]) - // TODO Calcite does not support project with correlated expressions. val sqlQuery = "SELECT c FROM l WHERE (" + " (CASE WHEN b IN (SELECT j FROM t WHERE l.a = t.i) THEN 1 ELSE 2 END), " + " (CASE WHEN NOT EXISTS (SELECT m FROM t2) THEN 3 " + " WHEN EXISTS (select i FROM t) THEN 4 ELSE 5 END)) " + " NOT IN (SELECT d, e FROM r)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]") + + // TODO some bugs in SubQueryRemoveRule + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]")) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala index f78017b77fd6d..6170dcb29986c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test /** * Tests for [[org.apache.flink.table.planner.plan.rules.logical.FlinkSubQueryRemoveRule]], this @@ -116,13 +117,13 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithUncorrelatedOnWhere_UnsupportedCondition1(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - // these queries will not be converted to joinType=[semi] val sqlQuery = "SELECT * FROM x WHERE a IN (SELECT c FROM y WHERE x.b IN (SELECT e FROM z))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -206,13 +207,13 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { util.addTableSource[(Int)]("t1", 'i) util.addTableSource[(Int)]("t2", 'j) - // TODO some bugs in SubQueryRemoveRule - thrown.expect(classOf[RuntimeException]) - // TODO Calcite does not support project with correlated expressions. val sqlQuery = "SELECT b FROM l WHERE" + " (CASE WHEN a IN (SELECT i FROM t1 WHERE l.a = t1.i) THEN 1 ELSE 2 END) IN (SELECT d FROM r)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + // TODO some bugs in SubQueryRemoveRule + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } @Test @@ -245,13 +246,13 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithUncorrelatedOnWhere_ScalarQuery5(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT b FROM x WHERE a IN (SELECT c FROM y WHERE d > " + "(SELECT 0.5 * SUM(e) FROM z WHERE x.a = z.e AND z.f < 100))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -474,13 +475,15 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { util.verifyRelPlanNotExpected(sqlQuery4, "joinType=[semi]") } - @Test(expected = classOf[AssertionError]) + @Test def testInWithCorrelatedOnWhere_UnsupportedCondition2(): Unit = { // TODO java.lang.RuntimeException: While invoking method // 'public RelDecorrelator$Frame RelDecorrelator.decorrelateRel(LogicalProject)' val sqlQuery = "SELECT * FROM l WHERE a IN (SELECT d FROM r WHERE l.b IN (SELECT j FROM t) " + "AND l.c = r.f)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } @Test @@ -570,15 +573,15 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithCorrelatedOnWhere_ScalarQuery4(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - // nested correlation can not be converted joinType=[semi] now val sqlQuery = "SELECT a FROM x WHERE " + "(SELECT MAX(d) FROM y WHERE c IN (SELECT e FROM z WHERE x.b = z.f))" + " IN (SELECT f FROM z WHERE z.e = x.a)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -602,13 +605,14 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { util.verifyRelPlan(sqlQuery) } - @Test(expected = classOf[AssertionError]) + @Test def testInWithCorrelatedOnWhere_ScalarQuery8(): Unit = { // nested correlation can not be converted joinType=[semi] now // TODO There are some bugs when decorrelating in RelDecorrelator val sqlQuery = "SELECT b FROM x WHERE a IN (SELECT c FROM y WHERE x.b = y.d AND c > " + "(SELECT 0.5 * SUM(e) FROM z WHERE x.a = z.e AND z.f < 100))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } @Test @@ -831,29 +835,29 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithCorrelatedOnWhere_Union1(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - // UNION with correlation can not be converted to semi-join val sqlQuery = "SELECT a FROM l WHERE b IN " + "(SELECT e FROM r WHERE l.a = r.d AND d > 10 " + "UNION " + "SELECT i FROM t WHERE i < 100)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testInWithCorrelatedOnWhere_Union2(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT a FROM l WHERE b IN " + "(SELECT e FROM r WHERE l.a = r.d AND d > 10 " + "UNION " + "SELECT i FROM t WHERE l.c = t.k AND i < 100)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -991,38 +995,37 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithCorrelatedOnLateralTable2(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE c IN (" + "SELECT f1 FROM r, LATERAL TABLE(table_func(f)) AS T(f1) " + "WHERE d IN (SELECT i FROM t WHERE l.b = t.j))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testInWithCorrelatedOnLateralTable3(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE c IN (" + "SELECT f1 FROM (SELECT * FROM r WHERE d IN (" + "SELECT i FROM t WHERE t.j = l.b)) m, LATERAL TABLE(table_func(f)) AS T(f1))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testInWithCorrelatedOnLateralTable4(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE c IN (" + "SELECT f1 FROM (SELECT * FROM r LEFT JOIN LATERAL TABLE(table_func(f)) AS T(f1) ON TRUE " + "WHERE d IN (SELECT i FROM t WHERE l.b = t.j)))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -1257,12 +1260,13 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { util.verifyRelPlanNotExpected(sqlQuery4, "joinType=[semi]") } - @Test(expected = classOf[AssertionError]) + @Test def testExistsWithCorrelatedOnWhere_UnsupportedCondition2(): Unit = { val sqlQuery = "SELECT * FROM l WHERE EXISTS " + " (SELECT * FROM (SELECT * FROM r WHERE r.d = l.a AND r.e > 100) s " + "LEFT JOIN t ON s.f = t.k AND l.b = t.j)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } @Test @@ -1432,29 +1436,27 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testExistsWithCorrelatedOnWhere_Union1(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - // UNION with correlation is not supported val sqlQuery = "SELECT a FROM l WHERE EXISTS " + "(SELECT e FROM r WHERE l.a = r.d AND d > 10 " + "UNION " + "SELECT i FROM t WHERE i < 100)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testExistsWithCorrelatedOnWhere_Union2(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT a FROM l WHERE EXISTS " + "(SELECT e FROM r WHERE l.a = r.d AND d > 10 " + "UNION " + "SELECT i FROM t WHERE l.c = t.k AND i < 100)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -1609,38 +1611,36 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testExistsWithCorrelatedOnLateralTable2(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE EXISTS (" + "SELECT * FROM r, LATERAL TABLE(table_func(f)) AS T(f1) " + "WHERE EXISTS (SELECT * FROM t WHERE l.b = t.j))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testExistsWithCorrelatedOnLateralTable3(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE EXISTS (" + "SELECT * FROM (SELECT * FROM r WHERE EXISTS (" + "SELECT * FROM t WHERE t.j = l.b)) m, LATERAL TABLE(table_func(f)) AS T(f1))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test def testExistsWithCorrelatedOnLateralTable4(): Unit = { - thrown.expect(classOf[TableException]) - // correlate variable id is unstable, ignore here - thrown.expectMessage("unexpected correlate variable $cor") - val sqlQuery = "SELECT * FROM l WHERE EXISTS (" + "SELECT * FROM (SELECT * FROM r LEFT JOIN LATERAL TABLE(table_func(f)) AS T(f1) ON TRUE " + "WHERE EXISTS (SELECT i FROM t WHERE l.b = t.j)))" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + assertThatThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) + // correlate variable id is unstable, ignore here + .hasMessageContaining("unexpected correlate variable $cor") + .isInstanceOf[TableException] } @Test @@ -1661,17 +1661,17 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { def testInExists3(): Unit = { util.addTableSource[(Int, Long, String)]("t2", 'l, 'm, 'n) - // TODO some bugs in SubQueryRemoveRule - // the result RelNode (LogicalJoin(condition=[=($1, $8)], joinType=[left])) - // after SubQueryRemoveRule is unexpected - thrown.expect(classOf[AssertionError]) - // TODO Calcite does not support project with correlated expressions. val sqlQuery = "SELECT c FROM l WHERE (" + " (CASE WHEN EXISTS (SELECT * FROM t WHERE l.a = t.i) THEN 1 ELSE 2 END), " + " (CASE WHEN b IN (SELECT m FROM t2) THEN 3 ELSE 4 END)) " + " IN (SELECT d, e FROM r)" - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + + // TODO some bugs in SubQueryRemoveRule + // the result RelNode (LogicalJoin(condition=[=($1, $8)], joinType=[left])) + // after SubQueryRemoveRule is unexpected + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala index bde051543f401..8b9339913e5d3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.rules.logical.subquery import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test import java.sql.{Date, Timestamp} @@ -83,7 +84,7 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { util.verifyRelPlan(sqlQuery) } - @Test(expected = classOf[RuntimeException]) + @Test def testWithProjectJoinCorrelate(): Unit = { val sqlQuery = """ @@ -93,7 +94,8 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { |FROM t1 | WHERE t1a = 'val1b' """.stripMargin - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } @Test @@ -109,7 +111,7 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { util.verifyRelPlan(sqlQuery) } - @Test(expected = classOf[TableException]) + @Test def testWithFilterInCorrelate(): Unit = { val sqlQuery = """ @@ -120,10 +122,11 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { | WHERE t1.t1e | IN (select t2e from t2)) """.stripMargin - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testWithFilterExistsCorrelate(): Unit = { val sqlQuery = """ @@ -133,10 +136,11 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { | FROM t3 | WHERE EXISTS(select * from t3 WHERE t1.t1a = t3.t3a)) """.stripMargin - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } - @Test(expected = classOf[AssertionError]) + @Test // TODO some bugs in RelDecorrelator.AdjustProjectForCountAggregateRule def testWithProjectCaseWhenCorrelate(): Unit = { val sqlQuery = @@ -149,7 +153,8 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { |FROM t1 | WHERE t1a = 'test' """.stripMargin - util.verifyRelPlan(sqlQuery) + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlan(sqlQuery)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalAggRuleTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalAggRuleTestBase.scala index 4f4a2e15b8311..714baa21623b9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalAggRuleTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalAggRuleTestBase.scala @@ -21,12 +21,12 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} abstract class EnforceLocalAggRuleTestBase extends TableTestBase { protected val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String, Int)]("t", 'a, 'b, 'c, 'd) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala index 4435a04857a30..31cd921531f16 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala @@ -26,12 +26,12 @@ import org.apache.flink.table.planner.utils.TableConfigUtils import org.apache.calcite.rel.core.Aggregate import org.apache.calcite.tools.RuleSets -import org.junit.Before +import org.junit.jupiter.api.BeforeEach /** Test for [[EnforceLocalHashAggRule]]. */ class EnforceLocalHashAggRuleTest extends EnforceLocalAggRuleTestBase { - @Before + @BeforeEach override def setup(): Unit = { super.setup() val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala index cd566a4c2bf56..72aceb8ab7594 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala @@ -28,12 +28,12 @@ import org.apache.flink.table.planner.utils.TableConfigUtils import org.apache.calcite.rel.core.Aggregate import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[EnforceLocalSortAggRule]]. */ class EnforceLocalSortAggRuleTest extends EnforceLocalAggRuleTestBase { - @Before + @BeforeEach override def setup(): Unit = { super.setup() util.addFunction("weightedAvg", new WeightedAvg) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala index 99bd7be733eba..52f3f53f894cd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala @@ -22,14 +22,14 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RemoveRedundantLocalHashAggRule]]. */ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala index 56e7c2d69d051..56a90cb44ba2e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala @@ -21,14 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[RemoveRedundantLocalRankRule]]. */ class RemoveRedundantLocalRankRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala index d407adcfca4cd..11be16997f2ae 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala @@ -22,14 +22,14 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Test for [[RemoveRedundantLocalSortAggRule]]. */ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase { private val util = batchTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala index cb6266d6c5518..a40028bd35766 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala @@ -24,14 +24,14 @@ import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.plan.optimize.program.FlinkChangelogModeInferenceProgram import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[FlinkChangelogModeInferenceProgram]]. */ class ChangelogModeInferenceTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTable(""" |CREATE TABLE MyTable ( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala index 456afb9e7a478..881f7c7dddcc8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala @@ -27,13 +27,13 @@ import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[ExpandWindowTableFunctionTransposeRule]]. */ class ExpandWindowTableFunctionTransposeRuleTest extends TableTestBase { private val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.buildStreamProgram(PHYSICAL) val chainedProgram = util.getStreamProgram() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.scala index c02376ef48c8d..9d37d60bcd5f0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.scala @@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.rules.physical.stream import org.apache.flink.table.api.ExplainDetail import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for [[WatermarkAssignerChangelogNormalizeTransposeRule]] */ class WatermarkAssignerChangelogNormalizeTransposeRuleTest extends TableTestBase { private val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTable(s""" |CREATE TABLE simple_src ( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala index 3e74c00480fd8..ce0b864d31172 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala @@ -25,14 +25,15 @@ import org.apache.flink.table.planner.plan.utils.MyPojo import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} import java.sql.{Date, Time, Timestamp} class CalcTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Long, Int, String)]("MyTable", 'a, 'b, 'c) util.addFunction("random_udf", new NonDeterministicUdf) @@ -102,9 +103,10 @@ class CalcTest extends TableTestBase { util.verifyExecPlan("SELECT MyTable2.a.*, c, MyTable2.b.* FROM MyTable2") } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidFields(): Unit = { - util.tableEnv.sqlQuery("SELECT a, foo FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.tableEnv.sqlQuery("SELECT a, foo FROM MyTable")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala index c98a049c5f0c1..d2f10011ce178 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala @@ -25,7 +25,7 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase} import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} -import org.junit.Test +import org.junit.jupiter.api.Test class DagOptimizationTest extends TableTestBase { private val util = streamTestUtil() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala index dcb594b748ee3..093b3a12415d0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions.{TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, TABLE_EXEC_MINIBATCH_ENABLED, TABLE_EXEC_MINIBATCH_SIZE} import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} import java.time.Duration @@ -30,7 +30,7 @@ class DeduplicateTest extends TableTestBase { var util: StreamTableTestUtil = _ - @Before + @BeforeEach def setUp(): Unit = { util = streamTestUtil() util.addDataStream[(Int, String, Long)]( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/FilterableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/FilterableSourceTest.scala index 89cf7c0f70ddd..5c10b6285bbea 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/FilterableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/FilterableSourceTest.scala @@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for pushing filter into table scan */ class FilterableSourceTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val ddl = """ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacySinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacySinkTest.scala index 9b066752afc51..82cdbfbf3715d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacySinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacySinkTest.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test class LegacySinkTest extends TableTestBase { @@ -39,10 +40,10 @@ class LegacySinkTest extends TableTestBase { val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a") val appendSink = util.createAppendTableSink(Array("a"), Array(LONG)) - thrown.expect(classOf[TableException]) - thrown.expectMessage("AppendStreamTableSink doesn't support consuming update " + - "changes which is produced by node GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])") - util.verifyRelPlanInsert(table, appendSink, "appendSink") + assertThatThrownBy(() => util.verifyRelPlanInsert(table, appendSink, "appendSink")) + .hasMessageContaining("AppendStreamTableSink doesn't support consuming update " + + "changes which is produced by node GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])") + .isInstanceOf[TableException] } @Test @@ -65,11 +66,10 @@ class LegacySinkTest extends TableTestBase { .registerTableSinkInternal("retractSink2", retractSink2) stmtSet.addInsert("retractSink2", table2) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "OverAggregate doesn't support consuming update changes " + + assertThatThrownBy(() => util.verifyRelPlan(stmtSet)) + .hasMessageContaining("OverAggregate doesn't support consuming update changes " + "which is produced by node Calc(select=[cnt]") - util.verifyRelPlan(stmtSet) + .isInstanceOf[TableException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacyTableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacyTableSourceTest.scala index 2c72dfa178be4..73ddddb52c7af 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacyTableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LegacyTableSourceTest.scala @@ -25,7 +25,7 @@ import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.utils._ import org.apache.flink.types.Row -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class LegacyTableSourceTest extends TableTestBase { @@ -36,7 +36,7 @@ class LegacyTableSourceTest extends TableTestBase { .fields(Array("a", "b", "c"), Array(DataTypes.INT(), DataTypes.BIGINT(), DataTypes.STRING())) .build() - @Before + @BeforeEach def setup(): Unit = { TestLegacyFilterableTableSource.createTemporaryTable( util.tableEnv, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala index 0556ea9fb9a7c..2d5333ddca9a2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala @@ -21,13 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} class LimitTest extends TableTestBase { protected val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addDataStream[(Int, String, Long)]( "MyTable", @@ -48,9 +49,10 @@ class LimitTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM MyTable LIMIT 0") } - @Test(expected = classOf[SqlParserException]) + @Test def testNegativeLimitWithoutOffset(): Unit = { - util.verifyExecPlan("SELECT * FROM MyTable LIMIT -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyExecPlan("SELECT * FROM MyTable LIMIT -1")) } @Test @@ -73,9 +75,10 @@ class LimitTest extends TableTestBase { util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 0 OFFSET 10") } - @Test(expected = classOf[SqlParserException]) + @Test def testLimitWithNegativeOffset(): Unit = { - util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 10 OFFSET -1") + assertThatExceptionOfType(classOf[SqlParserException]) + .isThrownBy(() => util.verifyExecPlan("SELECT a, c FROM MyTable LIMIT 10 OFFSET -1")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala index 8321aacd8df01..2aaea867fda19 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala @@ -19,12 +19,12 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.table.planner.plan.rules.logical.PushLimitIntoTableSourceScanRule -import org.junit.Before +import org.junit.jupiter.api.BeforeEach /** Test for [[PushLimitIntoTableSourceScanRule]]. */ class LimitableSourceTest extends LimitTest { - @Before + @BeforeEach override def setup(): Unit = { val ddl = s""" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala index f1e3755ea175b..9267831de172c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala @@ -20,13 +20,14 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.table.api.ValidationException import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.{BeforeEach, Test} class MatchRecognizeTest extends TableTestBase { protected val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { val ddl = """ @@ -223,13 +224,6 @@ class MatchRecognizeTest extends TableTestBase { @Test def testMatchRowtimeWithoutArgumentOnRowtimeLTZ(): Unit = { - thrown.expectMessage( - "MATCH_ROWTIME(rowtimeField) should be used when input stream " + - "contains rowtime attribute with TIMESTAMP_LTZ type.\n" + - "Please pass rowtime attribute field as input argument of " + - "MATCH_ROWTIME(rowtimeField) function.") - thrown.expect(classOf[AssertionError]) - val sqlQuery = s""" |SELECT @@ -252,14 +246,18 @@ class MatchRecognizeTest extends TableTestBase { |) AS T |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) |""".stripMargin - util.verifyRelPlan(sqlQuery) + + assertThatThrownBy(() => util.verifyRelPlan(sqlQuery)) + .hasMessageContaining( + "MATCH_ROWTIME(rowtimeField) should be used when input stream " + + "contains rowtime attribute with TIMESTAMP_LTZ type.\n" + + "Please pass rowtime attribute field as input argument of " + + "MATCH_ROWTIME(rowtimeField) function.") + .isInstanceOf[AssertionError] } @Test def testMatchRowtimeWithMultipleArgs(): Unit = { - thrown.expectMessage("Invalid number of arguments to function 'MATCH_ROWTIME'.") - thrown.expect(classOf[ValidationException]) - val sqlQuery = s""" |SELECT @@ -282,16 +280,14 @@ class MatchRecognizeTest extends TableTestBase { |) AS T |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) |""".stripMargin - util.verifyRelPlan(sqlQuery) + + assertThatThrownBy(() => util.verifyRelPlan(sqlQuery)) + .hasMessageContaining("Invalid number of arguments to function 'MATCH_ROWTIME'.") + .isInstanceOf[ValidationException] } @Test def testMatchRowtimeWithNonRowTimeAttributeAsArgs(): Unit = { - thrown.expectMessage( - "The function MATCH_ROWTIME requires argument to be a row time attribute type, " + - "but is 'INTEGER'.") - thrown.expect(classOf[ValidationException]) - val sqlQuery = s""" |SELECT @@ -314,16 +310,16 @@ class MatchRecognizeTest extends TableTestBase { |) AS T |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) |""".stripMargin - util.verifyRelPlan(sqlQuery) + + assertThatThrownBy(() => util.verifyRelPlan(sqlQuery)) + .hasMessageContaining( + "The function MATCH_ROWTIME requires argument to be a row time attribute type, " + + "but is 'INTEGER'.") + .isInstanceOf[ValidationException] } @Test def testMatchRowtimeWithRexCallAsArg(): Unit = { - thrown.expectMessage( - "The function MATCH_ROWTIME requires a field reference as argument, " + - "but actual argument is not a simple field reference.") - thrown.expect(classOf[ValidationException]) - val sqlQuery = s""" |SELECT @@ -346,6 +342,10 @@ class MatchRecognizeTest extends TableTestBase { |) AS T |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) |""".stripMargin - util.verifyRelPlan(sqlQuery) + + assertThatThrownBy(() => util.verifyRelPlan(sqlQuery)) + .hasMessageContaining("The function MATCH_ROWTIME requires a field reference as argument, " + + "but actual argument is not a simple field reference.") + .isInstanceOf[ValidationException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala index 0e0c57bdc6d02..55dde3373ca57 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_ import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} import java.time.Duration @@ -37,7 +37,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { val LONG = new BigIntType() val INT = new IntType() - @Before + @BeforeEach def setup(): Unit = { util.addDataStream[(Int, String, Long)]( "MyDataStream1", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala index f053cb4d779a6..6a22ccbb889bd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala @@ -27,8 +27,8 @@ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.Wei import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase, TableTestUtil} import org.apache.calcite.sql.validate.SqlMonotonicity.{CONSTANT, DECREASING, INCREASING, NOT_MONOTONIC} -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.Test import java.time.Duration @@ -258,7 +258,7 @@ class ModifiedMonotonicityTest extends TableTestBase { val actualMono = FlinkRelMetadataQuery .reuseOrCreate(optimized.getCluster.getMetadataQuery) .getRelModifiedMonotonicity(optimized) - assertEquals(expect, actualMono) + assertThat(actualMono).isEqualTo(expect) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala index 6ee0e9f6f441d..6a3285bacf5e3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala @@ -35,14 +35,16 @@ import org.apache.flink.table.sinks.UpsertStreamTableSink import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.table.types.utils.TypeConversions +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.{assertThatCode, assertThatThrownBy} +import org.assertj.core.api.ThrowableAssert.ThrowingCallable +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUpdateStrategy) extends TableTestBase { @@ -50,7 +52,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp private val tryResolve = nonDeterministicUpdateStrategy == NonDeterministicUpdateStrategy.TRY_RESOLVE - @Before + @BeforeEach def before(): Unit = { util.tableConfig.getConfiguration.set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, @@ -217,7 +219,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp util.tableEnv.createTemporaryFunction("str_split", new StringSplit()) } - @Test + @TestTemplate def testCdcWithMetaSinkWithPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_with_pk @@ -225,45 +227,55 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |from cdc_with_meta |""".stripMargin) } - @Test + @TestTemplate def testNonDeterministicProjectionWithSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + """ + |insert into sink_without_pk + |select + | a, + | if(a > 100, b+d, b) as b, + | case when d > 100 then json_value(c, '$.count') + | else cast(b as string) || '#' end as c + |from ( + |select a, b, c, d from ( + | select *, row_number() over(partition by a order by d desc) as rn + | from ( + | select a, d as b, c, ndFunc(b) as d from cdc + | ) tmp + |) tmp where rn = 1) tmp + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "The column(s): d(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "The column(s): d(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(""" - |insert into sink_without_pk - |select - | a, - | if(a > 100, b+d, b) as b, - | case when d > 100 then json_value(c, '$.count') - | else cast(b as string) || '#' end as c - |from ( - |select a, b, c, d from ( - | select *, row_number() over(partition by a order by d desc) as rn - | from ( - | select a, d as b, c, ndFunc(b) as d from cdc - | ) tmp - |) tmp where rn = 1) tmp - |""".stripMargin) } - @Test + @TestTemplate def testCdcWithMetaSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select a, metadata_3, c + |from cdc_with_meta + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select a, metadata_3, c - |from cdc_with_meta - |""".stripMargin) } - @Test + @TestTemplate def testCdcWithMetaLegacySinkWithPk(): Unit = { val sinkWithPk = new TestingUpsertSink( Array("a"), @@ -282,13 +294,8 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcWithMetaLegacySinkWithoutPk(): Unit = { - if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) - } val retractSink = util.createRetractTableSink( Array("a", "b", "c"), @@ -297,34 +304,44 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp .asInstanceOf[TableEnvironmentInternal] .registerTableSinkInternal("legacy_retract_sink", retractSink) - util.verifyExecPlanInsert(s""" - |insert into legacy_retract_sink - |select a, metadata_3, c - |from cdc_with_meta - |""".stripMargin) + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into legacy_retract_sink + |select a, metadata_3, c + |from cdc_with_meta + |""".stripMargin) + + if (tryResolve) { + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() + } } - @Test + @TestTemplate def testCdcWithMetaSinkWithCompositePk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_composite_pk + |select a, b, c, metadata_3 + |from cdc_with_meta + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_composite_pk - |select a, b, c, metadata_3 - |from cdc_with_meta - |""".stripMargin) } - @Test + @TestTemplate def testCdcWithMetaRenameSinkWithCompositePk(): Unit = { - if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) - } util.tableEnv.executeSql(""" |create temporary table cdc_with_meta_rename ( | a int, @@ -340,30 +357,43 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp | 'readable-metadata' = 'metadata_3:BIGINT' |)""".stripMargin) - util.verifyExecPlanInsert(s""" - |insert into sink_with_composite_pk - |select a, b, c, e from cdc_with_meta_rename - |""".stripMargin) - } + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_composite_pk + |select a, b, c, e from cdc_with_meta_rename + |""".stripMargin) - @Test - def testSourceWithComputedColumnSinkWithPk(): Unit = { if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } + } + @TestTemplate + def testSourceWithComputedColumnSinkWithPk(): Unit = { // can not infer pk from cdc source with computed column(s) - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select a, b, `day` - |from cdc_with_computed_col - |where b > 100 - |""".stripMargin) + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select a, b, `day` + |from cdc_with_computed_col + |where b > 100 + |""".stripMargin) + + if (tryResolve) { + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() + } } - @Test + @TestTemplate def testSourceWithComputedColumnMultiSink(): Unit = { val stmtSet = util.tableEnv.createStatementSet() stmtSet.addInsertSql(s""" @@ -378,45 +408,60 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |from cdc_with_computed_col |where b > 100 |""".stripMargin) + + val callable: ThrowingCallable = () => util.verifyExecPlan(stmtSet) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan(stmtSet) } - @Test + @TestTemplate def testCdcCorrelateNonDeterministicFuncSinkWithPK(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select + | t1.a, t1.b, a1 + |from cdc t1, lateral table(ndTableFunc(a)) as T(a1) + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): EXPR$0(generated by non-deterministic function: ndTableFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): EXPR$0(generated by non-deterministic function: ndTableFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select - | t1.a, t1.b, a1 - |from cdc t1, lateral table(ndTableFunc(a)) as T(a1) - |""".stripMargin) } - @Test + @TestTemplate def testCdcCorrelateNonDeterministicFuncNoLeftOutput(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_pk(a) + |select + | cast(a1 as integer) a + |from cdc t1, lateral table(ndTableFunc(a)) as T(a1) + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): EXPR$0(generated by non-deterministic function: ndTableFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): EXPR$0(generated by non-deterministic function: ndTableFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk(a) - |select - | cast(a1 as integer) a - |from cdc t1, lateral table(ndTableFunc(a)) as T(a1) - |""".stripMargin) } - @Test + @TestTemplate def testCdcCorrelateNonDeterministicFuncNoRightOutput(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_with_pk @@ -425,37 +470,45 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcCorrelateOnNonDeterministicCondition(): Unit = { // TODO update this after FLINK-7865 was fixed - thrown.expectMessage("unexpected correlate variable $cor0 in the plan") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select a, b, c - |from cdc t1 join lateral table(str_split(c)) as T(c1) - | -- the join predicate can only be empty or literal true for now - | on ndFunc(b) > 100 - |""".stripMargin) - } - - @Test + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select a, b, c + |from cdc t1 join lateral table(str_split(c)) as T(c1) + | -- the join predicate can only be empty or literal true for now + | on ndFunc(b) > 100 + |""".stripMargin)) + .hasMessageContaining("unexpected correlate variable $cor0 in the plan") + .isInstanceOf[TableException] + } + + @TestTemplate def testCdcWithMetaCorrelateSinkWithPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select t1.a, t1.metadata_1, T.c1 + |from cdc_with_meta t1, lateral table(str_split(c)) as T(c1) + |""".stripMargin) + // Under ignore mode, the generated execution plan may cause wrong result though // upsertMaterialize has been enabled in sink, because if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_1' in cdc source may cause wrong result or error on downstream operators") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_1' in cdc source may cause wrong result or error on downstream operators") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select t1.a, t1.metadata_1, T.c1 - |from cdc_with_meta t1, lateral table(str_split(c)) as T(c1) - |""".stripMargin) } - @Test + @TestTemplate def testCdcWithNonDeterministicFuncSinkWithPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_with_pk @@ -464,21 +517,25 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcWithNonDeterministicFuncSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select a, ndFunc(b), c + |from cdc + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): EXPR$1(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): EXPR$1(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select a, ndFunc(b), c - |from cdc - |""".stripMargin) } - @Test + @TestTemplate def testCdcWithNonDeterministicFilter(): Unit = { // TODO should throw error if tryResolve is true after FLINK-28737 was fixed util.verifyExecPlanInsert(s""" @@ -489,7 +546,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkSinkWithPk(): Unit = { // The lookup key contains the dim table's pk, there will be no materialization. util.verifyExecPlanInsert(s""" @@ -502,7 +559,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithoutPkSinkWithPk(): Unit = { // This case shows how costly is if the dim table does not define a pk. // The lookup key doesn't contain the dim table's pk, there will be two more costly @@ -517,7 +574,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcLeftJoinDimWithPkSinkWithPk(): Unit = { // The lookup key contains the dim table's pk, there will be no materialization. util.verifyExecPlanInsert(s""" @@ -530,7 +587,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkSinkWithoutPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_without_pk @@ -542,7 +599,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithoutPkSinkWithoutPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_without_pk @@ -554,7 +611,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkOnlySinkWithoutPk(): Unit = { // only select lookup key field, expect not affect NDU util.verifyExecPlanInsert(s""" @@ -567,7 +624,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcLeftJoinDimWithoutPkSinkWithoutPk(): Unit = { util.verifyExecPlanInsert( s""" @@ -580,7 +637,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkOutputNoPkSinkWithoutPk(): Unit = { // non lookup pk selected, expect materialize if tryResolve util.verifyExecPlanInsert(s""" @@ -593,174 +650,220 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkNonDeterministicFuncSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select ndFunc(t2.a) a, t1.b, t1.c + |from ( + | select *, proctime() proctime from cdc + |) t1 join dim_with_pk for system_time as of t1.proctime as t2 + |on t1.a = t2.a + |""".stripMargin) + if (tryResolve) { // only select lookup key field, but with ND-call, expect exception - thrown.expectMessage( - "column(s): a(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): a(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select ndFunc(t2.a) a, t1.b, t1.c - |from ( - | select *, proctime() proctime from cdc - |) t1 join dim_with_pk for system_time as of t1.proctime as t2 - |on t1.a = t2.a - |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkNonDeterministicLocalCondition(): Unit = { // use user defined function + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select t1.a, t1.b, t1.c + |from ( + | select *, proctime() proctime from cdc + |) t1 join dim_with_pk for system_time as of t1.proctime as t2 + |on t1.a = t2.a and ndFunc(t2.b) > 100 + |""".stripMargin) + if (tryResolve) { // not select lookup source field, but with NonDeterministicCondition, expect exception - thrown.expectMessage( - "exists non deterministic function: 'ndFunc' in condition: '>(ndFunc($1), 100)' which may cause wrong result") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "exists non deterministic function: 'ndFunc' in condition: '>(ndFunc($1), 100)' which may cause wrong result") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select t1.a, t1.b, t1.c - |from ( - | select *, proctime() proctime from cdc - |) t1 join dim_with_pk for system_time as of t1.proctime as t2 - |on t1.a = t2.a and ndFunc(t2.b) > 100 - |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimWithPkNonDeterministicLocalCondition2(): Unit = { // use builtin temporal function + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select t1.a, t2.b as version, t2.c + |from ( + | select *, proctime() proctime from cdc + |) t1 join dim_with_pk for system_time as of t1.proctime as t2 + |on t1.a = t2.a + | -- check dim table data's freshness + | and t2.b > UNIX_TIMESTAMP() - 300 + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "exists non deterministic function: 'UNIX_TIMESTAMP' in condition: '>($1, -(UNIX_TIMESTAMP(), 300))' which may cause wrong result") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "exists non deterministic function: 'UNIX_TIMESTAMP' in condition: '>($1, -(UNIX_TIMESTAMP(), 300))' which may cause wrong result") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select t1.a, t2.b as version, t2.c - |from ( - | select *, proctime() proctime from cdc - |) t1 join dim_with_pk for system_time as of t1.proctime as t2 - |on t1.a = t2.a - | -- check dim table data's freshness - | and t2.b > UNIX_TIMESTAMP() - 300 - |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinDimNonDeterministicRemainingCondition(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select t1.a, t2.b, t2.c + |from ( + | select *, proctime() proctime from cdc + |) t1 join dim_with_pk for system_time as of t1.proctime as t2 + |on t1.a = t2.a + | -- non deterministic function in remaining condition + | and t1.b > ndFunc(t2.b) + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "exists non deterministic function: 'ndFunc' in condition: '>($1, ndFunc($3))' which may cause wrong result") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "exists non deterministic function: 'ndFunc' in condition: '>($1, ndFunc($3))' which may cause wrong result") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select t1.a, t2.b, t2.c - |from ( - | select *, proctime() proctime from cdc - |) t1 join dim_with_pk for system_time as of t1.proctime as t2 - |on t1.a = t2.a - | -- non deterministic function in remaining condition - | and t1.b > ndFunc(t2.b) - |""".stripMargin) } - @Test + @TestTemplate def testCdcLeftJoinDimWithNonDeterministicPreFilter(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select t1.a, t2.b as version, t2.c + |from ( + | select *, proctime() proctime from cdc + |) t1 left join dim_with_pk for system_time as of t1.proctime as t2 + |on t1.a = t2.a + | and t1.b > UNIX_TIMESTAMP() - 300 + |""".stripMargin) + // use builtin temporal function if (tryResolve) { - thrown.expectMessage( - "exists non deterministic function: 'UNIX_TIMESTAMP' in condition: '>($1, -(UNIX_TIMESTAMP(), 300))' which may cause wrong result") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "exists non deterministic function: 'UNIX_TIMESTAMP' in condition: '>($1, -(UNIX_TIMESTAMP(), 300))' which may cause wrong result") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select t1.a, t2.b as version, t2.c - |from ( - | select *, proctime() proctime from cdc - |) t1 left join dim_with_pk for system_time as of t1.proctime as t2 - |on t1.a = t2.a - | and t1.b > UNIX_TIMESTAMP() - 300 - |""".stripMargin) } - @Test + @TestTemplate def testGroupByNonDeterministicFuncWithCdcSource(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select + | a, count(*) cnt, `day` + |from ( + | select *, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') `day` from cdc + |) t + |group by `day`, a + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select - | a, count(*) cnt, `day` - |from ( - | select *, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') `day` from cdc - |) t - |group by `day`, a - |""".stripMargin) } - @Test + @TestTemplate def testGroupByNonDeterministicUdfWithCdcSource(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_with_pk + |select + | ndFunc(a), count(*) cnt, c + |from cdc + |group by ndFunc(a), c + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): EXPR$0(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): EXPR$0(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_pk - |select - | ndFunc(a), count(*) cnt, c - |from cdc - |group by ndFunc(a), c - |""".stripMargin) } - @Test + @TestTemplate def testNestedAggWithNonDeterministicGroupingKeys(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select + | a, sum(b) qmt, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') `day` + |from ( + | select *, row_number() over (partition by a order by PROCTIME() desc) rn from src + |) t + |where rn = 1 + |group by a, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert( - s""" - |insert into sink_with_pk - |select - | a, sum(b) qmt, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') `day` - |from ( - | select *, row_number() over (partition by a order by PROCTIME() desc) rn from src - |) t - |where rn = 1 - |group by a, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') - |""".stripMargin) } - @Test + @TestTemplate def testGroupAggNonDeterministicFuncOnSourcePk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlan( + s""" + |select + | `day`, count(*) cnt, sum(b) qmt + |from ( + | select *, concat(cast(a as varchar), DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) `day` from cdc + |) t + |group by `day` + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan( - s""" - |select - | `day`, count(*) cnt, sum(b) qmt - |from ( - | select *, concat(cast(a as varchar), DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) `day` from cdc - |) t - |group by `day` - |""".stripMargin) } - @Test + @TestTemplate def testAggWithNonDeterministicFilterArgs(): Unit = { util.verifyExecPlanInsert( s""" @@ -774,47 +877,57 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testAggWithNonDeterministicFilterArgsOnCdcSource(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select + | a + | ,count(*) cnt + | ,cast(count(distinct c) filter (where b > UNIX_TIMESTAMP() - 180) as varchar) valid_uv + |from cdc + |group by a + |""".stripMargin) + if (tryResolve) { // though original pk was selected and same as the sink's pk, but the valid_uv was // non-deterministic, will raise an error - thrown.expectMessage( - "column(s): $f2(generated by non-deterministic function: UNIX_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): $f2(generated by non-deterministic function: UNIX_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert( - s""" - |insert into sink_with_pk - |select - | a - | ,count(*) cnt - | ,cast(count(distinct c) filter (where b > UNIX_TIMESTAMP() - 180) as varchar) valid_uv - |from cdc - |group by a - |""".stripMargin) } - @Test + @TestTemplate def testAggWithNonDeterministicFilterArgsOnCdcSourceSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_without_pk + |select + | a + | ,count(*) cnt + | ,cast(count(distinct c) filter (where b > UNIX_TIMESTAMP() - 180) as varchar) valid_uv + |from cdc + |group by a + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): $f2(generated by non-deterministic function: UNIX_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): $f2(generated by non-deterministic function: UNIX_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert( - s""" - |insert into sink_without_pk - |select - | a - | ,count(*) cnt - | ,cast(count(distinct c) filter (where b > UNIX_TIMESTAMP() - 180) as varchar) valid_uv - |from cdc - |group by a - |""".stripMargin) } - @Test + @TestTemplate def testNonDeterministicAggOnAppendSourceSinkWithPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_with_pk @@ -827,25 +940,29 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testNonDeterministicAggOnAppendSourceSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select + | a + | ,ndAggFunc(b) ndCnt + | ,max(c) mc + |from T + |group by a + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): ndCnt(generated by non-deterministic function: ndAggFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): ndCnt(generated by non-deterministic function: ndAggFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select - | a - | ,ndAggFunc(b) ndCnt - | ,max(c) mc - |from T - |group by a - |""".stripMargin) } - @Test + @TestTemplate def testGlobalNonDeterministicAggOnAppendSourceSinkWithPk(): Unit = { util.verifyExecPlanInsert(s""" |insert into sink_with_pk @@ -857,24 +974,28 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testGlobalNonDeterministicAggOnAppendSourceSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select + | max(a) + | ,ndAggFunc(b) ndCnt + | ,max(c) mc + |from T + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): ndCnt(generated by non-deterministic function: ndAggFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): ndCnt(generated by non-deterministic function: ndAggFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select - | max(a) - | ,ndAggFunc(b) ndCnt - | ,max(c) mc - |from T - |""".stripMargin) } - @Test + @TestTemplate def testUpsertSourceSinkWithPk(): Unit = { // contains normalize util.verifyExecPlanInsert(s""" @@ -884,7 +1005,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testUpsertSourceSinkWithoutPk(): Unit = { // contains normalize util.verifyExecPlanInsert(s""" @@ -894,7 +1015,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testMultiOverWithNonDeterministicUdafSinkWithPk(): Unit = { util.verifyExecPlanInsert( """ @@ -911,7 +1032,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp ) } - @Test + @TestTemplate def testOverWithNonDeterministicUdafSinkWithoutPk(): Unit = { util.verifyExecPlanInsert( """ @@ -926,27 +1047,28 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp ) } - @Test + @TestTemplate def testMultiOverWithNonDeterministicAggFilterSinkWithPk(): Unit = { // agg with filter is not supported currently, should update this after it is supported. - thrown.expectMessage("OVER must be applied to aggregate function") - thrown.expect(classOf[ValidationException]) - util.verifyExecPlanInsert( - """ - |insert into sink_with_composite_pk - |SELECT - | a - | ,COUNT(distinct b) OVER (PARTITION BY a ORDER BY proctime - | ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) uv - | ,b - | ,SUM(a) filter (where b > UNIX_TIMESTAMP() - 180) OVER (PARTITION BY a ORDER BY proctime - | ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) nd - |FROM T1 - """.stripMargin - ) - } - - @Test + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_with_composite_pk + |SELECT + | a + | ,COUNT(distinct b) OVER (PARTITION BY a ORDER BY proctime + | ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) uv + | ,b + | ,SUM(a) filter (where b > UNIX_TIMESTAMP() - 180) OVER (PARTITION BY a ORDER BY proctime + | ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) nd + |FROM T1 + """.stripMargin)) + .hasMessageContaining("OVER must be applied to aggregate function") + .isInstanceOf[ValidationException] + } + + @TestTemplate def testAppendRankOnMultiOverWithNonDeterministicUdafSinkWithPk(): Unit = { util.verifyExecPlanInsert( """ @@ -970,7 +1092,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp ) } - @Test + @TestTemplate def testAppendRankOnMultiOverWithNonDeterministicUdafSinkWithoutPk(): Unit = { util.verifyExecPlanInsert( """ @@ -994,7 +1116,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp ) } - @Test + @TestTemplate def testUpdateRankOutputRowNumberSinkWithPk(): Unit = { util.tableEnv.executeSql(s""" | create temporary view v1 as @@ -1014,7 +1136,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testRetractRankOutputRowNumberSinkWithPk(): Unit = { util.tableEnv.executeSql(s""" | create temporary view v1 as @@ -1034,76 +1156,95 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testUnionSinkWithCompositePk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_composite_pk + |select a, b, c, d + |from src + |union + |select a, b, c, metadata_3 + |from cdc_with_meta + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_composite_pk - |select a, b, c, d - |from src - |union - |select a, b, c, metadata_3 - |from cdc_with_meta - |""".stripMargin) } - @Test + @TestTemplate def testUnionAllSinkWithCompositePk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_with_composite_pk + |select a, b, c, d + |from src + |union all + |select a, b, c, metadata_3 + |from cdc_with_meta + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_with_composite_pk - |select a, b, c, d - |from src - |union all - |select a, b, c, metadata_3 - |from cdc_with_meta - |""".stripMargin) } - @Test + @TestTemplate def testUnionAllSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select a, b, c + |from src + |union all + |select a, metadata_3, c + |from cdc_with_meta + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "metadata column(s): 'metadata_3' in cdc source may cause wrong result or error") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select a, b, c - |from src - |union all - |select a, metadata_3, c - |from cdc_with_meta - |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinWithNonDeterministicCondition(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(s""" + |insert into sink_without_pk + |select + | t1.a + | ,t2.b + | ,t1.c + |from cdc t1 join cdc t2 + | on ndFunc(t1.b) = ndFunc(t2.b) + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): $f4(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): $f4(generated by non-deterministic function: ndFunc ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(s""" - |insert into sink_without_pk - |select - | t1.a - | ,t2.b - | ,t1.c - |from cdc t1 join cdc t2 - | on ndFunc(t1.b) = ndFunc(t2.b) - |""".stripMargin) } - @Test + @TestTemplate def testProctimeIntervalJoinSinkWithoutPk(): Unit = { util.verifyExecPlanInsert(""" |insert into sink_without_pk @@ -1112,7 +1253,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp """.stripMargin) } - @Test + @TestTemplate def testCdcProctimeIntervalJoinOnPkSinkWithoutPk(): Unit = { util.verifyExecPlanInsert(""" |insert into sink_without_pk @@ -1123,22 +1264,27 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp """.stripMargin) } - @Test + @TestTemplate def testCdcProctimeIntervalJoinOnNonPkSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert(""" + |insert into sink_without_pk + |SELECT t2.a, t2.b, t1.c FROM ( + | select *, proctime() proctime from cdc) t1 JOIN + | (select *, proctime() proctime from cdc) t2 ON + | t1.b = t2.b AND t1.proctime > t2.proctime - INTERVAL '5' SECOND + """.stripMargin) + if (tryResolve) { - thrown.expectMessage("can not satisfy the determinism requirement") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining("can not satisfy the determinism requirement") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert(""" - |insert into sink_without_pk - |SELECT t2.a, t2.b, t1.c FROM ( - | select *, proctime() proctime from cdc) t1 JOIN - | (select *, proctime() proctime from cdc) t2 ON - | t1.b = t2.b AND t1.proctime > t2.proctime - INTERVAL '5' SECOND - """.stripMargin) } - @Test + @TestTemplate def testCdcRowtimeIntervalJoinSinkWithoutPk(): Unit = { util.verifyExecPlanInsert( """ @@ -1148,7 +1294,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp """.stripMargin) } - @Test + @TestTemplate def testCdcRowtimeIntervalJoinSinkWithPk(): Unit = { util.verifyExecPlanInsert( """ @@ -1158,7 +1304,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp """.stripMargin) } - @Test + @TestTemplate def testJoinKeyContainsUk(): Unit = { util.verifyExecPlan( s""" @@ -1175,7 +1321,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testJoinHasBothSidesUk(): Unit = { util.verifyExecPlan( s""" @@ -1192,53 +1338,64 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testJoinHasBothSidesUkSinkWithoutPk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |insert into sink_with_pk + |select t1.a, t2.a, t2.`c-day` + |from ( + | select a, b, c, d + | from cdc + | ) t1 + |join ( + | select a, b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `c-day`, d + | from cdc + |) t2 + | on t1.b = t2.b + |""".stripMargin) + if (tryResolve) { // sink require all columns be deterministic though join has both side uk - thrown.expectMessage( - "column(s): c-day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): c-day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlanInsert( - s""" - |insert into sink_with_pk - |select t1.a, t2.a, t2.`c-day` - |from ( - | select a, b, c, d - | from cdc - | ) t1 - |join ( - | select a, b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `c-day`, d - | from cdc - |) t2 - | on t1.b = t2.b - |""".stripMargin) } - @Test + @TestTemplate def testJoinHasSingleSideUk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlan( + s""" + |select t1.a, t2.`c-day`, t2.b, t2.d + |from ( + | select a, b, c, d + | from cdc + | ) t1 + |join ( + | select a, b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `c-day`, d + | from cdc + |) t2 + | on t1.b = t2.b + |""".stripMargin) + if (tryResolve) { // the input side without uk requires all columns be deterministic - thrown.expectMessage("can not satisfy the determinism requirement") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): c-day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan( - s""" - |select t1.a, t2.`c-day`, t2.b, t2.d - |from ( - | select a, b, c, d - | from cdc - | ) t1 - |join ( - | select a, b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `c-day`, d - | from cdc - |) t2 - | on t1.b = t2.b - |""".stripMargin) } - @Test + @TestTemplate def testSemiJoinKeyContainsUk(): Unit = { util.verifyExecPlan( s""" @@ -1253,9 +1410,8 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testAntiJoinKeyContainsUk(): Unit = { - util.verifyExecPlan( s""" |select t1.a, t1.`c-day`, t1.b, t1.d @@ -1269,34 +1425,34 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |""".stripMargin) } - @Test + @TestTemplate def testSemiJoinWithNonDeterministicConditionSingleSideHasUk(): Unit = { + val callable: ThrowingCallable = () => + util.verifyExecPlan( + s""" + |select t1.a, t1.b, t1.c, t1.d + |from ( + | select a, b, c, d + | from cdc + | ) t1 + |where t1.c in ( + | select CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) c from cdc where b > 100 + |) + |""".stripMargin) + if (tryResolve) { - thrown.expectMessage( - "column(s): c(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): c(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan( - s""" - |select t1.a, t1.b, t1.c, t1.d - |from ( - | select a, b, c, d - | from cdc - | ) t1 - |where t1.c in ( - | select CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) c from cdc where b > 100 - |) - |""".stripMargin) } - @Test + @TestTemplate def testCdcJoinWithNonDeterministicOutputSinkWithPk(): Unit = { // a real case from FLINK-27369 - if (tryResolve) { - thrown.expectMessage( - "The column(s): logistics_time(generated by non-deterministic function: NOW ) can not satisfy the determinism requirement") - thrown.expect(classOf[TableException]) - } util.tableEnv.executeSql(s""" |CREATE TEMPORARY TABLE t_order ( | order_id INT, @@ -1336,87 +1492,99 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp | 'sink-insert-only' = 'false' |)""".stripMargin) - util.verifyExecPlanInsert( - s""" - |INSERT INTO t_join_sink - |SELECT ord.order_id, - |ord.order_name, - |logistics.logistics_id, - |logistics.logistics_target, - |logistics.logistics_source, - |now() - |FROM t_order AS ord - |LEFT JOIN t_logistics AS logistics ON ord.order_id=logistics.order_id - |""".stripMargin) + val callable: ThrowingCallable = () => + util.verifyExecPlanInsert( + s""" + |INSERT INTO t_join_sink + |SELECT ord.order_id, + |ord.order_name, + |logistics.logistics_id, + |logistics.logistics_target, + |logistics.logistics_source, + |now() + |FROM t_order AS ord + |LEFT JOIN t_logistics AS logistics ON ord.order_id=logistics.order_id + |""".stripMargin) + + if (tryResolve) { + assertThatThrownBy(callable) + .hasMessageContaining( + "The column(s): logistics_time(generated by non-deterministic function: NOW ) can not satisfy the determinism requirement") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() + } } - @Test + @TestTemplate def testProctimeDedupOnCdcWithMetadataSinkWithPk(): Unit = { // TODO this should be updated after StreamPhysicalDeduplicate supports consuming update - thrown.expectMessage( - "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert( - """ - |insert into sink_with_pk - |SELECT a, metadata_3, c - |FROM ( - | SELECT *, - | ROW_NUMBER() OVER (PARTITION BY a ORDER BY PROCTIME() ASC) as rowNum - | FROM cdc_with_meta - |) - |WHERE rowNum = 1 - """.stripMargin - ) - } - - @Test + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_with_pk + |SELECT a, metadata_3, c + |FROM ( + | SELECT *, + | ROW_NUMBER() OVER (PARTITION BY a ORDER BY PROCTIME() ASC) as rowNum + | FROM cdc_with_meta + |) + |WHERE rowNum = 1 + """.stripMargin)) + .hasMessageContaining( + "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") + .isInstanceOf[TableException] + } + + @TestTemplate def testProctimeDedupOnCdcWithMetadataSinkWithoutPk(): Unit = { // TODO this should be updated after StreamPhysicalDeduplicate supports consuming update - thrown.expectMessage( - "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert( - """ - |insert into sink_without_pk - |SELECT a, metadata_3, c - |FROM ( - | SELECT *, - | ROW_NUMBER() OVER (PARTITION BY a ORDER BY PROCTIME() ASC) as rowNum - | FROM cdc_with_meta - |) - |WHERE rowNum = 1 + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_without_pk + |SELECT a, metadata_3, c + |FROM ( + | SELECT *, + | ROW_NUMBER() OVER (PARTITION BY a ORDER BY PROCTIME() ASC) as rowNum + | FROM cdc_with_meta + |) + |WHERE rowNum = 1 """.stripMargin - ) + )) + .hasMessageContaining( + "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") + .isInstanceOf[TableException] + } - @Test + @TestTemplate def testRowtimeDedupOnCdcWithMetadataSinkWithPk(): Unit = { // TODO this should be updated after StreamPhysicalDeduplicate supports consuming update - thrown.expectMessage( - "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert( - """ - |insert into sink_with_pk - |SELECT a, b, c - |FROM ( - | SELECT *, - | ROW_NUMBER() OVER (PARTITION BY a ORDER BY op_ts ASC) as rowNum - | FROM cdc_with_meta_and_wm - |) - |WHERE rowNum = 1 + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_with_pk + |SELECT a, b, c + |FROM ( + | SELECT *, + | ROW_NUMBER() OVER (PARTITION BY a ORDER BY op_ts ASC) as rowNum + | FROM cdc_with_meta_and_wm + |) + |WHERE rowNum = 1 """.stripMargin - ) + )) + .hasMessageContaining( + "StreamPhysicalDeduplicate doesn't support consuming update and delete changes") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testWindowDedupOnCdcWithMetadata(): Unit = { // TODO this should be updated after StreamPhysicalWindowDeduplicate supports consuming update - thrown.expectMessage( - "StreamPhysicalWindowDeduplicate doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.tableEnv.executeSql(""" |create temporary table sink1 ( | a int, @@ -1429,21 +1597,25 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp | 'sink-insert-only' = 'false' |)""".stripMargin) - util.verifyExecPlanInsert( - """ - |INSERT INTO sink1 - |SELECT a, b, c, window_start - |FROM ( - |SELECT *, - | ROW_NUMBER() OVER(PARTITION BY a, window_start, window_end - | ORDER BY op_ts DESC) as rownum - |FROM TABLE(TUMBLE(TABLE cdc_with_meta_and_wm, DESCRIPTOR(op_ts), INTERVAL '1' MINUTE)) - |) - |WHERE rownum <= 1""".stripMargin) - - } - - @Test + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |INSERT INTO sink1 + |SELECT a, b, c, window_start + |FROM ( + |SELECT *, + | ROW_NUMBER() OVER(PARTITION BY a, window_start, window_end + | ORDER BY op_ts DESC) as rownum + |FROM TABLE(TUMBLE(TABLE cdc_with_meta_and_wm, DESCRIPTOR(op_ts), INTERVAL '1' MINUTE)) + |) + |WHERE rownum <= 1""".stripMargin)) + .hasMessageContaining( + "StreamPhysicalWindowDeduplicate doesn't support consuming update and delete changes") + .isInstanceOf[TableException] + } + + @TestTemplate def testNestedSourceWithMultiSink(): Unit = { val ddl = s""" @@ -1508,15 +1680,20 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |from v1 |where b > 100 |""".stripMargin) + + val callable: ThrowingCallable = () => util.verifyExecPlan(stmtSet) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan(stmtSet) } - @Test + @TestTemplate def testMultiSinkOnJoinedView(): Unit = { util.tableEnv.executeSql(""" |create temporary table src1 ( @@ -1593,15 +1770,20 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp |from v1 |where b > 100 |""".stripMargin) + + val callable: ThrowingCallable = () => util.verifyExecPlan(stmtSet) + if (tryResolve) { - thrown.expectMessage( - "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") - thrown.expect(classOf[TableException]) + assertThatThrownBy(callable) + .hasMessageContaining( + "column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism") + .isInstanceOf[TableException] + } else { + assertThatCode(callable).doesNotThrowAnyException() } - util.verifyExecPlan(stmtSet) } - @Test + @TestTemplate def testMatchRecognizeSinkWithPk(): Unit = { util.tableEnv.executeSql(s""" |create temporary view v1 as @@ -1629,57 +1811,61 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp ) } - @Test + @TestTemplate def testMatchRecognizeWithNonDeterministicConditionOnCdcSinkWithPk(): Unit = { // TODO this should be updated after StreamPhysicalMatch supports consuming updates - thrown.expectMessage("Match Recognize doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert( - """ - |insert into sink_with_pk - |SELECT T.a, T.b, cast(T.matchRowtime as varchar) - |FROM cdc_with_meta_and_wm - |MATCH_RECOGNIZE ( - |PARTITION BY c - |ORDER BY op_ts - |MEASURES - | A.a as a, - | A.b as b, - | MATCH_ROWTIME(op_ts) as matchRowtime - |ONE ROW PER MATCH - |PATTERN (A) - |DEFINE - | A AS A.op_ts >= CURRENT_TIMESTAMP - |) AS T + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_with_pk + |SELECT T.a, T.b, cast(T.matchRowtime as varchar) + |FROM cdc_with_meta_and_wm + |MATCH_RECOGNIZE ( + |PARTITION BY c + |ORDER BY op_ts + |MEASURES + | A.a as a, + | A.b as b, + | MATCH_ROWTIME(op_ts) as matchRowtime + |ONE ROW PER MATCH + |PATTERN (A) + |DEFINE + | A AS A.op_ts >= CURRENT_TIMESTAMP + |) AS T """.stripMargin - ) + )) + .hasMessageContaining("Match Recognize doesn't support consuming update and delete changes") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testMatchRecognizeOnCdcWithMetaDataSinkWithPk(): Unit = { // TODO this should be updated after StreamPhysicalMatch supports consuming updates - thrown.expectMessage("Match Recognize doesn't support consuming update and delete changes") - thrown.expect(classOf[TableException]) - util.verifyExecPlanInsert( - """ - |insert into sink_with_pk - |SELECT T.a, T.b, cast(T.ts as varchar) - |FROM cdc_with_meta_and_wm - |MATCH_RECOGNIZE ( - |PARTITION BY c - |ORDER BY op_ts - |MEASURES - | A.a as a, - | A.b as b, - | A.op_ts as ts, - | MATCH_ROWTIME(op_ts) as matchRowtime - |ONE ROW PER MATCH - |PATTERN (A) - |DEFINE - | A AS A.a > 0 - |) AS T + assertThatThrownBy( + () => + util.verifyExecPlanInsert( + """ + |insert into sink_with_pk + |SELECT T.a, T.b, cast(T.ts as varchar) + |FROM cdc_with_meta_and_wm + |MATCH_RECOGNIZE ( + |PARTITION BY c + |ORDER BY op_ts + |MEASURES + | A.a as a, + | A.b as b, + | A.op_ts as ts, + | MATCH_ROWTIME(op_ts) as matchRowtime + |ONE ROW PER MATCH + |PATTERN (A) + |DEFINE + | A AS A.a > 0 + |) AS T """.stripMargin - ) + )) + .hasMessageContaining("Match Recognize doesn't support consuming update and delete changes") + .isInstanceOf[TableException] } /** @@ -1736,7 +1922,7 @@ class NonDeterministicDagTest(nonDeterministicUpdateStrategy: NonDeterministicUp object NonDeterministicDagTest { - @Parameterized.Parameters(name = "nonDeterministicUpdateStrategy={0}") + @Parameters(name = "nonDeterministicUpdateStrategy={0}") def parameters(): util.Collection[NonDeterministicUpdateStrategy] = { util.Arrays.asList( NonDeterministicUpdateStrategy.TRY_RESOLVE, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala index bb0816e93f56d..90c40210984a3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class PartitionableSinkTest extends TableTestBase { @@ -66,13 +67,19 @@ class PartitionableSinkTest extends TableTestBase { util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable") } - @Test(expected = classOf[ValidationException]) + @Test def testWrongStatic(): Unit = { - util.verifyExecPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExecPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")) } - @Test(expected = classOf[ValidationException]) + @Test def testWrongFields(): Unit = { - util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util.verifyExecPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSourceTest.scala index 6d5862dfd0655..441bb7d1c96df 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSourceTest.scala @@ -21,22 +21,22 @@ import org.apache.flink.table.catalog.{CatalogPartitionImpl, CatalogPartitionSpe import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.factories.TestValuesCatalog import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util import scala.collection.JavaConversions._ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class PartitionableSourceTest(val sourceFetchPartitions: Boolean, val useCatalogFilter: Boolean) extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { val partitionableTable = """ @@ -113,43 +113,43 @@ class PartitionableSourceTest(val sourceFetchPartitions: Boolean, val useCatalog } } - @Test + @TestTemplate def testSimplePartitionFieldPredicate1(): Unit = { util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE part1 = 'A'") } - @Test + @TestTemplate def testPartialPartitionFieldPredicatePushDown(): Unit = { util.verifyExecPlan( "SELECT * FROM PartitionableTable WHERE (id > 2 OR part1 = 'A') AND part2 > 1") } - @Test + @TestTemplate def testWithUdfAndVirtualColumn(): Unit = { util.addFunction("MyUdf", Func1) util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE id > 2 AND MyUdf(part2) < 3") } - @Test + @TestTemplate def testUnconvertedExpression(): Unit = { util.verifyExecPlan("select * from PartitionableTable where trim(part1) = 'A' and part2 > 1") } - @Test + @TestTemplate def testPushDownPartitionAndFiltersContainPartitionKeys(): Unit = { util.verifyExecPlan( "select * from PartitionableAndFilterableTable " + "where part1 = 'A' and part2 > 1 and id > 1") } - @Test + @TestTemplate def testPushDownPartitionAndFiltersContainPartitionKeysWithSingleProjection(): Unit = { util.verifyExecPlan( "select name from PartitionableAndFilterableTable " + "where part1 = 'A' and part2 > 1 and id > 1") } - @Test + @TestTemplate def testPushDownNonExistentPartition(): Unit = { util.verifyExecPlan("SELECT * FROM PartitionableTable WHERE part2 = 4") } @@ -157,7 +157,7 @@ class PartitionableSourceTest(val sourceFetchPartitions: Boolean, val useCatalog } object PartitionableSourceTest { - @Parameterized.Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") + @Parameters(name = "sourceFetchPartitions={0}, useCatalogFilter={1}") def parameters(): util.Collection[Array[Any]] = { Seq[Array[Any]]( Array(true, false), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala index a6fcd9f0bc1b3..82547355d8286 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala @@ -22,8 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.utils.TableTestBase -import org.assertj.core.api.Assertions.assertThatThrownBy -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test class RankTest extends TableTestBase { @@ -43,9 +43,9 @@ class RankTest extends TableTestBase { |WHERE rank_num >= 10 """.stripMargin - thrown.expectMessage("Rank end is not specified.") - thrown.expect(classOf[TableException]) - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining("Rank end is not specified.") + .isInstanceOf[TableException] } @Test @@ -60,9 +60,9 @@ class RankTest extends TableTestBase { |WHERE rank_num <= 0 """.stripMargin - thrown.expectMessage("Rank end should not less than zero") - thrown.expect(classOf[TableException]) - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining("Rank end should not less than zero") + .isInstanceOf[TableException] } @Test @@ -185,7 +185,7 @@ class RankTest extends TableTestBase { util.verifyExecPlan(sql) } - @Test(expected = classOf[RuntimeException]) + @Test def testRowNumberWithOutOrderBy(): Unit = { val sql = """ @@ -195,10 +195,11 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE row_num <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[ValidationException]) + @Test def testRankWithOutOrderBy(): Unit = { val sql = """ @@ -208,10 +209,11 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE rk <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[ValidationException]) + @Test def testDenseRankWithOutOrderBy(): Unit = { val sql = """ @@ -221,10 +223,11 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE rk <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[RuntimeException]) + @Test def testRowNumberWithMultiGroups(): Unit = { val sql = """ @@ -235,10 +238,11 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE row_num <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[ValidationException]) + @Test def testRankWithMultiGroups(): Unit = { val sql = """ @@ -249,10 +253,11 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE rk <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[ValidationException]) + @Test def testDenseRankWithMultiGroups(): Unit = { val sql = """ @@ -263,7 +268,8 @@ class RankTest extends TableTestBase { | FROM MyTable) |WHERE rk <= a """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } @Test @@ -644,7 +650,7 @@ class RankTest extends TableTestBase { util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) } - @Test(expected = classOf[ValidationException]) + @Test // FIXME remove expected exception after ADD added def testTopNForVariableSize(): Unit = { val subquery = @@ -667,7 +673,8 @@ class RankTest extends TableTestBase { |WHERE row_num <= a """.stripMargin - util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE)) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala index 214d61bb635e9..2b195426e44a6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.plan.stream.sql.RelTimeIndicatorConverterT import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.types.logical.BigIntType -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala index 510ffe7baa379..40dea1bc89ec5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala @@ -24,29 +24,33 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.{BeforeEach, Test} class SetOperatorsTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTableSource[(Int, Long, String)]("T1", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("T2", 'd, 'e, 'f) util.addTableSource[(Int, Long, Int, String, Long)]("T3", 'a, 'b, 'd, 'c, 'e) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentColumnSize(): Unit = { // must fail. Union inputs have different column size. - util.verifyExecPlan("SELECT * FROM T1 UNION ALL SELECT * FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT * FROM T1 UNION ALL SELECT * FROM T3")) } - @Test(expected = classOf[ValidationException]) + @Test def testUnionDifferentFieldTypes(): Unit = { // must fail. Union inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 UNION ALL SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.verifyExecPlan("SELECT a, b, c FROM T1 UNION ALL SELECT d, c, e FROM T3")) } @Test @@ -54,10 +58,12 @@ class SetOperatorsTest extends TableTestBase { util.verifyExecPlan("SELECT c FROM T1 INTERSECT ALL SELECT f FROM T2") } - @Test(expected = classOf[ValidationException]) + @Test def testIntersectDifferentFieldTypes(): Unit = { // must fail. Intersect inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 INTERSECT SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.verifyExecPlan("SELECT a, b, c FROM T1 INTERSECT SELECT d, c, e FROM T3")) } @Test @@ -65,10 +71,11 @@ class SetOperatorsTest extends TableTestBase { util.verifyExecPlan("SELECT c FROM T1 EXCEPT ALL SELECT f FROM T2") } - @Test(expected = classOf[ValidationException]) + @Test def testMinusDifferentFieldTypes(): Unit = { // must fail. Minus inputs have different field types. - util.verifyExecPlan("SELECT a, b, c FROM T1 EXCEPT SELECT d, c, e FROM T3") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT a, b, c FROM T1 EXCEPT SELECT d, c, e FROM T3")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala index 2406c3823b8a1..2fc7d30f67bb0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SortLimitTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortTest.scala index da0c1af69a1cc..df554c3bbfc8d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SortTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala index 355d849729dab..29a498206623b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala @@ -20,14 +20,14 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} /** Tests for watermark push down. */ class SourceWatermarkTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.tableEnv.executeSql(s""" | CREATE TABLE VirtualTable ( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala index a8a97b4d44ce8..5cd67937c6a7d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala @@ -25,13 +25,13 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunction import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.runtime.functions.aggregate.FirstValueAggFunction -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class SubplanReuseTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index d228c206aa5c2..2cd7f047e9833 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.planner.factories.TestValuesTableFactory.MockedLoo import org.apache.flink.table.planner.utils.TableTestBase import org.assertj.core.api.Assertions.assertThatThrownBy -import org.junit.Test +import org.junit.jupiter.api.Test class TableScanTest extends TableTestBase { @@ -667,13 +667,15 @@ class TableScanTest extends TableTestBase { | 'changelog-mode' = 'I,UB,D' |) """.stripMargin) - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Invalid source for table 'default_catalog.default_database.src'. A ScanTableSource " + - "doesn't support a changelog stream that contains UPDATE_BEFORE but no UPDATE_AFTER. " + - "Please adapt the implementation of class 'org.apache.flink.table.planner.factories." + - "TestValuesTableFactory$TestValuesScanLookupTableSource'.") - util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE) + + assertThatThrownBy( + () => util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining( + "Invalid source for table 'default_catalog.default_database.src'. A ScanTableSource " + + "doesn't support a changelog stream that contains UPDATE_BEFORE but no UPDATE_AFTER. " + + "Please adapt the implementation of class 'org.apache.flink.table.planner.factories." + + "TestValuesTableFactory$TestValuesScanLookupTableSource'.") + .isInstanceOf[ValidationException] } @Test @@ -688,12 +690,14 @@ class TableScanTest extends TableTestBase { | 'changelog-mode' = 'I,UA,D' |) """.stripMargin) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Table 'default_catalog.default_database.src' produces a " + - "changelog stream that contains UPDATE_AFTER but no UPDATE_BEFORE. " + - "This requires defining a primary key constraint on the table.") - util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE) + + assertThatThrownBy( + () => util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining( + "Table 'default_catalog.default_database.src' produces a " + + "changelog stream that contains UPDATE_AFTER but no UPDATE_BEFORE. " + + "This requires defining a primary key constraint on the table.") + .isInstanceOf[TableException] } @Test @@ -711,12 +715,13 @@ class TableScanTest extends TableTestBase { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true)) - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Configuration 'table.exec.source.cdc-events-duplicate' is enabled " + - "which requires the changelog sources to define a PRIMARY KEY. " + - "However, table 'default_catalog.default_database.src' doesn't have a primary key.") - util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE) + assertThatThrownBy( + () => util.verifyRelPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining( + "Configuration 'table.exec.source.cdc-events-duplicate' is enabled " + + "which requires the changelog sources to define a PRIMARY KEY. " + + "However, table 'default_catalog.default_database.src' doesn't have a primary key.") + .isInstanceOf[TableException] } @Test @@ -731,28 +736,29 @@ class TableScanTest extends TableTestBase { | 'table-source-class' = '${classOf[MockedLookupTableSource].getName}' |) """.stripMargin) - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - util.verifyRelPlan("SELECT * FROM src", ExplainDetail.CHANGELOG_MODE) + + assertThatThrownBy(() => util.verifyRelPlan("SELECT * FROM src", ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining("Cannot generate a valid execution plan for the given query") + .isInstanceOf[ValidationException] } @Test def testInvalidWatermarkOutputType(): Unit = { - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Invalid data type of expression for watermark definition. " + - "The field must be of type TIMESTAMP(p) or TIMESTAMP_LTZ(p), " + - "the supported precision 'p' is from 0 to 3, but the watermark " + - "expression type is CHAR(0) NOT NULL") - util.addTable(""" - |CREATE TABLE src ( - | ts TIMESTAMP(3), - | a INT, - | b DOUBLE, - | WATERMARK FOR `ts` AS '' - |) WITH ( - | 'connector' = 'values' - |) - """.stripMargin) + assertThatThrownBy(() => util.addTable(""" + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR `ts` AS '' + |) WITH ( + | 'connector' = 'values' + |) + """.stripMargin)) + .hasMessageContaining( + "Invalid data type of expression for watermark definition. " + + "The field must be of type TIMESTAMP(p) or TIMESTAMP_LTZ(p), " + + "the supported precision 'p' is from 0 to 3, but the watermark " + + "expression type is CHAR(0) NOT NULL") + .isInstanceOf[ValidationException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala index 0df9269c97f36..e230b39a3850b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala @@ -29,7 +29,8 @@ import org.apache.flink.table.factories.{DynamicTableFactory, DynamicTableSource import org.apache.flink.table.planner.utils.{TableTestBase, TestingTableEnvironment} import org.assertj.core.api.Assertions -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.util @@ -86,11 +87,13 @@ class TableSinkTest extends TableTestBase { | 'connector' = 'values' |) |""".stripMargin) - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Query schema: [a: INT, EXPR$1: CHAR(0) NOT NULL, EXPR$2: CHAR(0) NOT NULL]\n" + - "Sink schema: [name: STRING, email: STRING, message_offset: BIGINT]") - util.verifyExecPlanInsert("INSERT INTO my_sink SELECT a, '', '' FROM MyTable") + + assertThatThrownBy( + () => util.verifyExecPlanInsert("INSERT INTO my_sink SELECT a, '', '' FROM MyTable")) + .hasMessageContaining( + "Query schema: [a: INT, EXPR$1: CHAR(0) NOT NULL, EXPR$2: CHAR(0) NOT NULL]\n" + + "Sink schema: [name: STRING, email: STRING, message_offset: BIGINT]") + .isInstanceOf[ValidationException] } @Test @@ -106,12 +109,12 @@ class TableSinkTest extends TableTestBase { val stmtSet = util.tableEnv.createStatementSet() stmtSet.addInsertSql("INSERT INTO appendSink SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a") - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Table sink 'default_catalog.default_database.appendSink' doesn't " + - "support consuming update changes which is produced by node " + - "GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])") - util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) + assertThatThrownBy(() => util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining( + "Table sink 'default_catalog.default_database.appendSink' doesn't " + + "support consuming update changes which is produced by node " + + "GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])") + .isInstanceOf[TableException] } @Test @@ -141,11 +144,10 @@ class TableSinkTest extends TableTestBase { stmtSet.addInsertSql( "INSERT INTO retractSink2 SELECT cnt, SUM(cnt) OVER (ORDER BY PROCTIME()) FROM TempTable") - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "OverAggregate doesn't support consuming update changes " + + assertThatThrownBy(() => util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining("OverAggregate doesn't support consuming update changes " + "which is produced by node Calc(select=[cnt])") - util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) + .isInstanceOf[TableException] } @Test @@ -360,12 +362,11 @@ class TableSinkTest extends TableTestBase { val stmtSet = util.tableEnv.createStatementSet() stmtSet.addInsertSql(sql) - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Query schema: [a: INT, m_3: INT, m_2: INT, b: BIGINT, c: INT, metadata_1: STRING]\n" + - "Sink schema: [a: INT, m_2: INT, b: BIGINT, c: INT, metadata_1: STRING]") - - util.verifyRelPlan(stmtSet) + assertThatThrownBy(() => util.verifyRelPlan(stmtSet)) + .hasMessageContaining( + "Query schema: [a: INT, m_3: INT, m_2: INT, b: BIGINT, c: INT, metadata_1: STRING]\n" + + "Sink schema: [a: INT, m_2: INT, b: BIGINT, c: INT, metadata_1: STRING]") + .isInstanceOf[ValidationException] } @Test @@ -389,13 +390,12 @@ class TableSinkTest extends TableTestBase { val stmtSet = util.tableEnv.createStatementSet() stmtSet.addInsertSql(sql) - thrown.expect(classOf[ValidationException]) - thrown.expectMessage( - "Invalid data type for metadata column 'metadata_1' of table " + - "'default_catalog.default_database.MetadataTable'. The column cannot be declared as " + - "'TIMESTAMP(3)' because the type must be castable to metadata type 'BOOLEAN'.") - - util.verifyRelPlan(stmtSet) + assertThatThrownBy(() => util.verifyRelPlan(stmtSet)) + .hasMessageContaining( + "Invalid data type for metadata column 'metadata_1' of table " + + "'default_catalog.default_database.MetadataTable'. The column cannot be declared as " + + "'TIMESTAMP(3)' because the type must be castable to metadata type 'BOOLEAN'.") + .isInstanceOf[ValidationException] } @Test @@ -754,13 +754,13 @@ class TableSinkTest extends TableTestBase { val stmtSet = util.tableEnv.createStatementSet() stmtSet.addInsertSql("INSERT INTO sink SELECT * FROM MyTable") - expectedException.expect(classOf[TableException]) - expectedException.expectMessage( - s"You should enable the checkpointing for sinking to managed table " + - s"'default_catalog.default_database.sink', " + - s"managed table relies on checkpoint to commit and " + - s"the data is visible only after commit.") - util.verifyAstPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) + assertThatThrownBy(() => util.verifyAstPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)) + .hasMessageContaining( + s"You should enable the checkpointing for sinking to managed table " + + s"'default_catalog.default_database.sink', " + + s"managed table relies on checkpoint to commit and " + + s"the data is visible only after commit.") + .isInstanceOf[TableException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala index 755a8a541c7cf..6189763ae06e9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala @@ -17,11 +17,11 @@ */ package org.apache.flink.table.planner.plan.stream.sql -import org.apache.flink.core.testutils.FlinkMatchers.containsMessage import org.apache.flink.table.api.ValidationException import org.apache.flink.table.planner.utils._ -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test class TableSourceTest extends TableTestBase { @@ -100,9 +100,6 @@ class TableSourceTest extends TableTestBase { @Test def testProctimeOnWatermarkSpec(): Unit = { - thrown.expect(classOf[ValidationException]) - thrown.expect( - containsMessage("A watermark can not be defined for a processing-time attribute.")) val ddl = s""" |CREATE TABLE procTimeT ( @@ -116,9 +113,14 @@ class TableSourceTest extends TableTestBase { | 'bounded' = 'false' |) """.stripMargin - util.tableEnv.executeSql(ddl) - util.verifyExecPlan("SELECT pTime, id, name, val FROM procTimeT") + assertThatThrownBy( + () => { + util.tableEnv.executeSql(ddl) + util.verifyExecPlan("SELECT pTime, id, name, val FROM procTimeT") + }) + .hasMessageContaining("A watermark can not be defined for a processing-time attribute.") + .isInstanceOf[ValidationException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala index 1d3c02dcfe5a5..d8e4ac1daeb84 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala @@ -21,14 +21,14 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} // TODO add more union case after aggregation and join supported class UnionTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable1", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.scala index fd1e4ff3d21f4..0eccb2947ca94 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.common.UnnestTestBase import org.apache.flink.table.planner.utils.TableTestUtil -import org.junit.Test +import org.junit.jupiter.api.Test class UnnestTest extends UnnestTestBase(true) { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.scala index 3d0a31c205720..0520d294f1f06 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class ValuesTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala index cf3e581ecabb9..38385a6989543 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test /** Tests for window deduplicate. */ class WindowDeduplicateTest extends TableTestBase { @@ -161,9 +162,9 @@ class WindowDeduplicateTest extends TableTestBase { |WHERE rownum <= 1 """.stripMargin - thrown.expectMessage("Processing time Window Deduplication is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window Deduplication is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -180,9 +181,9 @@ class WindowDeduplicateTest extends TableTestBase { |WHERE rownum <= 1 """.stripMargin - thrown.expectMessage("Processing time Window Deduplication is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window Deduplication is not supported yet.") + .isInstanceOf[TableException] } // ---------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.scala index 9a7b221a16459..fde6fbd50d647 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test /** Tests for window rank. */ class WindowRankTest extends TableTestBase { @@ -90,9 +91,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expectMessage("Processing time Window TopN is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -141,9 +142,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expectMessage("Processing time Window TopN is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -192,9 +193,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expectMessage("Processing time Window TopN is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } // ---------------------------------------------------------------------------------------- @@ -255,9 +256,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage("Processing time Window TopN is not supported yet.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -316,9 +317,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage("Processing time Window TopN is not supported yet.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -381,9 +382,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage("Processing time Window TopN is not supported yet.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window TopN is not supported yet.") + .isInstanceOf[TableException] } // ---------------------------------------------------------------------------------------- @@ -486,10 +487,10 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "RANK() function is not supported on Window TopN currently, only ROW_NUMBER() is supported.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining( + "RANK() function is not supported on Window TopN currently, only ROW_NUMBER() is supported.") + .isInstanceOf[TableException] } @Test @@ -518,11 +519,10 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= 3 """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "DENSE_RANK() function is not supported on Window TopN currently, " + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("DENSE_RANK() function is not supported on Window TopN currently, " + "only ROW_NUMBER() is supported.") - util.verifyExplain(sql) + .isInstanceOf[TableException] } @Test @@ -552,8 +552,9 @@ class WindowRankTest extends TableTestBase { |WHERE rownum <= max_b """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage("Rank strategy rankEnd=max_b is not supported on window rank currently.") - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining( + "Rank strategy rankEnd=max_b is not supported on window rank currently.") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala index 6b21078e36903..ff64550e35d65 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala @@ -17,11 +17,11 @@ */ package org.apache.flink.table.planner.plan.stream.sql -import org.apache.flink.core.testutils.FlinkMatchers.containsCause import org.apache.flink.table.api.ValidationException import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test /** Tests for window table-valued function. */ class WindowTableFunctionTest extends TableTestBase { @@ -118,12 +118,10 @@ class WindowTableFunctionTest extends TableTestBase { |FROM TABLE( | TUMBLE(TABLE v1, DESCRIPTOR(cur_time), INTERVAL '15' MINUTE)) |""".stripMargin - thrown.expectCause( - containsCause( - new ValidationException( - "The window function requires the timecol is a time attribute type, but is TIMESTAMP(3).") - )) - util.verifyRelPlan(sql) + + assertThatThrownBy(() => util.verifyRelPlan(sql)) + .hasCause(new ValidationException( + "The window function requires the timecol is a time attribute type, but is TIMESTAMP(3).")) } @Test @@ -140,9 +138,9 @@ class WindowTableFunctionTest extends TableTestBase { | TUMBLE(TABLE v1, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)) |""".stripMargin - thrown.expectMessage("Column 'window_start' is ambiguous") - thrown.expect(classOf[ValidationException]) - util.verifyRelPlan(sql) + assertThatThrownBy(() => util.verifyRelPlan(sql)) + .hasMessageContaining("Column 'window_start' is ambiguous") + .isInstanceOf[ValidationException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala index ded61ba8290b5..adf082b857fc2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala @@ -24,7 +24,8 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} import org.apache.flink.table.runtime.typeutils.DecimalDataTypeInfo -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test import java.time.Duration @@ -74,21 +75,23 @@ class AggregateTest extends TableTestBase { "decimal105") ) - @Test(expected = classOf[ValidationException]) + @Test def testGroupingOnNonExistentField(): Unit = { - util.verifyExecPlan("SELECT COUNT(*) FROM MyTable GROUP BY foo") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT COUNT(*) FROM MyTable GROUP BY foo")) } - @Test(expected = classOf[ValidationException]) + @Test def testGroupingInvalidSelection(): Unit = { - util.verifyExecPlan("SELECT b FROM MyTable GROUP BY a") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT b FROM MyTable GROUP BY a")) } @Test def testCannotCountOnMultiFields(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("We now only support the count of one field") - util.verifyExecPlan("SELECT b, COUNT(a, c) FROM MyTable GROUP BY b") + assertThatThrownBy(() => util.verifyExecPlan("SELECT COUNT(a, c) FROM MyTable GROUP BY b")) + .hasMessageContaining("We now only support the count of one field") + .isInstanceOf[TableException] } @Test @@ -391,9 +394,10 @@ class AggregateTest extends TableTestBase { ExplainDetail.CHANGELOG_MODE) } - @Test(expected = classOf[TableException]) + @Test def testApproximateCountDistinct(): Unit = { - util.verifyExecPlan("SELECT APPROX_COUNT_DISTINCT(b) FROM MyTable") + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan("SELECT APPROX_COUNT_DISTINCT(b) FROM MyTable")) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala index 9de0d258f187b..e4ae95bbfb49e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala @@ -23,14 +23,14 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, StreamTableTestUtil, TableTestBase} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class DistinctAggregateTest( splitDistinctAggEnabled: Boolean, aggPhaseEnforcer: AggregatePhaseStrategy) @@ -39,7 +39,7 @@ class DistinctAggregateTest( protected val util: StreamTableTestUtil = streamTestUtil() util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) - @Before + @BeforeEach def before(): Unit = { util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) util.enableMiniBatch() @@ -53,17 +53,17 @@ class DistinctAggregateTest( .set(IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, Boolean.box(false)) } - @Test + @TestTemplate def testSingleDistinctAgg(): Unit = { util.verifyExecPlan("SELECT COUNT(DISTINCT c) FROM MyTable") } - @Test + @TestTemplate def testMultiDistinctAggs(): Unit = { util.verifyExecPlan("SELECT COUNT(DISTINCT a), SUM(DISTINCT b) FROM MyTable") } - @Test + @TestTemplate def testSingleMaxWithDistinctAgg(): Unit = { val sqlQuery = """ @@ -74,24 +74,24 @@ class DistinctAggregateTest( util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSingleFirstValueWithDistinctAgg(): Unit = { // FIRST_VALUE is not mergeable, so the final plan does not contain local agg util.verifyExecPlan("SELECT a, FIRST_VALUE(c), COUNT(DISTINCT b) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testSingleLastValueWithDistinctAgg(): Unit = { // LAST_VALUE is not mergeable, so the final plan does not contain local agg util.verifyExecPlan("SELECT a, LAST_VALUE(c), COUNT(DISTINCT b) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testSingleListAggWithDistinctAgg(): Unit = { util.verifyExecPlan("SELECT a, LISTAGG(c), COUNT(DISTINCT b) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testSingleDistinctAggWithAllNonDistinctAgg(): Unit = { val sqlQuery = """ @@ -102,29 +102,29 @@ class DistinctAggregateTest( util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testTwoDistinctAggregateWithNonDistinctAgg(): Unit = { util.verifyExecPlan( "SELECT c, SUM(DISTINCT a), SUM(a), COUNT(DISTINCT b) FROM MyTable GROUP BY c") } - @Test + @TestTemplate def testSingleDistinctAggWithGroupBy(): Unit = { util.verifyExecPlan("SELECT a, COUNT(DISTINCT c) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testSingleDistinctAggWithAndNonDistinctAggOnSameColumn(): Unit = { util.verifyExecPlan("SELECT a, COUNT(DISTINCT b), SUM(b), AVG(b) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testSomeColumnsBothInDistinctAggAndGroupBy(): Unit = { // TODO: the COUNT(DISTINCT a) can be optimized to literal 1 util.verifyExecPlan("SELECT a, COUNT(DISTINCT a), COUNT(b) FROM MyTable GROUP BY a") } - @Test + @TestTemplate def testAggWithFilterClause(): Unit = { val sqlQuery = s""" @@ -139,7 +139,7 @@ class DistinctAggregateTest( util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testMultiGroupBys(): Unit = { val sqlQuery = s""" @@ -155,7 +155,7 @@ class DistinctAggregateTest( util.verifyExecPlan(sqlQuery) } - @Test + @TestTemplate def testSingleDistinctWithRetraction(): Unit = { val sqlQuery = """ @@ -169,7 +169,7 @@ class DistinctAggregateTest( util.verifyRelPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE) } - @Test + @TestTemplate def testSumCountWithSingleDistinctAndRetraction(): Unit = { val sqlQuery = s""" @@ -185,7 +185,7 @@ class DistinctAggregateTest( util.verifyRelPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE) } - @Test + @TestTemplate def testMinMaxWithRetraction(): Unit = { val sqlQuery = s""" @@ -201,7 +201,7 @@ class DistinctAggregateTest( util.verifyRelPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE) } - @Test + @TestTemplate def testFirstValueLastValueWithRetraction(): Unit = { val sqlQuery = s""" @@ -219,7 +219,7 @@ class DistinctAggregateTest( } object DistinctAggregateTest { - @Parameterized.Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}") + @Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}") def parameters(): util.Collection[Array[Any]] = { util.Arrays.asList( Array(true, AggregatePhaseStrategy.ONE_PHASE), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala index 01466bccaffac..1a74d0b41584e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala @@ -24,7 +24,8 @@ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.Wei import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_ALLOW_LATENESS, TABLE_EXEC_EMIT_LATE_FIRE_DELAY, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test import java.time.Duration @@ -44,44 +45,49 @@ class GroupWindowTest extends TableTestBase { |) |""".stripMargin) - @Test(expected = classOf[TableException]) + @Test def testTumbleWindowNoOffset(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable " + "GROUP BY TUMBLE(proctime, INTERVAL '2' HOUR, TIME '10:00:00')" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testHopWindowNoOffset(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable " + "GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL '2' HOUR, TIME '10:00:00')" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testSessionWindowNoOffset(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB FROM MyTable " + "GROUP BY SESSION(proctime, INTERVAL '2' HOUR, TIME '10:00:00')" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[TableException]) + @Test def testVariableWindowSize(): Unit = { val sql = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, c * INTERVAL '1' MINUTE)" - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } - @Test(expected = classOf[ValidationException]) + @Test def testWindowUdAggInvalidArgs(): Unit = { val sqlQuery = "SELECT SUM(a) AS sumA, weightedAvg(a, b) AS wAvg FROM MyTable " + "GROUP BY TUMBLE(proctime(), INTERVAL '2' HOUR, TIME '10:00:00')" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } - @Test(expected = classOf[AssertionError]) + @Test def testWindowAggWithGroupSets(): Unit = { // TODO supports group sets // currently, the optimized plan is not collect, and an exception will be thrown in code-gen @@ -92,33 +98,32 @@ class GroupWindowTest extends TableTestBase { |FROM MyTable | GROUP BY rollup(TUMBLE(rowtime, INTERVAL '15' MINUTE), b) """.stripMargin - util.verifyRelPlanNotExpected(sql, "TUMBLE(rowtime") + assertThatExceptionOfType(classOf[AssertionError]) + .isThrownBy(() => util.verifyRelPlanNotExpected(sql, "TUMBLE(rowtime")) } @Test def testWindowWrongWindowParameter1(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage( - "Window aggregate only support SECOND, MINUTE, HOUR, DAY as the time unit. " + - "MONTH and YEAR time unit are not supported yet.") - val sqlQuery = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, INTERVAL '1' MONTH)" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) + .withMessageContaining( + "Window aggregate only support SECOND, MINUTE, HOUR, DAY as the time unit. " + + "MONTH and YEAR time unit are not supported yet.") } @Test def testWindowWrongWindowParameter2(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage( - "Window aggregate only support SECOND, MINUTE, HOUR, DAY as the time unit. " + - "MONTH and YEAR time unit are not supported yet.") - val sqlQuery = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, INTERVAL '2-10' YEAR TO MONTH)" - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) + .withMessageContaining( + "Window aggregate only support SECOND, MINUTE, HOUR, DAY as the time unit. " + + "MONTH and YEAR time unit are not supported yet.") } @Test @@ -468,11 +473,11 @@ class GroupWindowTest extends TableTestBase { |FROM MyTable |GROUP BY TUMBLE(`rowtime`, INTERVAL '1' SECOND) |""".stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Allow-lateness [1000ms] should not be smaller than " + + + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE)) + .withMessageContaining("Allow-lateness [1000ms] should not be smaller than " + "Late-fire delay [5000ms] when enable late-fire emit strategy.") - util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala index bab43fafffdff..0bfa78fc1c1a7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala @@ -22,8 +22,8 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import org.junit.jupiter.api.Test import java.sql.Date @@ -403,14 +403,14 @@ class GroupingSetsTest extends TableTestBase { @Test def testCALCITE1824(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("GROUPING SETS are currently not supported") val sqlQuery = """ |SELECT deptno, GROUP_ID() AS g, COUNT(*) AS c |FROM scott_emp GROUP BY GROUPING SETS (deptno, (), ()) """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatThrownBy(() => util.verifyExecPlan(sqlQuery)) + .hasMessageContaining("GROUPING SETS are currently not supported") + .isInstanceOf[TableException] } @Test @@ -503,7 +503,8 @@ class GroupingSetsTest extends TableTestBase { val table2 = util.tableEnv.sqlQuery(sql2) val optimized1 = util.getPlanner.optimize(TableTestUtil.toRelNode(table1)) val optimized2 = util.getPlanner.optimize(TableTestUtil.toRelNode(table2)) - assertEquals(FlinkRelOptUtil.toString(optimized1), FlinkRelOptUtil.toString(optimized2)) + assertThat(FlinkRelOptUtil.toString(optimized2)) + .isEqualTo(FlinkRelOptUtil.toString(optimized1)); } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala index 3daab5cf72d87..63ecf9f9608c8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala @@ -19,20 +19,20 @@ package org.apache.flink.table.planner.plan.stream.sql.agg import org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule import org.apache.flink.table.planner.utils.AggregatePhaseStrategy +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.Before -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.extension.ExtendWith import java.util -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class IncrementalAggregateTest( splitDistinctAggEnabled: Boolean, aggPhaseEnforcer: AggregatePhaseStrategy) extends DistinctAggregateTest(splitDistinctAggEnabled, aggPhaseEnforcer) { - @Before + @BeforeEach override def before(): Unit = { super.before() // enable incremental agg @@ -42,7 +42,7 @@ class IncrementalAggregateTest( } object IncrementalAggregateTest { - @Parameterized.Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}") + @Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}") def parameters(): util.Collection[Array[Any]] = { util.Arrays.asList( Array(true, AggregatePhaseStrategy.TWO_PHASE) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.scala index 8e60a039136f3..3ef4d6b469432 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.scala @@ -23,8 +23,8 @@ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.OverAgg0 import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType} +import org.junit.jupiter.api.Test class OverAggregateTest extends TableTestBase { @@ -37,11 +37,11 @@ class OverAggregateTest extends TableTestBase { val table2 = util.tableEnv.sqlQuery(sql2) val optimized1 = util.getPlanner.optimize(TableTestUtil.toRelNode(table1)) val optimized2 = util.getPlanner.optimize(TableTestUtil.toRelNode(table2)) - assertEquals(FlinkRelOptUtil.toString(optimized1), FlinkRelOptUtil.toString(optimized2)) + assertThat(FlinkRelOptUtil.toString(optimized2)).isEqualTo(FlinkRelOptUtil.toString(optimized1)) } /** All aggregates must be computed on the same window. */ - @Test(expected = classOf[TableException]) + @Test def testMultiWindow(): Unit = { val sqlQuery = """ @@ -51,21 +51,24 @@ class OverAggregateTest extends TableTestBase { |from MyTable """.stripMargin - util.verifyExecPlan(sqlQuery) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sqlQuery)) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation(): Unit = { util.addFunction("overAgg", new OverAgg0) - util.verifyExecPlan("SELECT overAgg(c, a) FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT overAgg(c, a) FROM MyTable")) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation2(): Unit = { util.addFunction("overAgg", new OverAgg0) - util.verifyExecPlan("SELECT overAgg(c, a) FROM MyTable") + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT overAgg(c, a) FROM MyTable")) } @Test @@ -399,7 +402,7 @@ class OverAggregateTest extends TableTestBase { util.verifyExecPlan(sql) } - @Test(expected = classOf[TableException]) + @Test def testProcTimeBoundedPartitionedRowsOverDifferentWindows(): Unit = { val sql = """ @@ -419,8 +422,12 @@ class OverAggregateTest extends TableTestBase { "WINDOW w1 AS (PARTITION BY a ORDER BY proctime ROWS BETWEEN 3 PRECEDING AND CURRENT ROW)," + "w2 AS (PARTITION BY a ORDER BY proctime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)" - verifyPlanIdentical(sql, sql2) - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy( + () => { + verifyPlanIdentical(sql, sql2) + util.verifyExecPlan(sql) + }) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala index 4c8060a0d1c0a..4832e8198ec9f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala @@ -23,14 +23,14 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class TwoStageAggregateTest extends TableTestBase { private val util = streamTestUtil() util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) - @Before + @BeforeEach def before(): Unit = { util.enableMiniBatch() util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala index df866302cc3ba..6e0d5af0069aa 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala @@ -25,16 +25,17 @@ import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.TestPythonAggregateFunction import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.JavaTableFunc1 import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, TableTestBase} +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} -import org.junit.{Before, Test} -import org.junit.Assume.assumeTrue -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.assertj.core.api.Assumptions.assumeThat +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith import java.util /** Tests for window aggregates based on window table-valued function. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends TableTestBase { private val util = streamTestUtil() @@ -46,7 +47,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl */ private val isTwoPhase = aggPhaseEnforcer == AggregatePhaseStrategy.TWO_PHASE - @Before + @BeforeEach def before(): Unit = { util.addTemporarySystemFunction("weightedAvg", classOf[WeightedAvgWithMerge]) util.addTemporarySystemFunction("weightedAvgWithoutMerge", classOf[WeightedAvg]) @@ -87,7 +88,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString) } - @Test + @TestTemplate def testTumble_OnRowtime(): Unit = { val sql = """ @@ -106,9 +107,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_OnProctime(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -126,7 +127,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_CalcOnTVF(): Unit = { val sql = """ @@ -149,7 +150,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_WindowColumnsAtEnd(): Unit = { // there shouldn't be any Calc on the WindowAggregate, // because fields order are align with WindowAggregate schema @@ -170,7 +171,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupMultipleWindowColumns(): Unit = { val sql = """ @@ -194,7 +195,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupMultipleKeys(): Unit = { val sql = """ @@ -213,7 +214,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupOnlyWindowColumns(): Unit = { val sql = """ @@ -230,7 +231,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupOnLiteralValue(): Unit = { val sql = """ @@ -247,7 +248,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_ProjectionPushDown(): Unit = { val sql = """ @@ -263,7 +264,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_CascadingWindow(): Unit = { util.tableEnv.executeSql( """ @@ -295,7 +296,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_CascadingWindow_RelaxForm(): Unit = { // a relax form of cascaded rowtime window which is actually supported util.verifyRelPlan( @@ -319,7 +320,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testTumble_DistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -339,7 +340,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_DistinctOnWindowColumns(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -361,9 +362,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_DoNotSplitProcessingTimeWindow(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue // the processing-time window aggregate with distinct shouldn't be split into two-level agg util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -383,7 +384,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_NotOutputWindowColumns(): Unit = { val sql = """ @@ -399,9 +400,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_UdafWithoutMerge(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue // the window aggregation shouldn't be split into local-global window aggregation val sql = """ @@ -420,7 +421,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_OnRowtime(): Unit = { val sql = """ @@ -440,9 +441,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_OnProctime(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -461,7 +462,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_DistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -482,7 +483,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_OnRowtime(): Unit = { val sql = """ @@ -502,9 +503,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_OnProctime(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -523,7 +524,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_DistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -544,7 +545,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testMultipleAggregateOnSameWindowTVF(): Unit = { util.tableEnv.executeSql( """ @@ -589,7 +590,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl // Tests for queries can't be translated to window aggregate for now // ---------------------------------------------------------------------------------------- - @Test + @TestTemplate def testCantMergeWindowTVF_FilterOnWindowStart(): Unit = { val sql = """ @@ -611,7 +612,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantMergeWindowTVF_UdtfOnWindowTVF(): Unit = { util.tableEnv.createTemporaryFunction("len_udtf", classOf[JavaTableFunc1]) val sql = @@ -634,9 +635,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantTranslateToWindowAgg_GroupOnOnlyStart(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -652,9 +653,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantTranslateToWindowAgg_PythonAggregateCall(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue util.tableEnv.createTemporaryFunction("python_agg", classOf[TestPythonAggregateFunction]) val sql = """ @@ -668,7 +669,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testUnsupportedException_EarlyFire(): Unit = { val conf = new Configuration() conf.setString(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED.key(), "true") @@ -690,15 +691,15 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |GROUP BY a, window_start, window_end """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Currently, window table function based aggregate doesn't support " + - "early-fire and late-fire configuration 'table.exec.emit.early-fire.enabled' and " + - "'table.exec.emit.late-fire.enabled'.") - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining( + "Currently, window table function based aggregate doesn't support " + + "early-fire and late-fire configuration 'table.exec.emit.early-fire.enabled' and " + + "'table.exec.emit.late-fire.enabled'.") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testUnsupportedException_LateFire(): Unit = { val conf = new Configuration() conf.setString(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED.key(), "true") @@ -720,15 +721,15 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |GROUP BY a, window_start, window_end """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Currently, window table function based aggregate doesn't support " + - "early-fire and late-fire configuration 'table.exec.emit.early-fire.enabled' and " + - "'table.exec.emit.late-fire.enabled'.") - util.verifyExecPlan(sql) + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining( + "Currently, window table function based aggregate doesn't support " + + "early-fire and late-fire configuration 'table.exec.emit.early-fire.enabled' and " + + "'table.exec.emit.late-fire.enabled'.") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testUnsupportedException_HopSizeNonDivisible(): Unit = { val sql = """ @@ -742,14 +743,13 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |GROUP BY a, window_start, window_end """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "HOP table function based aggregate requires size must be an " + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("HOP table function based aggregate requires size must be an " + "integral multiple of slide, but got size 600000 ms and slide 240000 ms") - util.verifyExplain(sql) + .isInstanceOf[TableException] } - @Test + @TestTemplate def testUnsupportedException_CumulateSizeNonDivisible(): Unit = { val sql = """ @@ -763,16 +763,15 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |GROUP BY a, window_start, window_end """.stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "CUMULATE table function based aggregate requires maxSize " + + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("CUMULATE table function based aggregate requires maxSize " + "must be an integral multiple of step, but got maxSize 3600000 ms and step 1500000 ms") - util.verifyExplain(sql) + .isInstanceOf[TableException] } - @Test + @TestTemplate def testCantTranslateToWindowAgg_GroupingSetsWithoutWindowStartEnd(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue // Cannot translate to window aggregate because group keys don't contain both window_start // and window_end val sql = @@ -786,9 +785,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantTranslateToWindowAgg_GroupingSetsOnlyWithWindowStart(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -800,7 +799,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupingSets(): Unit = { val sql = """ @@ -814,7 +813,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupingSets1(): Unit = { val sql = """ @@ -828,7 +827,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_GroupingSetsDistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -847,9 +846,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantTranslateToWindowAgg_CubeWithoutWindowStartEnd(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -862,9 +861,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantTranslateToWindowAgg_RollupWithoutWindowStartEnd(): Unit = { - assumeTrue(isTwoPhase) + assumeThat(isTwoPhase).isTrue val sql = """ |SELECT @@ -877,7 +876,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testTumble_Rollup(): Unit = { val sql = """ @@ -891,7 +890,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCantMergeWindowTVF_GroupingSetsDistinctOnWindowColumns(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -912,7 +911,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_GroupingSets(): Unit = { val sql = """ @@ -927,7 +926,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_GroupingSets_DistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -945,7 +944,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_Cube(): Unit = { val sql = """ @@ -960,7 +959,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testHop_Rollup(): Unit = { val sql = """ @@ -975,7 +974,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_GroupingSets(): Unit = { val sql = """ @@ -990,7 +989,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_GroupingSets_DistinctSplitEnabled(): Unit = { util.tableEnv.getConfig .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @@ -1008,7 +1007,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_Cube(): Unit = { val sql = """ @@ -1023,7 +1022,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testCumulate_Rollup(): Unit = { val sql = """ @@ -1038,7 +1037,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testFieldNameConflict(): Unit = { val sql = """ @@ -1051,7 +1050,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl util.verifyRelPlan(sql) } - @Test + @TestTemplate def testProctimeWindowWithFilter(): Unit = { util.tableEnv.executeSql(s""" |CREATE TEMPORARY TABLE source ( @@ -1099,9 +1098,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |""".stripMargin) } - @Test + @TestTemplate def testTumble_CascadingWindow_OnIndividualProctime(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // a standard cascaded proctime window util.verifyExecPlan( """ @@ -1115,9 +1114,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |""".stripMargin) } - @Test + @TestTemplate def testTumble_CascadingWindow_OnInheritProctime(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // a standard cascaded proctime window util.verifyExecPlan( """ @@ -1131,9 +1130,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |""".stripMargin) } - @Test + @TestTemplate def testInvalidRelaxFormCascadeProctimeWindow(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // a relax form of cascaded proctime window unsupported for now, will be translated to group agg util.verifyRelPlan(""" |SELECT @@ -1146,9 +1145,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testTumble_CascadeProctimeWindow_OnWindowRank(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // create window top10 createProctimeWindowTopN("proctime_winrank", 10) @@ -1183,9 +1182,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testInvalidRelaxFormCascadeProctimeWindow_OnWindowRank(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // create window top10 createProctimeWindowTopN("proctime_winrank", 10) @@ -1201,9 +1200,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testTumble_CascadeProctimeWindow_OnWindowDedup(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // create window dedup(top1) createProctimeWindowTopN("proctime_windedup", 1) @@ -1220,9 +1219,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testInvalidRelaxFormCascadeProctimeWindow_OnWindowDedup(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse // create window dedup(top1) createProctimeWindowTopN("proctime_windedup", 1) @@ -1238,9 +1237,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testTumble_CascadeProctimeWindow_OnWindowJoin(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse createWindowJoin util.verifyRelPlan( @@ -1281,9 +1280,9 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl """.stripMargin) } - @Test + @TestTemplate def testInvalidRelaxFormCascadeProctimeWindow_OnWindowJoin(): Unit = { - assumeTrue(!isTwoPhase) + assumeThat(isTwoPhase).isFalse createWindowJoin // a relax form of cascaded proctime window on a window join is unsupported for now, will be translated to group agg @@ -1300,7 +1299,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl } object WindowAggregateTest { - @Parameterized.Parameters(name = "aggPhaseEnforcer={0}") + @Parameters(name = "aggPhaseEnforcer={0}") def parameters(): util.Collection[Array[Any]] = { util.Arrays.asList( Array(AggregatePhaseStrategy.ONE_PHASE), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala index c40cc18782b82..4333151b1be58 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala @@ -24,8 +24,8 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctio import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase, TableTestUtil} import org.apache.calcite.rel.logical.LogicalJoin -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType, assertThatThrownBy} +import org.junit.jupiter.api.Test class IntervalJoinTest extends TableTestBase { @@ -92,9 +92,6 @@ class IntervalJoinTest extends TableTestBase { /** Both rowtime types in a join condition must be of the same type * */ @Test def testIntervalJoinOnDiffRowTimeType(): Unit = { - expectedException.expectMessage( - "Interval join with rowtime attribute requires same rowtime types," + - " but the types are TIMESTAMP(3) *ROWTIME* and TIMESTAMP_LTZ(3) *ROWTIME*") val sql = """ |SELECT t2.a FROM MyTable2 t1 JOIN MyTable3 t2 ON @@ -102,7 +99,11 @@ class IntervalJoinTest extends TableTestBase { | t1.rowtime > t2.rowtime - INTERVAL '5' SECOND AND | t1.rowtime < t2.rowtime + INTERVAL '5' SECOND """.stripMargin - util.verifyExecPlan(sql) + + assertThatThrownBy(() => util.verifyExecPlan(sql)) + .hasMessageContaining( + "Interval join with rowtime attribute requires same rowtime types," + + " but the types are TIMESTAMP(3) *ROWTIME* and TIMESTAMP_LTZ(3) *ROWTIME*") } /** The time conditions should be an And condition * */ @@ -119,7 +120,7 @@ class IntervalJoinTest extends TableTestBase { } /** Validates that no rowtime attribute is in the output schema * */ - @Test(expected = classOf[TableException]) + @Test def testNoRowtimeAttributeInResult(): Unit = { val sql = """ @@ -128,14 +129,15 @@ class IntervalJoinTest extends TableTestBase { | t1.proctime BETWEEN t2.proctime - INTERVAL '5' SECOND AND t2.proctime """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } /** * Currently only the inner join condition can support the Python UDF taking the inputs from the * left table and the right table at the same time. */ - @Test(expected = classOf[TableException]) + @Test def testWindowOuterJoinWithPythonFunctionInCondition(): Unit = { util.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) val sql = @@ -144,7 +146,8 @@ class IntervalJoinTest extends TableTestBase { | t1.a = t2.a AND pyFunc(t1.a, t2.a) = t1.a + t2.a AND | t1.proctime BETWEEN t2.proctime - INTERVAL '1' HOUR AND t2.proctime + INTERVAL '1' HOUR """.stripMargin - util.verifyExecPlan(sql) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(sql)) } @Test @@ -530,9 +533,9 @@ class IntervalJoinTest extends TableTestBase { ) val timeTypeStr = if (windowBounds.get.isEventTime) "rowtime" else "proctime" - assertEquals(expLeftSize, windowBounds.get.getLeftLowerBound) - assertEquals(expRightSize, windowBounds.get.getLeftUpperBound) - assertEquals(expTimeType, timeTypeStr) + assertThat(windowBounds.get.getLeftLowerBound).isEqualTo(expLeftSize) + assertThat(windowBounds.get.getLeftUpperBound).isEqualTo(expRightSize) + assertThat(timeTypeStr).isEqualTo(expTimeType) } private def verifyRemainConditionConvert(sqlQuery: String, expectConditionStr: String): Unit = { @@ -552,7 +555,7 @@ class IntervalJoinTest extends TableTestBase { Thread.currentThread().getContextClassLoader ) val actual: String = remainCondition.getOrElse("").toString - assertEquals(expectConditionStr, actual) + assertThat(actual).isEqualTo(expectConditionStr) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinReorderTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinReorderTest.scala index 9cdacbebe81bc..b4ec8c0c636fa 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinReorderTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinReorderTest.scala @@ -19,12 +19,12 @@ package org.apache.flink.table.planner.plan.stream.sql.join import org.apache.flink.table.planner.plan.common.JoinReorderTestBase import org.apache.flink.table.planner.utils.TableTestUtil +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.junit.jupiter.api.extension.ExtendWith /** The plan test for join reorder in stream mode. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class JoinReorderTest(isBushyJoinReorder: Boolean) extends JoinReorderTestBase(isBushyJoinReorder: Boolean) { override protected def getTableTestUtil: TableTestUtil = streamTestUtil() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala index ac413fc4c70bb..4df4c3de8dd3d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableFunc1, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class JoinTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala index b0f870f72544b..f9e0749a159ee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.plan.stream.sql.join import org.apache.flink.api.scala._ -import org.apache.flink.core.testutils.FlinkMatchers.containsMessage import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api._ @@ -35,27 +34,28 @@ import org.apache.flink.table.planner.utils.TableTestUtil.{readFromResource, rep import org.apache.flink.table.sources._ import org.apache.flink.table.types.DataType import org.apache.flink.table.utils.EncodingUtils +import org.apache.flink.testutils.junit.extensions.parameterized.{ParameterizedTestExtension, Parameters} import _root_.java.lang.{Boolean => JBoolean} import _root_.java.sql.Timestamp import _root_.java.util import _root_.java.util.{ArrayList => JArrayList, Collection => JCollection, HashMap => JHashMap, List => JList, Map => JMap} import _root_.scala.collection.JavaConversions._ -import org.junit.{Assume, Before, Test} -import org.junit.Assert.{assertEquals, assertThat, assertTrue, fail} -import org.junit.runner.RunWith -import org.junit.runners.Parameterized +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType, assertThatThrownBy} +import org.assertj.core.api.Assumptions.assumeThat +import org.junit.jupiter.api.{BeforeEach, TestTemplate} +import org.junit.jupiter.api.extension.ExtendWith /** * The physical plans for legacy [[org.apache.flink.table.sources.LookupableTableSource]] and new * [[org.apache.flink.table.connector.source.LookupTableSource]] should be identical. */ -@RunWith(classOf[Parameterized]) +@ExtendWith(Array(classOf[ParameterizedTestExtension])) class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Serializable { private val util = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addDataStream[(Int, String, Long)]( "MyTable", @@ -117,7 +117,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, Int.box(4)) } - @Test + @TestTemplate def testJoinInvalidJoinTemporalTable(): Unit = { // must follow a period specification expectExceptionThrown( @@ -143,7 +143,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri ) } - @Test + @TestTemplate def testNotDistinctFromInJoinCondition(): Unit = { // does not support join condition contains `IS NOT DISTINCT` @@ -165,7 +165,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri ) } - @Test + @TestTemplate def testInvalidLookupTableFunction(): Unit = { if (legacyTableSource) { return @@ -262,19 +262,20 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri ) } - @Test + @TestTemplate def testJoinOnDifferentKeyTypes(): Unit = { // Will do implicit type coercion. - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "implicit type conversion between VARCHAR(2147483647) and INTEGER " + + assertThatThrownBy( + () => + util.verifyExecPlan( + "SELECT * FROM MyTable AS T JOIN LookupTable " + + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id")) + .hasMessageContaining("implicit type conversion between VARCHAR(2147483647) and INTEGER " + "is not supported on join's condition now") - util.verifyExecPlan( - "SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testJoinTemporalTable(): Unit = { val sql = "SELECT * FROM MyTable AS T JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id" @@ -282,7 +283,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testLeftJoinTemporalTable(): Unit = { val sql = "SELECT * FROM MyTable AS T LEFT JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id" @@ -290,7 +291,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithNestedQuery(): Unit = { val sql = "SELECT * FROM " + "(SELECT a, b, proctime FROM MyTable WHERE c > 1000) AS T " + @@ -300,7 +301,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithProjectionPushDown(): Unit = { val sql = """ @@ -313,7 +314,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithFilterPushDown(): Unit = { val sql = """ @@ -326,7 +327,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithCalcPushDown(): Unit = { val sql = """ @@ -339,7 +340,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithMultiIndexColumn(): Unit = { val sql = """ @@ -352,7 +353,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAvoidAggregatePushDown(): Unit = { val sql1 = """ @@ -379,12 +380,8 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithTrueCondition(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Temporal table join requires an equality condition on fields of " + - "table [default_catalog.default_database.LookupTable]") val sql = """ |SELECT * FROM MyTable AS T @@ -393,10 +390,13 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri |WHERE T.c > 1000 """.stripMargin - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Temporal table join requires an equality condition on fields of " + + "table [default_catalog.default_database.LookupTable]") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testJoinTemporalTableWithFunctionAndConstantCondition(): Unit = { val sql = @@ -409,7 +409,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithMultiFunctionAndConstantCondition(): Unit = { val sql = @@ -422,7 +422,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithFunctionAndReferenceCondition(): Unit = { val sql = """ @@ -435,7 +435,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithUdfEqualFilter(): Unit = { val sql = """ @@ -449,10 +449,10 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithComputedColumn(): Unit = { // Computed column do not support in legacyTableSource. - Assume.assumeFalse(legacyTableSource) + assumeThat(legacyTableSource).isFalse val sql = """ |SELECT @@ -464,10 +464,10 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithComputedColumnAndPushDown(): Unit = { // Computed column do not support in legacyTableSource. - Assume.assumeFalse(legacyTableSource) + assumeThat(legacyTableSource).isFalse val sql = """ |SELECT @@ -479,7 +479,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithMultiConditionOnSameDimField(): Unit = { val sql = "SELECT * FROM MyTable AS T JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id and CAST(T.c as INT) = D.id" @@ -487,7 +487,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithCastOnLookupTable(): Unit = { util.addTable(""" |CREATE TABLE LookupTable2 ( @@ -505,14 +505,14 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri |LEFT JOIN LookupTable2 FOR SYSTEM_TIME AS OF MyTable.`proctime` |ON MyTable.a = CAST(LookupTable2.`id` as INT) |""".stripMargin - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Temporal table join requires an equality condition on fields of " + + + assertThatThrownBy(() => verifyTranslationSuccess(sql)) + .hasMessageContaining("Temporal table join requires an equality condition on fields of " + "table [default_catalog.default_database.LookupTable2]") - verifyTranslationSuccess(sql) + .isInstanceOf[TableException] } - @Test + @TestTemplate def testJoinTemporalTableWithInteroperableCastOnLookupTable(): Unit = { util.addTable(""" |CREATE TABLE LookupTable2 ( @@ -534,7 +534,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri verifyTranslationSuccess(sql) } - @Test + @TestTemplate def testJoinTemporalTableWithCTE(): Unit = { val sql = """ @@ -548,7 +548,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAggAndAllConstantLookupKeyWithTryResolveMode(): Unit = { // expect lookup join using single parallelism due to all constant lookup key util.tableEnv.getConfig.set( @@ -571,12 +571,11 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri readFromResource( "explain/stream/join/lookup/testAggAndAllConstantLookupKeyWithTryResolveMode_newSource.out") } - assertEquals( - replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expected))), - replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actual)))) + assertThat(replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actual)))) + .isEqualTo(replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expected)))) } - @Test + @TestTemplate def testInvalidJoinHint(): Unit = { // lost required hint option 'table' expectExceptionThrown( @@ -727,7 +726,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri ) } - @Test + @TestTemplate def testJoinHintWithTableAlias(): Unit = { val sql = "SELECT /*+ LOOKUP('table'='D', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * FROM MyTable AS T JOIN LookupTable " + @@ -735,14 +734,14 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinHintWithTableNameOnly(): Unit = { val sql = "SELECT /*+ LOOKUP('table'='LookupTable') */ * FROM MyTable AS T JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime ON T.a = LookupTable.id" util.verifyExecPlan(sql) } - @Test + @TestTemplate def testMultipleJoinHintsWithSameTableName(): Unit = { // only the first hint will take effect val sql = @@ -756,7 +755,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testMultipleJoinHintsWithSameTableAlias(): Unit = { // only the first hint will take effect val sql = @@ -770,7 +769,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testMultipleJoinHintsWithDifferentTableName(): Unit = { // both hints on corresponding tables will take effect val sql = @@ -786,7 +785,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testMultipleJoinHintsWithDifferentTableAlias(): Unit = { // both hints on corresponding tables will take effect val sql = @@ -802,7 +801,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinSyncTableWithAsyncHint(): Unit = { val sql = "SELECT /*+ LOOKUP('table'='D', 'async'='true') */ * FROM MyTable AS T JOIN LookupTable " + @@ -810,7 +809,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinAsyncTableWithAsyncHint(): Unit = { val sql = "SELECT /*+ LOOKUP('table'='D', 'async'='true') */ * " + @@ -819,7 +818,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testJoinAsyncTableWithSyncHint(): Unit = { val sql = "SELECT /*+ LOOKUP('table'='D', 'async'='false') */ * " + @@ -828,7 +827,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } - @Test + @TestTemplate def testAggAndLeftJoinAllowUnordered(): Unit = { util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_OUTPUT_MODE, @@ -848,9 +847,6 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri } def testAggAndLeftJoinWithTryResolveMode(): Unit = { - thrown.expectMessage("Required sync lookup function by planner, but table") - thrown.expect(classOf[TableException]) - util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, OptimizerConfigOptions.NonDeterministicUpdateStrategy.TRY_RESOLVE) @@ -865,10 +861,12 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri |FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id |""".stripMargin) - util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN) + assertThatThrownBy(() => util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN)) + .hasMessageContaining("Required sync lookup function by planner, but table") + .isInstanceOf[TableException] } - @Test + @TestTemplate def testAsyncJoinWithDefaultParams(): Unit = { val stmt = util.tableEnv.asInstanceOf[TestingTableEnvironment].createStatementSet() stmt.addInsertSql(""" @@ -882,7 +880,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN) } - @Test + @TestTemplate def testJoinWithAsyncHint(): Unit = { val stmt = util.tableEnv.asInstanceOf[TestingTableEnvironment].createStatementSet() stmt.addInsertSql( @@ -898,7 +896,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN) } - @Test + @TestTemplate def testJoinWithRetryHint(): Unit = { val stmt = util.tableEnv.asInstanceOf[TestingTableEnvironment].createStatementSet() stmt.addInsertSql( @@ -914,7 +912,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN) } - @Test + @TestTemplate def testJoinWithAsyncAndRetryHint(): Unit = { val stmt = util.tableEnv.asInstanceOf[TestingTableEnvironment].createStatementSet() stmt.addInsertSql( @@ -930,7 +928,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExplain(stmt, ExplainDetail.JSON_EXECUTION_PLAN) } - @Test + @TestTemplate def testJoinWithMixedCaseJoinHint(): Unit = { util.verifyExecPlan( """ @@ -946,7 +944,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri ) } - @Test + @TestTemplate def testJoinHintWithNoPropagatingToSubQuery(): Unit = { util.verifyExecPlan( """ @@ -992,14 +990,9 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri sql: String, message: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { - verifyTranslationSuccess(sql) - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e: Throwable => - assertTrue(clazz.isAssignableFrom(e.getClass)) - assertThat(e, containsMessage(message)) - } + assertThatExceptionOfType(clazz) + .isThrownBy(() => verifyTranslationSuccess(sql)) + .withMessageContaining(message) } private def verifyTranslationSuccess(sql: String): Unit = { @@ -1008,7 +1001,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri } object LookupJoinTest { - @Parameterized.Parameters(name = "LegacyTableSource={0}") + @Parameters(name = "LegacyTableSource={0}") def parameters(): JCollection[Array[Object]] = { Seq[Array[AnyRef]](Array(JBoolean.TRUE), Array(JBoolean.FALSE)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.scala index e8dba6c10913e..f1b8c9d584a7d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SemiAntiJoinTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.scala index 6e522d240a024..3b4ea69d25b5b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.scala @@ -21,8 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.hamcrest.Matchers.containsString -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -128,25 +128,23 @@ class TemporalFunctionJoinTest extends TableTestBase { @Test def testUncorrelatedJoin(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage(containsString("Cannot generate a valid execution plan")) - val sqlQuery = "SELECT " + "o_amount * rate as rate " + "FROM Orders AS o, " + "LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123')) AS r " + "WHERE currency = o_currency" - util.verifyExplain(sqlQuery) + assertThatThrownBy(() => util.verifyExplain(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan") + .isInstanceOf[TableException] } @Test def testTemporalTableFunctionScan(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage(containsString("Cannot generate a valid execution plan")) - val sqlQuery = "SELECT * FROM LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123'))" - util.verifyExplain(sqlQuery) + assertThatThrownBy(() => util.verifyExplain(sqlQuery)) + .hasMessageContaining("Cannot generate a valid execution plan") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala index c385454a42e5f..29d6521540c9d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala @@ -20,15 +20,16 @@ package org.apache.flink.table.planner.plan.stream.sql.join import org.apache.flink.table.api.{ExplainDetail, TableException, ValidationException} import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.{Before, Test} -import org.junit.Assert.{assertTrue, fail} +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.assertj.core.api.ThrowableAssert.ThrowingCallable +import org.junit.jupiter.api.{BeforeEach, Test} /** Test temporal join in stream mode. */ class TemporalJoinTest extends TableTestBase { val util: StreamTableTestUtil = streamTestUtil() - @Before + @BeforeEach def before(): Unit = { util.addTable(""" |CREATE TABLE Orders ( @@ -618,20 +619,14 @@ class TemporalJoinTest extends TableTestBase { sql: String, keywords: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { - verifyTranslationSuccess(sql) - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e if e.getClass == clazz => - if (keywords != null) { - assertTrue( - s"The actual exception message \n${e.getMessage}\n" + - s"doesn't contain expected keyword \n$keywords\n", - e.getMessage.contains(keywords)) - } - case e: Throwable => - e.printStackTrace() - fail(s"Expected throw ${clazz.getSimpleName}, but is $e.") + val callable: ThrowingCallable = () => verifyTranslationSuccess(sql) + if (keywords != null) { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) + .withMessageContaining(keywords) + } else { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala index 9fe0b3e7fbfd0..792d2e40df8e0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.stream.sql.join import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test /** Tests for window join. */ class WindowJoinTest extends TableTestBase { @@ -148,9 +149,9 @@ class WindowJoinTest extends TableTestBase { |ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a """.stripMargin - thrown.expectMessage("Processing time Window Join is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window Join is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -255,9 +256,9 @@ class WindowJoinTest extends TableTestBase { |ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a """.stripMargin - thrown.expectMessage("Processing time Window Join is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window Join is not supported yet.") + .isInstanceOf[TableException] } @Test @@ -362,9 +363,9 @@ class WindowJoinTest extends TableTestBase { |ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a """.stripMargin - thrown.expectMessage("Processing time Window Join is not supported yet.") - thrown.expect(classOf[TableException]) - util.verifyExplain(sql) + assertThatThrownBy(() => util.verifyExplain(sql)) + .hasMessageContaining("Processing time Window Join is not supported yet.") + .isInstanceOf[TableException] } // ---------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/AggregateTest.scala index 6533785811525..ade4c8b1f031a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/AggregateTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.planner.utils.{CountMinMax, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class AggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CalcTest.scala index 5719ede551062..554eeba0a38af 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CalcTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.expressions.utils.{Func1, Func23, Func24} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class CalcTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.scala index 5e636774ad74c..2d5ea2c8634da 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg} import org.apache.flink.table.planner.utils.{CountAggFunction, TableFunc0, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test /** Tests for column functions which includes tests for different column functions. */ class ColumnFunctionsTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala index 2198c47046c60..deedcf48680cd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.utils._ import org.apache.calcite.rel.rules.CoreRules import org.apache.calcite.tools.RuleSets -import org.junit.Test +import org.junit.jupiter.api.Test class CorrelateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.scala index c0eb9543b11b6..a20ce81d37436 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{EmptyTableAggFunc, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class GroupWindowTableAggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.scala index 8a826ca9cef1c..49bfe1210e522 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} import org.apache.flink.table.planner.utils.{EmptyTableAggFunc, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/JoinTest.scala index 33a446d2bf1d4..4296b0c8417ff 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/JoinTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test import java.sql.Timestamp diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/LegacyTableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/LegacyTableSourceTest.scala index 9361ea46c79ce..e3742b206b9a7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/LegacyTableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/LegacyTableSourceTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.utils.{TableTestBase, TestLegacyProjectableTableSource, TestNestedProjectableTableSource, TestTableSourceWithTime} import org.apache.flink.types.Row -import org.junit.Test +import org.junit.jupiter.api.Test class LegacyTableSourceTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/OverAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/OverAggregateTest.scala index e0850e6764d63..8227a97cac264 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/OverAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/OverAggregateTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithRetract import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class OverAggregateTest extends TableTestBase { private val streamUtil: StreamTableTestUtil = streamTestUtil() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.scala index 348339d436519..14d30421fa032 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.scala @@ -26,8 +26,8 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions. import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.table.runtime.dataview.{ListViewSpec, MapViewSpec} -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType} +import org.junit.jupiter.api.Test class PythonAggregateTest extends TableTestBase { @@ -92,30 +92,34 @@ class PythonAggregateTest extends TableTestBase { false) ) - assertEquals(expected(0).getClass, specs(0).getClass) - assertEquals(expected(0).getDataType, specs(0).getDataType) - assertEquals(expected(0).getStateId, specs(0).getStateId) - assertEquals(expected(0).getFieldIndex, specs(0).getFieldIndex) - assertEquals(expected(1).getClass, specs(1).getClass) - assertEquals(expected(1).getDataType, specs(1).getDataType) - assertEquals(expected(1).getStateId, specs(1).getStateId) - assertEquals(expected(1).getFieldIndex, specs(1).getFieldIndex) + assertThat(specs(0)).hasSameClassAs(expected(0)) + assertThat(specs(0).getDataType).isEqualTo(expected(0).getDataType) + assertThat(specs(0).getStateId).isEqualTo(expected(0).getStateId) + assertThat(specs(0).getFieldIndex).isEqualTo(expected(0).getFieldIndex) + assertThat(specs(1)).hasSameClassAs(expected(1)) + assertThat(specs(1).getDataType).isEqualTo(expected(1).getDataType) + assertThat(specs(1).getStateId).isEqualTo(expected(1).getStateId) + assertThat(specs(1).getFieldIndex).isEqualTo(expected(1).getFieldIndex) } - @Test(expected = classOf[TableException]) + @Test def testExtractSecondLevelDataViewSpecs(): Unit = { val accType = DataTypes.ROW( DataTypes.FIELD( "f0", DataTypes.ROW(DataTypes.FIELD("f0", ListView.newListViewDataType(DataTypes.STRING()))))) - CommonPythonUtil.extractDataViewSpecs(0, accType) + + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => CommonPythonUtil.extractDataViewSpecs(0, accType)) } - @Test(expected = classOf[TableException]) + @Test def testExtractDataViewSpecsFromStructuredType(): Unit = { val accType = DataTypes.STRUCTURED( classOf[Tuple1[_]], DataTypes.FIELD("f0", ListView.newListViewDataType(DataTypes.STRING()))) - CommonPythonUtil.extractDataViewSpecs(0, accType) + + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => CommonPythonUtil.extractDataViewSpecs(0, accType)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala index c3dbfceb3e326..e5e076a534f1b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala @@ -22,12 +22,12 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} class PythonCalcTest extends TableTestBase { private val util = streamTestUtil() - @Before + @BeforeEach def setup(): Unit = { util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) util.addFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.scala index b590a4a801341..a254f337dc53b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.{PandasAggregateFunction, TestPythonAggregateFunction} import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test import java.time.Duration @@ -89,7 +90,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { util.verifyExecPlan(resultTable) } - @Test(expected = classOf[TableException]) + @Test def testPandasEventTimeSessionGroupWindowOverTime(): Unit = { val util = streamTestUtil() val sourceTable = @@ -100,7 +101,9 @@ class PythonGroupWindowAggregateTest extends TableTestBase { .window(Session.withGap(7.millis).on('rowtime).as('w)) .groupBy('w, 'b) .select('b, 'w.start, 'w.end, func('a, 'c)) - util.verifyExecPlan(windowedTable) + + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(windowedTable)) } @Test @@ -177,7 +180,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { util.verifyExecPlan(windowedTable) } - @Test(expected = classOf[TableException]) + @Test def testEmitStrategyNotSupported(): Unit = { val util = streamTestUtil() val tableConf = util.getTableEnv.getConfig @@ -193,6 +196,7 @@ class PythonGroupWindowAggregateTest extends TableTestBase { .groupBy('w, 'b) .select('b, 'w.start, 'w.end, func('a, 'c)) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.scala index a63faad289380..e297ecf25e465 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.PandasAggregateFunction import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class PythonOverWindowAggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.scala index e04970f9b2e0c..834cfa93e1009 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{PythonEmptyTableAggFunc, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class PythonTableAggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.scala index 1e0521550d6f7..f5f8e2b81b417 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.junit.jupiter.api.Test class SetOperatorsTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.scala index 42e173cfb593d..0d566db60d1c7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.utils.{EmptyTableAggFunc, EmptyTableAggFuncWithIntResultType, TableTestBase} -import org.junit.Test +import org.junit.jupiter.api.Test class TableAggregateTest extends TableTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala index 50cc948344d19..91691fc90f3b9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala @@ -17,11 +17,11 @@ */ package org.apache.flink.table.planner.plan.stream.table -import org.apache.flink.core.testutils.FlinkMatchers.containsMessage import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test class TableSourceTest extends TableTestBase { @@ -170,10 +170,6 @@ class TableSourceTest extends TableTestBase { @Test def testProctimeOnWatermarkSpec(): Unit = { - thrown.expect(classOf[ValidationException]) - thrown.expect( - containsMessage("A watermark can not be defined for a processing-time attribute.")) - val ddl = s""" |CREATE TABLE T ( @@ -188,10 +184,16 @@ class TableSourceTest extends TableTestBase { | 'bounded' = 'false' |) """.stripMargin - util.tableEnv.executeSql(ddl) - val t = util.tableEnv.from("T").select('ptime) - util.verifyExecPlan(t) + assertThatThrownBy( + () => { + util.tableEnv.executeSql(ddl) + + val t = util.tableEnv.from("T").select('ptime) + util.verifyExecPlan(t) + }) + .hasMessageContaining("A watermark can not be defined for a processing-time attribute.") + .isInstanceOf[ValidationException] } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.scala index 7fc8cf56d3215..af2be420d325b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.scala @@ -19,15 +19,14 @@ package org.apache.flink.table.planner.plan.stream.table import org.apache.flink.api.scala._ import org.apache.flink.table.api._ -import org.apache.flink.table.expressions.{Expression, FieldReferenceExpression} +import org.apache.flink.table.expressions.FieldReferenceExpression import org.apache.flink.table.functions.{TemporalTableFunction, TemporalTableFunctionImpl} import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo.{PROCTIME_INDICATOR, ROWTIME_INDICATOR} -import org.hamcrest.Matchers.{equalTo, startsWith} -import org.junit.Assert.{assertEquals, assertThat} -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -143,16 +142,15 @@ class TemporalTableFunctionJoinTest extends TableTestBase { @Test def testUncorrelatedJoin(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage(startsWith("Unsupported argument")) - val result = orders .joinLateral( rates(java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123")), 'o_currency === 'currency) .select($"o_amount" * $"rate") - util.verifyExecPlan(result) + assertThatThrownBy(() => util.verifyExecPlan(result)) + .hasMessageContaining("Unsupported argument") + .isInstanceOf[ValidationException] } @Test @@ -171,9 +169,8 @@ class TemporalTableFunctionJoinTest extends TableTestBase { inputRates: TemporalTableFunction, proctime: Boolean = false): Unit = { val rates = inputRates.asInstanceOf[TemporalTableFunctionImpl] - assertThat( - rates.getPrimaryKey, - equalTo[Expression](new FieldReferenceExpression("currency", DataTypes.STRING(), 0, 0))) + assertThat(rates.getPrimaryKey).isEqualTo( + new FieldReferenceExpression("currency", DataTypes.STRING(), 0, 0)) val (timeFieldName, timeFieldType) = if (proctime) { @@ -182,11 +179,9 @@ class TemporalTableFunctionJoinTest extends TableTestBase { ("rowtime", fromLegacyInfoToDataType(ROWTIME_INDICATOR)) } - assertThat( - rates.getTimeAttribute, - equalTo[Expression](new FieldReferenceExpression(timeFieldName, timeFieldType, 0, 2))) - - assertEquals(expectedSchema.toRowType, rates.getResultType) + assertThat(rates.getTimeAttribute).isEqualTo( + new FieldReferenceExpression(timeFieldName, timeFieldType, 0, 2)) + assertThat(rates.getResultType).isEqualTo(expectedSchema.toRowType) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala index f2538d18948f7..a732907df6c90 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala @@ -23,14 +23,14 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, StreamTableTestUtil, TableTestBase} -import org.junit.{Before, Test} +import org.junit.jupiter.api.{BeforeEach, Test} import java.time.Duration class TwoStageAggregateTest extends TableTestBase { private var util: StreamTableTestUtil = _ - @Before + @BeforeEach def before(): Unit = { util = streamTestUtil() util.tableEnv.getConfig diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.scala index 40b920f97d32a..cb8951f1a3d66 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.scala @@ -22,55 +22,68 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{TableFunc0, TableTestBase} import org.apache.flink.types.Row -import org.junit.Assert.{assertTrue, fail} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.assertj.core.api.ThrowableAssert.ThrowingCallable +import org.junit.jupiter.api.Test class AggregateValidationTest extends TableTestBase { private val util = scalaStreamTestUtil() - @Test(expected = classOf[ValidationException]) + @Test def testGroupingOnNonExistentField(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) - val ds = table - // must fail. '_foo is not a valid field - .groupBy('_foo) - .select('a.avg) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + // must fail. '_foo is not a valid field + .groupBy('foo) + .select('a.avg)) } - @Test(expected = classOf[ValidationException]) + @Test def testGroupingInvalidSelection(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) - table - .groupBy('a, 'b) - // must fail. 'c is not a grouping key or aggregation - .select('c) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .groupBy('a, 'b) + // must fail. 'c is not a grouping key or aggregation + .select('c)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidAggregationInSelection(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) - table - .groupBy('a) - .aggregate('b.sum.as('d)) - // must fail. Cannot use AggregateFunction in select right after aggregate - .select('d.sum) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .groupBy('a) + .aggregate('b.sum.as('d)) + // must fail. Cannot use AggregateFunction in select right after aggregate + .select('d.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidWindowPropertiesInSelection(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) - table - .groupBy('a) - .aggregate('b.sum.as('d)) - // must fail. Cannot use window properties in select right after aggregate - .select('d.start) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .groupBy('a) + .aggregate('b.sum.as('d)) + // must fail. Cannot use window properties in select right after aggregate + .select('d.start)) } - @Test(expected = classOf[RuntimeException]) + @Test def testTableFunctionInSelection(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) @@ -81,30 +94,37 @@ class AggregateValidationTest extends TableTestBase { // must fail. Cannot use TableFunction in select after aggregate .select(call("func", "abc")) - util.verifyExecPlan(resultTable) + assertThatExceptionOfType(classOf[RuntimeException]) + .isThrownBy(() => util.verifyExecPlan(resultTable)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidScalarFunctionInAggregate(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) - table - .groupBy('a) - // must fail. Only AggregateFunction can be used in aggregate - .aggregate('c.upperCase.as('d)) - .select('a, 'd) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .groupBy('a) + // must fail. Only AggregateFunction can be used in aggregate + .aggregate('c.upperCase.as('d)) + .select('a, 'd)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidTableFunctionInAggregate(): Unit = { val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) util.addFunction("func", new TableFunc0) - table - .groupBy('a) - // must fail. Only AggregateFunction can be used in aggregate - .aggregate(call("func", $"c").as("d")) - .select('a, 'd) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .groupBy('a) + // must fail. Only AggregateFunction can be used in aggregate + .aggregate(call("func", $"c").as("d")) + .select('a, 'd)) } @Test @@ -135,17 +155,15 @@ class AggregateValidationTest extends TableTestBase { sql: String, keywords: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { + val callable: ThrowingCallable = () => util.tableEnv.toAppendStream[Row](util.tableEnv.sqlQuery(sql)) - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e if e.getClass == clazz => - if (keywords != null) { - assertTrue( - s"The exception message '${e.getMessage}' doesn't contain keyword '$keywords'", - e.getMessage.contains(keywords)) - } - case e: Throwable => fail(s"Expected throw ${clazz.getSimpleName}, but is $e.") + if (keywords != null) { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) + .withMessageContaining(keywords) + } else { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala index 702ce4a6b5221..661781dd1b88c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala @@ -22,142 +22,186 @@ import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.planner.utils.{TableFunc0, TableTestBase} -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType} +import org.junit.jupiter.api.Test import java.math.BigDecimal class CalcValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testInvalidUseOfRowtime(): Unit = { val util = streamTestUtil() - util - .addDataStream[(Long, Int, Double, Float, BigDecimal, String)]( - "MyTable", - 'rowtime, - 'int, - 'double, - 'float, - 'bigdec, - 'string) - .select('rowtime.rowtime) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addDataStream[(Long, Int, Double, Float, BigDecimal, String)]( + "MyTable", + 'rowtime, + 'int, + 'double, + 'float, + 'bigdec, + 'string) + .select('rowtime.rowtime)) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidUseOfRowtime2(): Unit = { val util = streamTestUtil() - util - .addDataStream[(Long, Int, Double, Float, BigDecimal, String)]( - "MyTable", - 'rowtime, - 'int, - 'double, - 'float, - 'bigdec, - 'string) - .window(Tumble.over(2.millis).on('rowtime).as('w)) - .groupBy('w) - .select('w.end.rowtime, 'int.count.as('int)) // no rowtime on non-window reference + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addDataStream[(Long, Int, Double, Float, BigDecimal, String)]( + "MyTable", + 'rowtime, + 'int, + 'double, + 'float, + 'bigdec, + 'string) + .window(Tumble.over(2.millis).on('rowtime).as('w)) + .groupBy('w) + // no rowtime on non-window reference + .select('w.end.rowtime, 'int.count.as('int))) } - @Test(expected = classOf[ValidationException]) + @Test def testAddColumnsWithAgg(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.addColumns('a.sum) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.addColumns('a.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testAddOrReplaceColumnsWithAgg(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.addOrReplaceColumns('a.sum) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.addOrReplaceColumns('a.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testRenameColumnsWithAgg(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.renameColumns('a.sum) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.renameColumns('a.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testRenameColumnsWithoutAlias(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.renameColumns('a) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.renameColumns('a)) } - @Test(expected = classOf[ValidationException]) + @Test def testRenameColumnsWithFunctallCall(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.renameColumns(('a + 1).as('a2)) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.renameColumns(('a + 1).as('a2))) } - @Test(expected = classOf[ValidationException]) + @Test def testRenameColumnsNotExist(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.renameColumns('e.as('e2)) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.renameColumns('e.as('e2))) } - @Test(expected = classOf[ValidationException]) + @Test def testDropColumnsWithAgg(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.dropColumns('a.sum) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.dropColumns('a.sum)) } - @Test(expected = classOf[ValidationException]) + @Test def testDropColumnsNotExist(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.dropColumns('e) + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.dropColumns('e)) } - @Test(expected = classOf[ValidationException]) + @Test def testDropColumnsWithValueLiteral(): Unit = { val util = streamTestUtil() val tab = util.addTableSource[(Int, Long, String)]("Table3", 'a, 'b, 'c) - tab.dropColumns("a") + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => tab.dropColumns("a")) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeAggregation(): Unit = { val util = streamTestUtil() - util - .addTableSource[(Int)]("MyTable", 'int) - .map('int.sum) // do not support AggregateFunction as input + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addTableSource[(Int)]("MyTable", 'int) + .map('int.sum)) // do not support AggregateFunction as input } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeUDAGG(): Unit = { val util = streamTestUtil() val weightedAvg = new WeightedAvg - util - .addTableSource[(Int)]("MyTable", 'int) - .map(weightedAvg('int, 'int)) // do not support AggregateFunction as input + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addTableSource[(Int)]("MyTable", 'int) + .map(weightedAvg('int))) // do not support AggregateFunction as input } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeUDAGG2(): Unit = { val util = streamTestUtil() util.addFunction("weightedAvg", new WeightedAvg) - util - .addTableSource[(Int)]("MyTable", 'int) - .map(call("weightedAvg", $"int", $"int")) // do not support AggregateFunction as input + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addTableSource[(Int)]("MyTable", 'int) + .map(call("weightedAvg", $"int", $"int"))) // do not support AggregateFunction as input } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeTableFunction(): Unit = { val util = streamTestUtil() util.addFunction("func", new TableFunc0) - util - .addTableSource[(String)]("MyTable", 'string) - .map(call("func", $"string").as("a")) // do not support TableFunction as input + + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + util + .addTableSource[(String)]("MyTable", 'string) + .map(call("func", $"string").as("a"))) // do not support TableFunction as input } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala index 2d8beeef9c139..8c755f9b87b99 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala @@ -23,8 +23,9 @@ import org.apache.flink.table.planner.expressions.utils._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.planner.utils.{ObjectTableFunction, TableFunc1, TableFunc2, TableTestBase} -import org.junit.Assert.{assertTrue, fail} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.assertj.core.api.ThrowableAssert.ThrowingCallable +import org.junit.jupiter.api.Test class CorrelateValidationTest extends TableTestBase { @@ -84,65 +85,84 @@ class CorrelateValidationTest extends TableTestBase { * Due to the improper translation of TableFunction left outer join (see CALCITE-2004), the join * predicate can only be empty or literal true (the restriction should be removed in FLINK-7865). */ - @Test(expected = classOf[ValidationException]) + @Test def testLeftOuterJoinWithPredicates(): Unit = { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 util.addFunction("func1", function) - val result = table - .leftOuterJoinLateral(function('c).as('s), 'c === 's) - .select('c, 's) - .where('a > 10) - - util.verifyExecPlan(result) + expectExceptionThrown( + { + val result = table + .leftOuterJoinLateral(function('c).as('s), 'c === 's) + .select('c, 's) + .where('a > 10) + util.verifyExecPlan(result) + }, + null) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeAggregation(): Unit = { val util = streamTestUtil() - util - .addTableSource[(Int)]("MyTable", 'int) - .flatMap('int.sum) // do not support AggregateFunction as input + expectExceptionThrown( + util + .addTableSource[(Int)]("MyTable", 'int) + // do not support AggregateFunction as input + .flatMap('int.sum), + null) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeUDAGG(): Unit = { val util = streamTestUtil() - val weightedAvg = new WeightedAvg - util - .addTableSource[(Int)]("MyTable", 'int) - .flatMap(weightedAvg('int, 'int)) // do not support AggregateFunction as input + + expectExceptionThrown( + util + .addTableSource[(Int)]("MyTable", 'int) + // do not support AggregateFunction as input + .flatMap(weightedAvg('int, 'int)), + null) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeUDAGG2(): Unit = { val util = streamTestUtil() util.addFunction("weightedAvg", new WeightedAvg) - util - .addTableSource[(Int)]("MyTable", 'int) - .flatMap(call("weightedAvg", $"int", $"int")) // do not support AggregateFunction as input + + expectExceptionThrown( + util + .addTableSource[(Int)]("MyTable", 'int) + // do not support AggregateFunction as input + .flatMap(call("weightedAvg", $"int", $"int")), + null) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidMapFunctionTypeScalarFunction(): Unit = { val util = streamTestUtil() - util - .addTableSource[(String)]("MyTable", 'string) - .flatMap(Func15('string)) // do not support ScalarFunction as input + expectExceptionThrown( + util + .addTableSource[(String)]("MyTable", 'string) + // do not support ScalarFunction as input + .flatMap(Func15('string)), + null) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidFlatMapFunctionTypeFieldReference(): Unit = { val util = batchTestUtil() - util - .addTableSource[(String)]("MyTable", 'string) - .flatMap('string) // Only TableFunction can be used in flatMap + expectExceptionThrown( + util + .addTableSource[(String)]("MyTable", 'string) + // Only TableFunction can be used in flatMap + .flatMap('string), + null) } // ---------------------------------------------------------------------------------------------- @@ -151,17 +171,14 @@ class CorrelateValidationTest extends TableTestBase { function: => Unit, keywords: String, clazz: Class[_ <: Throwable] = classOf[ValidationException]): Unit = { - try { - function - fail(s"Expected a $clazz, but no exception is thrown.") - } catch { - case e if e.getClass == clazz => - if (keywords != null) { - assertTrue( - s"The exception message '${e.getMessage}' doesn't contain keyword '$keywords'", - e.getMessage.contains(keywords)) - } - case e: Throwable => fail(s"Expected throw ${clazz.getSimpleName}, but is $e.") + val callable: ThrowingCallable = () => function + if (keywords != null) { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) + .withMessageContaining(keywords) + } else { + assertThatExceptionOfType(clazz) + .isThrownBy(callable) } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowTableAggregateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowTableAggregateValidationTest.scala index 9a71c6d9ad9c0..4275b411ec948 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowTableAggregateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowTableAggregateValidationTest.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.{TableTestBase, Top3} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.time.Duration @@ -37,36 +38,35 @@ class GroupWindowTableAggregateValidationTest extends TableTestBase { @Test def testTumbleUdAggWithInvalidArgs(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Invalid function call:\nTop3(BIGINT)") - - table - .window(Slide.over(2.hours).every(30.minutes).on('rowtime).as('w)) - .groupBy('string, 'w) - .flatAggregate(call(top3, 'long)) // invalid args - .select('string, 'f0) + assertThatThrownBy( + () => + table + .window(Slide.over(2.hours).every(30.minutes).on('rowtime).as('w)) + .groupBy('string, 'w) + .flatAggregate(call(top3, 'long)) // invalid args + .select('string, 'f0)) + .hasMessageContaining("Invalid function call:\nTop3(BIGINT)") + .isInstanceOf[ValidationException] } @Test def testInvalidStarInSelection(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Can not use * for window aggregate!") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime) - table - .window(Tumble.over(2.rows).on('proctime).as('w)) - .groupBy('string, 'w) - .flatAggregate(top3('int)) - .select('*) + assertThatThrownBy( + () => + table + .window(Tumble.over(2.rows).on('proctime).as('w)) + .groupBy('string, 'w) + .flatAggregate(top3('int)) + .select('*)) + .hasMessageContaining("Can not use * for window aggregate!") + .isInstanceOf[ValidationException] } @Test def testEmitStrategyNotSupported(): Unit = { - expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Emit strategy has not been supported for Table Aggregate!") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime) @@ -80,6 +80,8 @@ class GroupWindowTableAggregateValidationTest extends TableTestBase { .flatAggregate(top3('int)) .select('string, 'f0, 'w.start) - util.verifyExecPlan(result) + assertThatThrownBy(() => util.verifyExecPlan(result)) + .hasMessageContaining("Emit strategy has not been supported for Table Aggregate!") + .isInstanceOf[TableException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowValidationTest.scala index 929fb5832399b..d163b7bca7103 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowValidationTest.scala @@ -20,275 +20,294 @@ package org.apache.flink.table.planner.plan.stream.table.validation import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.api.{Session, Slide, Tumble, ValidationException} -import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge import org.apache.flink.table.planner.utils.TableTestBase -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test class GroupWindowValidationTest extends TableTestBase { @Test def testInvalidWindowProperty(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Window properties can only be used on windowed tables.") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, String)]('long, 'int, 'string) - table - .groupBy('string) - .select('string, 'string.start) // property in non windowed table + assertThatThrownBy( + () => + table + .groupBy('string) + // property in non windowed table + .select('string, 'string.start)) + .hasMessageContaining("Window properties can only be used on windowed tables.") + .isInstanceOf[ValidationException] } @Test def testGroupByWithoutWindowAlias(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("GroupBy must contain exactly one window alias.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime, 'int, 'string) - table - .window(Tumble.over(5.milli).on('long).as('w)) - .groupBy('string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + .window(Tumble.over(5.milli).on('long).as('w)) + .groupBy('string) + .select('string, 'int.count)) + .hasMessageContaining("GroupBy must contain exactly one window alias.") + .isInstanceOf[ValidationException] } @Test def testInvalidRowTimeRef(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Cannot resolve field [int]") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - .window(Tumble.over(5.milli).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) - .window(Slide.over(5.milli).every(1.milli).on('int).as('w2)) // 'Int does not exist in input. - .groupBy('w2) - .select('string) + assertThatThrownBy( + () => + table + .window(Tumble.over(5.milli).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count) + .window( + Slide.over(5.milli).every(1.milli).on('int).as('w2) + ) // 'Int does not exist in input. + .groupBy('w2) + .select('string)) + .hasMessageContaining("Cannot resolve field [int]") + .isInstanceOf[ValidationException] } @Test def testInvalidTumblingSize(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("A tumble window expects a size value literal") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - .window(Tumble.over($"WRONG").on($"rowtime").as("w")) // string is not a valid interval - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + .window(Tumble.over($"WRONG").on($"rowtime").as("w")) // string is not a valid interval + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining("A tumble window expects a size value literal") + .isInstanceOf[ValidationException] } @Test def testInvalidTumblingSizeType(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Tumbling window expects a size literal of a day-time interval or BIGINT type.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // row interval is not valid for session windows - .window(Tumble.over(10).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // row interval is not valid for session windows + .window(Tumble.over(10).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining( + "Tumbling window expects a size literal of a day-time interval or BIGINT type.") + .isInstanceOf[ValidationException] } @Test def testTumbleUdAggWithInvalidArgs(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Invalid function call:\nmyWeightedAvg(STRING, INT)") - val util = streamTestUtil() val weightedAvg = new WeightedAvgWithMerge val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - .window(Tumble.over(2.hours).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, call(weightedAvg, 'string, 'int)) // invalid UDAGG args + assertThatThrownBy( + () => + table + .window(Tumble.over(2.hours).on('rowtime).as('w)) + .groupBy('w, 'string) + // invalid UDAGG args + .select('string, call(weightedAvg, 'string, 'int))) + .hasMessageContaining("Invalid function call:\nmyWeightedAvg(STRING, INT)") + .isInstanceOf[ValidationException] } @Test def testInvalidSlidingSize(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("A sliding window expects a size value literal") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // field reference is not a valid interval - .window(Slide.over($"WRONG").every($"WRONG").on($"rowtime").as("w")) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // field reference is not a valid interval + .window(Slide.over($"WRONG").every($"WRONG").on($"rowtime").as("w")) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining("A sliding window expects a size value literal") + .isInstanceOf[ValidationException] } @Test def testInvalidSlidingSlide(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("A sliding window expects the same type of size and slide.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // row and time intervals may not be mixed - .window(Slide.over(12.rows).every(1.minute).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // row and time intervals may not be mixed + .window(Slide.over(12.rows).every(1.minute).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining("A sliding window expects the same type of size and slide.") + .isInstanceOf[ValidationException] } @Test def testInvalidSlidingSizeType(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "A sliding window expects a size literal of a day-time interval or BIGINT type.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // row interval is not valid for session windows - .window(Slide.over(10).every(10.milli).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // row interval is not valid for session windows + .window(Slide.over(10).every(10.milli).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining( + "A sliding window expects a size literal of a day-time interval or BIGINT type.") + .isInstanceOf[ValidationException] } @Test def testSlideUdAggWithInvalidArgs(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Invalid function call:\nmyWeightedAvg(STRING, INT)") - val util = streamTestUtil() val weightedAvg = new WeightedAvgWithMerge val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - .window(Slide.over(2.hours).every(30.minutes).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, call(weightedAvg, 'string, 'int)) // invalid UDAGG args + assertThatThrownBy( + () => + table + .window(Slide.over(2.hours).every(30.minutes).on('rowtime).as('w)) + .groupBy('w, 'string) + // invalid UDAGG args + .select('string, call(weightedAvg, 'string, 'int))) + .hasMessageContaining("Invalid function call:\nmyWeightedAvg(STRING, INT)") + .isInstanceOf[ValidationException] } @Test def testInvalidSessionGap(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "A session window expects a gap literal of a day-time interval type.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // row interval is not valid for session windows - .window(Session.withGap(10.rows).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // row interval is not valid for session windows + .window(Session.withGap(10.rows).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining("A session window expects a gap literal of a day-time interval type.") + .isInstanceOf[ValidationException] } @Test def testInvalidSessionGapType(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "A session window expects a gap literal of a day-time interval type.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // row interval is not valid for session windows - .window(Session.withGap(10).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // row interval is not valid for session windows + .window(Session.withGap(10).on('rowtime).as('w)) + .groupBy('w, 'string) + .select('string, 'int.count)) + .hasMessageContaining("A session window expects a gap literal of a day-time interval type.") + .isInstanceOf[ValidationException] } @Test def testInvalidWindowAlias1(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Only unresolved reference supported for alias of a " + - "group window.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime, 'int, 'string) - table - // expression instead of a symbol - .window(Session.withGap(100.milli).on('long).as(concat("A", "B"))) - .groupBy(concat("A", "B")) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // expression instead of a symbol + .window(Session.withGap(100.milli).on('long).as(concat("A", "B"))) + .groupBy(concat("A", "B")) + .select('string, 'int.count)) + .hasMessageContaining("Only unresolved reference supported for alias of a group window.") + .isInstanceOf[ValidationException] } @Test def testInvalidWindowAlias2(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Cannot resolve field [string]") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'rowtime.rowtime, 'int, 'string) - table - // field name "string" is already present - .window(Session.withGap(100.milli).on('rowtime).as('string)) - .groupBy('string) - .select('string, 'int.count) + assertThatThrownBy( + () => + table + // field name "string" is already present + .window(Session.withGap(100.milli).on('rowtime).as('string)) + .groupBy('string) + .select('string, 'int.count)) + .hasMessageContaining("Cannot resolve field [string]") + .isInstanceOf[ValidationException] } @Test def testSessionUdAggWithInvalidArgs(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Invalid function call:\nmyWeightedAvg(STRING, INT)") - val util = streamTestUtil() val weightedAvg = new WeightedAvgWithMerge val table = util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'string, 'rowtime.rowtime) - table - .window(Session.withGap(2.hours).on('rowtime).as('w)) - .groupBy('w, 'string) - .select('string, call(weightedAvg, 'string, 'int)) // invalid UDAGG args + assertThatThrownBy( + () => + table + .window(Session.withGap(2.hours).on('rowtime).as('w)) + .groupBy('w, 'string) + // invalid UDAGG args + .select('string, call(weightedAvg, 'string, 'int))) + .hasMessageContaining("Invalid function call:\nmyWeightedAvg(STRING, INT)") + .isInstanceOf[ValidationException] } @Test def testInvalidWindowPropertyOnRowCountsTumblingWindow(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Window start and Window end cannot be selected " + - "for a row-count tumble window.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'string, 'proctime.proctime) - table - .window(Tumble.over(2.rows).on('proctime).as('w)) - .groupBy('w, 'string) - .select('string, 'w.start, 'w.end) // invalid start/end on rows-count window + assertThatThrownBy( + () => + table + .window(Tumble.over(2.rows).on('proctime).as('w)) + .groupBy('w, 'string) + // invalid start/end on rows-count window + .select('string, 'w.start, 'w.end)) + .hasMessageContaining("Window start and Window end cannot be selected " + + "for a row-count tumble window.") + .isInstanceOf[ValidationException] } @Test def testInvalidWindowPropertyOnRowCountsSlidingWindow(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Window start and Window end cannot be selected for a " + - "row-count slide window.") - val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'string, 'proctime.proctime) - table - .window(Slide.over(10.rows).every(5.rows).on('proctime).as('w)) - .groupBy('w, 'string) - .select('string, 'w.start, 'w.end) // invalid start/end on rows-count window + assertThatThrownBy( + () => + table + .window(Slide.over(10.rows).every(5.rows).on('proctime).as('w)) + .groupBy('w, 'string) + // invalid start/end on rows-count window + .select('string, 'w.start, 'w.end)) + .hasMessageContaining("Window start and Window end cannot be selected for a " + + "row-count slide window.") + .isInstanceOf[ValidationException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/LegacyTableSinkValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/LegacyTableSinkValidationTest.scala index f92823021f0df..30f0edab0af3e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/LegacyTableSinkValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/LegacyTableSinkValidationTest.scala @@ -26,27 +26,31 @@ import org.apache.flink.table.planner.runtime.utils.{TestData, TestingAppendSink import org.apache.flink.table.planner.utils.{MemoryTableSourceSinkUtil, TableTestBase, TableTestUtil} import org.apache.flink.types.Row -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class LegacyTableSinkValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testAppendSinkOnUpdatingTable(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) val t = env.fromCollection(TestData.smallTupleData3).toTable(tEnv, 'a, 'b, 'c) - t.groupBy('text) - .select('text, 'id.count, 'num.sum) - .toAppendStream[Row] - .addSink(new TestingAppendSink) - // must fail because table is not append-only - env.execute() + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + t.groupBy('text) + .select('text, 'id.count, 'num.sum) + .toAppendStream[Row] + .addSink(new TestingAppendSink) + env.execute() + }) } - @Test(expected = classOf[TableException]) + @Test def testUpsertSinkOnUpdatingTableWithoutFullKey(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -64,12 +68,17 @@ class LegacyTableSinkValidationTest extends TableTestBase { .select('len, 'id.count, 'num.sum) val schema = result.getSchema sink.configure(schema.getFieldNames, schema.getFieldTypes) - tEnv.asInstanceOf[TableEnvironmentInternal].registerTableSinkInternal("testSink", sink) + // must fail because table is updating table without full key - result.executeInsert("testSink") + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy( + () => { + tEnv.asInstanceOf[TableEnvironmentInternal].registerTableSinkInternal("testSink", sink) + result.executeInsert("testSink") + }) } - @Test(expected = classOf[TableException]) + @Test def testAppendSinkOnLeftJoin(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -77,26 +86,21 @@ class LegacyTableSinkValidationTest extends TableTestBase { val ds1 = env.fromCollection(TestData.tupleData3).toTable(tEnv, 'a, 'b, 'c) val ds2 = env.fromCollection(TestData.tupleData5).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) - ds1 - .leftOuterJoin(ds2, 'a === 'd && 'b === 'h) - .select('c, 'g) - .toAppendStream[Row] - .addSink(new TestingAppendSink) - // must fail because table is not append-only - env.execute() + assertThatExceptionOfType(classOf[TableException]) + .isThrownBy( + () => { + ds1 + .leftOuterJoin(ds2, 'a === 'd && 'b === 'h) + .select('c, 'g) + .toAppendStream[Row] + .addSink(new TestingAppendSink) + env.execute() + }) } @Test def testValidateSink(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Column types of query result and sink for " + - "'default_catalog.default_database.testSink' do not match.\n" + - "Cause: Incompatible types for sink column 'd' at position 3.\n\n" + - "Query schema: [a: INT, b: BIGINT, c: STRING, d: BIGINT]\n" + - "Sink schema: [a: INT, b: BIGINT, c: STRING, d: INT]") - val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -114,7 +118,14 @@ class LegacyTableSinkValidationTest extends TableTestBase { MemoryTableSourceSinkUtil.createDataTypeOutputFormatTable(tEnv, sinkSchema, "testSink") // must fail because query result table schema is different with sink table schema - resultTable.executeInsert("testSink").await() + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => resultTable.executeInsert("testSink").await()) + .withMessageContaining( + "Column types of query result and sink for " + + "'default_catalog.default_database.testSink' do not match.\n" + + "Cause: Incompatible types for sink column 'd' at position 3.\n\n" + + "Query schema: [a: INT, b: BIGINT, c: STRING, d: BIGINT]\n" + + "Sink schema: [a: INT, b: BIGINT, c: STRING, d: INT]") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala index 48bfa60ce6494..26cc005fdc888 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala @@ -26,7 +26,8 @@ import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions. import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase, TableTestUtil} import org.apache.calcite.rel.RelNode -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.junit.jupiter.api.Test class OverWindowValidationTest extends TableTestBase { private val streamUtil: StreamTableTestUtil = streamTestUtil() @@ -34,113 +35,161 @@ class OverWindowValidationTest extends TableTestBase { .addDataStream[(Int, String, Long)]("MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime) /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation(): Unit = { val util = streamTestUtil() val table = util.addDataStream[(Long, Int, String)]("T1", 'a, 'b, 'c) val overAgg = new OverAgg0 - table.select(overAgg('a, 'b)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => table.select(overAgg('a, 'b))) } /** OVER clause is necessary for [[OverAgg0]] window function. */ - @Test(expected = classOf[ValidationException]) + @Test def testInvalidOverAggregation2(): Unit = { val util = streamTestUtil() - val table = util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'string, 'proctime) - val overAgg = new OverAgg0 - table - .window(Tumble.over(2.rows).on('proctime).as('w)) - .groupBy('w, 'string) - .select(overAgg('long, 'int)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val table = util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'string, 'proctime) + val overAgg = new OverAgg0 + table + .window(Tumble.over(2.rows).on('proctime).as('w)) + .groupBy('w, 'string) + .select(overAgg('long, 'int)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testInvalidWindowAlias(): Unit = { - val result = table - .window(Over.partitionBy('c).orderBy('rowtime).preceding(2.rows).as('w)) - .select('c, 'b.count.over('x)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.partitionBy('c).orderBy('rowtime).preceding(2.rows).as('w)) + .select('c, 'b.count.over('x)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testOrderBy(): Unit = { - val result = table - .window(Over.partitionBy('c).orderBy('abc).preceding(2.rows).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.partitionBy('c).orderBy('abc).preceding(2.rows).as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testPrecedingAndFollowingUsingIsLiteral(): Unit = { - val result = table - .window(Over.partitionBy($"c").orderBy($"rowtime").preceding(2).following($"xx").as($"w")) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window( + Over.partitionBy($"c").orderBy($"rowtime").preceding(2).following($"xx").as($"w")) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testPrecedingAndFollowingUsingSameType(): Unit = { - val result = table - .window( - Over.partitionBy('c).orderBy('rowtime).preceding(2.rows).following(CURRENT_RANGE).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window( + Over + .partitionBy('c) + .orderBy('rowtime) + .preceding(2.rows) + .following(CURRENT_RANGE) + .as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testPartitionByWithUnresolved(): Unit = { - val result = table - .window(Over.partitionBy('a + 'b).orderBy('rowtime).preceding(2.rows).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.partitionBy('a + 'b).orderBy('rowtime).preceding(2.rows).as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testPartitionByWithNotKeyType(): Unit = { val table2 = streamUtil.addTableSource[(Int, String, Either[Long, String])]("MyTable2", 'a, 'b, 'c) - val result = table2 - .window(Over.partitionBy('c).orderBy('rowtime).preceding(2.rows).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table2 + .window(Over.partitionBy('c).orderBy('rowtime).preceding(2.rows).as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testPrecedingValue(): Unit = { - val result = table - .window(Over.orderBy('rowtime).preceding(-1.rows).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.orderBy('rowtime).preceding(-1.rows).as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testFollowingValue(): Unit = { - val result = table - .window(Over.orderBy('rowtime).preceding(1.rows).following(-2.rows).as('w)) - .select('c, 'b.count.over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.orderBy('rowtime).preceding(1.rows).following(-2.rows).as('w)) + .select('c, 'b.count.over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } - @Test(expected = classOf[ValidationException]) + @Test def testUdAggWithInvalidArgs(): Unit = { val weightedAvg = new WeightedAvgWithRetract - val result = table - .window(Over.orderBy('rowtime).preceding(1.minutes).as('w)) - .select('c, weightedAvg('b, 'a).over('w)) - optimize(TableTestUtil.toRelNode(result)) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val result = table + .window(Over.orderBy('rowtime).preceding(1.minutes).as('w)) + .select('c, weightedAvg('b, 'a).over('w)) + optimize(TableTestUtil.toRelNode(result)) + }) } @Test def testAccessesWindowProperties(): Unit = { - thrown.expect(classOf[ValidationException]) - thrown.expectMessage("Window start and end properties are not available for Over windows.") - - table - .window(Over.orderBy('rowtime).preceding(1.minutes).as('w)) - .select('c, 'a.count.over('w), 'w.start + 1, 'w.end) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => + table + .window(Over.orderBy('rowtime).preceding(1.minutes).as('w)) + .select('c, 'a.count.over('w), 'w.start + 1, 'w.end)) + .withMessageContaining("Window start and end properties are not available for Over windows.") } private def optimize(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/SetOperatorsValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/SetOperatorsValidationTest.scala index bc4ea4c34f92d..9588ac608b336 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/SetOperatorsValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/SetOperatorsValidationTest.scala @@ -25,12 +25,12 @@ import org.apache.flink.table.planner.runtime.utils.{TestData, TestingAppendSink import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} import org.apache.flink.types.Row -import org.junit.Assert.assertEquals -import org.junit.Test +import org.assertj.core.api.Assertions.{assertThat, assertThatExceptionOfType} +import org.junit.jupiter.api.Test class SetOperatorsValidationTest extends TableTestBase { - @Test(expected = classOf[ValidationException]) + @Test def testUnionFieldsNameNotOverlap1(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -38,16 +38,19 @@ class SetOperatorsValidationTest extends TableTestBase { val ds1 = env.fromCollection(TestData.smallTupleData3).toTable(tEnv, 'a, 'b, 'c) val ds2 = env.fromCollection(TestData.tupleData5).toTable(tEnv, 'a, 'b, 'd, 'c, 'e) - val unionDs = ds1.unionAll(ds2) - val sink = new TestingAppendSink - unionDs.toAppendStream[Row].addSink(sink) - env.execute() - assertEquals(true, sink.getAppendResults.isEmpty) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val unionDs = ds1.unionAll(ds2) + unionDs.toAppendStream[Row].addSink(sink) + env.execute() + }) + assertThat(sink.getAppendResults.isEmpty).isTrue } - @Test(expected = classOf[ValidationException]) + @Test def testUnionFieldsNameNotOverlap2(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -58,15 +61,19 @@ class SetOperatorsValidationTest extends TableTestBase { .toTable(tEnv, 'a, 'b, 'c, 'd, 'e) .select('a, 'b, 'c) - val unionDs = ds1.unionAll(ds2) val sink = new TestingAppendSink - unionDs.toAppendStream[Row].addSink(sink) - env.execute() - assertEquals(true, sink.getAppendResults.isEmpty) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => { + val unionDs = ds1.unionAll(ds2) + unionDs.toAppendStream[Row].addSink(sink) + env.execute() + }) + assertThat(sink.getAppendResults.isEmpty).isTrue } - @Test(expected = classOf[ValidationException]) + @Test def testUnionTablesFromDifferentEnv(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv1 = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) @@ -76,6 +83,7 @@ class SetOperatorsValidationTest extends TableTestBase { val ds2 = env.fromCollection(TestData.smallTupleData3).toTable(tEnv2, 'a, 'b, 'c) // Must fail. Tables are bound to different TableEnvironments. - ds1.unionAll(ds2) + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => ds1.unionAll(ds2)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableAggregateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableAggregateValidationTest.scala index 7e67a62b5a4bb..a30aedefb1456 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableAggregateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableAggregateValidationTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{EmptyTableAggFunc, TableTestBase} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -29,118 +30,130 @@ class TableAggregateValidationTest extends TableTestBase { @Test def testInvalidParameterNumber(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Invalid function call:\nEmptyTableAggFunc(BIGINT, INT, STRING)") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('c) - // must fail. func does not take 3 parameters - .flatAggregate(call(func, 'a, 'b, 'c)) - .select('_1, '_2, '_3) + + assertThatThrownBy( + () => + table + .groupBy('c) + // must fail. func does not take 3 parameters + .flatAggregate(call(func, 'a, 'b, 'c)) + .select('_1, '_2, '_3)) + .hasMessageContaining("Invalid function call:\nEmptyTableAggFunc(BIGINT, INT, STRING)") + .isInstanceOf[ValidationException] } @Test def testInvalidParameterType(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Invalid function call:\nEmptyTableAggFunc(BIGINT, STRING)") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, String)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('c) - // must fail. func take 2 parameters of type Long and Timestamp or Long Int - .flatAggregate(call(func, 'a, 'c)) - .select('_1, '_2, '_3) + + assertThatThrownBy( + () => + table + .groupBy('c) + // must fail. func take 2 parameters of type Long and Int + .flatAggregate(call(func, 'a, 'c)) + .select('_1, '_2, '_3)) + .hasMessageContaining("Invalid function call:\nEmptyTableAggFunc(BIGINT, STRING)") + .isInstanceOf[ValidationException] } @Test def testInvalidWithWindowProperties(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Window properties can only be used on windowed tables.") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, Timestamp)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('b) - .flatAggregate(call(func, 'a, 'b).as('x, 'y)) - .select('x.start, 'y) + + assertThatThrownBy( + () => + table + .groupBy('b) + .flatAggregate(call(func, 'a, 'b).as('x, 'y)) + .select('x.start, 'y)) + .hasMessageContaining("Window properties can only be used on windowed tables.") + .isInstanceOf[ValidationException] } @Test def testInvalidWithAggregation(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Aggregate functions are not supported in the " + - "select right after the aggregate or flatAggregate operation.") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, Timestamp)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('b) - .flatAggregate(call(func, 'a, 'b).as('x, 'y)) - .select('x.count) + + assertThatThrownBy( + () => + table + .groupBy('b) + .flatAggregate(call(func, 'a, 'b).as('x, 'y)) + .select('x.count)) + .hasMessageContaining("Aggregate functions are not supported in the " + + "select right after the aggregate or flatAggregate operation.") + .isInstanceOf[ValidationException] } @Test def testInvalidParameterWithAgg(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "It's not allowed to use an aggregate function as input of another aggregate function") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, Timestamp)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('b) - // must fail. func take agg function as input - .flatAggregate(func('a.sum, 'c)) - .select('_1, '_2, '_3) + + assertThatThrownBy( + () => + table + .groupBy('b) + // must fail. func take agg function as input + .flatAggregate(func('a.sum, 'c)) + .select('_1, '_2, '_3)) + .hasMessageContaining( + "It's not allowed to use an aggregate function as input of another aggregate function") + .isInstanceOf[ValidationException] } @Test def testInvalidAliasWithWrongNumber(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "List of column aliases must have same degree as " + - "table; the returned table of function 'EmptyTableAggFunc' has 2 columns, " + - "whereas alias list has 3 columns") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, Timestamp)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('b) - // must fail. alias with wrong number of fields - .flatAggregate(call(func, 'a, 'b).as('a, 'b, 'c)) - .select('*) + + assertThatThrownBy( + () => + table + .groupBy('b) + // must fail. alias with wrong number of fields + .flatAggregate(call(func, 'a, 'b).as('a, 'b, 'c)) + .select('*)) + .hasMessageContaining( + "List of column aliases must have same degree as " + + "table; the returned table of function 'EmptyTableAggFunc' has 2 columns, " + + "whereas alias list has 3 columns") + .isInstanceOf[ValidationException] } @Test def testAliasWithNameConflict(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Ambiguous column name: b") - val util = streamTestUtil() val table = util.addTableSource[(Long, Int, Timestamp)]('a, 'b, 'c) val func = new EmptyTableAggFunc - table - .groupBy('b) - // must fail. alias with name conflict - .flatAggregate(call(func, 'a, 'b).as('a, 'b)) - .select('*) + + assertThatThrownBy( + () => + table + .groupBy('b) + // must fail. alias with name conflict + .flatAggregate(call(func, 'a, 'b).as('a, 'b)) + .select('*)) + .hasMessageContaining("Ambiguous column name: b") + .isInstanceOf[ValidationException] } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala index 367b6fe42acf6..407d4914ad368 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api._ import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} -import org.junit.Test +import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.jupiter.api.Test import java.sql.Timestamp @@ -52,26 +53,20 @@ class TemporalTableJoinValidationTest extends TableTestBase { @Test def testInvalidFieldReference(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Cannot resolve field [foobar]") - - ratesHistory.createTemporalTableFunction('rowtime, 'foobar) + assertThatThrownBy(() => ratesHistory.createTemporalTableFunction('rowtime, 'foobar)) + .hasMessageContaining("Cannot resolve field [foobar]") + .isInstanceOf[ValidationException] } @Test def testInvalidStringFieldReference(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("Cannot resolve field [foobar]") - - ratesHistory.createTemporalTableFunction($"rowtime", $"foobar") + assertThatThrownBy(() => ratesHistory.createTemporalTableFunction($"rowtime", $"foobar")) + .hasMessageContaining("Cannot resolve field [foobar]") + .isInstanceOf[ValidationException] } @Test def testNonTimeIndicatorOnRightSide(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Non rowtime timeAttribute [TIMESTAMP(3)] used to create TemporalTableFunction") - val rates = ratesHistoryWithoutTimeAttribute.createTemporalTableFunction('rowtime, 'currency) val result = orders @@ -79,15 +74,14 @@ class TemporalTableJoinValidationTest extends TableTestBase { .select($"o_amount" * $"rate") .as("rate") - util.verifyExplain(result) + assertThatThrownBy(() => util.verifyExplain(result)) + .hasMessageContaining( + "Non rowtime timeAttribute [TIMESTAMP(3)] used to create TemporalTableFunction") + .isInstanceOf[ValidationException] } @Test def testNonTimeIndicatorOnLeftSide(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Non rowtime timeAttribute [TIMESTAMP(3)] passed as the argument to TemporalTableFunction") - val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency) val result = ordersWithoutTimeAttribute @@ -95,16 +89,14 @@ class TemporalTableJoinValidationTest extends TableTestBase { .select($"o_amount" * $"rate") .as("rate") - util.verifyExplain(result) + assertThatThrownBy(() => util.verifyExplain(result)) + .hasMessageContaining( + "Non rowtime timeAttribute [TIMESTAMP(3)] passed as the argument to TemporalTableFunction") + .isInstanceOf[ValidationException] } @Test def testMixedTimeIndicators(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage( - "Non rowtime timeAttribute [TIMESTAMP_LTZ(3) *PROCTIME*] passed as the argument " + - "to TemporalTableFunction") - val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency) val result = ordersProctime @@ -112,6 +104,10 @@ class TemporalTableJoinValidationTest extends TableTestBase { .select($"o_amount" * $"rate") .as("rate") - util.verifyExplain(result) + assertThatThrownBy(() => util.verifyExplain(result)) + .hasMessageContaining( + "Non rowtime timeAttribute [TIMESTAMP_LTZ(3) *PROCTIME*] passed as the argument " + + "to TemporalTableFunction") + .isInstanceOf[ValidationException] } }