From 60f053556a8698c5b857b3fc4d1fe228cc1271f8 Mon Sep 17 00:00:00 2001 From: James Duong Date: Thu, 12 Oct 2023 09:51:40 -0700 Subject: [PATCH] GH-37943: [Java] Add parquet file with all supported types Add a reference file with all supported types and corresponding test case to validate that the Dataset API generates this consistently. --- .../apache/arrow/dataset/TestAllTypes.java | 265 ++++++++++++++++++ .../org/apache/arrow/dataset/TestDataset.java | 41 +++ .../dataset/file/TestDatasetFileWriter.java | 41 --- .../referencefiles/alltypes/parquet/data_0 | Bin 0 -> 7998 bytes 4 files changed, 306 insertions(+), 41 deletions(-) create mode 100644 java/dataset/src/test/java/org/apache/arrow/dataset/TestAllTypes.java create mode 100644 java/dataset/src/test/resources/referencefiles/alltypes/parquet/data_0 diff --git a/java/dataset/src/test/java/org/apache/arrow/dataset/TestAllTypes.java b/java/dataset/src/test/java/org/apache/arrow/dataset/TestAllTypes.java new file mode 100644 index 0000000000000..fa92e87ec0b66 --- /dev/null +++ b/java/dataset/src/test/java/org/apache/arrow/dataset/TestAllTypes.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.arrow.dataset; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.SeekableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import org.apache.arrow.dataset.file.DatasetFileWriter; +import org.apache.arrow.dataset.file.FileFormat; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.DateMilliVector; +import org.apache.arrow.vector.Decimal256Vector; +import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.DurationVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.Float4Vector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.LargeVarBinaryVector; +import org.apache.arrow.vector.LargeVarCharVector; +import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.UInt1Vector; +import org.apache.arrow.vector.UInt2Vector; +import org.apache.arrow.vector.UInt4Vector; +import org.apache.arrow.vector.UInt8Vector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.FixedSizeListVector; +import org.apache.arrow.vector.complex.LargeListVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.complex.impl.UnionFixedSizeListWriter; +import org.apache.arrow.vector.complex.impl.UnionLargeListWriter; +import org.apache.arrow.vector.complex.impl.UnionListWriter; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.DateUnit; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; +import org.apache.arrow.vector.util.Text; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestAllTypes extends TestDataset { + + @ClassRule + public static final TemporaryFolder TMP = new TemporaryFolder(); + + private static final String PARQUET_ALLTYPES_FILE = "referencefiles/alltypes/parquet/data_0"; + + private VectorSchemaRoot generateAllTypesVector(BufferAllocator allocator) { + // Notes: + // - Float16 is not supported by Java. + // - IntervalMonthDayNano is not supported by Parquet. + // - SparseUnion and Map are resulting in serialization errors when writing with the Dataset API. + // "Unhandled type for Arrow to Parquet schema conversion" errors: IntervalDay, IntervalYear, DenseUnion + List childFields = new ArrayList<>(); + childFields.add(new Field("int-child", + new FieldType(false, new ArrowType.Int(32, true), null, null), null)); + Field structField = new Field("struct", + new FieldType(true, ArrowType.Struct.INSTANCE, null, null), childFields); + List fields = List.of( + Field.nullablePrimitive("null", ArrowType.Null.INSTANCE), + Field.nullablePrimitive("bool", ArrowType.Bool.INSTANCE), + Field.nullablePrimitive("int8", new ArrowType.Int(8, true)), + Field.nullablePrimitive("int16", new ArrowType.Int(16, true)), + Field.nullablePrimitive("int32", new ArrowType.Int(32, true)), + Field.nullablePrimitive("int64", new ArrowType.Int(64, true)), + Field.nullablePrimitive("uint8", new ArrowType.Int(8, false)), + Field.nullablePrimitive("uint16", new ArrowType.Int(16, false)), + Field.nullablePrimitive("uint32", new ArrowType.Int(32, false)), + Field.nullablePrimitive("uint64", new ArrowType.Int(64, false)), + Field.nullablePrimitive("float32", new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)), + Field.nullablePrimitive("float64", new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)), + Field.nullablePrimitive("utf8", ArrowType.Utf8.INSTANCE), + Field.nullablePrimitive("binary", ArrowType.Binary.INSTANCE), + Field.nullablePrimitive("largeutf8", ArrowType.LargeUtf8.INSTANCE), + Field.nullablePrimitive("largebinary", ArrowType.LargeBinary.INSTANCE), + Field.nullablePrimitive("fixed_size_binary", new ArrowType.FixedSizeBinary(1)), + Field.nullablePrimitive("date_ms", new ArrowType.Date(DateUnit.MILLISECOND)), + Field.nullablePrimitive("time_ms", new ArrowType.Time(TimeUnit.MILLISECOND, 32)), + Field.nullablePrimitive("timestamp_ms", new ArrowType.Timestamp(TimeUnit.MILLISECOND, null)), + Field.nullablePrimitive("timestamptz_ms", new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")), + Field.nullablePrimitive("time_ns", new ArrowType.Time(TimeUnit.NANOSECOND, 64)), + Field.nullablePrimitive("timestamp_ns", new ArrowType.Timestamp(TimeUnit.NANOSECOND, null)), + Field.nullablePrimitive("timestamptz_ns", new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC")), + Field.nullablePrimitive("duration", new ArrowType.Duration(TimeUnit.MILLISECOND)), + Field.nullablePrimitive("decimal128", new ArrowType.Decimal(10, 2, 128)), + Field.nullablePrimitive("decimal256", new ArrowType.Decimal(10, 2, 256)), + new Field("list", FieldType.nullable(ArrowType.List.INSTANCE), + Collections.singletonList(Field.nullable("items", new ArrowType.Int(32, true)))), + new Field("largelist", FieldType.nullable(ArrowType.LargeList.INSTANCE), + Collections.singletonList(Field.nullable("items", new ArrowType.Int(32, true)))), + new Field("fixedsizelist", FieldType.nullable(new ArrowType.FixedSizeList(2)), + Collections.singletonList(Field.nullable("items", new ArrowType.Int(32, true)))), + structField + ); + VectorSchemaRoot root = VectorSchemaRoot.create(new Schema(fields), allocator); + root.allocateNew(); + root.setRowCount(2); + + root.getVector("null").setNull(0); + root.getVector("bool").setNull(0); + root.getVector("int8").setNull(0); + root.getVector("int16").setNull(0); + root.getVector("int32").setNull(0); + root.getVector("int64").setNull(0); + root.getVector("uint8").setNull(0); + root.getVector("uint16").setNull(0); + root.getVector("uint32").setNull(0); + root.getVector("uint64").setNull(0); + root.getVector("float32").setNull(0); + root.getVector("float64").setNull(0); + root.getVector("utf8").setNull(0); + root.getVector("binary").setNull(0); + root.getVector("largeutf8").setNull(0); + root.getVector("largebinary").setNull(0); + root.getVector("fixed_size_binary").setNull(0); + root.getVector("date_ms").setNull(0); + root.getVector("time_ms").setNull(0); + root.getVector("time_ns").setNull(0); + root.getVector("timestamp_ms").setNull(0); + root.getVector("timestamp_ns").setNull(0); + root.getVector("timestamptz_ms").setNull(0); + root.getVector("timestamptz_ns").setNull(0); + root.getVector("duration").setNull(0); + root.getVector("decimal128").setNull(0); + root.getVector("decimal256").setNull(0); + root.getVector("fixedsizelist").setNull(0); + root.getVector("list").setNull(0); + root.getVector("largelist").setNull(0); + root.getVector("struct").setNull(0); + + root.getVector("null").setNull(1); + ((BitVector) root.getVector("bool")).set(1, 1); + ((TinyIntVector) root.getVector("int8")).set(1, 1); + ((SmallIntVector) root.getVector("int16")).set(1, 1); + ((IntVector) root.getVector("int32")).set(1, 1); + ((BigIntVector) root.getVector("int64")).set(1, 1); + ((UInt1Vector) root.getVector("uint8")).set(1, 1); + ((UInt2Vector) root.getVector("uint16")).set(1, 1); + ((UInt4Vector) root.getVector("uint32")).set(1, 1); + ((UInt8Vector) root.getVector("uint64")).set(1, 1); + ((Float4Vector) root.getVector("float32")).set(1, 1.0f); + ((Float8Vector) root.getVector("float64")).set(1, 1.0); + ((VarCharVector) root.getVector("utf8")).set(1, new Text("a")); + ((VarBinaryVector) root.getVector("binary")).set(1, new byte[] {0x01}); + ((LargeVarCharVector) root.getVector("largeutf8")).set(1, new Text("a")); + ((LargeVarBinaryVector) root.getVector("largebinary")).set(1, new byte[] {0x01}); + ((FixedSizeBinaryVector) root.getVector("fixed_size_binary")).set(1, new byte[] {0x01}); + ((DateMilliVector) root.getVector("date_ms")).set(1, 0); + ((TimeMilliVector) root.getVector("time_ms")).set(1, 0); + ((TimeNanoVector) root.getVector("time_ns")).set(1, 0); + ((TimeStampMilliVector) root.getVector("timestamp_ms")).set(1, 0); + ((TimeStampNanoVector) root.getVector("timestamp_ns")).set(1, 0); + ((TimeStampMilliTZVector) root.getVector("timestamptz_ms")).set(1, 0); + ((TimeStampNanoTZVector) root.getVector("timestamptz_ns")).set(1, 0); + ((DurationVector) root.getVector("duration")).set(1, 0); + ((DecimalVector) root.getVector("decimal128")).set(1, 0); + ((Decimal256Vector) root.getVector("decimal256")).set(1, 0); + UnionFixedSizeListWriter fixedListWriter = ((FixedSizeListVector) root.getVector("fixedsizelist")).getWriter(); + fixedListWriter.allocate(); + fixedListWriter.setPosition(1); + fixedListWriter.startList(); + fixedListWriter.integer().writeInt(1); + fixedListWriter.endList(); + + UnionListWriter listWriter = ((ListVector) root.getVector("list")).getWriter(); + listWriter.allocate(); + listWriter.setPosition(1); + listWriter.startList(); + listWriter.integer().writeInt(1); + listWriter.endList(); + + UnionLargeListWriter largeListWriter = ((LargeListVector) root.getVector("largelist")).getWriter(); + largeListWriter.allocate(); + largeListWriter.setPosition(1); + largeListWriter.startList(); + largeListWriter.integer().writeInt(1); + largeListWriter.endList(); + + ((StructVector) root.getVector("struct")).getChild("int-child", IntVector.class).set(1, 1); + return root; + } + + private byte[] serializeFile(VectorSchemaRoot root) { + try ( + ByteArrayOutputStream out = new ByteArrayOutputStream(); + WritableByteChannel channel = Channels.newChannel(out); + ArrowStreamWriter writer = new ArrowStreamWriter(root, null, channel) + ) { + writer.start(); + writer.writeBatch(); + writer.end(); + return out.toByteArray(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to serialize arrow file", e); + } + } + + @Test + public void testAllTypesParquet() throws Exception { + try (VectorSchemaRoot root = generateAllTypesVector(rootAllocator())) { + byte[] featherData = serializeFile(root); + try (SeekableByteChannel channel = new ByteArrayReadableSeekableByteChannel(featherData)) { + try (ArrowStreamReader reader = new ArrowStreamReader(channel, rootAllocator())) { + TMP.create(); + final File writtenFolder = TMP.newFolder(); + final String writtenParquet = writtenFolder.toURI().toString(); + DatasetFileWriter.write(rootAllocator(), reader, FileFormat.PARQUET, + writtenParquet); + + // Load the reference file from the test resources and write to a temporary file on the OS. + byte[] referenceParquetBytes = Files.readAllBytes(Path.of(TestAllTypes.class.getClassLoader() + .getResource(PARQUET_ALLTYPES_FILE).toURI())); + File extractedReferenceFile = TMP.newFile(); + Files.write(extractedReferenceFile.toPath(), referenceParquetBytes); + assertParquetFileEquals(extractedReferenceFile.toURI().toString(), + Objects.requireNonNull(writtenFolder.listFiles())[0].toURI().toString()); + } + } + } + } +} diff --git a/java/dataset/src/test/java/org/apache/arrow/dataset/TestDataset.java b/java/dataset/src/test/java/org/apache/arrow/dataset/TestDataset.java index af2abeee2145f..cafa63b7880ce 100644 --- a/java/dataset/src/test/java/org/apache/arrow/dataset/TestDataset.java +++ b/java/dataset/src/test/java/org/apache/arrow/dataset/TestDataset.java @@ -27,6 +27,9 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import org.apache.arrow.dataset.file.FileFormat; +import org.apache.arrow.dataset.file.FileSystemDatasetFactory; +import org.apache.arrow.dataset.jni.NativeMemoryPool; import org.apache.arrow.dataset.scanner.ScanOptions; import org.apache.arrow.dataset.scanner.Scanner; import org.apache.arrow.dataset.source.Dataset; @@ -34,12 +37,16 @@ import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VectorLoader; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.VectorUnloader; +import org.apache.arrow.vector.compare.VectorEqualsVisitor; import org.apache.arrow.vector.ipc.ArrowReader; import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; import org.apache.arrow.vector.types.pojo.Schema; import org.junit.After; +import org.junit.Assert; import org.junit.Before; @@ -100,6 +107,40 @@ protected Schema inferResultSchemaFromFactory(DatasetFactory factory, ScanOption return schema; } + protected void assertParquetFileEquals(String expectedURI, String actualURI) throws Exception { + final FileSystemDatasetFactory expectedFactory = new FileSystemDatasetFactory( + rootAllocator(), NativeMemoryPool.getDefault(), FileFormat.PARQUET, expectedURI); + final FileSystemDatasetFactory actualFactory = new FileSystemDatasetFactory( + rootAllocator(), NativeMemoryPool.getDefault(), FileFormat.PARQUET, actualURI); + List expectedBatches = collectResultFromFactory(expectedFactory, + new ScanOptions(new String[0], 100)); + List actualBatches = collectResultFromFactory(actualFactory, + new ScanOptions(new String[0], 100)); + try ( + VectorSchemaRoot expectVsr = VectorSchemaRoot.create(expectedFactory.inspect(), rootAllocator()); + VectorSchemaRoot actualVsr = VectorSchemaRoot.create(actualFactory.inspect(), rootAllocator())) { + + // fast-fail by comparing metadata + Assert.assertEquals(expectedBatches.toString(), actualBatches.toString()); + // compare ArrowRecordBatches + Assert.assertEquals(expectedBatches.size(), actualBatches.size()); + VectorLoader expectLoader = new VectorLoader(expectVsr); + VectorLoader actualLoader = new VectorLoader(actualVsr); + for (int i = 0; i < expectedBatches.size(); i++) { + expectLoader.load(expectedBatches.get(i)); + actualLoader.load(actualBatches.get(i)); + for (int j = 0; j < expectVsr.getFieldVectors().size(); j++) { + FieldVector vector = expectVsr.getFieldVectors().get(i); + FieldVector otherVector = actualVsr.getFieldVectors().get(i); + // TODO: ARROW-18140 Use VectorSchemaRoot#equals() method to compare + Assert.assertTrue(VectorEqualsVisitor.vectorEquals(vector, otherVector)); + } + } + } finally { + AutoCloseables.close(expectedBatches, actualBatches); + } + } + protected Stream stream(Iterable iterable) { return StreamSupport.stream(iterable.spliterator(), false); } diff --git a/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestDatasetFileWriter.java b/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestDatasetFileWriter.java index 10c06be2cca3b..86797c165b2b1 100644 --- a/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestDatasetFileWriter.java +++ b/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestDatasetFileWriter.java @@ -20,7 +20,6 @@ import java.io.File; import java.util.Arrays; import java.util.HashSet; -import java.util.List; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -32,12 +31,6 @@ import org.apache.arrow.dataset.scanner.ScanOptions; import org.apache.arrow.dataset.scanner.Scanner; import org.apache.arrow.dataset.source.Dataset; -import org.apache.arrow.util.AutoCloseables; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.VectorLoader; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.compare.VectorEqualsVisitor; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.ClassRule; @@ -99,39 +92,5 @@ public void testParquetWriteWithPartitions() throws Exception { Assert.assertEquals(expectedOutputFiles, outputFiles); } } - - private void assertParquetFileEquals(String expectedURI, String actualURI) throws Exception { - final FileSystemDatasetFactory expectedFactory = new FileSystemDatasetFactory( - rootAllocator(), NativeMemoryPool.getDefault(), FileFormat.PARQUET, expectedURI); - final FileSystemDatasetFactory actualFactory = new FileSystemDatasetFactory( - rootAllocator(), NativeMemoryPool.getDefault(), FileFormat.PARQUET, actualURI); - List expectedBatches = collectResultFromFactory(expectedFactory, - new ScanOptions(new String[0], 100)); - List actualBatches = collectResultFromFactory(actualFactory, - new ScanOptions(new String[0], 100)); - try ( - VectorSchemaRoot expectVsr = VectorSchemaRoot.create(expectedFactory.inspect(), rootAllocator()); - VectorSchemaRoot actualVsr = VectorSchemaRoot.create(actualFactory.inspect(), rootAllocator())) { - - // fast-fail by comparing metadata - Assert.assertEquals(expectedBatches.toString(), actualBatches.toString()); - // compare ArrowRecordBatches - Assert.assertEquals(expectedBatches.size(), actualBatches.size()); - VectorLoader expectLoader = new VectorLoader(expectVsr); - VectorLoader actualLoader = new VectorLoader(actualVsr); - for (int i = 0; i < expectedBatches.size(); i++) { - expectLoader.load(expectedBatches.get(i)); - actualLoader.load(actualBatches.get(i)); - for (int j = 0; j < expectVsr.getFieldVectors().size(); j++) { - FieldVector vector = expectVsr.getFieldVectors().get(i); - FieldVector otherVector = actualVsr.getFieldVectors().get(i); - // TODO: ARROW-18140 Use VectorSchemaRoot#equals() method to compare - Assert.assertTrue(VectorEqualsVisitor.vectorEquals(vector, otherVector)); - } - } - } finally { - AutoCloseables.close(expectedBatches, actualBatches); - } - } } diff --git a/java/dataset/src/test/resources/referencefiles/alltypes/parquet/data_0 b/java/dataset/src/test/resources/referencefiles/alltypes/parquet/data_0 new file mode 100644 index 0000000000000000000000000000000000000000..f5bdda9d6eff03b11857ef7caa851d6ef6a72d30 GIT binary patch literal 7998 zcmd5>K}Z~D6#i#2uA7aTRr610%``D-=0%Gpy1HUXny8h~!_rFQp@c#*(Y0CFUG47L z+R%e|k`jWH5Q6kjLJ2uYPf{dEkrGM?A(R|S>A_PErH9Z%4n5@1_y03H|IF@MS9jLB ztg|z3{{MaNz5o61fA2Xq{6Q~OD4`_IE-eHvw23x#Au&t?G@uhgWc*VIfe&>mM`9!p zj%Z3|HkG18Rn#$c3{F9?lg%JhX7j`N!x?E*V`OB5j5JBYuPS%&Uz_a#nGKaCSdw*0 zA~b-kuTM`~tgon7;CtzO(ZmQ2M1~wfp+OpqQ<+OHS)eVnMM$qHjA~sG+V$iJAjt%7 zDK6ELnfxGEZAns>B>3h4ypr|*U=%4;-7mS;146Dkl&g2ZCD<=@uu}Iw3HBoB6|67n z5^Ny|oAo;uEKI}hmBjU7zT{oaAX`oL(VRP6Rp(w%aR_se12S9PpAYF&u^pFLaiE_I zyC$h?60C<{DWr>;;%*DhyIJmmtzB=S$z{n|$M@#cIry#-zHHQs=K2gj9~STd25-9< z!(TZJi-&8C-%^BcdlJWQhQ;QJ@GI~gG~;%Da_@-n6K)IJQLH$y+6s%UOqz0Rwp$fK z<}a3bGse{UiyFVG=pf?B`;EdX#my#W7x5}9n#*7ahM&Jn z#S0eRXwDt7cJ2ig$BE5$ALDb#Bd%w=-PPc)*KoU4%)6}kAsTYW3bWX>xJr_#Nv21# zUytgv!In&!%})*TD3DPzaQi6y*~XUEnp6p-R1T6Ia309@*9BzYdNN~XKX*W1>hMhG zpnO^-pq1(Q;J5%im@=~;Cp^k-9)q=GwNUo%34wcv;}&WBtQ{V#LeZ+^i08=*73d43GFQLiAL^5rrdZSkAuq>&|5j8EcvVg7FR|5Bkoik=oNhNc6eyLBS66r)H&zJg|dJQ&*;Kv~y z=a;q2&L9`x5s+5Gvo-GH5qLP_6Yf8w0=JQ;9xU_N8hdpV?u}N6?did|%uL{qOAcyB zAeYb1PUH__-PS#EBbl1y_8nKpVdWAmUD^sO^6GSqXo1|sjYQg{7qD$($W}`ywT_EP z343I182b&YN{wUjmDXT0d(B2$-3X%{iD3`1FG{Fl4;bjT2lKktP#FhoFKv;hB` z4d8#10QRVmhdnE{q4ota>`~DQdsbM&9u$92vw96`R$}>3GJ#Je5cpWqR9~p5izieC zg7|}}Xh-Lanf+uofrZY@Ot+cMPJh<^X(F4${y}?h|CydMJ>8ez8y>s-?)&4I=3(+( OyDu4OBxDT#bN(+?mZKy9 literal 0 HcmV?d00001