From 40dcf69019adc455b097b20c1ce58ebb7195003d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 14 Jun 2017 16:53:49 +0200 Subject: [PATCH 1/2] [FLINK-6921] [serializer] Allow EnumValueSerializer to deal with appended enum values The problem was that we don't check the bounds of the array with the enum names contained in the ScalaEnumSerializerConfigSnapshot. This PR also adds an Enumeration upgrade test which makes sure that appended fields are supported without migration. Moreover, it checks that a field removal and an order change leads to a required migration. This closes #4126. --- flink-scala/pom.xml | 10 + .../scala/typeutils/EnumValueSerializer.scala | 13 +- .../typeutils/EnumValueSerializerTest.scala | 46 ++++ .../EnumValueSerializerUpgradeTest.scala | 199 ++++++++++++++++++ 4 files changed, 263 insertions(+), 5 deletions(-) create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index 1c4d7c42d2009..2df526e77e2a9 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -107,6 +107,16 @@ under the License. japicmp-maven-plugin + + org.apache.maven.plugins + maven-surefire-plugin + + + false + + + net.alchim31.maven diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala index 119db9381a06f..8be5e0b3b955f 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala @@ -92,11 +92,14 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ if (previousEnumConstants != null) { for (i <- enum.values.iterator) { - if (!previousEnumConstants(i.id).equals(i.toString)) { - // compatible only if new enum constants are only appended, - // and original constants must be in the exact same order - - return CompatibilityResult.requiresMigration() + // skip the check for all newly added fields + if (i.id < previousEnumConstants.length) { + if (!previousEnumConstants(i.id).equals(i.toString)) { + // compatible only if new enum constants are only appended, + // and original constants must be in the exact same order + + return CompatibilityResult.requiresMigration() + } } } } diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala new file mode 100644 index 0000000000000..500927158ae68 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala @@ -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.api.scala.typeutils + +import org.apache.flink.util.TestLogger +import org.junit.Test +import org.junit.Assert._ +import org.scalatest.junit.JUnitSuiteLike + +class EnumValueSerializerTest extends TestLogger with JUnitSuiteLike { + + /** + * Tests that the snapshot configuration can be created and that the serializer + * is compatible when being called with the created serializer snapshot + * + * FLINK-6914 + */ + @Test + def testEnumValueSerializerEnsureCompatibilityIdempotency() { + val enumSerializer = new EnumValueSerializer(Letters) + + val snapshot = enumSerializer.snapshotConfiguration() + + assertFalse(enumSerializer.ensureCompatibility(snapshot).isRequiresMigration) + } +} + +object Letters extends Enumeration { + val A, B, C = Value +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala new file mode 100644 index 0000000000000..c470cd00c0d2d --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.api.scala.typeutils + +import java.io._ +import java.net.{URL, URLClassLoader} + +import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializerSerializationUtil} +import org.apache.flink.core.memory.{DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.util.TestLogger +import org.junit.rules.TemporaryFolder +import org.junit.{Rule, Test} +import org.junit.Assert._ +import org.scalatest.junit.JUnitSuiteLike + +import scala.reflect.NameTransformer +import scala.tools.nsc.reporters.ConsoleReporter +import scala.tools.nsc.{GenericRunnerSettings, Global} + +class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike { + + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder = _tempFolder + + val enumName = "EnumValueSerializerUpgradeTestEnum" + + val enumA = + s""" + |@SerialVersionUID(1L) + |object $enumName extends Enumeration { + | val A, B, C = Value + |} + """.stripMargin + + val enumB = + s""" + |@SerialVersionUID(1L) + |object $enumName extends Enumeration { + | val A, B, C, D = Value + |} + """.stripMargin + + val enumC = + s""" + |@SerialVersionUID(1L) + |object $enumName extends Enumeration { + | val A, C = Value + |} + """.stripMargin + + val enumD = + s""" + |@SerialVersionUID(1L) + |object $enumName extends Enumeration { + | val A, C, B = Value + |} + """.stripMargin + + /** + * Check that identical enums don't require migration + */ + @Test + def checkIdenticalEnums(): Unit = { + assertFalse(checkCompatibility(enumA, enumA).isRequiresMigration) + } + + /** + * Check that appending fields to the enum does not require migration + */ + @Test + def checkAppendedField(): Unit = { + assertFalse(checkCompatibility(enumA, enumB).isRequiresMigration) + } + + /** + * Check that removing enum fields requires migration + */ + @Test + def checkRemovedField(): Unit = { + assertTrue(checkCompatibility(enumA, enumC).isRequiresMigration) + } + + /** + * Check that changing the enum field order requires migration + */ + @Test + def checkDifferentFieldOrder(): Unit = { + assertTrue(checkCompatibility(enumA, enumD).isRequiresMigration) + } + + def checkCompatibility(enumSourceA: String, enumSourceB: String) + : CompatibilityResult[Enumeration#Value] = { + import EnumValueSerializerUpgradeTest._ + + val classLoader = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceA) + + val enum = instantiateEnum[Enumeration](classLoader, enumName) + + val enumValueSerializer = new EnumValueSerializer(enum) + val snapshot = enumValueSerializer.snapshotConfiguration() + + val baos = new ByteArrayOutputStream() + val output = new DataOutputViewStreamWrapper(baos) + TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(output, snapshot) + + output.close() + baos.close() + + val bais = new ByteArrayInputStream(baos.toByteArray) + val input= new DataInputViewStreamWrapper(bais) + + val classLoader2 = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceB) + + val snapshot2 = TypeSerializerSerializationUtil.readSerializerConfigSnapshot( + input, + classLoader2) + val enum2 = instantiateEnum[Enumeration](classLoader2, enumName) + + val enumValueSerializer2 = new EnumValueSerializer(enum2) + enumValueSerializer2.ensureCompatibility(snapshot2) + } +} + +object EnumValueSerializerUpgradeTest { + def compileAndLoadEnum(root: File, filename: String, source: String): ClassLoader = { + val file = writeSourceFile(root, filename, source) + + compileScalaFile(file) + + new URLClassLoader( + Array[URL](root.toURI.toURL), + Thread.currentThread().getContextClassLoader) + } + + def instantiateEnum[T <: Enumeration](classLoader: ClassLoader, enumName: String): T = { + val clazz = classLoader.loadClass(enumName + "$").asInstanceOf[Class[_ <: Enumeration]] + val field = clazz.getField(NameTransformer.MODULE_INSTANCE_NAME) + + field.get(null).asInstanceOf[T] + } + + def writeSourceFile(root: File, filename: String, source: String): File = { + val file = new File(root, filename) + val fileWriter = new FileWriter(file) + + fileWriter.write(source) + + fileWriter.close() + + file + } + + def compileScalaFile(file: File): Unit = { + val in = new BufferedReader(new StringReader("")) + val out = new PrintWriter(new BufferedWriter( + new OutputStreamWriter(System.out))) + + val settings = new GenericRunnerSettings(out.println _) + + val classLoader = Thread.currentThread().getContextClassLoader + + val urls = classLoader match { + case urlClassLoader: URLClassLoader => + urlClassLoader.getURLs + case x => throw new IllegalStateException(s"Not possible to extract URLs " + + s"from class loader $x.") + } + + settings.classpath.value = urls.map(_.toString).mkString(java.io.File.pathSeparator) + settings.outdir.value = file.getParent + + val reporter = new ConsoleReporter(settings) + val global = new Global(settings, reporter) + val run = new global.Run + + run.compile(List(file.getAbsolutePath)) + + reporter.printSummary() + } +} + From c76edfc46307b395197664208b602c90ed9c8ebc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 19 Jun 2017 12:49:05 +0200 Subject: [PATCH 2/2] [FLINK-6948] Harden EnumValueSerializer to detect changed enum indices This PR changes the seriailization format of the ScalaEnumSerializerConfigSnapshot to also include the ordinal value of an enum value when being deserialized. This allows to detect if the ordinal values have been changed and, thus, if migration is required. IMPORTANT: This PR changes the serialization format of ScalaEnumSerializerConfigSnapshot. Remove backwards compatibility path for 1.3.1 --- .../scala/typeutils/EnumValueSerializer.scala | 65 +++++++++++++----- .../src/test/resources/log4j-test.properties | 31 +++++++++ .../EnumValueSerializerUpgradeTest.scala | 20 ++++++ .../_metadata | Bin 218895 -> 217183 bytes .../_metadata | Bin 218895 -> 217183 bytes .../_metadata | Bin 218119 -> 217183 bytes .../_metadata | Bin 218119 -> 217183 bytes 7 files changed, 99 insertions(+), 17 deletions(-) create mode 100644 flink-scala/src/test/resources/log4j-test.properties diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala index 8be5e0b3b955f..344b56d027e83 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala @@ -26,6 +26,8 @@ import org.apache.flink.api.java.typeutils.runtime.{DataInputViewStream, DataOut import org.apache.flink.core.memory.{DataInputView, DataOutputView} import org.apache.flink.util.{InstantiationUtil, Preconditions} +import scala.collection.mutable.ListBuffer + /** * Serializer for [[Enumeration]] values. */ @@ -88,18 +90,23 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ case enumSerializerConfigSnapshot: EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[_] => val enumClass = enum.getClass.asInstanceOf[Class[E]] if (enumClass.equals(enumSerializerConfigSnapshot.getEnumClass)) { - val previousEnumConstants = enumSerializerConfigSnapshot.getEnumConstants + val previousEnumConstants:List[(String, Int)] = + enumSerializerConfigSnapshot.getEnumConstants if (previousEnumConstants != null) { - for (i <- enum.values.iterator) { - // skip the check for all newly added fields - if (i.id < previousEnumConstants.length) { - if (!previousEnumConstants(i.id).equals(i.toString)) { - // compatible only if new enum constants are only appended, - // and original constants must be in the exact same order - + for ((previousEnumConstant, idx) <- previousEnumConstants) { + val enumValue = try { + enum(idx) + } catch { + case _: NoSuchElementException => + // couldn't find an enum value for the given index return CompatibilityResult.requiresMigration() - } + } + + if (!previousEnumConstant.equals(enumValue.toString)) { + // compatible only if new enum constants are only appended, + // and original constants must be in the exact same order + return CompatibilityResult.requiresMigration() } } } @@ -120,12 +127,12 @@ object EnumValueSerializer { extends TypeSerializerConfigSnapshot { var enumClass: Class[E] = _ - var enumConstants: List[String] = _ + var enumConstants: List[(String, Int)] = _ def this(enum: E) = { this() this.enumClass = Preconditions.checkNotNull(enum).getClass.asInstanceOf[Class[E]] - this.enumConstants = enum.values.toList.map(_.toString) + this.enumConstants = enum.values.toList.map(x => (x.toString, x.id)) } override def write(out: DataOutputView): Unit = { @@ -135,7 +142,12 @@ object EnumValueSerializer { val outViewWrapper = new DataOutputViewStream(out) try { InstantiationUtil.serializeObject(outViewWrapper, enumClass) - InstantiationUtil.serializeObject(outViewWrapper, enumConstants) + + out.writeInt(enumConstants.length) + for ((name, idx) <- enumConstants) { + out.writeUTF(name) + out.writeInt(idx) + } } finally if (outViewWrapper != null) outViewWrapper.close() } } @@ -150,8 +162,24 @@ object EnumValueSerializer { enumClass = InstantiationUtil.deserializeObject( inViewWrapper, getUserCodeClassLoader) - enumConstants = InstantiationUtil.deserializeObject( - inViewWrapper, getUserCodeClassLoader) + if (getReadVersion == 1) { + // read null from input stream + InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader) + enumConstants = List() + } else if (getReadVersion == 2) { + val length = in.readInt() + val listBuffer = ListBuffer[(String, Int)]() + + for (_ <- 0 until length) { + val name = in.readUTF() + val idx = in.readInt() + listBuffer += ((name, idx)) + } + + enumConstants = listBuffer.toList + } else { + throw new IOException(s"Cannot deserialize ${getClass.getSimpleName} with version $getReadVersion.") + } } catch { case e: ClassNotFoundException => throw new IOException("The requested enum class cannot be found in classpath.", e) @@ -164,7 +192,7 @@ object EnumValueSerializer { def getEnumClass: Class[E] = enumClass - def getEnumConstants: List[String] = enumConstants + def getEnumConstants: List[(String, Int)] = enumConstants override def equals(obj: scala.Any): Boolean = { if (obj == this) { @@ -184,10 +212,13 @@ object EnumValueSerializer { override def hashCode(): Int = { enumClass.hashCode() * 31 + enumConstants.hashCode() } + + override def getCompatibleVersions: Array[Int] = { + Array(1, 2) + } } object ScalaEnumSerializerConfigSnapshot { - val VERSION = 1 + val VERSION = 2 } - } diff --git a/flink-scala/src/test/resources/log4j-test.properties b/flink-scala/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..10792cd2949cc --- /dev/null +++ b/flink-scala/src/test/resources/log4j-test.properties @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=OFF, testlogger + +# A1 is set to be a ConsoleAppender. +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR +log4j.logger.org.apache.zookeeper=OFF diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala index c470cd00c0d2d..af725f6cfd272 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala @@ -74,6 +74,16 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike { |} """.stripMargin + val enumE = + s""" + |@SerialVersionUID(1L) + |object $enumName extends Enumeration { + | val A = Value(42) + | val B = Value(5) + | val C = Value(1337) + |} + """.stripMargin + /** * Check that identical enums don't require migration */ @@ -106,6 +116,16 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike { assertTrue(checkCompatibility(enumA, enumD).isRequiresMigration) } + /** + * Check that changing the enum ids causes a migration + */ + @Test + def checkDifferentIds(): Unit = { + assertTrue( + "Different ids should cause a migration.", + checkCompatibility(enumA, enumE).isRequiresMigration) + } + def checkCompatibility(enumSourceA: String, enumSourceB: String) : CompatibilityResult[Enumeration#Value] = { import EnumValueSerializerUpgradeTest._ diff --git a/flink-tests/src/test/resources/stateful-scala2.10-udf-migration-itcase-flink1.3-jobmanager-savepoint/_metadata b/flink-tests/src/test/resources/stateful-scala2.10-udf-migration-itcase-flink1.3-jobmanager-savepoint/_metadata index f4db7ff5b8ef391fc1b03d5318ac1734a04e0230..c18fd09f4d744567bdfe605d6640de9fdfc728a9 100644 GIT binary patch delta 1097 zcmd6lziU)M5XX1-T?h(6)JDY4D}*>W`+K)ii-;h27|*1X-M4Q+iwG8JB1E*X5t4A= zYa>C_Mvwzf3;%)?VqwI_*ot6f8+4BdvA4NuhFNyt^O^73tC`)+nf0Z!O%^EYCV+BA zf=a?b$XM(zH|EE0!|>~g{NE*`jc@GnodH&z!P2MeXI!1UOqqi+vG8jhAgD$IHr#47 zQmCc?kWTx4p*l5~+Bve%opc>)VIp{*GAM<5tgQwWWspV-sjVU8ypiDZ?J?DqbTu zvV1xZu1%UW3KrVH|&mm@QbXFnGzmmN~6oNr4;ny zL8wA21?MEzgFT5J7TK$M_6zmUUMvN#;!P1+G_^fw|AR^|F9~^{@P_<&o~60g>8GtT zwN4^5Gm=Y~;vj*P*p;M4LM#PVNy>B(C;*yo0|4B6@qxV9;a#ckb^4p(>-P@uL&iCa zcnDR95Tu+FD5z1;Xb}<}*joq!;LX0q-!4SWj`MuC``Ov~vM2?+(r) zuQ2=SW3L|&EfS>>ME2JqWgNo@kr2v=M$#yxkur>=E24Av$Jsemly3Q{E*6b4hM z5e{^sm{BPP;KS-R8(R|l4K}W>78)-eI@8z)7ef zNw|PihXN+t5GW;Mj7wptAo-zrX9F2YS(pTZTSg+=ZbAwrlOc?Z(t;|2B@zHEeAz~3 z3nSBbH0<8`7ncWx&GhWQ;XXToISIJd98zpuhN>iiHftm_5^F^biZrzj0t)%5b6$JC z>!!PB-FXzDG*ObnFxAG2O#~xJBPel76eWyD%7)su^K8+;U2~qLkXd9l8i@ zdEi|uMbt%*2Rckm(E42V&K?- zvD$#LfrHhe0#i8`8lzKDJxY4Pr&t+=8 z9AXlclvXGViIDBvql-IhC)m-qqgQVR^z|NfUCKtk{1-Ztg*&9Eug>KSAw_?&&I=_| z{mb)n`!hy5m+Bky`)4Ek`B?t1$av!`dvs^Wi_UQA6ctX)=1mRQ0fnI>;RRS4!#a)S7Xxk26(DMfv ztV~UbL4l2b=qU|eIt^S_QMgp0i;j>_cG576pAOi=e+b;W&o=3JYnkn_Lo9h%v!07p z%&pM5AY6h<5;KvU@SwGkNtiMy-~M>O{u0q#B#ViI4yZvnhdEhBLuwK?F(B7G=V{V{ qv3vK>c22`_)+pMKgt zQ|qQurKvyxjHEP>(g49I2sy-IfQ(25j#vah>umsldoMna7rVU63wxcxR`~k81N=0? zSj!}WSc(8rjS?ua9Vdt|7MPGwECG14uQ56AOn10#(+iS}I}6{=i&n>ZzT5rm?0oVS zw2Y2`WBHv`=b-ZSaAml*UZ*bV3*Q%iRz2XBz==^UE3)}eF*+;TOu1Se%cQCp|L-B+ zzm{a6qckMPWlu%cuJ*)g$W9bXz|ATdIFeslbvloh%A;9vqMiXSKGSRKz+jw#mVwiK zy|ycyiqG}gox1<*bhw%AHzQ?PDNmZ-TH-es`_bQ4bpe>DPb{o--6#L&*BUEJ?%m;8 z6f(N92pM4hqY}swj}l&Y+P9@G+sP(V$XS-zbtu6&W^^fF?Y_%rye_XpE1o-f(_)D zX-K7xpa?mEEEOsVx#cAZ^F#B_1~MT{A}nYCkrv7VMk1(Es}r3DQW=9(fUHgyzicD3 zg^_7Iy6)ck7ncWx&GhWQ;XXUg6GItc7)lm!NK<58f*6H53|A*m!!PB<0>VI1x9NqX=JIh9TY+*);xwYMFdt7ZJTGy9^9sRmPa-dQ;fa3&n4y}kP7Tt7qQVL*+?o0^9fqG&7z`&2!d$@H;u0l zL@NW9TP$`qRuN(f5gRS^4^XTHXQP$HHa0tOxtE#YT=;R$*3QD0=L@5oXGP+Mc}gKi zsX+>*z%i>JBoRcWnS0IP8cNu#SVLI)KzdgQbZ< zQXzU9S$2scw0mQ``QhsD%7kgJ%rf=g5Kc9&I7U`s~16tE8QpyyVvl*nM0xa_aY8z6ZF{>=c>%=}L>oLv&m`DTQ?QChS6iKB&Is)<&lQ13gURRx?pI`Q z*z7;Vi_KtzylovGBGJ6xAg3pTZ`G=57RvP1IahJ74Aq8-PN3hZ|ZBiX`&cnDBJRuJTp5bKNrIWzc;?r+>ggPVR;WmRYw%@qTs-+ zPJwepKxMiBN;Khud#+@P(DG<&W(nWl7j+$PYIXxRzn)i}7{A;dea1VVe?c8nbwta% z-*L~W!P-=#I^yd!eWf;|8?}k?q58!1&2W5s;25Fl$p(t^vjbrs9m^Z2WAx1MK-g7I z=jVn)P~Y#MBkiC+IeuW=Rp&0+3*Sb%u14*M>E>GRL-rrtr?s{c-J4h-ab@Anhmoq| zCZX09AY2i!@W4h1?tmmFrAd*6l`W``C)O!=VWjX(LCR$y$&7@IdIiD@&jXnv)eNEc z{o53__!S!zF8`>sUOmQnjNg{82jdmIqxJK8w1~^Or+BiODk7;<1{5vAYn#-A<~9XM zGS0M4d_q`xcnNR7aYh$R8y+g<1aMC?FoJQgLVClBOi5QD^y%m}oPUDTdU7qg^Ea4B zmCc+E+=^ZtBb-vsGXvZS8vMo$1k-u2T1euZAx6mHluCJWC_Yr~x*hFZZ<9)F=BzV7 zmA1fX=vBrf1uYnJ39*SJL5bUK>eFM{pqrkd3k4CW&ZV z8QCc@UbOwuuZ~@)#^v8AX`g_1UI$s8Y;|1-#KwYC1%olh%arO|he*iY{DC6yaIn4c zg7z!6J8btJ(3y6y%HDPk4~c5uud*}SgZb(%ZPQD7<1{9ZQpyNHq6D3=2t+0iu5m#% z%J3R_VbN)K`_=3|A@&a;m4g#OS_naw4k6dzlR=0n1Z8uuNh?A=;z1*{JY@GL5gO9% z>vz~MHUZ)aZ9lECS6$I~>9ebW=UA+Ux&dsmhG>`FIRHTNFyCJe^re?td*OIDTG|CcRT37kaRy~n z#_ho~qAtB~dB>ur$`5U4-}dBt6GlhTVXzujh3)gs*lcpW*fwUqIJ%9nkXb zcX;5`P;I(V9S!xGxl)@ojoReIaD8&-W<0(#U<$y@R0E{>*}*uEjpq%}HFjoXFs>@6 z^K&CnsQR!hSCfv!Ol!685&Mts(_URp?oBpvT4}!hFj~z# z6|Q5XECfe9lM*RSDGHo2%asz=XHEU_#2N)hsnXtYq-enqcP3IOh(}H<8+67wQxbsp z16vd}c@^svF8`>sUq69)3g4A)hSFuYt^Me;vN^s@WLZ4 zF{6S?&I2hAEy8s;#t<2Vq{vFFV#16?))y2N84*--t*vDMd^)-X=bzxTpI%Gu{0-)D zWh1A3x008~m=Lj=VurktE_PE4lEpT