From 9720820975be61c9a211a24785a28cf0e8b0d7b3 Mon Sep 17 00:00:00 2001 From: Manoj Govindassamy Date: Fri, 12 Nov 2021 05:18:40 -0800 Subject: [PATCH 001/217] [HUDI-2718] ExternalSpillableMap payload size re-estimation throws ArithmeticException (#3955) - ExternalSpillableMap does the payload/value size estimation on the first put to determine when to spill over to disk map. The payload size re-estimation also happens after a minimum threshold of puts. This size re-estimation goes my the current in-memory map size for calculating average payload size and does attempts divide by zero operation when the map is size is empty. Avoiding the ArithmeticException during the payload size re-estimate by checking the map size upfront. --- .../util/collection/ExternalSpillableMap.java | 3 +- .../collection/TestExternalSpillableMap.java | 42 ++++++++++++++++--- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java index d31b0aaa6c308..a6e8d5cfb35c0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java @@ -208,7 +208,8 @@ public R put(T key, R value) { // Note, the converter may over estimate the size of a record in the JVM this.estimatedPayloadSize = keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); LOG.info("Estimated Payload size => " + estimatedPayloadSize); - } else if (shouldEstimatePayloadSize && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + } else if (shouldEstimatePayloadSize && !inMemoryMap.isEmpty() + && (inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0)) { // Re-estimate the size of a record by calculating the size of the entire map containing // N entries and then dividing by the number of entries present (N). This helps to get a // correct estimation of the size of each record in the JVM. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java index 4fed5a80eb1f2..f7b45e9d839b6 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java @@ -51,6 +51,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -83,7 +84,7 @@ public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType, boole List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); assert (recordKeys.size() == 100); - + // Test iterator Iterator> itr = records.iterator(); int cntSize = 0; @@ -93,7 +94,7 @@ public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType, boole assert recordKeys.contains(rec.getRecordKey()); } assertEquals(recordKeys.size(), cntSize); - + // Test value stream List> values = records.valueStream().collect(Collectors.toList()); cntSize = 0; @@ -221,7 +222,9 @@ failureOutputPath, new DefaultSizeEstimator(), @ParameterizedTest @MethodSource("testArguments") - public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException { + public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled) throws IOException, + URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); @@ -274,7 +277,9 @@ record = records.get(key); @ParameterizedTest @MethodSource("testArguments") - public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException { + public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled) throws IOException, + URISyntaxException { Schema schema = SchemaTestUtil.getSimpleSchema(); @@ -337,9 +342,34 @@ record = records.get(key); assertEquals(gRecord.get(fieldName).toString(), newValue); } - // TODO : come up with a performance eval test for spillableMap @Test - public void testLargeInsertUpsert() {} + public void testEstimationWithEmptyMap() throws IOException, URISyntaxException { + final ExternalSpillableMap.DiskMapType diskMapType = ExternalSpillableMap.DiskMapType.BITCASK; + final boolean isCompressionEnabled = false; + final Schema schema = SchemaTestUtil.getSimpleSchema(); + + ExternalSpillableMap> records = + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); + + List recordKeys = new ArrayList<>(); + + // Put a single record. Payload size estimation happens as part of this initial put. + HoodieRecord seedRecord = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1).get(0); + records.put(seedRecord.getRecordKey(), seedRecord); + + // Remove the key immediately to make the map empty again. + records.remove(seedRecord.getRecordKey()); + + // Verify payload size re-estimation does not throw exception + List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 250); + hoodieRecords.stream().forEach(hoodieRecord -> { + assertDoesNotThrow(() -> { + records.put(hoodieRecord.getRecordKey(), hoodieRecord); + }, "ExternalSpillableMap put() should not throw exception!"); + recordKeys.add(hoodieRecord.getRecordKey()); + }); + } private static Stream testArguments() { // Arguments : 1. Disk Map Type 2. isCompressionEnabled for BitCaskMap From 93fd3517e319473fed74e62bf09e41e4348c843c Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Fri, 12 Nov 2021 17:28:25 -0500 Subject: [PATCH 002/217] [HUDI-2741] Fixing instantiating metadata table config in HoodieFileIndex (#3974) --- .../src/main/scala/org/apache/hudi/HoodieFileIndex.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 882636c4697be..c4e9bafb3fc63 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -129,7 +129,7 @@ case class HoodieFileIndex( // To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users // would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing. - properties.put(HoodieMetadataConfig.ENABLE, + properties.setProperty(HoodieMetadataConfig.ENABLE.key(), sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(), HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString)) properties.putAll(options.asJava) From 0e8461e9abc97537954a2c1dd716aed53e52dc62 Mon Sep 17 00:00:00 2001 From: xiarixiaoyao Date: Sat, 13 Nov 2021 09:12:33 +0800 Subject: [PATCH 003/217] [HUDI-2697] Minor changes about hbase index config. (#3927) --- .../java/org/apache/hudi/config/HoodieHBaseIndexConfig.java | 6 +++--- .../main/java/org/apache/hudi/config/HoodieWriteConfig.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java index 561460777ff6b..3d7e3a7941daa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java @@ -76,9 +76,9 @@ public class HoodieHBaseIndexConfig extends HoodieConfig { .withDocumentation("Property to set which implementation of HBase QPS resource allocator to be used, which" + "controls the batching rate dynamically."); - public static final ConfigProperty PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty + public static final ConfigProperty PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty .key("hoodie.index.hbase.put.batch.size.autocompute") - .defaultValue("false") + .defaultValue(false) .withDocumentation("Property to set to enable auto computation of put batch size"); public static final ConfigProperty QPS_FRACTION = ConfigProperty @@ -206,7 +206,7 @@ public class HoodieHBaseIndexConfig extends HoodieConfig { * @deprecated Use {@link #PUT_BATCH_SIZE_AUTO_COMPUTE} and its methods instead */ @Deprecated - public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = PUT_BATCH_SIZE_AUTO_COMPUTE.defaultValue(); + public static final Boolean DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = PUT_BATCH_SIZE_AUTO_COMPUTE.defaultValue(); /** * @deprecated Use {@link #MAX_QPS_FRACTION} and its methods instead */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index e167b12c82045..eb3df38428f58 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1295,8 +1295,8 @@ public int getHbaseIndexPutBatchSize() { return getInt(HoodieHBaseIndexConfig.PUT_BATCH_SIZE); } - public Boolean getHbaseIndexPutBatchSizeAutoCompute() { - return getBoolean(HoodieHBaseIndexConfig.PUT_BATCH_SIZE_AUTO_COMPUTE); + public boolean getHbaseIndexPutBatchSizeAutoCompute() { + return getBooleanOrDefault(HoodieHBaseIndexConfig.PUT_BATCH_SIZE_AUTO_COMPUTE); } public String getHBaseQPSResourceAllocatorClass() { @@ -1357,7 +1357,7 @@ public int getHbaseIndexMaxQPSPerRegionServer() { } public boolean getHbaseIndexUpdatePartitionPath() { - return getBoolean(HoodieHBaseIndexConfig.UPDATE_PARTITION_PATH_ENABLE); + return getBooleanOrDefault(HoodieHBaseIndexConfig.UPDATE_PARTITION_PATH_ENABLE); } public int getBloomIndexParallelism() { From 994922a1593824744f93fd1408fcf1807cf4ebc1 Mon Sep 17 00:00:00 2001 From: Manoj Govindassamy Date: Sat, 13 Nov 2021 16:37:30 -0800 Subject: [PATCH 004/217] [HUDI-2472] Enabling metadata table in TestHoodieIndex and TestMergeOnReadRollbackActionExecutor (#3978) - With rollback after first commit support added to metadata table, these test cases are safe to have metadata table turned on. --- .../org/apache/hudi/client/functional/TestHoodieIndex.java | 2 +- .../rollback/TestMergeOnReadRollbackActionExecutor.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 5c75bb565a31d..c741c0266cc71 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -225,7 +225,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul @ParameterizedTest @MethodSource("indexTypeParams") public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields, false); + setUp(indexType, populateMetaFields, true); String newCommitTime = writeClient.startCommit(); int totalRecords = 20 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index 38be873e57ad6..5a829e294a4b5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -155,12 +155,13 @@ public void testFailForCompletedInstants() { } /** - * Test Cases for rollbacking when has not base file. + * Test Cases for rolling back when there is no base file. */ @Test public void testRollbackWhenFirstCommitFail() throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime("001"); From c2f9094b493ed28a63bdaed10ce4349261c3cea6 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Sun, 14 Nov 2021 08:51:54 +0800 Subject: [PATCH 005/217] [HUDI-2756] Fix flink parquet writer decimal type conversion (#3988) --- .../row/parquet/ParquetSchemaConverter.java | 2 +- .../format/cow/FixedLenBytesColumnReader.java | 85 +++++++++++++++++++ .../format/cow/ParquetSplitReaderUtil.java | 3 +- 3 files changed, 87 insertions(+), 3 deletions(-) create mode 100644 hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java index 5187660c8caec..80fda29aa4756 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java @@ -564,7 +564,7 @@ private static Type convertToParquetType( int scale = ((DecimalType) type).getScale(); int numBytes = computeMinBytesForDecimalPrecision(precision); return Types.primitive( - PrimitiveType.PrimitiveTypeName.BINARY, repetition) + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) .precision(precision) .scale(scale) .length(numBytes) diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java new file mode 100644 index 0000000000000..07a93e19c25c6 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.format.cow; + +import org.apache.flink.table.data.vector.writable.WritableBytesVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; +import org.apache.flink.table.data.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Fixed length bytes {@code ColumnReader}, just for decimal. + * + *

Note: Reference Flink release 1.13.2 + * {@code org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader} + * to always write as legacy decimal format. + */ +public class FixedLenBytesColumnReader + extends AbstractColumnReader { + + public FixedLenBytesColumnReader( + ColumnDescriptor descriptor, PageReader pageReader, int precision) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + } + + @Override + protected void readBatch(int rowId, int num, V column) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + byte[] bytes = readDataBinary(bytesLen).getBytes(); + bytesVector.appendBytes(rowId + i, bytes, 0, bytes.length); + } else { + bytesVector.setNullAt(rowId + i); + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, int num, V column, WritableIntVector dictionaryIds) { + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!bytesVector.isNullAt(i)) { + byte[] v = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes(); + bytesVector.appendBytes(i, v, 0, v.length); + } + } + } + + private Binary readDataBinary(int len) { + ByteBuffer buffer = readDataBuffer(len); + if (buffer.hasArray()) { + return Binary.fromConstantByteArray( + buffer.array(), buffer.arrayOffset() + buffer.position(), len); + } else { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return Binary.fromConstantByteArray(bytes); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index 6bb514b429b38..d87a2af706d84 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -24,7 +24,6 @@ import org.apache.flink.formats.parquet.vector.reader.BytesColumnReader; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; import org.apache.flink.formats.parquet.vector.reader.DoubleColumnReader; -import org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader; import org.apache.flink.formats.parquet.vector.reader.FloatColumnReader; import org.apache.flink.formats.parquet.vector.reader.IntColumnReader; import org.apache.flink.formats.parquet.vector.reader.LongColumnReader; @@ -366,7 +365,7 @@ public static WritableColumnVector createWritableColumnVector( "TIME_MICROS original type is not "); return new HeapTimestampVector(batchSize); case DECIMAL: - checkArgument(typeName == PrimitiveType.PrimitiveTypeName.BINARY + checkArgument(typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && primitiveType.getOriginalType() == OriginalType.DECIMAL, "Unexpected type: %s", typeName); return new HeapBytesVector(batchSize); From 0bb6d8ff80cd49dd05d2cfa427d36f2c97dd77a8 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Mon, 15 Nov 2021 07:44:39 +0800 Subject: [PATCH 006/217] [HUDI-2706] refactor spark-sql to make consistent with DataFrame api (#3936) --- .../model/DefaultHoodieRecordPayload.java | 2 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 38 +-- .../org/apache/hudi/HoodieWriterUtils.scala | 76 +++++- .../spark/sql/hudi/HoodieOptionConfig.scala | 73 ++++- .../spark/sql/hudi/HoodieSqlUtils.scala | 2 +- .../sql/hudi/analysis/HoodieAnalysis.scala | 5 +- .../AlterHoodieTableAddColumnsCommand.scala | 9 +- ...AlterHoodieTableDropPartitionCommand.scala | 6 +- .../CreateHoodieTableAsSelectCommand.scala | 9 +- .../command/CreateHoodieTableCommand.scala | 249 +++++++++--------- .../command/DeleteHoodieTableCommand.scala | 21 +- .../InsertIntoHoodieTableCommand.scala | 95 ++++--- .../command/MergeIntoHoodieTableCommand.scala | 41 +-- .../command/UpdateHoodieTableCommand.scala | 33 ++- .../src/test/java/HoodieJavaApp.java | 2 +- .../src/test/resources/sql-statements.sql | 4 +- .../hudi/HoodieSparkSqlWriterSuite2.scala | 47 ++++ .../TestDataSourceForBootstrap.scala | 5 +- .../spark/sql/hudi/TestAlterTable.scala | 4 +- .../hudi/TestAlterTableDropPartition.scala | 8 +- .../spark/sql/hudi/TestCompactionTable.scala | 4 +- .../spark/sql/hudi/TestCreateTable.scala | 87 ++++-- .../spark/sql/hudi/TestDeleteTable.scala | 2 +- .../sql/hudi/TestHoodieOptionConfig.scala | 184 +++++++++++++ .../spark/sql/hudi/TestInsertTable.scala | 43 ++- .../sql/hudi/TestMergeIntoLogOnlyTable.scala | 2 +- .../spark/sql/hudi/TestMergeIntoTable.scala | 22 +- .../spark/sql/hudi/TestMergeIntoTable2.scala | 12 +- .../hudi/TestPartialUpdateForMergeInto.scala | 8 +- .../spark/sql/hudi/TestShowPartitions.scala | 6 +- .../spark/sql/hudi/TestUpdateTable.scala | 2 +- 31 files changed, 764 insertions(+), 337 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java index 76474fde66eae..07ab5df9c7dbd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -113,7 +113,7 @@ protected boolean needUpdatingPersistedRecord(IndexedRecord currentValue, Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true); Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord, - properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), false); + properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true); return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0; } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index ddbd7fc06a95b..d1094bb5cd3a7 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -19,11 +19,13 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf + import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.DataSourceOptionsHelper.{allAlternatives, translateConfigurations} +import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties} @@ -42,19 +44,21 @@ import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.hudi.table.BulkInsertPartitioner + import org.apache.log4j.LogManager + import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession} import org.apache.spark.{SPARK_VERSION, SparkContext} + import java.util import java.util.Properties import scala.collection.JavaConversions._ import scala.collection.mutable -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ListBuffer object HoodieSparkSqlWriter { @@ -141,7 +145,7 @@ object HoodieSparkSqlWriter { .setPartitionFields(partitionColumns) .setPopulateMetaFields(populateMetaFields) .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD)) - .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME)) + .setKeyGeneratorClassProp(HoodieWriterUtils.getOriginKeyGenerator(parameters)) .setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING)) .setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING)) .initTable(sparkContext.hadoopConfiguration, path) @@ -713,22 +717,6 @@ object HoodieSparkSqlWriter { } } - private def validateTableConfig(spark: SparkSession, params: Map[String, String], - tableConfig: HoodieTableConfig): Unit = { - val resolver = spark.sessionState.conf.resolver - val diffConfigs = StringBuilder.newBuilder - params.foreach { case (key, value) => - val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key) - if (null != existingValue && !resolver(existingValue, value)) { - diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n") - } - } - if (diffConfigs.nonEmpty) { - diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n") - throw new HoodieException(diffConfigs.toString.trim) - } - } - private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String], tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = { val mergedParams = mutable.Map.empty ++ @@ -745,16 +733,4 @@ object HoodieSparkSqlWriter { val params = mergedParams.toMap (params, HoodieWriterUtils.convertMapToHoodieConfig(params)) } - - private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieTableConfig, key: String): String = { - if (null == tableConfig) { - null - } else { - if (allAlternatives.contains(key)) { - tableConfig.getString(allAlternatives(key)) - } else { - tableConfig.getString(key) - } - } - } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 0e3ede1fe3ebc..c1223d9792273 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -17,15 +17,19 @@ package org.apache.hudi +import java.util.Properties + +import org.apache.hudi.DataSourceOptionsHelper.allAlternatives import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE import org.apache.hudi.common.config.{HoodieConfig, TypedProperties} +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.exception.HoodieException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hudi.command.SqlKeyGenerator -import java.util.Properties import scala.collection.JavaConversions.mapAsJavaMap -import scala.collection.JavaConverters.{mapAsScalaMapConverter, _} -import scala.collection.JavaConverters.mapAsScalaMapConverter -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory +import scala.collection.JavaConverters._ /** * WriterUtils to assist in write path in Datasource and tests. @@ -102,4 +106,68 @@ object HoodieWriterUtils { properties.putAll(mapAsJavaMap(parameters)) new HoodieConfig(properties) } + + def getOriginKeyGenerator(parameters: Map[String, String]): String = { + val kg = parameters.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null) + if (classOf[SqlKeyGenerator].getCanonicalName == kg) { + parameters.getOrElse(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null) + } else { + kg + } + } + + /** + * Detects conflicts between new parameters and existing table configurations + */ + def validateTableConfig(spark: SparkSession, params: Map[String, String], + tableConfig: HoodieConfig): Unit = { + val resolver = spark.sessionState.conf.resolver + val diffConfigs = StringBuilder.newBuilder + params.foreach { case (key, value) => + val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key) + if (null != existingValue && !resolver(existingValue, value)) { + diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n") + } + } + + if (null != tableConfig) { + val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null) + val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS) + if (null != datasourceRecordKey && null != tableConfigRecordKey + && datasourceRecordKey != tableConfigRecordKey) { + diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n") + } + + val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null) + val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD) + if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey + && datasourcePreCombineKey != tableConfigPreCombineKey) { + diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n") + } + + val datasourceKeyGen = getOriginKeyGenerator(params) + val tableConfigKeyGen = tableConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME) + if (null != datasourceKeyGen && null != tableConfigKeyGen + && datasourceKeyGen != tableConfigKeyGen) { + diffConfigs.append(s"KeyGenerator:\t$datasourceKeyGen\t$tableConfigKeyGen\n") + } + } + + if (diffConfigs.nonEmpty) { + diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n") + throw new HoodieException(diffConfigs.toString.trim) + } + } + + private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieConfig, key: String): String = { + if (null == tableConfig) { + null + } else { + if (allAlternatives.contains(key)) { + tableConfig.getString(allAlternatives(key)) + } else { + tableConfig.getString(key) + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 963035cb638d3..543a8b997004c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -20,6 +20,10 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.common.model.DefaultHoodieRecordPayload import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.ValidationUtils + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType /** @@ -43,6 +47,7 @@ object HoodieOptionConfig { .withSqlKey("primaryKey") .withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD.key) .withTableConfigKey(HoodieTableConfig.RECORDKEY_FIELDS.key) + .defaultValue(DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue()) .build() val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf() @@ -102,6 +107,8 @@ object HoodieOptionConfig { private lazy val reverseValueMapping = valueMapping.map(f => f._2 -> f._1) + def withDefaultSqlOptions(options: Map[String, String]): Map[String, String] = defaultSqlOptions ++ options + /** * Mapping the sql's short name key/value in the options to the hoodie's config key/value. * @param options @@ -119,14 +126,13 @@ object HoodieOptionConfig { * @return */ def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = { - defaultTableConfig ++ - options.map { case (k, v) => - if (keyTableConfigMapping.contains(k)) { - keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v) - } else { - k -> v - } + options.map { case (k, v) => + if (keyTableConfigMapping.contains(k)) { + keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v) + } else { + k -> v } + } } /** @@ -136,16 +142,19 @@ object HoodieOptionConfig { options.map(kv => tableConfigKeyToSqlKey.getOrElse(kv._1, kv._1) -> reverseValueMapping.getOrElse(kv._2, kv._2)) } - private lazy val defaultTableConfig: Map[String, String] = { + private lazy val defaultSqlOptions: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields .filter(f => f.getType == classOf[HoodieOption[_]]) .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieOption[_]]}) .filter(option => option.tableConfigKey.isDefined && option.defaultValue.isDefined) - .map(option => option.tableConfigKey.get -> - valueMapping.getOrElse(option.defaultValue.get.toString, option.defaultValue.get.toString)) + .map(option => option.sqlKeyName -> option.defaultValue.get.toString) .toMap } + private lazy val defaultTableConfig: Map[String, String] = { + mappingSqlOptionToHoodieParam(defaultSqlOptions) + } + /** * Get the primary key from the table options. * @param options @@ -154,7 +163,7 @@ object HoodieOptionConfig { def getPrimaryColumns(options: Map[String, String]): Array[String] = { val params = mappingSqlOptionToHoodieParam(options) params.get(DataSourceWriteOptions.RECORDKEY_FIELD.key) - .map(_.split(",").filter(_.length > 0)) + .map(_.split(",").filter(_.nonEmpty)) .getOrElse(Array.empty) } @@ -171,7 +180,47 @@ object HoodieOptionConfig { def getPreCombineField(options: Map[String, String]): Option[String] = { val params = mappingSqlOptionToHoodieParam(options) - params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key) + params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty) + } + + def deleteHooideOptions(options: Map[String, String]): Map[String, String] = { + options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => keyMapping.contains(kv._1)) + } + + // extract primaryKey, preCombineField, type options + def extractSqlOptions(options: Map[String, String]): Map[String, String] = { + val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) + options.filterKeys(targetOptions.contains) + } + + // validate primaryKey, preCombineField and type options + def validateTable(spark: SparkSession, schema: StructType, options: Map[String, String]): Unit = { + val resolver = spark.sessionState.conf.resolver + + // validate primary key + val primaryKeys = options.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName) + .map(_.split(",").filter(_.length > 0)) + ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.") + primaryKeys.get.foreach { primaryKey => + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, primaryKey)), + s"Can't find primary key `$primaryKey` in ${schema.treeString}.") + } + + // validate precombine key + val precombineKey = options.get(SQL_KEY_PRECOMBINE_FIELD.sqlKeyName) + if (precombineKey.isDefined && precombineKey.get.nonEmpty) { + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, precombineKey.get)), + s"Can't find precombine key `${precombineKey.get}` in ${schema.treeString}.") + } + + // validate table type + val tableType = options.get(SQL_KEY_TABLE_TYPE.sqlKeyName) + ValidationUtils.checkArgument(tableType.nonEmpty, "No `type` is specified.") + ValidationUtils.checkArgument( + tableType.get.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW) || + tableType.get.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR), + s"'type' must be '${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW}' or " + + s"'${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR}'") } def buildConf[T](): HoodieOptions[T] = { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala index cf9c49ef02a9c..8e490335080e0 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala @@ -90,7 +90,7 @@ object HoodieSqlUtils extends SparkAdapterSupport { val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) val metadataConfig = { val properties = new Properties() - properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties).asJava) + properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties ++ table.properties).asJava) HoodieMetadataConfig.newBuilder.fromProperties(properties).build() } FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, getTableLocation(table, spark)).asScala diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index 87cbb8a7f0306..1446760a3eb69 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -202,8 +202,9 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi val targetTableId = getMergeIntoTargetTableId(mergeInto) val targetTable = sparkSession.sessionState.catalog.getTableMetadata(targetTableId) - val targetTableType = HoodieOptionConfig.getTableType(targetTable.storage.properties) - val preCombineField = HoodieOptionConfig.getPreCombineField(targetTable.storage.properties) + val tblProperties = targetTable.storage.properties ++ targetTable.properties + val targetTableType = HoodieOptionConfig.getTableType(tblProperties) + val preCombineField = HoodieOptionConfig.getPreCombineField(tblProperties) // Get the map of target attribute to value of the update assignments. val target2Values = resolvedAssignments.map { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala index 4123ea9499a6d..e7d77e7598aec 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala @@ -105,8 +105,13 @@ object AlterHoodieTableAddColumnsCommand { val path = getTableLocation(table, sparkSession) val jsc = new JavaSparkContext(sparkSession.sparkContext) - val client = DataSourceUtils.createHoodieClient(jsc, schema.toString, - path, table.identifier.table, HoodieWriterUtils.parametersWithWriteDefaults(table.storage.properties).asJava) + val client = DataSourceUtils.createHoodieClient( + jsc, + schema.toString, + path, + table.identifier.table, + HoodieWriterUtils.parametersWithWriteDefaults(table.storage.properties ++ table.properties).asJava + ) val hadoopConf = sparkSession.sessionState.newHadoopConf() val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala index 7c4d45649587b..f12f43d389c1f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala @@ -92,7 +92,7 @@ extends RunnableCommand { .build() val tableConfig = metaClient.getTableConfig - val optParams = withSparkConf(sparkSession, table.storage.properties) { + withSparkConf(sparkSession, table.storage.properties) { Map( "path" -> path, TBL_NAME.key -> tableIdentifier.table, @@ -104,10 +104,6 @@ extends RunnableCommand { PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp ) } - - val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams) - val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters) - translatedOptions } def normalizePartitionSpec[T]( diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 38c7e290a2659..2244e72123483 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -19,9 +19,11 @@ package org.apache.spark.sql.hudi.command import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path + import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.hive.util.ConfigUtils import org.apache.hudi.sql.InsertMode + import org.apache.spark.sql.{Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -73,9 +75,10 @@ case class CreateHoodieTableAsSelectCommand( // Execute the insert query try { + val tblProperties = table.storage.properties ++ table.properties val options = Map( DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString, - DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(table.storage.properties.asJava), + DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava), DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava), DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" @@ -88,7 +91,9 @@ case class CreateHoodieTableAsSelectCommand( if (!sparkSession.sessionState.catalog.tableExists(tableIdentWithDB)) { // Create the table val createTableCommand = CreateHoodieTableCommand(tableWithSchema, mode == SaveMode.Ignore) - createTableCommand.createTableInCatalog(sparkSession, checkPathForManagedTable = false) + val path = getTableLocation(table, sparkSession) + val (finalSchema, _, tableSqlOptions) = createTableCommand.parseSchemaAndConfigs(sparkSession, path, ctas = true) + createTableCommand.createTableInCatalog(sparkSession, finalSchema, tableSqlOptions) } } else { // failed to insert data, clear table path clearTablePath(tablePath, hadoopConf) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 8ac63126a4b93..d6c5160897e6e 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -19,13 +19,17 @@ package org.apache.spark.sql.hudi.command import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path + +import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} +import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.common.model.HoodieFileFormat import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import org.apache.hudi.common.util.ValidationUtils import org.apache.hudi.hadoop.HoodieParquetInputFormat import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils -import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} +import org.apache.hudi.keygen.ComplexKeyGenerator +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory + import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.TableIdentifier @@ -36,19 +40,17 @@ import org.apache.spark.sql.hive.HiveClientUtils import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hudi.HoodieOptionConfig import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath} +import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.checkTableConfigEqual import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.{SPARK_VERSION, SparkConf} -import java.util.{Locale, Properties} -import org.apache.hudi.exception.HoodieException -import org.apache.hudi.keygen.ComplexKeyGenerator -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory +import java.util.{Locale, Properties} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.util.control.NonFatal /** * Command for create hoodie table. @@ -56,9 +58,11 @@ import scala.collection.mutable case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean) extends RunnableCommand with SparkAdapterSupport { - override def run(sparkSession: SparkSession): Seq[Row] = { - val tableName = table.identifier.unquotedString + val tableName = formatName(table.identifier.table) + val tblProperties = table.storage.properties ++ table.properties + + override def run(sparkSession: SparkSession): Seq[Row] = { val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier) if (tableIsExists) { if (ignoreIfExists) { @@ -66,64 +70,95 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean return Seq.empty[Row] // scalastyle:on } else { - throw new IllegalArgumentException(s"Table $tableName already exists.") + throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.") } } - // Create table in the catalog - val createTable = createTableInCatalog(sparkSession) + + // get schema with meta fields, table config if hudi table exists, options including + // table configs and properties of the catalog table + val path = getTableLocation(table, sparkSession) + val (finalSchema, existingTableConfig, tableSqlOptions) = parseSchemaAndConfigs(sparkSession, path) + // Init the hoodie.properties - initTableIfNeed(sparkSession, createTable) + initTableIfNeed(sparkSession, path, finalSchema, existingTableConfig, tableSqlOptions) + + try { + // Create table in the catalog + createTableInCatalog(sparkSession, finalSchema, tableSqlOptions) + } catch { + case NonFatal(e) => + logWarning(s"Failed to create catalog table in metastore: ${e.getMessage}") + } + Seq.empty[Row] } - def createTableInCatalog(sparkSession: SparkSession, - checkPathForManagedTable: Boolean = true): CatalogTable = { + def parseSchemaAndConfigs(sparkSession: SparkSession, path: String, ctas: Boolean = false) + : (StructType, Map[String, String], Map[String, String]) = { + val resolver = sparkSession.sessionState.conf.resolver + val conf = sparkSession.sessionState.newHadoopConf + // if CTAS, we treat the table we just created as nonexistent + val isTableExists = if (ctas) false else tableExistsInPath(path, conf) + var existingTableConfig = Map.empty[String, String] + val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(tblProperties) + val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(tblProperties) + + // get final schema and parameters + val (finalSchema, tableSqlOptions) = (table.tableType, isTableExists) match { + case (CatalogTableType.EXTERNAL, true) => + // If this is an external table & the table has already exists in the location, + // load the schema from the table meta. + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(path) + .setConf(conf) + .build() + val tableSchema = getTableSqlSchema(metaClient) + existingTableConfig = metaClient.getTableConfig.getProps.asScala.toMap + validateTableConfig(sparkSession, catalogTableProps, convertMapToHoodieConfig(existingTableConfig)) + + val options = extraTableConfig(sparkSession, isTableExists, existingTableConfig) ++ + sqlOptions ++ HoodieOptionConfig.mappingTableConfigToSqlOption(existingTableConfig) + + val userSpecifiedSchema = table.schema + val schema = if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) { + tableSchema.get + } else if (userSpecifiedSchema.nonEmpty) { + userSpecifiedSchema + } else { + throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName") + } + + (addMetaFields(schema), options) + + case (_, false) => + assert(table.schema.nonEmpty, s"Missing schema for Create Table: $tableName") + val schema = table.schema + val options = extraTableConfig(sparkSession, isTableExists = false) ++ sqlOptions + (addMetaFields(schema), options) + + case (CatalogTableType.MANAGED, true) => + throw new AnalysisException(s"Can not create the managed table('$tableName')" + + s". The associated location('$path') already exists.") + } + HoodieOptionConfig.validateTable(sparkSession, finalSchema, tableSqlOptions) + + val dataSchema = finalSchema.filterNot { f => + table.partitionColumnNames.exists(resolver(_, f.name)) + } + verifyDataSchema(table.identifier, table.tableType, dataSchema) + + (finalSchema, existingTableConfig, tableSqlOptions) + } + + def createTableInCatalog(sparkSession: SparkSession, finalSchema: StructType, + options: Map[String, String]): Unit = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) val sessionState = sparkSession.sessionState - val tableName = table.identifier.unquotedString val path = getTableLocation(table, sparkSession) val conf = sparkSession.sessionState.newHadoopConf() - val isTableExists = tableExistsInPath(path, conf) - // Get the schema & table options - val (newSchema, tableOptions) = if (table.tableType == CatalogTableType.EXTERNAL && - isTableExists) { - // If this is an external table & the table has already exists in the location, - // load the schema from the table meta. - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - val tableSchema = getTableSqlSchema(metaClient) - - // Get options from the external table and append with the options in ddl. - val originTableConfig = HoodieOptionConfig.mappingTableConfigToSqlOption( - metaClient.getTableConfig.getProps.asScala.toMap) - val extraConfig = extraTableConfig(sparkSession, isTableExists, originTableConfig) - val options = originTableConfig ++ table.storage.properties ++ extraConfig - - val userSpecifiedSchema = table.schema - if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) { - (addMetaFields(tableSchema.get), options) - } else if (userSpecifiedSchema.nonEmpty) { - (addMetaFields(userSpecifiedSchema), options) - } else { - throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName") - } - } else { - assert(table.schema.nonEmpty, s"Missing schema for Create Table: $tableName") - // SPARK-19724: the default location of a managed table should be non-existent or empty. - if (checkPathForManagedTable && table.tableType == CatalogTableType.MANAGED - && !isEmptyPath(path, conf)) { - throw new AnalysisException(s"Can not create the managed table('$tableName')" + - s". The associated location('$path') already exists.") - } - // Add the meta fields to the schema if this is a managed table or an empty external table. - val options = table.storage.properties ++ extraTableConfig(sparkSession, false) - (addMetaFields(table.schema), options) - } - val tableType = HoodieOptionConfig.getTableType(table.storage.properties) + val tableType = HoodieOptionConfig.getTableType(options) val inputFormat = tableType match { case DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL => classOf[HoodieParquetInputFormat].getCanonicalName @@ -134,31 +169,39 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val outputFormat = HoodieInputFormatUtils.getOutputFormatClassName(HoodieFileFormat.PARQUET) val serdeFormat = HoodieInputFormatUtils.getSerDeClassName(HoodieFileFormat.PARQUET) - val newStorage = new CatalogStorageFormat(Some(new Path(path).toUri), - Some(inputFormat), Some(outputFormat), Some(serdeFormat), - table.storage.compressed, tableOptions + ("path" -> path)) + // only parameters irrelevant to hudi can be set to storage.properties + val storageProperties = HoodieOptionConfig.deleteHooideOptions(options) + val newStorage = new CatalogStorageFormat( + Some(new Path(path).toUri), + Some(inputFormat), + Some(outputFormat), + Some(serdeFormat), + table.storage.compressed, + storageProperties + ("path" -> path)) val newDatabaseName = formatName(table.identifier.database .getOrElse(sessionState.catalog.getCurrentDatabase)) - val newTableName = formatName(table.identifier.table) val newTableIdentifier = table.identifier - .copy(table = newTableName, database = Some(newDatabaseName)) - - val newTable = table.copy(identifier = newTableIdentifier, - schema = newSchema, storage = newStorage, createVersion = SPARK_VERSION) - // validate the table - validateTable(newTable) + .copy(table = tableName, database = Some(newDatabaseName)) + + // append pk, preCombineKey, type to the properties of table + val newTblProperties = table.storage.properties ++ table.properties ++ HoodieOptionConfig.extractSqlOptions(options) + val newTable = table.copy( + identifier = newTableIdentifier, + schema = finalSchema, + storage = newStorage, + createVersion = SPARK_VERSION, + properties = newTblProperties + ) // Create table in the catalog val enableHive = isEnableHive(sparkSession) if (enableHive) { createHiveDataSourceTable(newTable, sparkSession) } else { - sessionState.catalog.createTable(newTable, ignoreIfExists = false, - validateLocation = checkPathForManagedTable) + sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) } - newTable } /** @@ -170,8 +213,6 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean * @param sparkSession */ private def createHiveDataSourceTable(table: CatalogTable, sparkSession: SparkSession): Unit = { - // check schema - verifyDataSchema(table.identifier, table.tableType, table.schema) val dbName = table.identifier.database.get // check database val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName) @@ -186,7 +227,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val dataSourceProps = tableMetaToTableProps(sparkSession.sparkContext.conf, table, table.schema) - val tableWithDataSourceProps = table.copy(properties = dataSourceProps) + val tableWithDataSourceProps = table.copy(properties = dataSourceProps ++ table.properties) val client = HiveClientUtils.newClientForMetadata(sparkSession.sparkContext.conf, sparkSession.sessionState.newHadoopConf()) // create hive table. @@ -198,9 +239,8 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean } // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema - private def verifyDataSchema(tableName: TableIdentifier, - tableType: CatalogTableType, - dataSchema: StructType): Unit = { + private def verifyDataSchema(tableName: TableIdentifier, tableType: CatalogTableType, + dataSchema: Seq[StructField]): Unit = { if (tableType != CatalogTableType.VIEW) { val invalidChars = Seq(",", ":", ";") def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f => @@ -230,10 +270,10 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean } } } + // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#tableMetaToTableProps - private def tableMetaToTableProps( sparkConf: SparkConf, - table: CatalogTable, - schema: StructType): Map[String, String] = { + private def tableMetaToTableProps(sparkConf: SparkConf, table: CatalogTable, + schema: StructType): Map[String, String] = { val partitionColumns = table.partitionColumnNames val bucketSpec = table.bucketSpec @@ -280,24 +320,6 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean properties.toMap } - private def validateTable(table: CatalogTable): Unit = { - val options = table.storage.properties - // validate the pk if it exist in the table. - HoodieOptionConfig.getPrimaryColumns(options).foreach(pk => table.schema.fieldIndex(pk)) - // validate the version column if it exist in the table. - HoodieOptionConfig.getPreCombineField(options).foreach(v => table.schema.fieldIndex(v)) - // validate the partition columns - table.partitionColumnNames.foreach(p => table.schema.fieldIndex(p)) - // validate table type - options.get(HoodieOptionConfig.SQL_KEY_TABLE_TYPE.sqlKeyName).foreach { tableType => - ValidationUtils.checkArgument( - tableType.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW) || - tableType.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR), - s"'type' must be '${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW}' or " + - s"'${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR}'") - } - } - def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean, originTableConfig: Map[String, String] = Map.empty): Map[String, String] = { val extraConfig = mutable.Map.empty[String, String] @@ -322,10 +344,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue() } - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(originTableConfig ++ table.storage.properties) - if (primaryColumns.isEmpty) { - extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[UuidKeyGenerator].getCanonicalName - } else if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) { + if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) { extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator( originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) @@ -334,31 +353,20 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean } extraConfig.toMap } -} - -object CreateHoodieTableCommand extends Logging { /** * Init the hoodie.properties. */ - def initTableIfNeed(sparkSession: SparkSession, table: CatalogTable): Unit = { - val location = getTableLocation(table, sparkSession) + def initTableIfNeed(sparkSession: SparkSession, + location: String, + schema: StructType, + originTableConfig: Map[String, String], + sqlOptions: Map[String, String]): Unit = { + logInfo(s"Init hoodie.properties for $tableName") val conf = sparkSession.sessionState.newHadoopConf() - // Init the hoodie table - val originTableConfig = if (tableExistsInPath(location, conf)) { - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(location) - .setConf(conf) - .build() - metaClient.getTableConfig.getProps.asScala.toMap - } else { - Map.empty[String, String] - } - val tableName = table.identifier.table - logInfo(s"Init hoodie.properties for $tableName") - val tableOptions = HoodieOptionConfig.mappingSqlOptionToTableConfig(table.storage.properties) + val tableOptions = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key) @@ -372,10 +380,13 @@ object CreateHoodieTableCommand extends Logging { HoodieTableMetaClient.withPropertyBuilder() .fromProperties(properties) .setTableName(tableName) - .setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString()) + .setTableCreateSchema(SchemaConverters.toAvroType(schema).toString()) .setPartitionFields(table.partitionColumnNames.mkString(",")) .initTable(conf, location) } +} + +object CreateHoodieTableCommand extends Logging { def checkTableConfigEqual(originTableConfig: Map[String, String], newTableConfig: Map[String, String], configKey: String): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala index 987ce0e050be4..e8acebd787073 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala @@ -23,11 +23,12 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hudi.HoodieOptionConfig import org.apache.spark.sql.hudi.HoodieSqlUtils._ +import org.apache.spark.sql.types.StructType case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends RunnableCommand with SparkAdapterSupport { @@ -56,8 +57,8 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab } private def buildHoodieConfig(sparkSession: SparkSession): Map[String, String] = { - val targetTable = sparkSession.sessionState.catalog - .getTableMetadata(tableId) + val targetTable = sparkSession.sessionState.catalog.getTableMetadata(tableId) + val tblProperties = targetTable.storage.properties ++ targetTable.properties val path = getTableLocation(targetTable, sparkSession) val conf = sparkSession.sessionState.newHadoopConf() val metaClient = HoodieTableMetaClient.builder() @@ -65,23 +66,27 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab .setConf(conf) .build() val tableConfig = metaClient.getTableConfig - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) + val tableSchema = getTableSqlSchema(metaClient).get + val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) + val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + val primaryColumns = tableConfig.getRecordKeyFields.get() assert(primaryColumns.nonEmpty, s"There are no primary key defined in table $tableId, cannot execute delete operator") - withSparkConf(sparkSession, targetTable.storage.properties) { + withSparkConf(sparkSession, tblProperties) { Map( "path" -> path, TBL_NAME.key -> tableId.table, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala index 2b88373115b71..ac3fce5315680 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema import org.apache.avro.generic.{GenericRecord, IndexedRecord} + import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} @@ -30,7 +31,8 @@ import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.sql.InsertMode -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils} +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter} + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} @@ -40,10 +42,13 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.hudi.HoodieSqlUtils._ import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} import java.util.Properties +import scala.collection.JavaConverters._ + /** * Command for insert into hoodie table. */ @@ -194,46 +199,39 @@ object InsertIntoHoodieTableCommand extends Logging { s"[${insertPartitions.keys.mkString(" " )}]" + s" not equal to the defined partition in table[${table.partitionColumnNames.mkString(",")}]") } - val options = table.storage.properties ++ extraOptions - val parameters = withSparkConf(sparkSession, options)() - - val tableType = parameters.getOrElse(TABLE_TYPE.key, TABLE_TYPE.defaultValue) - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(options) - val partitionFields = table.partitionColumnNames.mkString(",") - val path = getTableLocation(table, sparkSession) val conf = sparkSession.sessionState.newHadoopConf() val isTableExists = tableExistsInPath(path, conf) - val tableConfig = if (isTableExists) { - HoodieTableMetaClient.builder() + val (tableConfig, tableSchema) = if (isTableExists) { + val metaClient = HoodieTableMetaClient.builder() .setBasePath(path) .setConf(conf) .build() - .getTableConfig - } else { - null - } - val hiveStylePartitioningEnable = if (null == tableConfig || null == tableConfig.getHiveStylePartitioningEnable) { - "true" + (metaClient.getTableConfig, getTableSqlSchema(metaClient).get) } else { - tableConfig.getHiveStylePartitioningEnable + (new HoodieTableConfig(), table.schema) } - val urlEncodePartitioning = if (null == tableConfig || null == tableConfig.getUrlEncodePartitoning) { - "false" + val partitionColumns = tableConfig.getPartitionFieldProp + val partitionSchema = if (null == partitionColumns || partitionColumns.isEmpty) { + table.partitionSchema } else { - tableConfig.getUrlEncodePartitoning - } - val keyGeneratorClassName = if (null == tableConfig || null == tableConfig.getKeyGeneratorClassName) { - if (primaryColumns.nonEmpty) { - classOf[ComplexKeyGenerator].getCanonicalName - } else { - classOf[UuidKeyGenerator].getCanonicalName - } - } else { - tableConfig.getKeyGeneratorClassName + StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) } - val tableSchema = table.schema + val options = table.storage.properties ++ table.properties ++ tableConfig.getProps.asScala.toMap ++ extraOptions + val parameters = withSparkConf(sparkSession, options)() + + val tableName = Option(tableConfig.getTableName).getOrElse(table.identifier.table) + val tableType = Option(tableConfig.getTableType.name).getOrElse(TABLE_TYPE.defaultValue) + val primaryColumns = tableConfig.getRecordKeyFields.orElse(HoodieOptionConfig.getPrimaryColumns(options)) + val preCombineColumn = Option(tableConfig.getPreCombineField) + .getOrElse(HoodieOptionConfig.getPreCombineField(options).getOrElse("")) + val partitionFields = Option(tableConfig.getPartitionFieldProp) + .getOrElse(table.partitionColumnNames.mkString(",")) + val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true") + val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitoning).getOrElse("false") + val keyGeneratorClassName = Option(tableConfig.getKeyGeneratorClassName) + .getOrElse(classOf[ComplexKeyGenerator].getCanonicalName) val dropDuplicate = sparkSession.conf .getOption(INSERT_DROP_DUPS.key) @@ -242,35 +240,33 @@ object InsertIntoHoodieTableCommand extends Logging { val enableBulkInsert = parameters.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key, DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean + val hasPrecombineColumn = preCombineColumn.nonEmpty val isPartitionedTable = table.partitionColumnNames.nonEmpty - val isPrimaryKeyTable = primaryColumns.nonEmpty val insertMode = InsertMode.of(parameters.getOrElse(DataSourceWriteOptions.SQL_INSERT_MODE.key, DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue())) val isNonStrictMode = insertMode == InsertMode.NON_STRICT val operation = - (isPrimaryKeyTable, enableBulkInsert, isOverwrite, dropDuplicate) match { - case (true, true, _, _) if !isNonStrictMode => + (enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match { + case (true, _, _, false, _) => throw new IllegalArgumentException(s"Table with primaryKey can not use bulk insert in ${insertMode.value()} mode.") - case (_, true, true, _) if isPartitionedTable => + case (true, true, _, _, true) => throw new IllegalArgumentException(s"Insert Overwrite Partition can not use bulk insert.") - case (_, true, _, true) => + case (true, _, true, _, _) => throw new IllegalArgumentException(s"Bulk insert cannot support drop duplication." + s" Please disable $INSERT_DROP_DUPS and try again.") // if enableBulkInsert is true, use bulk insert for the insert overwrite non-partitioned table. - case (_, true, true, _) if !isPartitionedTable => BULK_INSERT_OPERATION_OPT_VAL - // insert overwrite partition - case (_, _, true, _) if isPartitionedTable => INSERT_OVERWRITE_OPERATION_OPT_VAL + case (true, true, _, _, false) => BULK_INSERT_OPERATION_OPT_VAL // insert overwrite table - case (_, _, true, _) if !isPartitionedTable => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL - // if it is pk table and the dropDuplicate has disable, use the upsert operation for strict and upsert mode. - case (true, false, false, false) if !isNonStrictMode => UPSERT_OPERATION_OPT_VAL - // if enableBulkInsert is true and the table is non-primaryKeyed, use the bulk insert operation - case (false, true, _, _) => BULK_INSERT_OPERATION_OPT_VAL + case (false, true, _, _, false) => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL + // insert overwrite partition + case (_, true, _, _, true) => INSERT_OVERWRITE_OPERATION_OPT_VAL + // disable dropDuplicate, and provide preCombineKey, use the upsert operation for strict and upsert mode. + case (false, false, false, false, _) if hasPrecombineColumn => UPSERT_OPERATION_OPT_VAL // if table is pk table and has enableBulkInsert use bulk insert for non-strict mode. - case (true, true, _, _) if isNonStrictMode => BULK_INSERT_OPERATION_OPT_VAL + case (true, _, _, true, _) => BULK_INSERT_OPERATION_OPT_VAL // for the rest case, use the insert operation - case (_, _, _, _) => INSERT_OPERATION_OPT_VAL + case _ => INSERT_OPERATION_OPT_VAL } val payloadClassName = if (operation == UPSERT_OPERATION_OPT_VAL && @@ -288,17 +284,18 @@ object InsertIntoHoodieTableCommand extends Logging { Map( "path" -> path, TABLE_TYPE.key -> tableType, - TBL_NAME.key -> table.identifier.table, - PRECOMBINE_FIELD.key -> tableSchema.fields.last.name, + TBL_NAME.key -> tableName, + PRECOMBINE_FIELD.key -> preCombineColumn, OPERATION.key -> operation, HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning, - KEYGENERATOR_CLASS_NAME.key -> keyGeneratorClassName, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> keyGeneratorClassName, RECORDKEY_FIELD.key -> primaryColumns.mkString(","), PARTITIONPATH_FIELD.key -> partitionFields, PAYLOAD_CLASS_NAME.key -> payloadClassName, ENABLE_ROW_WRITER.key -> enableBulkInsert.toString, - HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> isPrimaryKeyTable.toString, + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 5ec15ce4d84fd..251ebc32d828b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.MultiPartKeysValueExtractor @@ -80,8 +81,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab private lazy val targetTable = sparkSession.sessionState.catalog.getTableMetadata(targetTableIdentify) - private lazy val targetTableType = - HoodieOptionConfig.getTableType(targetTable.storage.properties) + private lazy val tblProperties = targetTable.storage.properties ++ targetTable.properties + + private lazy val targetTableType = HoodieOptionConfig.getTableType(tblProperties) /** * @@ -124,7 +126,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab assert(updateActions.size <= 1, s"Only support one updateAction currently, current update action count is: ${updateActions.size}") val updateAction = updateActions.headOption - HoodieOptionConfig.getPreCombineField(targetTable.storage.properties).map(preCombineField => { + HoodieOptionConfig.getPreCombineField(tblProperties).map(preCombineField => { val sourcePreCombineField = updateAction.map(u => u.assignments.filter { case Assignment(key: AttributeReference, _) => key.name.equalsIgnoreCase(preCombineField) @@ -242,8 +244,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab // Append the table schema to the parameters. In the case of merge into, the schema of sourceDF // may be different from the target table, because the are transform logical in the update or // insert actions. + val operation = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) { + INSERT_OPERATION_OPT_VAL + } else { + UPSERT_OPERATION_OPT_VAL + } var writeParams = parameters + - (OPERATION.key -> UPSERT_OPERATION_OPT_VAL) + + (OPERATION.key -> operation) + (HoodieWriteConfig.WRITE_SCHEMA.key -> getTableSchema.toString) + (DataSourceWriteOptions.TABLE_TYPE.key -> targetTableType) @@ -436,38 +443,38 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab .setConf(conf) .build() val tableConfig = metaClient.getTableConfig - val options = targetTable.storage.properties - val definedPk = HoodieOptionConfig.getPrimaryColumns(options) - // TODO Currently the mergeEqualConditionKeys must be the same the primary key. - if (targetKey2SourceExpression.keySet != definedPk.toSet) { - throw new IllegalArgumentException(s"Merge Key[${targetKey2SourceExpression.keySet.mkString(",")}] is not" + - s" Equal to the defined primary key[${definedPk.mkString(",")}] in table $targetTableName") - } + val tableSchema = getTableSqlSchema(metaClient).get + val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) + val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + val options = tblProperties + val preCombineColumn = Option(tableConfig.getPreCombineField).getOrElse("") + // Enable the hive sync by default if spark have enable the hive metastore. val enableHive = isEnableHive(sparkSession) withSparkConf(sparkSession, options) { Map( "path" -> path, - RECORDKEY_FIELD.key -> targetKey2SourceExpression.keySet.mkString(","), - PRECOMBINE_FIELD.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field + RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp, + PRECOMBINE_FIELD.key -> preCombineColumn, TBL_NAME.key -> targetTableName, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", HIVE_DATABASE.key -> targetTableDb, HIVE_TABLE.key -> targetTableName, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", - HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), + HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala index b1c8a04429e27..0c7d1ef0b4cbe 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala @@ -18,20 +18,20 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.ddl.HiveSyncMode -import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Assignment, UpdateTable} import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hudi.HoodieOptionConfig import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.types.{StructField, StructType} import scala.collection.JavaConverters._ @@ -83,8 +83,8 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo } private def buildHoodieConfig(sparkSession: SparkSession): Map[String, String] = { - val targetTable = sparkSession.sessionState.catalog - .getTableMetadata(tableId) + val targetTable = sparkSession.sessionState.catalog.getTableMetadata(tableId) + val tblProperties = targetTable.storage.properties ++ targetTable.properties val path = getTableLocation(targetTable, sparkSession) val conf = sparkSession.sessionState.newHadoopConf() val metaClient = HoodieTableMetaClient.builder() @@ -92,32 +92,37 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo .setConf(conf) .build() val tableConfig = metaClient.getTableConfig - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) - + val tableSchema = getTableSqlSchema(metaClient).get + val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) + val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + val primaryColumns = tableConfig.getRecordKeyFields.get() + val preCombineColumn = Option(tableConfig.getPreCombineField).getOrElse("") assert(primaryColumns.nonEmpty, s"There are no primary key in table $tableId, cannot execute update operator") val enableHive = isEnableHive(sparkSession) - withSparkConf(sparkSession, targetTable.storage.properties) { + + withSparkConf(sparkSession, tblProperties) { Map( "path" -> path, RECORDKEY_FIELD.key -> primaryColumns.mkString(","), - PRECOMBINE_FIELD.key -> primaryColumns.head, //set the default preCombine field. + PRECOMBINE_FIELD.key -> preCombineColumn, TBL_NAME.key -> tableId.table, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, - OPERATION.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, + OPERATION.key -> UPSERT_OPERATION_OPT_VAL, + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", HIVE_DATABASE.key -> tableId.database.getOrElse("default"), HIVE_TABLE.key -> tableId.table, - HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), + HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index 0557d70405c4c..9372a36f4d78e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -206,7 +206,7 @@ public void run() throws Exception { .option(DataSourceWriteOptions.OPERATION().key(), "delete") .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "_row_key") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp") .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName() : SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql index 280fde59ff30a..135c83b4b975e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql @@ -18,7 +18,7 @@ set hoodie.delete.shuffle.parallelism = 1; # CTAS -create table h0 using hudi options(type = '${tableType}') +create table h0 using hudi options(type = '${tableType}', primaryKey = 'id') as select 1 as id, 'a1' as name, 10 as price; +----------+ | ok | @@ -30,7 +30,7 @@ select id, name, price from h0; +-----------+ create table h0_p using hudi partitioned by(dt) -options(type = '${tableType}') +options(type = '${tableType}', primaryKey = 'id') as select cast('2021-05-07 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as price; +----------+ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala new file mode 100644 index 0000000000000..e64f96ff8d0f4 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala @@ -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.hudi + +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} + +import org.apache.spark.sql.hudi.command.SqlKeyGenerator + +import org.junit.jupiter.api.Assertions.assertTrue +import org.junit.jupiter.api.Test + +class HoodieSparkSqlWriterSuite2 { + + @Test + def testGetOriginKeyGenerator(): Unit = { + // for dataframe write + val m1 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[ComplexKeyGenerator].getName + ) + val kg1 = HoodieWriterUtils.getOriginKeyGenerator(m1) + assertTrue(kg1 == classOf[ComplexKeyGenerator].getName) + + // for sql write + val m2 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> classOf[SimpleKeyGenerator].getName + ) + val kg2 = HoodieWriterUtils.getOriginKeyGenerator(m2) + assertTrue(kg2 == classOf[SimpleKeyGenerator].getName) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala index d6ae80d09af58..a4c98665ebdba 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala @@ -102,7 +102,10 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit( + DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, + extraOpts = Map(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator") + ) // Read bootstrapped table and verify count var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala index e1bc4a1f71764..e2521047f68cf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala @@ -38,7 +38,7 @@ class TestAlterTable extends TestHoodieSqlBase { | ts long |) using hudi | location '$tablePath' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' @@ -127,7 +127,7 @@ class TestAlterTable extends TestHoodieSqlBase { | dt string |) using hudi | location '${tmp.getCanonicalPath}/$partitionedTable' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala index b2ada77c21941..8cc092c2e5f08 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala @@ -38,7 +38,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { | dt string | ) | using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -77,7 +77,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -105,7 +105,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { | dt string | ) | using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -151,7 +151,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala index e40a484210188..20238a6e4318d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala @@ -31,7 +31,7 @@ class TestCompactionTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts' @@ -82,7 +82,7 @@ class TestCompactionTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 2af8fd782cb86..14bc70428df86 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -43,7 +43,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long | ) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -62,6 +62,53 @@ class TestCreateTable extends TestHoodieSqlBase { )(table.schema.fields) } + test("Test Create Hoodie Table With Options") { + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | dt string + | ) using hudi + | partitioned by (dt) + | options ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assertResult(table.properties("type"))("cow") + assertResult(table.properties("primaryKey"))("id") + assertResult(table.properties("preCombineField"))("ts") + assertResult(tableName)(table.identifier.table) + assertResult("hudi")(table.provider.get) + assertResult(CatalogTableType.MANAGED)(table.tableType) + assertResult( + HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType)) + ++ Seq( + StructField("id", IntegerType), + StructField("name", StringType), + StructField("price", DoubleType), + StructField("ts", LongType), + StructField("dt", StringType)) + )(table.schema.fields) + + val tablePath = table.storage.properties("path") + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(tablePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + val tableConfig = metaClient.getTableConfig.getProps.asScala.toMap + assertResult(true)(tableConfig.contains(HoodieTableConfig.CREATE_SCHEMA.key)) + assertResult("dt")(tableConfig(HoodieTableConfig.PARTITION_FIELDS.key)) + assertResult("id")(tableConfig(HoodieTableConfig.RECORDKEY_FIELDS.key)) + assertResult("ts")(tableConfig(HoodieTableConfig.PRECOMBINE_FIELD.key)) + assertResult(classOf[ComplexKeyGenerator].getCanonicalName)(tableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) + } + test("Test Create External Hoodie Table") { withTempDir { tmp => // Test create cow table. @@ -74,7 +121,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id,name', | type = 'cow' | ) @@ -93,8 +140,8 @@ class TestCreateTable extends TestHoodieSqlBase { StructField("price", DoubleType), StructField("ts", LongType)) )(table.schema.fields) - assertResult(table.storage.properties("type"))("cow") - assertResult(table.storage.properties("primaryKey"))("id,name") + assertResult(table.properties("type"))("cow") + assertResult(table.properties("primaryKey"))("id,name") spark.sql(s"drop table $tableName") // Test create mor partitioned table @@ -108,15 +155,15 @@ class TestCreateTable extends TestHoodieSqlBase { | dt string |) using hudi | partitioned by (dt) - | options ( + | tblproperties ( | primaryKey = 'id', | type = 'mor' | ) | location '${tmp.getCanonicalPath}/h0' """.stripMargin) val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - assertResult(table2.storage.properties("type"))("mor") - assertResult(table2.storage.properties("primaryKey"))("id") + assertResult(table2.properties("type"))("mor") + assertResult(table2.properties("primaryKey"))("id") assertResult(Seq("dt"))(table2.partitionColumnNames) assertResult(classOf[HoodieParquetRealtimeInputFormat].getCanonicalName)(table2.storage.inputFormat.get) @@ -129,8 +176,8 @@ class TestCreateTable extends TestHoodieSqlBase { |location '${tmp.getCanonicalPath}/h0' """.stripMargin) val table3 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName3)) - assertResult(table3.storage.properties("type"))("mor") - assertResult(table3.storage.properties("primaryKey"))("id") + assertResult(table3.properties("type"))("mor") + assertResult(table3.properties("primaryKey"))("id") assertResult( HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType)) ++ Seq( @@ -156,7 +203,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id1', | type = 'cow' | ) @@ -173,7 +220,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts1', | type = 'cow' @@ -191,7 +238,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts', | type = 'cow1' @@ -208,7 +255,8 @@ class TestCreateTable extends TestHoodieSqlBase { val tableName1 = generateTableName spark.sql( s""" - |create table $tableName1 using hudi + | create table $tableName1 using hudi + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName1' | AS | select 1 as id, 'a1' as name, 10 as price, 1000 as ts @@ -223,6 +271,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName2 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName2' | AS | select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt @@ -240,7 +289,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName3 using hudi | partitioned by (dt) - | options(primaryKey = 'id') + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName3' | AS | select null as id, 'a1' as name, 10 as price, '2021-05-07' as dt @@ -252,6 +301,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName3 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName3' | AS | select cast('2021-05-06 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as @@ -267,6 +317,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName4 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName4' | AS | select cast('2021-05-06' as date) as dt, 1 as id, 'a1' as name, 10 as @@ -303,7 +354,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -380,7 +431,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -455,7 +506,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -514,6 +565,7 @@ class TestCreateTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + |tblproperties(primaryKey = 'id') |""".stripMargin ) @@ -527,6 +579,7 @@ class TestCreateTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + |tblproperties(primaryKey = 'id') |""".stripMargin ) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala index 9ad717aba457f..6137c4c6394b0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala @@ -33,7 +33,7 @@ class TestDeleteTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala new file mode 100644 index 0000000000000..f91388ae9a10e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -0,0 +1,184 @@ +/* + * 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.spark.sql.hudi + +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.testutils.HoodieClientTestBase + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types._ + +import org.junit.jupiter.api.Assertions.assertTrue +import org.junit.jupiter.api.{BeforeEach, Test} + +import org.scalatest.Matchers.intercept + +class TestHoodieOptionConfig extends HoodieClientTestBase { + + var spark: SparkSession = _ + + /** + * Setup method running before each test. + */ + @BeforeEach override def setUp() { + initSparkContexts() + spark = sqlContext.sparkSession + } + + @Test + def testWithDefaultSqlOptions(): Unit = { + val ops1 = Map("primaryKey" -> "id") + val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1) + assertTrue(with1.size == 3) + assertTrue(with1("primaryKey") == "id") + assertTrue(with1("type") == "cow") + assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName) + + val ops2 = Map("primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "mor", + "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName + ) + val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2) + assertTrue(ops2 == with2) + } + + @Test + def testMappingSqlOptionToTableConfig(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true" + ) + val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) + + assertTrue(tableConfigs.size == 5) + assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr") + assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp") + assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ") + assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY") + assertTrue(tableConfigs("hoodie.compact.inline") == "true") + } + + @Test + def testDeleteHooideOptions(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true", + "key123" -> "value456" + ) + val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions) + assertTrue(tableConfigs.size == 1) + assertTrue(tableConfigs("key123") == "value456") + } + + @Test + def testExtractSqlOptions(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true", + "key123" -> "value456" + ) + val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions) + assertTrue(tableConfigs.size == 3) + assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type")) + } + + @Test + def testValidateTable(): Unit = { + val baseSqlOptions = Map( + "hoodie.datasource.write.hive_style_partitioning" -> "true", + "hoodie.datasource.write.partitionpath.urlencode" -> "false", + "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator" + ) + + val schema = StructType( + Seq(StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("timestamp", TimestampType, true), + StructField("dt", StringType, true)) + ) + + // miss primaryKey parameter + val sqlOptions1 = baseSqlOptions ++ Map( + "type" -> "mor" + ) + + val e1 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions1) + } + assertTrue(e1.getMessage.contains("No `primaryKey` is specified.")) + + // primary field not found + val sqlOptions2 = baseSqlOptions ++ Map( + "primaryKey" -> "xxx", + "type" -> "mor" + ) + val e2 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions2) + } + assertTrue(e2.getMessage.contains("Can't find primary key")) + + // preCombine field not found + val sqlOptions3 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "ts", + "type" -> "mor" + ) + val e3 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions3) + } + assertTrue(e3.getMessage.contains("Can't find precombine key")) + + // miss type parameter + val sqlOptions4 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp" + ) + val e4 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions4) + } + assertTrue(e4.getMessage.contains("No `type` is specified.")) + + // type is invalid + val sqlOptions5 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "abc" + ) + val e5 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions5) + } + assertTrue(e5.getMessage.contains("'type' must be 'cow' or 'mor'")) + + // right options and schema + val sqlOptions6 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "cow" + ) + HoodieOptionConfig.validateTable(spark, schema, sqlOptions6) + } + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 337317b8ef7b9..09711cc8f1bec 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -36,6 +36,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}' """.stripMargin) @@ -75,7 +76,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = 'cow', | primaryKey = 'id', | preCombineField = 'ts' @@ -115,7 +116,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName2' - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -146,6 +147,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' """.stripMargin) @@ -191,6 +193,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | ts long | ) using hudi + | tblproperties (primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tblNonPartition' """.stripMargin) spark.sql(s"insert into $tblNonPartition select 1, 'a1', 10, 1000") @@ -245,6 +248,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt $partitionType |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' """.stripMargin) @@ -273,6 +277,7 @@ class TestInsertTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + | tblproperties (primaryKey = 'id') | location '${tmp.getCanonicalPath}' """.stripMargin) @@ -293,6 +298,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) """.stripMargin) checkException(s"insert into $tableName partition(dt = '2021-06-20')" + @@ -305,7 +311,7 @@ class TestInsertTable extends TestHoodieSqlBase { " count: 3,columns: (1,a1,10)" ) spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql("set hoodie.sql.insert.mode= strict") + spark.sql("set hoodie.sql.insert.mode = strict") val tableName2 = generateTableName spark.sql( @@ -316,7 +322,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -325,6 +331,7 @@ class TestInsertTable extends TestHoodieSqlBase { "Table with primaryKey can not use bulk insert in strict mode." ) + spark.sql("set hoodie.sql.insert.mode = non-strict") val tableName3 = generateTableName spark.sql( s""" @@ -334,16 +341,18 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) """.stripMargin) checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( "Insert Overwrite Partition can not use bulk insert." ) spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql("set hoodie.sql.insert.mode= upsert") + spark.sql("set hoodie.sql.insert.mode = upsert") } test("Test bulk insert") { + spark.sql("set hoodie.sql.insert.mode = non-strict") withTempDir { tmp => Seq("cow", "mor").foreach {tableType => // Test bulk insert for single partition @@ -356,8 +365,9 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' @@ -391,8 +401,9 @@ class TestInsertTable extends TestHoodieSqlBase { | dt string, | hh string |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | partitioned by (dt, hh) | location '${tmp.getCanonicalPath}/$tableMultiPartition' @@ -423,8 +434,9 @@ class TestInsertTable extends TestHoodieSqlBase { | name string, | price double |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | location '${tmp.getCanonicalPath}/$nonPartitionedTable' """.stripMargin) @@ -445,7 +457,7 @@ class TestInsertTable extends TestHoodieSqlBase { s""" |create table $tableName2 |using hudi - |options( + |tblproperties( | type = '$tableType', | primaryKey = 'id' |) @@ -459,9 +471,11 @@ class TestInsertTable extends TestHoodieSqlBase { ) } } + spark.sql("set hoodie.sql.insert.mode = upsert") } test("Test combine before insert") { + spark.sql("set hoodie.sql.bulk.insert.enable = false") withTempDir{tmp => val tableName = generateTableName spark.sql( @@ -473,7 +487,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -495,6 +509,7 @@ class TestInsertTable extends TestHoodieSqlBase { } test("Test insert pk-table") { + spark.sql("set hoodie.sql.bulk.insert.enable = false") withTempDir{tmp => val tableName = generateTableName spark.sql( @@ -506,7 +521,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala index d911ace62a23e..5139825f9428f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala @@ -34,7 +34,7 @@ class TestMergeIntoLogOnlyTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts', diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala index bd8558710d0fb..baac82f4bd153 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala @@ -35,7 +35,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -137,7 +137,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$targetTable' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -203,7 +203,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -313,7 +313,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id' | ) @@ -369,7 +369,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | v long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'v' @@ -439,7 +439,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -457,7 +457,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -553,7 +553,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | c $dataType |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'c' | ) @@ -604,7 +604,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts', @@ -665,7 +665,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -711,7 +711,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala index bf73251e947d7..072a1257d4069 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala @@ -35,7 +35,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -145,7 +145,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - |options(primaryKey = 'id') + |tblproperties(primaryKey = 'id') |location '${tmp.getCanonicalPath}' |as |select 1 as id, 'a1' as name @@ -187,7 +187,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | m_value map, | ts long | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -251,7 +251,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | dt string |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -333,7 +333,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -376,7 +376,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala index 357954ebb1d57..2524d04ec81fb 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala @@ -32,7 +32,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -60,7 +60,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | name string, | price double |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id' |) @@ -92,7 +92,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type = 'cow', | primaryKey = 'id', | preCombineField = '_ts' @@ -117,7 +117,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type = 'mor', | primaryKey = 'id', | preCombineField = '_ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala index 05ee61c4879fd..19d8d0345c4d4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala @@ -32,7 +32,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | price double, | ts long |) using hudi - |options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' ) @@ -59,7 +59,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | dt string ) using hudi | partitioned by (dt) - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -109,7 +109,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | day string | ) using hudi | partitioned by (year, month, day) - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 8e9c81b12cb7a..2537c9c807a8f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -33,7 +33,7 @@ class TestUpdateTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' From a14d1040b909658920a8c95a9b9914d0806ca6fd Mon Sep 17 00:00:00 2001 From: Manoj Govindassamy Date: Sun, 14 Nov 2021 17:47:41 -0800 Subject: [PATCH 007/217] [HUDI-2589] Claiming RFC-37 for Metadata based bloom index feature. (#3995) --- rfc/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rfc/README.md b/rfc/README.md index 5ef97300fcc35..0ef9f7ae313f6 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -59,4 +59,5 @@ The list of all RFCs can be found here. | 33 | [Hudi supports more comprehensive Schema Evolution](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+33++Hudi+supports+more+comprehensive+Schema+Evolution) | `IN PROGRESS` | | 34 | [Hudi BigQuery Integration (WIP)](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=188745980) | `UNDER REVIEW` | | 35 | [Make Flink MOR table writing streaming friendly](https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly) | `UNDER REVIEW` | -| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | \ No newline at end of file +| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | +| 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | \ No newline at end of file From a0dae41409a4f2d509aae1b16a4b509ec774c454 Mon Sep 17 00:00:00 2001 From: xiarixiaoyao Date: Mon, 15 Nov 2021 11:29:40 +0800 Subject: [PATCH 008/217] [HUDI-2758] remove redundant code in the hoodieRealtimeInputFormatUitls.getRealtimeSplits (#3994) --- .../hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 9140969c60212..11ce0a66ad330 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -125,10 +125,6 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream Date: Mon, 15 Nov 2021 14:39:43 +0800 Subject: [PATCH 009/217] [MINOR] Fix typo in IntervalTreeBasedGlobalIndexFileFilter (#3993) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 闫杜峰 --- .../index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java index 50d31f9cc50d2..18b094890081b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java @@ -46,8 +46,8 @@ class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter { IntervalTreeBasedGlobalIndexFileFilter(final Map> partitionToFileIndexInfo) { List allIndexFiles = new ArrayList<>(); - partitionToFileIndexInfo.forEach((parition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> { - fileIdToPartitionPathMap.put(file.getFileId(), parition); + partitionToFileIndexInfo.forEach((partition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> { + fileIdToPartitionPathMap.put(file.getFileId(), partition); allIndexFiles.add(file); })); From 53d2d6ae241f86aaf337abc50d3f00232fa87d00 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 15 Nov 2021 07:27:35 -0500 Subject: [PATCH 010/217] [HUDI-2744] Fix parsing of metadadata table compaction timestamp when metrics are enabled (#3976) --- .../functional/TestHoodieBackedMetadata.java | 48 ++++++++++++------- .../functional/TestHoodieMetadataBase.java | 7 ++- .../timeline/HoodieInstantTimeGenerator.java | 10 +++- .../metadata/HoodieBackedTableMetadata.java | 3 +- .../timeline/TestHoodieActiveTimeline.java | 14 ++++++ 5 files changed, 59 insertions(+), 23 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 9ece523ff472b..49eddc24ef0d8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -223,7 +223,7 @@ public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Excep @ParameterizedTest @MethodSource("bootstrapAndTableOperationTestArgs") public void testTableOperations(HoodieTableType tableType, boolean enableFullScan) throws Exception { - init(tableType, true, enableFullScan); + init(tableType, true, enableFullScan, false); doWriteInsertAndUpsert(testTable); // trigger an upsert @@ -462,27 +462,43 @@ public void testSync(HoodieTableType tableType) throws Exception { validateMetadata(testTable, emptyList(), true); } + /** + * Fetches next commit time in seconds from current one. + * + * @param curCommitTime current commit time. + * @return the next valid commit time. + */ + private Long getNextCommitTime(long curCommitTime) { + if ((curCommitTime + 1) % 1000000000000L >= 60) { // max seconds is 60 and hence + return Long.parseLong(HoodieActiveTimeline.createNewInstantTime()); + } else { + return curCommitTime + 1; + } + } + @ParameterizedTest @EnumSource(HoodieTableType.class) public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType) throws Exception { - init(tableType); + init(tableType, true, true, true); + long baseCommitTime = Long.parseLong(HoodieActiveTimeline.createNewInstantTime()); for (int i = 1; i < 25; i += 7) { - String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i; - String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1); - String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2); - String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3); - String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4); - String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5); - String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6); - doWriteOperation(testTable, commitTime1, INSERT); - doWriteOperation(testTable, commitTime2); - doClean(testTable, commitTime3, Arrays.asList(commitTime1)); - doWriteOperation(testTable, commitTime4); + long commitTime1 = getNextCommitTime(baseCommitTime); + long commitTime2 = getNextCommitTime(commitTime1); + long commitTime3 = getNextCommitTime(commitTime2); + long commitTime4 = getNextCommitTime(commitTime3); + long commitTime5 = getNextCommitTime(commitTime4); + long commitTime6 = getNextCommitTime(commitTime5); + long commitTime7 = getNextCommitTime(commitTime6); + baseCommitTime = commitTime7; + doWriteOperation(testTable, Long.toString(commitTime1), INSERT); + doWriteOperation(testTable, Long.toString(commitTime2)); + doClean(testTable, Long.toString(commitTime3), Arrays.asList(Long.toString(commitTime1))); + doWriteOperation(testTable, Long.toString(commitTime4)); if (tableType == MERGE_ON_READ) { - doCompaction(testTable, commitTime5); + doCompaction(testTable, Long.toString(commitTime5)); } - doWriteOperation(testTable, commitTime6); - doRollback(testTable, commitTime6, commitTime7); + doWriteOperation(testTable, Long.toString(commitTime6)); + doRollback(testTable, Long.toString(commitTime6), Long.toString(commitTime7)); } validateMetadata(testTable, emptyList(), true); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index cf261cc8994a1..5e4c4ba86632a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -75,10 +75,10 @@ public void init(HoodieTableType tableType) throws IOException { } public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException { - init(tableType, enableMetadataTable, true); + init(tableType, enableMetadataTable, true, false); } - public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan) throws IOException { + public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics) throws IOException { this.tableType = tableType; initPath(); initSparkContexts("TestHoodieMetadata"); @@ -87,8 +87,7 @@ public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean initMetaClient(tableType); initTestDataGenerator(); metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, false, - enableFullScan).build(); + writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, enableFullScan).build(); initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index 817b39254ef05..f26f52c9435b7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.timeline; +import java.text.ParseException; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; @@ -33,6 +34,7 @@ public class HoodieInstantTimeGenerator { // Format of the timestamp used for an Instant private static final String INSTANT_TIMESTAMP_FORMAT = "yyyyMMddHHmmss"; + private static final int INSTANT_TIMESTAMP_FORMAT_LENGTH = INSTANT_TIMESTAMP_FORMAT.length(); // Formatter to generate Instant timestamps private static DateTimeFormatter INSTANT_TIME_FORMATTER = DateTimeFormatter.ofPattern(INSTANT_TIMESTAMP_FORMAT); // The last Instant timestamp generated @@ -56,7 +58,7 @@ public static String createNewInstantTime(long milliseconds) { }); } - public static Date parseInstantTime(String timestamp) { + public static Date parseInstantTime(String timestamp) throws ParseException { try { LocalDateTime dt = LocalDateTime.parse(timestamp, INSTANT_TIME_FORMATTER); return Date.from(dt.atZone(ZoneId.systemDefault()).toInstant()); @@ -65,7 +67,11 @@ public static Date parseInstantTime(String timestamp) { if (timestamp.equals(ALL_ZERO_TIMESTAMP)) { return new Date(0); } - + // compaction and cleaning in metadata has special format. handling it by trimming extra chars and treating it with secs granularity + if (timestamp.length() > INSTANT_TIMESTAMP_FORMAT_LENGTH) { + LocalDateTime dt = LocalDateTime.parse(timestamp.substring(0, INSTANT_TIMESTAMP_FORMAT_LENGTH), INSTANT_TIME_FORMATTER); + return Date.from(dt.atZone(ZoneId.systemDefault()).toInstant()); + } throw e; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 1ee8a78e3e2ca..6cc5533f1c515 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -184,8 +184,9 @@ private List>>> readFrom HoodieRecord hoodieRecord = null; // Retrieve record from base file if (baseFileReader != null) { - HoodieTimer readTimer = new HoodieTimer().startTimer(); + HoodieTimer readTimer = new HoodieTimer(); for (String key : keys) { + readTimer.startTimer(); Option baseRecord = baseFileReader.getRecordByKey(key); if (baseRecord.isPresent()) { hoodieRecord = metadataTableConfig.populateMetaFields() diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 5f2d6928cbaaa..97de617b545af 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -31,6 +31,7 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.text.ParseException; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -472,6 +473,19 @@ public void testCreateNewInstantTime() throws Exception { } } + @Test + public void testMetadataCompactionInstantDateParsing() throws ParseException { + // default second granularity instant ID + String secondGranularityInstant = "20210101120101"; + Date defaultSecsGranularityDate = HoodieActiveTimeline.parseInstantTime(secondGranularityInstant); + // metadata table compaction/cleaning : ms granularity instant ID + String compactionInstant = secondGranularityInstant + "001"; + Date msGranularityDate = HoodieActiveTimeline.parseInstantTime(compactionInstant); + assertEquals(0, msGranularityDate.getTime() - defaultSecsGranularityDate.getTime(), "Expected the ms part to be 0"); + assertTrue(HoodieTimeline.compareTimestamps(secondGranularityInstant, HoodieTimeline.LESSER_THAN, compactionInstant)); + assertTrue(HoodieTimeline.compareTimestamps(compactionInstant, HoodieTimeline.GREATER_THAN, secondGranularityInstant)); + } + /** * Returns an exhaustive list of all possible HoodieInstant. * @return list of HoodieInstant From 38b6934352abd27b98332cce005f18102b388679 Mon Sep 17 00:00:00 2001 From: zhangyue19921010 <69956021+zhangyue19921010@users.noreply.github.com> Date: Mon, 15 Nov 2021 22:36:54 +0800 Subject: [PATCH 011/217] [HUDI-2683] Parallelize deleting archived hoodie commits (#3920) Co-authored-by: yuezhang --- .../hudi/config/HoodieCompactionConfig.java | 10 +++++ .../apache/hudi/config/HoodieWriteConfig.java | 4 ++ .../hudi/table/HoodieTimelineArchiveLog.java | 41 +++++++++++++------ .../org/apache/hudi/common/fs/FSUtils.java | 25 +++++++---- 4 files changed, 61 insertions(+), 19 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index ce74aad6b0989..e11d06098fc77 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -119,6 +119,11 @@ public class HoodieCompactionConfig extends HoodieConfig { + " keep the metadata overhead constant, even as the table size grows." + "This config controls the maximum number of instants to retain in the active timeline. "); + public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.archive.delete.parallelism") + .defaultValue(100) + .withDocumentation("Parallelism for deleting archived hoodie commits."); + public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty .key("hoodie.keep.min.commits") .defaultValue("20") @@ -568,6 +573,11 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } + public Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { + compactionConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); + return this; + } + public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) { compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS, String.valueOf(maxDeltaSecondsBeforeCompaction)); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index eb3df38428f58..73629b4c97b8f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1135,6 +1135,10 @@ public Boolean getCompactionReverseLogReadEnabled() { return getBoolean(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE); } + public int getArchiveDeleteParallelism() { + return getInt(HoodieCompactionConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); + } + public boolean inlineClusteringEnabled() { return getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index d492fb6577a93..bff91c3e56c2b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -18,9 +18,11 @@ package org.apache.hudi.table; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -127,7 +129,7 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException LOG.info("Archiving instants " + instantsToArchive); archive(context, instantsToArchive); LOG.info("Deleting archived instants " + instantsToArchive); - success = deleteArchivedInstants(instantsToArchive); + success = deleteArchivedInstants(instantsToArchive, context); } else { LOG.info("No Instants to archive"); } @@ -224,19 +226,34 @@ private Stream getInstantsToArchive() { HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream()); } - private boolean deleteArchivedInstants(List archivedInstants) throws IOException { + private boolean deleteArchivedInstants(List archivedInstants, HoodieEngineContext context) throws IOException { LOG.info("Deleting instants " + archivedInstants); boolean success = true; - for (HoodieInstant archivedInstant : archivedInstants) { - Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); - try { - if (metaClient.getFs().exists(commitFile)) { - success &= metaClient.getFs().delete(commitFile, false); - LOG.info("Archived and deleted instant file " + commitFile); - } - } catch (IOException e) { - throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); - } + List instantFiles = archivedInstants.stream().map(archivedInstant -> { + return new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); + }).map(Path::toString).collect(Collectors.toList()); + + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants"); + Map resultDeleteInstantFiles = FSUtils.parallelizeFilesProcess(context, + metaClient.getFs(), + config.getArchiveDeleteParallelism(), + pairOfSubPathAndConf -> { + Path commitFile = new Path(pairOfSubPathAndConf.getKey()); + try { + FileSystem fs = commitFile.getFileSystem(pairOfSubPathAndConf.getValue().get()); + if (fs.exists(commitFile)) { + return fs.delete(commitFile, false); + } + return true; + } catch (IOException e) { + throw new HoodieIOException("Failed to delete archived instant " + commitFile, e); + } + }, + instantFiles); + + for (Map.Entry result : resultDeleteInstantFiles.entrySet()) { + LOG.info("Archived and deleted instant file " + result.getKey() + " : " + result.getValue()); + success &= result.getValue(); } // Remove older meta-data from auxiliary path too diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index dc4df23a4c3d2..209e4ae422702 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -670,19 +670,30 @@ public static Map parallelizeSubPathProcess( .filter(subPathPredicate) .map(fileStatus -> fileStatus.getPath().toString()) .collect(Collectors.toList()); - if (subPaths.size() > 0) { - SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); - int actualParallelism = Math.min(subPaths.size(), parallelism); - result = hoodieEngineContext.mapToPair(subPaths, - subPath -> new ImmutablePair<>(subPath, pairFunction.apply(new ImmutablePair<>(subPath, conf))), - actualParallelism); - } + result = parallelizeFilesProcess(hoodieEngineContext, fs, parallelism, pairFunction, subPaths); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } return result; } + public static Map parallelizeFilesProcess( + HoodieEngineContext hoodieEngineContext, + FileSystem fs, + int parallelism, + SerializableFunction, T> pairFunction, + List subPaths) { + Map result = new HashMap<>(); + if (subPaths.size() > 0) { + SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); + int actualParallelism = Math.min(subPaths.size(), parallelism); + result = hoodieEngineContext.mapToPair(subPaths, + subPath -> new ImmutablePair<>(subPath, pairFunction.apply(new ImmutablePair<>(subPath, conf))), + actualParallelism); + } + return result; + } + /** * Deletes a sub-path. * From bff8769ed438aae36bbfc1ebefca63889418df18 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 15 Nov 2021 22:36:03 -0500 Subject: [PATCH 012/217] [HUDI-2712] Fixing a bug with rollback of partially failed commit which has new partitions (#3947) --- .../ListingBasedRollbackStrategy.java | 2 +- .../table/action/rollback/RollbackUtils.java | 19 +++-- .../upgrade/ZeroToOneUpgradeHandler.java | 2 +- .../functional/TestHoodieBackedMetadata.java | 72 ++++++++++++++++--- .../functional/TestHoodieMetadataBase.java | 8 ++- 5 files changed, 80 insertions(+), 23 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index 266fa39cb986e..e6355526e5233 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -60,7 +60,7 @@ public List getRollbackRequests(HoodieInstant instantToRo List rollbackRequests = null; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, - table.getMetaClient().getBasePath(), config); + table.getMetaClient().getBasePath()); } else { rollbackRequests = RollbackUtils .generateRollbackRequestsUsingFileListingMOR(instantToRollback, table, context); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index 6ad4e1c986fb5..fccb992ef11f4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -18,10 +18,6 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hadoop.fs.FileStatus; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -42,6 +38,10 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; +import org.apache.hadoop.fs.FileStatus; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -56,7 +56,8 @@ public class RollbackUtils { /** * Get Latest version of Rollback plan corresponding to a clean instant. - * @param metaClient Hoodie Table Meta Client + * + * @param metaClient Hoodie Table Meta Client * @param rollbackInstant Instant referring to rollback action * @return Rollback plan corresponding to rollback instant * @throws IOException @@ -106,12 +107,10 @@ static HoodieRollbackStat mergeRollbackStat(HoodieRollbackStat stat1, HoodieRoll * Generate all rollback requests that needs rolling back this action without actually performing rollback for COW table type. * @param engineContext instance of {@link HoodieEngineContext} to use. * @param basePath base path of interest. - * @param config instance of {@link HoodieWriteConfig} to use. * @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected. */ - public static List generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext, - String basePath, HoodieWriteConfig config) { - return FSUtils.getAllPartitionPaths(engineContext, config.getMetadataConfig(), basePath).stream() + public static List generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext, String basePath) { + return FSUtils.getAllPartitionPaths(engineContext, basePath, false, false).stream() .map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction) .collect(Collectors.toList()); } @@ -127,7 +126,7 @@ public static List generateRollbackRequestsByListin public static List generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException { String commit = instantToRollback.getTimestamp(); HoodieWriteConfig config = table.getConfig(); - List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + List partitions = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getBasePath(), false, false); if (partitions.isEmpty()) { return new ArrayList<>(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 18815b2e132da..1aebbf6b4c42d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -102,7 +102,7 @@ protected void recreateMarkers(final String commitInstantTime, // generate rollback stats List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { - rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig()); + rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 49eddc24ef0d8..59e12a1515ad5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -82,7 +82,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -93,6 +92,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -307,9 +307,8 @@ public void testMetadataTableServices() throws Exception { /** * Test rollback of various table operations sync to Metadata Table correctly. */ - //@ParameterizedTest - //@EnumSource(HoodieTableType.class) - @Disabled + @ParameterizedTest + @EnumSource(HoodieTableType.class) public void testRollbackOperations(HoodieTableType tableType) throws Exception { init(tableType); doWriteInsertAndUpsert(testTable); @@ -1087,18 +1086,72 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte assertTrue(oldStatus.getModificationTime() < newStatus.getModificationTime()); } + /** + * Tests rollback of a commit which has new partitions which is not present in hudi table prior to the commit being rolledback. + * + * @throws Exception + */ + @Test + public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(), + true)) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 10); + List upsertRecords = new ArrayList<>(); + for (HoodieRecord entry : records) { + if (entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + || entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) { + upsertRecords.add(entry); + } + } + List writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed + // instant so that only the inflight is left over. + String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime); + assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + commitInstantFileName), false)); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(), + true)) { + String newCommitTime = client.startCommit(); + // Next insert + List records = dataGen.generateInserts(newCommitTime, 20); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + } + /** * Test various error scenarios. */ - //@Test - @Disabled + @Test public void testErrorCases() throws Exception { init(HoodieTableType.COPY_ON_WRITE); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table // should be rolled back to last valid commit. - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(), + true)) { String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 10); @@ -1111,6 +1164,7 @@ public void testErrorCases() throws Exception { records = dataGen.generateInserts(newCommitTime, 5); writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); + validateMetadata(client); // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. @@ -1119,7 +1173,9 @@ public void testErrorCases() throws Exception { commitInstantFileName), false)); } - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(), + true)) { String newCommitTime = client.startCommit(); // Next insert List records = dataGen.generateInserts(newCommitTime, 5); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index 5e4c4ba86632a..a91099976700e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -87,7 +87,8 @@ public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean initMetaClient(tableType); initTestDataGenerator(); metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, enableFullScan).build(); + writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, + enableFullScan, true).build(); initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable); } @@ -265,11 +266,11 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, bo protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { - return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true); + return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true, true); } protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, - boolean enableMetrics, boolean enableFullScan) { + boolean enableMetrics, boolean enableFullScan, boolean useRollbackUsingMarkers) { Properties properties = new Properties(); properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName()); return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) @@ -292,6 +293,7 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea .withExecutorMetrics(true).build()) .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() .usePrefix("unit-test").build()) + .withRollbackUsingMarkers(useRollbackUsingMarkers) .withProperties(properties); } From 6f5e661010a85f3426131111baff962a6d6ba91e Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Tue, 16 Nov 2021 13:46:34 +0800 Subject: [PATCH 013/217] [HUDI-2769] Fix StreamerUtil#medianInstantTime for very near instant time (#4005) --- .../src/main/java/org/apache/hudi/util/StreamerUtil.java | 8 +++++++- .../test/java/org/apache/hudi/utils/TestStreamerUtil.java | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 867621a66d8f4..ddbd24e3562d0 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -414,7 +415,12 @@ public static Option medianInstantTime(String highVal, String lowVal) { ValidationUtils.checkArgument(high > low, "Instant [" + highVal + "] should have newer timestamp than instant [" + lowVal + "]"); long median = low + (high - low) / 2; - return low >= median ? Option.empty() : Option.of(HoodieActiveTimeline.formatInstantTime(new Date(median))); + final String instantTime = HoodieActiveTimeline.formatInstantTime(new Date(median)); + if (HoodieTimeline.compareTimestamps(lowVal, HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime) + || HoodieTimeline.compareTimestamps(highVal, HoodieTimeline.LESSER_THAN_OR_EQUALS, instantTime)) { + return Option.empty(); + } + return Option.of(instantTime); } catch (ParseException e) { throw new HoodieException("Get median instant time with interval [" + lowVal + ", " + highVal + "] error", e); } diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java index c05e5b056344a..b9e2b916da3c8 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java @@ -86,6 +86,8 @@ void testMedianInstantTime() { assertThrows(IllegalArgumentException.class, () -> StreamerUtil.medianInstantTime(lower, higher), "The first argument should have newer instant time"); + // test very near instant time + assertFalse(StreamerUtil.medianInstantTime("20211116115634", "20211116115633").isPresent()); } @Test From cbcbec4d3831d6236a2dbf1f17d50b447800994a Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Tue, 16 Nov 2021 18:30:16 -0800 Subject: [PATCH 014/217] [MINOR] Fixed checkstyle config to be based off Maven root-dir (requires Maven >=3.3.1 to work properly); (#4009) Updated README --- README.md | 2 +- pom.xml | 2 ++ style/checkstyle.xml | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 764ba0d48bccf..f20ec82dd2bd3 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Prerequisites for building Apache Hudi: * Unix-like system (like Linux, Mac OS X) * Java 8 (Java 9 or 10 may work) * Git -* Maven +* Maven (>=3.3.1) ``` # Checkout code and build diff --git a/pom.xml b/pom.xml index 888d8d32c6ed1..8336097e67491 100644 --- a/pom.xml +++ b/pom.xml @@ -231,6 +231,8 @@ ${project.build.sourceDirectory} + + basedir=${maven.multiModuleProjectDirectory} **\/generated-sources\/ diff --git a/style/checkstyle.xml b/style/checkstyle.xml index 7dbce7973bfde..ff90b7f3cdc55 100644 --- a/style/checkstyle.xml +++ b/style/checkstyle.xml @@ -305,7 +305,7 @@ - + From 04eb5fdc651bce3b34f17c1828879b18f1b1b6f8 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Tue, 16 Nov 2021 23:36:55 -0500 Subject: [PATCH 015/217] [HUDI-2753] Ensure list based rollback strategy is used for restore (#3983) --- .../org/apache/hudi/client/AbstractHoodieWriteClient.java | 4 ++-- .../src/main/java/org/apache/hudi/table/HoodieTable.java | 5 +++-- .../action/restore/CopyOnWriteRestoreActionExecutor.java | 2 +- .../action/restore/MergeOnReadRestoreActionExecutor.java | 2 +- .../action/rollback/BaseRollbackPlanActionExecutor.java | 7 +++++-- .../org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java | 5 +++-- .../org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java | 5 +++-- .../org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java | 5 +++-- .../org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java | 5 +++-- .../org/apache/hudi/table/HoodieSparkMergeOnReadTable.java | 5 +++-- .../functional/TestHoodieClientOnCopyOnWriteStorage.java | 5 +++-- .../src/test/java/org/apache/hudi/table/TestCleaner.java | 2 +- .../rollback/TestCopyOnWriteRollbackActionExecutor.java | 6 ++++-- .../rollback/TestMergeOnReadRollbackActionExecutor.java | 3 ++- 14 files changed, 37 insertions(+), 24 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 444eae62b2ec4..d98da346e13e4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -599,7 +599,7 @@ public boolean rollback(final String commitInstantTime, boolean skipLocking) thr if (commitInstantOpt.isPresent()) { LOG.info("Scheduling Rollback at instant time :" + rollbackInstantTime); Option rollbackPlanOption = table.scheduleRollback(context, rollbackInstantTime, - commitInstantOpt.get(), false); + commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers()); if (rollbackPlanOption.isPresent()) { // execute rollback HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true, @@ -1024,7 +1024,7 @@ protected Option inlineCluster(Option> extraMetadata protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { String commitTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, commitTime, inflightInstant, false); + table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); table.rollback(context, commitTime, inflightInstant, false, false); table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 6046374ba107d..747470f1dbffc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -442,12 +442,13 @@ public abstract Option scheduleCleaning(HoodieEngineContext c * @param context HoodieEngineContext * @param instantTime Instant Time for scheduling rollback * @param instantToRollback instant to be rolled back + * @param shouldRollbackUsingMarkers uses marker based rollback strategy when set to true. uses list based rollback when false. * @return HoodieRollbackPlan containing info on rollback. */ public abstract Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers); /** * Rollback the (inflight/committed) record changes with the given commit time. @@ -490,7 +491,7 @@ public abstract HoodieRestoreMetadata restore(HoodieEngineContext context, */ public void rollbackInflightCompaction(HoodieInstant inflightInstant) { String commitTime = HoodieActiveTimeline.createNewInstantTime(); - scheduleRollback(context, commitTime, inflightInstant, false); + scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); rollback(context, commitTime, inflightInstant, false, false); getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java index 1116ef9a4dd82..facab71c6237b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java @@ -48,7 +48,7 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback } table.getMetaClient().reloadActiveTimeline(); String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, newInstantTime, instantToRollback, false); + table.scheduleRollback(context, newInstantTime, instantToRollback, false, false); table.getMetaClient().reloadActiveTimeline(); CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor( context, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java index db6fbc2620155..661cee4a2e608 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java @@ -52,7 +52,7 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback } table.getMetaClient().reloadActiveTimeline(); String instantTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, instantTime, instantToRollback, false); + table.scheduleRollback(context, instantTime, instantToRollback, false, false); table.getMetaClient().reloadActiveTimeline(); MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor( context, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java index 24edde27642cc..f95ec5d5c9fe1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java @@ -50,6 +50,7 @@ public class BaseRollbackPlanActionExecutor table, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { + boolean skipTimelinePublish, + boolean shouldRollbackUsingMarkers) { super(context, config, table, instantTime); this.instantToRollback = instantToRollback; this.skipTimelinePublish = skipTimelinePublish; + this.shouldRollbackUsingMarkers = shouldRollbackUsingMarkers; } /** @@ -84,7 +87,7 @@ interface RollbackStrategy extends Serializable { * @return */ private BaseRollbackPlanActionExecutor.RollbackStrategy getRollbackStrategy() { - if (config.shouldRollbackUsingMarkers()) { + if (shouldRollbackUsingMarkers) { return new MarkerBasedRollbackStrategy(table, context, config, instantTime); } else { return new ListingBasedRollbackStrategy(table, context, config, instantTime); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 85ad1364de9ca..7e25559bb1772 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -316,8 +316,9 @@ public Option scheduleCleaning(HoodieEngineContext context, S @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index 5ad87e0831e97..a65e03da761d3 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -120,8 +120,9 @@ context, config, this, compactionInstantTime, new HoodieFlinkMergeOnReadTableCom @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index a9e582110c206..4107adb8fb488 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -193,8 +193,9 @@ public void rollbackBootstrap(HoodieEngineContext context, @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 0971b87c44675..74d4718a932fc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -230,8 +230,9 @@ public Option scheduleCleaning(HoodieEngineContext context, S @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, - HoodieInstant instantToRollback, boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index 9e053aaa0da44..75af5d0f685fc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -156,8 +156,9 @@ public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, - HoodieInstant instantToRollback, boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index caffb476b8409..86d18fe28b006 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -495,7 +495,7 @@ void assertNodupesInPartition(List records) { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testUpserts(boolean populateMetaFields) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withRollbackUsingMarkers(true); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsert, false); } @@ -506,7 +506,7 @@ public void testUpserts(boolean populateMetaFields) throws Exception { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testUpsertsPrepped(boolean populateMetaFields) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withRollbackUsingMarkers(true); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsertPreppedRecords, true); } @@ -523,6 +523,7 @@ private void testUpsertsInternal(HoodieWriteConfig config, throws Exception { // Force using older timeline layout HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withRollbackUsingMarkers(true) .withProps(config.getProps()).withTimelineLayoutVersion( VERSION_0).build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index cb468e903e59b..2305d7bdeb4d1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -1307,7 +1307,7 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty()); metaClient.reloadActiveTimeline(); HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"); - table.scheduleRollback(context, "001", rollbackInstant, false); + table.scheduleRollback(context, "001", rollbackInstant, false, config.shouldRollbackUsingMarkers()); table.rollback(context, "001", rollbackInstant, true, false); final int numTempFilesAfter = testTable.listAllFilesInTempFolder().length; assertEquals(0, numTempFilesAfter, "All temp files are deleted."); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java index 3225dcd04ea3f..1bee6ac0ac622 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java @@ -86,7 +86,8 @@ public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() // execute CopyOnWriteRollbackActionExecutor with filelisting mode BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, false); + new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, false, + table.getConfig().shouldRollbackUsingMarkers()); HoodieRollbackPlan rollbackPlan = (HoodieRollbackPlan) copyOnWriteRollbackPlanActionExecutor.execute().get(); CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, true, false); @@ -168,7 +169,8 @@ private void performRollbackAndValidate(boolean isUsingMarkers, HoodieWriteConfi } BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", commitInstant, false); + new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", commitInstant, false, + table.getConfig().shouldRollbackUsingMarkers()); HoodieRollbackPlan hoodieRollbackPlan = (HoodieRollbackPlan) copyOnWriteRollbackPlanActionExecutor.execute().get(); CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(context, cfg, table, "003", commitInstant, false, false); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index 5a829e294a4b5..1c6015f70840a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -91,7 +91,8 @@ public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws //2. rollback HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false); + new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false, + cfg.shouldRollbackUsingMarkers()); mergeOnReadRollbackPlanActionExecutor.execute().get(); MergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new MergeOnReadRollbackActionExecutor( context, From ce7d2333078e4e1f16de1bce6d448c5eef1e4111 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Wed, 17 Nov 2021 01:21:28 -0500 Subject: [PATCH 016/217] [HUDI-2151] Part3 Enabling marker based rollback as default rollback strategy (#3950) * Enabling timeline server based markers * Enabling timeline server based markers and marker based rollback * Removing constraint that timeline server can be enabled only for hdfs * Fixing tests --- .../apache/hudi/cli/commands/TestRollbacksCommand.java | 1 + .../java/org/apache/hudi/config/HoodieWriteConfig.java | 4 ++-- .../hudi/metadata/HoodieBackedTableMetadataWriter.java | 3 +++ .../apache/hudi/table/marker/WriteMarkersFactory.java | 8 -------- .../java/org/apache/hudi/client/TestClientRollback.java | 2 ++ .../org/apache/hudi/client/TestTableSchemaEvolution.java | 9 ++++++--- .../apache/hudi/client/functional/TestHBaseIndex.java | 6 ++++-- .../apache/hudi/client/functional/TestHoodieIndex.java | 7 ++++--- .../rollback/TestMergeOnReadRollbackActionExecutor.java | 2 +- 9 files changed, 23 insertions(+), 19 deletions(-) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java index ceaabddec7d2e..17bc48f66f0c4 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java @@ -90,6 +90,7 @@ public void init() throws Exception { .withBaseFilesInPartitions(partitionAndFileId); // generate two rollback HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withRollbackUsingMarkers(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); try (AbstractHoodieWriteClient client = new SparkRDDWriteClient(context(), config)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 73629b4c97b8f..496cd2d057ddc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -120,9 +120,9 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty ROLLBACK_USING_MARKERS_ENABLE = ConfigProperty .key("hoodie.rollback.using.markers") - .defaultValue("false") + .defaultValue("true") .withDocumentation("Enables a more efficient mechanism for rollbacks based on the marker files generated " - + "during the writes. Turned off by default."); + + "during the writes. Turned on by default."); public static final ConfigProperty TIMELINE_LAYOUT_VERSION_NUM = ConfigProperty .key("hoodie.timeline.layout.version") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 71d61938c7aa5..6cbc595c55974 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -45,6 +45,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -180,6 +181,8 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withAutoCommit(true) .withAvroSchemaValidate(true) .withEmbeddedTimelineServerEnabled(false) + .withMarkersType(MarkerType.DIRECT.name()) + .withRollbackUsingMarkers(false) .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) .withSchema(HoodieMetadataRecord.getClassSchema().toString()) .forTable(tableName) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java index 044b258e8893c..e7d31f3a07b24 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java @@ -18,8 +18,6 @@ package org.apache.hudi.table.marker; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieTable; @@ -45,12 +43,6 @@ public static WriteMarkers get(MarkerType markerType, HoodieTable table, String case DIRECT: return new DirectWriteMarkers(table, instantTime); case TIMELINE_SERVER_BASED: - String basePath = table.getMetaClient().getBasePath(); - if (StorageSchemes.HDFS.getScheme().equals( - FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) { - throw new HoodieException("Timeline-server-based markers are not supported for HDFS: " - + "base path " + basePath); - } return new TimelineServerBasedWriteMarkers(table, instantTime); default: throw new HoodieException("The marker type \"" + markerType.name() + "\" is not supported."); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index 32ac8689687cf..6412113a59768 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -201,6 +201,7 @@ public void testRollbackCommit() throws Exception { .withBaseFilesInPartitions(partitionAndFileId3); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withRollbackUsingMarkers(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); @@ -308,6 +309,7 @@ public void testAutoRollbackInflightCommit() throws Exception { // Set Failed Writes rollback to EAGER config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withRollbackUsingMarkers(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); final String commitTime5 = "20160506030631"; try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 29bad0d949c33..9bca10892e28c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -316,7 +316,7 @@ public void testCopyOnWriteTable() throws Exception { .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); - HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); + HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA).withRollbackUsingMarkers(false).build(); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA @@ -507,11 +507,14 @@ private List convertToSchema(List records, String sc } private HoodieWriteConfig getWriteConfig(String schema) { + return getWriteConfigBuilder(schema).build(); + } + + private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { return getConfigBuilder(schema) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) - .withAvroSchemaValidate(true) - .build(); + .withAvroSchemaValidate(true); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java index db17ceae92af6..43bf36c501f9b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java @@ -278,7 +278,7 @@ public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws E final int numRecords = 10; final String oldPartitionPath = "1970/01/01"; final String emptyHoodieRecordPayloadClasssName = EmptyHoodieRecordPayload.class.getName(); - HoodieWriteConfig config = getConfig(true, true); + HoodieWriteConfig config = getConfigBuilder(100, true, true).withRollbackUsingMarkers(false).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { @@ -337,6 +337,7 @@ public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws E public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Load to memory HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withRollbackUsingMarkers(false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); @@ -383,7 +384,7 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { @Test public void testSimpleTagLocationWithInvalidCommit() throws Exception { // Load to memory - HoodieWriteConfig config = getConfig(); + HoodieWriteConfig config = getConfigBuilder(100, false, false).withRollbackUsingMarkers(false).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); @@ -425,6 +426,7 @@ public void testSimpleTagLocationWithInvalidCommit() throws Exception { public void testEnsureTagLocationUsesCommitTimeline() throws Exception { // Load to memory HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withRollbackUsingMarkers(false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index c741c0266cc71..fa5a3537bb1f8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -98,10 +98,10 @@ private static Stream indexTypeParams() { private HoodieWriteConfig config; private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields, true); + setUp(indexType, populateMetaFields, true, true); } - private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata) throws Exception { + private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata, boolean rollbackUsingMarkers) throws Exception { this.indexType = indexType; initPath(); initSparkContexts(); @@ -111,6 +111,7 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean enab : getPropertiesForKeyGen()); config = getConfigBuilder() .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) + .withRollbackUsingMarkers(rollbackUsingMarkers) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) .build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build(); writeClient = getHoodieWriteClient(config); @@ -225,7 +226,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul @ParameterizedTest @MethodSource("indexTypeParams") public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields, true); + setUp(indexType, populateMetaFields, true, false); String newCommitTime = writeClient.startCommit(); int totalRecords = 20 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index 1c6015f70840a..4e98b220f3613 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -162,8 +162,8 @@ public void testFailForCompletedInstants() { public void testRollbackWhenFirstCommitFail() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withRollbackUsingMarkers(false) .withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime("001"); client.insert(jsc.emptyRDD(), "001"); From aec5d11da2e8a025fb6c8659a648f9e2d36990d6 Mon Sep 17 00:00:00 2001 From: 0x574C <761604382@qq.com> Date: Wed, 17 Nov 2021 14:45:43 +0800 Subject: [PATCH 017/217] Check --source-avro-schema-path parameter (#3987) Co-authored-by: 0x3E6 --- .../java/org/apache/hudi/streamer/FlinkStreamerConfig.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index de2218acb0b94..1d7111f495c58 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -365,7 +365,9 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setDouble(FlinkOptions.INDEX_STATE_TTL, config.indexStateTtl); conf.setBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED, config.indexGlobalEnabled); conf.setString(FlinkOptions.INDEX_PARTITION_REGEX, config.indexPartitionRegex); - conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, config.sourceAvroSchemaPath); + if (!StringUtils.isNullOrEmpty(config.sourceAvroSchemaPath)) { + conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, config.sourceAvroSchemaPath); + } conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, config.sourceAvroSchema); conf.setBoolean(FlinkOptions.UTC_TIMEZONE, config.utcTimezone); conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode); From 4d884bdaa9affae9ce920c83f799ff9e1be99861 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=91=A3=E5=8F=AF=E4=BC=A6?= Date: Wed, 17 Nov 2021 16:50:04 +0800 Subject: [PATCH 018/217] [MINOR] Fix typo,'Hooide' corrected to 'Hoodie' (#4007) --- .../scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala | 2 +- .../spark/sql/hudi/command/CreateHoodieTableCommand.scala | 2 +- .../org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 543a8b997004c..16f83f482c29b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -183,7 +183,7 @@ object HoodieOptionConfig { params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty) } - def deleteHooideOptions(options: Map[String, String]): Map[String, String] = { + def deleteHoodieOptions(options: Map[String, String]): Map[String, String] = { options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => keyMapping.contains(kv._1)) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index d6c5160897e6e..94c6eab75664d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -170,7 +170,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val serdeFormat = HoodieInputFormatUtils.getSerDeClassName(HoodieFileFormat.PARQUET) // only parameters irrelevant to hudi can be set to storage.properties - val storageProperties = HoodieOptionConfig.deleteHooideOptions(options) + val storageProperties = HoodieOptionConfig.deleteHoodieOptions(options) val newStorage = new CatalogStorageFormat( Some(new Path(path).toUri), Some(inputFormat), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala index f91388ae9a10e..5cd66fd4838d6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -78,7 +78,7 @@ class TestHoodieOptionConfig extends HoodieClientTestBase { } @Test - def testDeleteHooideOptions(): Unit = { + def testDeleteHoodieOptions(): Unit = { val sqlOptions = Map("primaryKey" -> "id,addr", "preCombineField" -> "timestamp", "type" -> "mor", @@ -86,7 +86,7 @@ class TestHoodieOptionConfig extends HoodieClientTestBase { "hoodie.compact.inline" -> "true", "key123" -> "value456" ) - val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions) + val tableConfigs = HoodieOptionConfig.deleteHoodieOptions(sqlOptions) assertTrue(tableConfigs.size == 1) assertTrue(tableConfigs("key123") == "value456") } From 826414cff5adb5006066bdedb843ed864709b048 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=8D=A2=E6=B3=A2?= <26039470+lubo212@users.noreply.github.com> Date: Wed, 17 Nov 2021 22:47:52 +0800 Subject: [PATCH 019/217] [MINOR] Add the Schema for GooseFS to StorageSchemes (#3982) Co-authored-by: lubo --- .../src/main/java/org/apache/hudi/common/fs/StorageSchemes.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 1cc00df4ebd7a..5dcd0b156aa32 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -54,6 +54,8 @@ public enum StorageSchemes { COSN("cosn", false), // Tencent Cloud HDFS CHDFS("ofs", true), + // Tencent Cloud CacheFileSystem + GOOSEFS("gfs", false), // Databricks file system DBFS("dbfs", false), // IBM Cloud Object Storage From 1ee12cfa6fa8d18546af11a7ae03cb0e4969460f Mon Sep 17 00:00:00 2001 From: wenningd Date: Wed, 17 Nov 2021 09:09:31 -0800 Subject: [PATCH 020/217] [HUDI-2314] Add support for DynamoDb based lock provider (#3486) - Co-authored-by: Wenning Ding - Co-authored-by: Sivabalan Narayanan --- hudi-aws/pom.xml | 194 ++++++++++++++++ .../HoodieAWSCredentialsProviderFactory.java | 45 ++++ .../HoodieConfigAWSCredentialsProvider.java | 70 ++++++ .../lock/DynamoDBBasedLockProvider.java | 219 ++++++++++++++++++ .../hudi/config/AWSLockConfiguration.java | 84 +++++++ .../apache/hudi/config/HoodieAWSConfig.java | 148 ++++++++++++ ...stHoodieAWSCredentialsProviderFactory.java | 42 ++++ .../ITTestDynamoDBBasedLockProvider.java | 120 ++++++++++ .../test/resources/log4j-surefire.properties | 25 ++ .../client/transaction/lock/LockManager.java | 2 +- .../apache/hudi/config/HoodieLockConfig.java | 1 - .../hudi/common/config/ConfigGroups.java | 3 +- packaging/hudi-flink-bundle/pom.xml | 5 + packaging/hudi-spark-bundle/pom.xml | 5 + packaging/hudi-utilities-bundle/pom.xml | 5 + pom.xml | 17 ++ 16 files changed, 982 insertions(+), 3 deletions(-) create mode 100644 hudi-aws/pom.xml create mode 100644 hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java create mode 100644 hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java create mode 100644 hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java create mode 100644 hudi-aws/src/main/java/org/apache/hudi/config/AWSLockConfiguration.java create mode 100644 hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java create mode 100644 hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java create mode 100644 hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java create mode 100644 hudi-aws/src/test/resources/log4j-surefire.properties diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml new file mode 100644 index 0000000000000..19ceed7fce7b8 --- /dev/null +++ b/hudi-aws/pom.xml @@ -0,0 +1,194 @@ + + + + + hudi + org.apache.hudi + 0.10.0-SNAPSHOT + + 4.0.0 + + hudi-aws + 0.10.0-SNAPSHOT + + hudi-aws + jar + + + 1.15.0 + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-client-common + ${project.version} + + + + + log4j + log4j + + + + + org.apache.hadoop + hadoop-common + tests + test + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + + + + com.amazonaws + dynamodb-lock-client + ${dynamodb.lockclient.version} + + + com.amazonaws + aws-java-sdk-dynamodb + ${aws.sdk.version} + + + io.netty + * + + + + + com.amazonaws + aws-java-sdk-core + ${aws.sdk.version} + + + + + org.junit.jupiter + junit-jupiter-api + test + + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + io.fabric8 + docker-maven-plugin + + + prepare-it-database + pre-integration-test + + start + + + + + amazon/dynamodb-local:${dynamodb-local.version} + it-database + + + ${dynamodb-local.port}:${dynamodb-local.port} + + + + ${dynamodb-local.endpoint}/shell/ + + + + + + + + + + remove-it-database + post-integration-test + + stop + + + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java new file mode 100644 index 0000000000000..631b0fa8d5349 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.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.hudi.aws.credentials; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProviderChain; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Factory class for Hoodie AWSCredentialsProvider. + */ +public class HoodieAWSCredentialsProviderFactory { + public static AWSCredentialsProvider getAwsCredentialsProvider(Properties props) { + return getAwsCredentialsProviderChain(props); + } + + private static AWSCredentialsProvider getAwsCredentialsProviderChain(Properties props) { + List providers = new ArrayList<>(); + providers.add(new HoodieConfigAWSCredentialsProvider(props)); + providers.add(new DefaultAWSCredentialsProviderChain()); + AWSCredentialsProviderChain providerChain = new AWSCredentialsProviderChain(providers); + providerChain.setReuseLastProvider(true); + return providerChain; + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java new file mode 100644 index 0000000000000..4e9cf383906a4 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java @@ -0,0 +1,70 @@ +/* + * 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.hudi.aws.credentials; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.BasicSessionCredentials; +import org.apache.hudi.config.HoodieAWSConfig; +import org.apache.hudi.common.util.StringUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Properties; + +/** + * Credentials provider which fetches AWS access key from Hoodie config. + */ +public class HoodieConfigAWSCredentialsProvider implements AWSCredentialsProvider { + + private static final Logger LOG = LogManager.getLogger(HoodieConfigAWSCredentialsProvider.class); + + private AWSCredentials awsCredentials; + + public HoodieConfigAWSCredentialsProvider(Properties props) { + String accessKey = props.getProperty(HoodieAWSConfig.AWS_ACCESS_KEY.key()); + String secretKey = props.getProperty(HoodieAWSConfig.AWS_SECRET_KEY.key()); + String sessionToken = props.getProperty(HoodieAWSConfig.AWS_SESSION_TOKEN.key()); + + if (StringUtils.isNullOrEmpty(accessKey) || StringUtils.isNullOrEmpty(secretKey)) { + LOG.debug("AWS access key or secret key not found in the Hudi configuration. " + + "Use default AWS credentials"); + } else { + this.awsCredentials = createCredentials(accessKey, secretKey, sessionToken); + } + } + + private static AWSCredentials createCredentials(String accessKey, String secretKey, + String sessionToken) { + return (sessionToken == null) + ? new BasicAWSCredentials(accessKey, secretKey) + : new BasicSessionCredentials(accessKey, secretKey, sessionToken); + } + + @Override + public AWSCredentials getCredentials() { + return this.awsCredentials; + } + + @Override + public void refresh() { + + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java b/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java new file mode 100644 index 0000000000000..f2c01abefdd80 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java @@ -0,0 +1,219 @@ +/* + * 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.hudi.aws.transaction.lock; + +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.regions.RegionUtils; +import com.amazonaws.services.dynamodbv2.AcquireLockOptions; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBLockClient; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBLockClientOptions; +import com.amazonaws.services.dynamodbv2.LockItem; +import com.amazonaws.services.dynamodbv2.model.AttributeDefinition; +import com.amazonaws.services.dynamodbv2.model.BillingMode; +import com.amazonaws.services.dynamodbv2.model.CreateTableRequest; +import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; +import com.amazonaws.services.dynamodbv2.model.KeyType; +import com.amazonaws.services.dynamodbv2.model.LockNotGrantedException; +import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput; +import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType; +import com.amazonaws.services.dynamodbv2.util.TableUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.AWSLockConfiguration; +import org.apache.hudi.exception.HoodieLockException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.NotThreadSafe; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY; + +/** + * A DynamoDB based lock. This {@link LockProvider} implementation allows to lock table operations + * using DynamoDB. Users need to have access to AWS DynamoDB to be able to use this lock. + */ +@NotThreadSafe +public class DynamoDBBasedLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(DynamoDBBasedLockProvider.class); + + private static final String DYNAMODB_ATTRIBUTE_NAME = "key"; + + private final AmazonDynamoDBLockClient client; + private final String tableName; + private final String dynamoDBPartitionKey; + protected LockConfiguration lockConfiguration; + private volatile LockItem lock; + + public DynamoDBBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + this(lockConfiguration, conf, null); + } + + public DynamoDBBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf, AmazonDynamoDB dynamoDB) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.tableName = lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key()); + this.dynamoDBPartitionKey = lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_PARTITION_KEY.key()); + long leaseDuration = Long.parseLong(lockConfiguration.getConfig().getString(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY)); + if (dynamoDB == null) { + dynamoDB = getDynamoDBClient(); + } + // build the dynamoDb lock client + this.client = new AmazonDynamoDBLockClient( + AmazonDynamoDBLockClientOptions.builder(dynamoDB, tableName) + .withTimeUnit(TimeUnit.MILLISECONDS) + .withLeaseDuration(leaseDuration) + .withHeartbeatPeriod(leaseDuration / 3) + .withCreateHeartbeatBackgroundThread(true) + .build()); + + if (!this.client.lockTableExists()) { + createLockTableInDynamoDB(dynamoDB, tableName); + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + lock = client.acquireLock(AcquireLockOptions.builder(dynamoDBPartitionKey) + .withAdditionalTimeToWaitForLock(time) + .withTimeUnit(TimeUnit.MILLISECONDS) + .build()); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (InterruptedException e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } catch (LockNotGrantedException e) { + return false; + } + return lock != null && !lock.isExpired(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null) { + return; + } + if (!client.releaseLock(lock)) { + LOG.warn("The lock has already been stolen"); + } + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + if (!client.releaseLock(lock)) { + LOG.warn("The lock has already been stolen"); + } + lock = null; + } + this.client.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public LockItem getLock() { + return lock; + } + + private AmazonDynamoDB getDynamoDBClient() { + String region = this.lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_REGION.key()); + String endpointURL = RegionUtils.getRegion(region).getServiceEndpoint(AmazonDynamoDB.ENDPOINT_PREFIX); + AwsClientBuilder.EndpointConfiguration dynamodbEndpoint = + new AwsClientBuilder.EndpointConfiguration(endpointURL, region); + return AmazonDynamoDBClientBuilder.standard() + .withEndpointConfiguration(dynamodbEndpoint) + .withCredentials(HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(lockConfiguration.getConfig())) + .build(); + } + + private void createLockTableInDynamoDB(AmazonDynamoDB dynamoDB, String tableName) { + String billingMode = lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_BILLING_MODE.key()); + KeySchemaElement partitionKeyElement = new KeySchemaElement(); + partitionKeyElement.setAttributeName(DYNAMODB_ATTRIBUTE_NAME); + partitionKeyElement.setKeyType(KeyType.HASH); + + List keySchema = new ArrayList<>(); + keySchema.add(partitionKeyElement); + + Collection attributeDefinitions = new ArrayList<>(); + attributeDefinitions.add(new AttributeDefinition().withAttributeName(DYNAMODB_ATTRIBUTE_NAME).withAttributeType(ScalarAttributeType.S)); + + CreateTableRequest createTableRequest = new CreateTableRequest(tableName, keySchema); + createTableRequest.setAttributeDefinitions(attributeDefinitions); + createTableRequest.setBillingMode(billingMode); + if (billingMode.equals(BillingMode.PROVISIONED.name())) { + createTableRequest.setProvisionedThroughput(new ProvisionedThroughput() + .withReadCapacityUnits(Long.parseLong(lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_READ_CAPACITY.key()))) + .withWriteCapacityUnits(Long.parseLong(lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_WRITE_CAPACITY.key())))); + } + dynamoDB.createTable(createTableRequest); + + LOG.info("Creating dynamoDB table " + tableName + ", waiting for table to be active"); + try { + TableUtils.waitUntilActive(dynamoDB, tableName, Integer.parseInt(lockConfiguration.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT.key())), 20 * 1000); + } catch (TableUtils.TableNeverTransitionedToStateException e) { + throw new HoodieLockException("Created dynamoDB table never transits to active", e); + } catch (InterruptedException e) { + throw new HoodieLockException("Thread interrupted while waiting for dynamoDB table to turn active", e); + } + LOG.info("Created dynamoDB table " + tableName); + } + + private void checkRequiredProps(final LockConfiguration config) { + ValidationUtils.checkArgument(config.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_BILLING_MODE.key()) != null); + ValidationUtils.checkArgument(config.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key()) != null); + ValidationUtils.checkArgument(config.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_REGION.key()) != null); + ValidationUtils.checkArgument(config.getConfig().getString(AWSLockConfiguration.DYNAMODB_LOCK_PARTITION_KEY.key()) != null); + config.getConfig().putIfAbsent(AWSLockConfiguration.DYNAMODB_LOCK_BILLING_MODE.key(), BillingMode.PAY_PER_REQUEST.name()); + config.getConfig().putIfAbsent(AWSLockConfiguration.DYNAMODB_LOCK_READ_CAPACITY.key(), "20"); + config.getConfig().putIfAbsent(AWSLockConfiguration.DYNAMODB_LOCK_WRITE_CAPACITY.key(), "10"); + config.getConfig().putIfAbsent(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT.key(), "600000"); + } + + private String generateLogSuffixString() { + return StringUtils.join("DynamoDb table = ", tableName, ", partition key = ", dynamoDBPartitionKey); + } + + protected String generateLogStatement(LockState state, String suffix) { + return StringUtils.join(state.name(), " lock at ", suffix); + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/AWSLockConfiguration.java b/hudi-aws/src/main/java/org/apache/hudi/config/AWSLockConfiguration.java new file mode 100644 index 0000000000000..cb52f66a84274 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/config/AWSLockConfiguration.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.util.Option; + +import com.amazonaws.regions.RegionUtils; +import com.amazonaws.services.dynamodbv2.model.BillingMode; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX; + +public class AWSLockConfiguration { + + // configs for DynamoDb based locks + public static final String DYNAMODB_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "dynamodb."; + + public static final ConfigProperty DYNAMODB_LOCK_TABLE_NAME = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table") + .noDefaultValue() + .withDocumentation("For DynamoDB based lock provider, the name of the DynamoDB table acting as lock table"); + + public static final ConfigProperty DYNAMODB_LOCK_PARTITION_KEY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "partition_key") + .noDefaultValue() + .withInferFunction(cfg -> { + if (cfg.contains(HoodieWriteConfig.TBL_NAME)) { + return Option.of(cfg.getString(HoodieWriteConfig.TBL_NAME)); + } + return Option.empty(); + }) + .withDocumentation("For DynamoDB based lock provider, the partition key for the DynamoDB lock table. " + + "Each Hudi dataset should has it's unique key so concurrent writers could refer to the same partition key." + + " By default we use the Hudi table name specified to be the partition key"); + + public static final ConfigProperty DYNAMODB_LOCK_REGION = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "region") + .defaultValue("us-east-1") + .withInferFunction(cfg -> { + String regionFromEnv = System.getenv("AWS_REGION"); + if (regionFromEnv != null) { + return Option.of(RegionUtils.getRegion(regionFromEnv).getName()); + } + return Option.empty(); + }) + .withDocumentation("For DynamoDB based lock provider, the region used in endpoint for Amazon DynamoDB service." + + " Would try to first get it from AWS_REGION environment variable. If not find, by default use us-east-1"); + + public static final ConfigProperty DYNAMODB_LOCK_BILLING_MODE = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "billing_mode") + .defaultValue(BillingMode.PAY_PER_REQUEST.name()) + .withDocumentation("For DynamoDB based lock provider, by default it is PAY_PER_REQUEST mode"); + + public static final ConfigProperty DYNAMODB_LOCK_READ_CAPACITY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "read_capacity") + .defaultValue("20") + .withDocumentation("For DynamoDB based lock provider, read capacity units when using PROVISIONED billing mode"); + + public static final ConfigProperty DYNAMODB_LOCK_WRITE_CAPACITY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "write_capacity") + .defaultValue("10") + .withDocumentation("For DynamoDB based lock provider, write capacity units when using PROVISIONED billing mode"); + + public static final ConfigProperty DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table_creation_timeout") + .defaultValue(String.valueOf(10 * 60 * 1000)) + .withDocumentation("For DynamoDB based lock provider, the maximum number of milliseconds to wait for creating DynamoDB table"); +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java new file mode 100644 index 0000000000000..6c62b61528ada --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java @@ -0,0 +1,148 @@ +/* + * 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.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; +import javax.annotation.concurrent.Immutable; + +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_BILLING_MODE; +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_PARTITION_KEY; +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_READ_CAPACITY; +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_REGION; +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME; +import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_WRITE_CAPACITY; + +/** + * Configurations used by the AWS credentials and AWS DynamoDB based lock. + */ +@Immutable +@ConfigClassProperty(name = "AWS credential Configs", + groupName = ConfigGroups.Names.AWS_DYNAMO_DB, + description = "Configurations used for AWS credentials to get AWS resources.") +public class HoodieAWSConfig extends HoodieConfig { + public static final ConfigProperty AWS_ACCESS_KEY = ConfigProperty + .key("hoodie.aws.access.key") + .noDefaultValue() + .withDocumentation("AWS access key id"); + + public static final ConfigProperty AWS_SECRET_KEY = ConfigProperty + .key("hoodie.aws.secret.key") + .noDefaultValue() + .withDocumentation("AWS secret key"); + + public static final ConfigProperty AWS_SESSION_TOKEN = ConfigProperty + .key("hoodie.aws.session.token") + .noDefaultValue() + .withDocumentation("AWS session token"); + + private HoodieAWSConfig() { + super(); + } + + public static HoodieAWSConfig.Builder newBuilder() { + return new HoodieAWSConfig.Builder(); + } + + public String getAWSAccessKey() { + return getString(AWS_ACCESS_KEY); + } + + public String getAWSSecretKey() { + return getString(AWS_SECRET_KEY); + } + + public String getAWSSessionToken() { + return getString(AWS_SESSION_TOKEN); + } + + public static class Builder { + + private final HoodieAWSConfig awsConfig = new HoodieAWSConfig(); + + public HoodieAWSConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.awsConfig.getProps().load(reader); + return this; + } + } + + public HoodieAWSConfig.Builder fromProperties(Properties props) { + this.awsConfig.getProps().putAll(props); + return this; + } + + public HoodieAWSConfig.Builder withAccessKey(String accessKey) { + awsConfig.setValue(AWS_ACCESS_KEY, accessKey); + return this; + } + + public HoodieAWSConfig.Builder withSecretKey(String secretKey) { + awsConfig.setValue(AWS_SECRET_KEY, secretKey); + return this; + } + + public HoodieAWSConfig.Builder withSessionToken(String sessionToken) { + awsConfig.setValue(AWS_SESSION_TOKEN, sessionToken); + return this; + } + + public Builder withDynamoDBTable(String dynamoDbTableName) { + awsConfig.setValue(DYNAMODB_LOCK_TABLE_NAME, dynamoDbTableName); + return this; + } + + public Builder withDynamoDBPartitionKey(String partitionKey) { + awsConfig.setValue(DYNAMODB_LOCK_PARTITION_KEY, partitionKey); + return this; + } + + public Builder withDynamoDBRegion(String region) { + awsConfig.setValue(DYNAMODB_LOCK_REGION, region); + return this; + } + + public Builder withDynamoDBBillingMode(String mode) { + awsConfig.setValue(DYNAMODB_LOCK_BILLING_MODE, mode); + return this; + } + + public Builder withDynamoDBReadCapacity(String capacity) { + awsConfig.setValue(DYNAMODB_LOCK_READ_CAPACITY, capacity); + return this; + } + + public Builder withDynamoDBWriteCapacity(String capacity) { + awsConfig.setValue(DYNAMODB_LOCK_WRITE_CAPACITY, capacity); + return this; + } + + public HoodieAWSConfig build() { + awsConfig.setDefaults(HoodieAWSConfig.class.getName()); + return awsConfig; + } + } +} diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java new file mode 100644 index 0000000000000..051fe81e8b0ff --- /dev/null +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.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.hudi.aws; + +import com.amazonaws.auth.BasicSessionCredentials; +import org.apache.hudi.config.HoodieAWSConfig; +import org.apache.hudi.common.config.HoodieConfig; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestHoodieAWSCredentialsProviderFactory { + + @Test + public void testGetAWSCredentials() { + HoodieConfig cfg = new HoodieConfig(); + cfg.setValue(HoodieAWSConfig.AWS_ACCESS_KEY, "random-access-key"); + cfg.setValue(HoodieAWSConfig.AWS_SECRET_KEY, "random-secret-key"); + cfg.setValue(HoodieAWSConfig.AWS_SESSION_TOKEN, "random-session-token"); + BasicSessionCredentials credentials = (BasicSessionCredentials) org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(cfg.getProps()).getCredentials(); + assertEquals("random-access-key", credentials.getAWSAccessKeyId()); + assertEquals("random-secret-key", credentials.getAWSSecretKey()); + assertEquals("random-session-token", credentials.getSessionToken()); + } +} diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java new file mode 100644 index 0000000000000..8dc28328274fb --- /dev/null +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java @@ -0,0 +1,120 @@ +/* + * 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.hudi.aws.transaction.integ; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder; +import com.amazonaws.services.dynamodbv2.model.BillingMode; +import org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.config.AWSLockConfiguration; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.UUID; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY; + +/** + * Test for {@link DynamoDBBasedLockProvider}. + * Set it as integration test because it requires setting up docker environment. + */ +public class ITTestDynamoDBBasedLockProvider { + + private static LockConfiguration lockConfiguration; + private static AmazonDynamoDB dynamoDb; + + private static final String TABLE_NAME_PREFIX = "testDDBTable-"; + private static final String REGION = "us-east-2"; + + @BeforeAll + public static void setup() throws InterruptedException { + Properties properties = new Properties(); + properties.setProperty(AWSLockConfiguration.DYNAMODB_LOCK_BILLING_MODE.key(), BillingMode.PAY_PER_REQUEST.name()); + // properties.setProperty(AWSLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX); + properties.setProperty(AWSLockConfiguration.DYNAMODB_LOCK_PARTITION_KEY.key(), "testKey"); + properties.setProperty(AWSLockConfiguration.DYNAMODB_LOCK_REGION.key(), REGION); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000"); + properties.setProperty(AWSLockConfiguration.DYNAMODB_LOCK_READ_CAPACITY.key(), "0"); + properties.setProperty(AWSLockConfiguration.DYNAMODB_LOCK_WRITE_CAPACITY.key(), "0"); + lockConfiguration = new LockConfiguration(properties); + dynamoDb = getDynamoClientWithLocalEndpoint(); + } + + @Test + public void testAcquireLock() { + lockConfiguration.getConfig().setProperty(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + } + + @Test + public void testUnlock() { + lockConfiguration.getConfig().setProperty(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + } + + @Test + public void testReentrantLock() { + lockConfiguration.getConfig().setProperty(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + Assertions.assertFalse(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + } + + @Test + public void testUnlockWithoutLock() { + lockConfiguration.getConfig().setProperty(AWSLockConfiguration.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + dynamoDbBasedLockProvider.unlock(); + } + + private static AmazonDynamoDB getDynamoClientWithLocalEndpoint() { + String endpoint = System.getProperty("dynamodb-local.endpoint"); + if (endpoint == null || endpoint.isEmpty()) { + throw new IllegalStateException("dynamodb-local.endpoint system property not set"); + } + return AmazonDynamoDBClientBuilder.standard() + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(endpoint, REGION)) + .withCredentials(getCredentials()) + .build(); + } + + private static AWSCredentialsProvider getCredentials() { + return new AWSStaticCredentialsProvider(new BasicAWSCredentials("random-access-key", "random-secret-key")); + } +} diff --git a/hudi-aws/src/test/resources/log4j-surefire.properties b/hudi-aws/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000000000..a59d4ebe2b194 --- /dev/null +++ b/hudi-aws/src/test/resources/log4j-surefire.properties @@ -0,0 +1,25 @@ +### +# 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. +### +log4j.rootLogger=WARN, A1 +log4j.category.org.apache=INFO +log4j.category.org.apache.parquet.hadoop=WARN +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java index 33519b9adc0de..d5e87ec202392 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java @@ -72,7 +72,7 @@ public void lock() { LOG.info("Retrying to acquire lock..."); Thread.sleep(waitTimeInMs); retryCount++; - } catch (InterruptedException e) { + } catch (HoodieLockException | InterruptedException e) { if (retryCount >= retries) { throw new HoodieLockException("Unable to acquire lock, lock object ", e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java index fd7b310a57d6e..bf560810ae99b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -53,7 +53,6 @@ import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY; - /** * Hoodie Configs for Locks. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java index 08e1bb4a87a50..18cd8042763b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java @@ -31,7 +31,8 @@ public enum Names { WRITE_CLIENT("Write Client Configs"), METRICS("Metrics Configs"), RECORD_PAYLOAD("Record Payload Config"), - KAFKA_CONNECT("Kafka Connect Configs"); + KAFKA_CONNECT("Kafka Connect Configs"), + AWS_DYNAMO_DB("aws-dynamo-db"); public final String name; diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 868912eea4ff7..0c00c7de37b16 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -81,6 +81,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander @@ -149,6 +150,10 @@ org.apache.hbase:hbase-protocol org.apache.htrace:htrace-core commons-codec:commons-codec + + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index d55b39493dca2..32a9abf8f7c90 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -73,6 +73,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.beust:jcommander io.javalin:javalin @@ -100,6 +101,10 @@ com.yammer.metrics:metrics-core com.google.guava:guava + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core + org.apache.spark:spark-avro_${scala.binary.version} org.apache.hive:hive-common org.apache.hive:hive-service diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 63b3b7da8a4bd..c99a516fcbac3 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -97,6 +97,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander @@ -112,6 +113,10 @@ org.antlr:stringtemplate org.apache.parquet:parquet-avro + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core + com.twitter:bijection-avro_${scala.binary.version} com.twitter:bijection-core_${scala.binary.version} io.confluent:kafka-avro-serializer diff --git a/pom.xml b/pom.xml index 8336097e67491..2ae6ad7449a4b 100644 --- a/pom.xml +++ b/pom.xml @@ -38,6 +38,7 @@ hudi-common hudi-cli hudi-client + hudi-aws hudi-hadoop-mr hudi-spark-datasource hudi-timeline-service @@ -83,6 +84,7 @@ 0.15 1.7 3.0.0-M1 + 0.37.0 1.8 2.6.7 @@ -140,6 +142,7 @@ ${skipTests} ${skipTests} ${skipTests} + ${skipTests} UTF-8 ${project.basedir} provided @@ -158,6 +161,9 @@ 1.12.22 3.17.3 3.1.0 + 1.1.0 + 8000 + http://localhost:${dynamodb-local.port} @@ -333,6 +339,14 @@ jacoco-maven-plugin ${jacoco.version} + + io.fabric8 + docker-maven-plugin + ${maven-docker-plugin.version} + + ${skipDocker} + + @@ -1267,6 +1281,9 @@ **/IT*.java + + ${dynamodb-local.endpoint} + From f715cf607f0692e6500a295a0ab5604f3979df3a Mon Sep 17 00:00:00 2001 From: Manoj Govindassamy Date: Wed, 17 Nov 2021 10:59:38 -0800 Subject: [PATCH 021/217] [HUDI-2716] InLineFS support for S3FS logs (#3977) --- .../hudi/common/fs/inline/InLineFSUtils.java | 65 ++++++++++++------- .../common/fs/inline/InLineFileSystem.java | 4 +- .../table/log/HoodieLogFormatReader.java | 5 +- .../fs/inline/TestInLineFileSystem.java | 58 +++++++++++++++++ 4 files changed, 104 insertions(+), 28 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java index e4570f94227cf..a2c60bc318e4b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java @@ -19,6 +19,9 @@ package org.apache.hudi.common.fs.inline; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.util.ValidationUtils; + +import java.io.File; /** * Utils to parse InLineFileSystem paths. @@ -29,46 +32,58 @@ public class InLineFSUtils { private static final String START_OFFSET_STR = "start_offset"; private static final String LENGTH_STR = "length"; + private static final String PATH_SEPARATOR = "/"; + private static final String SCHEME_SEPARATOR = ":"; private static final String EQUALS_STR = "="; + private static final String LOCAL_FILESYSTEM_SCHEME = "file"; /** - * Fetch inline file path from outer path. - * Eg - * Input: - * Path = s3a://file1, origScheme: file, startOffset = 20, length = 40 - * Output: "inlinefs:/file1/s3a/?start_offset=20&length=40" + * Get the InlineFS Path for a given schema and its Path. + *

+ * Examples: + * Input Path: s3a://file1, origScheme: file, startOffset = 20, length = 40 + * Output: "inlinefs://file1/s3a/?start_offset=20&length=40" * - * @param outerPath - * @param origScheme - * @param inLineStartOffset - * @param inLineLength - * @return + * @param outerPath The outer file Path + * @param origScheme The file schema + * @param inLineStartOffset Start offset for the inline file + * @param inLineLength Length for the inline file + * @return InlineFS Path for the requested outer path and schema */ public static Path getInlineFilePath(Path outerPath, String origScheme, long inLineStartOffset, long inLineLength) { - String subPath = outerPath.toString().substring(outerPath.toString().indexOf(":") + 1); + final String subPath = new File(outerPath.toString().substring(outerPath.toString().indexOf(":") + 1)).getPath(); return new Path( - InLineFileSystem.SCHEME + "://" + subPath + "/" + origScheme - + "/" + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + InLineFileSystem.SCHEME + SCHEME_SEPARATOR + PATH_SEPARATOR + subPath + PATH_SEPARATOR + origScheme + + PATH_SEPARATOR + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + "&" + LENGTH_STR + EQUALS_STR + inLineLength ); } /** - * Inline file format - * "inlinefs:////?start_offset=start_offset>&length=" - * Outer File format - * "://" + * InlineFS Path format: + * "inlinefs://path/to/outer/file/outer_file_schema/?start_offset=start_offset>&length=" *

- * Eg input : "inlinefs://file1/sa3/?start_offset=20&length=40". - * Output : "sa3://file1" + * Outer File Path format: + * "outer_file_schema://path/to/outer/file" + *

+ * Example + * Input: "inlinefs://file1/s3a/?start_offset=20&length=40". + * Output: "s3a://file1" * - * @param inlinePath inline file system path - * @return + * @param inlineFSPath InLineFS Path to get the outer file Path + * @return Outer file Path from the InLineFS Path */ - public static Path getOuterfilePathFromInlinePath(Path inlinePath) { - String scheme = inlinePath.getParent().getName(); - Path basePath = inlinePath.getParent().getParent(); - return new Path(basePath.toString().replaceFirst(InLineFileSystem.SCHEME, scheme)); + public static Path getOuterFilePathFromInlinePath(Path inlineFSPath) { + final String scheme = inlineFSPath.getParent().getName(); + final Path basePath = inlineFSPath.getParent().getParent(); + ValidationUtils.checkArgument(basePath.toString().contains(SCHEME_SEPARATOR), + "Invalid InLineFSPath: " + inlineFSPath); + + final String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(SCHEME_SEPARATOR) + 1); + final String fullPath = scheme + SCHEME_SEPARATOR + + (scheme.equals(LOCAL_FILESYSTEM_SCHEME) ? PATH_SEPARATOR : "") + + pathExceptScheme; + return new Path(fullPath); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java index 4c693c5c5d0f4..712b6c7ff4e32 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java @@ -63,7 +63,7 @@ public String getScheme() { @Override public FSDataInputStream open(Path inlinePath, int bufferSize) throws IOException { - Path outerPath = InLineFSUtils.getOuterfilePathFromInlinePath(inlinePath); + Path outerPath = InLineFSUtils.getOuterFilePathFromInlinePath(inlinePath); FileSystem outerFs = outerPath.getFileSystem(conf); FSDataInputStream outerStream = outerFs.open(outerPath, bufferSize); return new InLineFsDataInputStream(InLineFSUtils.startOffset(inlinePath), outerStream, InLineFSUtils.length(inlinePath)); @@ -80,7 +80,7 @@ public boolean exists(Path f) { @Override public FileStatus getFileStatus(Path inlinePath) throws IOException { - Path outerPath = InLineFSUtils.getOuterfilePathFromInlinePath(inlinePath); + Path outerPath = InLineFSUtils.getOuterFilePathFromInlinePath(inlinePath); FileSystem outerFs = outerPath.getFileSystem(conf); FileStatus status = outerFs.getFileStatus(outerPath); FileStatus toReturn = new FileStatus(InLineFSUtils.length(inlinePath), status.isDirectory(), status.getReplication(), status.getBlockSize(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java index 36fa187aa4111..febdbf8068292 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java @@ -44,6 +44,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final Schema readerSchema; private final boolean readBlocksLazily; private final boolean reverseLogReader; + private final boolean enableInLineReading; private int bufferSize; private static final Logger LOG = LogManager.getLogger(HoodieLogFormatReader.class); @@ -62,6 +63,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { this.reverseLogReader = reverseLogReader; this.bufferSize = bufferSize; this.prevReadersInOpenState = new ArrayList<>(); + this.enableInLineReading = enableInlineReading; if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, enableInlineReading); @@ -104,7 +106,8 @@ public boolean hasNext() { this.prevReadersInOpenState.add(currentReader); } this.currentReader = - new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); + new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, + this.enableInLineReading); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java index 4553aa5a923ab..92f83aad7fd7e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java @@ -296,6 +296,64 @@ public void testsetWorkingDirectory() throws IOException { }, "Should have thrown exception"); } + static class TestFSPath { + final Path inputPath; + final Path expectedInLineFSPath; + final Path transformedInputPath; + + TestFSPath(final Path inputPath, final Path expectedInLineFSPath, final Path transformedInputPath) { + this.inputPath = inputPath; + this.expectedInLineFSPath = expectedInLineFSPath; + this.transformedInputPath = transformedInputPath; + } + } + + @Test + public void testInLineFSPathConversions() { + final List expectedInLinePaths = Arrays.asList( + new TestFSPath( + new Path("/zero/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://zero/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/zero/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("file:/one/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://one/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/one/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("file://two/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://two/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/two/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("hdfs://three/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://three/524bae7e-f01d-47ae-b7cd-910400a81336/hdfs/?start_offset=10&length=10"), + new Path("hdfs://three/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("s3://four/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://four/524bae7e-f01d-47ae-b7cd-910400a81336/s3/?start_offset=10&length=10"), + new Path("s3://four/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("s3a://five/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://five/524bae7e-f01d-47ae-b7cd-910400a81336/s3a/?start_offset=10&length=10"), + new Path("s3a://five/524bae7e-f01d-47ae-b7cd-910400a81336")) + ); + + for (TestFSPath entry : expectedInLinePaths) { + final Path inputPath = entry.inputPath; + final Path expectedInLineFSPath = entry.expectedInLineFSPath; + final Path expectedTransformedInputPath = entry.transformedInputPath; + + String scheme = "file"; + if (inputPath.toString().contains(":")) { + scheme = inputPath.toString().split(":")[0]; + } + final Path actualInLineFSPath = InLineFSUtils.getInlineFilePath(inputPath, scheme, 10, 10); + assertEquals(expectedInLineFSPath, actualInLineFSPath); + + final Path actualOuterFilePath = InLineFSUtils.getOuterFilePathFromInlinePath(actualInLineFSPath); + assertEquals(expectedTransformedInputPath, actualOuterFilePath); + } + } + @Test public void testExists() throws IOException { Path inlinePath = getRandomInlinePath(); From 2d3f2a3275ba615245fcabda96b8282cb86940ad Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Wed, 17 Nov 2021 14:43:00 -0500 Subject: [PATCH 022/217] [HUDI-2734] Setting default metadata enable as false for Java (#4003) --- .../apache/hudi/config/HoodieWriteConfig.java | 2 +- .../TestJavaCopyOnWriteActionExecutor.java | 4 +--- .../common/config/HoodieMetadataConfig.java | 22 +++++++++++++++++++ 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 496cd2d057ddc..7988e93075220 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -2179,7 +2179,7 @@ protected void setDefaults() { writeConfig.setDefaultOnCondition(!isPayloadConfigSet, HoodiePayloadConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isMetadataConfigSet, - HoodieMetadataConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + HoodieMetadataConfig.newBuilder().withEngineType(engineType).fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isLockConfigSet, HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isPreCommitValidationConfigSet, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 4a3f3d5bcef89..796d7b74a83c5 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; @@ -115,8 +114,7 @@ private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() { return HoodieWriteConfig.newBuilder() .withEngineType(EngineType.JAVA) .withPath(basePath) - .withSchema(SCHEMA.toString()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); + .withSchema(SCHEMA.toString()); } @Test diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 75d014521ff41..ea99014eacb50 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -18,6 +18,9 @@ package org.apache.hudi.common.config; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.exception.HoodieNotSupportedException; + import javax.annotation.concurrent.Immutable; import java.io.File; @@ -163,6 +166,7 @@ public boolean enableFullScan() { public static class Builder { + private EngineType engineType = EngineType.SPARK; private final HoodieMetadataConfig metadataConfig = new HoodieMetadataConfig(); public Builder fromFile(File propertiesFile) throws IOException { @@ -233,10 +237,28 @@ public Builder enableFullScan(boolean enableFullScan) { return this; } + public Builder withEngineType(EngineType engineType) { + this.engineType = engineType; + return this; + } + public HoodieMetadataConfig build() { + metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType)); metadataConfig.setDefaults(HoodieMetadataConfig.class.getName()); return metadataConfig; } + + private boolean getDefaultMetadataEnable(EngineType engineType) { + switch (engineType) { + case SPARK: + return ENABLE.defaultValue(); + case FLINK: + case JAVA: + return false; + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } } /** From 71a2ae0fd6163d3108c6557ce6295f649246adcb Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 18 Nov 2021 13:59:03 +0800 Subject: [PATCH 023/217] [HUDI-2789] Flink batch upsert for non partitioned table does not work (#4028) --- .../hudi/configuration/OptionsResolver.java | 8 +++++++ .../org/apache/hudi/sink/utils/Pipelines.java | 21 +++++++++---------- .../hudi/table/HoodieDataSourceITCase.java | 2 ++ .../apache/hudi/utils/TestConfigurations.java | 5 +++++ 4 files changed, 25 insertions(+), 11 deletions(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index 376b36e3dcd80..4499df87e5791 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.table.format.FilePathUtils; import org.apache.flink.configuration.Configuration; @@ -92,4 +93,11 @@ public static boolean isDeltaTimeCompaction(Configuration conf) { final String strategy = conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toLowerCase(Locale.ROOT); return FlinkOptions.TIME_ELAPSED.equals(strategy) || FlinkOptions.NUM_OR_TIME.equals(strategy); } + + /** + * Returns whether the table is partitioned. + */ + public static boolean isPartitionedTable(Configuration conf) { + return FilePathUtils.extractPartitionKeys(conf).length > 0; + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 855a2ca1b4d0d..5f156e839f1e3 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.sink.CleanFunction; import org.apache.hudi.sink.StreamWriteOperator; import org.apache.hudi.sink.append.AppendWriteOperator; @@ -129,10 +130,10 @@ public static DataStream bootstrap( final boolean globalIndex = conf.getBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED); if (overwrite) { return rowDataToHoodieRecord(conf, rowType, dataStream); - } else if (bounded && !globalIndex) { + } else if (bounded && !globalIndex && OptionsResolver.isPartitionedTable(conf)) { return boundedBootstrap(conf, rowType, defaultParallelism, dataStream); } else { - return streamBootstrap(conf, rowType, defaultParallelism, dataStream); + return streamBootstrap(conf, rowType, defaultParallelism, dataStream, bounded); } } @@ -140,10 +141,11 @@ private static DataStream streamBootstrap( Configuration conf, RowType rowType, int defaultParallelism, - DataStream dataStream) { + DataStream dataStream, + boolean bounded) { DataStream dataStream1 = rowDataToHoodieRecord(conf, rowType, dataStream); - if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { + if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) { dataStream1 = dataStream1 .transform( "index_bootstrap", @@ -161,13 +163,10 @@ private static DataStream boundedBootstrap( RowType rowType, int defaultParallelism, DataStream dataStream) { - final String[] partitionFields = FilePathUtils.extractPartitionKeys(conf); - if (partitionFields.length > 0) { - RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType); - // shuffle by partition keys - dataStream = dataStream - .keyBy(rowDataKeyGen::getPartitionPath); - } + final RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType); + // shuffle by partition keys + dataStream = dataStream + .keyBy(rowDataKeyGen::getPartitionPath); return rowDataToHoodieRecord(conf, rowType, dataStream) .transform( diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java index 285df4931e1d2..7d00a658b7f64 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java @@ -424,6 +424,8 @@ void testBatchModeUpsertWithoutPartition(HoodieTableType tableType) { String hoodieTableDDL = sql("t1") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) .option(FlinkOptions.TABLE_NAME, tableType.name()) + .option("hoodie.parquet.small.file.limit", "0") // invalidate the small file strategy + .option("hoodie.parquet.max.file.size", "0") .noPartition() .end(); tableEnv.executeSql(hoodieTableDDL); diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java index 0eafb1281ff4b..46cad3e826d3e 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java @@ -247,6 +247,11 @@ public Sql option(ConfigOption option, Object val) { return this; } + public Sql option(String key, Object val) { + this.options.put(key, val.toString()); + return this; + } + public Sql options(Map options) { this.options.putAll(options); return this; From 8772cec4bdfe71d4dfcc1b92cb39075c71a363b9 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 18 Nov 2021 16:40:48 +0800 Subject: [PATCH 024/217] [HUDI-2790] Fix the changelog mode of HoodieTableSource (#4029) --- .../org/apache/hudi/configuration/OptionsResolver.java | 10 ++++++++++ .../java/org/apache/hudi/table/HoodieTableSource.java | 9 ++++----- .../hudi/table/format/mor/MergeOnReadInputFormat.java | 4 ++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index 4499df87e5791..acb4af61110fa 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -100,4 +100,14 @@ public static boolean isDeltaTimeCompaction(Configuration conf) { public static boolean isPartitionedTable(Configuration conf) { return FilePathUtils.extractPartitionKeys(conf).length > 0; } + + /** + * Returns whether the source should emit changelog. + * + * @return true if the source is read as streaming with changelog mode enabled + */ + public static boolean emitChangelog(Configuration conf) { + return conf.getBoolean(FlinkOptions.READ_AS_STREAMING) + && conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED); + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index f0dbffd4732fe..62baac4cc6b2c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.HoodieROTablePathFilter; import org.apache.hudi.source.FileIndex; @@ -196,11 +197,9 @@ public DataStream produceDataStream(StreamExecutionEnvironment execEnv) @Override public ChangelogMode getChangelogMode() { - return conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED) - ? ChangelogModes.FULL - // when all the changes are persisted or read as batch, - // use INSERT mode. - : ChangelogMode.insertOnly(); + // when read as streaming and changelog mode is enabled, emit as FULL mode; + // when all the changes are compacted or read as batch, emit as INSERT mode. + return OptionsResolver.emitChangelog(conf) ? ChangelogModes.FULL : ChangelogMode.insertOnly(); } @Override diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 2bf5bd58edb1f..95c4bd4a5431b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.format.FilePathUtils; @@ -179,8 +180,7 @@ public void open(MergeOnReadInputSplit split) throws IOException { } } else if (!split.getBasePath().isPresent()) { // log files only - if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING) - && conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) { + if (OptionsResolver.emitChangelog(conf)) { this.iterator = new LogFileOnlyIterator(getUnMergedLogFileIterator(split)); } else { this.iterator = new LogFileOnlyIterator(getLogFileIterator(split)); From 24def0b30dabd3c447fbfbde636868f1913e9074 Mon Sep 17 00:00:00 2001 From: wenningd Date: Thu, 18 Nov 2021 01:59:26 -0800 Subject: [PATCH 025/217] [HUDI-2362] Add external config file support (#3416) Co-authored-by: Wenning Ding --- conf/hudi-defaults.conf.template | 26 +++ .../apache/hudi/cli/commands/SparkMain.java | 2 +- .../config/DFSPropertiesConfiguration.java | 155 ++++++++++++++---- .../hudi/common/config/HoodieConfig.java | 20 ++- .../util/TestDFSPropertiesConfiguration.java | 44 ++++- .../external-config/hudi-defaults.conf | 26 +++ .../hudi/streamer/HoodieFlinkStreamer.java | 7 +- .../org/apache/hudi/util/StreamerUtil.java | 13 +- .../integ/testsuite/HoodieTestSuiteJob.java | 2 +- .../org/apache/hudi/HoodieWriterUtils.scala | 24 ++- .../spark/sql/hudi/HoodieSqlUtils.scala | 3 +- .../AlterHoodieTableRenameCommand.scala | 2 +- .../command/CreateHoodieTableCommand.scala | 15 +- .../command/DeleteHoodieTableCommand.scala | 2 +- .../command/TruncateHoodieTableCommand.scala | 2 +- .../external-config/hudi-defaults.conf | 23 +++ .../apache/spark/sql/hudi/TestSqlConf.scala | 104 ++++++++++++ .../hudi/utilities/HDFSParquetImporter.java | 2 +- .../apache/hudi/utilities/HoodieCleaner.java | 2 +- .../hudi/utilities/HoodieClusteringJob.java | 2 +- .../hudi/utilities/HoodieCompactor.java | 2 +- .../apache/hudi/utilities/UtilHelpers.java | 15 +- .../deltastreamer/HoodieDeltaStreamer.java | 23 ++- .../HoodieMultiTableDeltaStreamer.java | 4 +- .../functional/TestHoodieDeltaStreamer.java | 4 +- 25 files changed, 424 insertions(+), 100 deletions(-) create mode 100644 conf/hudi-defaults.conf.template create mode 100644 hudi-common/src/test/resources/external-config/hudi-defaults.conf create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala diff --git a/conf/hudi-defaults.conf.template b/conf/hudi-defaults.conf.template new file mode 100644 index 0000000000000..175dbaf23d739 --- /dev/null +++ b/conf/hudi-defaults.conf.template @@ -0,0 +1,26 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +# hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 +# hoodie.datasource.hive_sync.use_jdbc true +# hoodie.datasource.hive_sync.support_timestamp false +# hoodie.index.type BLOOM +# hoodie.metadata.enable false diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 82688fecc3663..ef6416c0ca680 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -360,7 +360,7 @@ private static int doBootstrap(JavaSparkContext jsc, String tableName, String ta String payloadClassName, String enableHiveSync, String propsFilePath, List configs) throws IOException { TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs) - : UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getConfig(); + : UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getProps(true); properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java index f97e4add462df..88335db130ae7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java @@ -18,14 +18,20 @@ package org.apache.hudi.common.config; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.io.InputStreamReader; +import java.net.URI; import java.util.HashSet; import java.util.Set; @@ -43,72 +49,110 @@ public class DFSPropertiesConfiguration { private static final Logger LOG = LogManager.getLogger(DFSPropertiesConfiguration.class); + public static final String DEFAULT_PROPERTIES_FILE = "hudi-defaults.conf"; + + public static final String CONF_FILE_DIR_ENV_NAME = "HUDI_CONF_DIR"; + + public static final String DEFAULT_CONF_FILE_DIR = "file:/etc/hudi/conf"; + + // props read from hudi-defaults.conf + private static TypedProperties GLOBAL_PROPS = loadGlobalProps(); + private final FileSystem fs; - private final Path rootFile; + private Path currentFilePath; - private final TypedProperties props; + // props read from user defined configuration file or input stream + private final HoodieConfig hoodieConfig; // Keep track of files visited, to detect loops - private final Set visitedFiles; + private final Set visitedFilePaths; - public DFSPropertiesConfiguration(FileSystem fs, Path rootFile, TypedProperties defaults) { + public DFSPropertiesConfiguration(FileSystem fs, Path filePath) { this.fs = fs; - this.rootFile = rootFile; - this.props = defaults; - this.visitedFiles = new HashSet<>(); - visitFile(rootFile); - } - - public DFSPropertiesConfiguration(FileSystem fs, Path rootFile) { - this(fs, rootFile, new TypedProperties()); + this.currentFilePath = filePath; + this.hoodieConfig = new HoodieConfig(); + this.visitedFilePaths = new HashSet<>(); + addPropsFromFile(filePath); } public DFSPropertiesConfiguration() { this.fs = null; - this.rootFile = null; - this.props = new TypedProperties(); - this.visitedFiles = new HashSet<>(); + this.currentFilePath = null; + this.hoodieConfig = new HoodieConfig(); + this.visitedFilePaths = new HashSet<>(); } - private String[] splitProperty(String line) { - int ind = line.indexOf('='); - String k = line.substring(0, ind).trim(); - String v = line.substring(ind + 1).trim(); - return new String[] {k, v}; + /** + * Load global props from hudi-defaults.conf which is under CONF_FILE_DIR_ENV_NAME. + * @return Typed Properties + */ + public static TypedProperties loadGlobalProps() { + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(); + Option defaultConfPath = getConfPathFromEnv(); + if (defaultConfPath.isPresent()) { + conf.addPropsFromFile(defaultConfPath.get()); + } else { + try { + conf.addPropsFromFile(new Path(DEFAULT_CONF_FILE_DIR)); + } catch (Exception ignored) { + LOG.debug("Didn't find config file under default conf file dir: " + DEFAULT_CONF_FILE_DIR); + } + } + return conf.getProps(); + } + + public static void refreshGlobalProps() { + GLOBAL_PROPS = loadGlobalProps(); + } + + public static void clearGlobalProps() { + GLOBAL_PROPS = new TypedProperties(); } - private void visitFile(Path file) { + /** + * Add properties from external configuration files. + * + * @param filePath File path for configuration file + */ + public void addPropsFromFile(Path filePath) { + if (visitedFilePaths.contains(filePath.toString())) { + throw new IllegalStateException("Loop detected; file " + filePath + " already referenced"); + } + FileSystem fileSystem; try { - if (visitedFiles.contains(file.getName())) { - throw new IllegalStateException("Loop detected; file " + file + " already referenced"); - } - visitedFiles.add(file.getName()); - BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file))); - addProperties(reader); + fileSystem = fs != null ? fs : filePath.getFileSystem(new Configuration()); + } catch (IOException e) { + throw new IllegalArgumentException("Cannot get the file system from file path", e); + } + try (BufferedReader reader = new BufferedReader(new InputStreamReader(fileSystem.open(filePath)))) { + visitedFilePaths.add(filePath.toString()); + currentFilePath = filePath; + addPropsFromStream(reader); } catch (IOException ioe) { - LOG.error("Error reading in properies from dfs", ioe); + LOG.error("Error reading in properties from dfs"); throw new IllegalArgumentException("Cannot read properties from dfs", ioe); } } /** - * Add properties from input stream. - * + * Add properties from buffered reader. + * * @param reader Buffered Reader * @throws IOException */ - public void addProperties(BufferedReader reader) throws IOException { + public void addPropsFromStream(BufferedReader reader) throws IOException { try { reader.lines().forEach(line -> { - if (line.startsWith("#") || line.equals("") || !line.contains("=")) { + if (!isValidLine(line)) { return; } String[] split = splitProperty(line); if (line.startsWith("include=") || line.startsWith("include =")) { - visitFile(new Path(rootFile.getParent(), split[1])); + Path includeFilePath = new Path(currentFilePath.getParent(), split[1]); + addPropsFromFile(includeFilePath); } else { - props.setProperty(split[0], split[1]); + hoodieConfig.setValue(split[0], split[1]); } }); @@ -117,7 +161,46 @@ public void addProperties(BufferedReader reader) throws IOException { } } - public TypedProperties getConfig() { - return props; + public static TypedProperties getGlobalProps() { + final TypedProperties globalProps = new TypedProperties(); + globalProps.putAll(GLOBAL_PROPS); + return globalProps; + } + + public TypedProperties getProps() { + return new TypedProperties(hoodieConfig.getProps()); + } + + public TypedProperties getProps(boolean includeGlobalProps) { + return new TypedProperties(hoodieConfig.getProps(includeGlobalProps)); + } + + private static Option getConfPathFromEnv() { + String confDir = System.getenv(CONF_FILE_DIR_ENV_NAME); + if (confDir == null) { + LOG.warn("Cannot find " + CONF_FILE_DIR_ENV_NAME + ", please set it as the dir of " + DEFAULT_PROPERTIES_FILE); + return Option.empty(); + } + if (StringUtils.isNullOrEmpty(URI.create(confDir).getScheme())) { + confDir = "file://" + confDir; + } + return Option.of(new Path(confDir + File.separator + DEFAULT_PROPERTIES_FILE)); + } + + private String[] splitProperty(String line) { + line = line.replaceAll("\\s+"," "); + String delimiter = line.contains("=") ? "=" : " "; + int ind = line.indexOf(delimiter); + String k = line.substring(0, ind).trim(); + String v = line.substring(ind + 1).trim(); + return new String[] {k, v}; + } + + private boolean isValidLine(String line) { + ValidationUtils.checkArgument(line != null, "passed line is null"); + if (line.startsWith("#") || line.equals("")) { + return false; + } + return line.contains("=") || line.matches(".*\\s.*"); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index ed2b90eeae4f2..6ae1ba6e34dde 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -58,6 +58,14 @@ public void setValue(ConfigProperty cfg, String val) { props.setProperty(cfg.key(), val); } + public void setValue(String key, String val) { + props.setProperty(key, val); + } + + public void setAll(Properties properties) { + props.putAll(properties); + } + public void setDefaultValue(ConfigProperty configProperty) { if (!contains(configProperty)) { Option inferValue = Option.empty(); @@ -167,7 +175,17 @@ public String getStringOrDefault(ConfigProperty configProperty, String de } public Properties getProps() { - return props; + return getProps(false); + } + + public Properties getProps(boolean includeGlobalProps) { + if (includeGlobalProps) { + Properties mergedProps = DFSPropertiesConfiguration.getGlobalProps(); + mergedProps.putAll(props); + return mergedProps; + } else { + return props; + } } public void setDefaultOnCondition(boolean condition, HoodieConfig config) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java index ddce3216b0d6a..b7ea8bff57c55 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java @@ -25,10 +25,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.Rule; +import org.junit.contrib.java.lang.system.EnvironmentVariables; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import java.io.File; import java.io.IOException; import java.io.PrintStream; @@ -47,6 +51,10 @@ public class TestDFSPropertiesConfiguration { private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; + @Rule + public static final EnvironmentVariables ENVIRONMENT_VARIABLES + = new EnvironmentVariables(); + @BeforeAll public static void initClass() throws Exception { hdfsTestService = new HdfsTestService(); @@ -73,12 +81,17 @@ public static void initClass() throws Exception { } @AfterAll - public static void cleanupClass() throws Exception { + public static void cleanupClass() { if (hdfsTestService != null) { hdfsTestService.stop(); } } + @AfterEach + public void cleanupGlobalConfig() { + DFSPropertiesConfiguration.clearGlobalProps(); + } + private static void writePropertiesFile(Path path, String[] lines) throws IOException { PrintStream out = new PrintStream(dfs.create(path, true)); for (String line : lines) { @@ -91,7 +104,7 @@ private static void writePropertiesFile(Path path, String[] lines) throws IOExce @Test public void testParsing() { DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props")); - TypedProperties props = cfg.getConfig(); + TypedProperties props = cfg.getProps(); assertEquals(5, props.size()); assertThrows(IllegalArgumentException.class, () -> { props.getString("invalid.key"); @@ -119,7 +132,7 @@ public void testParsing() { @Test public void testIncludes() { DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t3.props")); - TypedProperties props = cfg.getConfig(); + TypedProperties props = cfg.getProps(); assertEquals(123, props.getInteger("int.prop")); assertEquals(243.4, props.getDouble("double.prop"), 0.001); @@ -127,7 +140,30 @@ public void testIncludes() { assertEquals("t3.value", props.getString("string.prop")); assertEquals(1354354354, props.getLong("long.prop")); assertThrows(IllegalStateException.class, () -> { - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t4.props")); + cfg.addPropsFromFile(new Path(dfsBasePath + "/t4.props")); }, "Should error out on a self-included file."); } + + @Test + public void testNoGlobalConfFileConfigured() { + ENVIRONMENT_VARIABLES.clear(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME); + // Should not throw any exception when no external configuration file configured + DFSPropertiesConfiguration.refreshGlobalProps(); + assertEquals(0, DFSPropertiesConfiguration.getGlobalProps().size()); + } + + @Test + public void testLoadGlobalConfFile() { + // set HUDI_CONF_DIR + String testPropsFilePath = new File("src/test/resources/external-config").getAbsolutePath(); + ENVIRONMENT_VARIABLES.set(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, testPropsFilePath); + + DFSPropertiesConfiguration.refreshGlobalProps(); + assertEquals(5, DFSPropertiesConfiguration.getGlobalProps().size()); + assertEquals("jdbc:hive2://localhost:10000", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.jdbcurl")); + assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.use_jdbc")); + assertEquals("false", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.support_timestamp")); + assertEquals("BLOOM", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.index.type")); + assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.metadata.enable")); + } } diff --git a/hudi-common/src/test/resources/external-config/hudi-defaults.conf b/hudi-common/src/test/resources/external-config/hudi-defaults.conf new file mode 100644 index 0000000000000..1133adb4d7735 --- /dev/null +++ b/hudi-common/src/test/resources/external-config/hudi-defaults.conf @@ -0,0 +1,26 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 +hoodie.datasource.hive_sync.use_jdbc true +hoodie.datasource.hive_sync.support_timestamp false +hoodie.index.type BLOOM +hoodie.metadata.enable true diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java index 3e567f31fa63a..851931f0d76ba 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java @@ -18,6 +18,8 @@ package org.apache.hudi.streamer; +import org.apache.hudi.common.config.DFSPropertiesConfiguration; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; @@ -38,8 +40,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; -import java.util.Properties; - /** * A utility which can incrementally consume data from Kafka and apply it to the target table. * It has the similar functionality with SQL data source except that the source is bind to Kafka @@ -65,7 +65,8 @@ public static void main(String[] args) throws Exception { env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); } - Properties kafkaProps = StreamerUtil.appendKafkaProps(cfg); + TypedProperties kafkaProps = DFSPropertiesConfiguration.getGlobalProps(); + kafkaProps.putAll(StreamerUtil.appendKafkaProps(cfg)); // Read from kafka source RowType rowType = diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index ddbd24e3562d0..4b9a516106426 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -103,7 +103,7 @@ public static TypedProperties getProps(FlinkStreamerConfig cfg) { } return readConfig( FSUtils.getFs(cfg.propsFilePath, getHadoopConf()), - new Path(cfg.propsFilePath), cfg.configs).getConfig(); + new Path(cfg.propsFilePath), cfg.configs).getProps(); } public static Schema getSourceSchema(FlinkStreamerConfig cfg) { @@ -128,18 +128,11 @@ public static Schema getSourceSchema(org.apache.flink.configuration.Configuratio * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option). */ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List overriddenProps) { - DFSPropertiesConfiguration conf; - try { - conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath); - } catch (Exception e) { - conf = new DFSPropertiesConfiguration(); - LOG.warn("Unexpected error read props file at :" + cfgPath, e); - } - + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath); try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index d8ed649d9b08b..8e0e3eb5d01ef 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -103,7 +103,7 @@ public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throw cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString(); this.sparkSession = SparkSession.builder().config(jsc.getConf()).enableHiveSupport().getOrCreate(); this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration()); - this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps(); log.info("Creating workload generator with configs : {}", props.toString()); this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration()); this.keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index c1223d9792273..c89c19dde69f5 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.hudi.DataSourceOptionsHelper.allAlternatives import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE -import org.apache.hudi.common.config.{HoodieConfig, TypedProperties} +import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.SparkSession @@ -47,6 +47,7 @@ object HoodieWriterUtils { * @return */ def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = { + val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala Map(OPERATION.key -> OPERATION.defaultValue, TABLE_TYPE.key -> TABLE_TYPE.defaultValue, PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue, @@ -81,7 +82,7 @@ object HoodieWriterUtils { ENABLE_ROW_WRITER.key -> ENABLE_ROW_WRITER.defaultValue, RECONCILE_SCHEMA.key -> RECONCILE_SCHEMA.defaultValue.toString, DROP_PARTITION_COLUMNS.key -> DROP_PARTITION_COLUMNS.defaultValue - ) ++ DataSourceOptionsHelper.translateConfigurations(parameters) + ) ++ globalProps ++ DataSourceOptionsHelper.translateConfigurations(parameters) } def toProperties(params: Map[String, String]): TypedProperties = { @@ -170,4 +171,23 @@ object HoodieWriterUtils { } } } + + val sparkDatasourceConfigsToTableConfigsMap = Map( + TABLE_NAME -> HoodieTableConfig.NAME, + TABLE_TYPE -> HoodieTableConfig.TYPE, + PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELD, + PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS, + RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS, + PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME + ) + def mappingSparkDatasourceConfigsToTableConfigs(options: Map[String, String]): Map[String, String] = { + val includingTableConfigs = scala.collection.mutable.Map() ++ options + sparkDatasourceConfigsToTableConfigsMap.foreach(kv => { + if (options.containsKey(kv._1.key)) { + includingTableConfigs(kv._2.key) = options(kv._1.key) + includingTableConfigs.remove(kv._1.key) + } + }) + includingTableConfigs.toMap + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala index 8e490335080e0..50c9e539ccb21 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.common.config.DFSPropertiesConfiguration import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord @@ -273,7 +274,7 @@ object HoodieSqlUtils extends SparkAdapterSupport { */ def withSparkConf(spark: SparkSession, options: Map[String, String]) (baseConfig: Map[String, String] = Map.empty): Map[String, String] = { - baseConfig ++ // Table options has the highest priority + baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options)) .filterKeys(_.startsWith("hoodie.")) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala index 2df9ec86998cb..ec71a9d3c1aaa 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala @@ -43,7 +43,7 @@ class AlterHoodieTableRenameCommand( .setConf(hadoopConf).build() // Init table with new name. HoodieTableMetaClient.withPropertyBuilder() - .fromProperties(metaClient.getTableConfig.getProps) + .fromProperties(metaClient.getTableConfig.getProps(true)) .setTableName(newName.table) .initTable(hadoopConf, path) // Call AlterTableRenameCommand#run to rename table in meta. diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 94c6eab75664d..e449a91a12799 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.hudi.command import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - -import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} +import org.apache.hudi.{DataSourceWriteOptions, HoodieWriterUtils, SparkAdapterSupport} import org.apache.hudi.HoodieWriterUtils._ +import org.apache.hudi.common.config.DFSPropertiesConfiguration import org.apache.hudi.common.model.HoodieFileFormat import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.hadoop.HoodieParquetInputFormat @@ -47,7 +47,7 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.{SPARK_VERSION, SparkConf} import java.util.{Locale, Properties} - +import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.control.NonFatal @@ -100,7 +100,10 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean // if CTAS, we treat the table we just created as nonexistent val isTableExists = if (ctas) false else tableExistsInPath(path, conf) var existingTableConfig = Map.empty[String, String] - val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(tblProperties) + val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap + val globalSqlProps = HoodieOptionConfig.mappingTableConfigToSqlOption( + HoodieWriterUtils.mappingSparkDatasourceConfigsToTableConfigs(globalProps)) + val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(globalSqlProps ++ tblProperties) val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(tblProperties) // get final schema and parameters @@ -341,7 +344,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean } } else { extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true" - extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue() + extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue } if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) { @@ -374,7 +377,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) // Save all the table config to the hoodie.properties. - val parameters = originTableConfig ++ tableOptions + val parameters = HoodieWriterUtils.mappingSparkDatasourceConfigsToTableConfigs(originTableConfig ++ tableOptions) val properties = new Properties() properties.putAll(parameters.asJava) HoodieTableMetaClient.withPropertyBuilder() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala index e8acebd787073..98cc4dd726eaf 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hudi.command -import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _} +import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala index 339f4b52cba0b..a57a3d1bd4570 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala @@ -45,7 +45,7 @@ class TruncateHoodieTableCommand( // Create MetaClient val metaClient = HoodieTableMetaClient.builder().setBasePath(path) .setConf(hadoopConf).build() - Some(metaClient.getTableConfig.getProps) + Some(metaClient.getTableConfig.getProps(true)) } else { None } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf b/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf new file mode 100644 index 0000000000000..c883b5bbe83d6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf @@ -0,0 +1,23 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +hoodie.datasource.write.table.type MERGE_ON_READ +hoodie.datasource.write.hive_style_partitioning false diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala new file mode 100644 index 0000000000000..1a8ac0e645899 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala @@ -0,0 +1,104 @@ +/* + * 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.spark.sql.hudi + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.config.DFSPropertiesConfiguration +import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} + +import java.io.File +import java.nio.file.{Files, Paths} + +import org.scalatest.BeforeAndAfter + +class TestSqlConf extends TestHoodieSqlBase with BeforeAndAfter { + + def setEnv(key: String, value: String): String = { + val field = System.getenv().getClass.getDeclaredField("m") + field.setAccessible(true) + val map = field.get(System.getenv()).asInstanceOf[java.util.Map[java.lang.String, java.lang.String]] + map.put(key, value) + } + + test("Test Hudi Conf") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = tmp.getCanonicalPath + val partitionVal = "2021" + // Create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | year string + |) using hudi + | partitioned by (year) + | location '$tablePath' + | options ( + | primaryKey ='id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // First merge with a extra input field 'flag' (insert a new record) + spark.sql( + s""" + | merge into $tableName + | using ( + | select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '1' as flag, $partitionVal as year + | ) s0 + | on s0.id = $tableName.id + | when matched and flag = '1' then update set + | id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, year = s0.year + | when not matched and flag = '1' then insert * + """.stripMargin) + checkAnswer(s"select id, name, price, ts, year from $tableName")( + Seq(1, "a1", 10.0, 1000, partitionVal) + ) + + // By default, Spark DML would set table type to COW and use Hive style partitioning, here we + // set table type to MOR and disable Hive style partitioning in the hudi conf file, and check + // if Hudi DML can load these configs correctly + assertResult(true)(Files.exists(Paths.get(s"$tablePath/$partitionVal"))) + assertResult(HoodieTableType.MERGE_ON_READ)(new HoodieTableConfig( + new Path(tablePath).getFileSystem(new Configuration), + s"$tablePath/" + HoodieTableMetaClient.METAFOLDER_NAME, + HoodieTableConfig.PAYLOAD_CLASS_NAME.defaultValue).getTableType) + + // delete the record + spark.sql(s"delete from $tableName where year = $partitionVal") + val cnt = spark.sql(s"select * from $tableName where year = $partitionVal").count() + assertResult(0)(cnt) + } + } + + before { + val testPropsFilePath = new File("src/test/resources/external-config").getAbsolutePath + setEnv(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, testPropsFilePath) + DFSPropertiesConfiguration.refreshGlobalProps() + } + + after { + DFSPropertiesConfiguration.clearGlobalProps() + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index 5f9b199fe6952..21393a01b5ed8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -112,7 +112,7 @@ private boolean isUpsert() { public int dataImport(JavaSparkContext jsc, int retry) { this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) - : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps(true); LOG.info("Starting data import with configs : " + props.toString()); int ret = -1; try { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java index 24e2828a5e2f5..28d484c56589f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java @@ -63,7 +63,7 @@ public HoodieCleaner(Config cfg, JavaSparkContext jssc) { */ FileSystem fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration()); this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) - : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps(true); LOG.info("Creating Cleaner with configs : " + props.toString()); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 2bd9ab43a747f..a96a4b75e0f39 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -69,7 +69,7 @@ private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cf return UtilHelpers .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) - .getConfig(); + .getProps(true); } public static class Config implements Serializable { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java index 1996fb8b5f6ed..74a4ea59f39cd 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java @@ -59,7 +59,7 @@ private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cf return UtilHelpers .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) - .getConfig(); + .getProps(true); } public static class Config implements Serializable { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 3771a7d34b5a7..1d74aced5f1d4 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -161,18 +161,11 @@ public static InitialCheckPointProvider createInitialCheckpointProvider( * */ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List overriddenProps) { - DFSPropertiesConfiguration conf; - try { - conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath); - } catch (Exception e) { - conf = new DFSPropertiesConfiguration(); - LOG.warn("Unexpected error read props file at :" + cfgPath, e); - } - + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath); try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); @@ -186,7 +179,7 @@ public static DFSPropertiesConfiguration getConfig(List overriddenProps) try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); @@ -196,7 +189,7 @@ public static DFSPropertiesConfiguration getConfig(List overriddenProps) } public static TypedProperties buildProperties(List props) { - TypedProperties properties = new TypedProperties(); + TypedProperties properties = DFSPropertiesConfiguration.getGlobalProps(); props.forEach(x -> { String[] kv = x.split("="); ValidationUtils.checkArgument(kv.length == 2); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 1649759f72528..408a71484d8d4 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -29,6 +29,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.OperationConverter; import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -121,15 +122,18 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Con public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf, Option props) throws IOException { // Resolving the properties first in a consistent way + HoodieConfig hoodieConfig = new HoodieConfig(); if (props.isPresent()) { - this.properties = setDefaults(props.get()); + hoodieConfig.setAll(props.get()); } else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) { - this.properties = setDefaults(UtilHelpers.getConfig(cfg.configs).getConfig()); + hoodieConfig.setAll(UtilHelpers.getConfig(cfg.configs).getProps()); } else { - this.properties = setDefaults(UtilHelpers.readConfig( + hoodieConfig.setAll(UtilHelpers.readConfig( FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()), - new Path(cfg.propsFilePath), cfg.configs).getConfig()); + new Path(cfg.propsFilePath), cfg.configs).getProps()); } + hoodieConfig.setDefaultValue(DataSourceWriteOptions.RECONCILE_SCHEMA()); + this.properties = (TypedProperties) hoodieConfig.getProps(true); if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) { InitialCheckPointProvider checkPointProvider = @@ -148,13 +152,6 @@ public void shutdownGracefully() { deltaSyncService.ifPresent(ds -> ds.shutdown(false)); } - private TypedProperties setDefaults(TypedProperties props) { - if (!props.containsKey(DataSourceWriteOptions.RECONCILE_SCHEMA().key())) { - props.setProperty(DataSourceWriteOptions.RECONCILE_SCHEMA().key(), DataSourceWriteOptions.RECONCILE_SCHEMA().defaultValue().toString()); - } - return props; - } - /** * Main method to start syncing. * @@ -370,12 +367,12 @@ public boolean isInlineCompactionEnabled() { } public boolean isAsyncClusteringEnabled() { - return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig() + return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getProps() .getOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE.key(), false))); } public boolean isInlineClusteringEnabled() { - return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig() + return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getProps() .getOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING.key(), false))); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index 7e49d9b88f69f..2b788dc6d43dd 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -77,7 +77,7 @@ public HoodieMultiTableDeltaStreamer(Config config, JavaSparkContext jssc) throw FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration()); configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder; checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs); - TypedProperties commonProperties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig(); + TypedProperties commonProperties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getProps(); //get the tables to be ingested and their corresponding config files from this properties instance populateTableExecutionContextList(commonProperties, configFolder, fs, config); } @@ -116,7 +116,7 @@ private void populateTableExecutionContextList(TypedProperties properties, Strin String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX; String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable)); checkIfTableConfigFileExists(configFolder, fs, configFilePath); - TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig(); + TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getProps(); properties.forEach((k, v) -> { if (tableProperties.get(k) == null) { tableProperties.setProperty(k.toString(), v.toString()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 014a0c140d62e..9cf040ceaa5b0 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -365,7 +365,7 @@ static void assertAtLeastNReplaceRequests(int minExpected, String tablePath, Fil @Test public void testProps() { TypedProperties props = - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig(); + new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps(); assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism")); assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field")); assertEquals("org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer$TestGenerator", @@ -485,7 +485,7 @@ public void testKafkaConnectCheckpointProvider() throws IOException { String checkpointProviderClass = "org.apache.hudi.utilities.checkpointing.KafkaConnectHdfsProvider"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT); TypedProperties props = - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig(); + new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps(); props.put("hoodie.deltastreamer.checkpoint.provider.path", bootstrapPath); cfg.initialCheckpointProvider = checkpointProviderClass; // create regular kafka connect hdfs dirs From 4e067ca581975596df04a30296a537fe1905bbfe Mon Sep 17 00:00:00 2001 From: Udit Mehrotra Date: Thu, 18 Nov 2021 05:33:50 -0800 Subject: [PATCH 026/217] [HUDI-2641] Avoid deleting all inflight commits heartbeats while rolling back failed writes (#3956) --- .../client/AbstractHoodieWriteClient.java | 9 +-- .../hudi/client/heartbeat/HeartbeatUtils.java | 31 ++++---- .../client/TestHoodieClientMultiWriter.java | 66 ++++++++++++++++ .../hudi/client/TestTableSchemaEvolution.java | 4 +- .../hudi/testutils/HoodieClientTestBase.java | 78 +++++++++++++++++-- 5 files changed, 156 insertions(+), 32 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index d98da346e13e4..c2089466fe14b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -852,18 +852,13 @@ protected void rollbackFailedWrites(List instantsToRollback, boolean ski if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { rollbackFailedBootstrap(); + HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config); break; } else { rollback(instant, skipLocking); + HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config); } } - // Delete any heartbeat files for already rolled back commits - try { - HeartbeatUtils.cleanExpiredHeartbeats(this.heartbeatClient.getAllExistingHeartbeatInstants(), - createMetaClient(true), basePath); - } catch (IOException io) { - LOG.error("Unable to delete heartbeat files", io); - } } protected List getInstantsToRollback(HoodieTableMetaClient metaClient, HoodieFailedWritesCleaningPolicy cleaningPolicy, Option curInstantTime) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java index 80191d4c3cdc2..a20469429030a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -30,9 +29,6 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; /** * Helper class to delete heartbeat for completed or failed instants with expired heartbeats. @@ -55,6 +51,8 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String deleted = fs.delete(new Path(heartbeatFolderPath + Path.SEPARATOR + instantTime), false); if (!deleted) { LOG.error("Failed to delete heartbeat for instant " + instantTime); + } else { + LOG.info("Deleted the heartbeat for instant " + instantTime); } } catch (IOException io) { LOG.error("Unable to delete heartbeat for instant " + instantTime, io); @@ -63,20 +61,19 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String } /** - * Deletes the heartbeat files for instants with expired heartbeats without any active instant. - * @param allExistingHeartbeatInstants - * @param metaClient - * @param basePath + * Deletes the heartbeat file for the specified instant. + * @param fs Hadoop FileSystem instance + * @param basePath Hoodie table base path + * @param instantTime Commit instant time + * @param config HoodieWriteConfig instance + * @return Boolean indicating whether heartbeat file was deleted or not */ - public static void cleanExpiredHeartbeats(List allExistingHeartbeatInstants, - HoodieTableMetaClient metaClient, String basePath) { - Set nonExpiredHeartbeatInstants = metaClient.getActiveTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - allExistingHeartbeatInstants.stream().forEach(instant -> { - if (!nonExpiredHeartbeatInstants.contains(instant)) { - deleteHeartbeatFile(metaClient.getFs(), basePath, instant); - } - }); + public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String instantTime, HoodieWriteConfig config) { + if (config.getFailedWritesCleanPolicy().isLazy()) { + return deleteHeartbeatFile(fs, basePath, instantTime); + } + + return false; } /** diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index c70a2cf6a5832..cef6641f1743e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -18,7 +18,9 @@ package org.apache.hudi.client; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; @@ -136,6 +138,62 @@ public void testMultiWriterWithAsyncTableServicesWithConflictMOR() throws Except testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.MERGE_ON_READ); } + @ParameterizedTest + @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) + public void testMultiWriterWithInsertsToDistinctPartitions(HoodieTableType tableType) throws Exception { + if (tableType == HoodieTableType.MERGE_ON_READ) { + setUpMORTestTable(); + } + + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000"); + + HoodieWriteConfig cfg = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withInlineCompaction(false) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withMaxNumDeltaCommitsBeforeCompaction(2) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(FileSystemBasedLockProviderTestClass.class) + .build()) + .withAutoCommit(false) + .withProperties(properties) + .build(); + + // Create the first commit + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + createCommitWithInsertsForPartition(cfg, client, "000", "001", 100, "2016/03/01"); + + int numConcurrentWriters = 5; + ExecutorService executors = Executors.newFixedThreadPool(numConcurrentWriters); + + List> futures = new ArrayList<>(numConcurrentWriters); + for (int loop = 0; loop < numConcurrentWriters; loop++) { + String newCommitTime = "00" + (loop + 2); + String partition = "2016/03/0" + (loop + 2); + futures.add(executors.submit(() -> { + try { + SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg); + createCommitWithInsertsForPartition(cfg, writeClient, "001", newCommitTime, 100, partition); + } catch (Exception e) { + throw new RuntimeException(e); + } + })); + } + + futures.forEach(f -> { + try { + f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { // create inserts X 1 if (tableType == HoodieTableType.MERGE_ON_READ) { @@ -294,6 +352,14 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) } } + private void createCommitWithInsertsForPartition(HoodieWriteConfig cfg, SparkRDDWriteClient client, + String prevCommitTime, String newCommitTime, int numRecords, + String partition) throws Exception { + JavaRDD result = insertBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::insert, + false, false, numRecords, numRecords, 1, Option.of(partition)); + assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); + } + private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommitTime, String newCommitTime, int numRecords) throws Exception { // Finish first base commmit diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 9bca10892e28c..dda396a135676 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -303,7 +303,7 @@ public void testMORTable() throws Exception { // Insert with original schema is allowed now insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert, - false, false, 0, 0, 0); + false, false, 0, 0, 0, Option.empty()); checkLatestDeltaCommit("009"); checkReadRecords("000", 3 * numRecords); } @@ -438,7 +438,7 @@ public void testCopyOnWriteTable() throws Exception { // Insert with original schema is allowed now insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, 2 * numRecords, 1); + false, true, numRecords, 2 * numRecords, 1, Option.empty()); checkReadRecords("000", 2 * numRecords); // Update with original schema is allowed now diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 0a010dde5b63a..05d4b5c7df557 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -244,6 +244,18 @@ private Function2, String, Integer> wrapRecordsGenFunctionFor }; } + private Function3, String, Integer, String> wrapRecordsGenFunctionForPreppedCalls( + final HoodieWriteConfig writeConfig, final Function3, String, Integer, String> recordGenFunction) { + return (commit, numRecords, partition) -> { + final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig); + List records = recordGenFunction.apply(commit, numRecords, partition); + final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); + JavaRDD taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table); + return taggedRecords.collect(); + }; + } + /** * Helper to generate delete keys generation function for testing Prepped version of API. Prepped APIs expect the keys * to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is @@ -285,6 +297,15 @@ public Function2, String, Integer> generateWrapRecordsFn(bool } } + public Function3, String, Integer, String> generateWrapRecordsForPartitionFn(boolean isPreppedAPI, + HoodieWriteConfig writeConfig, Function3, String, Integer, String> wrapped) { + if (isPreppedAPI) { + return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped); + } else { + return wrapped; + } + } + /** * Generate wrapper for delete key generation function for testing Prepped APIs. * @@ -355,12 +376,22 @@ public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, Spar public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { - final Function2, String, Integer> recordGenFunction = - generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, Option partition) throws Exception { - return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + if (partition.isPresent()) { + final Function3, String, Integer, String> recordGenFunction = + generateWrapRecordsForPartitionFn(isPreppedAPI, writeConfig, dataGen::generateInsertsForPartition); + + return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, + partition.get()); + } else { + final Function2, String, Integer> recordGenFunction = + generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); + + return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + } } public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, @@ -453,6 +484,16 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); } + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function3, String, Integer, String> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + boolean doCommit, String partition) throws Exception { + return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition); + } + /** * Helper to insert/upsert batch of records and do regular assertions on the state after successful completion. * @@ -478,10 +519,35 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws Exception { + List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); + return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, + numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, + expTotalCommits, doCommit, filterForCommitTimeWithAssert); + } + + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function3, String, Integer, String> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, + boolean filterForCommitTimeWithAssert, + String partition) throws Exception { + + List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit, partition); + return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, + numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, + expTotalCommits, doCommit, filterForCommitTimeWithAssert); + } + + private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, + int numRecordsInThisCommit, List records, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws IOException { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); - List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); JavaRDD writeRecords = jsc.parallelize(records, 1); JavaRDD result = writeFn.apply(client, writeRecords, newCommitTime); From 7a00f867aef79f2c0f411f0f9ff22572fb61093e Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Fri, 19 Nov 2021 14:30:17 +0800 Subject: [PATCH 027/217] [HUDI-2791] Allows duplicate files for metadata commit (#4033) --- .../org/apache/hudi/metadata/HoodieTableMetadataUtil.java | 6 ++++-- .../apache/hudi/sink/StreamWriteOperatorCoordinator.java | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index b028056bb70d9..9078bd08db4a7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -102,8 +102,10 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : partition.length() + 1; String filename = pathWithPartition.substring(offset); - ValidationUtils.checkState(!newFiles.containsKey(filename), "Duplicate files in HoodieCommitMetadata"); - newFiles.put(filename, hoodieWriteStat.getTotalWriteBytes()); + long totalWriteBytes = newFiles.containsKey(filename) + ? newFiles.get(filename) + hoodieWriteStat.getTotalWriteBytes() + : hoodieWriteStat.getTotalWriteBytes(); + newFiles.put(filename, totalWriteBytes); }); // New files added to a partition HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 08a04e3f84bef..c1b56b95ff427 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -333,7 +333,6 @@ private void startInstant() { this.instant = instant; this.writeClient.startCommitWithTime(instant, tableState.commitAction); this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant); - this.writeClient.upgradeDowngrade(this.instant); LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant, this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE)); } @@ -360,6 +359,8 @@ private void initInstant(String instant) { } // starts a new instant startInstant(); + // upgrade downgrade + this.writeClient.upgradeDowngrade(this.instant); }, "initialize instant %s", instant); } From bf008762dfa3215823d47184e6f4e6d4f7bd03fc Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Fri, 19 Nov 2021 23:39:37 +0800 Subject: [PATCH 028/217] [HUDI-2798] Fix flink query operation fields (#4041) --- .../org/apache/hudi/io/HoodieMergeHandle.java | 23 ++++--- .../table/action/compact/HoodieCompactor.java | 1 + .../hudi/source/IncrementalInputSplits.java | 67 +++++++++++++------ .../format/mor/MergeOnReadInputFormat.java | 10 +-- .../hudi/table/format/TestInputFormat.java | 17 +++-- 5 files changed, 78 insertions(+), 40 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index b13b561f5dff1..d1d67efff4b96 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -250,11 +250,17 @@ protected void init(String fileId, Iterator> newRecordsItr) { + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); } - private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Option indexedRecord) { + private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { + boolean isDelete = false; if (indexedRecord.isPresent()) { updatedRecordsWritten++; + GenericRecord record = (GenericRecord) indexedRecord.get(); + if (oldRecord != record) { + // the incoming record is chosen + isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); + } } - return writeRecord(hoodieRecord, indexedRecord); + return writeRecord(hoodieRecord, indexedRecord, isDelete); } protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOException { @@ -264,12 +270,16 @@ protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOExceptio if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) { return; } - if (writeRecord(hoodieRecord, insertRecord)) { + if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) { insertRecordsWritten++; } } protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { + return writeRecord(hoodieRecord, indexedRecord, false); + } + + protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete) { Option recordMetadata = hoodieRecord.getData().getMetadata(); if (!partitionPath.equals(hoodieRecord.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " @@ -277,11 +287,8 @@ protected boolean writeRecord(HoodieRecord hoodieRecord, Option compact(HoodieCompactionHandler compactionHandler, .withSpillableMapBasePath(config.getSpillableMapBasePath()) .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) + .withOperationField(config.allowOperationMetadataField()) .build(); if (!scanner.iterator().hasNext()) { scanner.close(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java index 653e182bfad9c..d0fcc854dbe46 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java +++ b/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java @@ -44,6 +44,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; @@ -156,28 +157,52 @@ public Result inputSplits( } String tableName = conf.getString(FlinkOptions.TABLE_NAME); - List activeMetadataList = instants.stream() - .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList()); - List archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName); - if (archivedMetadataList.size() > 0) { - LOG.warn("\n" - + "--------------------------------------------------------------------------------\n" - + "---------- caution: the reader has fall behind too much from the writer,\n" - + "---------- tweak 'read.tasks' option to add parallelism of read tasks.\n" - + "--------------------------------------------------------------------------------"); - } - List metadataList = archivedMetadataList.size() > 0 - // IMPORTANT: the merged metadata list must be in ascending order by instant time - ? mergeList(archivedMetadataList, activeMetadataList) - : activeMetadataList; - - Set writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList); - // apply partition push down - if (this.requiredPartitions != null) { - writePartitions = writePartitions.stream() - .filter(this.requiredPartitions::contains).collect(Collectors.toSet()); + + Set writePartitions; + final FileStatus[] fileStatuses; + + if (instantRange == null) { + // reading from the earliest, scans the partitions and files directly. + FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf); + if (this.requiredPartitions != null) { + // apply partition push down + fileIndex.setPartitionPaths(this.requiredPartitions); + } + writePartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths()); + if (writePartitions.size() == 0) { + LOG.warn("No partitions found for reading in user provided path."); + return Result.EMPTY; + } + fileStatuses = fileIndex.getFilesInPartitions(); + } else { + List activeMetadataList = instants.stream() + .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList()); + List archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName); + if (archivedMetadataList.size() > 0) { + LOG.warn("\n" + + "--------------------------------------------------------------------------------\n" + + "---------- caution: the reader has fall behind too much from the writer,\n" + + "---------- tweak 'read.tasks' option to add parallelism of read tasks.\n" + + "--------------------------------------------------------------------------------"); + } + List metadataList = archivedMetadataList.size() > 0 + // IMPORTANT: the merged metadata list must be in ascending order by instant time + ? mergeList(archivedMetadataList, activeMetadataList) + : activeMetadataList; + + writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList); + // apply partition push down + if (this.requiredPartitions != null) { + writePartitions = writePartitions.stream() + .filter(this.requiredPartitions::contains).collect(Collectors.toSet()); + } + if (writePartitions.size() == 0) { + LOG.warn("No partitions found for reading in user provided path."); + return Result.EMPTY; + } + fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType()); } - FileStatus[] fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType()); + if (fileStatuses.length == 0) { LOG.warn("No files found for reading in user provided path."); return Result.EMPTY; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 95c4bd4a5431b..7a72bca0582fd 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -683,13 +683,18 @@ public boolean reachedEnd() throws IOException { // deleted continue; } else { + final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos); + if (!emitDelete && rowKind == RowKind.DELETE) { + // deleted + continue; + } GenericRecord avroRecord = buildAvroRecordBySchema( mergedAvroRecord.get(), requiredSchema, requiredPos, recordBuilder); this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord); - FormatUtils.setRowKind(this.currentRecord, mergedAvroRecord.get(), this.operationPos); + this.currentRecord.setRowKind(rowKind); return false; } } @@ -746,9 +751,6 @@ private Option mergeRowWithLog( RowData curRow, String curKey) throws IOException { final HoodieRecord record = scanner.getRecords().get(curKey); - if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) { - return Option.empty(); - } GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema); } diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java b/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java index f4da947f3bfc2..e6424a1abb751 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java @@ -39,6 +39,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; @@ -221,8 +222,9 @@ void testReadBaseAndLogFilesWithDeletes() throws Exception { assertThat(actual2, is(expected2)); } - @Test - void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testReadBaseAndLogFilesWithDisorderUpdateDelete(boolean compact) throws Exception { Map options = new HashMap<>(); options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true"); beforeEach(HoodieTableType.MERGE_ON_READ, options); @@ -233,7 +235,7 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { TestData.writeData(TestData.DATA_SET_SINGLE_INSERT, conf); // write another commit using logs and read again. - conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); + conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, compact); TestData.writeData(TestData.DATA_SET_DISORDER_UPDATE_DELETE, conf); InputFormat inputFormat = this.tableSource.getInputFormat(); @@ -242,9 +244,11 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { // when isEmitDelete is false. List result1 = readData(inputFormat); + final String rowKind = compact ? "I" : "U"; + final String expected = "[+" + rowKind + "[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; + final String actual1 = TestData.rowDataToString(result1); - final String expected1 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; - assertThat(actual1, is(expected1)); + assertThat(actual1, is(expected)); // refresh the input format and set isEmitDelete to true. this.tableSource.reset(); @@ -254,8 +258,7 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { List result2 = readData(inputFormat); final String actual2 = TestData.rowDataToString(result2); - final String expected2 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; - assertThat(actual2, is(expected2)); + assertThat(actual2, is(expected)); } @Test From eba354e922e5bf28f961e5ac7905b110315d71cf Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Fri, 19 Nov 2021 21:39:08 +0530 Subject: [PATCH 029/217] =?UTF-8?q?[HUDI-2731]=20Make=20clustering=20work?= =?UTF-8?q?=20regardless=20of=20whether=20there=20are=20base=E2=80=A6=20(#?= =?UTF-8?q?3970)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../MultipleSparkJobExecutionStrategy.java | 5 +- ...HoodieSparkMergeOnReadTableClustering.java | 119 ++++++++++++++---- ...eSparkMergeOnReadTableIncrementalRead.java | 8 +- ...arkMergeOnReadTableInsertUpdateDelete.java | 3 +- .../SparkClientFunctionalTestHarness.java | 6 +- .../table/log/HoodieFileSliceReader.java | 46 +++++-- 6 files changed, 146 insertions(+), 41 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 044b77362010e..69e21bec9dde1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; @@ -191,7 +192,6 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(table.getMetaClient().getFs()) .withBasePath(table.getMetaClient().getBasePath()) @@ -205,6 +205,9 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js .withSpillableMapBasePath(config.getSpillableMapBasePath()) .build(); + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + ? Option.empty() + : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); recordIterators.add(getFileSliceReader(baseFileReader, scanner, readerSchema, tableConfig.getPayloadClass(), diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java index 8f7a500d13f1c..f0ece849c058b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -45,6 +45,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.List; @@ -52,6 +53,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Tag("functional") class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTestHarness { @@ -111,7 +113,8 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 400); - insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + Stream dataFiles = insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); /* * Write 2 (more inserts to create new files) @@ -119,7 +122,8 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese // we already set small file size to small number to force inserts to go into new file. newCommitTime = "002"; client.startCommitWithTime(newCommitTime); - insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + dataFiles = insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); if (doUpdates) { /* @@ -144,28 +148,101 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); // Do the clustering and validate - client.cluster(clusteringCommitTime, true); + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); + } + } - metaClient = HoodieTableMetaClient.reload(metaClient); - final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); - clusteredTable.getHoodieView().sync(); - Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) - .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); - // verify there should be only one base file per partition after clustering. - assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); - - HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); - assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), - "Expecting a single commit."); - assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); - assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); - if (cfg.populateMetaFields()) { - assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), - "Must contain 200 records"); - } else { - assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty())); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testClusteringWithNoBaseFiles(boolean doUpdates) throws Exception { + // set low compaction small File Size to generate more file groups. + HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder() + .forTable("test-trip-table") + .withPath(basePath()) + .withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withDeleteParallelism(2) + .withAutoCommit(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(10L) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(1024 * 1024 * 1024) + .parquetMaxFileSize(1024 * 1024 * 1024).build()) + .withEmbeddedTimelineServerEnabled(true) + .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withEnableBackupForRemoteFileSystemView(false).build()) + // set index type to INMEMORY so that log files can be indexed, and it is safe to send + // inserts straight to the log to produce file slices with only log files and no data files + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0) + .withInlineClustering(true) + .withInlineClusteringNumCommits(1).build()) + .withRollbackUsingMarkers(false); + HoodieWriteConfig cfg = cfgBuilder.build(); + HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, cfg.getProps()); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { + // test 2 inserts + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 400); + Stream dataFiles = insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files"); + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + dataFiles = insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files"); + // run updates + if (doUpdates) { + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 100); + updateRecords(metaClient, records, client, cfg, newCommitTime); } + + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + hoodieTable.getHoodieView().sync(); + FileStatus[] allBaseFiles = listAllBaseFilesInPath(hoodieTable); + // expect 0 base files for each partition + assertEquals(0, allBaseFiles.length); + + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + // verify log files are included in clustering plan for each partition. + assertEquals(dataGen.getPartitionPaths().length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); + + // do the clustering and validate + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); } } + private void doClusteringAndValidate(SparkRDDWriteClient client, + String clusteringCommitTime, + HoodieTableMetaClient metaClient, + HoodieWriteConfig cfg, + HoodieTestDataGenerator dataGen) { + client.cluster(clusteringCommitTime, true); + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); + clusteredTable.getHoodieView().sync(); + Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) + .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); + assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); + assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), + "Expecting a single commit."); + assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); + if (cfg.populateMetaFields()) { + assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), + "Must contain 200 records"); + } else { + assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty())); + } + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java index fd2f63a26c638..db55d36876a02 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -57,6 +58,7 @@ import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -94,7 +96,8 @@ public void testIncrementalReadsWithCompaction() throws Exception { client.startCommitWithTime(commitTime1); List records001 = dataGen.generateInserts(commitTime1, 200); - insertRecords(metaClient, records001, client, cfg, commitTime1); + Stream dataFiles = insertRecords(metaClient, records001, client, cfg, commitTime1); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); // verify only one base file shows up with commit time 001 FileStatus[] snapshotROFiles = getROSnapshotFiles(partitionPath); @@ -142,7 +145,8 @@ public void testIncrementalReadsWithCompaction() throws Exception { String insertsTime = "006"; List records006 = dataGen.generateInserts(insertsTime, 200); client.startCommitWithTime(insertsTime); - insertRecords(metaClient, records006, client, cfg, insertsTime); + dataFiles = insertRecords(metaClient, records006, client, cfg, insertsTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); // verify new write shows up in snapshot mode even though there is pending compaction snapshotROFiles = getROSnapshotFiles(partitionPath); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java index fb44c14f59ad2..254d75779f99e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java @@ -95,7 +95,8 @@ public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean popul client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - insertRecords(metaClient, records, client, cfg, newCommitTime); + Stream dataFiles = insertRecords(metaClient, records, client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); /* * Write 2 (updates) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 79fbdcaad93af..40ef54b14cb6c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -203,7 +203,8 @@ protected JavaRDD updateLocation( index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table)); } - protected void insertRecords(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { + protected Stream insertRecords(HoodieTableMetaClient metaClient, List records, + SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient); JavaRDD writeRecords = jsc().parallelize(records, 1); @@ -228,8 +229,7 @@ protected void insertRecords(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java index f7a7acfa9022e..d8d00acb1ae02 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -23,33 +23,53 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import java.io.IOException; import java.util.Iterator; +import java.util.stream.StreamSupport; /** * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice. */ public class HoodieFileSliceReader implements Iterator> { - private Iterator> recordsIterator; + private final Iterator> recordsIterator; - public static HoodieFileSliceReader getFileSliceReader( - HoodieFileReader baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, - String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { - Iterator baseIterator = baseFileReader.getRecordIterator(schema); - while (baseIterator.hasNext()) { - GenericRecord record = (GenericRecord) baseIterator.next(); - HoodieRecord hoodieRecord = simpleKeyGenFieldsOpt.isPresent() - ? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField()) - : SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, scanner.isWithOperationField()); - scanner.processNextRecord(hoodieRecord); + public static HoodieFileSliceReader getFileSliceReader( + Option baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, + String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { + if (baseFileReader.isPresent()) { + Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); + while (baseIterator.hasNext()) { + GenericRecord record = (GenericRecord) baseIterator.next(); + HoodieRecord hoodieRecord = transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); + scanner.processNextRecord(hoodieRecord); + } + return new HoodieFileSliceReader(scanner.iterator()); + } else { + Iterable> iterable = () -> scanner.iterator(); + return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false) + .map(e -> { + try { + GenericRecord record = (GenericRecord) e.getData().getInsertValue(schema).get(); + return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); + } catch (IOException io) { + throw new HoodieIOException("Error while creating reader for file slice with no base file.", io); + } + }).iterator()); } - return new HoodieFileSliceReader(scanner.iterator()); + } + + private static HoodieRecord transform( + GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass, + String preCombineField, Option> simpleKeyGenFieldsOpt) { + return simpleKeyGenFieldsOpt.isPresent() + ? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField()) + : SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, scanner.isWithOperationField()); } private HoodieFileSliceReader(Iterator> recordsItr) { From 459b34240bc041be0fba7312f096980da024a5d9 Mon Sep 17 00:00:00 2001 From: Manoj Govindassamy Date: Fri, 19 Nov 2021 15:11:29 -0800 Subject: [PATCH 030/217] [HUDI-2593] Virtual keys support for metadata table (#3968) - Metadata table today has virtual keys disabled, thereby populating the metafields for each record written out and increasing the overall storage space used. Hereby adding virtual keys support for metadata table so that metafields are disabled for metadata table records. - Adding a custom KeyGenerator for Metadata table so as to not rely on the default Base/SimpleKeyGenerators which currently look for record key and partition field set in the table config. - AbstractHoodieLogRecordReader's version of processing next data block and createHoodieRecord() will be a generic version and making the derived class HoodieMetadataMergedLogRecordReader take care of the special creation of records from explictly passed in partition names. --- .../apache/hudi/io/HoodieAppendHandle.java | 7 +- .../HoodieBackedTableMetadataWriter.java | 25 ++++- .../HoodieTableMetadataKeyGenerator.java | 52 ++++++++++ .../hudi/table/HoodieTimelineArchiveLog.java | 3 +- .../table/action/compact/HoodieCompactor.java | 1 + .../MultipleSparkJobExecutionStrategy.java | 1 + .../hudi/keygen/SimpleKeyGenerator.java | 6 +- .../functional/TestHoodieBackedMetadata.java | 10 +- .../TestHoodieBackedTableMetadata.java | 19 ++++ .../functional/TestHoodieMetadataBase.java | 4 +- .../testutils/HoodieClientTestHarness.java | 17 +++- .../common/config/HoodieMetadataConfig.java | 15 +++ .../hudi/common/table/HoodieTableConfig.java | 3 +- .../log/AbstractHoodieLogRecordReader.java | 93 ++++++++++++++---- .../table/log/HoodieFileSliceReader.java | 9 +- .../common/table/log/HoodieLogFileReader.java | 14 ++- .../table/log/HoodieLogFormatReader.java | 19 ++-- .../log/HoodieMergedLogRecordScanner.java | 21 +++- .../table/log/block/HoodieAvroDataBlock.java | 22 +++-- .../table/log/block/HoodieDataBlock.java | 47 ++++++--- .../table/log/block/HoodieHFileDataBlock.java | 25 +++-- .../hudi/common/util/SpillableMapUtils.java | 34 +++++-- .../metadata/HoodieBackedTableMetadata.java | 31 ++++-- .../HoodieMetadataMergedLogRecordReader.java | 46 ++++++++- .../hudi/metadata/HoodieMetadataPayload.java | 18 ++-- .../functional/TestHoodieLogFormat.java | 4 +- ...sModeWithMultipleWriters.COPY_ON_WRITE.zip | Bin 2494616 -> 2592485 bytes ...sModeWithMultipleWriters.MERGE_ON_READ.zip | Bin 2910151 -> 3015940 bytes 28 files changed, 423 insertions(+), 123 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index a33383a05c02d..03e3d991141c1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -360,7 +360,12 @@ protected void appendDataAndDeleteBlocks(Map header) header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString()); List blocks = new ArrayList<>(2); if (recordList.size() > 0) { - blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); + if (config.populateMetaFields()) { + blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); + } else { + final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header, keyField)); + } } if (keysToDelete.size() > 0) { blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 6cbc595c55974..6886573c16deb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -41,6 +41,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -76,6 +77,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; @@ -91,6 +93,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); + // Virtual keys support for metadata table. This Field is + // from the metadata payload schema. + private static final String RECORD_KEY_FIELD = HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY; + protected HoodieWriteConfig metadataWriteConfig; protected HoodieWriteConfig dataWriteConfig; protected String tableName; @@ -202,7 +208,15 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withDeleteParallelism(parallelism) .withRollbackParallelism(parallelism) .withFinalizeWriteParallelism(parallelism) - .withAllowMultiWriteOnSameInstant(true); + .withAllowMultiWriteOnSameInstant(true) + .withKeyGenerator(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) + .withPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()); + + // RecordKey properties are needed for the metadata table records + final Properties properties = new Properties(); + properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), RECORD_KEY_FIELD); + properties.put("hoodie.datasource.write.recordkey.field", RECORD_KEY_FIELD); + builder.withProperties(properties); if (writeConfig.isMetricsOn()) { builder.withMetricsConfig(HoodieMetricsConfig.newBuilder() @@ -395,9 +409,12 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(tableName) .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) - .setPayloadClassName(HoodieMetadataPayload.class.getName()) - .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) - .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); + .setPayloadClassName(HoodieMetadataPayload.class.getName()) + .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) + .setRecordKeyFields(RECORD_KEY_FIELD) + .setPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) + .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) + .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java new file mode 100644 index 0000000000000..e9d7aec8c8e2a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.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.hudi.metadata; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenUtils; + +/** + * Custom key generator for the Hoodie table metadata. The metadata table record payload + * has an internal schema with a known key field HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY. + * With or without the virtual keys, getting the key from the metadata table record is always + * via the above field and there is no real need for a key generator. But, when a write + * client is instantiated for the metadata table, when virtual keys are enabled, and when + * key generator class is not configured, the default SimpleKeyGenerator will be used. + * To avoid using any other key generators for the metadata table which rely on certain + * config properties, we need this custom key generator exclusively for the metadata table. + */ +public class HoodieTableMetadataKeyGenerator extends BaseKeyGenerator { + + public HoodieTableMetadataKeyGenerator(TypedProperties config) { + super(config); + } + + @Override + public String getRecordKey(GenericRecord record) { + return KeyGenUtils.getRecordKey(record, HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY); + } + + @Override + public String getPartitionPath(GenericRecord record) { + return ""; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index bff91c3e56c2b..21a4fb58932b8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -346,7 +346,8 @@ private void writeToFile(Schema wrapperSchema, List records) thro if (records.size() > 0) { Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); - HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); + final String keyField = table.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header, keyField); writer.appendBlock(block); records.clear(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index 09a4e2eb588ea..73e1413d9dde0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -180,6 +180,7 @@ public List compact(HoodieCompactionHandler compactionHandler, .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) .withOperationField(config.allowOperationMetadataField()) + .withPartition(operation.getPartitionPath()) .build(); if (!scanner.iterator().hasNext()) { scanner.close(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 69e21bec9dde1..92dda123fed47 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -203,6 +203,7 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js .withReverseReader(config.getCompactionReverseLogReadEnabled()) .withBufferSize(config.getMaxDFSStreamBufferSize()) .withSpillableMapBasePath(config.getSpillableMapBasePath()) + .withPartition(clusteringOp.getPartitionPath()) .build(); Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index 60765b05da74f..b84a8abdcc796 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -47,9 +47,9 @@ public SimpleKeyGenerator(TypedProperties props) { SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) { super(props); this.recordKeyFields = recordKeyField == null - ? Collections.emptyList() - : Collections.singletonList(recordKeyField); - this.partitionPathFields = Collections.singletonList(partitionPathField); + ? Collections.emptyList() : Collections.singletonList(recordKeyField); + this.partitionPathFields = partitionPathField == null + ? Collections.emptyList() : Collections.singletonList(partitionPathField); simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 59e12a1515ad5..005d031cb9df2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -88,6 +88,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.nio.file.Files; @@ -358,8 +359,9 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { * Test that manual rollbacks work correctly and enough timeline history is maintained on the metadata table * timeline. */ - @Test - public void testManualRollbacks() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testManualRollbacks(final boolean populateMateFields) throws Exception { HoodieTableType tableType = COPY_ON_WRITE; init(tableType, false); // Setting to archive more aggressively on the Metadata Table than the Dataset @@ -369,7 +371,9 @@ public void testManualRollbacks() throws Exception { writeConfig = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) .archiveCommitsWith(minArchiveCommitsMetadata, minArchiveCommitsMetadata + 1).retainCommits(1) - .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction) + .withPopulateMetaFields(populateMateFields) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) .build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 5242e9f33766d..cac1e1bbe81e2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -29,6 +30,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.io.IOException; import java.util.ArrayList; @@ -90,4 +93,20 @@ private void verifyBaseMetadataTable() throws IOException { }); } + /** + * Verify if the Metadata table is constructed with table properties including + * the right key generator class name. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throws Exception { + init(tableType); + + HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, + writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + + assertEquals(HoodieTableMetadataKeyGenerator.class.getCanonicalName(), + tableMetadata.getMetadataMetaClient().getTableConfig().getKeyGeneratorClassName()); + } + } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index a91099976700e..5617058bb8af8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -288,7 +288,9 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(useFileListingMetadata) .enableFullScan(enableFullScan) - .enableMetrics(enableMetrics).build()) + .enableMetrics(enableMetrics) + .withPopulateMetaFields(false) + .build()) .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) .withExecutorMetrics(true).build()) .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 9ed98b15cb7ab..1df77b8b17f6d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -283,13 +283,26 @@ protected Properties getPropertiesForKeyGen() { return properties; } - protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) { + protected Properties getPropertiesForMetadataTable() { + Properties properties = new Properties(); + properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); + properties.put("hoodie.datasource.write.recordkey.field", "key"); + properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "key"); + return properties; + } + + protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields, + boolean isMetadataTable) { if (!populateMetaFields) { - configBuilder.withProperties(getPropertiesForKeyGen()) + configBuilder.withProperties((isMetadataTable ? getPropertiesForMetadataTable() : getPropertiesForKeyGen())) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.SIMPLE).build()); } } + protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) { + addConfigsForPopulateMetaFields(configBuilder, populateMetaFields, false); + } + /** * Cleanups hoodie clients. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index ea99014eacb50..0aa0593693e7c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -132,6 +132,12 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.10.0") .withDocumentation("Enable full scanning of log files while reading log records. If disabled, hudi does look up of only interested entries."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty + .key(METADATA_PREFIX + ".populate.meta.fields") + .defaultValue("false") + .sinceVersion("0.10.0") + .withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated."); + private HoodieMetadataConfig() { super(); } @@ -164,6 +170,10 @@ public boolean enableFullScan() { return getBoolean(ENABLE_FULL_SCAN_LOG_FILES); } + public boolean populateMetaFields() { + return getBooleanOrDefault(HoodieMetadataConfig.POPULATE_META_FIELDS); + } + public static class Builder { private EngineType engineType = EngineType.SPARK; @@ -206,6 +216,11 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } + public Builder withPopulateMetaFields(boolean populateMetaFields) { + metadataConfig.setValue(POPULATE_META_FIELDS, Boolean.toString(populateMetaFields)); + return this; + } + public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { metadataConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); metadataConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index dc57fd1c6ff8b..df196fe235c6f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -364,7 +365,7 @@ public boolean populateMetaFields() { * @returns the record key field prop. */ public String getRecordKeyFieldProp() { - return getString(RECORDKEY_FIELDS); + return getStringOrDefault(RECORDKEY_FIELDS, HoodieRecord.RECORD_KEY_METADATA_FIELD); } public String getKeyGeneratorClassName() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index e2e76ad7d6503..d495badeca4eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -120,28 +121,32 @@ public abstract class AbstractHoodieLogRecordReader { private int totalScannedLogFiles; // Progress private float progress = 0.0f; + // Partition name + private Option partitionName; + // Populate meta fields for the records + private boolean populateMetaFields = true; - protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, Option instantRange, boolean withOperationField) { - this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - true); + int bufferSize, Option instantRange, + boolean withOperationField) { + this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, + instantRange, withOperationField, true, Option.empty()); } - protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, Option instantRange, boolean withOperationField, - boolean enableFullScan) { + protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, Option instantRange, + boolean withOperationField, boolean enableFullScan, + Option partitionName) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build(); // load class from the payload fully qualified class name - this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); - this.preCombineField = this.hoodieTableMetaClient.getTableConfig().getPreCombineField(); HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); - if (!tableConfig.populateMetaFields()) { - this.simpleKeyGenFields = Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp())); - } + this.payloadClassFQN = tableConfig.getPayloadClass(); + this.preCombineField = tableConfig.getPreCombineField(); this.totalLogFiles.addAndGet(logFilePaths.size()); this.logFilePaths = logFilePaths; this.reverseReader = reverseReader; @@ -151,6 +156,22 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List> keys) { HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants(); HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights(); try { - // iterate over the paths + + // Get the key field based on populate meta fields config + // and the table type + final String keyField = getKeyField(); + + // Iterate over the paths logFormatReaderWrapper = new HoodieLogFormatReader(fs, logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()), - readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan); + readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField); Set scannedLogFiles = new HashSet<>(); while (logFormatReaderWrapper.hasNext()) { HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); @@ -339,15 +365,34 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option> ke } totalLogRecords.addAndGet(recs.size()); for (IndexedRecord rec : recs) { - processNextRecord(createHoodieRecord(rec)); + processNextRecord(createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, + this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); } } - protected HoodieRecord createHoodieRecord(IndexedRecord rec) { - if (!simpleKeyGenFields.isPresent()) { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN, this.preCombineField, this.withOperationField); + /** + * Create @{@link HoodieRecord} from the @{@link IndexedRecord}. + * + * @param rec - IndexedRecord to create the HoodieRecord from + * @param hoodieTableConfig - Table config + * @param payloadClassFQN - Payload class fully qualified name + * @param preCombineField - PreCombine field + * @param withOperationField - Whether operation field is enabled + * @param simpleKeyGenFields - Key generator fields when populate meta fields is tuened off + * @param partitionName - Partition name + * @return HoodieRecord created from the IndexedRecord + */ + protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + final String payloadClassFQN, final String preCombineField, + final boolean withOperationField, + final Option> simpleKeyGenFields, + final Option partitionName) { + if (this.populateMetaFields) { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, withOperationField); } else { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN, this.preCombineField, this.simpleKeyGenFields.get(), this.withOperationField); + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); } } @@ -418,6 +463,10 @@ protected String getPayloadClassFQN() { return payloadClassFQN; } + protected Option getPartitionName() { + return partitionName; + } + public long getTotalRollbacks() { return totalRollbacks.get(); } @@ -451,6 +500,10 @@ public abstract static class Builder { public abstract Builder withBufferSize(int bufferSize); + public Builder withPartition(String partitionName) { + throw new UnsupportedOperationException(); + } + public Builder withInstantRange(Option instantRange) { throw new UnsupportedOperationException(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java index d8d00acb1ae02..a786e8305bc27 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -46,7 +46,8 @@ public static HoodieFileSliceReader getFileSliceReader( Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); while (baseIterator.hasNext()) { GenericRecord record = (GenericRecord) baseIterator.next(); - HoodieRecord hoodieRecord = transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); + HoodieRecord hoodieRecord = transform( + record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); scanner.processNextRecord(hoodieRecord); } return new HoodieFileSliceReader(scanner.iterator()); @@ -68,8 +69,10 @@ private static HoodieRecord transform( GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass, String preCombineField, Option> simpleKeyGenFieldsOpt) { return simpleKeyGenFieldsOpt.isPresent() - ? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField()) - : SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, scanner.isWithOperationField()); + ? SpillableMapUtils.convertToHoodieRecordPayload(record, + payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField(), Option.empty()) + : SpillableMapUtils.convertToHoodieRecordPayload(record, + payloadClass, preCombineField, scanner.isWithOperationField(), scanner.getPartitionName()); } private HoodieFileSliceReader(Iterator> recordsItr) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index cdf3065587d13..9a3913fa118d0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.SchemeAwareFSDataInputStream; import org.apache.hudi.common.fs.TimedFSDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieCorruptBlock; @@ -66,6 +67,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private final HoodieLogFile logFile; private final byte[] magicBuffer = new byte[6]; private final Schema readerSchema; + private final String keyField; private boolean readBlockLazily; private long reverseLogFilePosition; private long lastReverseLogFilePosition; @@ -76,11 +78,13 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readBlockLazily, boolean reverseReader) throws IOException { - this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false); + this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false, + HoodieRecord.RECORD_KEY_METADATA_FIELD); } public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readBlockLazily, boolean reverseReader, boolean enableInlineReading) throws IOException { + boolean readBlockLazily, boolean reverseReader, boolean enableInlineReading, + String keyField) throws IOException { FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize); this.logFile = logFile; this.inputStream = getFSDataInputStream(fsDataInputStream, fs, bufferSize); @@ -88,6 +92,7 @@ public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSc this.readBlockLazily = readBlockLazily; this.reverseReader = reverseReader; this.enableInlineReading = enableInlineReading; + this.keyField = keyField; if (this.reverseReader) { this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen(); } @@ -251,11 +256,12 @@ private HoodieLogBlock readBlock() throws IOException { return HoodieAvroDataBlock.getBlock(content, readerSchema); } else { return new HoodieAvroDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer); + contentPosition, contentLength, blockEndPos, readerSchema, header, footer, keyField); } case HFILE_DATA_BLOCK: return new HoodieHFileDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer, enableInlineReading); + contentPosition, contentLength, blockEndPos, readerSchema, + header, footer, enableInlineReading, keyField); case DELETE_BLOCK: return HoodieDeleteBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, contentPosition, contentLength, blockEndPos, header, footer); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java index febdbf8068292..2db5437697094 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java @@ -44,18 +44,15 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final Schema readerSchema; private final boolean readBlocksLazily; private final boolean reverseLogReader; - private final boolean enableInLineReading; + private final String recordKeyField; + private final boolean enableInlineReading; private int bufferSize; private static final Logger LOG = LogManager.getLogger(HoodieLogFormatReader.class); HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily, - boolean reverseLogReader, int bufferSize) throws IOException { - this(fs, logFiles, readerSchema, readBlocksLazily, reverseLogReader, bufferSize, false); - } - - HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily, - boolean reverseLogReader, int bufferSize, boolean enableInlineReading) throws IOException { + boolean reverseLogReader, int bufferSize, boolean enableInlineReading, + String recordKeyField) throws IOException { this.logFiles = logFiles; this.fs = fs; this.readerSchema = readerSchema; @@ -63,10 +60,12 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { this.reverseLogReader = reverseLogReader; this.bufferSize = bufferSize; this.prevReadersInOpenState = new ArrayList<>(); - this.enableInLineReading = enableInlineReading; + this.recordKeyField = recordKeyField; + this.enableInlineReading = enableInlineReading; if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, enableInlineReading); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, + enableInlineReading, recordKeyField); } } @@ -107,7 +106,7 @@ public boolean hasNext() { } this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, - this.enableInLineReading); + enableInlineReading, recordKeyField); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index a8d97ac1b5f18..2e47e695d3144 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -76,10 +76,13 @@ protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List instantRange, boolean autoScan, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, - boolean withOperationField, boolean enableFullScan) { - super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - enableFullScan); + ExternalSpillableMap.DiskMapType diskMapType, + boolean isBitCaskDiskMapCompressionEnabled, + boolean withOperationField, boolean enableFullScan, + Option partitionName) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, + instantRange, withOperationField, + enableFullScan, partitionName); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), @@ -187,6 +190,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { private boolean autoScan = true; // operation field default false private boolean withOperationField = false; + protected String partitionName; @Override public Builder withFileSystem(FileSystem fs) { @@ -272,12 +276,19 @@ public Builder withOperationField(boolean withOperationField) { return this; } + @Override + public Builder withPartition(String partitionName) { + this.partitionName = partitionName; + return this; + } + @Override public HoodieMergedLogRecordScanner build() { return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, bufferSize, spillableMapBasePath, instantRange, autoScan, - diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true); + diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true, + Option.ofNullable(partitionName)); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index 31fc352acad09..1d3f5f3b01c56 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.SizeAwareDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; @@ -58,22 +59,27 @@ public class HoodieAvroDataBlock extends HoodieDataBlock { private ThreadLocal decoderCache = new ThreadLocal<>(); public HoodieAvroDataBlock(@Nonnull Map logBlockHeader, - @Nonnull Map logBlockFooter, - @Nonnull Option blockContentLocation, @Nonnull Option content, - FSDataInputStream inputStream, boolean readBlockLazily) { + @Nonnull Map logBlockFooter, + @Nonnull Option blockContentLocation, @Nonnull Option content, + FSDataInputStream inputStream, boolean readBlockLazily) { super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); } public HoodieAvroDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option content, - boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, - Map header, Map footer) { + boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, + Map header, Map footer, String keyField) { super(content, inputStream, readBlockLazily, - Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, - footer); + Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, + footer, keyField); + } + + public HoodieAvroDataBlock(@Nonnull List records, @Nonnull Map header, String keyField) { + super(records, header, new HashMap<>(), keyField); } public HoodieAvroDataBlock(@Nonnull List records, @Nonnull Map header) { - super(records, header, new HashMap<>()); + super(records, header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index 2e4338ef785d0..66c9571487dff 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.log.block; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -29,7 +30,6 @@ import javax.annotation.Nonnull; import java.io.IOException; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,39 +46,62 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { protected List records; protected Schema schema; + protected String keyField; public HoodieDataBlock(@Nonnull Map logBlockHeader, @Nonnull Map logBlockFooter, @Nonnull Option blockContentLocation, @Nonnull Option content, FSDataInputStream inputStream, boolean readBlockLazily) { super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); + this.keyField = HoodieRecord.RECORD_KEY_METADATA_FIELD; } public HoodieDataBlock(@Nonnull List records, @Nonnull Map header, - @Nonnull Map footer) { - super(header, footer, Option.empty(), Option.empty(), null, false); + @Nonnull Map footer, String keyField) { + this(header, footer, Option.empty(), Option.empty(), null, false); this.records = records; this.schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - } - - public HoodieDataBlock(@Nonnull List records, @Nonnull Map header) { - this(records, header, new HashMap<>()); + this.keyField = keyField; } protected HoodieDataBlock(Option content, @Nonnull FSDataInputStream inputStream, boolean readBlockLazily, - Option blockContentLocation, Schema readerSchema, - @Nonnull Map headers, @Nonnull Map footer) { - super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); + Option blockContentLocation, Schema readerSchema, + @Nonnull Map headers, @Nonnull Map footer, String keyField) { + this(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); this.schema = readerSchema; + this.keyField = keyField; } + /** + * Util method to get a data block for the requested type. + * + * @param logDataBlockFormat - Data block type + * @param recordList - List of records that goes in the data block + * @param header - data block header + * @return Data block of the requested type. + */ public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List recordList, Map header) { + return getBlock(logDataBlockFormat, recordList, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); + } + + /** + * Util method to get a data block for the requested type. + * + * @param logDataBlockFormat - Data block type + * @param recordList - List of records that goes in the data block + * @param header - data block header + * @param keyField - FieldId to get the key from the records + * @return Data block of the requested type. + */ + public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List recordList, + Map header, String keyField) { switch (logDataBlockFormat) { case AVRO_DATA_BLOCK: - return new HoodieAvroDataBlock(recordList, header); + return new HoodieAvroDataBlock(recordList, header, keyField); case HFILE_DATA_BLOCK: - return new HoodieHFileDataBlock(recordList, header); + return new HoodieHFileDataBlock(recordList, header, keyField); default: throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index a1e0c129803f7..35ea41dd3f637 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -68,24 +68,23 @@ public class HoodieHFileDataBlock extends HoodieDataBlock { private static int blockSize = 1 * 1024 * 1024; private boolean enableInlineReading = false; - public HoodieHFileDataBlock(@Nonnull Map logBlockHeader, - @Nonnull Map logBlockFooter, - @Nonnull Option blockContentLocation, @Nonnull Option content, - FSDataInputStream inputStream, boolean readBlockLazily) { - super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); - } - public HoodieHFileDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option content, - boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, - Map header, Map footer, boolean enableInlineReading) { + boolean readBlockLazily, long position, long blockSize, long blockEndpos, + Schema readerSchema, Map header, + Map footer, boolean enableInlineReading, String keyField) { super(content, inputStream, readBlockLazily, - Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, - footer); + Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), + readerSchema, header, footer, keyField); this.enableInlineReading = enableInlineReading; } + public HoodieHFileDataBlock(@Nonnull List records, @Nonnull Map header, + String keyField) { + super(records, header, new HashMap<>(), keyField); + } + public HoodieHFileDataBlock(@Nonnull List records, @Nonnull Map header) { - super(records, header, new HashMap<>()); + this(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); } @Override @@ -111,7 +110,7 @@ protected byte[] serializeRecords() throws IOException { boolean useIntegerKey = false; int key = 0; int keySize = 0; - Field keyField = records.get(0).getSchema().getField(HoodieRecord.RECORD_KEY_METADATA_FIELD); + Field keyField = records.get(0).getSchema().getField(this.keyField); if (keyField == null) { // Missing key metadata field so we should use an integer sequence key useIntegerKey = true; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java index 5dd0c5a8bc9c2..934b5b5f616c6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java @@ -115,22 +115,38 @@ public static long computePayloadSize(R value, SizeEstimator valueSizeEst * Utility method to convert bytes to HoodieRecord using schema and payload class. */ public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) { - return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), withOperationField); + return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, Option.empty()); + } + + public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + String preCombineField, + boolean withOperationField, + Option partitionName) { + return convertToHoodieRecordPayload(record, payloadClazz, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, partitionName); } /** * Utility method to convert bytes to HoodieRecord using schema and payload class. */ - public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, - String preCombineField, Pair recordKeyPartitionPathPair, - boolean withOperationField) { - String recKey = rec.get(recordKeyPartitionPathPair.getLeft()).toString(); - String partitionPath = rec.get(recordKeyPartitionPathPair.getRight()).toString(); - Object preCombineVal = getPreCombineVal(rec, preCombineField); + public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + String preCombineField, + Pair recordKeyPartitionPathFieldPair, + boolean withOperationField, + Option partitionName) { + final String recKey = record.get(recordKeyPartitionPathFieldPair.getKey()).toString(); + final String partitionPath = (partitionName.isPresent() ? partitionName.get() : + record.get(recordKeyPartitionPathFieldPair.getRight()).toString()); + + Object preCombineVal = getPreCombineVal(record, preCombineField); HoodieOperation operation = withOperationField - ? HoodieOperation.fromName(getNullableValAsString(rec, HoodieRecord.OPERATION_METADATA_FIELD)) : null; + ? HoodieOperation.fromName(getNullableValAsString(record, HoodieRecord.OPERATION_METADATA_FIELD)) : null; HoodieRecord hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath), - ReflectionUtils.loadPayload(payloadClazz, new Object[] {rec, preCombineVal}, GenericRecord.class, Comparable.class), operation); + ReflectionUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class, + Comparable.class), operation); return (R) hoodieRecord; } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 6cc5533f1c515..766bc68ec1151 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -133,8 +133,8 @@ protected List>>> getRec // local map to assist in merging with base file records Map>> logRecords = readLogRecords(logRecordScanner, keys, timings); - List>>> result = readFromBaseAndMergeWithLogRecords(baseFileReader, - keys, logRecords, timings); + List>>> result = readFromBaseAndMergeWithLogRecords( + baseFileReader, keys, logRecords, timings, partitionName); LOG.info(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", keys.size(), timings)); return result; } catch (IOException ioe) { @@ -175,8 +175,8 @@ private Map>> readLogRecords( } private List>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader, - List keys, Map>> logRecords, - List timings) throws IOException { + List keys, Map>> logRecords, List timings, String partitionName) throws IOException { List>>> result = new ArrayList<>(); // merge with base records HoodieTimer timer = new HoodieTimer().startTimer(); @@ -189,10 +189,7 @@ private List>>> readFrom readTimer.startTimer(); Option baseRecord = baseFileReader.getRecordByKey(key); if (baseRecord.isPresent()) { - hoodieRecord = metadataTableConfig.populateMetaFields() - ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false) - : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), - Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()), false); + hoodieRecord = getRecord(baseRecord, partitionName); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); // merge base file record w/ log record if present if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) { @@ -218,6 +215,18 @@ private List>>> readFrom return result; } + private HoodieRecord getRecord(Option baseRecord, String partitionName) { + ValidationUtils.checkState(baseRecord.isPresent()); + if (metadataTableConfig.populateMetaFields()) { + return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false); + } + return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), + Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()), + false, Option.of(partitionName)); + } + /** * Returns a new pair of readers to the base and log files. */ @@ -241,7 +250,8 @@ private Pair openReadersI baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice - Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice); + Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice, + partitionName); logRecordScanner = logRecordScannerOpenTimePair.getKey(); logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); @@ -293,7 +303,7 @@ private Set getValidInstantTimestamps() { return validInstantTimestamps; } - private Pair getLogRecordScanner(FileSlice slice) { + private Pair getLogRecordScanner(FileSlice slice, String partitionName) { HoodieTimer timer = new HoodieTimer().startTimer(); List logFilePaths = slice.getLogFiles() .sorted(HoodieLogFile.getLogFileComparator()) @@ -323,6 +333,7 @@ private Pair getLogRecordScanner(File .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) .withLogBlockTimestamps(validInstantTimestamps) .enableFullScan(metadataConfig.enableFullScan()) + .withPartition(partitionName) .build(); Long logScannerOpenMs = timer.endTimer(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index 131ca3b91762f..2c9ca39fdd50a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -25,7 +25,11 @@ import java.util.Set; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -49,13 +53,17 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc // Set of all record keys that are to be read in memory private Set mergeKeyFilter; - private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, + private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, String partitionName, + List logFilePaths, + Schema readerSchema, String latestInstantTime, + Long maxMemorySizeInBytes, int bufferSize, String spillableMapBasePath, Set mergeKeyFilter, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, + ExternalSpillableMap.DiskMapType diskMapType, + boolean isBitCaskDiskMapCompressionEnabled, Option instantRange, boolean enableFullScan) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, - spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false, enableFullScan); + spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false, + enableFullScan, Option.of(partitionName)); this.mergeKeyFilter = mergeKeyFilter; if (enableFullScan) { performScan(); @@ -76,6 +84,23 @@ protected void processNextDeletedKey(HoodieKey hoodieKey) { } } + @Override + protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + final String payloadClassFQN, final String preCombineField, + final boolean withOperationField, + final Option> simpleKeyGenFields, + final Option partitionName) { + if (hoodieTableConfig.populateMetaFields()) { + return super.createHoodieRecord(rec, hoodieTableConfig, payloadClassFQN, preCombineField, withOperationField, + simpleKeyGenFields, partitionName); + } + + // When meta fields are not available, create the record using the + // preset key field and the known partition name + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); + } + /** * Returns the builder for {@code HoodieMetadataMergedLogRecordScanner}. */ @@ -107,6 +132,11 @@ public List>>> getRecord return metadataRecords; } + @Override + protected String getKeyField() { + return HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY; + } + /** * Builder used to build {@code HoodieMetadataMergedLogRecordScanner}. */ @@ -161,6 +191,12 @@ public Builder withBufferSize(int bufferSize) { return this; } + @Override + public Builder withPartition(String partitionName) { + this.partitionName = partitionName; + return this; + } + @Override public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) { this.maxMemorySizeInBytes = maxMemorySizeInBytes; @@ -202,7 +238,7 @@ public Builder enableFullScan(boolean enableFullScan) { @Override public HoodieMetadataMergedLogRecordReader build() { - return new HoodieMetadataMergedLogRecordReader(fs, basePath, logFilePaths, readerSchema, + return new HoodieMetadataMergedLogRecordReader(fs, basePath, partitionName, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter, diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, enableFullScan); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 16eef8a5fd85a..0b0d144a6e7e9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -61,6 +61,12 @@ * HoodieMetadataRecord for ease of operations. */ public class HoodieMetadataPayload implements HoodieRecordPayload { + + // HoodieMetadata schema field ids + public static final String SCHEMA_FIELD_ID_KEY = "key"; + public static final String SCHEMA_FIELD_ID_TYPE = "type"; + public static final String SCHEMA_FIELD_ID_METADATA = "filesystemMetadata"; + // Type of the record // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810 private static final int PARTITION_LIST = 1; @@ -78,13 +84,13 @@ public HoodieMetadataPayload(Option record) { if (record.isPresent()) { // This can be simplified using SpecificData.deepcopy once this bug is fixed // https://issues.apache.org/jira/browse/AVRO-1811 - key = record.get().get("key").toString(); - type = (int) record.get().get("type"); - if (record.get().get("filesystemMetadata") != null) { + key = record.get().get(SCHEMA_FIELD_ID_KEY).toString(); + type = (int) record.get().get(SCHEMA_FIELD_ID_TYPE); + if (record.get().get(SCHEMA_FIELD_ID_METADATA) != null) { filesystemMetadata = (Map) record.get().get("filesystemMetadata"); filesystemMetadata.keySet().forEach(k -> { GenericRecord v = filesystemMetadata.get(k); - filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted"))); + filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); }); } } @@ -231,8 +237,8 @@ private Map combineFilesystemMetadata(HoodieMeta @Override public String toString() { final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); - sb.append("key=").append(key).append(", "); - sb.append("type=").append(type).append(", "); + sb.append(SCHEMA_FIELD_ID_KEY + "=").append(key).append(", "); + sb.append(SCHEMA_FIELD_ID_TYPE + "=").append(type).append(", "); sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); sb.append('}'); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index 3368c17c7bd10..5be3b9674573f 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -1672,9 +1672,9 @@ private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List header) { switch (dataBlockType) { case AVRO_DATA_BLOCK: - return new HoodieAvroDataBlock(records, header); + return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); case HFILE_DATA_BLOCK: - return new HoodieHFileDataBlock(records, header); + return new HoodieHFileDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); default: throw new RuntimeException("Unknown data block type " + dataBlockType); } diff --git a/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip b/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip index 48bf278bd6c6f189d3a7fabb7b5a0405237a8a78..42c0301b310feb65d7eeca166a6cc60dbca40e8f 100644 GIT binary patch delta 2329747 zcmeEuWmHvN*EZ4(Qj&sncXvxkhjh0z2ncvUy5rE@sbC=8(%mTtf`CdRE%5CF+_yeH zH}Cs?Kc6uk$5`r*v(`TQ>@}}>&1+t3C+8&J`I3=&heTN(1{N6#@&~QKQSItKH-F(n z5y!?7@M)+cLBSxujyHx3HxCr3JFo|LprBxJfRFu}jQ+=DGRWiya_~XN|{j>y-D@v9j_0L0C$e)cDu+TC%}J68{CzYwe*F1}!w~Y! zwp5&t(_qpGf=p03L2&)}efV7y#q!m6eY;<5-ry^Jb!BVHgo)77x|=d^&&i{aY?-x& zH+8lBNrv&4lO$vGI7ecoJSXB{S>jF=oK7dsjKO7(7X&}`eRu7ZcMcy}5OA3L-WN*E zvdZn&>tvog=128e;aOA`dk4RhUew;w<2W0z9RP1mUyp49%75^Yz-L|~X#e}M&16T7 zh8+FFUFGh*vr@<0#gAnc6HRBD+Wv)WA}gzRiq);>oZv2^&?X*NrYpl(^Q^_5!-(X? z7s!@*2|FDo$e+&Q_Vbje$EHyf`0tYO8Qwje{nGY4nR`%RL~A`FbdCpeHPqZub1=d? z-49(L!eHmJ6@&%z&9aJG&tD)2Bd^8cxyTB1qV<{?t(?Pu^9XY{_9hmGPuEmBANmjYc zr;)@Sw2HESq?6-$mCR~~9hns*YyK+5{@7oB+haai~**CwrV+T&6rzAd|7)(gfH!*uLuw7vw-HYTy(eV1m_zvk2q>-_}dSa4rvXQZ`m1@s(49NH&5dzA) zn>DiEgUT?I6;+G)MkmeWdi+A(?UL0nJBf%oe`FDA1vL`WQ~74G@xa2|`H@3?Ogl2A z_xY!y<5~R_U3;^&vWN>i!WYdz64-|xO`rxJ59bSdxZ|>xwfpYW%nl5q-b=j7_b%?q zy;jVLM=7{ZhpYcg)z!|}mHDx-+QWyfsu^0(=?TV{JHo(tKV(9AklRZlwtY?XNf)}%rZcP7j&ccmJl4V;8nbQQrJNqu%2oAynkw{!xlc9w!Gcz1 z-A^CK)Au`jUv^l20m%*!-cp5YKM}8j9STYk0wm0dtyK5Ku43C2h-=9BBeqSQO@Rr} zSBP|t6JLOZ1d58{767nvM1X?gxyA{mB-?H(lxv(Y0M>zwKi1mR=u`Z_irH|)keiMq zCoYf3b6=_yHuswZF=nvc+d#ywD&lHo%*&`v%15)QN$qf<-}|36rlhzT2t+MAW3qz@ z^01#$u!ou=y>-kmUDJ3IrsRZ35!qyukQ}3famK*KpTO3fJW=+*|{gk z!K639MYB%Mlaz{b&g~rqYj+%9M7HZM0i%;mTeN+GMv($*5|v*5G>Xg25Rtk{yZ>bP zRQvrC%}LHivm*+HWEndi`aN;$$F$bGlvrwB1H4|XLvnI_qu(-FQZuj;1!sf!OvM>% zUdl7#Qb>1$P~#rzqGZ4ct@-M#3~fk?g#6Li9@Yg2x zn$IWldU!XNM*ubZUPlfH$A9gTF(!u;-j6g$teWVg)j|3lN$Y^)#lwA!hhw$h*L}ZL zb6A7MiDmrYJ1yxJk*R3!vmm$*W$ zLHQ%`ld|@PS@yT1d#Sdhz8hssab-cu<5(e^Bed$^_e175lc~aAsYt4h%4tE#1Ka!M z`-kYt^01Jkm(0J1eTCcXO4&0iva?XAlp zD(vfwdCNZiwb8eLHJ(qjzJ;s^8G=xdZ^8!FLqxsKgpR<@gp5CSC)f&+wA(Aowbu$*g5|HudGBt2F%Rmz99}x$95vl8NyCs$W9&Xp{VdWoI zF^0ZbMGv?>Wc=|R{(Y4tV3mLRVZo%B*wscj$MzXcY$u<|{30tILVITx3PT>^rn={=dg*+(l^Th&!~6(jHw62h zeU|4#gPi7j-6m|sP!Ab#-+%LaHH(uQCpm!E*Ri>0Q9i0s8286Rv@|uO7ZxxIM-_jGg75(Uvp+nBRAk+jT$#X| z!NcalvT~-=dA~7jy{$2&JY0T19^(M`lhP^D;2_g7Zs^sQD{Yg(Jv`T82l<&HVALv) z-27*{1uIe9WZPSS=)bY}{}qc%|H-kq4s4Q`79p*Kf34k^z|q7=*U@MOL?UGT5siP8 zZcZC+_Ve>Nh{IC;-r2!!WW(dr)Z(%9vQ$oJmq+My+J$D;dGdY(R8XJNmb|TkhR&4o zRAR8rlZ{mcdz*1Nc8o$sTGyXdKE&C@nap48%=}qAxC3$>;Qd&5$Qp9bg+0wsKZ->w zyu(ATD13K~uf>BaynDj?ZgAE;Gh+CA569AzUvaz?yyWf|!-~r5>vO_U0`o@X(MBv= z@n<~Th2=Odf?A%>UOXu4XSe8u5_-lfG9npU8=+q{LJrh!M2^MTW$-)kH?^BX;8pDg zGP$bVW;BA{c}65XWwnow>s1lNdHb`RKAo2Ls8sv?cDcr6_SX}o+FAs9joGJqejp?0{^ zij#k%4Ihk>l}dY(*w9kmGcNtc#s2&a)zgsR$Fzwr2*nj}y+}IsT7yCZu*WDtMQe`skpKXYYeJ;*ytk$YK1w;<)R0w8CDjXWJ#`PP86Sf?W$fa zPU*#QY`>Vs;|^sIV6-D=if_}F*-$_G;UjR0LNpSj(nk7Shc`{4S|U$G(b6k13bdKp zC}I0F^`Q_4hqc0Vd%fiHsOW`;7Km9?Te>E1X-+)l=|O#zxNWe4eA;^#bBiSW6r=sK zvZPpNaYH1QC+FwUO<(OFsaGVGKT_RCxVKF6J;3ZOrvGa02J7Y=_0X%AsC{6j!C>|J zrj-!YeAr#<{vW>MbBdt%$d!iI$V)JCXd5EX)#JSX|$K{&)e4RfFTA*Ny@yD?5s_9@`glE7L`zL^Kbq z^C=#ynE88JLWz6R-uRbo|FMd|HKpLo8xbhUxb})n--TlVH{$=sKW~X>um{Wynf?xd z!oTy+D@u+f^*4>R0YCvV{s4yGDdcs&-v*W|T%&vy3t)`?jXuOSNX31PFV4UcknzVd z16qCdbNra|M+~U1OtmzL1qQdOc;KKbtzr=BiedZ;JGt+Msms!b+8%!r$Tv%gE09h@ z`IvBLFy(`%RZv$whQCh%~gPT}a0vH$WOw$G(*G! ziOumx{mc4!M6}Xk!xila5NCiuk_pA9XzF~T8#dpX0bAeJJa;MK zcuI5c0f}-+9JQ16&ll(8kKaWo!^jG(W}XK%i-nC+ei7P^@%bDlTn)FcO`l-73S}+! zQFbsXK)70EdV93flcEkegWc&Ix=rvkj$&_0Ts3yyyG;b8RK-QXLP{!{p!yW$8WD zXtPU8D>MrhPTSP(U7|-_;${kO{EaPl>u<1y_wO407CZnC!_wV>N_x<8EhUosQe z8}o+uJ_fD`8Gl?gP+ja=Wx`b%L`-1d(Fs!z4J)BM(POd5d5#7jxZ zEF_hx9X-WS&lj0XtC<~o7i_|0$VH)mSR9gfwr=+LgJq3>99U{W&|;WmnRkd0zl#U0 z*jJbdL@76@t8B`Z;5nEs`K0zRfY!ymq~7skAZe^A;-_L%L>A)IaA>#r^LdYts+34I z0vvqkgYQ=qY)j4|} z6w+#^)+;q@%Z}!1wWD({c)itd4 zzoUB>Me9^N2iZ9aCBAAr&knw$Ctr`$vTV@R^-QcIbv`c+M{?OjODk^#36scN>kEr< zE_;&#JD9nZHfKJc@zH&%Ra|_@x_N0Kz<<5l zu9L_b@Kmj!D3kxKn%N+}d8#JB6NQXFp6c(_3{iYgBBnh~m^zZ=L@+NI`kWcc-6%Ug z`y8?hk~!JbN-maA6s)O~gJ&EZ-y5okyS|W_(W^-pZAj;1(O{Wv@zV^Jnc!wELMKYv ztW>q*)X5;BW24pCgX*ELv_q-~T(pjEoo^Yr0PS#sW}Te_q8(CSX@?Os5baPRHIKG% z`-JOCJNz-_)n(&M28&(KXyI*k{!)&c!8X<4?HOCA0%a8uu4fKse*!15NoZ)JMO@RO zB~>7?tmn|ZfzyDF=qlsEWK4PHGA#Lyjw|Q`??a;IA5=$+45bGNb(To6eLjXao zUaWPFC#A3-=g#Ef@(cm>-njY3+l&C?$JIaKiRILmfAYzP_`Ne+IhjX7>-BgpO$chs z7@G3=T8(&b>@}KGS-g6aNdAnEHg~LxL7DkXgI7hd82ALE)(47SV-ydc5Y+Y5hEZF( z*e{=Czx%>3ODwDxU>jL6K40RRs^In?LD&YYwV0sH;8Z|UmCE!Gz&2>E*v1)ro8TK9 zW`J!(qvX9KL0~!ds%-I{1aZt^mba382v%7Uz<;4Z_WoqV6`=&kZ)GhT9FO66%78Z1 zs~2S&WHLJGoB0a(?K-Pr;^w0sy7R)NGd!psxg*ER%F9=^%m8s=9Z<{Y!*!MMb87Ah zJrDH3E{QZrn*L@QzKM9WtIT+30wJ09a1x_7hzXmGrVvTdtO>L_d?zs%17C79X-e$u z4K=F31KM_cnJ^-$uZ8!pJKaZ)twSWcQnf|8Dd&9_y4{WDGVc8hl6P)p4zV-FuBrO) z8OFLCXHfh}Q60L-@rdC}NL2*Zh5AI>3Go~fqzyGM#eZPSvjNipw4vPk`-|C(^Jb|C zli)uW+0{as(}9-IgIG|^XD7C)kzbLLGd)>d>+KijX75~e*J7dS2xz*SId?iGa!NfP zRIHd-^+#CH=XTR6Dn!C=rIZMrApc!Poi@qjN>Z9c>NvDdxYpviGJ?;Zx8x7~W zK`pN7ydAL~U^*%Ua^_C|#k%%$s9o!3SJ)$G#sm=5^UjZ$sIV~-kvl566)Ei}Buj_x z71+orj(F-r>HVU$8pa5%cQUt09Y`~Y4NESc;~sxp9&G<&2PdP?eW)II8HoN;m3Bou z=OMC?yCy{wbKN*YR%jmBfNn@WQx(qb^<%rwDfj8_(ln~9bx0Ffu?bpg8fBuYaJo>( zb4^DmZ)ZNI^V+qvIFmU>G|!*OLbGa{aG?XbP?RMrk8*rVOPh2Po*CZDtUGpbyeI>@ zP{&tYsB8R(i1XA{>8DG-HIINIf)!Bb5w8o0Th#eYjYkfjSuq>;f!I5t9f!3}GD#V~7_#fx*2ywEv(Av4<*d&a)UFSz{j6#qP%2Lh z!c|lJiWT?1EU@QAic3H7-YNl+U2KVG;&QA4-$s;7pHLQGp7du#c$G`?n#Y|^!}8CZ zSR*j~zhtAF(^4<729f>LA(NGE#Cf@7^~{|FTY3K}^=7or z9j4*78Z|6?6+b+v4K1-6rrNL7PR#n=x7Q zK^Di(A*z+53GBU*G{IkapKj)bjF)S8OceFDg)(>Mz}O^0l@&M~B^00Wh6J+q9V8B4 z*swWZM7*s0Nf&`3I&^?n;v->YgED?#g;Up42DFb1xO`aU`K`D0RW4Z5fE2KHf&h2j z#7P=yy(XFD)Bp=$x#}FyBYu6=W$?wwh+Kbff+>q*BN|& z&A=41uzq50ZRYa7ez)sQzY1*nR^}JNaX>!P{Vkt?8S!q?nGryEAmb0x15xMZcrhD~ zwD9@640@b4c&n)gpzl;olaWqK{AH|2%#ZT`)HxE>w+cO~c@h|aI=42k3{mImS^h4c zdq6lGJ63=?w`lWPf?*c>acnw~*Iu6B2hbSqvSZTp2P46(*smL(y*--Jhk~*BQ!rR8 zp+0=!B)|9Wgts9bY>b~yqQ_yvJ^04#B9Byz7@cJ(ILTNLy-Nirs%ylI9mSF&;_?%5 zL|4V)rZl8__PD8@z4gmSR>*%aeewmWXQIq^0rTxO2|?tovOB{*GkcZr9+y~^u9SP;*gcXRzSIy&kfkvktJKV7bWcV)=s*Pi8;4cykhvQ_lz z{YVYZ{ZMI}*1!qaD#CdHTZLUc#8%N3qGVd?mA^oFK2JY9-E5Ai&=opHENjYMup-*t zLT#AVWCeqZ$q;Pu)A8`#LjAcc#9#(EEuq;@IR5yY#3GahZ>l=$$AK zk*r)*&*VV$EC*E2u~<$qK=pjs160p^?~wP6LRJ-L9&7lkHXbj;u$VJk3_c-3BFzco zoXflOsK!%~s{lHZGNFzRR#{`xok4ZmXTBu-6MUB;GK=BiCCslf6(X{sr%KWO;hJIs zlak)V_ALeVzr=Pqr9f=PTvZzM9i1mJ7#lh0usVculC-MwSEebS;CM#0x8v(?TUN1r zw+9@BIX|qO7malHg^sJMwUTyxinH+;mSk{n6`G|8^FX#$Wwld$u4(B3sw+gbS94hp z3=h$hN*ddBeg8PZvgZj~=0m9E2@O%^JgN1eduAJ65aE6O71v8a(EU}i!>(3uVf0*U z;!|VHV`Kb&3l+>qto|V6;XyPkgXS>CygJn3f?yg#7WT9^1KjBbr_6gati{>Xfb1^$ zp~rDDz@ZkpgrOcgH;@|CPv`^4?kf0?&b1MirR5$lb-W2?Y%W9YLg`Gm(cG-JO_;EK zcEQYj6oZ&jMWVIH^Q5>f_Y37n%u|>9>nz741&oh$=J);Hu_#co=(l>~s^ZlOtMcO? zy?BEwYfz_`j4{^k*>{GVq4InmSx;mXb+9)j_K0KHBzT15LIgh-lvD%g?rKKSOZxJA zFr0wyZuvs}ZD9X>kF$azcRt3(?*%k=DTxamqsx>GLsjDWI3`rcDWYg}4FerC=uL$! zs)36`jmr;BkLAFO)qrP;`q<(t3A3IJV~6ffXuvaNEjt8wrts1L&(s5JWM@RSgdc7Y z&s49ev@1A03}n)g9n5%)$-V=4rsM(7RKYY-rf7tpS&dr0jgCtB95u2CT4w2E6;p&- znJQAAJ+yU;IppU9uzdT2q1MLW8ukyP)YO6TKyatnBsDoa1R0ah4hOvMs>7RH9o|$f ziyoWFELb+EFWeqZ%~(|#>Q^Ad=5}IMso{(P%_N|JC^;$mM|bRGi{Bp9qkEeMD@fGa ztjQfPPm*`89%->qdE(;~=Ifwckxs#bIh?5YrTLtMATVrYY|$)07bDm1$}WVw!qDb4*Lc zQrmKpv3OWr0geW#RXksnL?10j=0Hr`aXoT$xwJahK;FTK8WP%4&|8r9t!Z=0X8y?% zZvr{`zd&y54K={HUlG1^g91)!B=?!tc^Btv`71la^Pnd72K5q(#l1 zbM-D_TWz4a@{|$J^GaBSiHK(BF_|{%Vv2(;Sh9fnH9ZYE<@NkYo&wm?zEcacZy9BR z8YgFqKWvvy#+D4PSp~!{0mD&dbEoh=|F%Ph9hu^JQF~fgZwVnd312X<# z9uT^nCW_g3rbRZG1N}L9SQlIqS0MT3XU_dn;!ak3<;3|3_NU9zC)@M-WS9-sIweJ= zQ%4q-!AA?|N)$u;RJ8QPnBG)P#V;IXMH9Ateq?f%oe)>QHZ_Z>Kr0-vYjsQn5e*+H z=_o_b6NPN!e-{lrlUGR$v|A!77ie`aXdE}dKU`ukTW_0UM`+_gAMgkagXcwpw!qLM zjzV`^QK!oZVoY@2`dO^W2~cf=a0itBfL?tbo4t3MuSeTtx)EL62i_k0HPe@LbfB##t zs!lpJtL&hn|Jb8c_~j?*@M}mWuTvx)6Z4%sGQ}}3w27BlqjFiJ@^*PcX+RvwD)<*e zt0Kvdw(ZTZY&0z=FN>L8BQr%r=P$w|RYDo@SgWy6(!VY{B(u{32^lKh12GWQQ-95> zNsRbfDnxjBns$5;(|kr;xAgsi&4`WGH%9kN)G?=xS+Bd+`nEfTO5i}AI{aYxadwym zxlh|=%12I^w$}HjY{k#YIvNP@XRJIKA~9No*R>>rqlU-tC)?Vp*Y$)Cx#1Xap4;F5 z+Hkp5dJ3937uuE{61fBHPr@4?$1<=<9KF*8LjiWG_&Yl##k|R2w^)YOb(8gjVj?sX!*C;uD0uTIfLtR^?la*9q;ouJ!o~k%SP$B9L2xjaKzMuM z;F$q*FYAZ=nj{e%4iC!~s|xp-x!8n_yX9nG0I8*aXx+7g7i948aF}Sok~QPe;MrQ= zNrMP_*ZxHZ>YL_tbYu3k=%3+yKYX?~7E^s^c)QY1e0W~8Y-eGc5k~wRaa@m<{;=z@ zih+Q0S%LE?)NpB^hM=pzV5mCnpiaE^1$XJ_v1D9H}sAz#~ks z-XpN~708O7`{)2-7doOT!qk$J)yU(fcC5@IMeorHx7E4ja{Ap;nle8Y_B0PGV(o&KkS4x+&d6rt4q> zG<5>t>!Ie_=pn2W9aXKs3J+Rf!u+`W{&`BI)qwcokrY$<&;@7Ch(pmk>wLAQryT}w zxCJKi5>p7&sh=pB}3HE30$)K1qq z8^=ug8KLR?g9HArmzsXF7XuPT{2}R|Lu>$<Hp9#UtaNWPOH0s&UF^#1i{MdHTVL1c?p?cjMz&sClp)(UhaB@( zeWMumvhk1stGiviP@0{~eVp7)V_coVK{n4{r)S6|QxE3->&ND*9P@!>=r(!%@^a83zIP6vX}^#M0@{P_fg|Y#&IV5G&3}5Ob9?q{S+!aQsY!}sN=*xnw=IEe+!=| z<}sfsd6xy#MT!u;eF(bg7lX^mZ>)M(929&;j(pv5xy2^*D^o$T1tKaH_w5-RvE+>IgfmYe+7WD3dXkVeEgk+;TgHw&!70^TG>{$=sJr56Dp9U6pV>*^BX{XuK!5281ZhX)-7h&U*F=-vogSqgPnxvueU!ca8L#^{_&r6jWjy9pYU6P9>6be}W1f z%qn!FZr+kgH#J5S+gNdY))*3n@*W7EQnO-Xh4#VkwqmMPhcmm{jp<`j{;ao~dh-T1 zG*qdxaWj2xZYzDP-e+jvTa3~o*@=5uiCrQj)@aGAdcojLn4twu|Jge!Ms8WI^|Nzi z9N14>p!Ui5$?SlEkTuVsLLHX_ zXDsazm1CY`Tp@RTu1=K#=6XbWtP6{jRAT#3YQa_`f%OF?Vo5#qpzuzy!7OXUm0XI%klV;PSdehV?aDZk6+&jddC0ovW zu>son{=06p)FB6p?Nt#k5R2DBZV`4dg6ZhbVyYZCeEcff9-J83J0U6Hs!WuacGS@f zR{e8Ne1wq=s)!e?sGZba+#~J^vJW*5;47mCJiVSxTWoyyPb5l&PC$>(d3E4Glk3R#@ z^5}o9^X(tjd9z6+0^Sru2eVk+1g{l9)*<5`b|Qem!jeGn0^3#ud|wLV@BaL4l|rH9 z1E8mWIduiXyF!YPtNd~X-2z*`1J3o{ha`?@UN=)EfI*@Gu66nAA3zwq<%)hS>fe$% zGBoGx0a4#)TkElB5Mz?6)u(4UvvRt)cjkR)HO#w}8{9U!xhtP^G;dEkwd7Q4+hM-w z=W`LfKgaQ0gq*Usn7yBsu5%AKGbSBGND|Aj7e)KorL(nB#3Q_Vo^mqz!vVZ7Nad9; zhYg;K@;g5ZAX&utDQIKN1ekOJ?&__GMQ&1A)ZHh0JF63>Y{K?;toh z1RS0*=D-U()!ikGpK)OyRo)_Io?Ivy6S5<1Ig^upzl~YG**L4)q$3 z>~)>hP zA9$!8uN3`f3oTJ(G9En3lC;QZrQ}bO?9B+2RZo+AQCPcKACKfOZu;25ikP!V_JCN; zTUWH&2nvixUZ#~>>EYicJ!wvR#H8MTZtI7MU5Imte}{aHdAJg>vRhNT z|J7$5(>sh7IC3In*wJ*UKsO_mZ!~O=YQ0fbgC$HTm&b&;SthlUTrHD42_&r6L2O4t zziLJEMue?!e>E=Ntx(lDJ2##AJ#5=coH504#$swUD6u$k)BTzfy}~ZheN|+Ml;MCS z1r0enj0djtNkLD%!l>0L?L|rKnbIg;df4|{d6@yWo|5Sfw}(s5S-)X)Dft_NSl10} z>DL7=YFmU`V;iC8s)hBi_ds4Pm#!Cfyq*=^X|YN_swB60I-&;<{vPar3c1vv`1?&? zBOK%?&+GkFTpj@7-XZ-$xFp{lsvC&22kr$j{K7GU=vyporRH6 z@yOj;N}_&~=bgMYmrBB)P3PO;_gx$PDnP}W`2fV7tt5Fwd)fz7B=>oGUwkaC9^4&6 zC%}1WZw3IIhg{wv?HdLCp7I7w{3bYb;RVFyXl-#cZ`NW4PFfe%ZW)PMBFDx<|G}o` zwj@37@p0cAIrFuL${r-9BVc(>cu~E|W#7i&+y0NU{pbsh6mnrn(Xt1!bGii$uJ!_?xPr6G; zGhnPbU>bIbhslzN6#iANhc^Mq5kYZfytQ0L7-7T}4$0Hpq7{}6g{%YBOoj$T{4tvV zCew=es1gbJncw@q)6x&80G##fPYztc87!N~j!5;KPIq#21)QWpvg(Y40w$kJV4+k_ zppH3vb+~P$9iRfsoCnA>DBvT1SF~OgW#~=H#=z5Pq&p;c>ri-3p4zFYY(So5P+}5U zrHf-*8P2PRKUNDzX`UGhiYG3n?JGzWO)fB*b&7?XFoPzOQFVOMHP>rLh>*^sir*VR zjjC0uY9k<&gI&vMHg&?4L+*tAZO2MG@-<5>U*3E|ECFqo3D1;%VAn$S7^(2lX@)_C z%lBsqM_(BH=F=&j!A&SJMTqh>IU6=HYJ1c>U(2>I_N~pm`-@hvq8nD z#RQ{W%!aifmo+d(>&_mrtf0l@Ww_*AO5pqZ*+0gIB zVto*3I(3vV{x!j_;LxzTZMi|~WbMpoB`N0;o3i7QODt9ZM>zFMs8A+iJ}etNRQC)i zE&`bz^Rz&{fWhceN|(2VUtvJuWJ~OBo5kH;uS&6`V_+Y+O|{R^cx$PK#m@RLa}ud$ zzSIjQaH5-nF2i~#)lTO_)zdp6P9v7((nh=oS|vc?ZgEWfG8{`sGlyBy~8! zy|xd+=Y%JZIP@!tnCKM|%gpQ#6S7ZvLu!ZbeT`Q(pl>Z?8hdFFEpDs0{=SZ_rVL5S z4$-=*| z`U20101Ow*B5}itlN1+#%r`heb#oN-wyHE+g81gG2wgxi-PZ6tVHdb;39$LZudBkO zrbDWmb8bOEC5`s)g{6IaCG7n}SB!oZJ>*?(og?P(l<;clFHu z>iP8TScz{%nKaYf>rcW*TSfKLVaL>fvvFeWx?j~{GmuIdq3=0v^%sW@V4&wwM~45_ zH-V(!GMQlPzw`j@>jws;V4~~ibxY<21AGB82MP*OQ~Z&{A+$sHS_`>_1N?j9hSXGl zmgP4!72b7ea!Wn+FL3eSiu^yTB8OxPp4D^nzqa4wtL?`Del9@Gf7NrlB}e?Ve+FdD zoqmL93^Dg~*CldAJ6v~M3hykzo05l~IFoxXC>5W@1`)ij&62PCk9FRkW!3NIv{>KAj z|J~&Ok(>OQr9-gI<7Ge?B~T`7-;~KfwpF-(; z{juq>PzyqcZ07F1ne$J_@Zu?cZu4wD%|R*CQ#6fFacCO!&KRr_en8n<0ucd&AR-_< zL-og)fBN|tc5Hsi8^5I94(v__CbRHVpR(n2`YV0P!&Y(i%Fe4zANf|kc z&X|TsfH<>tvpMHZ{1Zl{p$f(D45jt4w345;GsS**N@UFi*}qRUY?h8Ro#tpa*{+`J z$|*u@d~yL(1o!#K$>?~mdwK#?=hVzowL^=fYb;j_ZyDsCw+@3JW+&m#mjmWA8^nC| zFjo5uYgoB-tg{bS7-62w@XOaeN$lpz&Ogc!BZKi+ODxzrlw3mA!s9z!OU%-HaB5(R z*qvV6s-}x^>Wj-QB&TqQ^%Ki@LKBQ4A>89M2i`X2H1wS;N9!?P#Sni#$dIU`z29^G zr1@)HXTjZbn9DVbF+_s!V`7#zD?B$4!gTd9P z9fwAIr>-Et%wCkL00k=xl8+kjaQTtid1vhvP7r$z(g!&W=}kduSMqy7PuAX1tLJZt zl2j#5AQlA+aLfu5r1(2C9y-sfd3;0TKVOos`1JLF-`zg6g^MB;rcrR zvTeOA;uN0wUD<=s+CQ_&H7&W}*SF9SAd~!6*g$%YLil=0L_n<4|B|$BV)d40V}|A< zH6T_S&lpr)g0yj%MC_j%F=^P!-r2ez@U~3>Mw_EY9ZNa!bF%{t*bg=8?RyX9Q@3W< zX4g7>(z{q+)FEeUBu?bnU|=L7qq=dmDZ-B9og6=9HFnI^m2lxXz6;C(6|1!IaqW{F zGcBAnRJJMge_^z5u1xKErMdr^oJ{$3LxOf^L|zH(W23% zbkU;497tIrvIaGSmegab&t9!sB4sa^L>S$);w=uI;z!o4A9%;`O=>WColz|LLEoZ( zJb}+8aXb8#n1yE(eAx9>-M@sIJ%{jR@s@P%@Y(Yt9 zp1TfF$1c-s9tvg~qn~O%5$Vdu2ULRu3LpY}y~>X}s2z>$kBH#O#(KlX=}Qo$;l# z@?>f?38W%bD6KT7o8(j2Y57hyFn|!XR~;}3`?QKVbdN6lt7;D~n9gx6y|=wqAN(0L z_bZm_UT1$glF6!QFLrl~B5zGL9ZaT8gqHyVQe#_hj_`L@kR1*-M{wOf2sB0&DNphF z-$K;4yF{Lo0h?)}G_F=&&pI8$TK?$W1vv=j!C7DrYqVygW%gp=nKVd4niw^5D12GQw=w{B1U<~10h=F#RY^Y1rVYJ12za` z0ny9LZ#A-G%>;gg)z;u14A^fQ)Wr0OZC@S`klI~9U$Z6%*Mig_cf+8I{?aUz{;oj? z9|Kjv|D_E1z<^FhO0Wyog8i6r#JjDr>`t;Sf{9hONOB8`f9Dnn}TU2Q< zngpcthq(->?A+I1ENSsG73{S%uL{W6knx9i_tCAnG) z=j*1z^)GOXta1IzfyD%`e>lazAJ1*qEfIvsR!zUlR>arJ$1RH0zt@=%V|xGSFeVd# z??Jz~7^ItwcFo6bvE~36?*Mcx8rQTQdj&DYWarm@@h_H%eBoOe>`VRueX-NF;)#UZ z-OEPPihB*BZ#BdalRftZ_n!=HtVYDp=0QAt_`O+S1o-$iu9D?F@Qv$PZ0K8;;ng0x zJa0Nb>N_klfJ$+kmx`&Pv60S3kDvpOaM3$$(yMe7|AYK{|c>Y8Nv;qJ_o+k6oC2? zykRx~G2^WE0kaz`JO1cHj0|68Em>vj0HK83>kH^`Eig;(g{^_1asFn+Luu%5RreyG zf#aRgi)6SM6FByKY;#02PYtr%lAF21LTmTy4nQNgkth_W2p4Nl zbx{cRk97cIMppnKW*kXYI+Qxa8q_ZJHIvhxVE{3^C;9nYR4v(EjsrnQr3uzYS^c1J zx_$(g<0RCI(80YHg8EXCRd;Xz)JI1#<=_4$n{tKvz0M_65Yz_&urGE6`#8jNyH(C< z05Qw>ifW#8qnph*_2p>66#fFiwkv z3MI6s-rZ$To{GGg90H>#T_*|qi=1LYq!^t+N&9r~bj3X*z;iE;&|Q}{c{7)hIpa3! z3>NU+bDQQ6{0(@&hZB4zE$g(LYlg)4FDt}9*1dogBI)wSUpgWF!zyFcN@AZos zA%KfvsnGIAK88qzf8+fvkPCJddvGn+Kq~0Jat!2ONS*;IXwBt4{20PV zIJ#J8T6miF*mSF$X9pB&6DW{p@y6wOxO?)N^l`i1;PRGB-h%zpZ?CqrTe6MZsK(+6 zy41pEbK2S6=<0vy2!28K6W50&gsUR<*zJw-!wDi^8*9g9L= zVo_AeHEez#K1t&>`~3#oU@66v^;e?FcjcG-uT{-ej2hMRN#w6 znlO@bW55brPn%~8X<8Vjsmal$IksEYdqm(wWXbQ_CaBNpy+G>Yl&-o2h zR5US-XA*mKTthf3X;poJk&YLVWn=Uhsv8XUUVhG_d1*zs)lKk)Fr8o_BG=uBcFTY; zoH)XtF;UR8Uq=)758EY1PDPX!A2IK=+!;>+253Z!6rORHe)&K=wY#yeyUWwP&}K2o{#k9T$7b1 z(hm+O$ahzA0GXg51?Cm4;fsYd^@H4LHFpM!J(Lm7$;dsN+8=VbE|moZ$$1&=po{k8 z!80k<)fAZ?zp%G@Hb?>5^o1n%w3Q!GL%pDc>sqx|t5s0CvzBn3j#5hFXRAbkA_Q%w z3JEAeKKk;1)X}71H2KQ|^ybO~lq0f%COvd6R`F?DzrJlX?KrSybm%VN0SXg#y=ZuU zsl@TA5S(<`&&kC!^IM4Je^b(bWJgp96!EhF6F_3f zi@$G-nOCOdn!Nb$YmCCwnxj#8$Kz&Cx3v6)T~#5!x*RnTFe3(eF?SqWZIC- zkGJ)E*t-`45{a=6H-%p<*toyDkd%p6gS@jXVg-1K3Mv{GKY5Z_B<*L8&Q>4L9-hPZ zg2~5io9NF$xtefmP#!+xO23ahq#wXxp;$Ty z4&#Q3Z`bQ%OOlGLVS5kW+pJLeZb-blRrVjD>!g+Q|8H1 z=~|@{xowNasrmDn2$6l7i{7M=eAbij?XjXPPk8qerLSRzddBi2Td+Cihb4Dd&&aCk zQd_3}eNxvDWGI_-&RKRY``qOHPUoe#k`Ze<^3USRsnCxc3xrJ5lxwXq-3rv42Cm8p z07X77`1GMXvL&`+c&s8mg&_H13;*}WW8w8o6b=fJeb{q#-5&w-MBy3te%2kUZB7v< z>dOh8lLM12V0^`>SB6bDPk@M6Ey|+7h3^RL( zIb{nAw=G|2KEBV_hWh`_W#m__*KyOGKw5cO5hin z`;7+pn@7DkNPBQb)DO?YmGhoY?~D8KTtoRwy#I!cCH$k@P_|yI?aSx~X0dk`m{4E5 z;ArukVQn|-)a%Kzg8>ysbzO+56KAokNt?ir<9M?){*+Xb?_Xni(t1l3+hx5i@z;*7 zJH-dZvpcSmy!f+CuhovH>u)ImWoyiD$6dxy_n5bfhyY5|CVLCY_ZJTrmhnt>ZcL${ zErJw>=ii;aTOV@DOYhqzFxovjntxK*&N_|w!mLhvsT+%ppDfd;b^B#puRWdnhEEBh zDKp~T!ny2E=#OXu8K=4t1z#fXMMv(E;ep&(#%T>)Uw^8I# zKsfh_T`jAO)J5AX?!TIPHHj1cp47wob7k8SZ%Zq1>+{(ddK2*!1ZpAU+U2JLHyt5G zdQr#6=7>jcUe&^b09oN|s+sSS=n|{P;-t3?UuFB5y*8h$8P#I@4&{9598ElD(2o`T zn&5-`<%}H|)5tWV(A((k;sM&>=zHZB@wMXcK{?f4IOGMmtaq1i#wcao{%Lvo?W1hz zEcPkgN};u{c-V2$!bSpgV)v19^h0>uvfbr)CLJi(q1gT5MUy!52JDT`i02gFjyKxt z>N#n5OvRo7&!0Wm9`q@F&L>}xOO1J7u&Or$M2_~{5HUV1nzFhX8UO*=YNl%$xjY^? zKB0*8XhuHDz@FcD3?YqE3so;%H}~ySri!I-xL|{5Djw+T9wzIWFB0V}D-u;7^4MQk zT+33iDeJEf*D9B!O`44zN+0xYJ8J+cCXQdtGV zQl_@ry|Swq>#{?}{DU;r#{@a|*oOV8in$y6Pl&fj9_MN9pA*YD5j=jUpRKfS--h^oK6zn4tOUVH%LKZV_? zfTHRj7Vd8nC4Y&k1e$mfW+EkbOgQ3Mnb=@pYJdDS zasYNcR;JgiOq`s5Z222DseA?JbgE(k23Zv%{247+(3FzP^guW2tB07X0Kq@?-8v+$x z5dv!*mI!*By^=5f6uYv6QfNYHPKkkB@HkitnIMl4H+KzXJ0%Cs7nF?^v%J*el)u>R4Ga>0v}3a5K}jq_8#-(8HwJW!TTp zeM)1IigAI7z&LzV?A%^XBh+pT6qSxh^fz}k=)>rjSt4rEU9ZHMV)68KSViO6&!yMQ zsi0`TL}$Xt5j9nQ1Am_=3FFhMq^+qPn}R4Kg6sXcg^9Zgj8-!-Svf^3e9P#;{5!H4 zy059Omf$R>W}0Te)q4FEu=()jEx1_N|DXJ9V$TB&H{dbS+a}F=y(Hn zV|Qz_40?+Y8k^A?lntig^TtPoKlWtXCy`zGQQm=w{Z%k<h~1P=0;H+Dz~NQ9NGYa7UDRY7&Sni}cVhUGd3y^;nzdU?vc7nT{&h zhUSW^@@0@+K=R|>7$g>U_kKD_1IKMTZ`}v!hsUZ=$UDO*=?stWPkkm%{kPWLM2{dE znpvx&)Kf2>uM#Cv)(u{X4(<+A211$2 z$L*S_pS~5AOdxA0(KFeaT{?!uIfohnnh+bMZAb*O-IXr;KF~?snzH!oNev3cp}U=* zwcW*t!lDqq$#&Ph9JPe+x~|ZZ8s?7$kW;Zx_K@T_ZIMEjbn6a|023KX?$B51N=u_r zPl7kUK_>oSB9}lg`9dpinliR-KX}%wDw-IMJbWo1Unq<6T`{{*m16qvkTqf$;M#LW zHLOw~kZNfA1Hk8pE1Dq#Jx7-1w{j*n0uK+7hfjqiA9*4RF^)@Bi=|s)#){9X!OFwx z&I-jUwnwH;8bWt_3+;|4)!AFHldgpaGgLz_9O&T9%sNP&xP?Q-${zk%krhF_ixw9; zS0D4A-`Ilp>cf!v%{%_%+NPIuA&R$`Kh-lBym?+azW_l#33UHgfA{Z_g`Gb1FYYAV zA34p__gOrD?eSdJ`vNM;NtriPu)hj>f5S`T^1S}s<0UG~w|qUI`V&ZAYBPO-a&d#v ze-42CO+5Y&V2Byie+fhW1tEa{I~ejW2m$=x!H|DJ2>gEuLx}$!hPXHo*}q^&babPw z*3>r}`3*V2GCoFP)gtxeAk*zdZio}M(`xp=yM{n2#C)~(5c zbf$MMP=3EYeto-9(d%)L5V?F6>`ut3`B78xYQ#gcAcA-)1I3#*zSQ0O{TU&m0gLE6 z<(0O%6?+yT=QhwK`|7GJ!7+22@+W}^LdXJF#Q-nOL)DwR!Mys{7?v`e%E6`P`*kLv7t0yU8sFc!_gNJ zY*aaMk#QN532_B*A#igpq8Xxl;~yR%+QKRIH>S_!%8(#`XkwA}@Ce}~8)3+tg`wl( z{mjgAJZkc^4rHBFwzAbm6bw*L)MCBa7Axq}0T z%!UlR!`pq*;|R4OhEebZWo@N+U{X0IVhx&+lZBR{xm-vrmZVe}3iElWGUEAJks)bs z)agP|f(lzXX%(Q5vWAjF2z7vdFe^019LQ62s?R1ZD|$uBzb<@vK5p(f%Zf^ciT_ALfchG@bl#0YGFp)aA2s;H@w(tn+%amgM7 z(I?h8geR}4Pg1TwbN0Irw(EA@kB{%;`hcpEW8;Ic_k`dZaaEZbfQ-gaEcA1wPHZfS z9T^2U%z{Lp!_V;qpBuCZMG4bATkCw%G)S%QQK35_dcnWsNVlg#Y7J1*P|AXBL^}GG zdxJP=_$~z;YvuOS_#JdxDulfH0N&z6nWXg?$Qfy6Q@0SeG>M#LX{$wLmZj}Zfiz*` z!})YlT@q1*`ue-ufY=XnFr3yRNxN+FNx`H5&?^ekIl|z&$xo)WyyhGIXhN0CTu zk33HeHk$mx5jXa*aK0RI8v zHQ@^h`}|XNuX8PtpN|LXQ|wV&nUaHugFl1Jh&`V$BYJ0%}|4qt_!Af~n zpHsh5-%W$m*ei`^b;q_eS@HZ}T{?%yw8;KAn>is7{c#+$JH0>fJ z&jca+8D!MK2on^kHGXO0-~8eu4L#Wps)X%7^rF#%K9cuXoG{nooADNMU|fKU&(&&T zL4Ft4Ht|^W7!tTx@z`%`kOWe4&<-Ucalk30IiT)$GV*3>=TK|N_#4>;j=+Ar%F@Iq zOqC(=)To9UKUz&K@`wSzk{l> zzpQMVS(%-+XlOY4-csSD=Ug!2nx<%oHsHvLj${~OTpA7sb-5V87Pc0q0M4i&T;Z%9 z>63;3IS-tG)B$C&kv7aqU$UDEye68)mk}j4nwfW4W>_Dt`r-f}xgIUaK=|c+q!tIt zRT5nU?yPjgkx(EbX>F4o-}5aaoa8B;q{3#kdlb82y!vG;Y}fdIufq-RlmC zeuP3Eo&^VEC&Q@W#a0WQHv76C7h9d(+s=M7A6?YmeErIhHf9$_r5ba_$+Ap9Oi*6O zzh9Kz(rS|A0g&$_l9|rLvv}|t9^5(v-U6R)+o6ml9ov6)rOIn)(_~&h;6v!)G7X1p z)fq>=+i8q`1g5CrFU83G=S^?yV=8w%>SAS08I3kTA{t?CVs6 zkS-?)55JOoTWGUx^HojU#n8E)TSF7kW1E10E?<%85g>}`dTwuK*3Iyct;ij~D=@i| z!JPRT<4<7H-Qi+sHDbRvCK0MwI0lHu#tOBJ*Ow-wW{*1Ul4ryiMrJx33e zMm_uD5R%xKM!ku-zTQtU#k(x4Wy$KJ2Mq+yK&68^ZUWh&!1M<1j&l|Es@mFG!*+}I zIz`YxIv^r7iU7dz-Hi2!)%u!?rXz1?&&E01^?qZ({3mSN8!O(k;}X;bNp66pDLeaD zuCc@sHQw1pb#DuXO=g|>njFseM9uuIEw{*<8ZJAsGaO0Go);l*)UUt0e>x{BjzlNw zQH3Ypb#?-2+EnZlqY zc}e}jWZ2?_3h@5dI!2>F@s}3*{tQ!u11cjDVyI$+UC zQ9=uz0rh8L@P9ndOU(sl`Y7rzxQG6SyeohBOWysa>LO0x_fO<~sVetE#nz>Kew@D~ zBKSR(?ypbecO~UyKC~~>zp6a{1bwYv?$U3b{x4$`mZF^SZ+Gb>T77YfIimX$^Zyiu zNiV_b*N6F=K=m(HSvB}OXUXcn<}CfUD*V$}`j4u@y}z9$SZqe!f2qQlKb@tvCKR5Z z7tAX2ij@i{yWfbWI5x|6bgeb_8YZ%iubSH;4oQ@@P{d8GkS6CZCu2l2&mVOIA!U(_ zVM5=9;xLctv`L1YTmvXk+RhC>@2C{;VVTwII&6mDT4>TzzJ6?v?khTmU2(XQ7TPWg zQ$1X=R@ZGd?3!d^$a8ZR3U~4R2Ji!UWE2__-cuFragZYwh-W-)U<E|wkC}@ z_q}=5>~}@V`P`+Dup;Owa`|OHtU)REY+M(vBQ_R5O?9^Y!{d@7=-HyO6}fEPPn|>m ze4^J8o9IRf5|h@I3=|SO7JIU>oH zF~*tR^CR|0NJ$_g9zo(kCaofnusK&mu|zz7VZi`D=``?%Ok$Co2Up+P5Xc#Z4Q5jh z;|K+ui1UXgj=bb6+V8YQuJ#W`=&oI~*=0X<-}z1YbbTK5o%AE~A(KXJLXeUj9A}vr zmV}@d5;*y$JjAL{t_seccJAOC!d3Mw)O=$M{$i>B)~519B!%I??KS!5-Qq7M6+HVX z=8Mam_00W#;eh5Z2>6XX^*01a{SE?*{@WP(ub9jKX=7;pZwP?=Cj=z^4-mkM(0d4% z{4{VNXET&H@}sW?sT{%Ht4-%BaV~%TRwByf!|O)v(W5uS#e&NWW2}CAU>{Pa1S8_F zAG7V`U3M%0>`b6hBEGNsn{^bH?%Www11xH-^VK|==c9My(#nkcBie2UhFG`V=&S9z z-zbl_30l@#mW33P?kJr2u7bW2(^+TU*VU+Na$crwn_zV0nJ_A=+7NY^G+w0DD=3U5 zQr3+uG}mn1N~>=LmZeB(Nu*NLd1qU*@FxUq*0cmV(idi+se4TA5fWN6C6VF| zg{(iY4@;d&s9kbDQIvp$D#qt=a}-sRUjfww9jGn+Xd#D@y3!$7bki6j(pGd207L8o zZRsEHC~f+Pl+!q!-beD>kFN7LXAX;;e}g45(V1^7279qm8Q0G5oa zXi$A24%Pr|=VWiQjB>wh44(Q)t?}gg*LQ!wLKcUFL8OKiM-2oEq!rl0fA(gDI)qs$ z;FIqzT9p}kgn??%1GZ!1(D@6!u@5;=;(@3zfnW~0hBR<`EQi@ z{~u-k78HM>%)b{D0yg9PNp7@8Zk?g`OV#Hjti-3~DpSvyY76VHrwW(rx^{tZfsmdS zrnBY`u6E3v$L1bK0I7_V8 zt{}a2<>1NAzIBVs%p(Gb;5EOm^FyIkdGJ#4)fMf5!#+#*j-*Y zKt8wQ_6AK5)xu`SwD^=o|yNY}k9u$}Ai`#^a>vRx@{}gadj~@<$ge{8_PK*SOr072ITGCM3+uK)H z9G&%Y&#gdBDbSa_3?X_wEfZ0;CWPwt9eg;_ze#rgtuM5}*pjF;U=s9EkeBYe#uXfGfD2iWIH*xyl&yGe+3|k!F68+O=XJ6L8X^#EN z*BC2OOCvr8(*`f)e!kN3Wkhh2e(#Hm{(RY3Qb8{j7W*H&$UX1yvVOBmGqgFtYe8%1 z&mI04!b19NQFRrN{RiuRUarsW9DXSJtN!Qt!C&4TM_Neo%b)q*JO=<{C-ZO8_R>XQ zBB+(1*#ZN`dx{8V`fInOpUBkoanaw1ehIYS6fuQ28LMc2nBWQ)wRJ)CUpuz$@|$jK zp=@ky2<75(ExdVbjoj1b(hfXH2a-&es9*E6WE2+^JFBL*%_^d+8ua;gnfvtK zo)W>L!NNd%W`qFW=opU-$tFW2KqP>L1t1a-z%np!50}5eW(dM1W(dP1wycN^p`z~{ zsnQL)g2je~D8Pc=!2*Jt#y#~A6QV${;Xonp2XQeRJTF*SZUTZQP@}0qz|HD{Z4NRf z9b+YI!YEN90iI&j{O(VU#SU(;c|*Lon+Q@={f^BHsW$`Dr*@bLUd{di&-Zczbncfn zIr1XX?|f{(J`beb^DEP)T7sC0y$}C1Y@li<=i8W@yZI&V1oI@C3h{|xzOm)jfjyL@ zctV$cY3R&6{b>*^9F3Ybyug6J)m6*2wW5IFsKGS^&I%`~h#;6{=Q^{vDmNZnzRb(X zZ?W`_$phusmRz=3!t>SzM+xZ!08ZEo@#gLEZLHh&LVdo(7-l@<0*svJ$?2&OF-l*r z*B5%7Mik5J@9SZdQ=bm)k&}Q|15t+i-UbboGHsA5jzV4w76*%0%T5Z7WNj?rqSrK zV%D@L_RL%7J&Z87=dw;)<cOpzxm`;Z;%t79@f42@8_>;!N>Bvr5D!X5spV0LN426&0f>sx&TN zGBPR-^5h)hb;M=j(XW#gk`$kqQ}cf=$nia=`+hRF1!Xk+qf$LHW8<7`Xu8Y>@ z{Uik^Hx}tx`9V;REkj1<7ihPoex+$W5^7$dT&GF%-P11EG-X0;PY zI|X8LedNi@bjGqM9l!A8KHr8$rKq%!`keRI=B&|Y-ek&mXK`7Q~!4CK}4*3tA z6*B54Hli~;d;k_2Sk&=~&{SCYGyrO;24?c|ci{Fi?cJ+l4KAFTOjLlyI(;~OL;*w2o{kU&agU~@kdh-|= zWq||M{FMFQnDZcEX`@)v$9f*k4$jzYE)$bJ`ivvE^s#L}k~O|I z9bTQi&O<;keSm&wrJO_9q<9V4v67UfY7Ui7_UiU`SJ6BDQs80%zvmSz(P?mv`i_H*d-irE&t*))AW%TVm%YOQbf> zwR^?uHj3w>B%N-)Z+)sOy!H$mbA%QelR?YfO-n=GWrj1*ESRxy0#fZE5-Zq*ALYLC ztMUZ|u+heCLPterszH^-1zq(e?JN5Vo19yJYdkPtNr{dX(I6D}q9_)As%cu%byFTYlQs1Xcyu{f?vT zZ33CoLk=oFTn-M0MI2>`F_&@Ij)6-?{O?Ku*$KA@4JcdHSTxGjp+ad5de;puD}nYV zrl&Mq2G1E|h97n?19iezyci8#r_^2~sNU-NP`DzWeHDQGX^%ZmWj|oi@q**<=>*N0 zR+CicrcahtsP*Xf5uJ&ZU?6QhU&+1iNYS*OEVoh9Gimekh0X!B4mEgJ8j=;(6^RxD zFb*2H#!FU^LejMHXtrv!%8tQ~S$~>vDc8x?1xz5{$?b@4dZLYlyssV+b!# z7K<|ze?m9hbe`6`E=Bv+q1;{#6g}X8?ijt##x2}&IWXJ?#nny)i(S&Hc#b2jUqy)VOw`)h4NP+W}sXIQ_f zQ{9U91*mex;Cpk&apPv9}e(U$)(;^+G*osO`@otpvNnwZ2AHzqdAXNi&5_R!BG}@ zMRi+VP-YLS0gS~-GV)H@9S;W4$-}6Uf@*Psc*T*L^O`wbVJbLW8EQ^jU9Nz;rVNCe zG`LINJ&`wdMni3gk8WVPknwyc7obMTPBY||H2^iLc6}@J!0&5m)DUJ5|0gISnG8SmGXXSa#SDrJDu14Jb<ot}gBU$vTv&R^{{`G?Pl0*mHM?PNiSo% zsr8dqjb3|a%a1W|FyC5}_bmYz7}i_wclI!W{Uc|0iJmAOO9q4!kp&2lO>0f>kabe@ z_aFp`2&4kEQ@p{>sA|3RF-RK7k_6uP3|?Bwa}KP2c-2q(qq_xiRtVsCY?(JkowqSW zh{$WPGVn2qm^%j5@yQ>3-t%KRzLG`=4hP*D9X}X({EE@W@Oe?IBaK^M`lQ*()Xxe* z^9*`;3P)}VFMiYK+vmLG9`1gwP#eh_F_H$3l^M@?VK)o*oWJ?ltSyG)QL;GNF_Kqm zN=lV$m|ai2ng0QQpy(B#DlLpOorecLp((<&a8?xDAAw3V{q}48(U9LfTmO2>V$9*x znI5{1eEaI8k!;QW#e#092$v77sXM7DHQbn-0rPQCH{{3DDEdgVS+(Z*G{l*l3YE1f za3$Forf|{CIy3G93Pl=(I{ML@Vg~Pu=&?cfpoP;VU_RoyG(Rq&9aTStHqIQ74mrzV z$QK+iG$?7%>%B;8059++flJ2sP}*he@}YyT{#I-4T2_wl6-#Y zx>6MC>?}3Pc3YgphqPuy<;o159X^uKw1y)qRC*NeD^>qs+rVbT!^PnD*$*&9{D+^W z?P)(Py_b_*in8JY&>W`s(r7(p7gVGrUo)98lor+-(c^c(RbMRzP+5RMvhI@5?DRq_ zWp5P)%w>oxb4RgsZTF%JDY+uFme|54E8J`ZPz-n%YZywikJ!PUd!<0=*XqfUBIHx_ zAk_J*2-5-HdW4`Hs=V2l%YM#yk#kM ztT?_y47g90$NNLU&Lf3NR{!!6XMZsKt2IywSZtOQ8k5PIbC*4E zdF(p;YBNdSrJp(*2d`#Qo1{B*1o{Fvd?2>>hLSD_a%-=SkC1Z3BaZz2Wa{lT#lubP z3Ao9(w}xZ;fRkKs!KvdGMymH_>2$r3&hJ_}VBB=Q`3Ec-`HF)P8HJ3#P#1lSE7FPb1e)uW2QOWALx728qI)x>LfVD}b<&ty*eeB6 zNi7b&4P}OW0r^3vx6K{i0FdK@qrH!7>5N+UR>d<(hBuWo)M;48gfYj1P!AHKHvx2r zpeb1aoTk@{@9qz-+?fcN5XIxBoYawYq)0Ey?As127kAeRwJJnCpZcOp*P|5QOi;WZ zecKSm34mKNP0V<0wl#e$Fp30mair#V$N*+2l47$lO5!;~FkeV3T+;IaaW2qUBurzr z;_)3P4cF})-mCl^J$-TBgY?tjl(oKtZz3Wv)mro;^l6k>mX`&M#U5jy#r?;paSfSX zX*470;EQtXiZ0pI&!mg+&I}Z2r73c2dn%_glK~u)dkVU1=-yL6$zGtwWrlx70YO$i zhNj8jDjiCAXkNrM7=5+Jfm6K~Ip`OD7eX$hz4-}x3E^an$9|y&2co<5_FX~d4_nUK zH8ta(*hm{z(VaSH-v>IWO=N|Z0!)nBa5>9p$#sV`QrIzbRWUe44N%Dh>b!oCMZjP}#|?95Sa_LsP$c=r`PWuLaE~ zoK!w5y{FtYIhjwr-W^xUI*;F!JiFRLBdW0WBv8zj_2~(S>3gq+BcyiOz-{d0gLos8 zhYPeL$X0HsJB$px+>BrQU?ddGBTfyGWCaAk@kD zU)_&R2UQQ!>>TPk+?ce$D8PzfckIC0)F+j))YBB|SHFYLUhvfMf6de{Ody85jRy-j z!G@yO^2gs2x9@`esvtmI$bCc)OTGQBxI#`UYEHg6hP#oP|EUBR@k#j;J9ydbhxLi+ zLZ8YjXXNlRmzsV=%tGz`!<#SS?gEYG$R)3nDnES}$ym_wGR*Mh^nP;?0bP@a`TZ__ z$3Z*{vLBatm#<0L5^MN&bG^aP>>zCg0LAuZK_0(9YT~t8X)%v%GtWh|Mdr{Jn%xDE zSE=Bt)S)a1hL5s*H4dU#6in8!dLKAAZS4$cL${k6u=SMrpsCmB7D9qDRdpE8!O`E? zsWO@ogoL{Fvr2gp2QP&GBtTll2}4LU_S}oBqf-PuQB4`QPvCG3zYa4Sv5{XA;D0Gc zq~U?O*|@0f8l}h=p;&UIo2Y$fqy)>scsVj1NE9DZ%D2+wJj6bQ)I=H)7h1`kA2|%j z^VHL?O)k{dt03%&`Q*$3N5f#mKDSPcwWh;qE z2Y#T+6Et)YHa6XYz^PxeRze5uIE(gjwRL@bjG3Prf6g|RCWHVspI75sKu^if^Wp4Z zv*i{-hxY*#xyNP=mw5s?Dx+jX;;>t-<2X{{n_!X|4?VmWSgXc)I>^^)h?K*M_mnc9 zLam)$aJT74I!6$OwZ|Vm**9r-Ozz#-pXtzjYmKsj+uvCvFIsrTiOvSzuoa)m1ku%5 z#SD71ksrMK>^zt2?c#W&4tT4}q=T1lwVKI3drS$Vv@+v-i1~ z(lF>_-n?a2H%yXWNe-=`*|NJ@&r>PNd(!)SXWHr>BL*_Ba@TE_H37tv7V^H2;#616 z{GO|JR&(#AvSp5F0`ikeSWTg!6K*5KPsMkZGaPf%C`IhG)Jbg7`Nd_c+!E68GIrt2 z+*PZ+jAlAumyH7@T84y#88jyv~B2b|c4n{G^ovo&Z>Y9_p2i0ph6;n>xMnVKuJ5PC)Cj2;k zTboSaBo;6$<5WM^s8o+xENIwi8Kf+dgDB85#vrvk-u70GrHESdi2rG28N3TaJ8F>E z-dyld!<>_sh<6EPYefSIhyx#$4HvB_Pc#31Lga99nISf}HW2_1EZZ3G7OYKj_z46U zY3!IERm`eE;>eLqWIL0I^}}wHeis^--RiM?-2gjCL@VQJZ#GFS{v}-dq(gboX3QJd zgWUoC@alCgxkIl{ZjPzJG|p9yCiABPvFma0E3P~0z)o#QH3vD*&`?JhvrbNY23u*< z(Obc*VaUXdQQ3EXpt_MDM7DSTfMMOu!Aw{y za77({UA9&Z5Y0eMr+O>rsIU67rdTbg3oizGE3-(y&@0eu6wW_SjFRhvB3Gq@GZIF4UIZgrqN0XIA$KM%JUu)9%4!XS6 z)*oHY{J^FaE31s;z`81EY#sO-oACf20qE+3AC+tQZDo;Dr0E1IYYN$=|0-92S&Ife z>qvfEp${WLfzCLo-?4g#KjaRYv@ccyYD>aqwT*ZK&l)XOb=SD4%H=bgly|AP2=fU> zta%CRG(IZ;)o67SN5KiL<~|)?7p?K~?vRi+ zL8GXfUB6Q*_GS41(@6`r&fbG}xk$CJ7#a0KBY1`GTu4!Tvl3MUPcCl18z%X1O?u`` z9fkuv=i-{oIoNgx*?R{Pa1EiauS>JIWN<2Fsqh5>QvwtrX1+JD#8n5+ZHp)~$&4}2 zt>5a_b@`LRq!>QKK6vnUX5;C3Kh-}Z_RO8>xQRl)g-V$w{$LX8J#b9p426>i;us4J zwI{N$cH%#dLvKD$<6-BykUZnQK?Q;*2wkd&YMgX}aZ2SO8CSJIRY6lvFn+8U3gT{H zT7<3#cy1xXy~h^oQ%&i>L)i|3!<=RAP(lkfg3*xNwOFhu+hMT*CM9Fs?_^3Ww|v91 z`Lr}#qsxFU+4@SPU>ytqM7Er%nS1*MrC@ql56swLmP~h#p61+*Qe%^WQS|0>qA88q zm$RHAw`_rK{@X!XbB~YaeVW6R)vk|NmFB}xfGZe5wN|^`^6-mp@4}6e3ue*#dqV=L zlK6JQlpq-*s6KyYEGub5V?CJ;vyaLmB+^M!lK63;S+z)ipx_(kG!rAmn77Ed^pp1%1Wpsj$#UH6r<8UMpvS6;~ue<6hF$ z%(b!0dzszim6dePf<+EQNiLy}Vz8sK6(H*%TS2S+kOrihiIq3oF4Ut9|0X)+ZVaik zO3MuchNA>uNpS`fz}2g=_71Iz9B%g_MCo zJ=tT^s!FDdv|fg=cN{YLmkj1|QTeji{xu3t4TrHdwqV}xU`+D!ByTW6ofP7s3jlq2 zT&Hh7?tYdjW%|Obb(T7Z-yx&!K`ouzfD8pN8;P0>RURr<4bl^y32Ja{ z8nGO6h#a^IDNJ7n+l`vW5K<1m$6oLhO-a z$7@ejm2QG~`skTLH)Lu*rBz=zzZA|t=q+Yz@6#(%p_)(}XofR6_+z}%pIuAovU7S~ zBN@*QE%>7mRJLls#RtW%Z+H^rBco{TjF+WvG9{soT^q9@DnASLt;lab3-d^tbQxXPcaKS4oLlsHf!)DO7V)VQC+t7`Aq7e_52{pg8yJRXcSt;wDZUWm?TYM%0_qA6gQf?3 zkk_;)zO@MNOy~A@o#6o)`ZsYMQjErp@(4kPQLN*# zO$dv5Pjh1SI4~0tezFQZ{4sa0o-M&=9H+ysaY+R8^d*l}4aMkl%d!LP`JdNSi z89{6ATm9IfCnH_q&9_5mY==)J8-!m^9gH)z*9@=E60Wsl1>JoCo4DNcLX~1!$kUln zGne+z60;1CQ>uAOr<)z;89ejod=eG!ayp&!Yi~k?djmpk$v#AN3gHb9Tl`eSex)Vb z&Wr|=MiQV{aIZy-{S(Fx`kNaQPV!ewsf-dP1ZD)p8`7oO&OqJOSu=|ra|9j33us$7 z(Ud@b2=rq;G69(|fZz`=SzPlec{K*vnuP#qDU!r8%mbCe?(d+-lhAS6lTM?cT?B3d z=QJw{>ig`7F4-OHmH|g{%Max}^;!=|iNhEt6K=RooT*GJls_w?Vq`s`X!X3+K@`^5 z7FpWGDMxp74Y)KzT2;tA*W4y5C1PE>gcw`31PiizPJY@=0Nt8&?1|-Ic#Y2M?NXmYanRJHb)Eq23L2!O3 z_Bzaoy-_rs48kBEM2r1=JH(b~K|G)b+LMdCC0zd$ku2e}PsM(lEau^w1r0@gJ}y{Y zuo506!<0{U2|)bKZ{IZ5I2*<+dRihFinvWR zpmf}@NC4%pFFum2s5gVGZf`LTB;yUeUiEstia%(oZXLX;Bjjm)>p4>Ygi*}!ak_$P z0C%QO6zUb)6_cT!*-VP)>ESCk68cfncF|fJsJupPC9xuytYar}kuPdd8S7pba2Ubk zRK3N<+kkV4v(M|-g;#>~;TN|lJ?kIKRZ|Say1pJ4z0w=@DJMF5(oyVx)POf#C}H_d z0DSB;EQ^ou%I~75tU&pGpInP^g+ACwK6ud<;*8yB?1PHs&l6e`HM)}eQGDd76Y+MU zDN0Ekz-pQ~`Pjk^#T?>|R|Ef`!7G#vG94uN5&=lzyW-)GUd#l?JRb*}V}S6C4>b>i zFmm;S_V>rGpOKJPS)dHy_7qN}ufRCr5I8_LmEzl(L4X#r1RWW zt7{!i|GeuIWsT?bmla@p!wCo`?N@SjG}Sc_bpPrd#l6!*uu(kWgW8QsN0<#~m(bZ2 z1Ox;N5~}4z>4?k>wBJ^8RbnyAoGV6?@h)B2_p0V~JawJJ5e3uGxydSlD6$HM@4v=> z(jdtdbr7W95aNxB3mwYT2a!pO$$Y*BC2gjB!rxfEj|auY9qR4wCJ#s;@dMaE2GNyY zpXU6Ce*gr0Vm_lBwXhH8t=$5I_aG5T?*R?5ZjNlhAmiKB3(iY}FM|8WqH7&N@6*0r zAQms7vAL^A9Foj{P#<8UK4_4jBn)+ghfIOk6&@=Xbc{m*Sgr46%r~B#_&`g5c7;$7 zLsagq`jfl~@To1uv^jKmcboPX|k_-Jo(7yDQMEyAV&whik!EvRE5I98Zk^+?xB2(x8f` zolvJip;W9<*$2?{J_ytP2DHj;`vyer_3i1c^lhOVKj;I^Cz6=nTJKi~laDet+IKeD zMA#2FmLq_L;pnG%&9pLr)9Xrb0I2enU;B?8lFh)0pd?VqTji%r@jbxP&IeJ?ZxeH% z`XYtLjx9^jPUveCG4>S+NDDcXzBq3c)9OXo&It;+dx|4b(bM&FP_q)N9&_}Ao~tn; zN#5?Lt^a8iw+d*uUkKgU60B$$;18mkECB;uiZ`R92pD}8qgK2xu8+2=Qf_jhTDCH{#079kVy9?=uAUva_O#JiDnNYyi+f3#pUN{{7*jD;)72c z>2up;-mnkFKPkBzjIQ4|T6!H-*r-koZgd?4o&c41F6L{bXRs6jy5HWNEgz;I*?{e4 z+sRPui|@dz!$k?17Vr)Wrmd7+7QPQ|$!J=uHf}s{npG_cE95X&G}-R6bY2%8c~Y$# z(Z3ya`Ep8K>~*n5cq_RMNbU-N#Vl_@eqvaD%WDs%JI04xr>L5$fg#oR zCAljp>f@fLImi!tcdU~?)5yhPb9hdVWF|@SMmY*I@HNzU02pAT$T|xOI`sQuBbV>7 zN5?<8UP2P{Z8pPvsBl{y(%rBS0oleZ5Z0~^o}$!pgB@5NVy7HlP4jB1zy37mGP^9_ zimKE`4}80;GiK%uE$`wkDyD;DkHd1^jD3#rLs03jlBS6G@e{)$Co?AC7a?q(I2+7;;Q>97hMD9>m&v!KdP5~m8k}-4PsMwa{N|N4(V%6PJi}Jn>Fx@FF#>^ zzmcufecJAUl8$AE^jNf(l?HF?IzWldDjIBIjxm4To(CFK3-k+Fh4*B4(Yk` z@xqp1uP)R8WyqlpI`Fo@yo#{=?d)n2buS)F80f`OIxkt!jQMRYyt|1v(}6+-DOai4 zs-M~{fER^f7bLT{($1Z8^~wucuNPYfD*UOCC{}Npj;Y1PQ02A0C+W@#h{cXKV^7KZ z-MePbUI47cgEA_amQlx|Pahyp=zU$}Zn!$v2()z`$9+Awm5Cn^th%9O?di4!?iB7R z>r>qsMpFo2WnsJG1$91aP{j!8-O5?lTtq1cWa!-M`eSO>Dd%7^P);{R^q_b$$b7q} zE*`m4;6xC9TU`DVN4OL^)!r{Ti$~oieifb^h8Yldrv2UhLreZziuXoJBr-zSJx987 z*cTMEF2o(=%VQg_iWQpp$bBto)Bb@#;Rr;wgX5sC8iWxTY=(d=8E@dS_L2Y0wQ6|3Ui@Xs6VFt zT^n`v5;r<;^iEV?gpl;y*k#zn>i7-Yv#`=lC(pMQGSa!CcdSS!p-+e6EAG$4xv}>8 z84+#Wdg{bnS!mjzHw)@Swd={YMl}*uo5!Md+2e&b8EV#asG9`G;P6jclq)DaL_>%G zq&e%iCbxRuEJ}|4)V3AxVKOrTGGZN`~wnK3`2 z$hDCiJB%1rwN~A&tI6!U=|{lCc(?t7iUkt-{W?jaTBfd?d)oxt{{tsL*uP8!@k$M| zCaSMll~4?I^$y}op7!_qsucWf+c>uS=$rVb5J`U_483jVgDu?lB5du*scYHnz>gEo zxXe5`IkwMwuO#L=(8;i0|K8jHPv*Z$0pc&Qy(Uo)rf@eFc}{N5E_fiD_lftAcZ&y7 zTw~Vlnlyf$^z)Zrr64Le+G`J*6>;)*1>I!$NN>vZoNm0FJ8BAP2Iypi7#T)V9gJ|D zj5B|5(0$z-iHCpj2GWTO9Q=w|ss<6-1f6oWteibvpT<#QS4}EIVAq_!4ScGI%AD<- zw8wOAQ3^dV;y$gmFWu7hm~Z*5bA^BYhtpO84CK*$)ZQv&@D*5Gpc5`8^X)nbNP!FH zAy2tPdvDA5t7*@4M_hN+$|p4eFA%J2&%}Q!aSH2gt{mO9f`&Zp-rrhlDg=LiNlMA$0sntfDHpH90PylqZOJ*Z@37Q z?o!2tn^6%8))`rdaywT1sNDqfRJ!;s!mbt1r!RV<)9H;TuR>m1K3GVsC0*pF^+A81 z{^ESkj)F^~H=KWBLR~jc8q0o5HJE5th>bP?UN~JA^Kf&Nr_^Ldti76Tubp3Cb|KLF zH{B?XJ)qg?Si72VzUV^InD|e>&~`y&4b?+)Pfvb2g;6&lwC!eOKZ#T|$3r|Y(cf4K z(x8QWTw1Kg$ze_wfksf>NpCsSvajl~!KhX_;0aQ%Gu zqwi4puEL-My5mcrs@WQ#u%I#3iS{4hO)W)l4AC+XUkwMCMK;7~Gn2HSJCC>E3^j}_ z5k8@^j>5F%;B!_ZZUiYiwfpRJtV1~Tzb}KLuEc-wO!8F6fA#ZNX)Pp)72#;@3MR=Qs$h^rzl_sE<@ct$Nys|W7<3P9-OU%Fo=a#Q z1V}d+_`-K^j1c%JM6GKgDulph`S<#Gr^8N?#r~Q^;QIk(Pjksbx9}wa6Bkc$)>!76 z;n0>*#=}o?Pdg)@?qGeGeOrHD&dtl3g6DPqWss-9^Wm1`d2-ErSbZD-(0>JK_kj__lh0nU)el% z&+CxT_=eby_}h-g2sFRln0UchCNHl8j|pR$PBbHK1^0i*9$)&P-_j&{UF^eI;WX*n zAMI27(_@8fm!I$2SLlC_va^FrM99u3C{MHmn@}0neYs=$90JVTw|0Bp2;snsD_33i z6%Q={JbGl91>js?arjfXDIlhwumm5pIX)B;ZFOMei@vqw?&EDJ@c8@mdsi)dy~AX} zEmeFtC8|Q5TO~#1b<1y?b5~0di004yv~Kow((`{qStJ9#&5?hhJdU6Url!2+N?mh& zf7?O04qxaS%evw;#GrY;pPbDTe+>?~?Y>%X%2h^jqZ~Wt?Q+50VxlghUG7GsC_s#_ z5z59wm+q{=f%F`^ao~B8tcEgjPo>AR6{wgGJl#sH?B*LnL3P0%Yo{_fEG;m+*J!VP zE924x1i}VZc!_^LK(?obyPaHp-Nl%yq@2$o#V)uco}Aam`Qqz;o9mh^vJ|cIy`6?P1%-ccJ}bx{NJLS_0{;8VAx<*yX^3sfk!+!fkE|U z`a+13k?DVj15oTw)T7Aho=+u{;ffu+7C26j-ZyMr5C#4hH@qcIE#zwjQ13kVq{@A$ ztL6^0>XN5$7GK|*YHB*TefPV)f*}Un6?E9;iRFS=jp2WcR!kEN(;k1!(NEbV5RIhh z-(dN+cyPI-Qfw;@$7(3wFIGhTyHM1SGk)ozY=M8vFoWCSvcDufE(WWvWR+?k%}Uwx zE~ZhVwj*j$Il zv>lz@0rQ>5lNjBm3x|2@<;%SoGj+6*GPzwAi7DJr7i1}3OgFx!mivz>)Tx+wZf^wy zy3l_II1-mbRPs4!W3+l21m@7hEKnQ$5TY9GFmZOa_|lUbsO5#c=c_OlgF?_oC&Y%& zq&a8hnD1o31uTt9&VI~v5u~s19U#0j?4p<7sM8+xHf75+Nc>AoPs~<~86;BdSwU2Q+>8NKgGu`axJ@xmACyOq<}fAh9rgB}CBWvz|mLj}`k3?GDkQ zJe)|Ad3)J|f1d1wwI8#qpE7DE#0wHaCawjFjeY0i=g!4X zXzi!Y-gxnq-8$hb+eY{vzw_Umz3;-YD8w%NGjeGB59_@S1P`>(J$}2h7PeEjgb{zE z?@I5PIWW2QKngL3PrQuMPEf7tdA9gVS^ZA<0sj^nq#4Wat#D=G&NJ4jq+BB(x!o~@ z(iqC~BU{=V6C9!aDl6GrAfnCY_esA%tpKVK?Lw>dh}C)$S2oD6<@-UvbxPy6&Awdh zset!dOELr3iw(1D>sJUm|maWC)qJ16($ z)pog}>M)X(n;tpr?#^ciX&%MZ%L7rR%J<_neksU2N(2m}It>}C-tLSlyb%^*w%1R}lJ1Pw{>UVO zOEj9B|MIuaC&-^x8&Wsy0LE9N=faESl}7zXyOo+Dqv=3K@^&gsb<2NeiMDRT6G30? z_UNfZ#xfV)+-1l~^Qz#ro^KC&mQr&mv2d^bKgn9zBR~bJ*g`9zSIeW_vhhbpzB>Lq zw^ZH)GEbFrD}$SwbjxikkW7EvVN*So-w--O-Y15We-jNBYH{ym@%Osb59J-bWNxPt z{p~=<@Q?$=)Hf3#6uW<`Cig=(v%kPe2mNM*O-xZltIlnuL3w=>!IvW&sfI#R%Hc}% z>O}+FBVZ3ER<|$gFf)<~!ubih9`MN?mfDM_~oD+uOeSiqUWM( zC}W1{Ewo-1z5*BXUJeIyCV|t-tU}IwZD@%;{*-K${4<25tv7#NF0{1tqHwdcs8F1# z{(HeaeN7&oFM1_DN+L85xy^OkD*iOBt`%OWkrUTAD`(Tni8(=3G}-QnZbo2tCIk`u zOp~1*P4}$;o)SF<%-1z2r(5#ZJsR5wUk(2~mMJOYWrZ5L3 zX3w@(zo*>9+0TC`ac}XSxveQpB72v~9K?eWXz69#d8!yH_r?gr$>DwLIQLM>bvrZh z7%+Sc(D`vwVm#1FeyJM;e|IZr3GChbOb>w@HG~UmTQAl&Njuem{RyT)q`C}Y?`#mw zK8fm^KBT&Hl*I#rCBV=f8ISVIu(KslXc7ljKAGv4BsOW&4Wcqn{Y;TqM$XeYPcJ3E5IDl)4TOnGHT!Hm) zr1lW^f~PAzG&Hz6>>jFHb#Lhlq)(VT0VV}oE1rKhNO_?m1$eK07O4C@%z~xRQP(T3 zlTW+|Ela+)2QkIRl^6nIXU-29D^;<75b)g{nMV zEC_$5n)|T=om;kmUaoDwgliASy~CgOmSO^V$7_>PgG5DoQ}C7gqwHJ4U-}>~qvFL- zzRgLMUY0!?3=ITn$R-&@yX!{u7HTknL)RO(;gu^YwI%EgS;&9wP+9{WIfkO|(rpv6_t|(4$g55^R>Fh# z3gNULD?+SYq1y9sfai*e;nz#{m;w@0tZzE>_rV=SEw=Q?>>D|$!H_Pj7nnqNpo!|@ z{#HD0KVEJ7`g0ad@5&+*`WGnAhJ*nIkj04R1&8Ev^+HKYpZez*Rzyq}%3^jQ2&I4X zB8aaaQmv;TxCGDd(`YwkY%LZTX&PU6-M2Q%AbZRlA!8`k z04x9fkcKznFoEy-9!kfo&U5=jX`vPHJf$84W{tl8IZAXhNNk(NF8Q zW_DU-Y4O-(AU#E;ewxP=*zxe+2}+U!=18wvlcF(jXjpb)qmIn{XW( zXFkfv3>*w|@|&Ozcpz#cemS0!F03FsrQc6Z;iu0A;!Mx@C`+j#fNe_3I#gil9rrbo|X%yNQa z_ZzR~Dz6#&mNY498u-HLp-60XNTKxn&rIFtBIbrsHDPY;9oZi!1V}QI zBn8a&H2J>Desez93K<}Pm{JLN7Az#nf#;{F_j$l8a~Zl`*Vy`Yt9}>$T&4-x(tn~c zZOxFICnTIYWGenhH?GTrx_}SOpIYA6!=57udV?!nH1=abzxo^};f~Fmph8liFF)bV zz}&b+m&u)p;J2^V74Uz<)oPRMiT+_|$Zh}e+J?kLzQ;+3E_W)A>E!ydOz^EKu}mRz zY;CusngIip*T6+c4B$X9fLoPQ@dqX4C=}53+fSR_6!(2ydf8~oJAoeiH0686caQhA zMEXKohhWFjW8Hbvg@?^O|58?o@2^nK581EY(O0d(rd7~c+X;WyVi@}old_00waKA! z4QDYSjol-GU55FYF<-rmR(k9%Rn5NQGH(b-{S-6XT9|o;gzX;g@iI{pCaGEb=C=-B zxWAsm`Q010>A+i)`MwZ|7o+QlOZLF5D4=}YJY2z@BcB#<#^47tmQ50pJg`5Fb6HH| z$8p{QTby;m-{pTz^aDNH2?trmmN;p@^zc-KKR^6x<1IYc@xHf2466hF@p_T@h7bWA z#88zoHk#`GI=c8A79Lb6YDTy|<)Ff>PR7MF|7}5$1VJTv>UspfpCFUYb>a9E7%QWq zo*Y4_u$^oZj&tIwf7KV1;vLoOv%o&yb+g!=$IBY&#B_ge>%v9{**DzNSlSK%Y!I|P z2$CM$bXRR$c$(n>%;FMmhe*SRKUE`Q7SIUs`o1E%YKObG9p4!Sjh>nv&K!|E+7Rzr zXvo$bz2a)MVIt<7!e7Ir=L;Vn6_4FcVSXD3yP!jZj6X-Sar#L{_QK5UL0^ZQ4 z3^T+fQ{Q@}|0E%tdS2q$Z|8EAGBaV)0 zzD0jT4!hvG0L90MzgJ4UngQ;NeT@rRip>utdaS|75XUH<`WOoAgCba(iAO$+5!|y&GApMn#iO9P(RaHEj*GpGQ(PWBY|jmd}4u zi@!0Cr84lQ(%UBl-Oxtk2_W#TTR}XLyo_eb065n}ME# z*d{%_?PBZAO9_2VH}7YBeTPL9&YyJ9I!!XGpA+RvDuRi#^_7jY(8v+95fTP(4TCYz z9&XqxV<= zGlLHKV!v2nQ*|1=LLhbPZw7x8anP(^8a_d!DU1Gk^fA|4^)!cM|FsLHqyFybCLg} z6FWUwi#D9&OsP94$D~o)>seo}Cf>TW?fg7h>T==D0oXs=Q{@QgLiX6(e_DC z+}BDo$HpFpTIjKijS4>MS+;xWcL+( zlu1lv#jSrg#9y6Xbhoi-24K#%T9*A!G0Laj2piy4P&eqSQj(0?Fo=)o=DrcA?-)_{ z>*(zSv`)6f`-wZ`K*Vw@tby<3MJAM$Gbv3Kl@~Q!e3rqo5=9z&3_Hy9Xupz=FSk8i z5WPXW_>ZtEMswZZK49_mpM0k7)Y}FzBYBr+jah%k>;OqWSAWB+<5fhJh~DgK=2xg+ zsOF-A-@YL>exmaxRxUvhV7EbeI(@h?2R|uGW)E z@ylnty6v`RRr|Uu#erf5R`|h7eV2RQq$d=>z*ldk;z+CwYv}7!G{>1!?3O~6h|)uK z<0XH9LFAKZk>ZI}mphmFaGLh0cKP|`<$lbC?J$MKmpd~L1d&2#90eGcyDx&KSXEh# z@CxG&a22GqSIE}Ent3$(CBs*7Vi7Ia-UevuDQ)}FCJvGXEycOvn-9g;{^~wrs`OO6tTDK%8B`UnZc2@)X6q{w$lv zoyIskEM1^OxK?e^h40NQmw-juHfIP0DOLF<~Bv zQnXkW0y~4(D~HW#8Gw7M#6Z$yMP<0K__eP~XPFQ#cEa;F6+ag=S!Qu%@>yxPF+YEI zjP1XNbtT@K?yj*mjD;h?VZru ztgKNW!v-mfC5~~$RXOT0a7w5SI_ZBnh{)pY$}1SL272#i7I-N7y`|7GCTY>&ogB#t zPKsE>6U=^jRr=gssTyL(i6AC1PKg$QbBE$1Iqq3#aJR`u5P z`E*RPsToK!TBqYH=R>8uo_2&o233$5BNd~zHJJl`ek8O`u-HV?_X|dygYbWyH&4_X zN-MeDF*;W0xA|4|)&t|xlNav2c(47Ie3!9}qKV`zalShuffpk* zt-Ob%e@8VNg~0Jh`0U;gxm5qGIxp27R7$j!h1QSQoG*b~wVCGR(ABvyS68k-uL$(z zyByewQR|3P6DO-wFuF%YF@t|L_qrqZwzci2Q3u9v3GC+xYuv>iV?|6psS$tat$)aV zjb;NDP5R9C*Ky-t_$=~!Qy4#)e23$p~z`a(?w&5-WG5wJgAof5q8{%n6VCy>k6yauRV zViZz)tfm{kF4smXzVS~ifLdhq;Xry>3n%P+^mf@&{+NReG7afm=BXmN*#DkV#%e5Y&)xL6%^dz>pO1cP+3sX?w@7{9 zN{Z2iuBICf7StQ{N zlhsXJs&W3QENGr=N4(6_0H6^ptgI{RqlH4~*Qe=5fA2JLz7B~tonixw?(Kz;hn? zEwI26R)y($(k+_R&xaq1R zHtI`h`A~S zz8Kmr5rn6>ON@dVbKGkeVEjn-uDVy**QCQum!tHC(lH9DjisWy&-?}jg)K7%p436q z15fJvtc4DGCEfUd;9Yd8v)}J#oQ4~^M%%Dc@r^H+-hqGGJcS|8#@XPl2YwZ}?47j@ zIxW;LZ`o0jaMT!0;dp=)08(sl zSURfaap9u7zlz45h@tGc4_WHWpa>z3$B0^p7{`xrG8(239xe3)<*N>HMlNpJjg}(& zf_NzY0Y-mX{!wP3{s1kE+|+QqoKkhZJ!rP>APSK+(gq!|f^Gh;k6w zb#<55PD%|KjI|osmkQ}SPb587?zLhH{D>SOj=dfy4_`@#`hHnHREPd{yL;hww{W6b zCYe^9YL=k^(Y!Pc3N-hKZb}lHP+0$18Tjbo2 z+>^|${B_W&ZC8Az@`ACV2m$--rD+h4Dy)bKppbnz4)AY`@0^$;Y>CyHWSw^HT%vjD z?<+_aK*J{j$Wp!qsyD}TCyH#cz#<(|scgYgLPiu41#nber(5$6noO>wyVZlL*(NzX zobZ2YE8lMlu~Ws);m2bt>zv;rH6ZjU_2C4zBDM4+wRnh=M(vU!zsMj~ zX#SMk$%K2YyaLMIJkoYrm!}IAEu}y}nG_Sxa^m=Uzxqp9M8y{?(Fko|TLf1_T0COr z71X~8G_u3U&owKo*d!wKiudG@QqKDj1F?T99uJ`!f^3d60Fo6_(Q;}j)D0c4*P%z( z&vy2q5zJK@oN?sd#8!VaVO8tXHoCmAZ3y+B8FQ9mN*hQnN)&q0{aobma>X@%yaK5V z2pKeXcr;D)?X`3_Uhq#m0P>?DsQ3_n2~9+bcyNN4fm7!m`QLbXBr3>B7A`y=$7X-a zkjn|d0~6gSrtPX7lot`!aA@;P031{e#;F`oAYpmofLsza%9g+V*P}}8bCu)n;|}WW zXAS9U_XO>Vg+ewwu7K2iNXEBul~R#yCskK;z`-D`DtaruO55)%lu4g3e6;RLUx&D` zfJ>~@mo<`&y7x;e;Cg4-hSc3aIy`@iLseryn{T&5{sI|9zw(JU4rZe#4!Zh1Tb^-Q z&0kkvHzdtLyruy+gPhN|1xCU;79~1<*etgyU7J5@-oRzo9iLOCb^;Qn3|vc*4?O+S zG4nkk?|#-ZDcp?`%Neenu1qZQ`$rn8{#E7|ETD4+{6VgA$;hDi19Im0vK)WxS5B60 zt?K`GJMF~M4;)%aW3#at-z8+v>d$L9KHbPFPuMjSBnnsDWb8%43cj zB2LXs)$5h?PC|HunXRHh#eRJK`s%EEROCDcEC>xhKtH~_4laNAo8YA742`%ZyUeFcnpc(Aump%t-HFyE5WVymeUPJC(v=0p^! zjZs?{k{$@DV^j0-H>+#-;My0}G0DWTgc>kPwA9Fi;+{l3hWWTu%Fl5P%IuElXyBs{))*1X)a zE?_uHL2D*8)YrO+Q?pj{4s-%b~_826}n#{7#}si~=Y<2slIyJvHuJLu%Ew}I$eB(zoR zL4&$cX6d>|vVska5!57CkcdJEM;icTmPB;6Aa3LB5LvcBX^JHNesltKDQ)%JWTWR> z^dT}Z@y$yGL?aTCLS(4FNqhRmG=3(j*Cz8(a=7b;EYr`qa2kK@L!F3H5g@{7XY`#0 zz<0|8=EzbHo#D_fy@t-~sPzt=!sZjic^roT{1EYj4W^sP@ahPihgjFK@XF7Bp>(N` z*=h{fDcyvb1G__J|8#Pt7JCJS#pjzXR||X^n&ns_A!*!TNvJ9(4M(PPx70igcXVRW}U!;A!{gG=Lb;7WV|&LIS`9iCx^M^>(-7 zZuq_OMR~nWhNSP&=Urych_&;}O$s`4!vPS953JFY&B-KFkOH^SO&~l>EK;3ZTV!ks zU=D};TU~Ag@iuf%F!dX}I568n(_q%vwWJRvIQy4T9>9O%SGWbQuaeiri`qm#S-mBQ z)I9$s(Dh!(>`E^k1w|4N- z(w{xhAUcE0kmt8J_H1{q*m9U|HIiUH(r24C$>Cx4$Y#cKgh&2?_o{SOeGr!`(AY=* zTL6J#gdL3nA(O9--X17Dr4V3-sqc{sdTwYw>!>diM}#aDvR4Chb8sW>>-4rJ2!nwU zyCQ%35Je-XHHdp)rtLqX0oN`AuXM=LjIeaKDV+?x4!F_p<>}b%N;N_@kKahc>vFCx z3EI<-y_T?ren41Q-*e02w;7Psl(tIm>8Wi@L!?z`(T!aCGOK0Pg_IwoDcwDo#2 zwwxqVBeqwxd>mD;f?Sal19#r;>Y2@Ay5|bE>|zvA9}yaU;GGJIwKJm5s#$%Jdfrap z?vXQ}J>W?D{Sbo-pEhP!&15{8bBx7Vg2H$cphb8#^!ZQod_q?(P4^dFRpE+4sIh-E z10&}C<$d=ujf>UXQoP-TQXy&blH%=0&+0)+!f5;u{vx50i4ZImT3a1Qv`YYR4sGKz zB8N)>LUs7DqMS+x><@D5$%FxkDa%J{;#IXVYTwUgZ0kO`CFxo|PJetu`->dDgiNa_&PlRv7f$Fc58q` zNhD0J))8Fq`mOyFbik+bPW02tdKj)p+(2gaj#}BnEt~BXnK-Oqx7NB))fEYx$UE;<&~Q#n0l(E@`_`O>$U^Re^{(tcimD^Pc$ji4^{6Nn&(U$;+Q32=DW9RcI&q-!AB37Uh|u30pBW5oNqrW zL9hXNa%bW+oxYE#Twd0Vw2t^r#GPB`V{}3r9N+ z!h$auk=&5>oMjk<%wM|bx~j(o;x$RmL%F;Rf-AOyc*<|~y~XlFjwyexh*MpU*N9ANF4a}EjVV-mL817}npX8|Qt>&?vH zNftZ9k4RsMidhqdQ}ln)F%gf{o=|=)1c?9Xbo{G*OR{xj;QKDi{ky`$s#$#Uhd?s* z5kTJeMHM6GQmeqU>bS^0q;~QW_%1iViK6u)=Ie1~JdK>4Jgz<*rU{m_qoY#A=_+5) zQPLN`lA+=MR*~uM`-EtEg#La~ZTEww`f9%z#?slSete3danyfibZIunSZ-D@eW)1{ zn~cqGZU40YceGNejB4j>#ATUf1OF{crwRBt85NVW+}hDbR=d31PEnT6)^)=St(nTp zz(Je*P#~)WLyg#0o)9Pd*WeLH2V%p1fB9LNkl)159~xG_JS>)>bmry`%AciF-(iZZ zke;$uxy5jZV$6Ts_`5!hE~{-m?lU@$;T;!9pbaATeM&h8Z z`Gg&RGP`bFm*n}krosFsFr9A(uivxu5Xu{)Nc@%Iju-o4vX)0;+Tjk{L%u?8&POV^& zFwJ&;S-pQ#5vLRlCLj zrJ6Ux;V_YXa;PUcT7fMQ1r~eQ>v)`u)t}hM3D$q(4p@DKlf>NUoYBcNI(2+ln`@N~ z5ub2xQUui9qx9+p`EfdjdDY}gN2$Tvzet4Ku$!6zBMvA@*`rgXWiqtpuFKEnNEH<2 zvRZj5w-jouTNdf7AT?rBfsh@oR%nB8&D}nf?^-TaJc0*_OO5Tj5|?ICnjL8UBPH(chaqjh$GZ9JlF|i!TZUb5G&*!zu^} z8-oc)$@W+`a?h8T)8;RbeKIbal6|VWUT-BVX}>u=>`b*o+2@(UV7Iuv4L|PEjER3! z?Z6BBz4}@*cntLGzd zJ&fNt)rPD7>?4-7c)Mkiud*^_ zuez9t8hopk=PLNt!BjK&yrO><7WT)Q;n%4MRf8%R>D4us%eCu3HQ{-|e=DgM$gUz3 zV}jTiyepHr@;4Zr^G8t!uKmCeEMq3Yqrrtg&f{xDS0t1+q^LrH=!UNGfN*RdAp}76 ze(w0wn!2cN;5%(+aiOBBaQ07iqdOx zhR1+~JQ#U5`(tz9szHBLq3XmV{1`2g66yM*!JeK`F(f$e)k7WZw8>*5z%G~XsBsb_ z#&6m<2wyu}7$+uh1-3TN(ZuGd9@x_S&Qo1nA0}lgJ%ccdgY-3^Fb?|9BdGV`%7>7U z!nZv3+UE;dT#p>wPA&f5QjNLEtnGPyr;5D!=By6Ig+!PeByoSi%kUv{6j6W3G6UkB zUQfK;AQP(_E+l8CQ|@(zC)<)Rk@f=VHC-&g<4LvsZ~=~?Rp`$Fw~XU@FXbG0xQNTR zbBz?P6PtlaRbK<~K&jUBh^To-p8*i@e!96mAqt*B2jzRvd)5vWw-Tg9B`7njW=WYm zSEz#ybD85^@koC?I{hVsGLX4Nqaf0Aj@VxQ`c(NrUSw;l*NH5{CU`j1%HA1NGgbJd z1~GwZ@>+jcv4(e3FY3~#oK)3gq-KK`@9c_XUP4Ldb;YBRoo|7^T)|+#w%U&jsuqy$ z|F^~iQNcq~y;6vzXuMmsR~D0E7li&(N4VC2^skMliVA;Qe%UL0_MDP^70wtMqVgZ@ z649?mm6aJJ^LA(`yqZq0eR)tyG8&QK;|#SDDnjP(0_5T+$svb$5ZGvKI@=ZHrX6L( zWaNPab;g4xtl``@M|}^>Kz1^^4D%<#n5HcTKM!G!^nB~^-(OQvE-)q!LL?Wtn%j;B z%R+0;MKymh^(vsfrO*82>XS06qL+(H;$YT-^7ZF=l%fbdWBlt9?5vK!R@#89(7}V7C<8 zIRG@J0k7l^6+6hdi^DW4_3${o+TlR4yqmHa*`4K0HxJT3a} z`e2l@ysEPd^;qQMW=5Jc!lY{9zPy!1VnKgPCZTi7$Q#_vY2;{;1!@)fTx47&D~X9) zB3n`D%{FI~A3_@I2tIFqco8Mr7&nNGP@|=G;6N@(u;aw(7EUaKeX}{863|wgvfq^eY?h{_1 zjflTl&u0dDIp+Z%gBA3WFQUmlreWyiu!SNE5R~`zc{=E52q&YWuk7>zl4a;cS(E&V zp=U!1qFb{`tdi8%@?9XyJ-<$_1~t&fcLfb1z>E*}c=$b@er5UUr9So}Tr_`9>u#ND z*FW7MWZ@ulnbU!=dLIBE6Ku(5GArPZbP<^Lx*sP>moKaMa>6|=z>TZvjAK+yumi_iu%EASQk2fm7w!DHwc;HG4b!$Fbj8Ez8;uIbMBC4-BxQfSBn|s&CYnM6A&D$_Y2#h7*Q?^sox%*VAwup>l{{T{ z-&@Le$E$mxlU8Uz>IjtV`zV!}an^bnA3=JQ{02J`c?##jcmUYjI=wx6u06@m-p_|vniGgfH$ zIusHd6n&Z5Ti=5Vw!C^oCLBtBssX>s5IejTS91(@h2LNx_XMN(=ZTdhs6Qke8Q^#G1h z}U}DHc}rNH-{!{xO?4ekv8{KL8rP4_HkxpW#u$BbDhAaKYhOk+)gr_&phwQm0f?9=6>EQRgr2 zYG3B{=A?5weba{koUUVPWpGke{_(Me8{*4iL63j$BFL$(^)ek3qWodTM}#{HP2ntj z4w(`NAJ$}@S_oG}qaRY(G@8BuFmcgo8_bxu*b1BzRNzZ#vNIE#KB%vP#yv0uLyB_- zNjwQCA7FzJpaO39rK1WKYeDTciwyjnEKebRk7Gr=$GjEElBpBNm#>qqQ?k6XI1cm! z9;<&PTPB3|J{^0N&&jdMZW=0Yx*8e?fw#>p7sbonic7euj5>cd%rv!bHjM&63zaZ9 z`>hqEn%t<_jJO2-YYR^$GqTIHp=@oDlg|%Qf4c!Mn;o)mv>>Ow!To?k&V?vVUH!ZC zTFdCnOe5bLQ4WLLG8`Qfi&NqrJzJFs@#cSUN}LI3ns6z5L{=*|znq{HAP>@8yZ%h} zzQ|z3Wxk0M1?to>TDxS+3$+dR^tD)f@paUl^`jPN7BPG}nm@!Pc9UhgIQjK%AhBFVI7d4;xH6i&{AIs$ z7-D6XrkQzQB`pGF*4qr4v#8L;%-b7=HjcUng zg;PafhTftZrn~-x@uPawR5qty=74{s?j*6fK*`%Vz<=E}TL=Lg6D`ko^1jMWJYQ;^ z^U|Isn}^E14>*N-woYI0#GAP>X7F~m1A)p->n~`~u8Ku1Im5CPs12?4GL;RWX4UTt zp)sD_4J>ALIdev-oj6~*fI5Tugt4ovVf$6@&3|LwhpT0)aO!#84#IPQZasg2DsZ9l zu`CQMo=pbmq)0j=jZS@;Pn>-^H<4k!E!jWYn_q}a=FTME&cNF)UpsU5bwNWOXUtZ% zR!?8wkEsV?trsvYTyT=iqmini+*f^SZ*({> zWqhh0$<8wNfu-Pv%7|D7_nCjkjIJ#OEZH7>Y+*p~5fI*rs#g&Fx*ic94ib;rRi2}J z)N|>!n_yPXBNWbfKd(-0-97Tfq#Et&+2nqVd~|9}REJJR;fDIb64=(NLf`0?P`YGR zItGtXUBlRw4xJf!r%J5v&|Bp3w;CjpL`Z{`(FIu?T-c-VwjhUdr0stL@-Kgab9kQs zGS@??u**)hzo?|h!x$(p6(7eV^LMeFvo5$j1U|}))*NU=3Ix)pEOB%M-&cw^XgYx=cE{+SiKCYnnjq`(&p|(FZ`FPkdrHGIxJ}#msdM4SRzuEaMMLtp>aB)XmC(xpoe(_Z5FR%yy_aq0aW$ePNsH z&z|kMjZ1XNLyHp$we6yI+X;Lr=@|v1a9LMW60g~c!+yPOHOd~DRo*ebgOak+9Z_GF z3@{qW^qSK5Bm@V812syLo>kI?h4mZ+|fg*ndpt>?y}tt17-xOF5$EA$0-TuI?NW z=*1V%K~aT7DysK@lpdx$ow`M1^R9+fOMS8#Ue>#c+8APv-GcqnOF5D{yTi(d(X^~Q zgm3*U^?>*LPvxoV57|c>9P*VeRfPru%0WK^q}62|nz?@}nE*vVy1(VbyS7TmkE_XW zMbB_@ZZA4VfiI;T1c+&T0+R?|xL^630Z}yRSL?p%^NP9>xE9GbcRj}dj(M}P*y@4S zpQI*Brr^9~Z80a(u{FwHE3E~*pTbl6H4ciA-=ZoRQtxWL$m#~yw@ri}_D@S4(*${2 zcV!EtiPYw{i7Lf^x;P&+3U$}v5f_Xe#C7F4LF%ASQtETgvzuZ^?!o$t>hD8skg7*6 zV<`(&i91MNm~Vmz&kKb#V%1c=8JWYF#gm_xeqNtPs0)7HudGYVtl2m6AtByV3qMct zP)+8TsyreLbvpqf>i9l5E-4FM^Z8QKdesv2fqJme8&nx` zLt^?5p|eWf@jInY8?x2zma3*6KlJNQ!ddzpMF-%MC0>$+lY-GES5K4O(g>NWlk}GQ& z4RpwGzCOu-=o7wkiSe{~d_rr4M%7!XQwz^-eK}&Z)K?9HFE)K}=8WnR`D%*A|>TnJ? zxH;{+xDCu*l_XUe^Xj#kvOUF8c|y-OCB?FI0hnh z>z9v*xs}s?zq$YQ8a}Kp5PT9L}^=?>pDw7MYHD4oHlq zRA>#ThdO~kjaRUmXkz00i^%(??%9)5EJU||?x%)>CNAna2bomRXWr)o6Nji#+m&g0 z$Rk$b@w&8*DknA zcISa>?F&+pYTj|C?d;2DxhzmIy=M@*9jD4y9|PM%f(bnPa&jjv`I`H-&b~MogSEK> zxFV^~pN)ts6MCep@H< zb@9_td%IpJps&y0nyj_VKgJxY;REM>hL=}$$L^YK|C^WLTHl<0OyGM571Yr}c==q# zqOi3thcsL2?Q)Tf)8O#}luWOG^fGXHNgv!Wo^mu{42R#pT?+Y~cibDMh!tfvC3Dk| zxqo%2uf4HozcX2c9diHh@qiQt*6rJG$E+r6zfFLa522Ue-CX-OU-YHC#N`nlRuu)&RO#E z`a|9Gnd}He5IJOj4t{>yJMx~to36fk33gxqX_TAK;e2hkE4B?wkkH zmV?%p+r`ZaLAv)4g1W5;icx*#c1y&U`BzTIzvdfYPc-^iQ|{oL4o7X;@FcLnK@{ee zsv--=tks9F199ae@MiIUE`Gs-um>>gCQlWeK2Da!=a9PT@t;BYE$Ew0!Ir@NBf>6& zX6WcnNr??pCl1ixBmF+a*4rBG9O3xLyZHvvqVEZkv>Xao8k~QCiKT@AX+$FGWFSF>9)8| z!WD-uCdI+kEYr%XOc(a#N;+0SIf{I`N~sKZE;O(z!jzG8E8yJQ#b9U-*g7Tm@NlSy z!yTBHu>oq$cd$IqeT`&lGu(?fKPu1s#Hk8ims?Y!yHs6&X9Mza+t1%nP!;s#avZA`-MDv+YX9uCmVfoT z>F9`uZe`)Mev7cLJ0Z+Fm(`3fqJKMoZ|=D;#M3!M2S)#phLa!_Y|tDh_x)G2}Pbg6uiz}J!;LlSL1M2j=~5&_ZuV6+RnPrudEWHaw&YmH`G*^ zWAME>#_%ix>gf2Cv6m^xiH?S9!`hgz6T=(W8%aQO>vwoNl($D+%s#-e>-TL(Yn@cY zWr8PvqTwOZ?sY?J>`AUy6|Som5_ zLo8MZPB;5~3>1c3>v$L!$;9J$%T^CB6#8ewyN*+twKv90`r{7h|PBIhTDA2Ee z*vXCnBdIk5gaB)}l;f5}_?=G>L);c(Mrj>jyElMj74^Ew>vPOxhMFdt0ETf6^w8us zQLEqi1gFc}1g)n=61^YlT#{DVui?w?B~Is)TqaS^J9|5Y#ot)C8k;Vz7eK;h5JF@n zouEV|;1dd?xZD&nC12OeAi191v>EAt>dS;^8|z;sO|c9-t)M~5qm7d#<}#u% zj;mi?k<(E{KTi5+V&4k9PHudiR5Th}UKq&*m(woCcHBX$NSP)?cF~#pb6@ln{ai#X z(&o-ZU6hJ?^Wv7=!5*S%zftYF#C_-oGO~=rt@9#R(e`4T0Ymh1aHxqtztwAh8z7fF zsaKA9K4jQ7BH~Q&e}g}Y%3AOORbJboD4@08S)ROEC5clu@@%EA+@ACaT3=gUxE>qd zz5Xm6u6L>t?QS1OP284m^@9{E=)_-tFAmN8ZPbhs!H_J>jOfa@KzVk+jMLkGgVGVe zY>gT*@dY`qJNjZeXe*AK}+);0Y4Czq7Lt$9I^oiSs$L1}~5TFAx-6 zt<{3X`G!{uCuK=imSR}xwq@TNT_WnQTStvLXf$hh`G0m^ZD~0vSxEcc7^u~3t63HZ znZNTrtHC4MG%xgoGoZz}l$AWc#(3ShHAx0O;pK@L1kiQ~Fc?2Wlx`t^5k4o)L*~w) zfEE~|Cq^oXx<-jEW{RUsWhcFW#3Jq$rHHXcF7LJjNAuJg`*L5c^#Ss&yX0i73~$57 zl+Q<6K^w<2-ewDWitC5io6&EcCF#cxEeR=0?ZXwZCbKBT%)OC15)NG%Clky+IO*xp zzH(*e@B0_fcExMlV=RDw+b(#HUE$5Hlxo5=WTeW71Mrp}h5dG(+6*dB?nzvN>9mRZ zF968FaE4BZ8=!3}=u_qSQQn1#qw)k=6)n`q80cg=Ki3*f728U3t3*3 z{Jp{FL%ib1B&76S7%-eI615j698JBa5Kh;kj1rpd$AKJ<+9Xzg9u;7SJ-cXmrn4u} z8Li6tFIt`{1C>Y*I%CH_SHg331YJOOkxgG#Q?%Z6D1ET4OWGS_OI-@9VI$hE3;+Mhsjg@fd zcrA`Gaq@GAfjxN-AE#HB0us`UAdIT2M4&J~02Hz@D<@I%B(@^I#7iNX$6jg&dZAq%M|W#QOcBg(6*MfQVEk1E+5i`N523gziSAiZvHsZc4-?^ z+iy)*EnFji3IcXLI|y?~H(Nm*0;iPWL=`%I!0cZV8vnSfMUSn&6_@bgr-;y7;wt^_ zX@RD`59%1MyooD@Ls(dI47E^ghMvM^t;zPgsebIh3wPIYs@UC|BI1Y{`oZ!S6w34&5exme%Xhu|q+^?A$~@R^Jxd}(v8&fWLd81R`dJnh(Tj@*E-d>zRKaB|zVlB`yL6-6TteH_Kjwr-&Zjp4 zk40=$G6`85!zFtq7g*EvQXR?0U*a46fZy5!T3;YqBQ*-JO1>19iMWaRzji?ez4W0b0v{KRaDK z5pT#Vvs42eROpX)R5yx(;Qh5<*53z>TBMpBfEd~rK=15f3rz?u&66d&Mmrt4UpT>) zi$fJN7>I6&$HosqArq3ik&iPI#iPUMbV{n?Y(F|L5q--0~(1us+XDfp39c`t? z?QKf!))~I{6<1p%rL!XD^uKd|5Q*UQD!3Ab*g6o`;cX1=@*lPL4+H00b7ZIjAAkP@ zzW|&t|1cf;q+H9pBr`Ook(JEia>QN%iy>**P;yq(1tIz$Ru;ep1FI0SrsVBl|UFYY=g!-ZX(=?L}<#M65l$mdg_p(p# zArySgL}a$NXhPZR^;2S|2!A?c5Xkm33`Pb$l%O-)cZ~l>mshX1w@glJYI$wR)BpgP zI{`Z#Qpgik=OIVxML>6C`wtu%9c#&6RKumnRBJouSM)TGB z6X52U6?^}G`qoP(cF2~F`bEkas)=UR0*$Vq!#6^z2qmHbL9+X%ThUgG&kMW71e z;N1=wm@8`0r%6@hs|K&;gVcfEv#cm(s;BD_+^4_4p216!bUDVCz(&88go}pF0S&C# z@4fEsCks6NYNM&8iKIVdAQ4Nbj3m7qEKAZ#Q03(K8{&96l*;0wml@JNMVO?I$@paC#gLjJ27aVnq}}Bt*?BsX zzwMnZ25tzWwIe&9-^v5dsc%h>-Olf-HxXISZe~hE1)QU}5+Fhx3m@D7rtM1lE+ori zfkR+_$=@7prHD~i&Aud61z0BO5l?w(I@_m(5BzbxWnjDNN{Z;A&1rAg07AVd)C_lt zM_dP{h6`5xqz5$L=J^yxM^&cc@lkCfx|bHeM1`I~N8AT!_j1nXm#pTYg(BhCCt3RQ zKUMxsH6ja6VZMGz)uZL|IBQE6YFn6N7@eMfD2i-|?pD9Q=M{T#%^Kbdve^9sYbhW2 zSSH;bKg(a}F&bU{xay>Z?i+NMmjat{;BUof06xo%ICx|Ytrne+q5Bt-e*11!UMyY( z7lhwHAo$kUQi1|gYw3RO+tF8N@OkBT=-Y{lNXPK&=g??o#7N2oNP^l5zO+V^1AfGR z^7UaxRhphDbm8aHOFwu=I$8a_xfH43jlu}vll1r=(ettr3j*o-LGfuI?AluG)1Dld z^-)suN7iG*w)ikdwI2w(9O_{V7N{R%2?X@ z%`W@-0M(O5>a5U`voryHcH+zzq@g>1n%fCw$N3IZs*|4GhMQ%K5!JuSlec_SLZeEX znprWY4(;@?uNw__uXOwv)eoZ1FV611ym>)``o1||cY@}JM@GYhat7P4jbiN5Jo)Bj z1~NdnY#MtB&-dW8xHgJB|G)a1pMP)6dB?+qtLCsL&xu) zmgG2^u-@18G!w}_tMurUqrS3srf>|^M`j=Q%5}Z{y2Benplxj!sbA~TOX1M>($kwG z(VvJ@v+{06okB5~<3Hi+cXl5;b1-U4XLTA|zl-`^$z)u+OWfS0<>;4xp_-Tz{hlkc zDL6Kq_z2eZVt+XMQ@TX{Gw7S5%rgcvaUelZ52} z2WEl$G6z<8q)Vwv{;&RjjQ0Wcsa;HUlpB-LG+FV=p)y2pE>TI9n4^jO)?Lf?8V2^} zOK*Y5kR%m_yFjV=;i2EM(@|^8<6ohg-tQVe8v+@UAc}pdsq&Vj%KNs~I-RG;I=*EN z%}&AW;Fx4HD;x)xf79?4&n0^M(#pd!pcIU7cDi&R$j9XbFPjK|>xB$78Eg^8-{M3l z`EgKymJ#AekKDzAhJVX_D>Pcch~-P1VmKMu`s>L}ito7L2JL6l&Heq8zCjOiWX(u% z4RqyuKB&U*A+lX5<#BRMZCqT|k9Eo`m8LV$Lo3X8pfL$iuPebl%2Y&@aO5>V1mY^@ zj&}jMRq8wkQ>?pxE(&~Jw=QLIzJ)qhIbNuH++6?17T7p+&Wp)E_Pd)oJroc1*M*C~ z_t~^6qMQ>S(n%`-k11Prx=;bQlY{^Dvz?3*F`s7Y2Cgr5m&6)hcDbco59!&o9a3M=`ZMXm#KQD8RBA6~8ylyKD_ZGUTs zjNz7`>CGiNvK$L6-DG6*A_BkBK34=ijk9!ZFtT|3;sRfEGSFMBbAw-syKFoqV@8<% z(2TEkoR`7=9{?YK;D5J~(hH9~x4!-x(;T<>D-4Sgmm|dzA-AbM3=tH6jJIV_9Kg5s ziF@z~2~G&X-Q5GhZEz<*aCdi?5G27}28ZAR&yllmi~%WdAJXO@P$6@w7b5ZfadgTc$Jfc6<3ik1E2$<8N#hDABH{sp}9w zX|3>+R|Veb=y=Ydod(x`V+wR9Y6vcxJk;X59J^~Nn}eHizkl2LzEj6{SvDkKOMx_2 zQ`B$O?;E~y6{=gXa~mjvm+Nb~e5W*<#{#Be|7|eo_K!*orqA*39>Bx@FG4%Ca~}vOukT!@8@t5+@&$%t`Tu2Z=&iDa zd-+iv9)=bx{Zq1kSI5AJt%V0b?dm_NxCc;<)Yw=~H7rV9cO`!IrkvEq0r|d6L}^k? zon3&j%6Ba_eJ}@U>jVo&MDL_EQx4gmY?r)4zuyjg59BhLI;`yT7(vz2>4w$HUA-eT zwx|uJ|EmQm$8e$i;WHN0?i^HcWq;c6FTip?CQ$}4riAR99L zlSzR%FgrZ+OJVl;VcH8AaFZy$<|d?YZ>?w2E#OCgbtZZ6ywu7d+Ny4PBeg0!*il+R z&MPL~R{2ZX|GqwAnvuJov)IQlUd0fFl0yW&o~@V4 z`$9rp^q8ke_vZ7AUf<=~(@d<)heJ48+Eb{MbIx>vx$$6MNGXRgv3hpi{nzj4oVTUo zF{wj;<5%D>Bik-iLp{!7__NZCLZ|iYB%1^Cz4`na-AJ962q?; zjWUXb9%|oK^KhJGZaloYY8?Y9KqmX`x=yX&ZdY@A7~*#2QD2pgaryJYF$*Rc>oUh_j*hQ;_dJ zkYO;@z-fRJgJ03mVvbE_tn0oZ3+-YZu;xpWi?6@MAy7PvSEXWY)NqGaN-aKD^LC26 zU%#3(BT-(#mUk5S@R%g(Wuu#FtYe~o(nhQRo;y9|YE0gMG%K%w`V@!VtL-CS0;woG z+RD%-G;lx5N^pxuoIf~!;c571EnA~>+r`vWzRq=|z8FdNE-~Qox@>Ej- z(`HdhE)*FILtCoEH-6&_+3lf^Kd^bfOImKuPV#RvI7qv9Un)XKatXwWSO!;r-E*CC z;{}q>`Whk}0<^gMp>MqnMj$&<}2e*N>rppPELNS^s|dyyZtBix_0qRY!I0H66j>8#LR}CC%R` zc&2zmLYp4$$Php(<%qv%qtnyXz&|hcLQbKE^~sjxwgWJu#k#CPF<_fHA+D<<{}WZS zYGl~yz@6H!j2qOz!f5W3r>OqE3Gt=S=20iB>-W(7dFr2`)rrcp_Lww(5`gfTLt%N$ zqTg2(u9Nv0{-_T)?)*kZvmQQwW*ObYK1&QFiU+0edAygj;UiX~EJm3!116;KkB)91 znH=^xuta$j9`G9pon0NALCZi54vm4n`C5hL(-i043=g|p1f1adn7M%+E_@U!}aH;%@Yi>TAD|Iqx z+Migxog%pn@PtC-`(weSxwpF+T4eL0lHI7S(T}5$C$;`#*soL0r1?)o$HnyK#Z?@f&HGuOwqO*ubw_}}M}`zkX~W5fJRWMgjaZ4VH;7v+tP_*&Z~3WLiK)D^+!?`D5ji(2ZArc zBTCZiok-Hu`cPI}Rd8=jaf}l<{xN<80-~1^Fqqt))bYh2XTmM`qQ}9{FP!&W$LekI zZhIDcg!XS(a`B;QJ~Qz*uSdmL%OCN=#+7Hi`sW#JHv~$*zx`=O!F6xRdY$s9I?M_M zh3_qY(w3aHqFe9*oHp8zd~0LrbZBDSoP^*zw0#6pX0aD|DN(nEl$!}w0o=EDCM$70 zwIs^CC9RF%*BYb^XA4VjJj8&9dX(*(GzIT4$M|hRs2u7y-8<>N=dV7cR}WwDCD`(e z_2CwLqD=3_7W4QhtTJQ$Dy+29t0pwn1Pso9ggKAJ2A^!RWqMvb)#(K71e{LIha*y7 z6VAW#y_)`=z81d&_zCP@#SsL0<5jf>Co-oa1Wdi!e-FPpQ1gFPx1+yiIcsR+ehRxz zy9(3?Ff5q_eC7Znfy`Jl*QM(T750-1!hAIEpLG^+dM}ys9oVV9(BXm)7I!M-*Rq~} zX2k59aI-QTRdpTI>~tlDdWU#s2Cs6hDGpdCgJWA- zxk{?$wmjti1)Y4=} zWjDk1&jC|`b^PiTTCn)PIY(ae%|aH0t#M#lTg*C z+ie1i#mKkV2gFV9m2PF!8ojTh13tr5?JK!&X1IPy4hI(E>DQBrQLydLl-b|E?xiG6 zGJnBEOpc-oxqW_+uqf66qP=?ZufKbO$O3xQKLetbH;)?EGsX)DuB8Og4^}06^1)-@ z(DkE^uN-{Nk>%EM0&lH$ag8v4fKO&!s`h!OyUfW%hhS94h!ZMC21xY8z-vlu-vU1o z%5rm%45W+xT5{&AQ)5Fbt=g#%4TLF#Mv3jptYd}3;Hc#8GJN{1O!Hr6Y2txw#PnVQ z#rkG(_Jn;}lB%j&vXjP$1m;l5um=3BO{irg8x(YWHoi*pL1jzS8>5SVt{0NHOh^qB zs)d1H#(N1iz4Ull^@rB9_JqG|(H2HAlzU2$RloTL%TQh> zhZwu0-VX0V`LN%d3QTX0SBML5EN05W2*N5d9aylinN=yCK0P%{vAB4fB%KZp$1;sS zh9}<&MXL{;Y~|hsKAFmzG0beu#4%3XQC3-$#z@KpaOAYyg=o@$2c#_?V*j2i6?;76 zl^c|yJ-Fgje|b1ax14=CyY51FQ|ZTbW%dfVH{Q3R_E(P&Qhl8=!#f}U&5TJH(0Wwo5&%53~t~iSR#CBOmTNTA~ZF|=IGB}nk z^Pn^A?k;n$atUUC0FKN(Q74B8o9t2TmDNmYV?T!7-f0;y{h28r-)oc;jm~3!$Bjs5 zD1RczeCtH|H2m2uchjX$MnDG|`pWx|EC0O!vupVKvmg27m#2x|7o<$Xo7-{b>+|2f z2Wu|I{!~c58k%ic=5NnJV-3q%ZiN1ki8n}QqKv=qs_FfIy2R(a!4x_ex_65x1!t`1 zDE3YTUZRY(WH(w-3X()E?09~vgJjy~MpmU8omp`TVxjLtp@Sq6a*c?DU!J!>?F&y# z9Tx8x5VU%7{w|Z`%#GbS47<0A>9FE|#6n%MJ4^WXR%#eT+4w!_a29GLuNWqLNKdMg zJ>{GknC)zT=hv01a)pXxj_(lQR8^%HD0dyHZHx+Kvx#t z@Bn)@myg!fTp}hAw(Y!Pq_paffO)(t$}E5WVjV}9SExvB&0LrhzMoem`{O|8;i|c& zD_NdcT}qBrVFT5iW$H ze}Np0^O*69WYpiqWeo=;!Kdv%v>Mqh#wXkxd6_Ee-KvdwUW27NkP&eUSh-776Cwft z1@ImqrUXC&pdcVn#bF@)uVrCvYAXLtS&?O&5fNd+FHr>hKS2NjfEXaa0@z^w&nYy2 z00#L7^zW8dW*+wL|JUX{0>l3+hVVaxtxQe5^T#pz5tuvDx3K<86$P+|3J66~0$}~m zH{?u8;)ZeH$V?j!r-|~_2O!K@lHn_%`{?s2kr*Vk~>NzGdNm;_z%AAg}Vx6q)W`{R+U88ang^0!1N0`ij-+ z#U9%4(Ro+uo#~I{cw_7p&pMFozQn|;R`YUeCD=D%M9Xw#)lVaHv0aoHBvbN zYnGpL1+2Uz9VV%Y*uM-*FCIllkKIvD5;^4`aa{iNT2>W*I?O!kaL?(EXVd-A7w`3Y zGI`_cxDqPreCTiD@9f2P__>1FZAHkwM{IU7MB>db2*7eJ{fnht{g zUNbIEUQ3?;ak)s2Uh`+c#P0!|7nS?Yl<9pgZ;Hcz%;84yJHaeDETA2zR3qc&(UY zw-17U>?}WgHQsgeiu4C%v3Bjdfnl4{_qPXWu$x^ww#3@^XVxDqGZxqV!ksDJB28xf z?*D>4Me@l0W$akoKD;=>I!;pV!Or(!&0GF?7T6!bcxc%+GwiGxuyXjiVtv)-va;Ya z>U4bMJCh@-MV37Zm zm3eJyDdR4n8p_x4WTpS2NR#H2>H*nQWZzAls$JMOx3BQb1x0M%Hxb^KRSz7R?Q6ho z&P2DE{9(f;7Df-`Z*wp5x-te?Nlk9+k0sgW#>-H>#TNJq|CJK#g9A&;>4ih52iR?Y ztZP2QDcYnx3NrotShJk~yDPa1@T=j5TR{gmo*dhs9apF-i51>4&tK%86~ocUUQg1! zK`WyRir9I=%zI=D{~o%|jQ3Alp2l<7F9NF+sr)Oho|?yp*6&JE3|Wji)O0gqFYZ?Q z_H{5VcXAs}(AjkihhLewy^l+-+UC!HT5R{pvhUV&S^|ahP8y&5vzPggEpgZVhxA|K z9f5CR8D}F!4S}NkPWweb@aw5S(V6xB<3v=PAWZ*^q;lWQ(rw-G+9cjBUAwpDqT20* zyY%B^#_T|@USaJd5JFT&g&j9>^im^kqdmWK^}6&jphfk{bKFH^r=Q$Ux2(5~BgXl; zUP4aHriEsK;2Q$LN|tiJ+kH8cKW>yHCIj5ADLxHO8MzZnzuki-z19+cqZ%(P^Zrul zs{F@={9XLZWubcOzn|J9f?m%;6``{TLj^zX`qXJDF^fI*#TxFEpU{#c>FZf1+nJt| z$^Z5fA#X=^!#(r|JKb)7J&%>!!hG!Yb58H`!&BLIyf8oL{NS1F1KzFM_Y2meY{>Ik zpCGkKLdIyYR+( z%WX0It-w_Ey{{;LP%MwPJ~S?%>uxvvrY|x*y$-_GiB7;d?-qhJzx2#Rg$zX)gnF>~ zWI8#D9m7!$niXlqI$b-isx}Z22KT=jd3)-B4{p(137N0vlVD2L7ok@Vho!Iqm{~Kg zsh(-Cs5qou#lcmJ4pX&$|JeD}e$yeLQ4IE)NEB>0p*|sh;i&I#bGP{7=_lK&(pf|r zRI%|Hz0A^3TmRRUumEH!3I$9&cG`8u7PT7&Xx)8S>rILONMglF->iqY8KpBCqV$`# zoU7kW<=*?{vAUjb*!Wl0S}yw}rJ&R%oE%jGarD$AyL)+<@#F}HUNQqaqLtFK?-eVG zbRtbI$F3EBkhJM&u)Q}2iS3K11*nW{4OtjfUs^nF-gQuV+o(k&yJEhbLpuH?pzdo- z(va0wy3P4_YYwCC6KeJwcU!;Tz0Z2-Mo5sWB2(OR+rT|Tbko`1fq5|l}l$c_9ZF#|xU*6Wx!Yb06cug7H$c=ojW9UK;YGiv4QM!T zFUfG$zlu=Q4e{FEkad9hvJuGs6wChnv?o|l^rM5O9(DdiCD8k9GoL7rp&OaShv}S2 znwf)>+N{YeFi0ZuJrCY(cSSdz_A}CsXlk#2-XGqLN=iorPe5d3@&(I z2XLj_5*M+&KLv`9Vwk1S$h?2spG2)8ATkpEA)b#`)%XJyRKTZO@&HsSbgN0Ejy`jL z>*amF6Gk02$XYA5{$z!MOb_ena~C6w_w&VjM$X$b53^Wq8*6*#Nlw}914Apkdq0nE zcpz(fx~GGTizf_%b%niD4;3X*f{8^R%(@(2tk&s!BbTVU*i)=kJll&;H9Gl|D8_1n z3otp_vE!657|#*J4}@at7P=e*_V2!b5B0fzzDr|L(BF@}Vz+iz>38N;-7aG^WB(|< zRc>~NIS`t=G!=rYUsQ!Nv>;I?N)#+q#1ImXvxbMOz35Vn(0T|BtC79=!tS?D;E%@a zER5IzX#-q6b+xwBsKXp)nve%XgpbfcSBpe^GYLaNuTlpwSEVotj$p*EG*L8vZ1j=A z=HB}$v*Mi1lP2-vZDWE_DgOMnPER2aagmKYz0Zc|Wgjepn|kB$qP|sO9&Sfgn@nbP zrJNq>d1GB~u8TC}LfwwKXHLAX1jtErBS};!k891QoN_4Ms1k3)THxjpA^E-@zKdF% zCO@m-KXtc>x5=^}?x#3UM8@!cK0TAxCgH`S!;*6j`EO16CVW!BG8!)ZJ>PC?fW<(u z1?TZK+;OWRejKG9>?aEJ74-D?q)pl;Mq@#e zg{|uexkp3JlPLwjPw#m^n~vZ0ME6@JKCSK+^PrO5OMcgqB%B{|Lqht0N7DRB!pTPr z*BW*Jq{hK>+wDz=O4Cmk&29kb?zi?7B%1$e`Hn;3K|J;vdj}PVqE#x?NhEu)ABY8$ zXkIvqu7`_kwRwa;&W4#2=l1ys(SLA7SFVGJ1|meh65s|Enrus{N>^PevIF>gQE8?2nG8}`SJK`!(dN;qdS=dAhX$PRO3%|>m z4HN5CA(=#nMMUOsB|ZQ}uD0~3dyU$y<(#EI1JJrN?brLSCNoM9KuC_s{)y@#WbxNx*yJdtx&rpJQ)a^P=&?n|aL8B9>nS@dnSw$Jb}FhH;u4 z0wICTIlYSt{{1R*wqPgYJrYym^8m4ZcQ%smep(w;L+iA*i;^oWYy%_FYa3(zZ|~nX z-wjTZd)MOUf0(p?u`7+Su;KW69IX5YAcQO^+Z8;?($k2XKaU(*EJ|rz-==S*hnZc)TKRiFw+i6NY>J))Xl=_w< zF7!N9e(9mB!?j8xEp%-2P|!Um=q{%NkI?>-yJw5%-~@YPZQovdw*7dY{N6bD(~Zq5 zh7sw;ou)~D$wAgVTPM*T_P#{WM-@H+nM;A;QTRDnkr-)RY?C10hSC9T1&>O*_mtpn z1KMA{j|w(7q;4+G8EDgP6=g_0KrK0cR|^A%$Nu8u&f@}yIQILepr@IcFg7fn#Iki~ zNjo68+YM&<;(Y+kRo&j1-;wNYVAnx{yKP+N>(KUp{pJCl^RV^;MZ!e~wei~9fQ2YI} z2v#wF-Brd0uL3WV+S7S%Nfu?%o_zt&ea5IeB}0&&B8nJ}!Ug#~Rov}`<=v^#7uAb( zeID9y1qzic)pTARbpz zMTK7#zYC&7fj-s%Pe8E0g1g*^@kdPI5y5d$jC1R&f725@RR{_z@g}>2NmhxO+12vt z=do9wbVOZe+rJZ$`a1)z#Yq;l_JWA=%Ae2_KTGajHK5$vtgBjg~g)&G?xL6UuSG ze@QF}?aHFqFptv-t{~Y>+VH!F#oODonki*^aId$~`t$pdXM!gg6We;$unB&)T)zm! zWrdi=@>_lr2^Ma23g-E-s3GC=aD9#iiks;^!wav!{`t&wj&Nfm-A4aBg%>4Zrt~mk zAo@CUIc;&iU=c;Bv^_b*s}=gxobje6e-AQBH1WE>TgtB)KNmwn#Bu@9{agfAW2=rp zY$@849G?$6u@We@rOO(hB$m(xKFzihRfm9t-wZNR8VJ8r=WJ1{g<0#L)^pzi$=w6Z5)ZF*E$UMp-lNE=Ywb zXXHbaI~&;YZg;FZBh#I2DJi)jB|K_-WJVucRe<5TL2sbBoeE9weTQ0a;qvhN z&RO9HKzrDoKa-ikkg@^EblTfqe|>|G`R`HorIO-L2?^}$%viN`$7ci`J*gdlsqBKc zXHu{>YUGeJr&Op?R5Mn;HlKUv9ngx{y76IHCHe2m3Z9Ca+rNNkTnHKc(`dyuWUGi~ z*Gu16nRAsWAD@$TQ{dGr{*eng8}PkTv6*Vv&Ho*Gyj%#L(J@v<|CNh7f3kNbixWQS zd_aBWv4w^R+1f1gX@3|>5mVIVF(|QT+Jfd8U-i26li}z~q3a@NA()l@HLe$o7yRIr}n=VXo?}vVLv-BEZOfwF6dR> zh0UjQ0~tpx-qx&ZLZTLJlisi1p0bb^8BkbF6zh%4+ciAYP_f7@iI z+I04`ul$U8^g4;_Xi;DKi9p~J|L>jZ%Qya0@DdTCS7|PC1Q-M3oCHrmI96Wln}?x@ zcZt@=_vbW(1i|Ale_+rB;i{#kyx?%ID09s6=EGE}h*uBxc%G5V7%U9=<3HuVmd&Jl zpkcu$^2084Qg`ClJ`xFEq9`ok?$*@}AbO{#F6Ad}aHhpgy??A*4<3g#E@pqhfDphx zD9~&l|APKGBrL9-cS1b2+q(md=U2K zWeN~9C@67H(LUcJr3PKPth26DBY}vgG=$nz8Y6t}r*r;dB)YiqBvvj>E&&kH_x8QW z$j1>u`Y1dNK}fimJt0}S-(>{v-2_NRhacf@Ce~TR?c+Ej&4&u8lQ&uf`45x@KPMn3 zS_Wck3?immf1E1cSNZ$zTk`$*8>(@*G$H3vr|Cd!R~3MrvKs4VnfQs)?N8q6_z)cu zrLo~(hS&Jl{#xI@iit`p<4nS zl4501f9t2I&E{R2Dbz)kNd7K?gkuECl1IW$rg)c3%50ZVt#Tsp?T?p5n464sabK(y zqE!$_Y7M@;*9>A5f6!~TXl_ZE_6i(zOGd|3JWt0%X(m=RG4_%Q{5Lkr{mczDSeKF+ zGe=G{G@gI!Nu(I#+l*=4`;3An*-L36de=pntMd}Iwm5`(F(F|fv!coeG@(4c& z9Aly1Z-;v-peM$}7rn&P>r%~4jpTSA9WQc_O?JvF{2SNnl=#Q>jQU3g3jd>_S{WHH z%0S(0#B&xvIZV3!!l#awYsD+*r@j{pEC8S);{>cN&x-{lD|QB=Ru3YP!mq-7l=#AH ze|yIww$&0i>%uC8$VIM}u1=hSZCP*8kqM{>^}^*YxEP^5aOhn8(4_`hO|MU1%(=D7 z4cV|&2HNXSh-;9)zjY!zWS(=LSY*El>5&}Xz=oH%yd9+Y2j1V-sT}w?LU=);U{})-G0;eSS<*YY}AT8hc&Sl*XBuzMzb5 z97P*Ok5!9O(sy$8uk?HG{)buv`8}a2MlnWqVl(hTRq^=;yO>s)l~-?F>}-~M>@On^ zKN?8VDW1{9fmY@hTjOqXU$ovEQ&dx_A;gn+@_)UVMeI<%URX16U>7$sV<5+*f6PE0 zeJ~mKz8I#nzHoWF>?bNoj2rX4T1x%6IB_Xgcf92CL(d-92+=9_Qf@ofz;w9{d^OtA zgSIKC)04b2sBV)OJAwO`7WColq*j-~n%$p?XWB2A-RXs^HU!~fe9O13Dn4eY*1kHg zTtJ>wt91H>oc_lA#%x^vDyUquMOF zYSqbIz`up)6&=yOXJMyz9SxIzDr7<2~`IvDeWzKpCWKc5Dv*dHue}vP|Cgy_k ze}UAh`HYW6xsL5hZ!D5p+4nV<_|%85b$koA7xizKDJ`EysTHQe3q-bnU9_Vp<~$QP zr?C50m>3o?1;nWl>4g-n;M@484O!X>o!}0W=ZcakGBkZdC+|7q8VTWyYjq9lfP9K` z*pjaE3nXqg-;XP|b`zw0eG+FxoZ&Hzw%x{utame?CFhDSG`Z_^-VqrOzxgw*r}bp-5ENA)7z=_kRY=<%@J1 zm$Vv=QY6H?rM%D2@yWsh1f{9Ye+Ct?RGd{OrcXeV>IC!NmXO?&e}U;JtBZHIG)LL* zkIF=1QXk`}{-$pju^`#sh{9alHjBZ((^wHU0T7ZU75m}P6qVU0kVUn`l{N#}eK=&8 z0xvBL5WT=MV-)kj>R9lcgD!w-9+}*9MYjH>#P~BkK(Ec=5=^; zW({elt~Hz<&DbsXe@Fdi()8xc=}Jwhzh_%Fgw-;s ztI>5$?V#nU%OT%-a!#6v(oo4v`P<_kiPUFcLU#?*aUEG@-d+xOVqD;OS_@JB8f(%7 z>qW%VPOZw@1RZI^4@S~AI<(-<`=8{F9P1ce-bZ6z+wP;4f4RWb3wd_xcE|BiOF{0@ z{d$*@aoowD3SsyRfskM%clGev>XL9>=ckh_7}Ixwn{61#@f)BW z>r<<{Dc=CPe~B{W1nEukgEf0#?Ff}Ug_}&06L80VcFk;p65qM5gf%eMhz53w{xMBh zj#+Qjz=goDq5fHMQANt~!sbVhX7x_YwW!%|ig)`~yPaMhu6Ie-fVdhYM$M@!3ZP*E9}5j$5)7 zs7lTkFvLJjh_l&|VrPXl;$ z2*;?#t&(9g@9@&`l0O1uqG8taCDqpNIi%&;>MU>Ay)x#s4nSLisq>Xo=x>IM-mjH} zF;RAc=>=;00Y1ECzL|xe&9Jf0a=&uFl`2SFf0GEEb&7lggl+9uE-J47-dKjT=ZA}H zo6?ANWu~6Zn@J&`V`ZCG%9$syVQI}YtW$|41%pMhU^v;zapE0>Q+E1IC2rw-DcCX@ zsN|IvH-Q(~;~-k=y_V*!W=eW=_dNF2Wg0?hOO8n=5BK2wKqf$5gS)~8r^>YkVfmtY ze_$RhR4p3fyWz4i5XKKGpPt{t*wuZ3jeCutc;X*Pzgh(SPQufSNQ z2U%&Wm6haxfoV9vig0L?A@2JRy@8$WRu?iN;Ug(a${bORuk$7D%?1S%K3eS5Iu*xX z*iWq8vS>S4*+@a|g^$R_hNjG~Rj~-0f5>t)j+I&45xO6!$ImKX5V!=~&Pn)(x+C(z z%ItL?lgmLU!I!3SZgX++1ic@(!vBbqxBpoiEW=(P-q)}q4q~DZCo&mU%{o0K@x@m8 zGz2uMA|wX5NYI^)3p$PU9HZ($2j(AWZj~R1fvIMfK9%NRJ;R6Ie*${yK#v)2e|zJM zdq#`n(h<-})jAUWwAM}GaciK?Lqq;+1V{q`w+?zXk8l+7FF<6EAh&_d;Lt~R?a!t} z21`(CSNWQDs1igNaHi!WKahV3`eE28F3145Ocv;}%VK0CxOSPsxqlVdmx zPz>N%{bs@fCraZT@~(OyXRa;se;cc~D6zah`~x`N_oMk(r?>r!9@ax3-cVXhy`oj} zDchp2b>rXB2^o)H?fbdQIpdd#d&%D;c$Bx41`UsOJxd7QpL~N9_M7(We=d+ka$C8t zY?T>QZIX_Qn$pG0lTY=e{*l+tSut5 zD@v+L;JA2BulJ{sEUsTQOwyweWE6BZWz=+3XQcTym-YNM`pK=k;Xer{^R=S^v0EOo z|Lih6Z!-^f+&oioIbJ(^_-c27>JC6xF?vBOWXv{r zK=h6q_?SGV&g#%~^zeDjSE|hz@bG^4q`?<(!y%E-vEQhi6M%2 zTd(0c&w5zbf7ZT9BKsEuAd=^WThb>ifj%=MEhH29blGeiW z!H>tZ!fF*_tuA|15S?-aVKp)Ku2{#ksg%@6 z$U(!_4z;^ZdJx9-;3f(5Yqs2S48@h-4jcIF{$3am?~4Ou3Ltm`X!2<-S|Y5rAW3f2 zTeMPKe>MO`B)Xko{XwECm|4Gn11ndM79 zQsXmi*1_xD-}@SG-o*5A!s7M84Tx7ER)3JKIw1K;#P96i83XtCz>x7oF@Xt#^tDP% zZ^Luc?rgOUqM)0*1>ua+;ClAm7?VJKjHp1Df6$-T+rMyKiRXrJ44EpiYM|8(fAUWw ziZyy#oM>Hq3vN~DZV*Z@*>o%?o_+OnUuvoN#*sl`9Y#04Yg^J0Z=hb5T2Paz5T3js zlj+)wTw@)CLYK+-C9vSUf3TL+dbhi8ME#A71?Zc(9r`C?P4Mov;{c4YiZ{7d8By=>JN_3?g|QJc;I&q@0GpsN@qKKNnl8(bfhe^*gvr_9SMWKVw+_*Se`(ElaSSy)_;qBVW9X(eysTfjbR#R<_mxEfODZA&RWyPxn2j|f@w3v$_ z%d2xzXYfl1_>%|F@1|Dlr!ipPi$4%6w>?5rXuf5*5kc>F`C)L0ifsZqe}e}Q7o~J( z{VwFA15UB}&kQ9G2ARIn`;aZy=wFr|lmM9UoVnYttsDa^ekR5`&DE5KqUSAX^4|c& z*b4897~2v}F=}Lga^gHz$gV(1T|ku)(6^RKZnEYVIqyBq+hYfRAm)=~{?{AH-dL^L zmMMT+qXScD{r#b|CZ$Nmf4(&_zhYJKHon~0lhs?FUs)@`QA%;gZj6f>B4x*|lzlc{an~VD;(TLRx3R}hx{=d z|A$yuCT1`@plIVd;0IV{*6Bl!7VS?K>4(^x&8I;-B4W@(UYBz-e^z=PK`=Ry-3n-7r!}J5N&HR%I#Z zP}4U6lbXY7gvLm7e{*OEPuG2(!3Ha0Yg^ex`<~P}lASp8az^q&($GGlP$dVV$zBu*p_}Tea@@sU|4mI+nj;Kwp{3&5S78hH`3a zk|hG`vlzc6f4aJdV>TCRLC2>iyAQC9)r{y1gjF~R$)67x;vsF2wN2J1Jd=jylPq|E zmPwTl0@1!cS?dw{@n`)PL<#by2^|{sN*zQqOLiirbgWT_eXo{aOMe#w|GS%yQsShy zjUJxuH*$#5>^ImGXh*O-CK#FPg9xR-!?R`7h;z+de;FeUh(ry>ceJHdJ64aRb1`<)AZ_tDyC>hGXgK2?0+itC^rtUM0SN1Q-xse#t)q`}f4{rdN~sxGtb3|SMeE#HLFn3Aw|(<@ zioEz?(O@)+KqgQ#2C5k-lvJ*_b=CQZ+i|Jo4 zf1VIasWg36F81?=z_t!h*dd@|viJR91WV%ZrRfD>u8kF@)(JYXvB1A8SwNM*=b9zA z!R#Rrdk^b$Yy>Z_lXbS98u)g?QUrZ*w|YYdPUM&wwY8BGYo)XPQ@Q?eF${G}Yc4wf zUhR3@r~M$Sk~)wR9Foo)j4ERjEv|Eee+^LS+{2{t<*Vx6DwGZ&J}A%MM69#fO)9>- z(b8vk@1Pm=GFrSl&% zrp_ezQx>RuQx>d#kDBo0LQ^T}MLsL)QUF(3JZHYy^aMVm@#g(##`0%yII;_~e`)9c z&A_N1`7-jWPH9aNu(e{A1^Fq!NrL5B0}n0L*l&WAavT@=xZB-7Kq-vH@);o|Sk|ct z#EisvN(g3Sxa28%bbsMi^w8+eHAX_r?Kv06J zmv0w9n(e~lMf##wwi1IOn$KsTkJ~p67A;Kk3o^vNFaF*~ZWV8Cf3?g5U#oq-=>xxP zrlCJptH!OY3ug%x;|If$<)A^*ZbhbGx+>v^SxCD)?hdM2<}pI0+WOfiM>1jt6P>Zz z-rzZne5s(r(NE;ooAFemusA9%L#%4IzNi(gtsx|=0NHB0pdSe&OZyBnutGX z++v7`lD^*F?*u1FjC%H*()O=xQ7fl?buDHW^C9Z|{v|-=f0vE-+YguIw&z>;r@iM2 zF?F})DK4ttCDVcA=5dOm4`gvjO1-={eF{u<*$;qQsd$IP4BwlYcd@>b#tHB+6jxiO z#M(Psk;2VrmsU-<+p@m^P1;s7-5r8SlhDfsMWKWyf9yG^{Ppi=s6jVc;hu<|;dOju zP;Y^9lm8Y;e+tv=8oDp&50u~fC`#UtRPNd)e7Tm5E1K~~Yw+d2iv`#TF!K+gzp_j6 zzq3Rj%OafAL|4g+mb>y3G=Dow2lbD{43W+hkr^lRLFU(*oc!ep#C6a2=;l~Ns+`p8 zx$u6u(l}!mFWRG856jE=Y|r;Lvb_(|k?fy;G{XPTJ|`BAJyL6em& ze~Tke9%AZ_+HS|&bOk-wEu^9m>_#5?GYL<~^-ACVmmgklSy3AT(vk%sMsH1L}c>pGIwF*#eVIO#@iO2 zuXQ}6Rxbj5X(=r(0X^Op7jIkb8{GyQG~5BoP(vqXeMlt-Hs_1gH^2SJR++7of3Q=j zTF!-EEvycDrhWbG70Fu1D;iJ(3gWp@Rm*HnhE)X6YSmxpdbV6;<;N?dCOR1_o^Jh; z+eR>R$Y{KuRc{#*!O|{UQUEE=V(27vu&!@!oxTi|d~V;?dWlp3PyNG0i#-+wh4z=7 zv$LEbRSN%_)#ahU5BI-1(vn%{e*|(k>ls(~hBYg&ZUkMO%dK8hO$er{r;NK(JZlh(A|18p*gWCvn4XA9bhh_Hb3 zTS1SI`UA1@)(>REN{5Q?KWuUTu)SK+xmjmRz`9Rj}ED$@b#vzM71 zec96#K9t*cnvCvcQs@^hyB$!HFfuaK(qt#jpnnz-3yZufjhdTr+0xK(kpB$3%HpP* zoJ*WR8u^0SVa0QNW@f6nRaCW8b2n2VC>7xQja>>r6j)mEqC6luf9NKVTOxd35-lsX z+MNo_h@5p(@i7`zG)6I*#{k#(fJ-voSCm?eadmI*$EhYK@a<7j+nuGBsUWwWPq-?A$xdba*6r|Oy2@MPfD42u3lsjYwga<~y2hmvb~ZBsfP z)bUSGA%}>P#^+S$b(J}XDZq22>MKS{ZZkF`y0)8qJ;foharVZ&I~V5Xsud=~=9Ry} z=Hls)&YI>La7T&I_i;_{D}PDDjs7dz`PzbKb;|8`@W7Xpe{U}nQ!SW5c1!z@qpBw1 z=~6si;;&=O2qX$$C4TztC&jBKHi&3(1)g6Goki_^V#;0PZ9y8ex_V3x6*jK&0sGHg zYw#&^VR#SGl$eM-*|PBwOZ_CPoc!pPZ&&u1G-&=S$it?IWZa7!E5~-3x>IJ?bJ}kH z3xl?UW3Z-PfAIn}#vw7Uph{CC$SNF+;LH-fGh4pEp36e@R90m zH_`W_(b`-)8!ii~SIxDH0g^d04~ABPRVgj$h+|v3n&iw;uOZAritUC6LDK!Pwn0dj z5UbB(v&(#JNkZ;)F~o;hR=I?v<+n<#ROsiLse*&Y+2LC^BC8AH4!V8Zr$s(EG zkrM0M^3|Ckuh3C`&vBq%KygYa=>%p=udx?w0xl7^_qw@4t*{BrVst8LenTY-MF@sK z?h)Tt&j*7v7+;HC@l3g;nVsd5Nu5N@E54{BtA)bHXtVgC^~S*D^x3JYy&+Yoe}zx_ z$xRo1f7&y1{0b9gaYkp>;b^>koKO?D<@lkRyehzUVHZ<)G9su9lQA z6w>X4ZewxhC&pW0^6rB%W2X@9?v+5Pnt^<7|DSOq4D~+kWjjP2f5WNXKOt{7xVj;e zko!@Fd;tCq59!W0o%S=bU=^Youcdf(Zk+?5f1mC?Z_7^Mk;l?q1crh~ZY_6(?cjlf zyrCW)8Q@0xa|3)}#>ZgbeX1JOd(HEMs_-=MDr>GqRFR!S3pX@?iZ~ZiW9$3wB}&y* zYO`kXs>YurYDecT;`G9yi_v83&Y0-Oxq%n&yC@U>6BE&Ql-NwjzjIK=^(6T6o~$yeobV2jDkQ8d zJ!vU_w-#ON1}{X5R99szm9 ziW1RBOpjdD_HU^W>ma#V`d)`Ybr|Sj(+Ji+7TLYy{p?*_*|sRpx}ftsZdPN1e^xf_ zd;t}{d^?A@b?%@sZB*T4aTDAMVWG^G>Bu5w?Y%Lsji|c?xyhW<-Pxw9J7pr*e9IMu zgE1LI482b++^X?Ea=uGECAvki>nrJeAkfa&WKH?ev+|}gK81{HlK!a|V7$l&Dz;qZ}ZxZke{#Rx@ zA_o2YkF`tu7o^}VGcUz#U|JP_+BUKbhqU&XH_O)xE3tt4S4)pQY0Fs_f0OMCzU*E0 zv#76{TFTQ?D2!1{A&ULf_YI4#lkp`V7!VKC!1#1St!BEHBXAEpW0lU zP+3J_7^uLzvg`*o&LEPymv*&O)?k0RJXW3uQZhdVnKfsUpm@Ms)VdJ|0j~f9ZGdtL*qD>fIN% z`woQU%*+XA*ggDNfsR zUq6q*s~JpskSoPib?SeV|Ha}=3m82jvG%CD_yx_U31Iwp_2qQVn~j_gr+e&N7Spa% zfB0lPY=OahRg(fah4MosDGma}NdH;5j5RGA14&BDg_vz- z_(kGtsan4%omO2LkT4W$jRF73_iky$q@}73eq3A0J|y;%&;qQt<+qs{yR~S~==vJb zy2x@>u_Hdl=lEjS1bC-;tf%iGb_?+0ye#1PEisdiB+-klf2!qWx1VY?XiU`R;ktXd zrWq}byA%I|VA+;U^Oo1Y9P7Qz8DDj&W077K#mGt*qPbwdO#~+$eg{T)26BD5+1?pi zD0~bim3ci#MDDBHD`j&v6Q4^e@(Dec8Aq12Z*|@f8B~0P)=91PetM~#3FEv}6cWOd zbz^1Fv)m~*Fiw+A!o)nZobE3Z}6y9 zS#$G3HN-=_wB&&=jq+;F5aL>&vlYLe9h#7r%A;9P4(j1UKNDgMJF5pQdv>-?$^RIh zf3@syof+B}Mc+%HElxKUjg{)vVdpNgR%U9JehD+se_`g)+DGI$qxH+1EKk`>uc;3q z!Ak-HXLxyQD1+Z8ZT~29GH_LbJHpA;xKHaIvxz#ssBA+!;T!e%%2rC5=+2u)9KT>9 zT=KG7hc-bkXqo$iNO3ivj(Md;lI{XBr^$GdhTyA)*S1|+ZgN9E*<>ZHfT zgdgKoe{$^Ftz}90V*&;GGUgfvS1k^fy6#Xpg`E;HWMfNxTB;klK$VwRr%=?0>(R3E zD}-xxXyfD#O(eaoMmEIyVcz+%D|wJXLM8f7Q??veGnsP*J`_{#bc4j%3iKSXWXY zB4eU+_2BXef4oX)(BSPcHT!EqkRT&&mDuW+H8(}u-3^<|XXu4D@QYA*N6?k5$(yau zRk7UygY=Q=H}-c0j^TnPutY~B^?C!-9tl&rrm>)VaAM&<+n=-flEhbba&YgxjkuP5 zf9ZQWq#8CQ0+^`eOMC3bM8Qw9_Wsd7>oYvA44BxZRZifQGl_SPPzT3hkfWK6(`oI8)a7w=e+`v-DaGnHWW;NQRgaZ~SgBP0spqEzAq~V| zZ)?~dT3(K=$n+OALMnWdVbFVt2^98d`{f440| z7U_`iIhXJ^yWfmNtCS#6O^0aNb(4SFy}D8(pd=O^0NT_dBYMPk`B!%>tLQfqL;0iqxKI`CGI^B0e?=8(j?v!e0w_@`=U06bAahGLoxaC&vF_e6;8%@N zE#+s>sc-u`Zt?fxqVH5n8Ew2RFGv!G_dESNKpJS54M8r~5k=El+&3A_r@yLB5#Lv- zoN8ONJZX|w=>lVz55%%S|N63jPn!52<13v)gn~DKPU2JFGJj{nWwS^O*0A3T-e+!zmjPpI{SqWLo%*%M(u~^-f&dX;IWCo~Z^m43X2J_`v zoP6|C(Izq0M3T4>YsWh%z38bha>+FNenWks{qEOXKJma&^0l^JW4mpYquqk{uSXU` z=k}p*`9I%9z%BdyOV* zwWaFR`6s^XSF^e(e}dmt+D;HRT4~Jve*EVz?lC8^=sj$nCUd*v_#E2SQDs{f3>}x^ zLLUU#3k(&SFDIvGZs)E5>z=#?=H*%P;oksbcYBroO}&?p1qQD-l(DO)r>5jB@#e6e z14{8qInoy7Ls{fDW6Gy;HbPDi$oIA)v{E~%0)a^_MnVl-e^su!14QY@L;g@8xf{8S z4}hu~E2pXI+#wyLBvdd8*sfzj9_y;D1Y~*Vk5<%;qZhQm^~UeSD@%WEOI#u1Bjj$q zS9ZgY$u&&2u+jV(*#fD963BSKR{7i3`P_{bgJ~vEgqzyb&9a{z|NJ&`kxDSkv4=)s zNh6x)tnFzIe-W5nw3C!{w%xiqz_tE(Bm$iqM;}#?CM)%DV4#bo#=WG*#puyLeewHq zur92+m5EvO5UDMixyhyV8wdI8@(wgg+ySC86esVSrGhR$OCIs4S-*!-NFaIl<`>ku z{Wve%t$qg5J$*GhYP+|kM)>DJyL)_DrA)&qp3<6}e;S(RrI$ye{2XeUP!agZc=0IJ zR95@s7yj&PJ;>8LiCWzfzXqAQc7Ql4I#JA#;QRNw#8ZV6c6ykGZsV!mq_3)3E3A`a@gBiZ8Wt3vJPckT4`gm%L zFaGW0?Q`r;Ip^KW%+i)Au*ZQm^ElOCKiVhgYQQ+79wp^PhqV*Bvb)Z|w=9wxLA?=b ze}s0@Y5s1mMiP_>yK}PbHMgD#cV6C#XE24f;F4j&zmGZX_Xy>hI@On3`|terkMecz z_)X{pE^Y^esMZuczu$dQHTVTq(f$kD5~H-6t;V)wWCLZbMD7LHELy3}th_v5doOq@ zUx#iH0BKCSYT15t*^OOgt!VOOJVcN?e^Aoye4`yD-qX|s7_Y)nJ0XalhC-kPns;%} z8z}rB+sq8BGjYFfp*qwvLm4D9&(_LfGD#$VZ}kT*7u$Wf()kaR{WbPA+}yO5^0~pq z(XcDb2UTwpKBlDEw6kgFx&}?n3A49H=EB<|8-$`g{X-OYHAdPwe{=KAoLaxfh;$8MfKNX zOy%&_m@8OJw<|WXods~&q}wuXahzEtPHdp~W5UA6FPy``i-%tFBIO>wf126@sZT>~ z6!Scjo^LhqKi*%@rU0@v(#vHS+7R1>+3Ncj0S{XCUSm@$pCk%|jsv4BEB034!kfdk znJT1)3>39fqs#C&Z+=szF2{$4HS>0BAOPW_&6Hk~1DUli9tx9knZeYe>DT zyj$2$PFj}OlILN=U{T4p17lh~PH+*`ORa=JVrHoUhxb5Qn;n0b#UMJU< zVY0@%Uh22)j9pWuX?MqWw?p0GBT9ABSXEbYUYNll0M8+oC28Ewf76J-N|&wtzPsTE z?6R!kyhkrFVdHa z?{P0;uKpobisn}#Tui$bNx2Os4pMp2LC>U7H;Nd7PikIGOo;}U7}6>;(j1PzR00EtOz+FUr|M4c#NP@$y9<~N(iNsPy~Mjv$L(y- zyHk&IYf3M4%1_+Bv)kr+-ykutx^q~Ph*Zj?3!(ZE*6tp8c6BCTiB+@KtAzvB4lKgq7vYNxTu&Wc%i}@VY8l_Y(lUUTUPhVur^X!*)6y5Y;AS)nuH%KsW5*ezGf_emWms&5SxZN*|X^jWBiL=)UIU zU${%dB|dg#I|aJQ^ID@Xd~)%e9jANZZ;`r_v^#$CGI@K|&ht?Fvpi9K#5HK)vDf7} z_K~7?e?=fQLn~YVuGh{pCKD-XK=@9Gqmhn+iI9QeL#DX9N66$_e%nn)d5Koc340>R zTNn*QaK7%C{}+n7<((aZxU>#S?ak@*MP}der&G+Nj=2Yi_CkJ)3vF7^c5|&b(N40Rdzcu5{>4&zPvYMhn+Jsl9Cf+y(RaC`G#s ze}7zMMee2!JI7*HC1ULv);95a*I(*{5uH0nJ;|&Vh;8vw`yH^Q*5;V?mW6-5(A!ic zDB#Bj5L@;=&$RAsHjnD44Q0}4$*#jY{!>E*mmfi}yLb1EOm?{RujLLEyRhek1hL>3 z$af0rDnTu{uqTs}Bgt3CD}Alb(l*i+f18QKFTa(QRI(aCzIm}17(Zehbz|CFe3nE< z@4XU^YU?J9H^1m;g{)%#S%qxR^(eIxf60mcrOaIJk~328hfEKPA%{$ zi9A@RhQDSwO>ofmq2gLz;p1AuxUkmO$8k%%slJl9^ErX4N6&FYc2e;&6?tWYf4V?k zayLCEj;NZ>WkVgNIq1pYRaqF2@~>)OnIxxiuG05yBe>c3L~id~#RgIj6i;`HFazHxIP`bXAp))JJ%1Ql++u zgMbztAX~TUNxPo?*sr5!_wdK{fA1#e3P#1X4<{LAqi%gFO;l&#!qNw_Zzt?_m8epU zZPltM`fsI1w)U0RKerB;QZ`)DzxE2wT9)+--{L)j|- zwPPKuxefk_asE;qJkX_=x06_izh~e5P5HZ{G>K5(``X2E*gog~F!h;nOkfiG%llEakB-KblKmuQJnvd@GrL-l6SO>4%ove5C!WA?&@Ml} zML>Blp>yXU(gk|zbPq~ee+RnIerP{@sb(gP+QS{0JZQky1|Q_sjQmmKhQhl3zAHfz zhq5+;q&4$0GFvZC+Vy7NF56|px|1G9kHvTr=77$-O7CoOXmnStn_?F?qnb6}&Zb}L zG;ql}e%pS19`AWqH9G_3Gf=){cjV-;()d|c9_@$+@pnygH&QUPe|vGK-PL%oICJyp zPvu|JghsGPQl3`I!QpOG-dvQv>{CU0&rW?hWYNLMI_n4^>es5m*DB1*PO znywl5Z3DcdDc{=(*HqeG^21h`1@p0g741PV=sRi%;5FfNdhJ_huSsOjxSsV9xb^Vi z$nR=9)6#JR&p)N{f9-e8x=Odhei^y({Mx&Y^x!2~*cw4b+7_gRsVlI?G#nZJxbH=* zLi=LvgOMGGhsDwf*7EQKO%N~fD`>gcH+Pz7@h?pAge}5EuRvAWnz&>OHOo^oXRj?d zGmPsGr%*UtQ$sE@2{Ce)XMBslma;&ny}b#VA@mFWm$*2-e;ioH$Jgg_j!L4<n{Bz3Bkhz__{Wz{EQP1i@NA+%6g;X6=Ytx#8 z=^%tO-XW2E>k;Hzn16WrXg~M1Vm$TjPN$!e)g-+GL0Qwn#E|{4(`Vew^O>rm`cfxJ zOVn8Eb@(vQWvx*IHYKkE8r+s0U6Gc)V#}O7f28&h{21gp*YB%hZ>R~ZdEfz9f0TGmk9iFE9Z(_MIVk_1ZRJ@FU09fD{p%y zmOH*PqJy<>2c6#{Zn}wRZUvo>S-=HP`|F09`>^anFJRl{K8MktG=*p!JbxqU7F;-C zu@2td-ggH9Gk+IUW9vIGDC8-jdl59FIj5gCHO2M!dW@s+s>2Cq<_67ZkEW|OCB6+t z{+F!pT?E}0$)5epu#UmYxx3`F$@zZ+%vpQf%GhsQH z5Rt^Y>aK3V=8L^y8!_dXn`<2s_jJJ%N-cCC6FI3*!hb>@Nvh4k`K|C=i$>=XV`=&O zk@6R{S9u+e&W7(B5=&osDsKjyGo7IARbG%o?Mk)jPHFe#^+mm|yEUe>`Hky*gQVuu zJZ(W9jX0v3KgtlX)uJy+w~l3O+LN%zSP@}#GL*5=V+qFf;wKVn9(dgJi&N~d&4o;jU1uz2X_46{NIMt- zQC@Cex=+Sb>e~zBOA}RhBOdi-<*v zt^vOTJ!m>>8mzpVD@N-vmr6 zF0Iik({wVR8%QM8ei;xyfmPg?$8b65spd4o z2P)a*Y0IEL(}$Mv)L!=%cV=>_h;yJqiYX;tdaLcukfy5j^lFTDsOH@`+i0I>iFEs3 z4lwQgf8776MPiktI}P~t#Ujvzn|k5A{^fbBXB)gSKHs6bW*4Y?d4sH|zGW`SCx7>0 zu=*heKWlFNndFfFx53;-NS$RapY?Qwx{Qa3=3OtdDi`lto5ba2jC}k>mPr+6+3vR< zei9r|!57BlFGzYshD`!i156t82>P2QvjH3J5c5tO{)_U$lanYvFC*o(#%DIWcYG~m z`vPd-dfyH&x7$5fMBwJ`JDy_w8-Lt9>xa@g+W)C)kdZ+B8U$*C5g{v+Ecg0e-aW4NBWz&~Wdzr0y5_+b;S(DvA~(8u`MMUjDnw;t8?4sF8s*IwA$+bxfPD18{}v1L zD~QhE9a1oU?UcOo9z*MU8GmnKFgT=NBNB0+)Kg=G?1dc=7IGjKydEvAwdYhw6iqIPe2xDx==Q_t@4jpSaf&cwtn~n;QdEBqybx{9PjyYqkmkd44R9A=&kOL zmDfWQ@33rfy@9K8$|LJ`iKDu&BvE#}mG3XYq>t9wIb#c4UNljE2xcb--c9R7Ov^&U zCnf1jBzt-_ex|c-hO)3_67aVd@`8Aoy2!S|?x)Ued&w5ZV9$Pubgy>@w?@gkH)Wj{ zQMmkANRP0We%w@xyMLBgHD1}#XE`wGcYU&GhYM?RP3;BvoIHjz!-|MwH`WLe^UD(S ztAHVJEl&srX8aq{DCz0=Jti{d`O%((gY#UEo?q%`M5LXk5u zvV@dFqXsQpW5w4|xr==ubWB!V=BVSSU8JFy-;)0Gkz%lNS$~n_lMY;hT?wD72$S#0bAgY}Cw5Satc{6ZEd5Fs1*vjf? z;-Qu4$N~;QSqQKo4bAIteP6UBLTl8%s#rFV;bzh9;j?Sw4NlTS`T4*_r2NtzM{@-YQR?dkao6m zDxmHb$R1m3naCzz{q#rcQ?%3KJj8^T-s~uJ-HQ>XW&KYh>%oP|K@#~RM4xovmXP6e z+-yF2vi!4$^Y#Jm6|7Uf_D}+pqO)DP_3LaM^nX;5jre`;w{2w{xsDHE@YXInFJXs` zD9mU>deiPJ%52z~SG-V5IDA({erRRy!L#x=xeRWRO5)t7XJDpuh7n6Oh|p$ppl z!>4}_`gM?T7Qv)C?y;~j8(DnSBWaQStU+DdQIzjf=Ylq|Q7qz143Y9>*3-N@?{J5$ zb$>bg4g8h$R>7@Bz_DGa=d?>3`%as<%=SQ|*VA63Zq{_GfFUcOMvFL8Ur=%hB1Cx? z`x&hk`{N(T!%4lxMQ4fl78$(7DOSb}XGs;~wAO^Re8f=4Fn=H|>IB}o5FoqW(aSV$ zBD&{ELAd}N@;1)N*IBD$D)WqQ<6le|SbvE8BGm0>IOmTB{GbF)Aoj)xq)WW;%992f z6AT%`=ET<)I*etFAvfVTPxX(4M)e5P@ZO{!89B-c>=>C@v_e0RNe%H9BR4ud|WGPh{1n9x&q6{p)sfI=};M8Gkh# zxJKjjl+`ll|4gO6ANpB1@d^&G45ys86HH+_3BpWBa?eS88cO(c2nQ({e3B$Y**H3E z>Fm4hpdTB`xUPt+AoY7RG#qf|{#NjDqeur87K2ne-w9y|} zNh71z+MTfL{z8cuD%j?}*8N6?C7Cz<0;bL68T&I$I9t)7gsTj_b6jD-Z9doXQH!_M za-Be|DT)Z*^UjyzM!BEDUwfusZr+-iFi73`;(;1)IS&bF%M><2{ENHH{6LzHz_I0e z*PmH+Oh^?sV`GDn&4q*{cYht09UBXt(lgDlV?$(GMAL@HGZ3YMpeb0!xO}dlV{@pg z{o}-rr!gSbD_-m!Lz~G81~OzRiuGrLLv9WUM$==EI8eGTfMF;w9#eB+z-RcWO8t;- zI)$ovDAb2>E-BAEN6L7XGZ(fl!_MN+4u84o9+lWvfobgqN~x9x?tdGH15JHMI$GEi65=dv`%RVEMmM423n606 zUVgh>X4>+`!5!DZFP~03tBcVIcOuxAdZOG0`n+vKDF!Tk;@>qNsj32$Hsn3Y9TMBK zSi6V^m+2QlU@?A%Mt_fed9{DH9FMTPv9Yy8GY;-_9za#gTJ-@hDgd^6^3BSH^Up=W z8CJu@w(Zum@2p_RS6>(N03Pd6+TUtxVAtzV!uON!&$75U?Bf^Q=S;qa5xjZ>0V%2E z84EsDMsP3=qC8x(Keg0WXvdZ#S~6>T+a$|xSpq*Ua{qKen1AI7km3-105L|NiyNZ~ zJ-9FI7SNzbCKv_%(pzx`7v(P}ta3#VKI2>zC1JgJQI{3&0L21)v<{TpTQigT4Z#IT z?@+@wrUM*D*mUcD zl~jb2D1M3zQ0ywzYHk(bpe8~ehAD?SG`i1}4JKH); zCuCP;A6(gtF;m^%r&p9o=V*04IMBEO#VVX&c^})D34gJMPf#h%4@Af5zvatYs+fI* z=3XUAZEsl5;#H3Oho03|!y$=Nn}fwZWMp2`kOLC0w-@8Dg(|D2auawHhI7M zZ-jpDdVk&#X-RE4C1#_azGY7Jjo?6_!0KVD1!A{46~kLBLdD`qsXoX0s^?URiw(l_ zy>XV(^na{+atNy=o;`geh-2ykR*n*1ljQwn zOw!>o$$L0VgL7Cln+fE!^Z5qaz;zEwj#kV4A0K}s@p8yH>kQU1Z0oRPvmVg06zTuy zFu~bTS~*i^HFZOltgH`r}W-O@|fc&O+{_7cgce(!rFzyxD`+9r7?7dHj_F|TqK#8h` z(e46g`piF1Y@EjorjSGgvx{K4+iQIa9SXCJOni+J&8T^bo;*Qj9&?9&WggbRG=+O< z5c&ob3z36Q6z`s=75jZyUX(bDIrkp6ihqsE72J3EQQ9|E3*(SAae&cN6M+HiMHbv% z`7(NSLv=z{*-v2v#$nDQo??%6J`63r=h#=ll0njEMPticn?o*`dt>E_zk|Gri&TW) zraXn6obt|Y0At-4S@u}MQ;+PA<5-@P!`L1%XPrs|BE{DvSDjHYTg<;R(hhVC4Sy=8 zZiHhND2-!z& z($+WHV>}necyCJF8xQq;&N*zqXj|Xk%+`*O{<2+h$h~}~*w{B|jHPxEjgvj7Om8*Q zujK1i!z_rBPxfRB8tj%93bIGR8h@+l`~C9u%$gRK_aiuG;nwUB0r;$fHQJA1=>stm z_(_usj~IHHQ;%;Of$$TleCT}uf7Of>C4yM^kIyQ;smhuW)yTb%7o{5Vc>E`UX?vlN zM-Dr(_wyf{Sis0jTm|mZ_)|F;4^GKh?mU((uKxo11_aJC&lM{K4^rPD?|&{#EE=IM zOCUIULa34uEX^X$th`bG+PgeX@hW2$O}L3d4*wL@DWn!o3jLnMxjI@y`X%>c_U&}E z!F4`h7UJ~SQEGkt``A3Lw`%PuS3loD*}eWSiTxSYAdv;97qa(J+FvU_MxyKUtS%kJ z#G_pDT7$-j?QOfwS|G*Z~59A$iz(&Jtox2|g$?kkn}*G_lLr12+vY-c%l@w?CePjKkW0e@sQN$aim<%&F7Cb`d6{#*EM=N?$_C9F#kgT(4BNZXSA&Mg-Ri$Wt= zc$(wI{{9;rJZ`t-UGlHs@td5}oZqClXv06aZA05&RyR5yRe!uJ%s>u$*Qsq*%lc_J z`1Ez>l`@CK$aNt7ecRxAREMP$XKge4LtRw5`~A~!6fl0KH;<~-m`~&|3gpjO%e#~l z5RRW5{c>A`hUO{N_vh!8n~C>1>MVV0kMGXiiXUXLyCad;>q%r*BDoe`&dM%Q{PVXiqa!HQqO69v-U>&+}ifd zRwODmuk?DTqEMek#OOykAIGOY%Ly1bovdhbp$^UKaetUj^ebHR1ax&N)3mbGvI9%I z_%S!YdoW$)O!39i$**tu{(cnSd=9_iY-AiBt~@)qe?J5OCM!Q&9;Pv^h+#DUT6X^S zAQ@ueAeywM<@wMSwNSK$rgP!&xPCAB>2W};fkS|?9~=dbHK-)SX8kTZh27gU%tWAR76W=cB*)F$7D1AD+>6)u<*Ox%ebHA^5?~LT)+0FiBKiLjmr)pP- zmWi_v)3h%yth%O+v7*C|U=~Lw$Ps>NTH4TKN|-HskHacWXWQ%bZ-3uL?1w&cUSH#9 zY4_s~)L|Pby{!uqHiU)vzY*RDOc-qFx6`8DD}O85c;~XwGAK0&AT-_ACt|q03zdqi zf4&L04%ezbV91c5K;5-k!oNz-?`GnG3(+I}k&3}!VAz3kRDy(dH+b~ z+}~NHhkuXM0onAbRNgj2+L8EpkoBvA9N2>z&Z3mSNE`N4fq_Nml0%a{w*&kEL({+e zwtvnG|Nef-Mx!fN0j)}eMa}#$?l&@~IA`9O&bbrPRx9~tgiVBf-xf_)+TS)+v>BsrkC2`yS5=nP5uaH%u>4t!do{e-H8S z?^JVfIq1RhpnR&b#H%tx)kxig0)}tJecSc>_G7d{w$S!ejz`jxs4DA% z)_V|2>LIdXGiOAnony>5rC%Q(nSVx9%jYJeAa+^X!&R6%wQ*l(@@k-JN+Fj}UE}YH zfNbdxgYcKt)y~8-%O*0iP~U_60_hU(+J=0ma17%wae=u$sThaLhSggOAtt?+bY5r& z47ovVUq1x!+9_->cb%gGQ`DKL_^-tSLVP)Mql{2g3j%z-OEeab06LQys(=3;*o@vC ztDg_W{O6aX_(T;Rl7(GirFu%^`I4eC{y{I9rW+etHUqGYNp(N;ok(fii!Qys|E0do zp8!0(SCRCGk&tXzT*iZqgJJzl&CWR~50z@2o>vd~u}nL!w3^F%;P2R?ZqWDi%pc14 z-tGxvp-PfEd?u+q^xnJ#)e;}=u3qJ__rEh}S# zEIi)6KA#%TXK|Y_jRL4a;)U$Ga+v=B6Ko8vN1j@SA4^D$0a`)0pIhq)EyfP@2G zI>^fmyND;|(@7D0`l!bMXsZ-%8zZfYa`#^|!=AiY`DsI*ovx;tfIPR6(|dFh`!yX| zkG(k}u;A=8wt}_-w|{C6R)!GXbjYtzaMP1?XGXxuxpQj6B*O8<$+3Ye!-UsX)792jOKPLEMJJ6 z`J|TJ8CU}^%zxxiziuN{xMO+)+NnS_Y8NuF@nqW3*!R*Sy-0GreCwG9l#BmP-c>~? z@wE^bV^fgHBX=;^HHa-iLkRxPvf+immn+6+=M}-c*f_yE2ahXfagylfkv| zCq0C*;sXynNTPIZyGnLPKgD7fEyN5{rRBvY-M}JY)j0l)w)In1$EEN*6HM(<4 zcfXxv6*N68TJ9oW0iH{eixPJpgZC%E+u(y{LVUlmk;G%#hOsh%?3<{Lg(hckc;#W= z&z5rni<$c)ULGTw5`z7x2yezDCf~>N#|_29x;K@odxnO@%}LpRY#wP`?*k0k%gO## zFdupy!+*Pi=zjRH-RIYfe>gCaeLRWoOGBoZy%6=ItV_QBvXSxmXr;kg7j?%&4e~+> zjnuP*x~FNtqD*TIcSo6v@AIn)~dghjTHW-h=b?8@+AkZ#kn{C9{@Fg%73>y z(+tsAe?R{`p_QY+n6Z2)DB9@eqno0HfQMIieBWjf3Onp~FVo?JAEBSuhHlrvd(Qg@ zt9JqR13}=mI`mB1`Ud)4)6buB7}k#rwCIZ>38At;@Z+6~4d?jk)wxs;j~N;K zvv-WYRfUZF)+Z?Z-SqMKD}J5lb3dR(skp4 z8!ABgbp5I-Dz^H5DVbkFvb-OESJCq-bDbxA_XuJr`?^e33&YMq86@OK`yPU|*XLJ& zl0y61KHWhIFAC-<|3TwZ4Eg4v26%NPcchNGdLJw3=jqws7txQ``rFzXYp7Q>DZk3$arTx7OONy!(_Ai2EdQ+KWvGN{V4}K!gjkBqO>x(#@eGlQV z%U-!rN()Nb_?{sS(PGr){`vxJTXag=e|g{^K%o@`-)07_9Nho>5_E8m!VrAIZhmsc z@1#!Nsw)I{_t!t&XyjbbD?UVsYMI|L3-m4ENb37wE#~*?O;Js-7sKXwz=rN+d#mZz zeS0xiPZHj+E&VB#fV1_mF;yJ_wqgJLV(N3vFA%n&?LkhI;ZN8or|7#oKA_0iPqZwe z?Q0;uh2$GcVOiB?@5OmRt7SIMuCFeXf+V|6yw1IF`wS5f~T?Ns9 zq|4Bi(6-NAYTmu>w78nEU?Bg(UC&`3h|?gm)|EGYkw+>L`vW>efn@&l{q^3#uj0P zYErc3SGP&YAN>@sr*S3Y2hfZ&y`W{09}WeA#o77ydK&to8TFbM{6{^*3_eaw3t-@Y(W){m*sQz?7HmTBu=OC4krkYDbAZ!yByd3eOWghdb$t8oWknmK-nni@+(3; zv3Y!9y}v)3?Id(2pfq>P3eOc_#su=jkH;MkDOg>3)$m#}zNpbTRQIqF_I&(}sR#~{ z#rkH=^fFx0KPOds_u~`ls>BMx_TAlvb;%v=_BCc|Q$^?ZTo*b_R2f)i7KUb|#yWLU zvY<%$^iXnAK-elNR)v?KDI7|FevlUO_@^M3EZ{ZsbwfuuZ%CB$WD?i)I5g9pdPo;x z)a1JH@IOftGb7XV&i{PO^;&loGJMJKX@=75)q_zH$2O|{UKNfLUow5k?RpO^YGGqQ zYg*~Ryjkk)!`!d}i_^d=-by-^kOQ2{gi&X(}8QR*yp& zCuQ$@3!bM&y*18!K7IK|NJc5@J#1Li_v;T~8V9eye4IKwD44fOZE@#x@q7I6c#I6L zZA)BG6Ma1{Ni0_-o!&mO8~h+*_ub(oQ*_I{XZWBOr(BTy23%pu(LAOQIw-s}2$`t{fLNrLxP7}+m9 z@qCyrVR-xJ z)4JH$r4d!Ft{_#aCi^v|R8U9+z3YPxuKA!M9kHh2uo>~CeXH-q^fy*0Y!JMe zbJAA^A7n&1l*-!r^cZJ?a`CWzW_3mNygKxLYk?ByfBo^UCmSce;e-3e61D2&LVx#R zwl3K8?QJWFN7J!x20nQ%!2O{*D6OK1>O827{J*M%hm~!`$pZg5{z6nqswbv;(CUD# z%+hcl3d3}_*ptKOEH0P~CcNdkm-?p%_XvgY;dm0^KV0`^e+>3WRh6kI)zNg`p&(twN2r-Kd<|~?myGl-mgAn z>#iFBXDP#vz|WZa2!7a$Y{$?~6M}!eFYvtlXM*-@m;id|EDmg|b+UBT=Z8yF=QTl4 ze*qTDzbpYOUN^n&`8L77vA((Fa}+t?y-xI>(Qi$&;R0%^oE3AktYVY{^mVvJ?w9(4l*4%`l5<(&~kMnvWf|xC#M*#PI<5}E?ALvl!66Ne`0Ik z!TkA82Sx)*Xjo~ML7xV_L)Eh}_s#&=_saXSWme{h=r{xqkMMZf1OVCyHbPAtm>X%)0b!f18*?QRzzD zm_LCF0e%(U(Q(+NAbAm7D-2KRe*n#Z7{QG9?)~U}IUdh9H|AxLXj}AFkS;#R^$YSf z470yEN!Hy{oUG0Eaa4JkK3D6n>Ssa4uist+UOC02MTtYr=7Y+Y3gOFCQq#x#5AQ=M z9Wp%2fKtta&jratS^9l^?oyTISE4k!=HtSd@}zkyP}Vl*q98UG;reEPSJ@kTJe)PQa>%Dzf+xu}*b z+x0vGYA>*b8x)U{I0wCRFQXFjvtX(4_mtNk9A02~KhAeVH})5WnGUO7%lqviwdX2p z|4yK9v0%1xFBSCLqpO~ufAF`jp^~mH2)p=R(+3vPMem>SNl3}(=Q^-@2d}X@oBN^+ ze1fMQzW(S{rtot*s$x3ECB^h2iKncoGW}2igPo%HU-E-*)%)qPdV-^LRi)&3nDZN~^99%mNhBiG$X&>&L>QNUDR_e{#c)((><vl z6(@#jQpeBd*_G`Yh=o+W6Mb7meWMC1U5@2CHu@}Q5`r*>V{RW@r@X4#6~dajBOW&n znzQ)!UFIalL`9(_20a-`CjDKOF&sdD>!%Y~@ja~U|6beoq&J9pON2}ACkAZxt;QNLz!-iSB zpMR|)NX+~dz0m5sB^BIT#RY@erz19IzgJKxAh?#zw-klli4u9t7+gJ*WfI(}&OaYk-H`Cn-Z?iZ3#d&Fi1Z2zs0sB0*?$ z128WzxuiFjU59@1{CqtSPCW`~C?qrYLx~p?75$G{c_zMp{_!kADL@Igf_K5OICd!# zoGI)BWx(E>>3&A`G{H@bR*lE=f5Kl-d_Ko{E%z2|5m?j-pZ^;QaQq#j`#i)EL98^oO3u=TUTuj0Cvw&lS+|ewZ+A-IqOwLVN+5fA;&vpQ{P% zk$3!bY#v%LLC=r;!N%cwY0g6enuI{dVG3%a4BL9!rc+7Fg0Z6RxlTPYPd@kC{T_@! zQQtwAuJ}gFgLz!y|2xk}pdy7vpapG~YPug07`jtHIrP)+ih`kTi7cL>gbWV`s846Q6&ky^NVSjix zsZ@M%?M}n=#{&w$dd`_XNE8Geo@c^95J#u@VH;mBuVXFFkX28g3CH>KBRv@x68-74 zc|}193%W7=1LghXzF&v!D1`E?$qOA8z4d~OBO7`;(S8o_m#%Cle+BcggK{%@JqvRo zfj`0JPX`iRD!Ma|_07iTM};?*LNY{#*!;ji?dyFW}Xce7tR zet;yZGRQpMZ_=k>e}OZwWng`8-Li(CFcOS-9{G8;lKy1?z~f+`@L3FBlEpew%^E4ZpdQTk zhPqqHAuzpCf38W}=Ps9GsV;oYo3mZfnZ<8$s~kWV+kF59NnX6{e#lqAzv4#eurgmS z9al8xKlisOe%(*?sxTQf{oOnCqn8T&*LiNLX_a~!?h*6}VHS4UF&*J#6*1Ua+uxgS zjp!4u$-ZecWn|qC$XlT|6)rxnm`Gq$Qb6WFbyADPbiYn~2=`B;; zp`%$S`XkIL9g$BjL$Qh3vd$G%Ore7|7*}U~zlA3H@;tL54uTuMz+n9d=Hlp8;gF7I zaSR{tRSzUJc>mN#I8rKYtlx#Ta!&7`zRL5CC$K91qVA;ki#|WJfCpR446V-lVIeT- z(E)*~e_U9uTyKJRsJ#3nRVhB7M!#&~YQ;(>_anlmQS!f8HN6mC@jdm7%Qcp6ZL&v? zSex%{3yE)-kK6YM=8G~DLO7Fng|rm9gui{RO5m`O)bcvuL0W;udBep5m22r=u^mzk zc(2m3_svQ|ngBsA^l3n}LaFGao|P56zrCV%f05&j0ydq!r)E!5H~HzQ^FQepd%dc_ zd92Ym(1|JRFTjLBzZ3h$vs~w-LKMmd!A!kSS63fD=pIFtim0CZ{-Db3#F<8|vAX`t zBqF`~ok%Fm;-9aMOru(WWrkyHG@V#@!|0D-A@b=(VppmnW-GtF_MDf;-JAvf_|n&( ze_x7Qfw$!!Y$)GL?Z}KTB@0387yoDbdsAl}A))oc%f(XFe11l#I=%uNpC03V%grfa zxJtix7x*;DFbINX9qIb*Tb;Zof{gGAf(Ik38+F|0)!&R?J(F~l0rj`){f#|Rr%Qs37lnCL|C)DX3xOr5Yi<{n;Le)>;;0h}>0qswJ z7iU@3JpwZ3YMwHuQazn*Pvo-F%e9{Z zCYc>+&-*^E7EBJ!fUMhRxRTc%ZTh`wsv!df}-W?_Y6O_fdr`h3Rx0Hg>`we_>hQyRi9( z!p2dCNNCY7*G*Qns{ioiUh?-)Bwa^A8k-RoT-M&-o|mc$s@Y{aa?|vdj_`9?g!^`n z^E{faM7k`9Kb(V%uPWsiTQ!fq?)#)q$3~gf25D?36->;gE!|fo{DFDq_dXVagZLYi zYWtjof{C4g3WTMqX!$+~eJI;h6yB7eN;m-NmB%riV4`+Q-k7|z{U)a5narrrz_l1!Une~&-^t??$%dN6(+ zmXX)in_5%YraS7s1zivvDOEi#`23j-WJQLi5Hiqq|NP?xJKA}Lf-=xbu^gYhlPNO< z9CgFK{jIIo9J79Jw7D8Ts+&S~&9~-$?&p&!6m+mSpfJG~V9`fv>8Bv1_%!w^KJS^j zQLwgtVQ^j$>mACwf4Xm~OQY(U>U)S0_I4jbHcvbNuQ#~)onw}@A^D6Bg`3;t&|7ht?MKT5wI^A zaKVZS?~b!uY69oNYvx1ACGJPyE`@uV5_T%SUzUZm@NPj3O*=Rrcv55@!6>{UXY9Ur zvQ(^75cO>HTJ~Hw%azP)g<_QkUDv*$d7|5`qSf|rnZpKXk_b~OHz(Iy!oP9x78EU; zsUSQ5Iq!nwWL1~%(h?ef$69haAhutIMAo_r>yLh~Zt7sI#y;PJz`{@>P3BMhpqrfG zXXp6p7VtRieypsFASh(&KkORA7oZL(m;tfU?tSGF^bvGK8_Rd*(*VWL6dVIR_UZc? zs*w|w_|Jy1LTV3whWfrm=?9tZ54R6p<%FKJ&Uy9zhCRQMub|C;&bxd5P?aLU50%7I> zUg3E(+t;PuC$g@Fw9r~Le|oAHIaGa1q>+rjrtjNWF4~jzp2vOCRUTFo>b|sciZR|s z&Oh|5UUrMZH?P32IU;nq?)dtr8F|JiaGIxnf!BF^BAA&>Xd0bf{DdEc%?H2xIF6FD z_u*zouYRz9MPl=}j$WU_p$;o<<#YMIvue$I5r*jr{!pvjH@%b~{2++7^5^fIYI_#y zqZt=8#nZ?+)%{Jl7OR+fotLZ2ueYfC5e%1%$fWRP9-kB{AJqXKUm(X{`u731$nwDB zhR?66b`%q=avd*y@8_sk*1d!`$A&b!A-{-ZsIZ@xd_Ej3e^%#H9~Bn6G8f`c3!4#( z$21^>sjeH@=Ppmfwh4np*N-)Q+ex!psa#hk2hTI*DhL%+!A7_{^(fjCB`Ose=+)3)%5AuC_b~;pUuy8p_c|u z)N!slv?}`l=Hg3*I4miCR6%(OJ+&VbVkI%u^6h>^^!4fAFXNTsK%xf(m~ARhaUcJD zwGOi4Wso4vzryy_GNh1KX&@=2e*Cb~9%IQfJS{YhCvqmeKP=r8$q1uoc5xXhVqfetGy137n$Yb(g%7$*i-~1 zLdD{=y-q1$qMq(~Kkk7=l8^>Nbn3i@jw?j;{OVbJ_wo!C z8B5WTZgOe9U)G@=>8}`N%J~)(rV)Jk!~El5e~BU_zqJ1TJIu@nzXp9Ti@dPrl?Nz#Pm>Ac7jbW-(dqv;u$AM*}?S-rJP zrTF0?f%t%~Mlk>Q1|M_2NAU%RA`JBE$G=}sfuORN^@rFj;qaTitP1cGK0bfmP1z51 ze-rww=(7%-)`%uI=#+Lp&q>MT#GD}Vfg|ucSvroY*^iKvijDpGjP~Ke9F4tMn3SsI zP}X{aN-bLF&GR{n;!5BfsX0~Ks=HskX;l&W)NBL#-0!p;?}At!;Oh;1E>!xfu2te_ zM^WGB`+}1(bkxO;;T(ZapkUFpPg-vh>e~9N?)C z9-FBxFc(6BX5|Mz;ejBnhEPabjC@$p#$Z6vxgfvE=_aK;7j zh9QCJL_1GaSt6*Ah>G{jvCW}}=JG3`6Lg64=kJGOLqL60S(K(b`H1m6oXTJdqu%$% z#uYDw7a=eUzORk^0(Q1Ka(X5GZ!e#hZ`l$eLPs|t3po0mPGuv{e1CgNtsXXG@lQ@h z4r?QSdW@siA;9c9N^C}~N4PyNh!sk1@5l4tsRyQg&ouqX;`{M}x%E7k%t9O-e_m;P zm|i6Q=Y%Dl76q~-{NnuG6c?-E0ydTmVeTyx>qRtp=m7a?O7u_k2UX(9$ysp(Fb8F2wxd9v4wrxK<7RkBB0|= zRgs4To+mtC^O0xE6l2qvs<@odGQs@Ly)IMo6D9`CDwT3$AlmtZ^s znN_6m4He@mr`iV)|4 zmdE_33-^~g&*mCz9?@~sYk8@NoR{0i!dsoewfSWMB|BhNz~u^y z%=YQTToi?SRKXiRJvZ_TfASO&rNETGb1FQG(+U)G;xqb~s)g`5Q`uM_TyQO}semj9 zm%|)Wsjv$;R@*dEJ&!TsJZ;{Y1)Rcam zAAMKVSMT`KJ*T6Df0p$HjI;Og9)!-Hjf>KQ8wFM`#9d^(YnsYYuH>b$~A5j1}P!+YjKp;Ag0_$MZVDdA>q;xl|oJWti$F8xvoTj5B*y zNj_aiX(NE2p7YaNr!K4LWf(}npA*Tx{pH`3I}5OwcCx30M2t>^0w%>Et4`tTZ(xwf z{v~||n>3(Pu7KVG8pQGA{l5s%!T-ZbsQ~UeL%1r0f6d=o!}~)=@Bd{h;4%NLO}=FF zpaK+Igjnl(mUvjHY*t}M9*hd5f>%*#cGnjx*;YP3zOEM`boel`Dp+gaOU$wjcw4u~ z=L>qR?~Ux2u?1neje|9KL3H-#r}0DWG>aT}4Kw2}p|qq4T@AKx$2yQ!eLV0w&i#}|d=wlG+6R8$+D z_fg4bdvqrFFD$#R%o?{)`qZ~Y)YtQo0zreXe}ZaVdEw_|Rn`ZK&~>&BuKPER3axZg zjWH)duP-Q4tE}BZYeFKv*HIMk7uE~=L4Oxr2&4BmecL;9Gn@0iRDUsIN$7^sry=A9 zft_|0+$(!y?bFZE0i__Kc45C;Ax+hyE!9!1rs@2Hb%hjJFvY5j{Dw{kvUfV8D$9~0 zf0p;T6lGBx&@Y;%f3?|*d29_&nR(3Zy!x)2Sb0&g^WG+knuxt;L?F{{Eq!x{!9aJ@ z)iqt0mpPp>=R*Qi?D+F1X;Wtrlx9N#erw2+UiW=PkkB5TpZ+I16XkS%1yTfMr>PsDB<(UxKXJ-;OQI-k9l4H#m z=7&EQp#g|C5i9Fb@U!m7$lt0E_~2OXdfb%8tU#iRfNC8S55=K?>!+?|c-+shvlxRK zShKy)jZ`zkp$O#*5u(35bD5Z-`e*xK50_2g5+hPiVto)nLh=k^+XMZVfw7yOjcOw2 z392qb`TQ5nM-ipl0hwE|PcWf}b{%Ic3~XCHn%3{cIcp_F%%JoszBJp93Fq?#lr46#Alu)`fkE&dIJKWp?|uo51L3%RlQ~UL)kizJZ#0G zENt=R1)=H9JPsCvvCD_=$)VST0)p8ZVa=zXT)DqW&u_3{PSF7oD370xbz|*DXhmhaeuRJXG6X)lz+2ECfXBH) zlOdd{^90}#^m3f700@snIE;D=O90au(zWpm?(7V64m@*;0YwRg86D^2d}RPgK)1hribL~>{Q%oHU@ukWwO|9&MrDGJ z@1qJqsZ94g(Ldclra+N@9#v8(ver+hGNUff-tjThF;*s2KZGBIhon%_gX{AvS(J@h zir^^ydvGyICY!Bo)hIa3*?GALO%M;Z?J?t@g6YZDwWoEdp!wwmCB=iy9_6ghf%Q8OX=mW=1J)?8ZghP2o^&Ogex{LljSe#U{(|rSf$C|EzN|aw^(Oxx* z;E>M|M~4ui3sKMyEAY&f^6(eBa4i*oy$_P1!BU-Yq*Mo>2cS>z=zq`i2`I#W=F6zhsQjyXB80Q`JpO#8hrprY5NeVIb91xv$q(CtH`Xvdy`VfoI|xad4hFhH zk#wiOq*#W=xZ|~d`Yx;mf9X;m!GxKVX-nWhcf08iUB{ZfDX2Q}G6_ zyRP@DSaKDTIQOQbPChamArz(<{~&p2&*c%`Xx2AlfB&_;82ozCR8xvO>+A0-01h`+ zhW8NAmmfV3{kc4$VG|X?5XF8v61OeoK^690eK4#OSV&Q}RY13h* zskoPFMrFzBJOzcb^}Z9s?qj^K#z_?1Zx>aPvxf8LE{0r4>E8%%WbC{{kS%Sv98uw1_xO1{IGj`-?Jv(ww))9g@j@++{K|{C- ze;?*SIH13O66Uq3Gai2AjG)C`cFCI(tzu(Of4|!dqo2Hh$bI< zweW%um8QV;r=2aGBXlmnZHADx4tVqR(Mb7lKRQk>H)^@3gW+aK{i@z;qqQeeDzEQL zktauug4Sy{QwIYm5F!SclVSO@bAna@mi+J9k>_52RHx!cdYn_dqUS++OAKzTv_yQF z?Q9nPL(>6)srrakp?ZoLo|e$=6=ej5Wf{~s-BORLFZe~3XEcq^Sc|Gcb3B2u-+TrkIJi8kr7$xX+zfbHzuW_rYS}+5k8wekiROy!) z$*m55e+#=R5-Ud@{NIS!AOv!@`x!Bv~^? zKs}6W?Y&f)Z1j^JQtg5(E_(1j(5S}x#eCWrQf7FOD?sUm|H^Lz7h8xS#pUlT9xCkRRJLor+c^K;KL`QG<(&%) z2ld5%G_dqDp;yJK23?Z@0Suq>|l)L0}?jhCDe4vrAyu@ zje#rXkrIiL{vWj`riDQALXO#QKA=% zymni9dmqd!fA&FgrV4Y%o|t)5+s;EpGlywf2gac&D-&Po`Cs4cMVjY4e+J8cru$!X zX!E&VIU;QVazAcNru52<(Zc%r$K?iwMhBW<1nByfKXWpiI6+eyJ+PS z*&YO7toA3fA0z+E58IY}0m}e|mXai@*iBo0pXK1UjzvZk#lg_!+$Q6H-_nM7S>`h= zd=A+V7d5(bgXFtNQS{9DXxY)eu?O46JKAlA{0TLOy>4NI_=@mdyt5GiQ9cD*Y8Tb( zT-^|Dl^h$zfW^5c1LA6Okk_7f{C6T6`FOVxM4x!N?Lv5u`PhBI=>R(_%v~8&>zXhoo z-@P!|@GF-U;KTyO`kvR_^k;|3mop}r*K~fpozQ01_GJIUgJCGRvWqdOw$`<)3s!@E zLhyxE!02RVNkp;dS`*!g-Zdz$or2-*6B^G8Y( zh`rx_Xpwi(s2TK7ehwct>vFjg50xfCSDoN7R*5pdd?UVp*wwpZr^snetJFhJFIUVz zQq+nYnMy|AjCTza9wOc+=w}8qPnXhsZ{Q6cZ6AdO(^<20>||{Z7a%knRHW3OLVReK z6b6d{;(tz&c57~?g0;>B%lJ%}fC$dL1b?k6twE0m$5a&4y-~bTfyb^}@LVOMYv@+U z6Pg5axrj%9yslRGL4GPOP{F=fKfNq$yo6rYW-~i`hy0Fri!L1iw2Gc`sWWp zG~}QE9!JK8hK$UNtScOhs~oInLHE=ByejmjTr7W1*~KTQ|8oQF4_X)u+Ab6=+CI&H zhCrf0;IiQ>EATs zRw#rvihsPf3;f38FOdlMHqC&?Ngw6Am07De^0~6h45rSxw*1`rU$?a}ETg2nObRnx zCO1ca5&nK~KvW&=xr_qLgGEInZ{P79N+ZE;Val(dcjb)FI52IgygNf@N{iS)`<(UV z0gbWrdf(^NOtX0 zGJ#mnfTyjFX5G&*f)$_xK#)&hf~BSHj8c$)?tZUsO4XXCVrjQ97x6I)7}zKGLvOEU zq%D>56S}F^pvXVoM20KS1q^sw0 z%BvgFPVFv8l62#=r`+|PUbNtl#7fNda_Gy_@lKS&&7k|BZhq`KU6by~#CzFgpBVyw z(!`_90=!yPvSqrLsG42wo;!bg4zJSJ*nd=Af;CB|+ zl2b<#G4DsnTBKj>opGmsfaqMH2lgzs8o(=^>y2Aj1P>wI(Ds1w9>VN?F>CRhlaHnh zXZ4XJY}2#AN;MbAdBgMhJ&s`#`FJvaHEN;7e0{L{q;NKsdtW9bQ68!`)Ac_fG;LWl=>JXF=7xrpld@8Di)?>?fb3Cg zk*L=YQ*DAK*}(ryBFsn0{C8zn!=c^6{;$f0akerwbEaebPq1SCPX*)uqHVM|0|jg= z)X@rAC(9i@Tyj%YdPC0)%OX=kx>TNmzfZvAVgTa?avE;ie>0hX3?|f&tN1dANUwrr3CX=QnMN{52 zXGbAYUxbJA%p{q@dFc=!fqAeH@`2F)HsiP59{dP)`l(BG(JzhDHQh!$Wg-Wb_MjZ3 zs`q4>KWDVWZ&6dI;&7g|6?GMR#Rzu~H1s&!iIz5uZ|NCUPlv8tVSM?2{tSn$MuBKG zTM6zK=vD`02kmDtTHj=g7`kv67!}R$8?g9re!QUSE%IEWk$&66O-?JHjEd5TL)s|@ z6uFRWc?b~X&fahF#;XT^Jji9LRfgj|ow|)F$ZA`#RG#okO&Q4o(>YsK9-2OOjL^SM z+>B0?-cs3)ZMAR*C#HjclD=!FZ&bRS61u#O_P^MCwtP<<`#kZyD#$Oy{Z7`aLzhI5 ze;wPN%LJ6OehS7wyHZbXji7jXIi%?sS~*@{E8+eCgcsqlw5-nPPe^-mqL(s2BlHM{ z%*Vc~=%lVC)O}UD4ko=>cmo9=!BtCoaGg}e${cAK+sf6?6!zVJR5VMHpV&DG^hwB1 z6{BskZRuG_iXi?XSh&jv^eS~&KS>-$>;Sds1;p?ZP#_rAkq|TQ`S40~2iFFVV{Gmi zrq8r~qG418UP~A#n~;0z%>2MyH5q}cwG1zbO-p5UyxSNH0f~L4)iX}1q%M^2B6VC zizL%_DkW>$D(1bm|3u99=nkM!esNfS5p=hOq^g7b7o+(=XuAI=qeg~?l2ntlGXDiB zzG^QL_8ww?tV7T!2l$`z1>s%A|4&F^O`WaW{|BZ2bM=2Q{lA$0Urhi1VfqK{?=CRf z`Xumw!S?6BV8etoW5S3p{<#ytg#5$0w)JO3MF9N|<^N0fe*q6Q-Gevv4rh!?imdsX z)QmOdU$t*_X_9myDXCSalyanSit2cbuy|5N#&x29+x5)Gkk*Tw%(;hufI!`o_w8A4 z?UhTXsg7WBCMK;GSdpoylu$B^78$vS0v+O{o%MI z<9_GKvzYu1CUlJ7u;n(C=uQ?3C*um7oW*Cjkd9@#Nw5n=af!uuci&^^yT=F|(VpL_ zjtUZA(;URb6rX_s&9KZoJ7%=Sln_#Z11ZQGwwpYR^pLzb8nWMHyN&vE1j0hr*!Y z67pbN0YBwzRDzyIQa*~k-3{x~U$%DuUI3V!@Uq5H)(a+1u)vLhn8!bpQNqoC318#WO=d0F^EhHoLbZ(Zi`9BMbgC1fK5sW2#Xiq#tBNux5T6~)% z>m2p1i=%0Qn#pzw2IDV-dasUi_GZw+5PGHP1ayIpk@kl~640+-)7!1eq2baR%Etwa z-2B_IrgeQILC1@aPXTEP#BCmb;|{sIjJ~zzGdd>|5CFhxFE155>dn!=ixtQE=*r(? z4OS3-35Ytjnw-DQ63cKddhAhQ?2J2kFMk6r%wkQ-@)g4G5Oi&ALOo8H3hoAU&#N>0 z`U)BeEUx^wb9hDuPK%VUZsHLCKzz0c^97xmA4xUQ`+mXO;zOnGi-;(HP?V8YqW28N z3H0K6B>7aedJyvGYshjxt>-ya-;5Wd&llA73g+UTkZq_lfocnde0`! ze-OFns=(q&g&`L@ncxmLFmzw(s=nbXUtZg7`|={EHV9tNuCGGJ9^@3LFHDDj?>zHbUg@!JaiIz_ zADD|%P^WsUCM4WWu$RVkx(JtZO}t%yP1b2@no%yW0t4pr>-Q$i>(RJs9<_ZXYxjJc zxKpBgX}Gj1T1b~0m)u>e@gQb2PW$=T;LBs@5C}jqlz!4ODh7E?c?>KSj$VnIN82)T z_hM7jmO$!`!vk026lD~F}at>A!Op6n79?Tw=I(Rk7%C61G>(Tq-lB~*uks`*=WYq+w_)U_^w+jqJ{l?*uPkbEq@s5XfrCn+$ zGeQmaq&4<`HGZyB=DAJqwa9Mgp2n49&aJ+GQBd>%>|v>Dk#tN&c9vhVxR~0|`A^og z60FM292*wX0prS&wxU*y5RP97W1IfED{8L^s(;{}WujQDHALk|j?9b!Ykm%HhN0PT zsEAv1Zzd&P1)R9T5uFw#8%YZ7!h6in9@<(OJ>Ecn)*SAMgN0YR^xQe###*p98;1>> zH;=c7{F}StXdVXGiN*s!_C*$KeJ>32H}gBd7UQmUrd6su48bTrRrkAHtBp4(sWmAqOu03Qp9n)!)LQ6XrTBX5c1{;$P+hp%kiT4eJBUw~Bz= z`XtPM`Ls#3j7^2$N8pvv8VqR0_8|eri)o5uyPtVi-h4PJIU*Tk_!z-Tx0|hnEx&bL z#118LFU*V)vaK}8Te1TC=d~+OXjX2t#+XM*Pe*|K3l0E2-Lz+_v;#h9U$@&`I1$W1 zA@Rfg&U*aSu8YgF8e1&-^IHf1b)di{u844d8TvsDXmMnK)DeT8x3VWXY}th%hZn8= zjaz*;vraAEBmnlM>c>R*y6AO)9Un)S2SPjqTVAld@uvAZ7h&80>7scdBXV83%CaM8 zNU?+LSO9r9*p9(KoiSjMuqS|@ELx=BZlneK5|lk1ZJ`rB{Ig$s5URm@_RlBvLV9z5 zUa;XM?#q1^Jt)5Jeca~ANHnVVebf-C<))uZK!UCJ6cnI``O}FajVW0HPO(|ec zdy%lE!-RI90p}@TW=D%}I_$AhqWW?JIvsZV$Mt&&l;0xG--DNI-9{v9RMRSXUNRNg zfJ8mC3@hNmTLNsa!$s>}RhNSLH-{*H3d3!G{d`q;j-T2%?`OkmUHUH|Fj$y6Tu1SK z;`_X#%7jweeT23;Gcq2h_>QJEe#m9!LoW8AXx{Rb`+mC_c|gFs@vGTOvqemH#GV0z zYR({nLBIeGuoFIAoMJ$d{7+7cxej9`>-Hj?(=!cs8|MbYxM)3#NZ^>M<>!UCn`$ch;;m5fu@os zjNB=ue;#0Gwx6*zG1zBwp~=$Ncp8X+(5QdMii-a{NO`>G?vqRCa0efmcEx5K=tpF< zETy>SFaIQJ06txw=4W;W7@XgKFrM(7_!C8G<(A8_nYilFLvCio_o{Hhx=IG+No9}3 z)|MWg^%99M0I#&rS6(=?*he7FK#pRC?y%p_?b2ueD+6nQ#_lr5eDxq(MAOR? z$9Gbxv2?*bU{B>_h^bQEsrU56zu?B_QNV}oSQrUeD_?uFfE(w2LtUwm8F$9@6=sVM zt#|QMF*U^>o$f zFs8xfEP*E9(LVn~#go=uiJN~N@D|kZcFX6@nTxYLaEX~HnX_QzS>3I!nll^j?ed{~ zCr8&GBPVH0a>7>PeP6!D8LfE11p8l-IwL>G`0f7bWPqa>nR_LFYL$G5=r&Xs&V#qb86ZmKcNu_?E+^IKXm#pGjJfB^GJjQuXdL3>>Z1gOgvnvp zh|mzmptn-%H%88Y7qvo}@6Jrkc!dhmNRaN458QaF9b@a-fWC;MdfchaY+ z1)s#|dpE22u!MkvaU~77+%cn9iM^n?)970JDULB!-{a)O3|+X65y*!od$yx)>##15MQHkQ8NzS(#L8)i>Rs_iCQ*~&1}g8=kt z46*}hw2jgBe~|}z3$3{nV~p+Uw_O+o7;GBH^q#Ov2WRgE>!%h1-Yb1SZ(emk?l*XU zsmIFHvP*3geUYa^#T)4W*G>ong!-}FjT2)8HzhkokWh5wh$ zdN!?FV(_?DCh}{E!#UnNG zrHW)@MlruGF4CvJ6&T!biT}&i`AojN%`x8x`FNm7Xe$0rvZo!-nOR$C%nDV&=1G>9 zC!wFmOMal?-t8{mwc;1jQ??95EjR+nV7h1qotep9G{zcRoHFR%jqdaY!+O@Lip(2= zB&D0ryq9$ZXw-X>{Q^2ZIzc;sMlmbR{t!_a!*|x`0V3Dlbt0k1r~?a3lrGoXidgz? znL?n_F4$gf3mak!_bX$G6)aDYiw}Uxd0wu^)Un+faZ3QtvrYkLu=>2)xJwh75+~2Rg*r58&D&W|&ynrEfq6@@p4=>#l3E1Q$7r z!-El!A-I6tH$xmve&0&+ba~?f9M2u_LTTbcGJbc$1(*SS*>raZS2d#g$mW;y^_G>$ z@=qe`)4kW_C|u(Uiw^j9PM9z@)K`SK>O=wxC#aRdV>csN{9A|llODOJ;J26U{STs{ zokIw&4%|_@*t2ZUj#+tsl^i2%x;g)&PI10>Os!qs3jIo-_&&WlFaP25_kNils$oe7TkWq6ht@=x? z@j;*E-B&OJW`44NLP{p2XYj93uZwUb^pQtA$EepLQPqpSnectrc)ppF2@w_}@7u|^ zh!VB`2IbakVmsuvD^=$Y1(sbJygqkZ3j)O9BL_?K`@MZf16zJ061I_DfT^g!phjJX z>vTCD7xGyv7o4BaR#alk9NpNx!$ge+85eNw9@DR$0whL%{Ow13Rub6m3Z8^U?{csN z^JhG#>4MRQL;DA5T9`=az>BX4zHH@PBKsKhsouMO@4PPC@Qt?k{(9u~6-TvM^=JXL z(~&?jhU&5Ek9OyJ2h3|Gf&3Ez%6+@f+8J2ddO1f1&9d%te0L^{NG$K>aRF=*Rbqo zHO2fQudkru78SO|NldMdZ2oK7xqV3>rJ;RHG?l=yV|q%Z`m!G3 z&rmiz15epC0d7`On_rK&H63t4f`;>DCPq@rk3ws=>Y^GDATfAlKJpvhj&JKpa$?=qkq=6LT=M!^i zic6^WLQk73lYY-xpKIwGepWZ5)f%?k{8>8YJ9MzlQHBPIHsWk#@)q_FyGfPLIhFcF zb#@JAj74TgrOv{jkP6jAwuPZ#ZZ*EbmZ!0Q4>r5Ie)k0 zyU9@H_Q;{SNBayU+fb~^mI>%)hz2e13UV9QdK~2}-?l8*gI{+~HN0>n*SGzJ>*@>O z(5)>{v)RH1#wjP55iNq#8QwjNJg{I_TRyty>_p0sBfOTAg*10BzItP41Wk~in)md7 zfa6%J&;BIJs(Klq*@Z+M_%x{&dT7g z#i7rKBCkYq6&kU&r(*Ht=@_|)q2m%7`JB|pQbBE-4*y!ik0U#F7h}19g;9AlUc27s zr$r2B=i~p@{U#xC{jaEl?AFY4BWu!^Tmj^a`{lo{6moI``BY zrf!j{++Lu;>PU(kc(T&JdCyvGz2d0=d`QI7<<0+nQyA#iL3|u1lPMykorDYB)aPs_zCAz_Ia}(dd6d2y12}KQ<{xMc30kk7v=jKl&Jr0a9|bz4k}R<$=~>!&#g=F-w$-%5=!MccJ)u~TlDi<8b! zvH%tfiUs!4ZM_ZXaiKne!kR%3EHy(R4GFu|UjH`&p*KwWqPRl(<^>}_$47GB7TgQ9 zHtCb^=bN_V!@+KUyNV;j{q{_>E4{%s=m!~9p+#>DO&8ITT(*Tf@^^8TbdOhvFOtGn zgwl&({&)V=UD+rT4&)#*NOAq>Oi>3y7GS$fefJ0xSZH)g6=Pt*DbF!ZPl?Ypwui+_@m z|8vl@8mh*Z4)EKbnLozdwItvSh+GLS%F1O!-%S^tbVG`ke6_gWr3BU3PVb)_nU(jK z!@DQnEA_B{NnrBbv2OT!5t8&U;~URMYLG1+*Z1roWT<0^bV&UG{?raudxTfLMw*?W z-muqCUmCa+dUM?T^Ee*#W3mO-2hnkeJRekWoZC6+9P7 zOVvz5K+S9YJy*mPoEY$=T;_ z4C@~4XwMwVg}y5~Vj4O=xrI0i;>$%mM_&l_8;EUKL<=S?h10%^w~-jxqmoF)B{zO2 z0$E#s!Q{+qNW@$j>BoA_(C7l@yw}Ixbt<%ng(0z#J-s6t)No-=W$8oT>tX@?E*=aS zcvo$RuK|_c$giJJ-`h$pn78=R%d(E`b@mkmz_JIzmvTkJ2&GNz^t2fC>#;YU<$1>m zU+ykw=8)Nf(pHp~ule-{j_|};TEH99`suHK<7v>xgh0KIm~C)QY$3UfR%1c(6PtHDv@+MrD4$IQ(qtNru^!lCB+^x z4T(nw4N`Va44ncuJDtjIkl>RF7C~{#{m#FB-fOMs!yeunT)h+rS2YI2&=@v|@1${m zj=q+l*xv@H-qe+ZefffKi&sh!k7YFku*eVgju~Ys^f0-#2NzHbHujIMw6_Jhfib%rQX$97El4OzrFRx%7VbSQtMk zJG~HMqy5rJaL-RC19s8Jnb(}$!%#1O`V2pA`^+uX57pm{w5fTlfHGDR$7>6S(D4x8 zinlxzj*t6)Km5Uu_Nde=-F9r)eD#-`?!fM>nC%2VS)*Bh7^E!ml8{;{0 z5sZnGPJaqA!1V{PdI~d$wxQQaJUF7_;Ed@!2;P;2-V}e0W=k-;EVY!>vo5DBW}bYG#tC<{8lABqY#FN3G)~?T zGfytuviOAlMkL41G_Gl`ruQx-$cl>=M0qP;aYk6Hkt^v4`2do+x&q@#&x@WPNC zWGfo9Uu&>i61;k^-xdsixtOd?Wm{4Y`>%vPK5r^v*rO%n$p?K0eyil%*p4rzE<3!b z3!YA`IOR!^E5AnW?0j9qkUE^q=%W=5JRUq4%uRB6HZMPX10#Vyd*Q3Ed4uwIZC*4u zj*+sDE45s+Z78lk`@5`xF<;UVbE&wqMNDo!yx;m@^Dd^?Lt!X?<>%N+RPYN99rU4N zflK%#>Q(OKSdiV{iCo3WI$u4wAKX#+1qLgspECR?CjvEGuQvKjo<#Vn7X0Gptd|{E z656nbjbCnE6Gmc}gEa-cD1RhAc0au}tU=v7J+k#>_7AS@P1w7|_vU+hgm$-1irDV1 z6BgypLj|SZp6D2VmtY>m@+r9$kSKq(2{}ta@J^$Ky{MccLK9)ogvJ&QMq{Kt<;LfH z&T$gl;{P!ro+453Mi&0mR>nLbYdisrnh zdFxuaJ4U1xxV^ElcM#7C)%xieq_5^koK?9*zdj2{jZ%AmL_y4g(eyi^02@W4D#_TQ zNW8gga5}PP4zYbP?l#jwtZAO6dmupqDkVnNBddXA^jgip%A*|**9g>>s_`Uwk-0Ws zB!!14XA?*&A9=8O;n88^(o8nsDSP?{O~$Y)lzI~6m9=;Z0MsYuP#z4h-3b>qYfkQZ z(Gl_{r0~&y{0QxDkkLHXNE@+QJl1T~1Ld!{?R}^;;M{ zBDkY@yNksq3u7T#TmAs7s&Osq7>imo9`?Kbx{)ZGa+V=s=RH<{axgvs{OtN3uL?zCyCUFc?h76^Jl9iZ%ZG``K5m zEGgy@YIf45#4fp72g_2ztOh!plLN!g@}T+C6}01T(RidYSnmr_UOVZ1 zoDvp0Kt*_RNeNk}YV%kzfH}w(y7wv)2eP_YZk`B`vs1Oiph+30+l^TaM0x}q_ey{j@m!SW4 z$X%TMlH!k+1Kd?Z9=v5~t9vUY^~#D9a+Og+^9~NIV7-(nC0R<;Kxtw3+(mwWP44lW zqJJ9r;k9*!8ENz;-N}ION4O9F8=k}AA9I-X*6fCu2H$sBus+B$G|yCONxqSAPrA|5 zY7d0hWH5^Td#nVcbA=X|$LSbN_ftBVIlRXD-4p=QyU+1?@-Bo0PkPFmcY};K=FZmj zfImt2)>PA>7k+MOY`;a%J_@ma<=l6k`$_6LzTuCLw1zXgT> zmlHf)p;a`D zIfy)mM?Zx-DEq6diAzD~+*_F%Cy$M7V&UdN4o7GP*x2^*$h;qa>nwdKAQZrrj{Ef= zrwzIr8Y$I@g4Ar5H&wS0!0==CNjbDmx0FqHVwKM)ab-=%pk4&-5viTBpR0Gg{BR;* zgW^ud-{{c>_TQ*GaPvCDqt{2@{2l`s6*3;JZT+RC7DWm7w9ljBtp8aZT6k91#Ft&M z%9Q~Vvv&a$=g)C}pVCu?ufzV`DWJN3utf$U(g(@2hud#^%C)II1+KfCzLfPaHxwC{ z?~2|cSDMnX8Jff&RlA}|ISz^e=4XYB@`3hxVwrjT20}CjQnX4eG=9FJ|i(4czS^^Oe{TG?FX8jsBMa zi92)(ZL5=NA9f9A0`l#b&hB^=4}@MirthWYmPq8tXQJR|I`YQio8!;eYB&8t2k<$~9w%XbF z{zdKG+Gmcfo4wU7b8GtiP-;IBuj=xQ56y3PiL~k8AM^(L&-p|IO>rzu-j>1+O_QCt zN(F%E!#e(6f7cue-@Y7u{(|E6u^i6<;u5D+C z`hUs|p6;1eS3{FmnZ6WR8kbckm?Jbh=Op*pUJLj8ne1$|wAHtKp-51L1%7y-v2msK z85ZS^m|0H{8kfyprC=Zl>2ejoxq+*j6_1W-DlL$R>#Nlb3M~faWT9Iy&o_yRo^Fyc zLwfW0e_vh1^^Yv6*}vBG=k>-J96$z$I~9u&T!2AXC}b=>X)i`reX}PC*fSrDwYR^oA39C@<+UPV z98V^H)fK=9Yf`qo@(O!#|K%;XH_7@>`7wZ?`dKu=lO>UT z%!s^0W^sXj@A)zN53r|4BM?$DSU|lS{F8Z=qDmX!A!IM#VWR4;yuXwtwvva~f1d!2 zTWiE*=)&-NidF=DFQ1u=eV1rtyTHXsFD=wRhxiMuNALR8CQ9qQ30FWOEM)Q9Y?zgJ z@Hm-fio2)5=~XAw-_b!|p|`dHnWOTlG0*cir;VeIn|4lUa&d2RdtQFJIy(}r?W#X% z!fGWFj#PAGvudntRmel+tnCI|f3b#Vo(qVC}a(@wnrNe zVjm`wkh3|kL6e+`Y0agGUvGQ0Gd=>>7y*m=r&4|P?|Z}%6#nduVy3k=3smHRzSoaB z^eX#ivPSl)$g9eELFobmqCHCTPOn7&S=~%4v<=ip7)9@E)nA;f5Gip!e+Dg2f4Wp} z?$lsUk;^p@W)ebK-d0(S+3>vNVfZIbc7LAlND^Mk5i^Uc?-+aF;@fQAt_3sLT7C5Y ze1d3*Xc`Pp$}W&FyHT7x{enW84v^^C5V#Om6||OGqQUsSH?!m7O(69)I!-JZA?#@U zA+ENT4?1sWWzdaOyQ5#2f1GO%4sNr=T+^~l%4PSZ@v3aX3P#CmK53yl!_+)dtmFp! zCakf9>hnzdI(I*uSO^vc*xQhg$c-wnciKFZxXgR;|#f(i(|Qzui<^+;H$2+TCW>dhy9u_f+H8;03RHf0j}qq<0**TpvSA zA)tItOx!cn=M}MS7qId7f(ql$a&~1r1}5~*YCNAAiSL$Qvy(6dx1JY*Q5lGPEy^5D zHk1<|7VCJvcwtI(*lJ4|E`U;759|{a_%d1aeaPikNyv#q+b5&rt-&n*eN6J=qIzMW z$(6O^+eFEcZ`!tXVcN=^Z zUw;~1w1$-X%+1J=8ri--KX|d&s1FdqPB3yYa@HGHO`j2|f1qGmc=HEh?<>=5>*0$L z3ob*Clz$S+PTZ`z6bx@mwf-(lMPbanf!@ti5IJ_8cd9!=OF-DeuNMyuI-A+cgqnN9 z^nKE6ckKo{A{c^IxF!k$Jb>Q0Vu6IVSUBL)8Iv3jN|e-&B+oa_B)GxPh)%{eF2Ia| za_N+wKVGvVe>6rOc3mQ96m0Sx<{!JV&O7#63`kScUaLaE+o6_o_`Pe4mkqTI@4o7} zz>Tu*YDFztwwD>dH6D-r{t{>`?102PgCp-9i7aO&GwEm4eyf0p_fk4&(k+R~@^Vmq3H3jEp< zbfIkr7mW<;`k+!qc;ME!IT!_ZD0RD-Cc&NTlw6^;CO(YSc%5`{BVVfka2H>uBp3LhbZ9Svt5RblYx*5RGS`uGNPCPiBETX>P_SP-96 zk3|S}!<;;u#FD1ec!-RR@+`lSBL#f=Qs1YU5=`dXoK6inaEnb=w_DAHxEHP_>O~oJ z^U9xV>8OUnEMHu>kz6@l)BMY{2=AT=u}54(9mUUA$^J0N@ZG0vvV*sgmFl(K-Rl zF@GGiDr{=+I>zN`7;SgCLgJ@ZEp8OcOx8=LX*T?U8>x{vz(Mtif25!O)+r+}FQw&D ze%|TI-$|>_NKjtJNo^Bm`vU}G(Zp< z|9?#Tob(9#_T*d}rSOAvYpZEK(awO3rdwZbwnq#jvhi4DTP$eZyHNTLWJ({`$$gr; z4c+3`vObfbUfp#;U*Rwdy#KqJD{&Qe_^mcCYbo?csBV6^3t1=IAEhksb*N70)Au#t z_t(H-By5GxmJ<_(Ajab$pV>g*J(myaNqT+yrxpSi?dILD`=UD5<1y_o?3M9(B@4vSIl;(d+tuS zdli;Y<{NzImjEu!^-wNPp*l)et>VhDE(Q3wEi?MRy7MMwD_F0Ge!W={54xacXjliK8;;wq1- zVoOEDN~Z+%yam{e=<5Z18GLNtq?8z(i1o$Nvi(-+{f5mBu?d#%@@H9A1h4JbO}r5l z@nfg)#F$5w#d1l!sNA2O@qb_@qVvH4PG)$zrd^~A)*utGfj#%v`Gll=bdNBV7=MG2 zqOn?LMlSPTt^wJ72dT{(3+VENRnO5I#!Jzoi zobu%UCdVhLJlx^Y=~TRtt3?|kzwQF#r)Z>NSj!uuvbKFEE9Ku+vwtDN-0fv=u6I6v z@Mtt=><=(NyHHm}dh~07>PCua+gSy^W*fw2ZeXbcWcVuzYmw1Fc05?NP{x)02) zz=futxGh^|OZO~9rEky#<-;HqkOJL>zp?0Z*`S-_dcdhIN*q^`LwbA@;={*!rrHgw zuj*npA6|d|sox#+uz&kXeu<`E$AAZY_2#cDpDmu)2iMwG2ZZH#_KxmHc`Z~bMV@pC z+R8qs$o1$=C@(2@R2RS5Yr&&IVQI83g6mn|A_=>bBJ0ypYSF&FO2tIi@iHbQg-6tH z22noyW1v@rC4&DP#Zum@d;VYZS_)CLw_p2$=qC*qp#&JihJR|XT{<=`cXtx3nHT&Q z>2BZ%KW8iQVn6Yo(zMmuYaq?2IRgir;gFY#PxG~PR7f$pqRMoYgGLh^m9CT!A(XZg zW^c6iPWzjtGMP)lRF^}n31eK5Xx@$(3f(szd@Y>&Dio?G_g^O7)ZkzM8JfEt5C4c8xs8bkhzuHy1aI|qT{K8p$ z)6*np)+Wn{g&$%{KX3dc=QQ;q&Y! zhNcvjczVBy54kTE;#CThA=g>=2lK2#Ri7<-jYlm!?J<;;ZSlsqs$i_qeZc&3yJHLX z5`Y}M8-EHbQL$je3Fz5m4foN$+m0$ANFjS>lbLedg#WrI@_PD@m+9O<*BKA!&-Bph zyQ2bF`C}j?lJD=+Lw1w{J=8ZT;M0!Xg=GCF^7^`@58B60-RmOr4Nk?}>olx`>;LUg z-8lhq%>7`$Y3u>rj=BJS`0JwDCZ-muB?+>Odw=AA6Bm7n4w)X~1MPjv{H9bJ7mfSA zM52(GIBFOB{)n=pJkj3lhCwzFnFeU`f7sADk98R1h;)SfvN{XMzwbvzlS;n`*$>fno* zZhybnO2qsaU0=9Xbavy{&S1y9f0*-^()LtN=HspjA~`x)&lOju>-h`D2e#kq!#_zm z6|2_Ap{R=YC-&p;iJbV64Wu_}Irl<1e%--`LjWR_qg<0MeVBcHd3>E`!;mr0hqi6) z$b!=g@0e^$$9CG80CxmeL<(bV&N{_Em50^HBW@=iRnN=(EwSSQN^ zBF9=HS+T$bTCBgjXew^_uv=%$DWF_f1*ZPQy)p4@<~B6lj~(|DJ|6^Ifq$TAd&x#b z96nA54O5!EhItqSksFxDGGYgr7Zj7G=(EpXUB+5e=L!V`E8fGfgIsnn-@o(R&eco3K{Hi=%G5jBbAok~TVsDI!Sm&%&jH_|^ zx@C9Zu|e6jkWSZn%Wx;ga7lgFI<^QVuvDhWL?_K0r~7#bjO#X-DSuwhkSc`%oPD&C z=~+e>!yNG@)Sueh-_7KZQmMR4(h(!mF25bABzm6rM#;{zl}4^x(fS(jSm9>D&6(Pc zJpM8=l{&j<%SPYVakPE3h`5V&=?Va$^RU~7dRib4hsRC9CTYR_&qh1#McsM;lgGP* z1<_5->&=%GIOJKi`+rCg^ewU3q0aqR&e3%?MA*5>G{X+r9C46a3?~I=z;eU#y~a19{@U#--TlGk z_V4QkY#Fhq&84#!9I<)%0n$hyEbB-w?AZ=^!6e6F>-?g9d)^LX`|GfUBX*F042);; zRf~zqsy05b_1P6m)21=ee?CU_||eREix zqfpFN#?S&Gp@X95+r<}?Zj4W=%@kSCx#QI#@C0Z?>3`lc!XmE3e4mAkKA-{VeqCHB z01i%>!`2$GTuRs?D(>RR2PJ-R#-H&Fe>exew=K&zu*-eiQqB4Uf_r_E{91%>?vLzV zifzK_8%NZ;osTvVBZGktnryVkzv+|29yaY^B-)@iLW%#`X(tc}qM=lI6A%X~%1<&~ zTq8GhaDNBaiMEJ3Z0&hgPK%9ld5Lb{YBKl@kP=Y|Bcz}pB7KWv}Z@8@o{e( z?N%oVjTYQerbp$YGyksgp$-!s*KO_olM$z&S?%baL?zU(aW>K=;K}kpm|AnxC1ZJvP#Q3}BTw!sC} zovKULEVO?HG3+08C%_!iW(^boED$5?un}7crU8`?dBZJ(StdeOzrMui3OYP;P;lnq zcJ!rHrEsQ~zW(m!VG(ZvF{_UAQ=Usaq-*sPvOtK}wJq94t2ZF*WfvQrNCz=E8P!z| zCVv{yW!rVlu^RNN=1Bgdc;kSpF@a7eN-9HDkyo5R_9B!LV%eYvB}MI;TrAp3yhBWd zEVimhN-Cp8E;GJ9$`5Wb&1f>=OgC}2{kz}AJ9p;QH|LeJWSX;FzQ7w+qE>c1ItGq( zh%@gNxSt|j2-}fm7OYS9n*U>Y{f?-_ zZ2tuAeHqF~_~<|Uu|$9_TqlRIUWnjr20}0=5nK3)-jnPUAFO$ByVx@9n=dyGJ%Ud< zAWQ#d?Q_nlJp&U1HV13;(g2jIMGZ#-yu{0}$f=miWg#s?A3RH*BNY3NihFats(+k# zBUBRkx^uM$fvEN>?r{PMRcZw&W`??rTQ_FOx+g+wq*#&;^P$dP?OX`U;{wlN^1ON= zBO#{aItJwfa#@@LNIs!%>$%m{Fgd^~Xjcs7iJn`Uoj4AQjy{Cf^jBJ63&~ErzjSqE zIvI~@^9tCa8pyrDLK8STS_(R-uzy=1_NhI6P()J`{?w}c)Vf1tuU$aXSp$krUkeLA zOZ1nq!R;v*aR=uZ#u{tpUJLFP4Bn4iaL-kv^^MBQvBt)8Y4lDY3pZL5K3j?qDh63VN;*|^`!<))p!^?y|Ok;&jR_&JbO zXd^akI9acVC&_<+Xaa8K2&&GLhjyg{_3-eQ>d@qa6?>Or^4oFbb^~}{=fFcM3D0J; z;Q;pVGn3?@Un^A0vwxsMyn#%}Z3%cW`LjTw+Up)*@wdwBqwa7+qh8xZl54dJ65c;~ zI_ue=^Ve?sBcTmqn+i_*i>PP3sh6!lM=R1bHe0&^&C8L=(GcP8GQ?%qpq51FnhL{#!rkGa47PvXA%FPl^u&JuWuSq>mED_~ zye?%2GpzUTfXW^!bBbFrKaMWNhp|IMzo{Dy>sF5bj)7rdm7HzMur1eh39)<0srrye zrtOGvYk%)U4CkjC-sv-E;?FgHw(rmD*mEpULfvSYjzrmz;{4%HU@6hLr!MCcQ#HE2 zgc<2|X=VdJOMgv58yV-An5qVUx2}Cua-$@V3pDe%<24{j#!>Ekf6WJrRg_8qnZnMf zzxZNPzc6v!vf!rV%%1qSDMfC^d0Cfgr3w11-QgE%Kw_pWBODg+^+QmMPtN|Vc|Jt$MP5{y259a z4eSXJL+~7bE`psX8uoC@gB(>(DCZj{Vz$Fj>9jrn9DdOiI^0Tc<|vj}+$ky{>Q#;1 z$r(r5{#&21Fx#+*cb}2eDe*#B?P#(@4(y8YmVd5veV`97uV zDw(YVmf;Y>-$}PBy08KhFUpnrjQzEsatoz37{!Ip#KjiT;4%%QIzf39Z38%JKHO`e zU;oz`J3CuTh%5dZC)*H{L|~a~1guVp*Oz9|`i7l=yPY`$s`Yg3x9RTW{h5MI7+>*V zO@B&)US3Rzu(;R0CvSu+Cryc2WxEb2?07uXM-iBOeDC8O_`{z~m5!!c?zlp#DEox> zdHMJ?bWH# z7dGcnvP<(OT(gsu+J7cDZaHD}Z<@R44u5yq-G!I9(CEnl)UO@v`ZDK^I`*}Q{y**M zycllL)J}en#HyjT)Pb8^q;JHtX!u5y_mJxRd&#D9@?x&xsRcTyIe~J{qQmTXyRG4B z8{^1M`9|-T8B;{FQV-aqV*bAs4~&0C;V5N({6DleIH&D7ORa0JLp$P2l)eD|n172; z0oO2gqu0f{+elvQ$nGh>|1{+LM{(88%0tN48T4V5A3y(MY^Qix4PG6p*W}hak$T#T zlQVR+A@$YzLCxdf2~P35sahZm275os&4t06p5K*iJ~bjWWmQvaY6JcD;OnsG(hO|d zT^2o!aY>@RTbw>(Lz4p@mY<;!NPi~G-$hS0{6!aS;OWPk``vxZ9W_Wm8px{$L#Wu} z9=N&zWSrY-r#i-`F6^j-jYZq~DEt?WgkisHkf<}cu}1kAP8~p4faMRS+!yj_0dL$Z zZx_H*EHk3-!Fczr96GIfE|<_^7^xfUNkxhyMAT zMk7bOpQH6t^oSR=6O!X^(cIVgvAms)ps1Uamg5#w$~qQ-cN+_2=p5R3gO#_iMj9e< z{T`|aDVC!cT6RO9B){FeU4QstXzR0!?34bVliEeyIc3k~6|`;w%rTR+C2O6<@PnJV z0GYs@zi}#77srFfka*5M1sdam2AitwHmGnzI9VD|q6~P30Db=2vvShIOOAY34UeRi-uT%HFod{o7R{KbjVvY41a47EtRhKa|7m@-HVquG(Hh&XXvW>7UR=TVEx~mJsT}*b@kFZ`j_1|EGK@1OkCkju1RbS`f zD81{LgdN|{o==%lpT(!Zn@9{G+b75m#ofSnlWLAHuHizb(H#y?@mC|oXCM#6&I{pV zr5Zh5Ox>rX=NI0?JgptC3aW96`AYr5ZD2AE%{<{5ZsMvqh<^lzxvK7|3(g0ci=Z%E^W-_X?*WZrl_zc=bi%;b>* ztIQ52W{I+{W`8$m-6JoojE+#K@Ok7z0GcMFS=IU}D4a>!*jn-dAo5FY3AM**0%w9u zx~b#rv*6e~)Bg|=jtK*VyY8B1tg>%*I$?&(upIQJ2ID3)wrJr&&!J1nu!zU_xdCJe z{79_qrprleP~NQA5;Xw))E8d0*|=+MC2)}@C&Xf_RexZ`E~tvnCPK{7NR|4xQU$r@ z9P{n8%`qXJ=L)Q~-Qhw=(m`ttdvZ&Rn8#c0-0_E_j&{G>$}bncKW)f{8%W8~V5cRBYIxw&Da>SR z$Sy9$j1K;f0m11yfY|Un@sZ(d48X%6s4s%{tbbO;NQkhsad4Lyi2L$=qjKX#d$Nav zbW$JIv0SaiijK{|+f3F=)?)O#1l8}-quD@=HoaGvQe?=P{~l`VF%hf7-Bxf%jpD03 zlQngnm~Pio2+k#)ys9KQSipM~wIODT)2=VX_pL77XC&C?>8d`D0WF znSaM_O5t+($|q}pzNe*5{Of(2kZ`_5y+Rwoi}Ia>GpcT<8vx~K>%nZ~0ObcCR`jj7csb0R@G#>tx!^A>D;v0G57FDE58y_*78sp5 zzMB6>Gd8qRHIENj^!9d2&Yoq|GV9borNKF|4qM!2bq@@ zJVjS7rm;$V{#sPqWw{F^Zek*{ky*L+R44Nw+5j9-G4^k!BSXR63V5Vgh|m@T?aCiKW;x*BwhPDq_AFJR^IcGBO8>$y@vwB0)PG%$Jnx~ zl|bm;p%^IVxETG#6^z#SZRzJh444HVb;In8P+cGIPvk@%R2JE?62$phd8YvleD+jA z4f|!-11LQSw&PM>6~n$38Qr0C6fC9ig?*ltpWKbW_lfGQeQgA|oo}kMP+bgHKAiRa z96)j@(pCBQ=Mz4p-EXy38Gka*K3nMCFcVm&DIK@j{-&$?lv;>@5evN4LUzc!A@mMM zdG&}~<>bU7{WF=bD-hLPyAm5{dhAU5#SzP98k6kwKwTQ6&qc;DKe}QN85tgyCvDeq z+lGrB-pjetkUe7+JhOT)wfhibbSuYP#=NOQin#?&fy!!GA)@m6w9BNwsmL zt`+0q2o8OLa`Kyi4~{tHyE^CQimGi#%aQ}jbi}}hj|+Gaq9AuqYg1*@-2{AnbjHH3 z(Zqp~y?~Crx7nef;0>NemBNfAXHZrlIL1KUjgIMGolJ{?yWY@-wbh@O8yIKzSpK3+ zI}$_&s0j>heZ|WTKz}2DJaJ%&BR#3}crMa*g$EtkE+(rJmwE5>@$)iEy~THQ()Ezx z1c}8O1aWv2?Y6j1*kptiV@a`z1tmYzQm3(GW7dCnNcdESfranQuxyOdzUdW)`pm@t z%{}2Gt(F+vEOJ9*y>%bcg>QruXjGFSu z*swcln8vDrFE6Lhjefuc{A)t)1}Bj8KuV6jmz7JrI_1O3xkA(5y~ z^4O)~I`CHjb$@w;E=ZijHK8YaPaH)z-SoAlKZFf9=exkIgawQ9sJ_hh)r18VERV}k z2kxl;tfpklW{hy-XfWB}cMm|og|TjYyUw1jYJMvo>@q6)CXEg^vq3`>tgJP^U<$Q? zz#c49Sajj)`bYn=Lht?rdyvH~ZO#P3GJR%eFqyhPaDR-6N(o@FQx6jvw{|%#rE?w6~n4jW`nO8Q>h|^&vmdswi?3g_tNP>V3}_@m9Y-iGNfs@mqr$9pWQuK zvPUCUO|wN_W88}X40t66-zg)xZxvQ@L_^7==SyF(h!a##*&E2X=b_&d6W7$>o3!8M zXlkq!`G0iM=St<}c5TT)xFvVC+0XF4Fwd|*_w9V7YHoy?sd=#79hG;~NSIu;y%9?P zAh3&ku-BbRry`GsCn~ZyxCrH=o9FjP0F1wCTV9?GgEerTe;9O}?o<%9x9h9-Hf6>z z=6tdl=aS@?==$)m2!(C9K7;3oq~=HRJ$1Bk;C~+uG$6R!tPPE8O38)azk827Z@2zX zS6@9?L7|%&SjdsBrY+=(eUn|;SF=G-+TFS9#E{TK?MXaj;_2W0{&IQ9;V)cPzZR zM1MT(n9)82-iQ;f17cy7-m@yCYk;Q~oQSs}a35d7`Vhtna4 zgd5rj-0wcVT0FG}C@VQxJW+Hj#-?p@a0SmgR0NZ11NBI#pFEpx=s$Fp9c62Ku8iY zoMHohlHekXfbqb)HYDxooFbVCM1Lndr;N!O2rk*X4j&%PzX+<45H$a4{13Ow%m2;o zl0@Kavg*J4THb%_20T;$m)qt4+5KPLE;GGU zrG&}MAf=R{sFW83QBoCIZF0w!c-J!KE@Gge4z~6TKJ$9iE&Ar3b^&fRYJc~+U~C+% zEr-?a{jawpu9VWdU$o`6V*2wx6B4V!>42Ex^=r)jwf?GrQICduHKW&d{u=v2cHhF6BJcN4ANwD2`^_zX; zUMk4OmPrcn0A>uV`RgZyD1XuYPc7F!l0Z(5SK+e!T%c2~g|ny~k#f0IX3hnM;R_cN zESH!MLnjBY3C%E&%?FKaNM^8!FlINTE1i34lc&J^Q+?3tb3YPiD@?c(z+!T4%V0%~ z|C2{7zvl@4CT?#&2Evl)G48F0@#A6dRK-G0`DWulqaQm6I_z@A-G8OHS+{FU02GeB zg9**tIUwI^l6B5AC4|plqC~5-#zE=q=0Y5hrIYeAck&C`f)T9RZ zG{M^(PkkrEbPC-L&jv^Y)pAV>^LhC4@J|Ax7B;mmcpBcOX5a?RN;Kb?Hxeds&tv@o zqB2V66IoQ|f@70(%eeuTY#Zxq*(c+}$7qX%`WmLALUsFRVSiM7dH|I^#)-G0_u{?H z+%%#ux0;^D%gaD-alAbwkx?; zFFZ^_$r#NvLVp0Yy5rG(>~jK1wDG6=>r_ANUxl{FN~hEM_Zogp$x>+JeDUL2r^tP^ z%Ve}l)#wj*xC{qT5Fc#DdF@1(65h$&YoUZWv6;NU`YRoUuA)~b-Di2 zjG$_(?`IN!=L~JX4R^SPYewjQL94JZPj7qPw}0+aDgB3GL4%*)swG0e@!+y?)?bo= zrB*5MkA?kS$v5ZN5Q@?6Yk2hM{bvFDjSUpo?7utafRVVt$13Lx7gUr)Zj=_3CkVkt zxi1oC9y9|O|9`&LD2El()2R=P%(`EvP$*~+`Rd3t)0^rbtqKZ!8!bo@)~gB@xQNCX zAAe6~WwGJ=*H&#>@^MlY`I8r}qwvXr49NuN*bjknAEtegF9{ z8ggF}o+*1kJwoHQ^39bEa&`$3wdcYmosIk>NW9B8>Daj)zTqzxWi9{CQ3$2&vv`2m z5%Sz&-l_dD!#w_4bK`Mn8FC}MIvr6_m4EG620b~A5q`brSSq&jjmXt+UU-7RB9Ei5 z)F>)fB(1)CthnE%6jtKE721kwfB4X(#*V(@Uu>TpoTQ{c zdB;OlUCB(bPc$QpsxtgNFs8U3AJBJ+U&^nu(yjLfg6q3Of7wPl8~7MHl@;?R7=L8G zCev^*CXP#d7vBp3sW4npi01ewVsv?#M9;?^LsDJHhC(@Cm(^ybPXobr(joKgzalcM z{WZ{T&a@j}tM0?w7xOmTm!0<#Jde4V!u?!|HPw#p@|pWmxbv^kaRi*G_OxSnvFT9Q z$l?>-2MA_^MD0T*&DxN+g3L_z`hWMLWiWMm)hK&Ul}GV2Emq2`25(Q96EcPg-m2>^ zn2eo)wy9=Rv4J$-sKy4TH>3tq5{cx%qVAz_mBBS4RzOXTn;lf!KZGntTBO(5{ipKG zWc-$eQ1ALy;^2Ofr|{Z*-2;Pm@sTz-v`wAYN(N)w#w?fR*6Duets~MD5`QlEX+*X> zO7C1h^TDct#cA%ed;Q%92W+GX-(&XnMzmsar&`PdTanEDGGf7!!j_>$Ju4b+NYXTX znuWmFK%PJ=o39I>LA~btK6-sBYSh4>sJT$-^j*W&9}#D@RS4aoCg6@h=_H$dg>#W5ezJUVoMF8j`MKz`U98P1tc)@$=Q7z3MXF$J6naO9W_lv@Opg zq>ry_{0oOt8Eed_*m_|PZJ*soYxCU?JwHjY!5RIv5YvuM2^eo zZ{d8=0B)M5^zeZ_AQE%$orK&qLe;Oq!GL!Lmb~;hVT$Xq+mkY?^nW=2?)1Hpk;8v? z);+F3#@TZpl3uYQnrFxWA=y7T@2wph$_`HPL;g#?(d#HcbtNL^u<8M;KOM6aLsty${C0-_^dl^u7JP>(vUDM$&T# zzy@EhpmG1^iBSA^*Uq;PV`N>zOXSlI-zp|@+(ij81${|<~M##zb?6r^NvLjMy^!*iD;Ma z8}|Nyf-$Y4w|`|YzJ8a|Z4833+^MgtXmgnApiF)nxEHY+$DNoEfh1+Yo7tfy^*?`Q zbDf9;E?HUGUyK^5>OLmKQ+SdIu@T!hz~dO`Uq*739RJnz`*MGRXD1Ei^9O0+)9m+R z?Ag`$n}O<(m|I1%?#uY0)f@Wfg3imi+76^WHJfsbnSab=FDG&&>VnrdrH{`mJ$=aa zaI>3?S~a3$3bI!bY+fx{wsn_s_&kp+wZm)p) zs_4OzDu1nfb`S&YS7wO`6Q5ah&h}t>4vo$|W4-`2>(jcH{sMLACa{zvo-P>|>DIw8 zGW0xjTl0L)uFI6e++KeK!tH1NZYMB0QbaeT6_$yrs_3d+GnPk`KL!v?;X_fCS3K`J zGj&z??qL_U8c8KluG}4nMVx5%<@PB==rYzh*MFg&XvcDT`!S6~6z|SiDaH2N?qM8I{0DdLl!H%(+<8Bb0q|-G)(+9en9|qOpONwPb-)#PFSMM848m4BBgMknHkyZ)neD2A|YJ*cm@Tp&nVFll0+ z$Ya_#2ry?|9vC~1S9pcqF5{Yf9kgRwtZe?MKtz^DWcSC431sDIU%>J`hr zZleF4vj;JCHwiOYB(2WijO)+lz-a|r-`l$V&2IDWDB-RH!Oq%Gc1Xu|TMg|i zVQyCZDyMBeh?+djFA4AYH-Du@U-bu{@=(R0UK}eetgkXjPc{sdTD10S5CS2=$_(z; z7gnOup*u!rxDrLneT*(O3Pm2OE*EE7e8=1&IRu+eV&~aOFzuL->WDb@;&i)Ym!qJu zRp_5M`Qm5>8ZN~n+QNFV&w1CcIhBl#qBoZtsTP5>bwHiQzrCETsDEXcHZ@#b8b;o$ z8{))MqiF%wVggA=mj-*0_GDZbn_<+Yuu3e=1$^(yZoxmXH$~P@P`x|vM~}=+M(Eb9 z;lfJZf6P)2#-iiv9TOh;GSmmcC$h#G#x$Ptoql`X4cxIkOQYrMUaCbBY<(SI2Lx7= z^}S`t)J>q(tSz=epmjg_FB8`--j;`e&CVh-*~<_+!*06k9^;J|8sKsJ_gL8 z>tLNH=>JGr3WwVT?HJa;Er|T{An6N52rBko`B#`5J3swRVO&CUz$Dv;j=~Wk4EVOO zz94PxL83jnsDH1Ykgz}JIk;=sz|;1|-%0mhcCviUQ8JZb*vrRQgo|En8{q!b^t_{K zk*IA#B%fkQrcGNm(qxG@dCCctbNKMXbw@fASV5%6VutJ+9~RNy6p47NzZZ7qh8wYB zcA2{=?Y}+1s~z~#r;V0jS8urr-l2ZwO8u5=?=B2*I)9ca92<@m@B6LpuOnMhcy|IB z=b`8kHO6nS;Frdk;W?L7^*yTb`WwH4hTz`CPZ0j{@w>=xkiQ@Kx|DNSmQ_%VTn#v3 zgySJ8T*DcC5g}n@dD}R`*W|jcf7IfNpnxf>4uJAWs5UEnJT#;kxJx@^D0vOuZkoRc zR|j@3q<^PNv9d6BQ*?VyP2(-^{NsOOiQ`#}L&6R*f*sW>YC_j}Lx!Aeje0NXo_BH( zD^lO;k+HhT9#-V#wXoaP4ms6Ug@{t$uu9u%5(=`=)FpAZy|)p3R2v;OGwM~2*{uzT zQv*g>Ig!wqB?ymnZXSkWV*b$B+g$GM1b@0nq_C&QyATlx<$Z6~qrjhlFo5-(6iq#W zuV4&n105N?=>fG!>1g(>dE*R`{0$hhBWNSp`LsWzKx2RuKqkuwpkM9+)d-sZj9Q`xJ?fqN8HcSBrkek2C*Q0&9dK}7Jn>^ ziVi1x$hBGnepL4YlFzg09`YLr4GyWv1Zh}fJ4VycrImpC0l3sS%8vf?SkuGU!wPpX zG}-Vq2djSFab#Sv59jN09jegCGaAwD_NbtOu+tLh{Qzv#N4rPp?CHd28ZpJ$epBij zNNW;NZw6`s!2qn<<{htOf0;b634fMjrEy~Bqn{f$#*klQg~d-T)&oj;eV=R*1lY>p z3M$W$zUxq{_*ScXsPJx-r0@{7sCQmpPM_vYqx>H z!X<=~&xf+-!j_?OhD3d~Zn(SZ#$4H8w z!exM$P3pUQBiREn>l9s#^(TKrx*R=;G_AH-n7MnS8|F}l&M{a4+e$H{3Wss+p4|3q z8H$IZ8v6N?u>luC=;T(xUJS+MzR3xO#dxW69&#f~JnUj!p8bDuBvyB+`DBr5b4=$_!&`#E3fvw?y zg;)r2%yIG5TOfu9hL@UyKDP<|mwAI#5rj`GWcLnjcgXJ_pv0IUJT$|~2gRp>Ki&h> z3PIqAILxzX;GY!M8Gkb5K=Bp%b#!`qVK6RHT)Dv!jsZL#8E%D|dogNnaXN8-;#wTF zY3{S;Ve)((#DWv?CB>_T)&7EvAX3Co+yNz94K>A2ZN>lheF}@YY&1l319@@GKtU*Y zU^gP{3ilZ&Jmbo}<>0&*QzQ42^ZWuItw(?}UE9ae^w$-^cz^GNBZ3B5Kfhp8`{@g0 zyJyq-+Ak769@^>1vbA4*c;p*2JlweJ2cv2ZCZW^Y`h465s8xXft^9``?TNL(Nh&*J zVs0i}EcJ~N38SHz+&iMP*mbBH>PH}J2fo5!mW3jyS2HxhbOsa^BMVs!y}8|^?BC(3NVdkVXQD^s@^N8gCY z9nc?=CO|Wn9fGq^xr4@Wo}gV?CfI}?LUllr+4kb%2kFgJlL696@iM(- zAo}+=PCs*giTLYfJfoP+hciL7UNW;|c1$8;CARGl=fL?z%3FD#5IAOXEX@Z&ur;a2 zbI%jTp10$emO~^sW?f~f!cF+PqE9`2jNIfrR4zS}1?z6E-M2TF|1Y%IO&vii#D9Qc zDaZc%J9T;8Vs}S*5kjt2;m2%d?EQPU%Om$egPPUop!6i+^`6~Rhs|QVU$_N)K>SI6 z&&8#slrjBs&UNGO4ek)I&{t2?wf=baG?w3IQse;!O{=Bo1EU@P>S#UX=n?^wfIFSZ zq4#w0!eKJ5?5^)OJDDXS9P=Pd^nZ6n{p9)wqhA7qd(pyfP za$L`Ajo2;fg>af`89(luk#zJfa=@;^EO4HjY+klUN7cB%3V*onpa$6)=urJ* z14ZJBB}(1K{P#t^dD2CN|FBJU-%w80u-xX|3)5_!7JZXl07_99?Dcl_nj@?)HtRM1YhSyg&ON2L&-}m~aO(CYt^?zItSUe!TnY=qB7pnSV zF?`oI@ zNsnSUAQZ%Tw|`YD`CuyFsifVe3q-Ag91!eo#vQfz_$s`HI-lzapbdO2{>^|BLo+!W zBVukGI1YZJ{P1>bUPOLpoSbsX$Z;ensCcd;$i%L{_O=)_2g3N+@`%DNb|S`E0Pu8E7!6D_X_M95e z1;S_Vtx18CLf!XVvo?8Apxt(@;#2WS8y|b$A|S;pCcGT&oi9yxEx(OpRrwt2U(b%K zjoCxBwVY#^x-|*vb_kx4+0yQQ40vjy%?v~>dH+?S0IT=YZVc`KLqNR0Y{^TvNn5FU&gWSraR^L(nK7Tq@MI5&S5@_L!@ z-Ti=u#o=&d?81W0lW@5Wpy1?O;;l4xh0VOXWv?J@AS!=qv^@eALk@Zr;f^CuB=kPo z5ioNKsv6EYvN`-9e5-lT(qN3BQa_k>pkYqY(A{f2o?~pa<1Jz#C;yAj)9vznXMwrH z?ei0&$2GlX&t_On5S-XpAg85+FKSoqDhZ>7s4D|sPD5(6ZgY5>AVKk%@wv0fQc=F( zsX|j;CR~5#yeF_p1(Ytu5!UDKO6*6>qRe3gswJ0<-*KsWep!?MiO`UkRpXGs$r|KF zy!NHuLw2Mvr_r?60=!OX8gfki_}~-&{SX8p93ik~ev5N3V0=)bz)efPd|XWMvshYm zH`&X@Rj0X*&$#Mg(2Y`cIt8wwvm~ew_>g*^aGHO%O;SwwV)+BV6Pvgr23|M2IDaZd z@s(LP-E=$eWCwNuv=s+IrIRTYCk|t8_u4zWK^QXEIVwWH05}rB7?}a9qI7)kX!VF$ zMR=MZw&;_9t?tJ6rn#g=o7p%tKm71b(XZV_Z_5lBhCr_hvgmX=zeFa=@~pcw%~{`$ zBdUL&GKJJ~7`cvKM$hvJS(DQHy}MyWpk58#p~F@^C-_cBbqkI3g0mknW63kDJ7l*x z6veBQWeXya9woHAo;l%iH!=KOUh>*|A6J#%ESJBY;|@#j=mJN{p@DGCH03fJJI+++ zr5gS&J0q^0J2`lj+v#Wa!Ir+lxpm{K75aagzz#0p=NF~t#QSWYLF?(*s3KSjtA7QQ zwUAh>6MpM0W0hscRcuWu{7_#G>+TLKm1wiq(ko|VeT4o$WV}OrEsO#+`JNNowr$(C zZQIs~ogLe@ZQD+EY}@>9_oy*lKVUsolS;2TTX9LG&>>{d*lGrbBOFgQ4maK*QL29w zW1yC(7TCQi$lQ+qsrI8b`n1aXPdp%wph>M}G+-&XYjgeEa+|uWf+HHHqGF~w8%$4= zwxpfnjTLX{Fb`qld(Y^+$nVAp)FOAqeGMOt!dHMs@Ty9fzx3eNLrXs1sKHy%$qgGS zIujy}8vnpp=$ybw3yWKn6yAZFZxequw*J4$*M+$V7oYA39e#L*4usy_hp+F99)Ta@#|jYHTi$lvEYR( zcxUWDEvD|3kJtzwMPx)ZcZ%byhb*(wkx|i$lo9Ui0Zp54pSw9jN;hO|)Q5(~ObBi`wERRt&le8c1qa5@KG$wzeS zoHGM>g0_l5bJl7|gC13#=Z@#=uMalE8vkGG=*~u)4Xv7az2OkNti^1~hKP!LTO~kvI2El;6x67T8izSIG>n z^vh*AJ)Cm~#WJuMv`cm8sr&XBWkRr;qVco2ASSWSBYtz?!`pwoi(+JB_6^13C2IUbvQOx`Uc&-u{2!R1g4cfCESvX$zy}tPer>+y?&VGvRdRT^nn#DEQQ*9$8zz$Hr>_ z`itsD7F^wtG@U9p=D2u&znd*zZNB;;fI;zfE8nU1Vnala_uhN|q|JSn1aacOBYhN| zUR11)ho_;I=iO4AsPucdjgA9?|Civaxtx0!4C4G4-NJwQ1IgQScKn1RZ(fj_{O!I5 zp>a_zz|M9eKK_=zvt8?7*yBDQ%?2A|>#p|Im-K(LX`y5YrdD^`jsNT{NksFBNJwWo z!CV-Cp2d|hLS7RVS?oA}UQ%nw95&}hBz`->^Biy>vh2@h)T#oDgzutQ*w`e=`BocP zX+rns>r#Irr9e%d9r&ApQ{wmG z?hs2(S7b#l&jj$gjlEyPxYT%Hbb;hG1-vo*=M8`Fi;}@bKVTBlT0Fut(!ci2)&1i% zQwgRCUN!v5=kYpb%(_fI0e*9c0xvDn>;og%Q_0Czdcb z?OmuS2}3XCnCB?~d_@<|63;Ah?oQ|KO1dgsM&RV>Xf*fzT=8dF1Kj;L&EgR0?~~!h zHPn9)z5+Ub>jf$fs(awY*QDNZP^jR`L*jP8=zQb1yRQ<(K7>C(xW3-(=<+KbN|}Sl z{(ar3$jekmKC9;#YM;@*{bJ_#O~;|-q_&Ukr5pG3_NR$Y^QL1DKZOiy9()f)p$)sw z3lV?&m>yqt7*Yub`w}0hfaD}9y{t{Jdrf~xY9|!dmBdaoIZbrD{0D1!Z(1OO|A1Tv z4OqNyi|)>`mH=zzlP{STZgMTDMLcfhp5j?TYsy_G-Z4bq>u!D#aNqVv0MQ4{uCbjs z9OA^lW-2mEk8#5iDVJk8#z(>!JB_}yfcU+oFyxxjw>$Gut%#?fmvU{l&I*%0{I-8u zdd_t3ZWvEzLW!I8#okxZ${<$rwV=r2C5h2T@AjY3DdEAI!{Q_F*k3R8%I~l%V%r?Db>_rS2b|A-cID*5Aq-ilYA z>)O6l1Z-#$G7@5#up%qC!DC#;b@+KcY4u)aD#?(VhzeNpfS2`i})6^9dV z2yv4>Ek{6OS1$NDEd54Yu&nYs^u!qnKtAkkQ6^%)=sUHh+;_XJOUV9#CoHmQ+e(}g z?dx+t#l#dkuXQ5Kc?{< z&1$YWgW~tCXbc0wlvI56dUinWo*g>dz2^7d{DCC$uyDaB5Z9`oT#i@SNpFVrP%6+q z8tIsDd!$sO#x^uyfXP%V{MSPwb)Ud5=N#jwsi?`-dKg5DB99f$444dxG~b8s4dP>p|;pR=B(O+uba#VIIhxW5Rk29s~@5ZDUkL#@%1cV^^MOl}dj|8>ePwNPpSLcI~zx ziGH~F+qj(r5kyK}UH-N?@2h{?{fZIae&{E_bB{yiW9sPVqH~9^8S@V{ZwxukZm@qj z3*q=Pl@^QEz8s#YErD6h6gk!>vI+PYp_17E8P@EL`_BY20~6FRaasI%e}1~rCtN-i zVq0R2Vn0t}4Vr%>7&GraUpkvi6iI8kU0v<)grg2$BGQ$New%puN>k$UX5|8er$?R0 zl;S%$l3`DSx1yn9JP@7Q@mk7>L6Bm)ygJ@j2r0Hto(al?OeIFH$!_zED~`j`fVoRZ zj`qhShYq;Yq}T7tY1@H=B10E(Iy!_W0e9&eOY%<)-z*8O| z)I`a*2KTD@;rw6-uX1&B&7=dg786;2`0({lBuq~3i9*`Vd+>6;t zlP4<3{14-m@siF2_=3931vubv&Hfp4-HR~g1`V4vPUeApG*CS{L%;IEU{lm3lH2hI zC0mu_xa9!`5oZcx>Hm5v_x8=^nF@9#w;n%$q1S)Llr1*n@sk|e52ZSi@SFEir%zU6 zll$t6NQo2&rr)My20#_n_hmc^HpB(l9|B9^AaD$|jI>KBq(;PAPC57m@WLgJc6tXo z@yf|r0_vjo29!QsE%Dm^=s%r}!%R|n_Dz2{Um+cQQKW94!PNDC6xBRYJ3xD8^!-qi z!4-dma2tD*WWvw9BjaDl%i``(vKR<=69hQ)KL0RI1PTW(RANG5OaFv+TsDo|)rZayk@9O!o|BJqaQ<>md$CX4kNVV@{hV24V zIJ-6?)rF?!WWVKrFcZ@{DU0Z%x%6sOzX#pN0{%AZRO5we|7=IT(OuWC;i1`c$_*Tm z)iFgb$y)Yonz|aeLK0sk@ACgD<+r|)e&%~)P*)=4%x|LxqHl;i{_xC-9TJos=gEIj zjWa1pCbVyX{rN5*SfXV=Lh1V9+e!#tkXykTjNy;i>a1@T<{~h#r+ssZ84Ozf0yb-b z*C|nU*$pgiDrnA^Y|-{sVLaU0tx6;BjMI8vct2e+9%ZKX$v4gPNQ)o z0A2S?b{>BH-TBBu44mW!7oU5^i@*w2nTR@=j=B{;NPQ-cF3;Oa#NMY}j?-i~;iJ21 zObp_1(bU~{)PrsQV8MS3BJAD| zZ2DTxF_F0N5r;V$le>q>jnt`2i4d(6`SUa-$F+|-nE;uws@|yn3%JAaopSR4u37?g zGLX*9o}s&?<4o)wLeEdqGI55ayTZ*F?bB?9J+`|x{-m0J)-!l}Q%g$S3`(85|_OS)G*CEaf4ACCa*u4G>{ zVXxMXh2b^zH&_h}w=8!N*WqcRj;O#5uDoFbg{S`i=|$7;4C9mDqA#6G=sLC^wdVOK zC-X2MC(B1gFuM=O$BGq1&ZC4S%N(2#MfcmWyaT)nQE7LZ89<04GmOD7eFME3=;DtOen1lck z=C2wp0xr%QKY(X4FDjmjSq2*7_u$N4vq@hcxiKp<*8 zp5WNTkG?hHdSe0w42VzHG<9g#O7hHf z=&7L5(+-r&Q@ww7mYlEk|iQ_DjkNfWDx ze|$SPI?@G!15y@EhakLZ*1L#ZGBLm4XIqeo+=1JKo6jNyu+x|GODT~!Q`j$a+>UZ_ zmv-mddWS<4g=UbwPVNYEoJs#@L4)QtCDQ%Mi#<|m*!F)Nt^p}1(CMgw(r_!=mWczv zS9bF5wusG*5B{ZBy*0HbYX3NB+}&`_Xy<;m94J^aeY)UFN%7`@t6e!-tlV|B3#y?LfZMO}SUn~kj zIdOk6vcQE+KG$lEP0h}UT)2nu5Ye>-bXJcl*B(x1Wv_Pkf;Peg7NZjNC)+!BLx;; zmglY9&RCGkm&Ww813bsYFZIS#Sgg9=6n8C%Wn)GqU;WzP|19cC(#b{`??dSkIktb- zBKTy8Ushhc@}Xlf0e z&@moKQsjgo<(qw0_i!y{2D+z+eN-Xo@1t^Xl~pSP9R<_TIU@x{4O`8yDgDs z;8%>0vfE5yIBal@j^bk(;H=hJP%xXafPjXA%GMwY-u71<`o5Cse(>Mf0~K3#(cT;f zmX7og8F}2k&?4J)f%&*+AJHMEm9{`@(YiApnYvQMZuMh>1z)cfO!0?VT-bjTXeGg= zPPEd=;M{0k?f(`?x~Sh#stzqZ^K{YP9bMox<+!9@xO_2t-j6coD*8JM#Jsw0kaxVz zg6E#}5Ad&{r~TbY(CTKJ@Q976DC13bI$u&aZ+(L~8Vhd=@W!G9egNPkx6yKoW=fdN z+cq5!5r}X&9y*rG363R~V)}p1bSS)s4dK^bXr2wOK7coD{?#cOSRZ*OCECS?P^87s z&KhlM3_Z#_pQ4%0Ql@|B*Wk+vUVfCVho$5t^92u0k9(DZ@ZNbBpbR-T196Nm2o0{} zzPTzxKVEl8ww=J&h3!9@dc0ZPb)K;Demo>LSbXdBM0wfoNdU>UsYQPjaI5$ ziaLI=HSLjKT65ZL?{u8`6Ina^e<5q_{&!J&@QrZ#QporcC_R{HJr3Ntu4J1DM=qL@wMzWw=Xg>=LEtX=jKYHJRHz63Bz2Q zLtX9SJt(*}zpr4g^UkUCZU;Y;3z((E`%YrUo7cO&+LkLl(5}54EK=~k58b3n=&Z#} zAn2jK3f>|4Eg1$uK;EyZz!31_vy+ICf5oe*gFWL?gQQ4{kN1DF$(o$r&>t-O^~q^r zMbWQ2UIQR|l{Tmp8Mo)`wS*p>oMQXN)K2U?aCRWLcA54Tpo6JSP#lEwvp6y2XIA$7 z3oZ!n2%GX(Y7o(-X7z*v!T9C~KZHYIGDhWEhc4L6$?=~yg3N$`yOKDNj3;7Ak~RiI z)X$wyHyH@swatGbs0``*b>T*JES33JNuw@p7Nu)B!;3EqZxz@W!5{is4-=fzye7Sp zLn*t}Y$^hckqdIw!xk7D|GbvAkuf#Ui?KgbWb8_6_0vqfJ1yF1w*R>&b03mc@jlJH z^any4xT;gvRnWqypmbvh^VdgJd{>})jMwil`(EuZ0VJ&q_U1|z@p*1{@HxDjzK5Mzc_hq3ycXWgu6ctqbT+}_l9bJZ6kSEae=NtCSWTeB z{RUfSajU7MvliU?a5o01hBAPc+Qvr22Fnqsik6ynO}-;Dr-Z`gqyu>Cnu-ZO@X55f z7mi_X8T@|~T-^wY@?Yj&KGnbZHwg0@+1;|7~nc_#|7MUID3k%c>P zgD-#iV@M;p@KGh%)h{H#)hUjNLzFUJr-3Q` zblZ=1yZY51Wvsf)v6TJnV(3P8oM~e#Fzr?{cJKwdPBlwiFTfjhbF%eIeuXV0pcVh3 zR*I$kM|;l0*G{BkBSh#;8 zY26mj`xE&{4!HaMMcGIJIJ^Ux>fD1p&UoQTz z(UTC~qkJdn_8U7kDch}Y;t_Yr0=>&X4ik06oj;L0sq)xOxg9&P{hBAF=Yra;^znL(-M zD-HbdGm7JMGHu}SQ7ATnn0PJOonkn<{Ga62O8aG9qv4_KLyC@+W9;gVftL#GIw>~i zhn^1%65sKs&_2CmOmewSi`>5rldh6BWZ%r{SIO);9GJ&&2n7E}>+qeXm!(;qs)T}e zukdFq=tYLK+rx5>hamqfL+RdLc4wD3^w7S zhLdMzrPp|7qyXdC%JpREBkbmEB09#Hw|0#VlXo?DFm-i zW$UPNz9-Btd*yy3$-sYYQL#di47u9#8d(Z~7Fb>T)T~hk zr*Ai_g9|^om1!Q#&_f%@ zA{W?JHTpF22&-BhRoD6^EWUhn2r*U%AbdQluGjp76BRHU`P;P+!s0NtyyQE!kLG-C zB{oCT{rmQ{)cDUd{>*7hOF7n(n~j3#6$Gb}ICE`^0I|2XY%M~>N^p+=z)}|KQUC0f za;dbSKTTIyn`eKr8`wl$DRDxjcsHfMIye;gcVazB=4CgZr26-r8)$}GOfJb+uFD7g z`<-O((mCej^-avTDuIn|iBk<>VKE}v03R_e? zqBrNIpR4if;#Yh5+ts^^aT;?FQEdDL7fi1Bx_^6h^&e;*S@Mk&;fcC|Rau!jv2F;sv=M|>D^o>T}}uj`y`b&%QY$VuGJwj0>P2PVsBd(11e$K zl06?`fe-+8RJ_eZhN{Q2lFtQ5+MPx$!re(hw6wdpNx^OQUZ>6nC*E<-${(O4B4SLA zW>Wkz)l1F-4oiudR^YXPaAQ*isO7k7OdJe`!p46C4GNF;IL<1A&&?4YYrf zz8YJftaVTuU;=~Akhv!wk~uBWS7Q1r-;Oj+;NH7A08BzRlPsj*E7yd(wCq6{kG!qn z%HfC0%(BgX3wp-3>V`kuAOb0?uVBF^{W!q-Y9y<%9_4ax+GPF7-q$KNTqX3i-wlOYHO2i?CKeCA0Vp~KQ6z$s#A6p!+^za^frKLvi&JW0d z?OFlYJ=`Y6RnH3J*j(=JxXGrw*6gz_(iBwOvq^DeM3J4$5){HaLCMEwy=GC5Z2=+_ zBDZTWv3{=oR;WF+G~rkM|xim0$ELtRkBSjPw)s2S+&_2_oNRl34>Srir`^0$L2h0GWXh( z_6irh_MDJ-e$7h>aM9D31iH2Jj&2jkm98zJRl;H1C&2sq5<(MCB_TqH0i-*65J745+!s=(m2w3%W2MY}Y@=+91qqZ#6^ z$m4l^f;I9x7`$vr(PMusv&qQ=oD2?SaZeI_*^e3ZO35RortV_&-Kz#(llP5F{K{C5nE&C{k!a_DiT`pB7ZO0CTCh<=%N zXjI!b`FVTjc#W(OQDH)4N^|POcHiq_o~zbZn$@7h-xENe2VQ@-t7eSDE4)Bzo@8h` z8)B5+61L|{Ax%@Hwb<8H;MF6#Z+U@P8x`^Gi!FYqFjTbNXVIj@M5N(47z^Ro`B?~( zsCt!iU&$0OG)%ec^K&Ai3aV23!h64GtZL8Y zF>3B6tX(^v=v;y5JgR3Mo@7kKQ`g-s?uL2#ZKlE&@vVQe=eU34F9W~GAWp6F;_MVa zX?uEA=XCYl=OwX|Iu024hg-A&9y0#9y5f)X_xG&sBUPPx^!8YQEd_s!U9P8IrigdA zb_Uqsp#xn_-8=41Pb{JpOHU{H#>S>3M8Z;rv&9cw+CMQH$4~9zG>`@H`)!_is5DZX zioQ9|z7&7??mF#Vvx8t7s^NNkh1`h*YLkNHP2K2q|eXCKnjF(P&wLsYMVo^&e zlPj;n>okHLW>8f3Sz1`rwkT1S37c z+OaNCh3RlB6h{cK7^ee2v}YDqr40_ad%Y~>^59`KBW>dsO^itvIkc#_cJ-{7VCX8x zgJ^$htX)U!0yw5z(c}_+p!ROfc4kZm2Pe%o$S_14@Gpv`B5onq6t%GWu%TTRLip`6 z>hzDnV)|5QgYfTDE^W&e5+FY(x#ASwfC5%eB3(_FbILQ8FNF}zEaqIm@DIQ=C43R; z2cEVJQXb>}dQ7o0l;s_k^WH6rQ32vsx&?neLQQ7o2DD@Tk?e+%JCWE)aU`n+$(K z-jjLHjpwq{B#wc9JZT3tr754Hb3@bo8Z&vPH%>*`c*xoL)7V50oV;si!i$M>{m`m? zYql3c=dY-33Q&_X30I%krvkj;@q_gT9=V#p%sZ-^_Mh|mqC^0cef_S@1tD>2tet|N ztv#Y&&9~7NT{AHR0UeofntZJB;fH@K$PfgpExK`8H~$i#F!CPydDtG)HEw$mp5jYJ z?%_S{m>}NHrF`rW48zSE$vck;+Nn2SQjU0IoDT^jZq|Rfp&EZe%iXXkDAdIGlk@9v zvG}(DVUTX;cxFA+7q|COW5WBIaG3X|lXZ(?HnO!)zUyVc^z$wOIF;Ui--Unu0Aba> zvRQTBX^*;nrRj8D%AQXyucClCe4Eofu%>pNI7Gedam4kZE)p z;h|;#hqtCjm(QU?j@^G|q?wY1gm;yzUj=EZlo#@z)?21Sr(1$jE%*p3&}OFux3z;e zgi{DO!!ky@_ER&1g2xSp*!oFdww#+zg3vA1TqhlyyPD)uDM4~6?C5vcrC6q#n)VA-|TX zZN$#-knM!af`0>6D4+e^P@@w!L3< zs=OpUU6fx>Bc&x@F&M_{xtZ3WAyy9HG0PT$Xu5ybB~`fIwH}$F%okCr*49slyj--q zkCJX3brqvfWv>5SQL(_hPr=GIuNk<&Nd*`5`V_@5L(LdmKH4~A=I_90d%DL&=iLtD zQ5k60e~l=9Z%AYNCk!4c%g+naG1=iijE^>&fau+<;96FIY1H?@iR71d?XxE=8NHNF zUi5$9hHz)XIv`(3T)m@3;l(=kX9%>&W+Z}>{2RZ?y)o?LHdp`{3qsb26U4hk`xeW1 zxix+~%e)ntXq&tJ9QS!Ehg4>FSX^(Hn8$xa1B!j57GW@zA(9Pc2yku)bDxqQPD=A; z@&3QdjK)hvDJMvA%PZSy473&n*O#4=6OVd~g>Y6gJ!Zxh5h4pIAh0pEWE2RZ4Mtx) zpQA+?^wrYo@ZQ)`qa8}yb8}ETNIzLzGM~y}JFfzvAZzNuB=pqXJ(=)r49`%v)|r0^ zzIwJuN%CRMm4m9v`bf?OQ3lmCWW0r}#_pQluwB4R^@+DfN8fPk#RVkzT)p}3UlA+R z*cFM?)$-W=z!=4piv%Od=SE%r4TQqo!mIr_S@zPC)<}q>r~Tf%i? z;q@|-S*jr$^vDi7Q1AK=0R_W#DIWlQKP+)f$cZsU5ivU>35kb7;XgG<3leI>hF zA-2@LgNH00PT-xL#TX`EA;FJ=hjZTl5I4B-D9e&e{6>JqZemg-(%!{B@eLEb)4cKM zf@YvkT(#nMl+*RE3zqM+{nm5jTPm&`FOOnnuSM$s8;A0^8(mU54wDeT7kq!5j6Lh( z;!?3ESi!vu9L-;7Wr!jKN?{(u)YnTV=p`Bt%I3`0m-pG`Sb#>;5r$e?T$`zog}+U; z(mlVR+E(F90$bmi?FID4!m!8OA?Rhw`V4a1TYQIlG~rk!n#0e8IGGxk>N)wZwOgdu z3Ow^NfBVb$?N^94Z~8wU{Aqt$>75V_q(^Mr2DeC`zj3YOkaFILJ|f5Gp%qof;|>~r zvIKO0xYV8;f1j;mLUtZX8KwpznhWz)^@-GN_hjAZaUdjkjNsft;*P1d>(;x@6k{}t zNXVvzg<@UuEB@k8T}+aIcXhN6|L$&31+`&FfbQ1#i3qtm^i!=tBDa6~!IF`Aqh#M6 zC3CozW1(0BuYa6u2BEo*k7Pu9{nrtbiGY+9&Ij2QMZAy##ckW&;Eg9q!ZxH@wjI#v4Xt~H^!+GF?Q*7=W@Ap?5&4ZJm z_hvh69iFlM-P~OMbyZjs^jyd~*rX;eoG9=^+i;t=BoA0zGarAvY3>NUU`D6Ew(gBz z7JRcdb@ga57jVbsvtNI-_^($5Na4^;6KWVe*Nq;4FRkzav2NpGtRZ2Mw~IM2P-eb# zi}|DQw%esylaOTMbl+#^A>*PY6^ zQ{ERJ{N+s6+m?U4|KGfsD&9^58uqHuYL`^H(2_9g^(Q^}UVLqbF=r!VY0~H$_s&0Q z_|tvxyfc9cvhBIPUy#b^e_7qjl=Jy&uNf&-5Snr|!W;1y_N!hzivYA(5X1ssYR)Hv zi|_Yp2r%q#n7L>J%xGuqc^f~kCIlgV7S+P!;fQCiQW1Y;Oc6HK=%-(3y3PDr_xB5h zPMRb7lMaTh#=zk<@?AV#Oiz4cF~70`oeFdKGM@%9MmI*vwCK3+w9u$g?K{H>DX8INDmwM4Jx*Ky2W_}g@$F71vX{joQ4T|W z$xZ<#Je&_I3PeV`oZAd7?P319!d#cI9quoP1ce&l|B=1?`Tt}uk!)9eG(P_Cz~z62 zOMX{-{Qp<>^8dg7zq6MaUYx48IHPLG#mZJH1+n9a?1@{dt-YvZQq*+Bbd*xU5t0#! zIn93?DYjZox1ank8B8=R$$Z7T@84`|ZYR38UUQH9bAACgm(xeZF3fNc^9L`B+n=fe z<`O1W4mBN+d%D}nvv~bSkLSOu1el`}N*m>bm*VR=?YP-~jW9tCkvxEp7{jw<(u%~v z5c(eN_5*hsWD$_twp|jTF2{Zw&sTf|Kr>htnZNX#ct_ z_i9{W{D}i-?+!;i?TuvHm%qTKp``!^qkol`OW)lYv58($uT)m-nj0t*3=+$R=QSL! zuT?R|1HNKwQ8$1GTTXx3 z6@BHY3$Q-pr$th6u5lXt4*_3-e{f!}^f+$_ha=8zYr#3FF=+wN3wb7ms>e!gf^={d z&rsnj@Q(d12kVQw&jLa`Uhl3>8W@dvbhMz2Fp8Vat}0Rg;dB+US;I|b-k%$dI@9=| z&yIdNr9r+5f3&djDR#Td1!VlkrU!ALHts2#J})g^h8x&pdyfcY2(b zSX6+3Mn2t&y3sS+e~`X+$*el(tZG$Jm2CzKAibm+c1dUC?=Dv<%*SaHX4jx3Fflgm zfq35o>Pzx=T{=E)5`K7!H;zgcm7nFvJm3}yxz4g{R0hKR0_3^Lt?}kP=+}}i@yTX4 zm`b}#k#**z@5Zm`gYLcQBDa47mg*C{-lKLY;Prdh^=%7=hmS$+G32Z;zXk+E=Tct& z;d+bZ%l-27ddK4Eb`1$V5etQj`4wqiXnhOshQw6mCczVPP$AYJUDBB-7d&|ke7`5! zo>|}!1Y}JMt?X=@$pK~V5!M*I{0oEq?H^+-Auu);GjwJH%m&t1%angFxzNHW80dL8 zXGXLBT^yN_lQwtsQcnL4i8c=AYE1U9MaZPuBUIr3r69cO*7{;p9p8^d<7Taflv>#! z8XxXS?fm5?J=Yw$tCo*2GxEC*r#3lNI^*1RgR&={T{?@k3%#|F6`k|occL6qP1+KV zop8t1YqDnk`~!hKXncR7%Z5(wjG^vFZ(bzXu{7Rs#uxPUfKBl~|6$k3_7h|HZ`D+c zLn}kbwjHZWjyUGe0(hHlT{StSmnW9S@G(g<*+p=b4z)R_1Bx|8XM(B__nxOQXy?}! zFai15QS&-w@6`!2Nbj!!+UG?B)5}L)h9k7@qe&~zZ-gkj6uEyKm3ZwKH1fB$D2^5fBKy?WP%ktjJg11l?b+ZD=rcNssRb>2H5()=t&|&N%!_udEtbjr)GA z$9I!Wp^ot`@7QWPpLZF7+{AME)a0+#_z zLzAy>F`M^Hge(3)B=kYQv20j$&UItl89ES@Q;gR3P?{LR=c8~FkDpNys-QRv1TM$) znd*8Mg|UAbi!Az88)qY2^mngIGjMiK@ToDq8&Er&>!HX7bmX3k7hNm~v zKXrfIHF5!nPw1gn3@Vc;0ROI8h^Mso(0i5rap4ovJve zQr@$YH&n4S)O}I#;n~NOkI6rxko`c8nZe{nEtaIv4z0TC^;|^kM7^_ zDWd3K#4@!?BIK1*{}F9hPpw%uGE)DV_1D z^T|xmQwx~1qmM@numJ`J8KVEiA4T8FtqI?3Aeno$7%o)4Ub#pq8Is)b5HG%AD&SGp zW@zQF9dY8AWI(w811m{U{d*xL@)-x$)%J}y%0JN8I1`M!&HzAu(3O|y9*c=TKDK{# zH9*@;uvd6nWNNSru=t!`EPa*$))27g6{)P#mBib{ee*MM!|pt3k4iWo*TV4EWL4gZ zi1$-`hd!!Kb4L1bhr0?1$jO)l0s`k?9V$=@>qyP%UGsE_3p96B!MQ zK1aOQ3TsfS`H0xOr8-dz(vD0;!Qp>|8|Ho+BT?a8Wa~<7UY!(^1=F3E7`M1xa{XPg z2kAT7He@hfgDk#)Z_B|Hj(T2RSx>?c*k8}4{(5GHJO=W?H~59O29rCRyy^%S21R@f zv7PfeMm@G`^3=L`2y;JVD?`%(-i<6A5JkV|)C6&d2g28Ma{HFhK?leo{-S@dgZ7_^ z`Qitik+*(XmwD&;HFfFutkt4ZDLkYMm*l3u32=aM`cU@}t2!Y|d{3OtIe-2+u)~<7 zSdt6qX6`2l3!HA2xq*W9={EeLqtJbK)`hl4`$ zpM#Z(Ga*~&I{xeeJt&c9Hx;&Ry>{pG{+3H(*og4D@;9Xg(Ccy*ZDY~AD^3EeN(QmO z6#|mS%pW2}1pRVcvxWWAj?P(~uqb^foY>dPvvL)vpF73ypn4VNZg77Z2_yD^n`c($ z{mf5ES;ok)Pz^jOjKuN11z%sIFK&>YRv%LmX-US00(?o^OR$a&1BK$4kSx4quVbF) zcFrHehhu7b>8s7*)q2i${0Y{3X7WIeHewdhp6lgH@2Y}-t@J`8s*^fQ zaLme|Gtge-#s#%Zz=w{w;5y~yel@Ql#JUbN5={NrPM4@eZGOCXANGp}jzG{Rg&pt3 zXRda&&45MTDmYePYN!;=!8fue95P!M5+h5y*PSuQr2g`yz0MXq%_P&D`!m1itG(7u z%`?h7&heZL9kyUU`f+K0;#>_QI%-zaW@;2O#;jeqNss63t>~ftRBJ z4b=PXp??<_u>)@Cbaz=HqQ^Qb^9zlUox|Wz_#+H_E+}l)E*P)Vqutnh_K&*z$ODMx z5Nqe@(eL!hNXS**Z=+(V%D*%S?h0H$IY@6*uNSZe_^<1t*=Z|BEbPkQv*y@1*EPOY zkK{q11q`Uh3uBFc7quT7k2Z~#8~OZ3kh$VnP~CsAgh%~I{L4^o>?oPBKLNSV=tmRB z3rtq%jqPP!oj>jeYd9c85+&TImnF#$5{!lvbcZ(8E0AttBdVu};8DtC z>f01gl{Zp9fc8Ot*A?+z%Z~oQ+knev2UJ$V(dqh~Qcfs;=Zn*&9p4*q2!kMKD$D;; ztJ2(KbFcnMLOud+ppEH`e|a}(G9tBvNXQ)6VKA;^0ZIXkNc|?-{&BhCK|Fy3>k^cR zsQ%3(`*dg`Q(rTDT@&>kQb~?kQ-hg5(_kCO^t|fe9HV7V#3y?wfuCe8mfovfHIvzY zG+o^>8 z$N8UCf>B-AD07WtSP74?&9*P`?7yrL@A=Cf=aa{t@wRAAN}-X%VyWBJ79yUM1J|6r z34v$(E)2$ftR}m0YK_2Q%5~H9Pb#ugh!73uu5Ao|kNpvhiXb^&*yOQSxDzZDL7I5r zUHBgQ|3sP|bO$zMaYa3|NeT)7H^yKSO?u!k*{Zl33-q@ls!DjHA|HpUcj5|X0L`Z? z=2YSXsj}%|fUZGSR?>>2d^D!N^kgRh#RfQlxO%wKD{|KEDKu1qgx;#HIo}Z^+S4cd z{M1u_6SkByynXmXYCHdM?5kikFur3BvK_)cyya+O5%FpWA>{<3lM%deM2r`EY3eCW z4T`#pFQ&Vwpts*en;x6ucmEY1*J_6QlM#rG>0lBY;B-4!Y9sH%r`uUTu+{ke&wzRy zAFPv~?pHe@7I z2)=3``}^$Q($4Zv#qR}CtufMvS0?m~4~O^PPY5QOCCN>@@u1~r6brPAigiQ2`qNtq za7!p4K_bXzy^B4K+s36)u>w8IM77~RiuS41nHwDRx}_U^=RSyM6oKiyEeqj&je>W7 zKMeTkCutmxCXxB#eN>mA_FmT`(W@NEK?faSUG42`V`G>V6Cu)!X8o&{9 z7O%ktbM#(4ZiQiNkHp2gj@rl5xsf%uWWy$7UYE>}@fhh#ZO8G;hzzT;Iyz*StToLt zQ^`?r**~Aiz0#pnesD(I6m*E}U#>)de8KR_T6-tItv2JlZZfk{lgA1^h~3VH!|1ZE4H@Bd*t_4cIdyw5T@J$ERlTMv#TwWg(%1D&C z&V|Y34z*KysqmkpqZFFd`Qi7K4NPnCUCK)T^iL-d%c^L~TZ>*SKWhsS5eY+>#IBT?yj5Ew#7#NHzVqZ0tf} zOt`K577$H~RZR%i@FWYrx@+{iA>}999l;dowg+$GuLHEfb8TKxJTH7gpgUI|NcU{Y)7qz@blvgnsYf0r*Pf-^Jotv|! zKivlqwI9|RaG*jpO<61L$tw~(I7fOHQ`2LtIyg~Kf9{uZF=^Cr^=tg6_7Tl@LuS4K`rpD&MF`c(>9L6`cv^o-Q)GwJFNyBw;2?<(mq86!s@ zw?V5k_E8qG4hXw03vD3mF1eASK^n3u<)bQb-cg(VQAlq+ZYBmS0a4;ozo5ezt2O~jY`i_>|k`@t4}NM?qOGvZtRVPyPYJO(fT zJnM>ceXHkzoF+R8@9`7#Hi({~#q_FT%~^ca$g|Eq-r2bsPRs->u{*W81%L|bGpe=M zHP0&KO(#HS=nJwu=P~2hS5mOI@0m36VN!HNMvdQ|YA7Eio9Pp;_lCZ`LWO&&^u{=b^R#zOP@*Ljk_9 zefM^w=~Na~2mf@JR8*|NYxc#8gIzM<77sqj{6JTK@w_Cl1h*NklJ|#SD(t&19+GAk z-!rUs$1;ZpdiC5m<0sm5yyMw(>x1XtlDUNSA=wMD5_p)N5BJ&B8gMp{U=4ii82+iaSinaXLWMl8lJqO{OJ2%58Z*&mEXV!eN^rPmR2yi8+gLIzdfO+mJNBjSb?*EbTK zs6@Z>jH=%D;Y1d;kZ;59-VUYg$_c3bIUZaq4%ziY8zA}9P!;53HNN|}%D~@^G{^eU3xjV?K!d7Ntq$+(widOz>9R8d8)4LKIfOcrjaR(S|UnzR3)?-LatbN@=A$t2+ z34ivl0%uIZx}I-M9v8D8Xb`)gif}{S68?+vxbp7R8b^E9XYcT>tO|ZFHQ_~MgKqnO z^r@DoRnmXahDDY%za926T)5LuFBt*N-+LboXV4P;t`KFDFjqC-@#(=L84M-!lzM#N zFL?tsy5?Vx@3;J-_!;+6%Z6nf2Rn>8EZG-#9kR$3`YLwCt!cNTE?;XOrg8{Mkmtw4 zU+c|F&>9AFt<0yR?16t!=E4D+f?hy>?Q(Hadn)=Pz1`eN`vx~Y+Yp^++J@Y@^Hw3^ z{oepW5<)g`YUC<+cjLgG5AkVttHsTP>)_Mh-(Jlwa54HdrO&~f$f2h84orl3HC=X_ zRKrp6__geH(3EZqQ^Pcm4^!wzatHqv}>+&-7L5;Q>-LC`7!4U42M1?1}%u4!+qX zZ|zkoNs-Zy-D9MN2r~Xboz(25KfOjxAB8{Usa?%zn~tPhhjyV(fc?LY>{BGR03aeY z)!nhJ1;4-kI#Ty=Jw`!YnDVWEc}md<7f(mbc#3NhTFhxLC;_Y_+mSTK0NYmR(!63A zp4KUIS|z|xI-gAcn;3!xsMAMa0XHDklJlK*=g{)Bg`8*zJa#U}b2TXAX_uQnSRNqqYeL4x3c>e ziiOb2#qWAwiM|sZOMu?GWWRkcJj4-*MB7Tc)ey6%{XF=BL(i=m!`}1Rb-2Jt%cTKd zb6qJ~+cD1%n9A||6kV!+w<5z}0GOGF*%tmoc^bA{qG#vV7}u-b)`*mWvnT_~H!wW{ zByJ}JnUp=g8Um(ohAwrEMab!C+iRNCR8mZcY_o)0CZw2G^)}7!B_p6#1c@0s5)tR| z9e2qZ|CDuvFW-jL=PzgT=B4UJnW(JlZ-1V9K6^!C_B}9a!_aGgJN>mn-14zjKHp^2 zW^c8^PK`BI8ON+sLI?YzGPq%{vkCs7z^};(&|=(8C14sK3)Mc6*$C@YhRm34JS5ZH zvwgQ)$5sa!MSL*Oa2-szxILZW0Tu6V9rvVd)qM4Vq4(^TrUj zf;`YmdbjC@E|k_@{LtYx%GqZBO4*Al)Prg|*|(Jp=yO{(vw}&*cS`2_%!rA>%&Ym? z9l!gf={la8bQk@NaklAY`Y)j<{92(Q%xQ0PfP+GpijnVsD>HGf`UbR1(cbl><2ngG|UZHnd+%s6dfHg z8g8Da1tP6QYhwaRsSmcH{n;?QRt3pK{{p@^iA6dyR`MR}|)O z^Ps1c34GZq)8i}DIRH<4p@jlJth`Qx|+vmGrRTk@3 zeTiUyn^~woJO*ZjB2}x;T~cJjx0RHX87GHBV8`1vpk;Tgz8od*!2JaW zvR-6A@|c{S++*2PLc-x~KQcB9G?>!>Y)$rmBEQoC7n-^FeXS)4B@qHzTh=PX z?f48s_c|Pn*GYn-Nc>TI^LGv$VG8?P{{4E%Ztxu(!ho#RSij9mqHlMs@@N;>_xP2t z;y}Fn=t3<;p8JxEvjk#Pu9u_Ib|l`%=esQ6Un@pQ_(XtSzA=r|Zwradf4L?fO>w$^ z%+*h!g&+x_thz;7+53Iz_MC(Z z3qg^Ur#@d&EoUz=Fp4P`g7XiJ-+-Ebu!UrM0)c=q=#pKe#R-OJ#<^Y%U$Pcll!SsPQ~Tw3Pihg_^Ds!wVDW7iMdIc zHGG1T!NbS>1{)bs>*O>V#h{QOGu=XBL#_Hlj775`C`%hq7+*e3U{2={*`5?Y* zq*v~DG8^bZ#QOi-@SJjC3*EOoA&BpE6D5yN79xP`aO|B1>8Z-lc&i-OXakH%C9#DtG>u*=8ROLWOe$AmkLqNs{z~t6L|p1N?b^r%U+^tc1DXtUlK3$jK3YNA;uTta zCgHS~NQ#=o?ZTQ0o&(LJB?{iGZIitarGcG`RBIVc+?$G6VJ06nMOY-#+tiYBy_!M3-X>XsPYN#Wwo#VL$ z&SNNnF1ZqMWhfL$0uSl&}X7(ydR{X1a!u?3sCc{V~*S`}7 z*X57qCu6zQ_A_YTss*NhGJRH~gJov3@e+t~>plQl9z^K8o3)2DWmO4O5^IOea7M&C ztpmmR=u!}HDeZq~LufE_T!}8qv2H$JzvRGOwMN*Z-D8Yx9+X4>mGGUd@wjTo?!CDH z7((p)r6g`$1?@MzxH~bW7Po-7cQkO`@ReqJsaZ)LqYFyJWw(`olN1lCN>lHZ?+QqJ zZIcZ;b_3$h<`RZCl2rcMR`^(CzC+XD2|k|nsFPT9Cs9(UwA|WAs0^b)P7Mdli)*Z) zs(lRE_>})-wle6wzX%s+4U&3XWB#n|ZM?nzAaO3US}n=yD4u_9cGmiWiqPm=;!wR5 zjZG68_9q3~KyXff9@3L|mZm)%Xt06orrH0l3EjmmpC8|@eKy2rD8}H3u(D=qtJwEZ z6lL9;&BW=@-PJ{Cdsg%Olg>&a{=wO(*GbMnL05^{i35$!gTQIHZ#gAnZjgV47_-4L z=#1}s0k@iJRY&dFmPWfFd)oHwIENbC{hM1`-j~iARV|c%sZoZsa9W1pqx+j0ruB)r zosKZfoA>?kxnRzY@46Nc`Em_L*3L8)G8MEDkkU0vdonm$ex(|%jEkPtEY`=*^-s=y zs1gF_aqnl_(AhmO*3aO^;ZSSs&(b9S)K9y!WVPKVQqCr*)|AaJz6K)_wU0=2VKR79 z*@4zhDJrCYKV!14vQ~I$RCesboWOp6ae7EKJ4(XJUH zx<{vRTQw<|Ar7LAb~6Cyh^f=@M~;Pp#xuRKj1gCV(#Q;8RSl^~#p$v`06+N&jP~#v zx=KUERm=(lTaW`U>Qasv_g(kdW^BYUYB#P7T%rSR$Z+5%SR_qX{tLx*gpnEO1MSEU z3|8q7Fx9$urUIqYrnVFk#lOQ9n6sc4g}1)c-C=C12cB#{gEgn1$b`~++h2)C`=n`J z(VJ6$G^9qs@zxbe5Ow4`QuY``INa3qeW{9-&3@A8$zL z#pgkPEsbP~46a;_zaq4)<8D7g%-x*wYQ4jModdNS=BIO8#x`&>)ylIQH_wQlO!V84 z8+^_kUH;rBQGj~MdGF&y?#ivwh+Uyt?pDLI-o%aMP85=yDpD^^=b%Viq;)ore`E+j_dVH^!7b3<{nQW7QDY; zr+-giyqN|yyNS2EaX#ebS>bkVw-oeF`1Dy|(K+r?YfVsOcVkYX0&s_W*&JNOA8_`Z zv|rp#HmpO&y>lj=am30;56H4`rRz11oezhH*sH5x&U)&YpbHjQ#-zhO?iX5rD~?Dj z5QMLG?z{f7MFUe#`4qg#7~J_qi;pN?PVHhT3j|hef5@o*=oG{ z6dud$!c$%0$vKo8$~{IelrfKD5+LoRYWOGm#y4TO zaZAseu|D6j5f(%^X?}kUkao1HByxzQhj@PS=N3zbxh$U0OB=${^5?i)mdWEAyE}ugU3mk@ z^S)lb3g_zo9;&(%;Zb-g=U#Vcf%SG>wqGVg{_g3^;cn)EQQQng^T1vd{GK>}*8J#( zuIb z!H36yn=GnBBRYtjdGzraZwFSMnjH)TvF$=nq^uUzXY%sASu%nAF}C;n^H&x0L^WXA z-DF7bbM(T_D7YMdr}NXg*Aa7-1V;)F9rkT|?`x^ z%IL5SynC}>?_n-qNCSIhYjUpA*TeYXcYQf@Q`KBF#tlw?*wcMuFq|wqsPr;d-ZZe` z-gdN5Ku&jE@2!1U+8MH&!tZRendm7KFA^8NGHiw{$)XfIqg}s7`%s7F-=H7hApi`{ zXiejLrW@XGzNcELyQKNgTF7y*7XkA7bIeH}e2bB93$j~O65)`KuU!#mK@tv{R1dL7 z`?&nk3a);CE}Y#W+2>*m@s)Iw4{Jj&!aaB>1U0?+z;m>01}PVOEsjI)_X{)JUy2vg zN`qU`M#8ytvjfOChh{xM%4O&hHt;2!JC@VsSpcxnBVtM2g7e9y3yJVUo@LmEY^eZ0 z8u6)NA~KEw{F}zU&T3_E;fotKOJ7GPQ#LUlXUlVc!#y!Dtgv6TTnDH5%w;baxE=b< zEpM01A;O5~ckTIMhQrIMYga;U^B+=6y__h1xwI4PB{so-b0&XydJJ|P-K*a-=0w6h z!yYeZtcD0voX4&qmC(5FODTV3c%$ScVSJOU$0{0LZw^QC99?#Pe4bIzKhh$ec6Yh{ znXI~hh!l$cpvvR(_*2#6btnAtIjRv2IF-Augs~p&hs1ck9b*MmT7 zY%n(EwlckC1llZxCPcQHqCU^gY3mjfy^B3W2&1ED(cY(@vx(+PtGwdyC}fLBx4z8b z)L{C290PQCU~F!DH5SU47~@$cKCea@TY0a4uU21(bJyCZl@wnn(38|g<<*o3oqnf- zwuArZ0<|r=jF%uitBYPwK|TAoF9SbrOwmt{TP{P^*p09D@(T|)h?-I1A*2~dZmKG< zjVw=t*M6g41!*tnvzaE&t6sDJtU`|YS0ix zr`R75qg04e)--($wy3eEbDkYPWrX@!ET(9%e>Q%sI7x@M&@u zO!AMaPtQ9#&V9xGXKB=5- za-lWk#_&#_#l~}Ol)Q>(1Ye?muKv+FaEFBa+PF^?9Ll!cpD=_FSf2BgY3vvk-Ho5H z4~UpjO5(A^_((k!*^LN_<|71o!osikQH8&1Un2k#ArzN;iG80uGr&`p0>g1XYqfQ)zYP!RtST-x8umbLip1YRaAl%`v=cS=u zi>qokV@^`&;TIdtfYSBz4e#q(_-A7zfeztBI}#U7y{=kFxVHs(f5FM_4)NQoWXAfJ zrNWE_P??yI?#=#CB0>CrHDdve{=PuyPj6`{^MFteJ+9A-D{}YW?hN~LInuIHYF+iE z$svRU!}l5mJ1OnvZ>6gu5{=7IjFl4xyA^epe@)SWuy_R8P!`NK8KCa^gds3fG(I@W zmiPmgHb`BZ>Hc_QqE}$7{7e!SC&q?k$T9}oak>z!*3Bgh!7nI(9-YUHF7z(xY^Y8B z!=qVvYvpC7tjQ@qlOnF|!L7(?NeiUUlihy93g|KI6G00@L9WjYh@)fkxOO~8LWf2Sb^3``L$#isRMf zHN}D56&S65tEzJq+S|QY7k@Wz_wJ%)YMvXrndG|dc4dC~s;VI%Q5)jleDDqV56?zx zdq~}UJH2L&in{e`0Keh>q+^u5{1Uv?Ln~%aI4GmO01{1ZI<|L0h>lSfM^3LTGj1y` ze`x;0MAT`+^_V0*_Z0Ja|dJt~*u5 zs6#iY^IL>ooB@A}#wR+;jR&M|jFD>Ht(WC~RT#!m+_n^Fmcb>OroP`~;li!d6|EZ|w>K*2-><;EC_7dmB@ zYob)h1j$(rXLlHRb=#Dgq`HwDjDgXBkubr3MK2ZF(@m(UwPrskfs1;U%!m7kQAGJk zz9-+l4z$#3WH$YCIYn>tzDFh+l$vOpQFsdHdY^8aE;n$9ogrv$r~=zBzYO8eRl%X% zDy5oNw6kGvjWki5|9#ZgUEx-oxGT@=ZUeGmQ1(i}hl>p=?v}dykv3`(<+IpJ=2|a* z16Zz@WNfch@?;ge^41u>>5vu^WH<;LZhD@%p|P`C$AmVsK6PT0wqql8-I&4A+UEzWuNV1Q&WDM_Tf^;`$wCs ziygZVdfOX;%Y*F@C!-w_7gyqNUu-W$x(%>Z8wne*cU;u=o_7m6fgraW?bkGa+`_={ z9(DES9}`~o&Qp1yxcFiCoD6j2Pt8<=roTij4h)wS%X1ghb{6!3j_yw6B1h6rth4Ud zhthfG$s)whGKSX=LbxL7qPdIc8BRBNF@?bfqf&`?i>ALH9)DkiWRg;OplRdaL(d^= zx*W8g-reO`txB?A7e4HvDG|eer@I4%FFnF)Wv4sQ$@$#eO+eqE_47ABNvGnmodvUlCD8Xk#b31G@l7(}66*+LpO?_o1H6#dhf>ijMz(?a$nV(lsfE ztXg#0aJpinR=2J7_Qo6PW!jB38cDzOa@luiNS25GYR7f;<7t3eM;Y3V=J3;C!`F^O z{LD(78{N>~_@*#(ZQ1cL_NKhUm`$Iq)B8S;kQ-Tcy7O}y3OVEUd7m`?{kpz6WMu36 zMP_2#lnHX*48lRNOwA;JvT)b=)rQ_*_bwyU{>QwMp(lPTQ$O*O z_|&ZrLOK*Ku)a8dZdtWO|M>%j1{Z|k^Qp}Z`Ag7R>9q`5uu6ndtAoD(`b8n2Un0~u zAUAF(La=IJ%~x@l_W$UlttRO0$yEzo7MCXEjY&%Vi0cu+~86CM)!=4P9{A zT>{X-D9a~x6`oqi{VP_J$#wb7dWFK#S<`5W_=o1RuzeMO4^n?a*`%QWD$okerzm2z zgA@1NQHvSRJDM7EdyRAa;)Z+0DHQflf;+s%ys{sqAi>Y*jl@{?mScjR(TRTbDl3o? zM%@icfc}J?=rE$UKlNI4=iW(g5mjn>Dm3jEw%KLBX_8xY+78pWm`{mM)W4Z_!=_$| z4Q|XY&KtLXZ*bJZ=9V34pNO;#JF^c^AS@wm+^|Cl7i4A&!4uLy5n42G!N;W1X` z7N`}U&^S|nT}){Blu>$SN34*rqnZPGWJvHk#erehzP{F{y18A6P|J}AlsM)UU62b0 z!Z_XTzf-;n9dWUztyxNaxg+ZLqczmbHIgVBm zU_5F>Jdvoc^i65!fGbJ$O+Re>8CPo~uQiGzyhe7bK`H?|+J>5QC1cq$ByOqat%Q2E zbI$p@6mkn(|7yuG3_^dgcoJkYo#s-HrEdF92I9 zHQ5z^s91P2XOo4V%oBfNVQ57R46LDG&AugPEf{=n`3gK&3aO~J%O+`GzPgSqevVo} zwCj=nju&>;9pMaKjNj^Qd}%Jl=G*Mf0)G$x(s&PGn1E16o@Dcrbz@FPu_M#lWOepB1FfFvw!y?~;HMkX6fYUpF zMx7jUBT2tzHE^BFgDu#6b#|x(pI_B`-cL3s*pgbp^5@E#kJtQC7xYAQTg*-M3q(V7 z(7)^uv=MF13;!gS$A_gN+Uc0KHnm$&(qa`82^TcVq#pP#{&d$p3DDwvKe2RB_yd(n z;Ia$9$|fZ5;DdWCAlxRFH|MYH5ZaZ0)qAaq3Kx?Om-cO|NhrW9ubAw%hEw z<#U=AV2p*~Ok^Q!35WakE^aO_+K^D`^E>r9M7G{muupdG%ILH=_lSVMfD`NO|r9GY8MXXk7#!cVjYwCFn&8OynU2Mod< zaV}CzhWpNe2koHGRr^8B`0k_RUzg7iV0* zMUR>XyCZyh6ms~C$<-NOKsGs|TDI|GJ=ix6v4h}APw}AeDKXlx%{o3^p=vFc)_-13 zY3C07&zQZg9)pr7P`B$x^~(iX*Gew=zio$l7+EPaG9|(ef4M`)HNwPyQ2VWtWDS%1 zSDvc6)&?VW)oC4-qL?p>YldTO+PyOdg$e^A?xOF|2Gv1DL+{1s1!pX)mBZo}8v7vV$Nlm^Ua z(!9H#S4}Sz1Zxf{!!Uxee7mwfYc}*iH8?<0jJPZ=o}08(25IB&{F_R+48M8x{8=nv zuS$n=&3K}Jz)zTg^D=Ay?V@^w9fg|r;w!PWu5yGQFbrz2!&}^c5|BY%(r9>SVwm;N zjZKZ=%!)v7PwL!MecWg%3_PhUs{@yFsvM2H+O2+&KWkV89is!^?` zP$*p=J;bEGay2r4=8+bl);z!ObdOfD7(K7XK=;Zog5B7X@~etZdN^+qz7iKE60wH! zx$V{&lb+Rav1y5|FMd^*`%3h5+V`hs1-Ya?GP$$iT$ntunyYFAQ(LB>u$H^p-kTgqh4O=&moF5t>%PsCod~kf3-;KP z#20Z?_IpS1wLjmZaWk0#d*@<*qopE$Hhywzq#P#&S^L{OcVmTut;=?^&Sz@8eAisZ z9s4({$e+UnYG=Z;sn2syhs--_l=jWMgV3$UW=(T{F_1uq+7JIZ#_;hzx$a>b)^)^< zl0T7Pbd8hbu=a=jZtN3>)3y%8VQ?i~ZjJiR#_A)3_gK2!U%`Lupg~9`UE)Wbi2ZCb zxGHxHk*n?FpU+IG>s4f(s#1?Cg2UxlnM zIgJj1ujKjAL2Q@Bj~(%wzHNSAfCiN(XyWQdP1OMXYJ6m$D+==KD;f&|$I#*7(!Edu z{241#P8dDh(DQc`>KX$n?smMo;l^cUHUodDJ zJCgt14{-kD^%(Jn*K7*9D1W~NV7vh^qD`Z@W3D;p?ejwEUvc4y-=`O@v7|8G zc#E}O+yVoZQl zRB9HiDq5GRQ)(gfW}poW191o!!jRla?2Ck6lpJg6JE zKHYenhjxWh#Z6@)A#Bx=>8a>{?u_h}v$WGIOs0|)daaPnkQsvz@;XT2_DYI& z{gpbmaOFnlIWOkp-+&K3lv{G53qHG9*zkR~=SWteagPyg>DNBCn_>__f2rER%ey`% zUKacjEH|U~{0XIqLBUNXTJ9TeFa?!{NP7fJc4GRnXtN-Q4f~Y(reo>DvgG!E`P__u zl87fWB2dMT5SfOsUcI1=k$iRn#4^qs4;lKDxe9)d zi_gTCXxvD+9qx`F?EVd@lk_{xEmZ7C_wt#O>S@4yR085WthElzGU(lIK11I{lVT<5 zvkE>>hkjE(*3@J8t2`&ONBG@;hm~ZJYeqoLG2If#idf=iF!fdc@h6CuC=0ea!#~%S zx5p7O<=Zkza}&NcGxh=)fqJTF^SK*g?!+>Def$1sG-ODqth5^Gl93XN2~!H6P@y_t zV8?5Pl8T^4DOZSs^rT2qCW=xyC}*e;5vhbtKl*07a2AOuWh}P!3H*#H`a7i@y<)rN8%cwaHu7FKqC{YLp_1^k!SVBA`I#w=Y22D; z97mRU|IPlZ1(rqrG!l`^kN$T?{~8yPX0c6fIaUZr zSUY%&1~Vo(&7#5+`T9y%ad`zVQFU7;qVcy0@kQwO%=ujzHwVUlH=l(k5cpyr8eY=X z?{9(b)?7~T_noRcZXb3lJL^R<rI~`ttqnowugqoJ}hcPG{ulX%M;`^7g{12&0&iD)iKhqfh zGV#QiJrIT-K5B6nfcax7X4QGLu9ikV?-m=E&h7`6rIk>BBZncWEhRzIwZyo>++=rM zN@VE)G4^Q@Q$T+mApv-{z{DxZ>^mwhQuZ)OTJ4{#nosWA}Myi$n^;%6!cg(J(M6 zFz!T__(HJ`Z=EM3>k?>t_FyqZpigmgQ$Ms=-B}DPB2`5-bHs;S6%n(onO$HZg^g~j z+2UA$k-)!yPIvPvGBj}b>^v05Sa6A@l^!RZR)9|sE~Q|69vox}`o_9}At?n5a*faO zaj;X(3kglvjXc{`ay9k7QBpJ<+z<)WC1ncJNeOkoB~aNHeK7e+B_T55-X8#fl5~}u+Y^aR*OX3|sLsD?wPY&OJ_-+^oG2=0R9cxdI*1qO&f4al(fdR6E&U)F; zJ~W>)0RaO-c{*BFZR_jKry+vtJ^HX=(UW8d>O@sK9!)WHOsDG6^f2KPjE7G|a&;!K z5CND+mw5#J&yKN0xwh0HoqZ4yaf<_HI9vu#Y-my_(;O$2q{H96Z#^?FFzUMLNw=F5 z{c%r!p^gAu@HcosOQnQTg1g6*&?&~Vp$>3Y+ocesW%aNlS(Uw&+_L@A-jtYIM^`6N zC0Lf=W!@g9@YJiqCyVy^0Ob~)E|v=%6~2xtqF|wai%4f7OFZ}%D&P;#`5a}E%oWn+ z%pk!~Z07FaI%Z6Vt%EhEWOkeRaI`Bmb@Q`-`?dy+_>|lUbH%qUYcl_inBS22$(wjW z7I2-C|JB%sVnAq0ef5INwk$u$Hc2FfY9Ra8 zr_gAz~||U>|uMzhl`KId(uN0Kd=--!frvm^iDyN-Z9 zu7=<89PmMH`ZTa$=hh&;8q46P?2~Yv4GMV#C@~U0t;|uz99(a9jRk4Ck^fKH0Lh z|4}tM0JSeUeaE-wm4`F>yQSYNe~L^s-%M*)OOu>x==++@&2i_H%}%`O6$*c?yJngM zT+bX-v5|HI25^#@2uB2cLc>-5c+b%x{!0MsK12$5DP*OJJFRSg(E&nBGWB{AmUL^ z?9jUB9}dW}W#-m_4>a-1twX8WyRg*mJYa&EP`cP4IE?)IUcBK~milXdV1)LYj|<20 z`_O}B4tGG1!Xl&1OiG)f*UNY_{9oFxfc1)y>Ubj=lVc6$i$87IQzBFKA&NTQ9PWsL z-_kkdH`H5LUrC*}_z`iM#F!epRoYqP9v>VMuIxENNEa|bL^DJv>M+Vv_l5^^EuAg0 zv5E=syQ$JoKGi*iUJ*=xPN8?%U%_1b?pyhD{)=m2(U&6?zz5EMzC7v1?MfP&se3B9 zHstv&@hFvsKaRd6FRv3vIQuxsev%|~mg6;IF^MHwR-;W-){RohdpoBqXD^sw)lxo3 z#-2REav~md&1kUo?5lCcGyX^g6dO?;y!KY}T^9hwSYa+j_O#1e&$;$3f@mu`@uxdNUc=CU2!~`pm`K&B;Q%BdUL_V$ zKNaFg%TIc}QjDi7rNZ=-ZGs1`=l!mrsUPYr5LNV41V>~cJAIHTK`g>^?9fOnAe}ai zZS^!DHKH@LLb#@X9fs(fF@D)YZcl^=tfVqY|maexI4rZ$45asvmmP{S7sO?7#_Wl~BGH zq|y*SF@rb*XB`{dHlh43Z|UiENACJG9ai6nA@Y!6n%W0{gXjUc3991iPKbt$>M=MN%U4z9o=LiU1o8Z;<|CB-QMo)rU&Ual7x1kV7x(?^mZsJw_6ip~ zsHRlw8W>zoQp)jKa|3po05(P3jx$j7Xfsk;F+K@^@}3|12kQ`}ceW(e{HK!m2srspXT`a{q=m$DbKV*i}~(Qaa*@n zh5{8Ov?We1HrVb?;&}oDIh&#})w9-L$T2Xub~WvPk7TDTnhg9>e-+@KG~T;l7BWX_r-_i=(e)NQn-Rre*bG!-&*~$--k;Ea zk@}FL{POE)lt=*?nu&FzS2(k@sBzX%%Kle8`}HuNuLQXTtnE7D2qK_)CSK;-By87a zx0E>sorTI8kxzXhspnhnU`{w2Db%{G5x=PC6xh{9n-*N9bXZ{;&~M*~rr$IHLR0K+ z@SCBdU2MPDy~FiHIR2z1kq!^{Ndq>2?a!DT2h}Bqt+}Aw&;jaIsjdGmMSa+2*fAEf zJb0D_P`u{eQ4S>Dj*C~etfo$&o=D-Ve!5~|pBjX|abUt_QiKcL2OP^3jnl6c6;v~udOF*>0^3rXz{Bz;PvZ@#Zrbyjq zf6YM`3IhOmT2f;ARm%9g+~kQd-JSA0F~S1M{Ch{-)X2Bl6+92QF_0q19<#x=XFutz z7nf&Y!&;-kM>Zy!tO|CFCTsbR<-e=!$Mld|9C-3dMu$Ui;e>*}DGcStSEIA-QZYdj&Xd>y*i?2TudSP1MVO(hVyrbQHjwKLGc6@^oFi^fXf!KBO;jOrTt?qD ztN*^})W#kGal(y>6{6ddLKD74Y3-l?h1Dj_2y=tWDSI~)s@69l&Vjy??$nt0A+&hoUao4A z0l@-|hi7eM92n`+zm>?K7~tw=43$Wo2R`>s;Y4`kPEASy8j42ctt7N<|3FvUC96`w z_jSCIh@~qhA<e;@?PH>n~;BD`O$>NC#}%D(0-K2Wj)J~_$G`|{n3Ps0@m z^6){@U=DQ}=8QzUnW%5)i}z0d%x;wW2c?mC5U4ERfokkSH<>-n-9c8C^j%_#Mg;3T z>Cv;xJ~!t!gT_0O{AO=E7zao}yvj*Cy%H%_&3dJXV_|=yth@>ke;?kyfQ`lJLX-E> zU0gX81l!jTycQM_bbjy5p;Jq?qXB*9SDbNo)1f+Rl;rN}6&1Tv@8MTY_Ut+Azyw+p zTctZuhPty_YRrB1xAFIEuIIyU=`voHsyH2Mev{Lhny^@e<4lc)_=#^@sp^BNiLywq zM3NZ#>t>Jh202*ve;n!$yQyqIwG>eCm_1Lv4W@O+ln zMdB2j8A+|hO721gO8*sEn9EcKxnsk__y`M@=8Ke5AWqjJt&+x?~z#$`3nw__j*SeBVE+5H@g)|6el;THf(H)qAWT&eJfFkMAIkC%0lrCwW$v|%lw!8x3TI z>l=J_GwS1#MzJrf!+W(q*6+n^YSSt3yh(MmK^Yd~K%nfz9CYucxLQ@iaFL;(>L<&C zvJG*Ua6^bk@e}XwdDv~pRtpXlJ)Q1M_`w=?Ba=chyYU-s<^zAI>vE-j%Fa!Tz0n&=1|?7h$2q!dE;!x@-P7|6gs!Sl zowCgw+vE5RwnUr}zIBe6vB(OVm9E0I`l)c>sl`w}Rs_mnr~CKbmKmBa#VJZ9SPs5R zg^q?dGVs&Gc>4q=_Mpi$np_tjs3o-K42xaFf4{(ZS(f)hUj8Iz4oW9Be~#0$CK?Tx z=+_Wi!G&d4C{h9qo)_`uA~-D>4@8@Fqc_x8Cq#358B+9V4bH0s+mM$nyepMkk>cmH zaIdeRu;7P`0)ppw>T?@V*Uly*cFnZgyX7JCh5-Zf#ba0cV;*+TYj$xQiE8dU_tACn zf9nOjTWh&E=@oL)P!{~vNiptxg~i=livEPOB<5N{@8CXi$^NzpfgE;r?#;;v`ilhC zJs>rh^J~H2q`;^c3`4U7Y8h{Az3vV zAk~9dri%$DUn)wlFi?WYP9Ka_;_IU|f1{b8ZkZZCBXZv3s^BK%4C8ycnjkW@?r!+^ z`NRr4pnhG>3G?1y`vVwBddJQ z5~W;%htGamXJ6ll1SLmD299bGe^9@_eAXNiRS-bk%*%}-IcXgdxUE0v1q);g zkh$D+HhO%T-Y9rf*F}7twdh=7JK~Ipi^m<)fm^zi#)EW|~ zTE}+PcNHU$?$$uUTLuQ|sMliZ%QWEf09Z5k&6#@QJhxtkaAsSz_&ZsG73zoW>t0hf zs65@@aQjV;-NCYaeaBFc!S$eRvBOnO{#(lvKQgS}fh#kW8(Du74PH`Yf259rz3^nj zg7g)HD>eLWFgV#np)Prc^%*Q8@v z1mpW30p|aI`+o~C(=7O-e{HZrqu>)KRV@Svg2{<5p$h~XQzbQ%JzE6c@8L?lXd z7JJhcd+9dcOkOK2G9%ncD(*gfXX-LMZofKepSWRt7uXOZM>FszqL>U{kqheQjxI@? zc!5LHXFqT-0{X(p=ZAx4SS&rT@CBfrj)Z>Ne|Gj}dIO5@olM_a z#|+O@Hu*dV8j*K9+orr-l7`2gJ9o*M5i$MBJ%}}ls^?{;SN;u$bZO2gE*#3uPg%X& zaqxj;`RerQu`LCpEnzGME)YkXe6zk7Nb~PJp8V9cHP+wr);=f$W7MUhW~*T98&#Lf zTFe?kH=zajZco|$mJgJ z+upCdn7>(b?>W{pv4udSs-+)dG3N?Q)$ z0e|PE`DrL%2y{G^DuaAiCAP#6wxf|jJimsPmgCbO@<`jUk-H7K?l=a^!bcvA$06R4 zDkxNIv*$&z3Ga>hqp4FC#(=zJ`<*xQFR;7L41x`2==FW8grf_Jz|OAmq*j@3(|AuT zyKm9|BZO%gH_9OV*VhCdlPf}6NNyCvlz&c@(g(d!+GKx!wFLm;k5i#GkMMT)55>3z zC`yYiu4r50<0MgnGPv*qkpyUby@gZ8gg!Jhl}Rp8CQqiHDk(0kwG*9Jjvc{<`x{CMs(k1&X9KqJ3hPjHO^B!0b{;PP~liS@6ANuYZ#v zUbFKmZ>&n7xa@0G7)7aG?I7p<2AMOt&H?Vt(Tktb>Heo=GwMrYV-X7^UA;9x>9#K+ zvew9K1noi&zvN4&|Lm?ue0ID|`pz6^qTY1|%5c_(YdNKpEmi34JWdj9M0zVXIGW}rg)aAf7L?nXFgStE-D z3vv0WSRP!vL=f*!o7oC61PPaj7RZvuI^|fpw`)w47?@XZ+D7~V_h3==fPY-bcx>J) zkC|7(4nCa;<`P)ZnQE9GMcbgvlf9lM1BQviFQ0paQ#LneSABPS#5K; zp%39&36c=UBj0JWWU}nF30XsvR6mT&~+Y;#jcC6eI2!G1jO6lpeHG8}! zrPFVkG+F8s_*+J6F@<-Kj+>|I7tfd)~ybj0p%c40T1m|ZN99dld zwd)aI%#U*MC3wpXFCXj)1{Pc2IRv&(%k6!&8_A}yGg2%{nt_~!eAdl@!uXGOhUcjf zg!?YFVjZUKaLr2$oqz9hJc7`s)>SQ5RF`Oxr`9{Eg7?uZ1;`Qrbhmc)ue}E!^VAa` zFW*j7hOZw3Kp23#-JI4r@u@~%SKjxT0amGX(EDg+Bz~TF##>Y=(Lv>UO7ewcwFGT9 zU6C}1+|VkO03sVhOWVDT>?m5bHlwAb;*hZUpwm!NTKYN90e>E`y&3ekS=>d5@lySb zy376t2~Yb&?z7xrF2nvvRRxSr$-^u4^Amv;?j7#z{78DcO1x;?ifp) z<dMwchxfW;sW3fNT$=7U6 zuP_|@7A&^D9l?;r4?#t8nM*y)^9*g{b!eubs>He zmA7r#L3Qy>m3a+r^YT33dOSJ!=es1Cloxt}8|rAD3!8q;mEKK!umZX7hNrhiKk#cY z8VIkNLD9EF+`H_JY_*OIWg1o;@Qf7e=14U3&RN>9qf4f|8KZpDKp0HKMI72JW%SVx z>zC{mD}SnBqAhD<4!ivB1F6KPEG#s(bfNa6YVi8pnY>R8k;g*velVLsiCEPm ze6yL0#Ag-kdr#RtIHmnS(~q&P3^1BWLI?nrsXzf#dWN?`>E+!*yeSV4 z+!kCx$4ALe7?ONPfynjnt zn{2z`^VX6)IvK_Ut$oDjfoiZ@f<#WUQXPWl-G1?jtjMg6JRUgbX8fU*+LDO(P(&6VN2`jLL@36b`=MMq*y&tX`X`)Ye zhn3JeVY;w6IWhORHUiS4oVN-rJ^5|pnyyrKGX$bt)rSjoK(VbLB22^m>N;aN+Nz;L zgFM;+&%-aGsXL|>$pI&umcTtJSjJbhZz@%SATtC%quojFbd7S)U59hQs(J6TX;((;_Stk0RuHDBlZC1mFF_)@7sY_aED@@J(z5 z((Y$AWsdGrqM~#;(Gv&|8^QF60^)*~R^5bGPE^9Pw^D5v?7{W!KqZnr{NUU78+(dz zft_{X&~;zd-<>L!zsgb3$A8Zz1gr3`k0P=pe7e!WXx$#QTqic*F5a}_+I$14m@+~; z2B;m9#x-j`c-|%g0^Uk9#90=CUbrW&1`3YNEpsDARuTH!EoIRGpZ%|_xVJ|J;aE*I zt(nnW^{C@F;oB?QDeZ1iI#?kzlSh?HjCifvj>p|rZ+Du4jfpA7`G2=kBBd`KoYNeU z*S(MRP$6zo#ICrQg1bn0kzNldN4v{gpNs9tuVSK{-{E^9}^_drissl4?!(V@S}UsN~&+xG_+nH*X1 z`qro}rOSaeqz+wY7JsDQBI1>zLF@A|y+`ru%TXeommVc8E%@690sb#DTHBr4$sKu_ z>93kj>W*nrVB`-gN}57~ZtIoC{!as7Y!_)6XQF}q!(Sq?yMzX(BeSrnY1wC*+gRjm zi=nxBkRr_)gkbHppX8<@#60NVq?}&KZL^8d=F}}}X320lwSNjk?vI~?9;LWYd?DUv z%l4S@ABcdQ(k=D!HU0LhUTYDZ+3_KsOKo-Dh&i6N`PhgW=per~h+0L?jt#vccm@f^ z+T~kLYlo)}-y8;SxcP&v ztYfQHA^VQ0}9IV-+>=06p4D4$TtZL>Hd?9i_*W-Q3V4B&+>6k>_8DPwyC-+5!3qezFB*iNChw1 z0l`XLdsG+EMyh)RsS`MXfns~&L0-J9H`)z!cQa_#pAGCcR1SWs5~5MTx@SZCHQo2= zkYGzfW3tBP^Y&+SM+b~84&v{sc1Bokf;v1eupOr`?`?E2uj$hwyUBw?UEN0S2aNLX z-b@g_w^=KCdZo+*2UmqvS9P#N#KC%h{Ts07p4s*HQBr~1Tg8{vSs(B?r)}+%7$moS z#Qol42&jx=s>kbyxId&%qToeve?RxXjg3*D1tXwcO}{4nJ}r(cMnd=P5zo=ESZ+EL5n!&v{d5DsRzY#c3 z^9`I+H;kd!7JiMGn{(k1Qqr^g=YI$&ma1||z;FLHYA@B|$pKm~of{bQj^pX^Pb6%dri#?ml0S{$3QE>{4^Zpp#*zb;DGnfnrry2sJJ=i~-^ z7lcW7I}=f^!8BiX!TCQHvd=!ufdfP1uqfZM3V8LH-SLWIRnD*wF)Mr8PoCX|hYQiU zTNBf@Ypzms8nhw0ow< z(h_AcpF+b-`t5XdGohec#idv`7z!9Fj05zBLL_ssz+yL?hj`W}<^EKuuP0*EpjK$i zcRz?K&{hL@M@C6Ky}9AiPa`Dii)@954b0}At??8WGJj&!>5UN!rpk*NB2E!xJ5=sX zr$_Pdpa`ZM`zqU=K`A0uhorqxP0V-zlc`P}=iwRkFPEr{j^{=~-YKcg)sY^L=aIeIyRYtgO9kJt!a65A`=0W*Nj?8tyX}Ehx zo{Tl2BC@xAvG{FYj>tsx$IkR7kvpWX>N;9uM^{gLJAV%-&>vp2FvooD_kD818KJ-! zmU+6VMm%ybnUy zS!>bcHh+XlU+-1f?@pH`g=;QJ>^1g}@GgV6UnL|wjDC$uV1VXgyH313--P8Z4q?PJ zVQ)jMt!FNakK7b#-UzxH3GbpnO<26(TaSgSXOID%O^Xm_%_%Ol#x=4N-du246vM+AJ) zD1YpARgR;}Wa}Yb^MT5-$!YL%g7^{LHZve)v}>23m=q~OkxVF=oD%%d>qlszbJ6$b zl^kF$vIg7q-MYz9Ux+@R*?i3{KNC*;UIbyH-ywi0!-*pu~)ND-hZ z6`&1eqL%R6kIr4ZYeGU&xy(g|MGq3Ignu>nFyb&f4m|EH#MtwkS1vkM;+2Xq`93JB z%P+g%@sx~=mM-(rB*0m)H?7^q__0B`v|<+2sX)xFdL}1qtwYZCX*5P%k*h0yN9o~U zgPom2?qTQYPIuuD+D(-5e~f-gXWGJ6M=38}|H1~6DNx(&^K0Sa@ z94{0cETg${M@hU7CdQH3!UdVBF4)>$g0R+{pIKwY5FmOo^{`jH zP;nKPf;NRJ@Ij)*;g65Hz0PqXHDIX-IkQm6JNzwYbM}uGjgb)>*0l^At7tYaPqC7> zC5?1gis_lx@(+jA}O8e3nRQpVMY6_?0Tb zs==+E^DO9U|1yooMw2EXjTc$>P4l3?f!Uq*E| zkC&%P!t(I2L+to5bO{-@*PDg;M5GxIY}fBS9bY2e(_`e_1J&#^c%D+I-CQuyj(J)I3m)DRYTmWF&Ja{kT^D1eSdBXZna=qUn<&Y zBPE37K)r6el>VG|lihS@-ycs_n}E?=lGQ&y&!%mf5YZ<-vcOYTjm-NA<;_N`33V5I z25QKXF{j4ttr2;@bWP1ACap-LTXW_RAou?RAqxQIoZ`iX(VmgV$HMCcRIN9*s)C~qbCot zcax-yD;B>nFxRNY1z+{X$eVsY!~aHw40EU8Eza@w=|l(9)_;oWMz@U_R^w-oIarDq zef;>2Ltz0F=ed~J6bE^KCbfKIGYJiRe#()49}mxP$1udkU??CC%R zrE}FL-rd0#oJZ)vYJzdIo~mZTc*hjE(KRY!IT9SFA)*70Q_;WFdoC^_1xfye4tFXf zI?vAdsz{S}Uw=7-*f@8*sle6#x~O>JTYm*v>DN~qAV^>()*w!eJi3)VIYqUQuKi3G z8wMD2eP~oVk%G>pSZ^g<1|}!;6m0@HAui`Z(LPs>HM+XRI_e@fxYR~4CY3#Y2Ms5- z3+3BR<)`+j@x;EE;sgoHQzZD)zqL`7n2-;fUM)FuW`B5lp0KaB0F8XkTaV8Wzh@Pb&iB zrl7bht=wJMu;w(T)UdL(sCB~XUzQJ{SvEDYLwaG^~0Z{8i#8C zYYJ)Ia(|I=`0nvK@_F*WT#5dsO*bms;!F{g8)B~K%^?#)2ZS{_$H%O2D#**X|C!nb z&HEUKbpbwqfa|~!O}hNxCtqaeu?sdFN4YV7W{ za-^4_fLI>giXBuW``-40E=OiCy5U;#QC5W9_^ysJjZ$PGSTbHzTM{BI{lO_WCJ0=$ zVt=ukJ2}In6CmLl#GV;ch1D6_z{nZffQ4_~>pSTz)GrZHnvJ*nSs)#AZO-S_H!(0< zU;3I`2Tb9yDs+;#LI&^-I=*v(>k`4Q)oP9|JMw7o%UK_u=zt?8kv!ixg8O)bmyVR2 za_i>@U2TCdcguv+H(bcl)=%D?9?9*eXMe;HvcBHJ?Wf_~o-RQJ#jmgHN+RNNJtT6@ z)^n6+6&69fZqieVPSuQ?HxSEeEUz5Yb#k;$2g)2a2%Oyu0j(_2`slmQtvys_!`kS- zjppb<7X?ZckzyDMEEyiS8!W&$%f(R?BDr`hDi@#4YP^Y|)H9>EB?OPJ7PW8#kbm56 zo>r1$FbLIbJ&6A}zrUOEK_UX z9!mC%55^F;-e7l`KEAS!x9HZ2H`y2p`MpKH?!4CA{~(T2d#6kS!c(ZfAJ?y(v&ufH zgO2q|7bn!X15tt9r5%1e<|*~P7Jp-w68j89MC9K4Fr9LCCyA8HK#{DPFM4m8{{cJ;~!fT5ZmU+W{$;bQAPr{I zvqQYDnqBF3jw9mHFtqs-CkEekwCd1n)N{_cMu5{A)!tX@D_05%^Z-HOrHl9#ma3TW ztV~Mm`GlvZU@^Ko{1tKHMt^32ippW<8XC>ddNF<^0`xvHIWzaO29Ol9K!mbKrQbdD zy-#w6u->R$HhH=+{)K>xJU80|5s%Kc2?E@L~rDQ)0=x41Y-DMJ;%x08UoDbMD2;rj5pd{PMflX{)Jkqln%%iUE? za5%0kQK^=%9;xm$@hGK++4Odfn;9c7=d|F?+hB&Up~ww0s@17~jy;3Y6cF?aiDP-t zDu<#g`l~8Fym%6E1q&`mSD{|&+o6YWfn2J}+yiq1Q>31&L4OyEaJJE{Hw6lH%yaPY zQvjcZ?ra;bLd*KBVbwcI7Cdlyli!LY-CuDM`1lvNmpfrclTs5jI(ACiYeF-R;oa8l zn2)3Tre`jTb34F#_cWQ=qb$w4(8|O|AF*E~@(X6QjxX_Nwa$kQHCdWiMTGG)PLu5X zCwA|45Y-ZJC4ctC;L5w*>G^`62)4LeD^~%;VRuwBf`gn*+31|~!TAqq%!dtQa{>L+ z>R@|rE1b_5EhdF!xOe=Bf3(N-Tta<}Rw3VCbNI1-PjH0wa?!-vB7Uohb%^kDOl%Unp(evby0s0zB^RYe`fX=UKF=vuGc}#rvEK9b3aq|o@g%i5K zNa01U&sm>Hj(&iISsWcAQ)dZhsQ!AUoW})8-FLFHY}XR|0m}sY3`F9d(p0bQjpV>I z%itT-rho8HN(~g0C7R&2{iv45L+Bn;RBWq-*E_4~iZ#4YkfZGFq^Q_BIePItPwPsa zF0Y;aju5PaCUB~6PG;~&(|LQOhbLSdC;h!QAt0?JP$?)fhg=Pj<(vqiLu8Z1I#(^) zdts*07paTo~u5w`O%&%Oky0btz zqi{jw`RV&D*Z;-Br#!FQo3XP6EprnYi9u&jMt5?O_Lr9&fB) z`@ZUZ2S}WW2*k5fLBJ~ucTHb0;!%yb%YU~Ias8eA*DQTkZY<4FN5yHj|SMt+rn=(uP{c0sQuZ+ec0_3!cf;Tj5+AuzD|(?hS*qh}%JGe1Gi` zJiSyJ0>VsA&^+InoFyt)SK$n-F#?(Rw_2HsbazR0cSaP0Z7kp8GI6CZhlLy|(e2kU z*IlnkXJX+;TNvA?>04A4?Nv)XH<6s^pzSPV(`g2Ky1v&I$*DFVKHm8uwU{Gam^(8DtUweI^CpB6zbqHmb<<@4WfX=`vVq~qJ1e@Io&=zXu9)@ z;b~YmGJH?145|@fU=D+kdOJ4^48C z;b)VF7+fdDqON@h{AdH{BF|fl;L@&OBR2;w*GuI1=#V%ZmK!PJeZs#q6z57}7!4hp z+Qa;75BQyMP%kvpFDP9oMMnD{no6tO_0Y>Ge3A}%U!5)V0p`XJ#6yTYI9u#O7gzMT zUJb*>pfs~K?7XFT-mvlJbAL%H9Q8U_pOvmpNJK>jlHOsYqdv&TGT&f|d?nqz1q9}3 z=uQ<;@)wzWg-_wv?}_YiA6>kX8)J zS257TVnZ$)Hb%XITtxM=>I@z!R-f%D@~L*x_6`G)TL8;3ozl2-JF z*bL-A$1etiDo#1Lp??L)F8-&*)qwn1g1~*en7iy*m2)@jP{IeR-)d~Z1KroXE6HJZ zKt<4-E8O5%bvc_=Eei3L2+oM_2K;6&fic2tVE-=mfkT;OIW!scp7!)$Q?wSMq7u)z zN(L(jh19^97Jo)sUbq}##iIRL*x3&~57PNsR|*sb-pmX27{d9UEbFgUdJwESI21MJ z@at#t3!qf_`n!crVGsvM7ZsaNR7;W2QL11DbXLqW>~G%E3y-|VSHY?O9&sd9)EQE% z3&A71#oa_YskKhS)#T&bQ8?rjrP{{SaAbzxW@qpl-hZAWg}%x-uyk5zx$;SL!40|H zJ(L{jLeN&5%DPxTRjRYr-PFJ2>LaDBSf=_VTWa45YM?VoC~sGB>3n-DI2vwKY)Q>D z0&2}Lin{V6F*$GHwd`JzO<^)erZ#M^?@HWH%gT#euP+2Mn)6DJcxVM7w+Wy+%>^>d z&Vhy+Tz^gDDReL<#^$-+tb4oC{T$iPTb?fn-mb1(kGEUkgeBWg3E{fYlXQr#^=Kh! z`JmRk@xP2iF3GcqZy1LO(`v;Fz^iG^7m2|RN#>&4I25)ueP29jA2MBYeN&@AE5rP=8onFY@PKPcmWO`X)`R3V0l|NLC)) z(96TiWM~!C;*i20+t_;h^1RTc3|6D17TpZd=xD|;&ypu#LQJRftB zk$<7LW+IKY3Hb!{g%*vu(RGlErzN!T?Zc|hdu1B*(NZ_RhEnj~wTe<_k5@y7eYUw9 zym5I$jdEsdqoHAsx2i4Kk#OkfH1Si=-Il;#oUYXwP-#e$^I=F+!LKr|iwPzwxpJEC~6!#31cyE#n=o$Gxewzac<)@|U-8tpL z>vhUi`}V9)%o`z@ap{!wiR75D zt2c|Opd~dHJ0XIDt1vm8Q|H1lKSSeZa2zWqca#kj zY2+pEZPuJa!+KR&nkidozlmFFPKVg62Dn{xO=p8J(})^(B&cY@(u_=7uHL96*=hpE zWpC0(C30^-H8<dnGo#1eI*@YdG9UeR{AF5IM;QeXP|t_Rx_Sk?wI&x$s> z^o54QpDaw3K)+tSJiqB=(xbt0FRvFdRMgozb-tuw;5ea8m>{{1`lybvg9g%BpI@sO zLlF#j#z$qRWsdgNTdz6CTz>~c+%;Up-XBOlziO+T?83r%y2BfM;K^H8>CB$N2biy*;U%Js* zb=Oi>vJ9Eg7Uo$iB!3**Y%HQAp2IGLq+gy2rLKS1dM4jb5k2C2y*?Y-ANY&P$jfb& zaR{{_alU?B-r0)(ZtA@W0oH zq(v&>TxEHVt?n*^qsPYsI#L<%))@?@QivbzoPH_z4s$1@)9o)nxA9TT9ilb|`zDK< z1x?*M`EoqdOM0@SWn+HAZ?f>+s$O^QvU=#7?0$uI41ERimJ~1FN0Rr+Sl2xB4bUh~ zGgD5g3GhdRRevKyOOPk`CzTiGL7gkZpNBv|^U`TsOjJ@J6`wO@O${FN<_+@a-rNz> z!T=S1bajlU(%sb zEskPzILyDH(sE7k@_i!t(;$b>VbaPy<%NmRer-g1r z%YXH-apWFRx7S5M4%TrpB^*dep|fnHJY@SXoqqN@pa%n$rlKWGJNeMLnJTx}qhIl6 zR%Q`ScJ!Xj+cgf*mkXhLh*njT6iB!WiVe@R{JhU`);f2Uf@3;k zH)u=H7UuJ6nr()uuxFd3Z7O4!`(E6KL4Pxo5Y%48Xc+VoL6a~6jI)Y};ues9kE%Qxe01@5szei_g~xJOC|0!JZL<<_ zQPb)bhn?xH&U&TWe+UDCHFvrswJtcxbnJbu#NAa#+p(Ywy#VZ*dzjZBLLnbpI)6B& zTjl!@bvy=W2B5!sYR0e+to)Tnmcnm`vc_+(@@~S8@!j7ecd2&N?P+13W!5;L5w0Kh z@=ko#&^Xe*7|=mJRmYP%O~~7xUa2(k0|Ey6Ek6Z7!EQ1uxfz-1!=nqsMrXzx=v-)Y zW(ljn3u0fjW$P?PYUglDN3Rg5;SB*7a!f90c+am5P3}+sGNa0S6P`vb8!3P z=qCYn#891zkO&bvL+_WRJQ^>7p(0nW-QDFYu~dK@iwlX)*)e$Bm0hOT8?)7FKXdEv znzaa5M}xylw`qYscH`?4@f}L21EKDV(@J?l*xbB4JqMX+5Ngb_J0BuvgMXfD=6f6r zh_7vS6%5+Uk)vz^X@B0O1z)dNQO_zy?1>I{Uukj7?LO1sj+_Ex->M6*Lng-A-^&I> zRkMNS;`6gaZkY45y;|1?N|c6@Ot$0e-UC#8x6J}LK-WoXfQYYuez|S!H`hgi4g85?rfiteuEJxRH?@QF6KRkQ zpJCS{y7LH1y4E%duduvtrfWN2sWId7L(4#Bp!z>@rp)d2D3yc|V#t&j;T$g?JAKjk zg?V}E%@Aq?PxF&_dVpE=q(N(fnfuLh4FzRk?%`h_qj=5286H;Sf~goSaxtC=2C}-X(~4UGr`vtqcbC$aH71Qh%4+B;9GBrq5^EP{S14 za%XY$4PU>HcN4xx^YtWq316BaL1dViEGV4yYSZpKT|xI*Nd=r9n#{qk#KEdoa`;rb zWBZ-JFjjcl3<~bLKvr5?Xe+Wb=^b)AnJ%nIEC5oh)DXGCT6kpT7hb%n*!jJ~0FbQ> z8*CG&HtpvHKYwc^HAqH4a@V)tVxV!i1)uw!KKpiEPL+0kB*C1gg}nJFdN?b$@3M;d z2QPZY9v~Q@X)HzKN!r)BEHJCInf{_mj&T>2yylM?k{~IjhZ^UQ37UNd!C}-Z76-2Q zhkNo>xC>(^*%x}X4j1sA*rXR{8R6DCeRnZc9OEqO)_*;ZE;Ob;UAcVWxBVs>PbNq* zKk6SEm)~?#5cg_%C$JDQ`j#_c^dJEQ*FL>6w8xvoxayECb*-5GaZRdG8{{Z1WqEzy zZ|Sl01aC!IUKR09dfgKXU9cktM^Bxt+V|yG8=IJ7{o1o6?)llVwMa#2$Gy)6d*{Es zYr%4?xqr`F9L1X;Rl`e`5pSRMB;P})dk)yf)QQv|r_;d-o16Qodf3_%R^pK-a22CC_$CM^AfM0{eKKi`kXMu^KaN`%jB2;x6+di4gr5%&A~0e zqPiX&_Ff&f6|t8?$Ev1pNghM#4R3}$^q@zy)M?0LgXSgQJ_H3Rt52a<^$|GrHxWIJ ze0~B1=d9=*u+NGHZyDQIu^@%o)~e#&#RQZM`HDSMy|HVq9~*fW;4O&J^>XeE4W_6| z?d^X-SGVmw9gM#ca6i2}t~YrJmXW>&JNF8Bjme8>S*k9Qs{P4M~il4q-uEt9q3vv;~>ij%A2Y%<&e5) zr_9^`V~v5ZQ~TufqJZ^Q?R7)QI?T5^MSPjwll?v}&ggln23Vpszo2Fis}q}ppxHJG zD2u6ZOug4C$%E1bYM_6Xul;5aPq7fvEPAg(YB9dvOR9YE-&aqH@rHvPB_m7}o7x(x z19>z^8&c&x7rAPF;>}WJjlND1ipl~#VAP}iIooH4hAoZ9t6ZR;S$vF@H!3==BS7`3 z5ipNQnc3peM&`5oI6Yksjc=lhEUosEXU9kJctAt@n~gHzw?Kb%&>?8x`4-08J+C?n z4jIs2cuH-U6PjOdvHJ--kq*y2)j%j*g!~|j5No9)8c34zElZjO*c-`xJW>(%tN*;t z@*H(=no#NK+{>~5;CYl5ON|Xi0KxOcgVkp7oc0ra-8bbLIuN2qInhg5_xj4ZcXoOU zDkFgo7X>ekd|Q7u6z}0K(#;cc&ojEt>f5V7c)=fAO^n67O|#huHhYo9jkx!b5z}_K z!_|FqO@~HPA>pp|-wrY@Az91_CJTS<(7hSe!B`nQfB-#~%vpuv{t5l}HU;O#u0$`o zlc7GY2mVqmsm!J1GF}hrQ?H&*VQ1Slx_L9@@VRxYO|~8SD;vmn%gv#PInL7X1pl z+$W1zGTwAw(J#Q+oeRT=nY~k#As?ZNOsT#G@lz*5S4qsL-!))dl?;Stl2J8Nic< zg;>%lA_{6OO;Q_SrXeF#w(J=oMFTdoapD5!hxE0Lh=|-1AG$1Y(pukQEDz$oNU6vy zO_`bTtj2WEYg4{eaYiE?g{SD3Cbz#lSoah=i@h$w(X0Bs8^Ur5ARz)IJXQhgPTHF_ z3e)>?D)>NuLEq#0!uuf)c2HS6Ey8w*@bVh8wc!ZLTH9LU`q2|~F$;TvYuIPf;%R%M zoF`b^v_0Q4B(xr{>qKx(3U)kMk`y*eM&S5ZCvnlK8eILoL3_pa?=0+%7dzVcaD1=2 zN96EloY_w0@$TrTK`-!zODIrEuX~BhbuUaZ*|8yi^NhLI&Xal_VKb%k{4Ao)TC1&BD|@AuQq3dGVx~9LLb;WE2JM zso=tFRk=9>#wzYsU5N>DbeK1jTb1;5(j7zXYg2c4o!e#GB|P%Q=5ZHQE8~6#$NgIm z5{Ri_1Tdl`XndW1TGG{^fPuYnVJ;HVw<5lOnff6+l+d(_6l62LStDP0$q^Y(uS#PQ zJ2Q|!vSH*w!H(Laa*fX35YyCcw3@9c+y|zz`&85iWG`@EKMF{h{bMa=!<`}jhVa_+ z%bIUCskzTL7FWlN{nb`=7QL`i#R1&+Ski1tQwJkDw*mdcD zR3^=%Js-1AgFY$>$8U1%rINX|lRC-k%@>U-ndbZQAzR*sOGhV~=$ls+(eXqVQEK>&xV`tUm{EGe68 z&V?u>CbF^ZB zp_4f2-1v44A?wjRECUBGG7^(-5J^wpboY1W4bl# zTq3ntLhxo}XDfMgtYAGumxNgOx=q|}ZuMse9nS}64V5n@@T4v2BsYdYc6C>O-L((} zsKa!RNlek(athqBu*&G4^HWZ%dqwM^(dlF?|KKB(YSVedDrSx=xxTg z0nT=K+k?Dha_kL%KSd}^OB4(D(w49N`xqQ>9m^EtNvcDPCw3fA#x z+{#=qXqa0r=4`CB- zd>TH43P5eF(td$|dHYd@8S4cBYR2pJ4gc47=)LI$Tl>;s-MgR}YjcYdSM~zhAPnwJ z#*TggBY;lnx#%i^8Sj_JiRggO6{JT)MYSHB{mC(g5@=6_5A1~`)*l(D)8Pl-6qA+r znoAXSBwi&jQzeFc(xHR0sSj#F^=Q@;%%VIN$JtqHd{Fv-wCj8|P?lTo^P^=)o~@TM z>+-explF@y@^jbBIhzV;4H3z>sDImS?#^(~k0@2-HVglh+l^n6EINj7jC}#LFVx`f z>vVD{1HwB6jq&I2FF$5yKJ#3Eu!^whV$Ua?17F6EVl!;KuHtku1$a#L_UFy~hU@#S{7r!)EaoQkQL$D9 z*fk-`jOetE6 zKVZAB(C+WA>tUKgfM)H9qusgTIVBAef)@oLN`$q4v7qt(sBG}oHd@=uAwN{)eW@zC zFTtS$-rlcMsplSMGBNvH4kwHO-}fq_76L_)wemqLO&Wf?$>9e@$bRP1EJzsx2hgI9 z)%OjvpPJ56lE4t#kT$n+LqZ0A-svmzP9bIxKy!?v5oaYQf0*l`UgHz+bH1U?T+;-NVBz4}r9EdR{dIaU<@vM66E^%1OTP zQQRYk3eCgo+N&kP>*HxF-!2s#4cmfFV(e%J9^5vOCstjXM+!r_wDDedi%{`km7MfB zmZv*iIOua#;0rL0A{()9Ta`)j-IHNX`*#U{p3<$<)4J(+aGq25vU3|g8bvi3%9W+C z;)mdAAd*%&OZ#4jHa z9A6915YtF#G&4%i;ZEdvWNlf@0lpWdwAmJo|)1MPnpfH zY_LH6q5dYI?Y;b*eJg$q%7~>+t3=f$|74v#TWcv!%FQvBXALi~qhsS=$u$sbrLnfM zV+m1#xxWWN4XgzxS*`klM>^bXN@CkdiBKQe{(XZwqwQa~Q?2i;2$~$N*hnX=Yhje% zl&wYJOolc@ketr?4l|b+UP-^)rMY#Y9bc-j&vE%S2?5J>&d`Prb^3oiW|xnm4w!!z z@Hf#CLw(ub8dprU(;3Vc?F=H0&g*}|9xyuPjh%7Bn|&k_NFG7z^9+tRFeFzE#}M9y zP2Y0v4Tb@|c62KFenW#gF|4c9x=14jiwN=|&|;)bZ}LprFxYYbB!SeRa538S zOeOOc+&O?#^+m%a)Sdew4WSkh2G)Oc&{yQO@t(c1T{dwxFm$r+#&+1dc3}sFNz^?< z@cqnCfOcZu<=iX!z>EcfIasr$9adoXD7q#7#sfJeeEH?&F>mj@^V^b-CNsyP_4iv$ zRQK|^1Fk~LYgs|=BFz+E(4)$CIf;DCbBwxot`9iI`h!@X7mTDQC}p$8tQdbP_gwZz z9jZAgL@;n<9Br~qU6FX@uWLRQQ&Xr^KBJK)zAGaVc*mQgzJUwQ?tw?%dl;1Z;vnJ3 z<)!cD_Oy#Qt0kY!n=Bn+Pw!hh$#RN;m2*m}Kr^d7-mGRNVFE)q&p}x@6$r86h5$>l zC8nT*#4pom73vDPtm2(^6z?I$bDQ} zwxSshuTIYU-9+4dDbS|DbBz*zNHUW|f;rnmnhvJkzDliJNq*j(h`Po#{YwM2KQ;2^c{E;Ij05L1_ z-0MIY_zh|Wx55}5}nNY)OoPtKI+I89`d@QBp7`=LQ+Nm%4vFlwzWt6iNK$!JbKhkpU& z3~RKsrXBs1<#-h^ssB9X!na=p)2g8*}!J1{*!_=E>c`3w5Ed z@FUK>$y&Q=8#&iIFsMFjOc^yLdhot-FM2Hubp1n<(ndO)e|LY-@sEV->mZ}-{NZ&C zpOdc&?39#g;ohrbtu?2o9z$-AJ(UYo<}1RxAy}1~3H_zK>j}2ujjx#E?x>$sznt-j z5RcMi>vroFUD(RJGhj}Gv2`Lw0%IqqpXa7#&JLmHeU&F`!^Pm==pXD{h8vl_1+$6i zM4^N>OwW>M9V36ZssrAlrg}^KkE8nQGTNYCOc1L|NDTj7S+LG&mO^~gl!h#oUXf4q z8IF7QdAy&ZG;sKC?^W8b#Q+Rw_X*Il3TwlB!LK(Ogphr7URM&m>=2t{Bz##~tPljV z$W@mTcy*+FuKcxs*L)3)(HWS+-k0jvXr9D!F-DFi_7H!V1aVzgnJ>2Ig?0d9i9)gl zZQ6EU=DG6_4x%k#Ui`Vd(_8QJ!}O&OqqyQB@+Mhfn=?=Yqm?NU-yi=E=N^ zGkuk?Dupu}{tGQp#790B%Myf3rVKT@D>L0VQFo3nRqXRxeC*pv;A)et4}ANkrlXr2 zRWvHG_S5=UP~jU6otFTw>Ykna-PWL3P!6$#Q6PWR3A+(ifDZEXuY9ap(lNq!-s;Co zB^3TxIZec2|0acXRu?r;Sc#A>O3L3l z(z5!L*noe+fYcTQIjwA^T(WqG>G7APR*`>%biykcu((X%v5+Y)*HqA?C|s}WBCxUR zF&x8pRsEs7TNmbnkSnE=)$AJcp61@^*hT-nJn*A6kN=&jOvMU%w9f2W9s}BRR2# zU$W}1ML&iECX`Eua}EHJT&8cnUc-De878AieGt+iibeO7y8l zzgN7DFohIXOl?<82FlDJq=Eiiv|p22jB6?{l=&4<26FhSN~)aE)4+f;tRvA%zYGCoey zq9P=FG5ce+6YjCdU=j(EM+OE$ z?)3OP#%Cf!X>W3IX)}!d9*PWNQ6`2WF52ckRB$#`aiP*5Oa?9$EscLu)`*#+ZtAS0 zM(mS*4n)QAQQ_GeN|TPz^Lpvm#}kd>e=9G6QA39O(19DJLHtqwk^ish|5986^i%OIwh4UP)I+@m3c5VjDg$I`r66#dY>N3NN;|z>!egq zhp55(fa0G^%UOQ}6b7?{uJDNQJv+k*W3bZIwuSii{MO(~WTsUv>_Xn#OA+t%3F%toj1CucIoi=DlDX7 zcdy58SK=mEJ&jHqgM!0z=>~@GnFjlB?yb>Z_6?Q|sW-1Y>aE$+9+GD=fihLi&k{)L zU5MpD*sYrB1H3r!ON~p&JJ|-NmH6(T9p37ss11LUS=PCHUf>TEI9Bl|y=Jb_Q2uY|s|aY0`w=5VLKj?f zTJ?W<&ZD3mB})EVt`1f;qRW&X_SKEH29Ljl@kzbpYB^Ygeqy(Uonqw;ed~-)6h+)> z4b7-|9>3F3YJqBlH;w)eAa!e^baA9l`HB&GGeGo>r@m|4^C?*r7AhrdLFV*GLAV%V z#)aTOmik~p*nQ5nKX2S+c*EKmqpIty#-@Kcm<~yMHTcRK0IL3J|4Ng|@?k9Uv)yI> zAEq9MA};K-5JPV7!uo&2swrMO#lJ1e(q|yaW#_L!Oj69VJ+zvNS?6|*BZ5#4|E_@I zY+$HXL7>xxj`q>7YzqdLE}9ll8!y(HmJmD50I~JLbN>bxO}p;~V=*Pemw7@pvl-LH}zeK$O%6 zO~FI4-&3%5+EJk@zpX&eys~`+v5{8*lvHLp35d(HfLFxI)BKk{5#DITItxV@_HjxT zm4vzNm?+9pjcBpNE9Z=wAp}=yIRzKMcT*IisF#gvvqcv~pH@2j2e~j8bl+c)@0K8m zh3>dU^KX~runxn2pANUBYQf46)kQfkgB!Yx!Dz0Tt63K^yW?C!aYkC-bRjv`3~KnQ zX-~+%11e6N%?!q1-$7C<*L-3Q>&&&K&Qh=ak$=^%#S$I&?+bh~1ZP$CAg&J4>#N+9 z^V<790J4?y$LFmI@kWZvHf7%dSYv~~MNYFX?iwvNRyt^Zm)006?g4=9;6nCD0~IR- zqJQ}gD5={eJfkABgVPkCtx=7bAJ4{7s)l7)TF5kAUiI#U63II!dS5rhG0KrMd zRzy3gDIdCjP8H+6v-LQ2BT>Um4j^A3UdPdOMjse?b5hD)ZR*!Ta)#HM^KDFc+aNr5 z90}z0NV5UzgmQ!P4fn@OS!59LgAm}8IS&1qiPlnQ}xcu`YV)(*|xb* z^!R1;r#>14517|@-atU zvL5r!AjlUYJ?)xYExlVN!+%R&TR~18SH@q+TeYE)%H#eCopn<$%g-^x&o}J=e#B->@bD^Bbjb*WZ>s)L19_KTn9dY~h^Q zwZEym)n?somwW(0vl5r=hQXt6J+pgNk%n2&XkVI@zq6#2K-N9aB7fRW*^YRv3%J&4 z>NyusU6*WI!a#DCRojf+x;)s)zx{mY@u^!q9k{v%1%GIAIzW3i=hluS2~xRg ziOnQd&FBENs@Q7h6HP)%%j`Xl;k&$A5}It$7k83lb^XNJCXk`&0q;Yp<91NMog)7{ zdsqZ9xr&)Jk|?k;9=>K(z9I;?f5)?MXlxFy3{F|v^}C*wf$&)l#YRtV;L||(r0?0JnUr|yT6#Az!WT1A^am{$7 z%25$@MbJ;nx-TXpQVhr=3hLY57V+NL^yB@JJikLi*bWPFe&5R$wY3ek0#2S8i=u57 zko=0#r+$x4((-V$BTmgGY_jJS%Xy=g8?e zA-ztm4u7ozI%!+*C`aGXI?Kw4t7S~GIVc6HYcpf*2)^klz&&lRApciY^i(lowfUO-i8;(yg9!n+$%Q#M(;sEiZU@bTsvbg%x>ZaKy$MkRDeWFh%lF%!Z8KOTEyGW&*+Gx?xg;tL4=z=UK(OePBNe|*Kv&Nq823CSBWZ|F0<`R zR)EbZi!YG;PXAKDy&7D;B+~^lrW>sAt&FX{ER>Y*qr9eI(53BH&-Nus2_a$IYZ+&- z4X@}qfYTUu4^%yw&Be!>b!nR&R$6o#xKHAq4-yQy3uWn#KC=RGfOucxNV4YSS8>dV zrKk_8tuVf_GnDMD+Psi$k-B&E1LnXVa}$6J+t?ZpfH@z{Vg+O~E14Ex&>dIrdxY== zjX&3|&NutNw-US#<^q3|Tz}gH%*-P$bjPrIH37}K8E(#4xImzA;sC}gky0fm4*94*}#Q{11`beciB_pfhD}v?>2u?#*VB-0UD?{@SiVf zy3B%o#WOC|<&fnuBO)Q^WR+tZap;M`@2*l4oEL6TAWO8qFF=Xeh}9E2;_204cu;XvcSf3mND3Mn}!j$y3uL(H@=uMsmO6oH_+V zQ@QEa1~q?eWLwh0>?l|fD@2CN=-$|SD-s_{^w>%C z9J{CcNAxvaFY%n%SmH@dEMDz`vJ@HSA6zN|_l!~wG^X2h?B% zyG4I1r9!f2B#sS4iAJYNZ9A{bd=lDzModq1NrBD;j2yM+Yuz$dC1 z8>1=;*t&i;IS+Iu0z7?C75PZD^=0JA;baEhORwzABi|+TH5zuNy$|)5!F$NJ#?r}k z&6GM(+*DxcvU!L;Q?YOc%fPn8e2mL^7g~Rd<5f04qQ3dpx9jv^Yz5H<>30mUr<Z=2AYtyeHOAY!F-1lhRNL%?ip($*Amk+x`P;Fr8=25L{sX;6XGSE| z0sm<^vDsoocc;u~+lJjehKcDj$MM=ODEWDdP-QiaZNE+gJ%-Nn+)2QQiCimZfgYh;M&8G^(mHmeBN5rSd>5bEVqL>MIy+ zjU=FZVO5qM@GV3d*xsK7s?(~%d@p}Ztfj&vKF61aXs1Bz{Uf?VzOnu7;Y{2DT|D0* zB+iyA3yz)C?Gx-1{(Q7glI|HK6a5x%!@d?p$>%cMJhK-0L?i0%n{rXv^ z{L^*5+R-tJ5N~t!aF@17V))t~MaSg|$$}w)bkH0$!S^k-ubb9*pMtkVaRFXW2<{zP z0iEG77kQ?K=7qgD`fQ1XTgYJffiTdEXzk@ig}&BnHwtGIu?S$YtA0f!%#-rPTs_Tf zdH~!T%1~wDzyR3P)6k~|K@CUC(VX}0$CHSQrJF)scLhDR3W%(!$;%S3XnXu0IRQg| z@9Uz-YrnDC*3+XWN!8o8m(rePCzF7S4jg8*X%fc4JwSnU**~|n#13x)f6rLsQYUvT z))$b}j8kftQevMS4tMX$JwIf?w$?*^;JJ{*4-vCB62#az$zZym%j=+M6Un~J6Wvh8V_~|A#Ax~{0+S2XC2@~OouR}1v^X1;+hkNP07o^60V`p~g1N|^!sOd!Xuw_2k<1y0vZIK+@+&)gJmw5H z96G%g!=>g*Fn)~X&`Xv7R@W2*A%&D^%G&{pIDa>`+SQE?tq!idmihs0p$hnX z&#c~Qbp@%@$vREbsEP4zx1JH^C?D_nQ%G$v+Ad9Up`#PLHxS1uyXi7D zZUECrSO2i!CL4PJ31l`QBzF6$LVYo|YVKr&>$IZ&?j%KCLiE@0#iGh@j(iQEoOaOq zP`qux^=A3=tKSpc5-x?vZ)0t*4Mo|H;;@EhscmfjaGkP&ZoJ!g=sRX>b#y`&sIfbo zwgUryRvt{6fBiOVrY?!n=Ck2@9$!0gt;ADMgEu944grz(EdcYY=l@!(8nHbq{;Uz@ z{%C+ZvK!{!R03nQn>n`Sv5hk;$u*W5^nb=ICMz1tHk1|ky%&A(3PFFg!|w*^*Vzo= zg|0RKTtTzHs0WC&f0a`|M>d1~IY{%ZivxD>vpUGre<|5W7HDwS_Bj4*HUTALW*QIh zSK%rA*J~?q!EQDsAS-N#sW7OmDGOYqkd4~l0mU$zj>QkK40BJk15K46I<6vo$Gbf* zNHe}7gYP$Xa+VyorJTybP=a`grfe$;Nd(xlU9V9`6)VuI4kKx3Xk}VnR6wY}+ciB$ z1LVE|e~Ir5(SX3t(!c8X%$1>+&#T~iU+srbom*yQWfsnrVcWxLasQ`WQQ42u}SDybgn~ zJeOVlOcfUy{t8)jdV_r$gRYDuV5w2KgI2Bof9_oAVP*I+N*9A}FvCLrsAAtK_c0U| z^6SC1&U4{!X3!tNuFC)c)pUZzm1a}5-vCrh>96rc)J4^B|1=x)_Zlq<#D}asx~O7# zo}Buj2Q9L~@&sJ)N*PxOLT-m1|J{AIQ@NMP6e~rnxz1#^!}D=o0=