From 24157127e91b96435f9df030c3dcafb5306da87a Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 13 Oct 2022 23:13:00 +0530 Subject: [PATCH] Some fixes and improve perf Improve trino perf by separating executors Split size estimation and minor fixes Use caching metastore --- .../plugin/deltalake/DeltaLakeMetadata.java | 3 + .../deltalake/DeltaLakeSplitManager.java | 5 + plugin/trino-hudi/pom.xml | 63 +- .../hudi/ForHudiBackgroundSplitLoader.java | 32 + .../trino/plugin/hudi/ForHudiSplitSource.java | 32 + .../java/io/trino/plugin/hudi/HudiConfig.java | 83 +- .../hudi/HudiHiveStatisticsProvider.java | 894 ++++++++++++++++++ .../io/trino/plugin/hudi/HudiMetadata.java | 12 +- .../plugin/hudi/HudiMetadataFactory.java | 16 +- .../java/io/trino/plugin/hudi/HudiModule.java | 28 +- .../plugin/hudi/HudiPartitionManager.java | 64 ++ .../plugin/hudi/HudiSessionProperties.java | 55 ++ .../java/io/trino/plugin/hudi/HudiSplit.java | 16 + .../trino/plugin/hudi/HudiSplitManager.java | 43 +- .../io/trino/plugin/hudi/HudiSplitSource.java | 25 +- .../hudi/partition/HiveHudiPartitionInfo.java | 2 +- .../partition/HudiPartitionInfoLoader.java | 105 +- .../hudi/query/HudiDirectoryLister.java | 2 + .../HudiReadOptimizedDirectoryLister.java | 46 +- .../hudi/split/HudiBackgroundSplitLoader.java | 105 +- .../trino/plugin/hudi/TestHudiSmokeTest.java | 7 + .../ResourceHudiTablesInitializer.java | 2 + .../testing/TpchHudiTablesInitializer.java | 17 +- .../.hoodie/20221014130009989.commit | 74 ++ .../20221014130009989.commit.requested | 0 .../.hoodie/20221014130009989.inflight | 71 ++ .../.hoodie/20221014130117650.indexing | Bin 0 -> 979 bytes .../20221014130117650.indexing.inflight | 0 .../20221014130117650.indexing.requested | Bin 0 -> 659 bytes .../.hoodie/hoodie.properties | 18 + .../.hoodie/00000000000000.deltacommit | 97 ++ .../00000000000000.deltacommit.inflight | 116 +++ .../00000000000000.deltacommit.requested | 0 .../.hoodie/20221014130009989.deltacommit | 72 ++ .../20221014130009989.deltacommit.inflight | 80 ++ .../20221014130009989.deltacommit.requested | 0 .../metadata/.hoodie/hoodie.properties | 14 + ...l-stats-0000_20221014130009989.log.1_0-0-0 | Bin 0 -> 127 bytes ...-stats-0000_20221014130009989.log.1_1-8-12 | Bin 0 -> 11431 bytes ...l-stats-0001_20221014130009989.log.1_0-0-0 | Bin 0 -> 127 bytes ...-stats-0001_20221014130009989.log.1_0-8-11 | Bin 0 -> 11493 bytes .../column_stats/.hoodie_partition_metadata | 4 + .../.files-0000_00000000000000.log.1_0-0-0 | Bin 0 -> 124 bytes .../.files-0000_00000000000000.log.1_0-8-8 | Bin 0 -> 21950 bytes .../metadata/files/.hoodie_partition_metadata | 4 + .../2018/08/31/.hoodie_partition_metadata | 4 + ...524e9e-0_0-29-29_20221014130009989.parquet | Bin 0 -> 443915 bytes 47 files changed, 2014 insertions(+), 197 deletions(-) create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiBackgroundSplitLoader.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHiveStatisticsProvider.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPartitionManager.java create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit.requested create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.inflight create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.inflight create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.requested create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/hoodie.properties create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/00000000000000.deltacommit create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/20221014130009989.deltacommit create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/20221014130009989.deltacommit.inflight create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/20221014130009989.deltacommit.requested create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/.hoodie/hoodie.properties create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.col-stats-0000_20221014130009989.log.1_0-0-0 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.col-stats-0000_20221014130009989.log.1_1-8-12 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.col-stats-0001_20221014130009989.log.1_0-0-0 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.col-stats-0001_20221014130009989.log.1_0-8-11 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-8-8 create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java index 07733d89681..26e1ac2935d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java @@ -590,9 +590,11 @@ private List getColumns(MetadataEntry deltaMetadata) @Override public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTableHandle tableHandle) { + LOG.warn("inside getTableStatistics"); if (!isTableStatisticsEnabled(session)) { return TableStatistics.empty(); } + LOG.warn("fetching stats from metastore"); return metastore.getTableStatistics(session, (DeltaLakeTableHandle) tableHandle); } @@ -2612,6 +2614,7 @@ public static TupleDomain createStatisticsPredicate( List schema, List canonicalPartitionColumns) { + LOG.warn("inside createStatisticsPredicate"); return addFileEntry.getStats() .map(deltaLakeFileStatistics -> withColumnDomains( schema.stream() diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java index bfc55d8df49..76b70281e40 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java @@ -14,6 +14,7 @@ package io.trino.plugin.deltalake; import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastore; @@ -66,6 +67,8 @@ public class DeltaLakeSplitManager implements ConnectorSplitManager { + private static final Logger log = Logger.get(DeltaLakeSplitManager.class); + private final TypeManager typeManager; private final BiFunction metastoreProvider; private final ExecutorService executor; @@ -152,6 +155,7 @@ private Stream getSplits( .filter(column -> predicatedColumnNames.contains(column.getName())) // DeltaLakeColumnMetadata.name is lowercase .collect(toImmutableList()); + log.warn(">>> creating delta splits"); return validDataFiles.stream() .flatMap(addAction -> { if (tableHandle.getAnalyzeHandle().isPresent() && !tableHandle.getAnalyzeHandle().get().isInitialAnalyze() && !addAction.isDataChange()) { @@ -198,6 +202,7 @@ private Stream getSplits( } } + log.warn(">>> Creating delta splits for stats: " + statisticsPredicate); return splitsForFile( session, addAction, diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 1d873e46cf1..02629d47e1a 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -110,11 +110,6 @@ guice - - javax.annotation - javax.annotation-api - - javax.inject javax.inject @@ -130,7 +125,59 @@ joda-time + + + + + org.apache.hudi + hudi-trino-bundle + ${dep.hudi.version} + + + com.google.protobuf + protobuf-java + + + commons-lang + commons-lang + + + org.apache.hudi + hudi-common + + + org.apache.hudi + hudi-hadoop-mr-bundle + + + org.apache.parquet + parquet-avro + + + org.apache.avro + avro + + + + + org.weakref @@ -403,6 +450,10 @@ io.dropwizard.metrics * + + org.apache.logging.log4j + log4j-api + diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiBackgroundSplitLoader.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiBackgroundSplitLoader.java new file mode 100644 index 00000000000..edd5189d043 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiBackgroundSplitLoader.java @@ -0,0 +1,32 @@ +/* + * Licensed 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 io.trino.plugin.hudi; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForHudiBackgroundSplitLoader +{ +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java new file mode 100644 index 00000000000..608c25554b5 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/ForHudiSplitSource.java @@ -0,0 +1,32 @@ +/* + * Licensed 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 io.trino.plugin.hudi; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForHudiSplitSource +{ +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index ea323818bdf..abf18f4dcfd 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.LegacyConfig; import io.airlift.units.DataSize; import javax.validation.constraints.DecimalMax; @@ -42,10 +43,15 @@ public class HudiConfig private int minPartitionBatchSize = 10; private int maxPartitionBatchSize = 100; private boolean sizeBasedSplitWeightsEnabled = true; - private DataSize standardSplitWeightSize = DataSize.of(128, MEGABYTE); + private DataSize standardSplitWeightSize = DataSize.of(64, MEGABYTE); private double minimumAssignedSplitWeight = 0.05; private int maxSplitsPerSecond = Integer.MAX_VALUE; private int maxOutstandingSplits = 1000; + private boolean tableStatisticsEnabled; + private int splitLoaderParallelism = 2; + private int splitGeneratorParallelism = 16; + private int partitionScannerParallelism = 16; + private long perTransactionMetastoreCacheMaximumSize = 1000; public List getColumnsToHide() { @@ -99,7 +105,7 @@ public HudiConfig setMinPartitionBatchSize(int minPartitionBatchSize) } @Min(1) - @Max(100) + @Max(1000) public int getMinPartitionBatchSize() { return minPartitionBatchSize; @@ -114,7 +120,7 @@ public HudiConfig setMaxPartitionBatchSize(int maxPartitionBatchSize) } @Min(1) - @Max(1000) + @Max(10000) public int getMaxPartitionBatchSize() { return maxPartitionBatchSize; @@ -178,7 +184,6 @@ public HudiConfig setMaxSplitsPerSecond(int maxSplitsPerSecond) return this; } - @Min(1) public int getMaxOutstandingSplits() { return maxOutstandingSplits; @@ -191,4 +196,74 @@ public HudiConfig setMaxOutstandingSplits(int maxOutstandingSplits) this.maxOutstandingSplits = maxOutstandingSplits; return this; } + + @Config("hudi.table-statistics-enabled") + @ConfigDescription("Enable use of table statistics") + public HudiConfig setTableStatisticsEnabled(boolean tableStatisticsEnabled) + { + this.tableStatisticsEnabled = tableStatisticsEnabled; + return this; + } + + public boolean isTableStatisticsEnabled() + { + return tableStatisticsEnabled; + } + + @Min(1) + public int getSplitGeneratorParallelism() + { + return splitGeneratorParallelism; + } + + @Config("hudi.split-generator-parallelism") + @ConfigDescription("Number of threads to generate splits from partitions.") + public HudiConfig setSplitGeneratorParallelism(int splitGeneratorParallelism) + { + this.splitGeneratorParallelism = splitGeneratorParallelism; + return this; + } + + @Min(1) + public int getSplitLoaderParallelism() + { + return splitLoaderParallelism; + } + + @Config("hudi.split-loader-parallelism") + @ConfigDescription("Number of threads to run background split loader. " + + "A single background split loader is needed per query.") + public HudiConfig setSplitLoaderParallelism(int splitLoaderParallelism) + { + this.splitLoaderParallelism = splitLoaderParallelism; + return this; + } + + @Config("hudi.partition-scanner-parallelism") + @ConfigDescription("Number of threads to use for partition scanners") + public HudiConfig setPartitionScannerParallelism(int partitionScannerParallelism) + { + this.partitionScannerParallelism = partitionScannerParallelism; + return this; + } + + @Min(1) + public int getPartitionScannerParallelism() + { + return partitionScannerParallelism; + } + + @Min(1) + public long getPerTransactionMetastoreCacheMaximumSize() + { + return perTransactionMetastoreCacheMaximumSize; + } + + @LegacyConfig("hive.per-transaction-metastore-cache-maximum-size") + @Config("delta.per-transaction-metastore-cache-maximum-size") + public HudiConfig setPerTransactionMetastoreCacheMaximumSize(long perTransactionMetastoreCacheMaximumSize) + { + this.perTransactionMetastoreCacheMaximumSize = perTransactionMetastoreCacheMaximumSize; + return this; + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHiveStatisticsProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHiveStatisticsProvider.java new file mode 100644 index 00000000000..abd3da6f141 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHiveStatisticsProvider.java @@ -0,0 +1,894 @@ +/* + * Licensed 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 io.trino.plugin.hudi; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.VerifyException; +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.HashFunction; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Shorts; +import com.google.common.primitives.SignedBytes; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.trino.plugin.hive.HiveBasicStatistics; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.PartitionStatistics; +import io.trino.plugin.hive.metastore.DateStatistics; +import io.trino.plugin.hive.metastore.DecimalStatistics; +import io.trino.plugin.hive.metastore.DoubleStatistics; +import io.trino.plugin.hive.metastore.HiveColumnStatistics; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.IntegerStatistics; +import io.trino.plugin.hive.metastore.Partition; +import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.statistics.HiveStatisticsProvider; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.NullableValue; +import io.trino.spi.statistics.ColumnStatistics; +import io.trino.spi.statistics.DoubleRange; +import io.trino.spi.statistics.Estimate; +import io.trino.spi.statistics.TableStatistics; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.math.BigDecimal; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalDouble; +import java.util.OptionalLong; +import java.util.stream.DoubleStream; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; +import static com.google.common.base.Verify.verifyNotNull; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.Maps.immutableEntry; +import static com.google.common.hash.Hashing.murmur3_128; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_CORRUPTED_COLUMN_STATISTICS; +import static io.trino.plugin.hive.HivePartition.UNPARTITIONED_ID; +import static io.trino.plugin.hive.HiveSessionProperties.getPartitionStatisticsSampleSize; +import static io.trino.plugin.hive.HiveSessionProperties.isIgnoreCorruptedStatistics; +import static io.trino.plugin.hive.HiveSessionProperties.isStatisticsEnabled; +import static io.trino.spi.statistics.StatsUtil.toStatsRepresentation; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.Double.isFinite; +import static java.lang.Double.isNaN; +import static java.lang.String.format; +import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; + +public class HudiHiveStatisticsProvider + implements HiveStatisticsProvider +{ + private static final Logger log = Logger.get(HudiHiveStatisticsProvider.class); + + private final PartitionsStatisticsProvider statisticsProvider; + + //@Inject + public HudiHiveStatisticsProvider(HiveMetastore metastore) + { + requireNonNull(metastore, "metastore is null"); + this.statisticsProvider = (session, table, hivePartitions) -> getPartitionsStatistics(metastore, table, hivePartitions); + } + + private static Map getPartitionsStatistics(HiveMetastore metastore, SchemaTableName table, List hivePartitions) + { + if (hivePartitions.isEmpty()) { + return ImmutableMap.of(); + } + Table metastoreTable = metastore.getTable(table.getSchemaName(), table.getTableName()).get(); + boolean unpartitioned = hivePartitions.stream().anyMatch(partition -> partition.getPartitionId().equals(UNPARTITIONED_ID)); + if (unpartitioned) { + checkArgument(hivePartitions.size() == 1, "expected only one hive partition"); + return ImmutableMap.of(UNPARTITIONED_ID, metastore.getTableStatistics(metastoreTable)); + } + List partitionNames = hivePartitions.stream() + .map(HivePartition::getPartitionId) + .collect(toImmutableList()); + List partitions = metastore.getPartitionsByNames(metastoreTable, partitionNames) + .values() + .stream() + .filter(Optional::isPresent) + .map(Optional::get) + .collect(toImmutableList()); + return metastore.getPartitionStatistics(metastoreTable, partitions); + } + + @Override + public TableStatistics getTableStatistics( + ConnectorSession session, + SchemaTableName table, + Map columns, + Map columnTypes, + List partitions) + { + if (!isStatisticsEnabled(session)) { + return TableStatistics.empty(); + } + if (partitions.isEmpty()) { + return createZeroStatistics(columns, columnTypes); + } + int sampleSize = getPartitionStatisticsSampleSize(session); + List partitionsSample = getPartitionsSample(partitions, sampleSize); + try { + Map statisticsSample = statisticsProvider.getPartitionsStatistics(session, table, partitionsSample); + validatePartitionStatistics(table, statisticsSample); + return getTableStatistics(columns, columnTypes, partitions, statisticsSample); + } + catch (TrinoException e) { + if (e.getErrorCode().equals(HIVE_CORRUPTED_COLUMN_STATISTICS.toErrorCode()) && isIgnoreCorruptedStatistics(session)) { + log.error(e); + return TableStatistics.empty(); + } + throw e; + } + } + + private TableStatistics createZeroStatistics(Map columns, Map columnTypes) + { + TableStatistics.Builder result = TableStatistics.builder(); + result.setRowCount(Estimate.of(0)); + columns.forEach((columnName, columnHandle) -> { + Type columnType = columnTypes.get(columnName); + verifyNotNull(columnType, "columnType is missing for column: %s", columnName); + ColumnStatistics.Builder columnStatistics = ColumnStatistics.builder(); + columnStatistics.setNullsFraction(Estimate.of(0)); + columnStatistics.setDistinctValuesCount(Estimate.of(0)); + if (hasDataSize(columnType)) { + columnStatistics.setDataSize(Estimate.of(0)); + } + result.setColumnStatistics(columnHandle, columnStatistics.build()); + }); + return result.build(); + } + + @VisibleForTesting + static List getPartitionsSample(List partitions, int sampleSize) + { + checkArgument(sampleSize > 0, "sampleSize is expected to be greater than zero"); + + if (partitions.size() <= sampleSize) { + return partitions; + } + + List result = new ArrayList<>(); + + int samplesLeft = sampleSize; + + HivePartition min = partitions.get(0); + HivePartition max = partitions.get(0); + for (HivePartition partition : partitions) { + if (partition.getPartitionId().compareTo(min.getPartitionId()) < 0) { + min = partition; + } + else if (partition.getPartitionId().compareTo(max.getPartitionId()) > 0) { + max = partition; + } + } + + result.add(min); + samplesLeft--; + if (samplesLeft > 0) { + result.add(max); + samplesLeft--; + } + + if (samplesLeft > 0) { + HashFunction hashFunction = murmur3_128(); + Comparator> hashComparator = Comparator + ., Long>comparing(Map.Entry::getValue) + .thenComparing(entry -> entry.getKey().getPartitionId()); + partitions.stream() + .filter(partition -> !result.contains(partition)) + .map(partition -> immutableEntry(partition, hashFunction.hashUnencodedChars(partition.getPartitionId()).asLong())) + .sorted(hashComparator) + .limit(samplesLeft) + .forEachOrdered(entry -> result.add(entry.getKey())); + } + + return unmodifiableList(result); + } + + @VisibleForTesting + static void validatePartitionStatistics(SchemaTableName table, Map partitionStatistics) + { + partitionStatistics.forEach((partition, statistics) -> { + HiveBasicStatistics basicStatistics = statistics.getBasicStatistics(); + OptionalLong rowCount = basicStatistics.getRowCount(); + rowCount.ifPresent(count -> checkStatistics(count >= 0, table, partition, "rowCount must be greater than or equal to zero: %s", count)); + basicStatistics.getFileCount().ifPresent(count -> checkStatistics(count >= 0, table, partition, "fileCount must be greater than or equal to zero: %s", count)); + basicStatistics.getInMemoryDataSizeInBytes().ifPresent(size -> checkStatistics(size >= 0, table, partition, "inMemoryDataSizeInBytes must be greater than or equal to zero: %s", size)); + basicStatistics.getOnDiskDataSizeInBytes().ifPresent(size -> checkStatistics(size >= 0, table, partition, "onDiskDataSizeInBytes must be greater than or equal to zero: %s", size)); + statistics.getColumnStatistics().forEach((column, columnStatistics) -> validateColumnStatistics(table, partition, column, rowCount, columnStatistics)); + }); + } + + private static void validateColumnStatistics(SchemaTableName table, String partition, String column, OptionalLong rowCount, HiveColumnStatistics columnStatistics) + { + columnStatistics.getMaxValueSizeInBytes().ifPresent(maxValueSizeInBytes -> + checkStatistics(maxValueSizeInBytes >= 0, table, partition, column, "maxValueSizeInBytes must be greater than or equal to zero: %s", maxValueSizeInBytes)); + columnStatistics.getTotalSizeInBytes().ifPresent(totalSizeInBytes -> + checkStatistics(totalSizeInBytes >= 0, table, partition, column, "totalSizeInBytes must be greater than or equal to zero: %s", totalSizeInBytes)); + columnStatistics.getNullsCount().ifPresent(nullsCount -> { + checkStatistics(nullsCount >= 0, table, partition, column, "nullsCount must be greater than or equal to zero: %s", nullsCount); + if (rowCount.isPresent()) { + checkStatistics( + nullsCount <= rowCount.getAsLong(), + table, + partition, + column, + "nullsCount must be less than or equal to rowCount. nullsCount: %s. rowCount: %s.", + nullsCount, + rowCount.getAsLong()); + } + }); + columnStatistics.getDistinctValuesCount().ifPresent(distinctValuesCount -> { + checkStatistics(distinctValuesCount >= 0, table, partition, column, "distinctValuesCount must be greater than or equal to zero: %s", distinctValuesCount); + if (rowCount.isPresent()) { + checkStatistics( + distinctValuesCount <= rowCount.getAsLong(), + table, + partition, + column, + "distinctValuesCount must be less than or equal to rowCount. distinctValuesCount: %s. rowCount: %s.", + distinctValuesCount, + rowCount.getAsLong()); + } + if (rowCount.isPresent() && columnStatistics.getNullsCount().isPresent()) { + long nonNullsCount = rowCount.getAsLong() - columnStatistics.getNullsCount().getAsLong(); + checkStatistics( + distinctValuesCount <= nonNullsCount, + table, + partition, + column, + "distinctValuesCount must be less than or equal to nonNullsCount. distinctValuesCount: %s. nonNullsCount: %s.", + distinctValuesCount, + nonNullsCount); + } + }); + + columnStatistics.getIntegerStatistics().ifPresent(integerStatistics -> { + OptionalLong min = integerStatistics.getMin(); + OptionalLong max = integerStatistics.getMax(); + if (min.isPresent() && max.isPresent()) { + checkStatistics( + min.getAsLong() <= max.getAsLong(), + table, + partition, + column, + "integerStatistics.min must be less than or equal to integerStatistics.max. integerStatistics.min: %s. integerStatistics.max: %s.", + min.getAsLong(), + max.getAsLong()); + } + }); + columnStatistics.getDoubleStatistics().ifPresent(doubleStatistics -> { + OptionalDouble min = doubleStatistics.getMin(); + OptionalDouble max = doubleStatistics.getMax(); + if (min.isPresent() && max.isPresent() && !isNaN(min.getAsDouble()) && !isNaN(max.getAsDouble())) { + checkStatistics( + min.getAsDouble() <= max.getAsDouble(), + table, + partition, + column, + "doubleStatistics.min must be less than or equal to doubleStatistics.max. doubleStatistics.min: %s. doubleStatistics.max: %s.", + min.getAsDouble(), + max.getAsDouble()); + } + }); + columnStatistics.getDecimalStatistics().ifPresent(decimalStatistics -> { + Optional min = decimalStatistics.getMin(); + Optional max = decimalStatistics.getMax(); + if (min.isPresent() && max.isPresent()) { + checkStatistics( + min.get().compareTo(max.get()) <= 0, + table, + partition, + column, + "decimalStatistics.min must be less than or equal to decimalStatistics.max. decimalStatistics.min: %s. decimalStatistics.max: %s.", + min.get(), + max.get()); + } + }); + columnStatistics.getDateStatistics().ifPresent(dateStatistics -> { + Optional min = dateStatistics.getMin(); + Optional max = dateStatistics.getMax(); + if (min.isPresent() && max.isPresent()) { + checkStatistics( + min.get().compareTo(max.get()) <= 0, + table, + partition, + column, + "dateStatistics.min must be less than or equal to dateStatistics.max. dateStatistics.min: %s. dateStatistics.max: %s.", + min.get(), + max.get()); + } + }); + columnStatistics.getBooleanStatistics().ifPresent(booleanStatistics -> { + OptionalLong falseCount = booleanStatistics.getFalseCount(); + OptionalLong trueCount = booleanStatistics.getTrueCount(); + falseCount.ifPresent(count -> + checkStatistics(count >= 0, table, partition, column, "falseCount must be greater than or equal to zero: %s", count)); + trueCount.ifPresent(count -> + checkStatistics(count >= 0, table, partition, column, "trueCount must be greater than or equal to zero: %s", count)); + if (rowCount.isPresent() && falseCount.isPresent()) { + checkStatistics( + falseCount.getAsLong() <= rowCount.getAsLong(), + table, + partition, + column, + "booleanStatistics.falseCount must be less than or equal to rowCount. booleanStatistics.falseCount: %s. rowCount: %s.", + falseCount.getAsLong(), + rowCount.getAsLong()); + } + if (rowCount.isPresent() && trueCount.isPresent()) { + checkStatistics( + trueCount.getAsLong() <= rowCount.getAsLong(), + table, + partition, + column, + "booleanStatistics.trueCount must be less than or equal to rowCount. booleanStatistics.trueCount: %s. rowCount: %s.", + trueCount.getAsLong(), + rowCount.getAsLong()); + } + }); + } + + private static void checkStatistics(boolean expression, SchemaTableName table, String partition, String column, String message, Object... args) + { + if (!expression) { + throw new TrinoException( + HIVE_CORRUPTED_COLUMN_STATISTICS, + format("Corrupted partition statistics (Table: %s Partition: [%s] Column: %s): %s", table, partition, column, format(message, args))); + } + } + + private static void checkStatistics(boolean expression, SchemaTableName table, String partition, String message, Object... args) + { + if (!expression) { + throw new TrinoException( + HIVE_CORRUPTED_COLUMN_STATISTICS, + format("Corrupted partition statistics (Table: %s Partition: [%s]): %s", table, partition, format(message, args))); + } + } + + private static TableStatistics getTableStatistics( + Map columns, + Map columnTypes, + List partitions, + Map statistics) + { + if (statistics.isEmpty()) { + return TableStatistics.empty(); + } + + checkArgument(!partitions.isEmpty(), "partitions is empty"); + + Optional optionalRowCount = calculatePartitionsRowCount(statistics.values(), partitions.size()); + if (optionalRowCount.isEmpty()) { + return TableStatistics.empty(); + } + double rowCount = optionalRowCount.get().getRowCount(); + + TableStatistics.Builder result = TableStatistics.builder(); + result.setRowCount(Estimate.of(rowCount)); + for (Map.Entry column : columns.entrySet()) { + String columnName = column.getKey(); + HiveColumnHandle columnHandle = (HiveColumnHandle) column.getValue(); + Type columnType = columnTypes.get(columnName); + ColumnStatistics columnStatistics; + if (columnHandle.isPartitionKey()) { + double averageRowsPerPartition = optionalRowCount.get().getAverageRowsPerPartition(); + columnStatistics = createPartitionColumnStatistics(columnHandle, columnType, partitions, statistics, averageRowsPerPartition, rowCount); + } + else { + columnStatistics = createDataColumnStatistics(columnName, columnType, rowCount, statistics.values()); + } + result.setColumnStatistics(columnHandle, columnStatistics); + } + return result.build(); + } + + @VisibleForTesting + static Optional calculatePartitionsRowCount(Collection statistics, int queriedPartitionsCount) + { + long[] rowCounts = statistics.stream() + .map(PartitionStatistics::getBasicStatistics) + .map(HiveBasicStatistics::getRowCount) + .filter(OptionalLong::isPresent) + .mapToLong(OptionalLong::getAsLong) + .peek(count -> verify(count >= 0, "count must be greater than or equal to zero")) + .toArray(); + int sampleSize = statistics.size(); + // Sample contains all the queried partitions, estimate avg normally + if (rowCounts.length <= 2 || queriedPartitionsCount == sampleSize) { + OptionalDouble averageRowsPerPartitionOptional = Arrays.stream(rowCounts).average(); + if (averageRowsPerPartitionOptional.isEmpty()) { + return Optional.empty(); + } + double averageRowsPerPartition = averageRowsPerPartitionOptional.getAsDouble(); + return Optional.of(new PartitionsRowCount(averageRowsPerPartition, averageRowsPerPartition * queriedPartitionsCount)); + } + + // Some partitions (e.g. __HIVE_DEFAULT_PARTITION__) may be outliers in terms of row count. + // Excluding the min and max rowCount values from averageRowsPerPartition calculation helps to reduce the + // possibility of errors in the extrapolated rowCount due to a couple of outliers. + int minIndex = 0; + int maxIndex = 0; + long rowCountSum = rowCounts[0]; + for (int index = 1; index < rowCounts.length; index++) { + if (rowCounts[index] < rowCounts[minIndex]) { + minIndex = index; + } + else if (rowCounts[index] > rowCounts[maxIndex]) { + maxIndex = index; + } + rowCountSum += rowCounts[index]; + } + double averageWithoutOutliers = ((double) (rowCountSum - rowCounts[minIndex] - rowCounts[maxIndex])) / (rowCounts.length - 2); + double rowCount = (averageWithoutOutliers * (queriedPartitionsCount - 2)) + rowCounts[minIndex] + rowCounts[maxIndex]; + return Optional.of(new PartitionsRowCount(averageWithoutOutliers, rowCount)); + } + + @VisibleForTesting + static class PartitionsRowCount + { + private final double averageRowsPerPartition; + private final double rowCount; + + PartitionsRowCount(double averageRowsPerPartition, double rowCount) + { + verify(averageRowsPerPartition >= 0, "averageRowsPerPartition must be greater than or equal to zero"); + verify(rowCount >= 0, "rowCount must be greater than or equal to zero"); + this.averageRowsPerPartition = averageRowsPerPartition; + this.rowCount = rowCount; + } + + private double getAverageRowsPerPartition() + { + return averageRowsPerPartition; + } + + private double getRowCount() + { + return rowCount; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionsRowCount that = (PartitionsRowCount) o; + return Double.compare(that.averageRowsPerPartition, averageRowsPerPartition) == 0 + && Double.compare(that.rowCount, rowCount) == 0; + } + + @Override + public int hashCode() + { + return Objects.hash(averageRowsPerPartition, rowCount); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("averageRowsPerPartition", averageRowsPerPartition) + .add("rowCount", rowCount) + .toString(); + } + } + + private static ColumnStatistics createPartitionColumnStatistics( + HiveColumnHandle column, + Type type, + List partitions, + Map statistics, + double averageRowsPerPartition, + double rowCount) + { + List nonEmptyPartitions = partitions.stream() + .filter(partition -> getPartitionRowCount(partition.getPartitionId(), statistics).orElse(averageRowsPerPartition) != 0) + .collect(toImmutableList()); + + return ColumnStatistics.builder() + .setDistinctValuesCount(Estimate.of(calculateDistinctPartitionKeys(column, nonEmptyPartitions))) + .setNullsFraction(Estimate.of(calculateNullsFractionForPartitioningKey(column, partitions, statistics, averageRowsPerPartition, rowCount))) + .setRange(calculateRangeForPartitioningKey(column, type, nonEmptyPartitions)) + .setDataSize(calculateDataSizeForPartitioningKey(column, type, partitions, statistics, averageRowsPerPartition)) + .build(); + } + + @VisibleForTesting + static long calculateDistinctPartitionKeys( + HiveColumnHandle column, + List partitions) + { + return partitions.stream() + .map(partition -> partition.getKeys().get(column)) + .filter(value -> !value.isNull()) + .distinct() + .count(); + } + + @VisibleForTesting + static double calculateNullsFractionForPartitioningKey( + HiveColumnHandle column, + List partitions, + Map statistics, + double averageRowsPerPartition, + double rowCount) + { + if (rowCount == 0) { + return 0; + } + double estimatedNullsCount = partitions.stream() + .filter(partition -> partition.getKeys().get(column).isNull()) + .map(HivePartition::getPartitionId) + .mapToDouble(partitionName -> getPartitionRowCount(partitionName, statistics).orElse(averageRowsPerPartition)) + .sum(); + return normalizeFraction(estimatedNullsCount / rowCount); + } + + private static double normalizeFraction(double fraction) + { + checkArgument(!isNaN(fraction), "fraction is NaN"); + checkArgument(isFinite(fraction), "fraction must be finite"); + if (fraction < 0) { + return 0; + } + if (fraction > 1) { + return 1; + } + return fraction; + } + + @VisibleForTesting + static Estimate calculateDataSizeForPartitioningKey( + HiveColumnHandle column, + Type type, + List partitions, + Map statistics, + double averageRowsPerPartition) + { + if (!hasDataSize(type)) { + return Estimate.unknown(); + } + double dataSize = 0; + for (HivePartition partition : partitions) { + int length = getSize(partition.getKeys().get(column)); + double rowCount = getPartitionRowCount(partition.getPartitionId(), statistics).orElse(averageRowsPerPartition); + dataSize += length * rowCount; + } + return Estimate.of(dataSize); + } + + private static boolean hasDataSize(Type type) + { + return type instanceof VarcharType || type instanceof CharType; + } + + private static int getSize(NullableValue nullableValue) + { + if (nullableValue.isNull()) { + return 0; + } + Object value = nullableValue.getValue(); + checkArgument(value instanceof Slice, "value is expected to be of Slice type"); + return ((Slice) value).length(); + } + + private static OptionalDouble getPartitionRowCount(String partitionName, Map statistics) + { + PartitionStatistics partitionStatistics = statistics.get(partitionName); + if (partitionStatistics == null) { + return OptionalDouble.empty(); + } + OptionalLong rowCount = partitionStatistics.getBasicStatistics().getRowCount(); + if (rowCount.isPresent()) { + verify(rowCount.getAsLong() >= 0, "rowCount must be greater than or equal to zero"); + return OptionalDouble.of(rowCount.getAsLong()); + } + return OptionalDouble.empty(); + } + + @VisibleForTesting + static Optional calculateRangeForPartitioningKey(HiveColumnHandle column, Type type, List partitions) + { + List convertedValues = partitions.stream() + .map(HivePartition::getKeys) + .map(keys -> keys.get(column)) + .filter(value -> !value.isNull()) + .map(NullableValue::getValue) + .map(value -> convertPartitionValueToDouble(type, value)) + .collect(toImmutableList()); + + if (convertedValues.stream().noneMatch(OptionalDouble::isPresent)) { + return Optional.empty(); + } + double[] values = convertedValues.stream() + .peek(convertedValue -> checkState(convertedValue.isPresent(), "convertedValue is missing")) + .mapToDouble(OptionalDouble::getAsDouble) + .toArray(); + verify(values.length != 0, "No values"); + + if (DoubleStream.of(values).anyMatch(Double::isNaN)) { + return Optional.empty(); + } + + double min = DoubleStream.of(values).min().orElseThrow(); + double max = DoubleStream.of(values).max().orElseThrow(); + return Optional.of(new DoubleRange(min, max)); + } + + @VisibleForTesting + static OptionalDouble convertPartitionValueToDouble(Type type, Object value) + { + return toStatsRepresentation(type, value); + } + + @VisibleForTesting + static ColumnStatistics createDataColumnStatistics(String column, Type type, double rowsCount, Collection partitionStatistics) + { + List columnStatistics = partitionStatistics.stream() + .map(PartitionStatistics::getColumnStatistics) + .map(statistics -> statistics.get(column)) + .filter(Objects::nonNull) + .collect(toImmutableList()); + + if (columnStatistics.isEmpty()) { + return ColumnStatistics.empty(); + } + + return ColumnStatistics.builder() + .setDistinctValuesCount(calculateDistinctValuesCount(columnStatistics)) + .setNullsFraction(calculateNullsFraction(column, partitionStatistics)) + .setDataSize(calculateDataSize(column, partitionStatistics, rowsCount)) + .setRange(calculateRange(type, columnStatistics)) + .build(); + } + + @VisibleForTesting + static Estimate calculateDistinctValuesCount(List columnStatistics) + { + return columnStatistics.stream() + .map(HudiHiveStatisticsProvider::getDistinctValuesCount) + .filter(OptionalLong::isPresent) + .map(OptionalLong::getAsLong) + .peek(distinctValuesCount -> verify(distinctValuesCount >= 0, "distinctValuesCount must be greater than or equal to zero")) + .max(Long::compare) + .map(Estimate::of) + .orElse(Estimate.unknown()); + } + + private static OptionalLong getDistinctValuesCount(HiveColumnStatistics statistics) + { + if (statistics.getBooleanStatistics().isPresent() && + statistics.getBooleanStatistics().get().getFalseCount().isPresent() && + statistics.getBooleanStatistics().get().getTrueCount().isPresent()) { + long falseCount = statistics.getBooleanStatistics().get().getFalseCount().getAsLong(); + long trueCount = statistics.getBooleanStatistics().get().getTrueCount().getAsLong(); + return OptionalLong.of((falseCount > 0 ? 1 : 0) + (trueCount > 0 ? 1 : 0)); + } + if (statistics.getDistinctValuesCount().isPresent()) { + return statistics.getDistinctValuesCount(); + } + return OptionalLong.empty(); + } + + @VisibleForTesting + static Estimate calculateNullsFraction(String column, Collection partitionStatistics) + { + List statisticsWithKnownRowCountAndNullsCount = partitionStatistics.stream() + .filter(statistics -> { + if (statistics.getBasicStatistics().getRowCount().isEmpty()) { + return false; + } + HiveColumnStatistics columnStatistics = statistics.getColumnStatistics().get(column); + if (columnStatistics == null) { + return false; + } + return columnStatistics.getNullsCount().isPresent(); + }) + .collect(toImmutableList()); + + if (statisticsWithKnownRowCountAndNullsCount.isEmpty()) { + return Estimate.unknown(); + } + + long totalNullsCount = 0; + long totalRowCount = 0; + for (PartitionStatistics statistics : statisticsWithKnownRowCountAndNullsCount) { + long rowCount = statistics.getBasicStatistics().getRowCount().orElseThrow(() -> new VerifyException("rowCount is not present")); + verify(rowCount >= 0, "rowCount must be greater than or equal to zero"); + HiveColumnStatistics columnStatistics = statistics.getColumnStatistics().get(column); + verifyNotNull(columnStatistics, "columnStatistics is null"); + long nullsCount = columnStatistics.getNullsCount().orElseThrow(() -> new VerifyException("nullsCount is not present")); + verify(nullsCount >= 0, "nullsCount must be greater than or equal to zero"); + verify(nullsCount <= rowCount, "nullsCount must be less than or equal to rowCount. nullsCount: %s. rowCount: %s.", nullsCount, rowCount); + totalNullsCount += nullsCount; + totalRowCount += rowCount; + } + + if (totalRowCount == 0) { + return Estimate.zero(); + } + + verify( + totalNullsCount <= totalRowCount, + "totalNullsCount must be less than or equal to totalRowCount. totalNullsCount: %s. totalRowCount: %s.", + totalNullsCount, + totalRowCount); + return Estimate.of(((double) totalNullsCount) / totalRowCount); + } + + @VisibleForTesting + static Estimate calculateDataSize(String column, Collection partitionStatistics, double totalRowCount) + { + List statisticsWithKnownRowCountAndDataSize = partitionStatistics.stream() + .filter(statistics -> { + if (statistics.getBasicStatistics().getRowCount().isEmpty()) { + return false; + } + HiveColumnStatistics columnStatistics = statistics.getColumnStatistics().get(column); + if (columnStatistics == null) { + return false; + } + return columnStatistics.getTotalSizeInBytes().isPresent(); + }) + .collect(toImmutableList()); + + if (statisticsWithKnownRowCountAndDataSize.isEmpty()) { + return Estimate.unknown(); + } + + long knownRowCount = 0; + long knownDataSize = 0; + for (PartitionStatistics statistics : statisticsWithKnownRowCountAndDataSize) { + long rowCount = statistics.getBasicStatistics().getRowCount().orElseThrow(() -> new VerifyException("rowCount is not present")); + verify(rowCount >= 0, "rowCount must be greater than or equal to zero"); + HiveColumnStatistics columnStatistics = statistics.getColumnStatistics().get(column); + verifyNotNull(columnStatistics, "columnStatistics is null"); + long dataSize = columnStatistics.getTotalSizeInBytes().orElseThrow(() -> new VerifyException("totalSizeInBytes is not present")); + verify(dataSize >= 0, "dataSize must be greater than or equal to zero"); + knownRowCount += rowCount; + knownDataSize += dataSize; + } + + if (totalRowCount == 0) { + return Estimate.zero(); + } + + if (knownRowCount == 0) { + return Estimate.unknown(); + } + + double averageValueDataSizeInBytes = ((double) knownDataSize) / knownRowCount; + return Estimate.of(averageValueDataSizeInBytes * totalRowCount); + } + + @VisibleForTesting + static Optional calculateRange(Type type, List columnStatistics) + { + return columnStatistics.stream() + .map(statistics -> createRange(type, statistics)) + .filter(Optional::isPresent) + .map(Optional::get) + .reduce(DoubleRange::union); + } + + private static Optional createRange(Type type, HiveColumnStatistics statistics) + { + if (type.equals(BIGINT) || type.equals(INTEGER) || type.equals(SMALLINT) || type.equals(TINYINT)) { + return statistics.getIntegerStatistics().flatMap(integerStatistics -> createIntegerRange(type, integerStatistics)); + } + if (type.equals(DOUBLE) || type.equals(REAL)) { + return statistics.getDoubleStatistics().flatMap(HudiHiveStatisticsProvider::createDoubleRange); + } + if (type.equals(DATE)) { + return statistics.getDateStatistics().flatMap(HudiHiveStatisticsProvider::createDateRange); + } + if (type instanceof DecimalType) { + return statistics.getDecimalStatistics().flatMap(HudiHiveStatisticsProvider::createDecimalRange); + } + return Optional.empty(); + } + + private static Optional createIntegerRange(Type type, IntegerStatistics statistics) + { + if (statistics.getMin().isPresent() && statistics.getMax().isPresent()) { + return Optional.of(createIntegerRange(type, statistics.getMin().getAsLong(), statistics.getMax().getAsLong())); + } + return Optional.empty(); + } + + private static DoubleRange createIntegerRange(Type type, long min, long max) + { + return new DoubleRange(normalizeIntegerValue(type, min), normalizeIntegerValue(type, max)); + } + + private static long normalizeIntegerValue(Type type, long value) + { + if (type.equals(BIGINT)) { + return value; + } + if (type.equals(INTEGER)) { + return Ints.saturatedCast(value); + } + if (type.equals(SMALLINT)) { + return Shorts.saturatedCast(value); + } + if (type.equals(TINYINT)) { + return SignedBytes.saturatedCast(value); + } + throw new IllegalArgumentException("Unexpected type: " + type); + } + + private static Optional createDoubleRange(DoubleStatistics statistics) + { + if (statistics.getMin().isPresent() && statistics.getMax().isPresent() && !isNaN(statistics.getMin().getAsDouble()) && !isNaN(statistics.getMax().getAsDouble())) { + return Optional.of(new DoubleRange(statistics.getMin().getAsDouble(), statistics.getMax().getAsDouble())); + } + return Optional.empty(); + } + + private static Optional createDateRange(DateStatistics statistics) + { + if (statistics.getMin().isPresent() && statistics.getMax().isPresent()) { + return Optional.of(new DoubleRange(statistics.getMin().get().toEpochDay(), statistics.getMax().get().toEpochDay())); + } + return Optional.empty(); + } + + private static Optional createDecimalRange(DecimalStatistics statistics) + { + if (statistics.getMin().isPresent() && statistics.getMax().isPresent()) { + return Optional.of(new DoubleRange(statistics.getMin().get().doubleValue(), statistics.getMax().get().doubleValue())); + } + return Optional.empty(); + } + + @VisibleForTesting + interface PartitionsStatisticsProvider + { + Map getPartitionsStatistics(ConnectorSession session, SchemaTableName table, List hivePartitions); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java index bd2d83fa762..f8b4cd4b326 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -22,6 +22,7 @@ import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.statistics.HiveStatisticsProvider; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; @@ -78,12 +79,21 @@ public class HudiMetadata private final HiveMetastore metastore; private final HdfsEnvironment hdfsEnvironment; private final TypeManager typeManager; + private final HudiPartitionManager partitionManager; + private final HiveStatisticsProvider hiveStatisticsProvider; - public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) + public HudiMetadata( + HiveMetastore metastore, + HdfsEnvironment hdfsEnvironment, + TypeManager typeManager, + HudiPartitionManager partitionManager, + HiveStatisticsProvider hiveStatisticsProvider) { this.metastore = requireNonNull(metastore, "metastore is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.partitionManager = requireNonNull(partitionManager, "partitionManager is null"); + this.hiveStatisticsProvider = requireNonNull(hiveStatisticsProvider, "hiveStatisticsProvider is null"); } @Override diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java index bb1a4832bc9..145e7c1eba8 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java @@ -16,6 +16,7 @@ import io.trino.hdfs.HdfsEnvironment; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.type.TypeManager; @@ -23,6 +24,7 @@ import java.util.Optional; +import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.memoizeMetastore; import static java.util.Objects.requireNonNull; public class HudiMetadataFactory @@ -30,18 +32,28 @@ public class HudiMetadataFactory private final HiveMetastoreFactory metastoreFactory; private final HdfsEnvironment hdfsEnvironment; private final TypeManager typeManager; + private final HudiPartitionManager partitionManager; @Inject - public HudiMetadataFactory(HiveMetastoreFactory metastoreFactory, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) + public HudiMetadataFactory( + HiveMetastoreFactory metastoreFactory, + HdfsEnvironment hdfsEnvironment, + TypeManager typeManager, + HudiPartitionManager partitionManager) { this.metastoreFactory = requireNonNull(metastoreFactory, "metastore is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.partitionManager = requireNonNull(partitionManager, "partitionManager is null"); } public HudiMetadata create(ConnectorIdentity identity) { HiveMetastore metastore = metastoreFactory.createMetastore(Optional.of(identity)); - return new HudiMetadata(metastore, hdfsEnvironment, typeManager); + // create per-transaction cache over hive metastore interface + CachingHiveMetastore cachingHiveMetastore = memoizeMetastore( + metastoreFactory.createMetastore(Optional.of(identity)), + 2000); + return new HudiMetadata(cachingHiveMetastore, hdfsEnvironment, typeManager, partitionManager, new HudiHiveStatisticsProvider(metastore)); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java index 52bd1f7028c..978961160d0 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java @@ -20,6 +20,7 @@ import com.google.inject.Scopes; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HiveNodePartitioningProvider; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -35,12 +36,15 @@ import javax.inject.Singleton; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.BiFunction; import static com.google.inject.multibindings.Multibinder.newSetBinder; import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.configuration.ConfigBinder.configBinder; import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.Executors.newFixedThreadPool; +import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.weakref.jmx.guice.ExportBinder.newExporter; public class HudiModule @@ -52,6 +56,7 @@ public void configure(Binder binder) binder.bind(HudiTransactionManager.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(HudiConfig.class); + configBinder(binder).bindConfig(HiveConfig.class); configBinder(binder).bindConfig(HiveMetastoreConfig.class); binder.bind(Key.get(boolean.class, TranslateHiveViews.class)).toInstance(false); @@ -65,6 +70,7 @@ public void configure(Binder binder) configBinder(binder).bindConfig(ParquetReaderConfig.class); configBinder(binder).bindConfig(ParquetWriterConfig.class); + binder.bind(HudiPartitionManager.class).in(Scopes.SINGLETON); binder.bind(HudiMetadataFactory.class).in(Scopes.SINGLETON); binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); @@ -76,7 +82,27 @@ public void configure(Binder binder) @Provides public ExecutorService createExecutorService() { - return newCachedThreadPool(daemonThreadsNamed("hudi-split-manager-%d")); + return newCachedThreadPool(daemonThreadsNamed("hudi-split-manager-%s")); + } + + @ForHudiSplitSource + @Singleton + @Provides + public ScheduledExecutorService createSplitLoaderExecutor(HudiConfig hudiConfig) + { + return newScheduledThreadPool( + hudiConfig.getSplitLoaderParallelism(), + daemonThreadsNamed("hudi-split-loader-%s")); + } + + @ForHudiBackgroundSplitLoader + @Singleton + @Provides + public ExecutorService createSplitGeneratorExecutor(HudiConfig hudiConfig) + { + return newFixedThreadPool( + hudiConfig.getSplitGeneratorParallelism(), + daemonThreadsNamed("hudi-split-generator-%s")); } @Singleton diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPartitionManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPartitionManager.java new file mode 100644 index 00000000000..ffeec25be95 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPartitionManager.java @@ -0,0 +1,64 @@ +/* + * Licensed 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 io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.type.TypeManager; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.base.Verify.verify; +import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles; +import static java.util.Objects.requireNonNull; + +public class HudiPartitionManager +{ + private final TypeManager typeManager; + + @Inject + public HudiPartitionManager(TypeManager typeManager) + { + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + public List getEffectivePartitions(HudiTableHandle tableHandle, HiveMetastore metastore) + { + Optional table = metastore.getTable(tableHandle.getSchemaName(), tableHandle.getTableName()); + verify(table.isPresent()); + List partitionColumns = table.get().getPartitionColumns(); + if (partitionColumns.isEmpty()) { + return ImmutableList.of(""); + } + + List partitionColumnHandles = getPartitionKeyColumnHandles(table.get(), typeManager); + + return metastore.getPartitionNamesByFilter( + tableHandle.getSchemaName(), + tableHandle.getTableName(), + partitionColumns.stream().map(Column::getName).collect(Collectors.toList()), + computePartitionKeyFilter(partitionColumnHandles, tableHandle.getPartitionPredicates())) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java index 355a5fd5180..a969c34674b 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -47,6 +47,11 @@ public class HudiSessionProperties private static final String SIZE_BASED_SPLIT_WEIGHTS_ENABLED = "size_based_split_weights_enabled"; private static final String STANDARD_SPLIT_WEIGHT_SIZE = "standard_split_weight_size"; private static final String MINIMUM_ASSIGNED_SPLIT_WEIGHT = "minimum_assigned_split_weight"; + private static final String MAX_SPLITS_PER_SECOND = "max_splits_per_second"; + private static final String MAX_OUTSTANDING_SPLITS = "max_outstanding_splits"; + private static final String TABLE_STATISTICS_ENABLED = "statistics_enabled"; + private static final String SPLIT_GENERATOR_PARALLELISM = "split_generator_parallelism"; + private static final String PARTITION_SCANNER_PARALLELISM = "partition_scanner_parallelism"; private final List> sessionProperties; @@ -104,6 +109,31 @@ public HudiSessionProperties(HudiConfig hudiConfig) throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s must be > 0 and <= 1.0: %s", MINIMUM_ASSIGNED_SPLIT_WEIGHT, value)); } }, + false), + integerProperty( + MAX_SPLITS_PER_SECOND, + "Rate at which splits are enqueued for processing. The queue will throttle if this rate limit is breached.", + hudiConfig.getMaxSplitsPerSecond(), + false), + integerProperty( + MAX_OUTSTANDING_SPLITS, + "Maximum outstanding splits in a batch enqueued for processing.", + hudiConfig.getMaxOutstandingSplits(), + false), + booleanProperty( + TABLE_STATISTICS_ENABLED, + "Expose table statistics", + hudiConfig.isTableStatisticsEnabled(), + false), + integerProperty( + SPLIT_GENERATOR_PARALLELISM, + "Number of threads to generate splits from partitions", + hudiConfig.getSplitGeneratorParallelism(), + false), + integerProperty( + PARTITION_SCANNER_PARALLELISM, + "Number of threads to use for partition scanners", + hudiConfig.getPartitionScannerParallelism(), false)); } @@ -153,4 +183,29 @@ public static double getMinimumAssignedSplitWeight(ConnectorSession session) { return session.getProperty(MINIMUM_ASSIGNED_SPLIT_WEIGHT, Double.class); } + + public static boolean isStatisticsEnabled(ConnectorSession session) + { + return session.getProperty(TABLE_STATISTICS_ENABLED, Boolean.class); + } + + public static int getMaxSplitsPerSecond(ConnectorSession session) + { + return session.getProperty(MAX_SPLITS_PER_SECOND, Integer.class); + } + + public static int getMaxOutstandingSplits(ConnectorSession session) + { + return session.getProperty(MAX_OUTSTANDING_SPLITS, Integer.class); + } + + public static int getSplitGeneratorParallelism(ConnectorSession session) + { + return session.getProperty(SPLIT_GENERATOR_PARALLELISM, Integer.class); + } + + public static int getPartitionScannerParallelism(ConnectorSession session) + { + return session.getProperty(PARTITION_SCANNER_PARALLELISM, Integer.class); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java index 0081e35fa7d..e5abee081f7 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java @@ -23,16 +23,21 @@ import io.trino.spi.SplitWeight; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.predicate.TupleDomain; +import org.openjdk.jol.info.ClassLayout; import java.util.List; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; public class HudiSplit implements ConnectorSplit { + private static final int INSTANCE_SIZE = toIntExact(ClassLayout.parseClass(HudiSplit.class).instanceSize()); + private final String path; private final long start; private final long length; @@ -144,6 +149,17 @@ public List getPartitionKeys() return partitionKeys; } + @Override + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + estimatedSizeOf(path) + + estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes) + + splitWeight.getRetainedSizeInBytes() + + predicate.getRetainedSizeInBytes(HiveColumnHandle::getRetainedSizeInBytes) + + estimatedSizeOf(partitionKeys, HivePartitionKey::getEstimatedSizeInBytes); + } + @Override public String toString() { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 34afd7e042f..0ab251a210e 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hudi; +import io.airlift.log.Logger; import io.trino.hdfs.HdfsContext; import io.trino.hdfs.HdfsEnvironment; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; @@ -30,15 +31,19 @@ import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.security.ConnectorIdentity; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.util.HoodieTimer; -import javax.annotation.PreDestroy; import javax.inject.Inject; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.BiFunction; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.plugin.hudi.HudiSessionProperties.getMaxOutstandingSplits; +import static io.trino.plugin.hudi.HudiSessionProperties.getMaxSplitsPerSecond; import static io.trino.spi.connector.SchemaTableName.schemaTableName; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; @@ -46,33 +51,33 @@ public class HudiSplitManager implements ConnectorSplitManager { + private static final Logger log = Logger.get(HudiSplitManager.class); + private final HudiTransactionManager transactionManager; + private final HudiPartitionManager partitionManager; private final BiFunction metastoreProvider; private final HdfsEnvironment hdfsEnvironment; private final ExecutorService executor; - private final int maxSplitsPerSecond; - private final int maxOutstandingSplits; + private final ScheduledExecutorService splitLoaderExecutorService; + private final ExecutorService splitGeneratorExecutorService; @Inject public HudiSplitManager( HudiTransactionManager transactionManager, + HudiPartitionManager partitionManager, BiFunction metastoreProvider, HdfsEnvironment hdfsEnvironment, @ForHudiSplitManager ExecutorService executor, - HudiConfig hudiConfig) + @ForHudiSplitSource ScheduledExecutorService splitLoaderExecutorService, + @ForHudiBackgroundSplitLoader ExecutorService splitGeneratorExecutorService) { this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.partitionManager = requireNonNull(partitionManager, "partitionManager is null"); this.metastoreProvider = requireNonNull(metastoreProvider, "metastoreProvider is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.executor = requireNonNull(executor, "executor is null"); - this.maxSplitsPerSecond = requireNonNull(hudiConfig, "hudiConfig is null").getMaxSplitsPerSecond(); - this.maxOutstandingSplits = hudiConfig.getMaxOutstandingSplits(); - } - - @PreDestroy - public void destroy() - { - this.executor.shutdown(); + this.splitLoaderExecutorService = requireNonNull(splitLoaderExecutorService, "splitLoaderExecutorService is null"); + this.splitGeneratorExecutorService = requireNonNull(splitGeneratorExecutorService, "splitGeneratorExecutorService is null"); } @Override @@ -89,9 +94,14 @@ public ConnectorSplitSource getSplits( .values().stream().map(HiveColumnHandle.class::cast) .filter(HiveColumnHandle::isPartitionKey) .collect(toImmutableMap(HiveColumnHandle::getName, identity())); - HiveMetastore metastore = metastoreProvider.apply(session.getIdentity(), (HiveTransactionHandle) transaction); + HiveMetastore metastore = hudiMetadata.getMetastore(); Table table = metastore.getTable(hudiTableHandle.getSchemaName(), hudiTableHandle.getTableName()) .orElseThrow(() -> new TableNotFoundException(schemaTableName(hudiTableHandle.getSchemaName(), hudiTableHandle.getTableName()))); + + HoodieTimer timer = new HoodieTimer().startTimer(); + List partitions = partitionManager.getEffectivePartitions(hudiTableHandle, metastore); + log.info("Took %d ms to get %d partitions", timer.endTimer(), partitions.size()); + HudiSplitSource splitSource = new HudiSplitSource( session, metastore, @@ -100,8 +110,11 @@ public ConnectorSplitSource getSplits( hdfsEnvironment.getConfiguration(new HdfsContext(session), new Path(table.getStorage().getLocation())), partitionColumnHandles, executor, - maxSplitsPerSecond, - maxOutstandingSplits); + splitLoaderExecutorService, + splitGeneratorExecutorService, + getMaxSplitsPerSecond(session), + getMaxOutstandingSplits(session), + partitions); return new ClassLoaderSafeConnectorSplitSource(splitSource, HudiSplitManager.class.getClassLoader()); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 2726bbb741d..db169ff0172 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -14,6 +14,7 @@ package io.trino.plugin.hudi; import com.google.common.util.concurrent.Futures; +import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -39,6 +40,9 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; @@ -53,7 +57,10 @@ public class HudiSplitSource implements ConnectorSplitSource { + private static final Logger log = Logger.get(HudiSplitSource.class); + private final AsyncQueue queue; + private final ScheduledFuture splitLoaderFuture; private final AtomicReference trinoException = new AtomicReference<>(); public HudiSplitSource( @@ -64,8 +71,11 @@ public HudiSplitSource( Configuration configuration, Map partitionColumnHandleMap, ExecutorService executor, + ScheduledExecutorService splitLoaderExecutorService, + ExecutorService splitGeneratorExecutorService, int maxSplitsPerSecond, - int maxOutstandingSplits) + int maxOutstandingSplits, + List partitions) { boolean metadataEnabled = isHudiMetadataEnabled(session); HoodieTableMetaClient metaClient = buildTableMetaClient(configuration, tableHandle.getBasePath()); @@ -83,7 +93,8 @@ public HudiSplitSource( metaClient, metastore, table, - partitionColumnHandles); + partitionColumnHandles, + partitions); this.queue = new ThrottledAsyncQueue<>(maxSplitsPerSecond, maxOutstandingSplits, executor); HudiBackgroundSplitLoader splitLoader = new HudiBackgroundSplitLoader( @@ -91,14 +102,16 @@ public HudiSplitSource( tableHandle, hudiDirectoryLister, queue, - executor, + splitGeneratorExecutorService, createSplitWeightProvider(session), throwable -> { trinoException.compareAndSet(null, new TrinoException(GENERIC_INTERNAL_ERROR, "Failed to generate splits for " + table.getTableName(), throwable)); queue.finish(); - }); - splitLoader.start(); + }, + partitions); + // splitLoader.start(); + this.splitLoaderFuture = splitLoaderExecutorService.schedule(splitLoader, 0, TimeUnit.MILLISECONDS); } @Override @@ -125,7 +138,7 @@ public void close() @Override public boolean isFinished() { - return queue.isFinished(); + return splitLoaderFuture.isDone() && queue.isFinished(); } private static HoodieTableMetaClient buildTableMetaClient(Configuration configuration, String basePath) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java index cf3596092da..a5e314dba6f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java @@ -69,7 +69,7 @@ public HiveHudiPartitionInfo( @Override public Table getTable() { - return null; + return table; } @Override diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java index bf501a777d3..0538323cd4e 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HudiPartitionInfoLoader.java @@ -13,20 +13,20 @@ */ package io.trino.plugin.hudi.partition; -import io.trino.plugin.hive.metastore.Partition; +import io.airlift.concurrent.MoreFutures; +import io.airlift.log.Logger; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.util.AsyncQueue; import io.trino.plugin.hudi.query.HudiDirectoryLister; +import io.trino.plugin.hudi.split.HudiSplitFactory; import io.trino.spi.connector.ConnectorSession; -import org.apache.hudi.exception.HoodieIOException; +import io.trino.spi.connector.ConnectorSplit; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hudi.common.util.HoodieTimer; -import java.util.ArrayList; -import java.util.Comparator; import java.util.Deque; -import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Optional; -import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.stream.Collectors; import static io.trino.plugin.hudi.HudiSessionProperties.getMaxPartitionBatchSize; import static io.trino.plugin.hudi.HudiSessionProperties.getMinPartitionBatchSize; @@ -34,89 +34,64 @@ public class HudiPartitionInfoLoader implements Runnable { + private static final Logger log = Logger.get(HudiPartitionInfoLoader.class); + private final HudiDirectoryLister hudiDirectoryLister; + private final HudiSplitFactory hudiSplitFactory; private final int minPartitionBatchSize; private final int maxPartitionBatchSize; - private final Deque partitionQueue; + private final AsyncQueue asyncQueue; + private final Deque partitionQueue; private int currentBatchSize; + private boolean isRunning; public HudiPartitionInfoLoader( ConnectorSession session, - HudiDirectoryLister hudiDirectoryLister) + HudiDirectoryLister hudiDirectoryLister, + HudiSplitFactory hudiSplitFactory, + AsyncQueue asyncQueue, + Deque partitionQueue) { this.hudiDirectoryLister = hudiDirectoryLister; - this.partitionQueue = new ConcurrentLinkedDeque<>(); + this.hudiSplitFactory = hudiSplitFactory; + this.asyncQueue = asyncQueue; + this.partitionQueue = partitionQueue; this.minPartitionBatchSize = getMinPartitionBatchSize(session); this.maxPartitionBatchSize = getMaxPartitionBatchSize(session); this.currentBatchSize = -1; + this.isRunning = true; } @Override public void run() { - List hudiPartitionInfoList = hudiDirectoryLister.getPartitionsToScan().stream() - .sorted(Comparator.comparing(HudiPartitionInfo::getComparingKey)) - .collect(Collectors.toList()); + HoodieTimer timer = new HoodieTimer().startTimer(); - // empty partitioned table - if (hudiPartitionInfoList.isEmpty()) { - return; - } + while (isRunning || !partitionQueue.isEmpty()) { + String partitionName = partitionQueue.poll(); - // non-partitioned table - if (hudiPartitionInfoList.size() == 1 && hudiPartitionInfoList.get(0).getHivePartitionName().isEmpty()) { - partitionQueue.addAll(hudiPartitionInfoList); - return; - } - - boolean shouldUseHiveMetastore = hudiPartitionInfoList.get(0) instanceof HiveHudiPartitionInfo; - Iterator iterator = hudiPartitionInfoList.iterator(); - while (iterator.hasNext()) { - int batchSize = updateBatchSize(); - List partitionInfoBatch = new ArrayList<>(); - while (iterator.hasNext() && batchSize > 0) { - partitionInfoBatch.add(iterator.next()); - batchSize--; - } - - if (!partitionInfoBatch.isEmpty()) { - if (shouldUseHiveMetastore) { - Map> partitions = hudiDirectoryLister.getPartitions(partitionInfoBatch.stream() - .map(HudiPartitionInfo::getHivePartitionName) - .collect(Collectors.toList())); - for (HudiPartitionInfo partitionInfo : partitionInfoBatch) { - String hivePartitionName = partitionInfo.getHivePartitionName(); - if (!partitions.containsKey(hivePartitionName)) { - throw new HoodieIOException("Partition does not exist: " + hivePartitionName); - } - partitionInfo.loadPartitionInfo(partitions.get(hivePartitionName)); - partitionQueue.add(partitionInfo); - } - } - else { - for (HudiPartitionInfo partitionInfo : partitionInfoBatch) { - partitionInfo.getHivePartitionKeys(); - partitionQueue.add(partitionInfo); - } - } + if (partitionName != null) { + generateSplitsFromPartition(partitionName); } } + log.debug("HudiPartitionInfoLoader %s finishes in %d ms", this, timer.endTimer()); } - public Deque getPartitionQueue() + private void generateSplitsFromPartition(String partitionName) { - return partitionQueue; + Optional partitionInfo = hudiDirectoryLister.getPartitionInfo(partitionName); + if (partitionInfo.isPresent()) { + List partitionKeys = partitionInfo.get().getHivePartitionKeys(); + List partitionFiles = hudiDirectoryLister.listStatus(partitionInfo.get()); + partitionFiles.stream() + .flatMap(fileStatus -> hudiSplitFactory.createSplits(partitionKeys, fileStatus)) + .map(asyncQueue::offer) + .forEachOrdered(MoreFutures::getFutureValue); + } } - private int updateBatchSize() + public void stopRunning() { - if (currentBatchSize <= 0) { - currentBatchSize = minPartitionBatchSize; - } - else { - currentBatchSize *= 2; - currentBatchSize = Math.min(currentBatchSize, maxPartitionBatchSize); - } - return currentBatchSize; + this.isRunning = false; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java index 401e0f35e84..8a40daf37db 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiDirectoryLister.java @@ -30,4 +30,6 @@ public interface HudiDirectoryLister List listStatus(HudiPartitionInfo partitionInfo); Map> getPartitions(List partitionNames); + + Optional getPartitionInfo(String partition); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiReadOptimizedDirectoryLister.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiReadOptimizedDirectoryLister.java index 92aad499ce6..5b4e266fd67 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiReadOptimizedDirectoryLister.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/query/HudiReadOptimizedDirectoryLister.java @@ -16,15 +16,12 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.MetastoreUtil; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hudi.HudiTableHandle; import io.trino.plugin.hudi.partition.HiveHudiPartitionInfo; import io.trino.plugin.hudi.partition.HudiPartitionInfo; import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.fs.FileStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -32,7 +29,6 @@ import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -48,12 +44,9 @@ public class HudiReadOptimizedDirectoryLister private final HiveMetastore hiveMetastore; private final Table hiveTable; private final SchemaTableName tableName; - private final List partitionColumnHandles; private final HoodieTableFileSystemView fileSystemView; - private final TupleDomain partitionKeysFilter; private final List partitionColumns; - - private List hivePartitionNames; + private final List allPartitionInfoList; public HudiReadOptimizedDirectoryLister( HoodieMetadataConfig metadataConfig, @@ -62,28 +55,16 @@ public HudiReadOptimizedDirectoryLister( HoodieTableMetaClient metaClient, HiveMetastore hiveMetastore, Table hiveTable, - List partitionColumnHandles) + List partitionColumnHandles, + List hivePartitionNames) { this.tableHandle = tableHandle; this.tableName = tableHandle.getSchemaTableName(); this.hiveMetastore = hiveMetastore; this.hiveTable = hiveTable; - this.partitionColumnHandles = partitionColumnHandles; this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - this.partitionKeysFilter = MetastoreUtil.computePartitionKeyFilter(partitionColumnHandles, tableHandle.getPartitionPredicates()); this.partitionColumns = hiveTable.getPartitionColumns(); - } - - @Override - public List getPartitionsToScan() - { - if (hivePartitionNames == null) { - hivePartitionNames = partitionColumns.isEmpty() - ? Collections.singletonList("") - : getPartitionNamesFromHiveMetastore(partitionKeysFilter); - } - - List allPartitionInfoList = hivePartitionNames.stream() + this.allPartitionInfoList = hivePartitionNames.stream() .map(hivePartitionName -> new HiveHudiPartitionInfo( hivePartitionName, partitionColumns, @@ -92,7 +73,11 @@ public List getPartitionsToScan() hiveTable, hiveMetastore)) .collect(Collectors.toList()); + } + @Override + public List getPartitionsToScan() + { return allPartitionInfoList.stream() .filter(partitionInfo -> partitionInfo.getHivePartitionKeys().isEmpty() || partitionInfo.doesMatchPredicates()) .collect(Collectors.toList()); @@ -106,19 +91,18 @@ public List listStatus(HudiPartitionInfo partitionInfo) .collect(toImmutableList()); } - private List getPartitionNamesFromHiveMetastore(TupleDomain partitionKeysFilter) + @Override + public Map> getPartitions(List partitionNames) { - return hiveMetastore.getPartitionNamesByFilter( - tableName.getSchemaName(), - tableName.getTableName(), - partitionColumns.stream().map(Column::getName).collect(Collectors.toList()), - partitionKeysFilter).orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); + return hiveMetastore.getPartitionsByNames(hiveTable, partitionNames); } @Override - public Map> getPartitions(List partitionNames) + public Optional getPartitionInfo(String partition) { - return hiveMetastore.getPartitionsByNames(hiveTable, partitionNames); + return allPartitionInfoList.stream() + .filter(partitionInfo -> partition.equals(partitionInfo.getHivePartitionName())) + .findFirst(); } @Override diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java index b9ca3cbe60d..7a968050f82 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/split/HudiBackgroundSplitLoader.java @@ -13,103 +13,92 @@ */ package io.trino.plugin.hudi.split; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import io.airlift.concurrent.MoreFutures; -import io.trino.plugin.hive.HivePartitionKey; +import io.airlift.log.Logger; import io.trino.plugin.hive.util.AsyncQueue; import io.trino.plugin.hudi.HudiTableHandle; -import io.trino.plugin.hudi.partition.HudiPartitionInfo; import io.trino.plugin.hudi.partition.HudiPartitionInfoLoader; import io.trino.plugin.hudi.query.HudiDirectoryLister; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.exception.HoodieException; -import java.util.Collection; +import java.util.ArrayList; +import java.util.Deque; import java.util.List; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.function.Consumer; -import java.util.stream.Collectors; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.trino.plugin.hudi.HudiSessionProperties.getSplitGeneratorParallelism; import static java.util.Objects.requireNonNull; public class HudiBackgroundSplitLoader + implements Runnable { + private static final Logger log = Logger.get(HudiBackgroundSplitLoader.class); + private final ConnectorSession session; private final HudiDirectoryLister hudiDirectoryLister; private final AsyncQueue asyncQueue; - private final ExecutorService executor; + private final ExecutorService splitGeneratorExecutorService; + private final int splitGeneratorNumThreads; private final Consumer errorListener; private final HudiSplitFactory hudiSplitFactory; + private final List partitions; public HudiBackgroundSplitLoader( ConnectorSession session, HudiTableHandle tableHandle, HudiDirectoryLister hudiDirectoryLister, AsyncQueue asyncQueue, - ExecutorService executor, + ExecutorService splitGeneratorExecutorService, HudiSplitWeightProvider hudiSplitWeightProvider, - Consumer errorListener) + Consumer errorListener, + List partitions) { this.session = requireNonNull(session, "session is null"); this.hudiDirectoryLister = requireNonNull(hudiDirectoryLister, "hudiDirectoryLister is null"); this.asyncQueue = requireNonNull(asyncQueue, "asyncQueue is null"); - this.executor = requireNonNull(executor, "executor is null"); + this.splitGeneratorExecutorService = requireNonNull(splitGeneratorExecutorService, "splitGeneratorExecutorService is null"); + this.splitGeneratorNumThreads = getSplitGeneratorParallelism(session); this.errorListener = requireNonNull(errorListener, "errorListener is null"); this.hudiSplitFactory = new HudiSplitFactory(tableHandle, hudiSplitWeightProvider); + this.partitions = requireNonNull(partitions, "partitions is null"); } - public void start() - { - ListenableFuture> partitionsFuture = Futures.submit(this::loadPartitions, executor); - hookErrorListener(partitionsFuture); - - ListenableFuture splitFutures = Futures.transform( - partitionsFuture, - partitions -> { - List> futures = partitions.stream() - .map(partition -> Futures.submit(() -> loadSplits(partition), executor)) - .peek(this::hookErrorListener) - .collect(Collectors.toList()); - Futures.whenAllComplete(futures).run(asyncQueue::finish, directExecutor()); - return null; - }, - directExecutor()); - hookErrorListener(splitFutures); - } - - private Collection loadPartitions() + @Override + public void run() { - HudiPartitionInfoLoader partitionInfoLoader = new HudiPartitionInfoLoader(session, hudiDirectoryLister); - partitionInfoLoader.run(); - return partitionInfoLoader.getPartitionQueue(); - } + HoodieTimer timer = new HoodieTimer().startTimer(); + Deque partitionQueue = new ConcurrentLinkedDeque<>(partitions); + List splitGeneratorList = new ArrayList<>(); + List splitGeneratorFutures = new ArrayList<>(); - private void loadSplits(HudiPartitionInfo partition) - { - List partitionKeys = partition.getHivePartitionKeys(); - List partitionFiles = hudiDirectoryLister.listStatus(partition); - partitionFiles.stream() - .flatMap(fileStatus -> hudiSplitFactory.createSplits(partitionKeys, fileStatus)) - .map(asyncQueue::offer) - .forEachOrdered(MoreFutures::getFutureValue); - } + // Start a number of partition split generators to generate the splits in parallel + for (int i = 0; i < splitGeneratorNumThreads; i++) { + HudiPartitionInfoLoader generator = new HudiPartitionInfoLoader(session, hudiDirectoryLister, hudiSplitFactory, asyncQueue, partitionQueue); + splitGeneratorList.add(generator); + splitGeneratorFutures.add(splitGeneratorExecutorService.submit(generator)); + } - private void hookErrorListener(ListenableFuture future) - { - Futures.addCallback(future, new FutureCallback() - { - @Override - public void onSuccess(T result) {} + for (HudiPartitionInfoLoader generator : splitGeneratorList) { + // Let the split generator stop once the partition queue is empty + generator.stopRunning(); + } - @Override - public void onFailure(Throwable t) - { - errorListener.accept(t); + // Wait for all split generators to finish + for (Future future : splitGeneratorFutures) { + try { + future.get(); + } + catch (InterruptedException | ExecutionException e) { + throw new HoodieException("Error generating Hudi split", e); } - }, directExecutor()); + } + asyncQueue.finish(); + log.debug("Finish getting all splits in %d ms", timer.endTimer()); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java index f00491943ff..843e1a7fb34 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java @@ -49,6 +49,13 @@ public void testReadNonPartitionedTable() "SELECT * FROM VALUES ('row_1', 'bob'), ('row_2', 'john'), ('row_3', 'tom')"); } + @Test + public void testTopN() + { + // remove this test or make it work with tpch tables + assertQueryOrdered("SELECT n.symbol, r.symbol FROM stock_ticks_cow n LEFT JOIN stock_ticks_cow_column_stats r ON n.key = r.key ORDER BY n.symbol LIMIT 1"); + } + @Test public void testReadPartitionedTables() { diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java index 8d1cda74e28..affa66e14a3 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java @@ -155,6 +155,8 @@ public enum TestingTable HUDI_NON_PART_COW(COPY_ON_WRITE, nonPartitionRegularColumns()), HUDI_COW_PT_TBL(COPY_ON_WRITE, multiPartitionRegularColumns(), multiPartitionColumns(), multiPartitions()), STOCK_TICKS_COW(COPY_ON_WRITE, stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions()), + // NOTE: remove this table, it is only for local testing purpose of column stats + STOCK_TICKS_COW_COLUMN_STATS(COPY_ON_WRITE, stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions()), STOCK_TICKS_MOR(MERGE_ON_READ, stockTicksRegularColumns(), stockTicksPartitionColumns(), stockTicksPartitions()), /**/; diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java index 2a6c5f44b22..85bf391b3aa 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java @@ -31,9 +31,6 @@ import io.trino.tpch.TpchColumnType; import io.trino.tpch.TpchColumnTypes; import io.trino.tpch.TpchTable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.common.HoodieJavaEngineContext; @@ -50,6 +47,10 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.org.apache.avro.Schema; +import org.apache.hudi.org.apache.avro.Schema.Field; +import org.apache.hudi.org.apache.avro.generic.GenericData; +import org.apache.hudi.org.apache.avro.generic.GenericRecord; import org.intellij.lang.annotations.Language; import java.io.IOException; @@ -77,6 +78,7 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toUnmodifiableList; import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; +import static org.apache.hudi.org.apache.avro.Schema.Type.STRING; public class TpchHudiTablesInitializer implements HudiTablesInitializer @@ -263,14 +265,15 @@ public HoodieRecord newInstance() private static Schema createAvroSchema(TpchTable table) { List> tpchColumns = table.getColumns(); - List fields = new ArrayList<>(tpchColumns.size() + 1); + List fields = new ArrayList<>(tpchColumns.size() + 1); for (TpchColumn column : tpchColumns) { String columnName = column.getSimplifiedColumnName(); Schema.Type columnSchemaType = toSchemaType(column.getType()); + // NOTE: comment for now due to shading // Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(type)); - fields.add(new Schema.Field(columnName, Schema.create(columnSchemaType))); + //fields.add(new Field(columnName, create(columnSchemaType))); } - fields.add(new Schema.Field(FIELD_UUID, Schema.create(Schema.Type.STRING))); + //fields.add(new Field(FIELD_UUID, create(STRING))); String name = table.getTableName(); return Schema.createRecord(name, null, null, false, fields); } @@ -308,7 +311,7 @@ private enum TpchColumnTypeAdapter IDENTIFIER(Schema.Type.LONG, hiveTypeOf(HIVE_LONG), Function.identity()), DATE(Schema.Type.INT, hiveTypeOf(HIVE_DATE), TpchColumnTypeAdapter::convertDate), DOUBLE(Schema.Type.DOUBLE, hiveTypeOf(HIVE_DOUBLE), Function.identity()), - VARCHAR(Schema.Type.STRING, TpchColumnTypeAdapter::hiveVarcharOf, Function.identity()), + VARCHAR(STRING, TpchColumnTypeAdapter::hiveVarcharOf, Function.identity()), /**/; static TpchColumnTypeAdapter of(TpchColumnType columnType) diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit new file mode 100644 index 00000000000..9e93d46cc4b --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit @@ -0,0 +1,74 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "7196b21b-34c1-4e78-bf20-72dceb524e9e-0", + "path" : "2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet", + "prevCommit" : "null", + "numWrites" : 197, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 197, + "totalWriteBytes" : 443915, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 443915, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:3482" + }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "7196b21b-34c1-4e78-bf20-72dceb524e9e-0", + "path" : "2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet", + "prevCommit" : "null", + "numWrites" : 197, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 197, + "totalWriteBytes" : 443915, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 443915, + "minEventTime" : null, + "maxEventTime" : null + } ], + "fileIdAndRelativePaths" : { + "7196b21b-34c1-4e78-bf20-72dceb524e9e-0" : "2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 765, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.inflight new file mode 100644 index 00000000000..ceacf6654e7 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130009989.inflight @@ -0,0 +1,71 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 197, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 197, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing new file mode 100644 index 0000000000000000000000000000000000000000..b8f51e5350effa0fb981bbc2cb1f96fdde660fcf GIT binary patch literal 979 zcmb_bv2NQi5Op*b$Ql&sB5*bY*-g>dW80w~iW;pu8AfmsPZk4`Cy-|&LEyPlzMxZn zDPzB&pHWhll?HK&A}F9i;vL_+_wMQQ`NctU1Zzi`FDf+f>+o9eRgGdS92VrH=nD%b zet#sA3MW?L<$E$l@h4=Ez|gk%8j9FQZka)NuQ01tQe_RgA&ELPmI}4>Vm!az!LHGH zMK;v7ymO%}N63}dV$m0piU2yRluw(>sl!?aCZBSJ?NO;T`Wd~$52%q*PSH)lvEnVT z#1dAT#dxskcf3@igV->Oa_`w2-q6;%;Mpm=4E09F1*{=!kH+Hbwzm1tQ^pDY7*s{a zJ2?|ul7D(!a1O3QSuvVmb0_03%X|PGu0CT$`}{R+Iy(3%aP{=y|LXIOYM)8pYNmpag#qtnr&tG@xi CP*1)9 literal 0 HcmV?d00001 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.inflight new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/20221014130117650.indexing.requested new file mode 100644 index 0000000000000000000000000000000000000000..16883161bf9216f3f828310c54da2725e6950228 GIT binary patch literal 659 zcmb_YyH3L}6fK^Z*g!~Zo~tMc3^X$XorqL1pem|vW4CeTM`hnsR8=Itgsr=*OP)ejpRP}cx(RhPJfu1|mGvoFI8mj-?#s_O*?O}mjdF$Q$x3%`AJ@ubrD~`Yb zA?@T-z~blN4z190k2e)AU$%hJ(TF;E4GRz?528V;_Y;DU%7$QV{NLPSuZh-@N}2-JY66vB{9k|{|h&P)K;i&YBUXq1$oa)^{5P>W?K5RQ-{V1fumq(Xw}9I~hd0$)O7;0rLB z3?=y_3V^XRcteWE$ORf5)%YR zU`i6wii?0?xsYCnXniiJ(@r2{Yw#XY43hy?NCa=6^0v8cWlFnoFR-F|!x7X%)JL!q z*fiLJlmNrt_rhBu(z>-HN^Ef!4Wwl$${ZvBfvfedsc=J(u#8fW+GYP}A&wjvk&}?t z926}9E~UM!Re3bu?|S_EpiY!32cEggbM96LWD$8Sj+@!)C#Ue7Z67-L}I8@(At+k&lnEDQYl2K zOQShW9#9g%*Q6!~d@OQ%Vp63{&Ld%xXjw^1L`x>ndZ%p#wlbw9?~MWl^gL6^}p8-MjwtPzTCj9BdxJwNBHP7SYrLPy;}khzilinZ!Ws1Tha#tNsXt zCWKHDinasFa|fl0H30?5L0|+H-;XeK&d_3HwghV`0)JF!968xONRaV510a=1+XTl2 zsj7W+R6^?vcyyv{6Q8#fgF8TGaVV7}a^HO<$|a!uBz$qi839)E?oE#OXG-0#Zi@6hV!Gq?i~Lz|v0G4qZmK4HFTd zGFaL^+NriPb5bss8J0|22{0TKa9PyyO`0vj}XfIHB2j(1$r=1v1a z*)|gu;8+LF05x`;j2`^I*s!KsczeH8U8uppY=38TLHc*LM^EVdJpw0!P6#+gThz=% z<)}<4qY6i=uTeMsI4l<&MNL?MIjDX=QU=N3m=Bj3%=gNPx6(HEtu*;gQhGN5yc25Q4Qp@5 z2F=NOa|A!6RO-7A`1lR0aLgxT71Tn1k5k%}o|uyE+0`vkOW~{bzZPa7MuI>mpoUVt z06mfJRX*Y^>Z0bK7^dd6Z=IyjP~EwV@ji~2g$)D z6$n@a5O&8`Qns1}dXB%fYzNvnP32A>I_Mv^aqqVv|EQGW`kk&BCp9pd!r zi^lZo{-f^lZSggV!!?D4?zi0!{<`I5O`sRvJET(Aw|@WH^`Cj#E_aVOc47hQ7MuS0 z@6|r_($NVU7qiZM4e8Hs)bq~nb1ZR_PUx23(g)8oln%Q+*xBZb+-0ddR`L2iX?&i~ zb|~&{P|b@g#$F{1*)yiUb@L9=j@O?bQZBT9 zmcOMWpLxn{r;sP(pNeN!xl9iU`TDQs0fpTdB7c7whg##@o}5<84%O%k{OP8r(I3O{ z&9dFK&qM4gpL=t-(>Q+K0B_I1494JyckuSKe>3s)oSB&Uv;mHleMb43l;G|rl8Tjs zdx%es>Aj<($Zt#XsGDvNEKbZjklVz)c2Sspjr;89$KC^*m)$yRR(;NN-*NF7!P%?? z16QZ|d%=18!h(bE-Yq-*IO~?tk2Xu2cu!^b4=rB!xh!mKMSrpxH?XUXH<@;>ROpKN`pIw@1)Vy@|%Q*$EeSg2gSIJGMJQy#& zm5?^pT=h#-4}-@IgGNs|caO_k^X0-!e$bSAI}(zPTlTrNWOk!q%_hmzl-&UfSqxAA z;QpcWJdq1^q)YZBF zZZH|x`?u-8$0r_q;`%iH;M}+~%c7bt{3>5$oEh<>UV*XO2*LdfgQ{*-JJ;1^`0Cus z|IyfIgMazUrlI+ZW78EWudb~g-JF;2?_28s!td15f^s-5baecwmFMy+S7gOM7^+;c z*V%Ax9fNJ;7?`ng?~3}<8|L>#`y0P-h)*y3JTvd9KoKy=)G28Fs(+PS=)WuBdLvdi ztIUj3$03x(-LlgsUA>lSA0C)=a!rKyWcl5)2iC4ZyDHM4utU%4U){LwAa-=w;K8#s zx;L`isr~ux6w$+Y}8`7K){2F?DX^ls1+4SUx-steAk#}>f7+#W;qIJtuD-Ph< zjXHr>lZUQkHWW`;lM%5dC;#lB)O+HbbUgda#k#M&J-;$u;XTmwpv~#=Y%7QGqG2^R z{g_^rdNyS;i(|Q)t5%$qKQj6?Qg31E!0qngmo`dlFNEZN!zeXn?|*Dk2KAff$vZG| zf1$}Gqn^h_8{M`$1ZSMMcqcA9Z;#`}jYcV!{YUh3m#oiTrV71vuXf}Jk3~PkUOS}n z*?8lychp2UvUUhQKjBr(^S^>}o%5Is)sbXr{?O`f3af;qhx4x(F0^(JwdA`?M&8cQ zk8)dp=)R1cX4}KzS+(~YW5_loqevnu^|NNyGswu>m;dNSxpnrWD1^ER<< zS;5QI-8q|l_`NG87M*9P_QY5S6yK%o6hD~gUddF6=N2u36Ha|48sRjiE-b_MLQ~pl z6B+L+qxACERRtNbCl=0!Bt6CZ%R`61wA<4hU*NK<$dG&BcCXT#e*M@9`$f$8zKo>} zI+G3#Ae&DUF?(VShCZmTu=5#Im+KsDUQj&D?OKC};3dDumhX+v)Lu64*LT+yu60S! z$q6GSnhCB{PaiaDh)a@Z#DhxMZpV)D8M{0#40hrC^hhl7n4n|$0!xtD$8z*-@!(ai ziTS1zV=amYmxdKTcChr1O&9d~A>CDU-OT#T!H16{@tRo~-!rV{JrstFXfSxTy4qp) zxtzx~`~MKh=B}KTm&&XCBj8M3h3mQ?UUQO&_vPC3#OyzfeMVLvUiVlP$JkjBK_+h= z{ygr*7vyfl>%`iLPcq}GCIwd|p0`^u)>AU4&x~)r-!u7WWYAGxm)exl8zr^dRmWbK zzh0h+H#EHNX=2$!6#-&oa@4iFz*$tbXIjkEE$BHQp~p}T6g$^Ohs~I*>!#zw2w=>! zfA?2J6a&8>(WQ{B`Zq`)s$KA%?FkAj7>B{5aO&sHu2&ZWT?}+F(8WL(16>SsG0??8 e7Xw`kbTQDyKo>^-^Sqz;=ly&x z<J-x^Rpb^vmoD_@e^e>fo(Pf5l*ETe2-JE!Y;;7FKLF+tzlnt^A=Lx`au_gdXf=h)g72$&FtB4Po-v#AIS>9fFjin2;IVu_CoJ5(dk}B-4%>5@+@~djt}J;a|j{e2*cE zkeDxGE*zGUD2d`&NH|Q2zKF>%RE&hsp8dZN*m;9W;V2wn#-!c95RQadim&XL-g4)} zb#ekNw+;|Y6eB5z(OU5VLWJ-zS{)LF?iBK2b}~S1g@_?YMA9Llj~G&7lfdENVI(Y; zAwXHmue;XzOt1h43!|j*u_g(0@jDSR#W;{&2Yzqxr~^L*vNLFua6!ZfiBJbjao~73 zE=FKXIo(GpLm-seK1x8;d`JWn5D!5x2%IQ-Zd`C&mozI6!E_uWVH6`E6zoGPfhjRT zFb|iJkWyR(1Y-hvA)@oSq)t15ke$Ihi*Z~6SRoN<_mp?dZ6{O8jk|yqyQaboLBbMBL2{SKm13nkIr{iLo1oI|glIU1TNkm5`PLFPo$_M4|o=VsNhV~6eU9WBDp?3LIc6!;bK(o_Rl0a`J69^ zPEObt!zV#8KPn!7p1TkI=PVDD^Q2(&h*aq`WoZ#bJpeTTq=={xeVj=g#7+?N0JZXu zKxhI8C7~#Dpgbo~s*EEbUkn6BVDaM!L+1=7MrKE_rXuiXg~q|i?m>c#*Bbz-MA{`d z50I+5M@J>J-hf9Z$}aKE6ytCY=nJEU^#(%PaMu*N;4&bm?nxBT<9Y+6+;Z0_ovFOg zJxDrz^ae(GUh2}<^0KgdjPfYj3m}z8xF>PC%dLRa(E~+LqaZOZMES6|7q&x}(Ottt z_^1RHcTc&Cd|)qvRdO#EiWyVEi;uhy703aA&>IdC0ZLE-+Rlm%8a==r=sGQI;=0_a z4=CF_QN9%K!5N^&PLR-p|35aY=oa4JFI5*RFfhB{8C{V6mF>|JdVi0=iJ%h#j!_mh zo+yS&WD=@yr1~0p(=UZFA;N;3L5CrLKubawe{@YqSAY-!?h+)h859M|aTo(7ITndX zLAa+{B9O&Fo|od`Qj|JPbV&}X-_Mjm5;*$PWd`%3a^k(T?eSik{2(cPm;gQqwI7DH z_hWGVt4c@eQgXWC51R6^jEW2kyxLK3SxDO4?^8xBXXS*Z%uhWtrHu11 zMbkhhea8xDds4>9Jvr`86IbY#8Ot&)*|PDF1vDlpY5t3W9$ydDRpSXlL+=b@%nnMA zbmBd&&LJ1u-7EQ{{*cex(D@S zroHn{_O(1|LOeicl`U=Hpp`0 zlbuZS)Ae&T^wd**w)J<(KsY56s~$FPupQH6SuuF&;ah{G;ma8>beMCTtMkU5;5#DQ z7hpsB@ggL8noUoQ_U8u%vvS8rnFa~=R@MA@#v#YZkah0#5S1$gcM-WXYJ43_Js}p( zE)EhWJd)(JO0^` z7n11kwg)rG#iwpgnzZVkdHVTstCw-dz6!wD0lO|ei)05FU$|&+YJw-rK8-uoX-;8W za%Pf2#g#=&r-0| zF?L0wPRkBgs1*EBouyDO!{*(xYRc!WH}^Krr2$ zxX}!)i5CWpb~iXJbutiDd_PoOsAjfGMC{fT@>3>hO&ML_EBH-VyLqftUD(00%IVrV zN7p(W%bDUMJHN6lZOe*5_O_4j7nS_{^58)`2jAI4-n{;$&Fe{`VF4Z>yq#CQ@WC&O zFYFw3v_6_UUa#Ibo*NrrzGeUYON;YjV!0BVkrUW;(M79c!;5Y-s+g`jzo3!(VCTZt z60`WASi7J-4fmQ1XK-Ivu^#96lzTD?FKJIp4l*ynC#0SI=UBTq|7iQRJJ*U{-N<>` zo?TvdwgSFA`Tmx&ZAj-^xRfygx=*0^Ei8=NoU5Zb4F@~ajcXUea;%r+JC9_8)>&mW3x1}zJ7;Z zKuKej6d5k9yD-&dN_p!pqb5fC_)!u6k+Ha|?q)sN5ylPKE%Q`YG@~1De3Q6-2wQjdo~6|mx8JU7L5*LaWoV6glR(JL@%pZwAeGdtr2DDq7g$f6V&n@+H@}{Z^*JZC>Xz%N_-7Y(J5ox=&yCgnLVU z@bwB0!mo6ivGr|d?ZFW)qi0xX*RKC{?8XCnTaR0%`Me3@_0y}4X(w&JI1xB8)v$FZynV=vZV89U%VBGMMB zIwpmROYs`t?05a?PUDu_qzQ!H_No5?o9La>MLur?r{g4NlzY>4DI7oAN#!! ze;%=F(AFahj!CZOZ#(-RqtVES_8%e>l3`#4qQl3uE)5kyk|)$21x|bo8&? z{N>wcvl9-wl_XS6J#M2f{;v6~R$G4RrU9ZPUH9vTjT>^b9v<^vaXD%8oOHhV{Ao^y zU4`jW5(*aDPdQw5+39>xhSiHv>glZ2C4;keC2m#O>=_YqszGc&>wL|w#BDP+SL{h> zJ+7LpQE^%7bMA=s+T2GLPi<>=?CY0aKB&mbMkVU`Ysabqm42CrtlL6nzqKb~ToW}m z1eDsO9~osBsxz;`rD;9mb-fxEH2QAkyz`AYsuM$ZxAP*Mve?2)#UsyXR5EJgwMX21 z%WSyy8+%02H_{D8zlGU3v2-_I-Th|x-hK|7G>aQx108@~K*DQ4xr>l7I!q*()KJ(x9Xt&Ck(S@!p%(Q&t7*&r^R(CJmvbgzXvV~N#YQb85 z=7ly5_T`0T*J?%hfhhHH#xl2mCi`!nXIgHOvAB5Ci2Nm%6Y-W`f600&w9e)xo(r-* zx!D&+`+XZ*ryedTKDa44-Xuuhxs*FM;pDWF z#lhtVgB$cSJs!{eW=PzrI>RT1&oT}jcr|RD_Hu0=h>@kf{yTlRRIR`qwuI2(bL{H%Jr*zdq506ifaKrYqHKxcf~9$I-r5 w9|L_1^fA!KKpz8r4D>P3$3PzgeGK$5(8oX@1APqqPcQ)PPv2dkGR*4!3mL=Y-2eap literal 0 HcmV?d00001 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.hoodie_partition_metadata new file mode 100644 index 00000000000..24d26b944ac --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/column_stats/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Fri Oct 14 13:01:22 UTC 2022 +commitTime=20221014130009989 +partitionDepth=1 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 new file mode 100644 index 0000000000000000000000000000000000000000..9bf687c1a4b5fe36c79ae9759b132a53717475c4 GIT binary patch literal 124 zcmY%P2zBvPW`KY^AjOD;L85#HC;%h`1QtNd1jGi64ADOMMd^Bp1&PTSsd^cuDVci7 j`MJ6Id3w3|DXBSnE~z=GC8de`rL*ZaO}@8769 zxZBdyDa4_Z2L4HbQyQEDgr^c3M#Vw`5vnDXi_ke-233#BhFSP`2M&jUB2EYfGhhsM zk-WQ?a`ago2Eqhl7#uoiG8+=gg?R8PgaHMkSOCg~ zumA)SNrv2DUnT-N_@M}s!J}F(4HK#09YE;dP3N#!DCUi!Kry{GeV0<%K}>uc9>zu4 z{!}l~i1^Tm9~{KQsFrw>aJ}y2@sO2l&JPhNoTK*&B;<#P3xv5CilH2~cOZ-f{1B0T zC=>A}Ec<@}P;!D|$7wTw9+fcva(ET2-*(B8>MAr&xP%c1cM~Z=McEi`Fv3^dfFnX2 zKf*g$ZGPA0K2!+-_*8fyJPcupT&N2|V%=D9Ah@{-W(Fa^v&5Hs_H{cj0FCX3yGB=^ zvyafbvxp0W2us*nA2XGKJ z24sOXgfa};91-$k!v0j?jbeur`2|E{GY~$$3M6d_{F%drU?vm7y{paV+VVgU57-)O z%K_a*ZW|6Wh{blrV2mdkNeqN26Ns(Tvj<9u5`)+7fplUZ2$&2*c^FFnw)^3$b3UFX zMV`?`hrYO?7(8HE1j2(e0aI`QN)HhFM;C1%I53cj3eDb4k`wm%f!YZH`(gUAQ1(JR z9(RvjXZu+T1EmKSOdjEijV3NFg7F)m27uZkDnu-2j00jPh)EIJ?dx2Ywdz=bL9ewZ!ycNXc_@i`~*$>kp%IKceMCd3BX7&!b zt*~J~Dl5ib=qP&BHeB3<`%s2Z0T6lvK_WnTc!0JvF$5bufF0O%8chxFai(sdY;!{C zTuvX>0N!>Ai)j4cm~h)}q2qkbILRhz0Y%U1m_bloK7&wqu7h*(oV?P5_-kZRfDoF*dZV zto9}NElsKWdohAOCq{u!o(DQ!4G#GYNaB@z|BLi>x=bmL$k(9!Xx_+EhM zNbFU*u@?TI=6eL={d$*<;o9K4b8$V_quM--!-f432;b`CPRc-eD?rcgJdKd<&IS*u zK)~Vw$L{+`K@z>dp5q@4+lOsjTjfqHI_NvMab33{-?f$V3I`7y@k7C|!y&w=++sW1 zZML=JJRR`e3Y^BRZ3lQxISX&&Ibqvi{69G8K0a~aeB`INHP*TO@)iRJJ)O&Xm&+@3 zx5xTtv<{cj8|qV7EKQXfoqc|UG3cO^x=)j4yU4}a*$w|X4V=22<-1ABkJun>A$y>y zsW>I2X?=0pj`AJlf1j&u;zu>NDi#i)Y(}d6b%BtZJdZ31xnsd4(cvS~duqGffufs~ zs+l3mb~ML1wE1>GA?cSCin47dg=A#gJC-c^g`+-IQK3riz?>PCbutsx{#6iYttm5c z+526mv~5j;GiXCzxE)LJn)Fwm*Yw}=X*TZfDw?d$H7J-gzV=Jd+Iz?eazpo*TXi=@csDgLlscCv*d%*S11P;o{w>$^s{Yt4EhRhMUG;&Y1r=?9wY(<*j9^#>w$l zBj)&wb(r9oTwFVH&Llm^HOoeSv%*i8<7ol~s_eL^lsW6T<2bLm^2-9u`32GRteMy7 zr?OW+%%#Ru##YIVQU9d%X>-<{xb*l{;{v#A&nY;cco=zq$o`lcVP$?{>vUhqKE!gz zI4EuQUlS(3#G=sJ{LWW;O=9bm)XLCC*P{A62A?#CX6p`GdA4%n!sfT5;%yFYH}|mD zx;l~)mF71$+kAKBhCT9ZBekO1NIPv;%`4SQtOgBIGfLE*n%VS5vpL>0e(Beli>`cD zzif?n;(jSn8)_eejR@inpXKZpO77s4g357P-A_L=NJepp)kx z`)}sO#!`c4wJI&|8b7XB9P~AX!p~$W9V}5A6Jc_h!Hmt`RXJVml}?Mi^^n0ejL-pB zR#-psp3K$UKO#%-9b+ARrSG}BUs9#+HV#AfZFxj5 zuQx0BqVho7XQ1rE)gE4PKbJ4|dOeA|%_QQCcS&i}`N8I4Q}jxYK36q!I3{q8%Zv0k zQ~kL#?Lx&QYkhu7Y4Po+Ma!&9HdoWOrRnl_?0H;K9_cqgqw(R$GKPG@SkET~N~3lK zOlMMPn|0&sBJ0pA`zR~QhKA;amh;W$>v`VWR`aCl@Tt{OdZ+I`J#*&vxbdnN+~<0^ z)`cv5S^waq{NV>vj+DO4biB4VT>EJHSD9-Cw;+=VPVc6~%`B`@uM@;d+3mgdLS@>= zE5kp{#SSCZ_Hkwn`!>}~^LS%+z+29{wDP!8qEDDJiq>SYbNHo5LSB3sU9fVv{onR8 zpR!-K%4pp%O|Vs_P@cVVO?Pv_BR^Hcaqv4SMW3;{(4Bdcr3@+NQnRESC<`b{j61(( zLOI|C1CcBBg)gZb@OpCILK~2QVJ-}A`QWd)$b&?HM1VwqM1VwqM1VwqM1VwqM1Vwq zM1VwqMBsk|0q{(;{jn%TCqrF=uZllnCHbm=gDc{{l6=*FO86!Ds#N?XmiV=o@GYH) zGKjyYBKfMF*=8Mi@7C?qSNs(q!BYmWbxFP|$yddfRd?@%g>PP6Nxmw{R~3g1lCLU3 z`t6;u){%(Q>DNTTBFZmLJL(TYoruoVJ=wPtHK;oUNxmxnLztde)163E-2q9k1^dI) zLK{I^R#^v4cjf-VsRJM(i%zADLA?Z{yz z`Ko}H-Xo(_OoS!*sy+Pz_J5JDI>Ok{*x1O>$kfOL5LYcM%q@gZPDd}4psc2WYiBz# zI4S4b8Ju3%rz>1ERFtkPnLB5T;>cms4DWdDyUn!9K=+B*P~|l2X$h@9ucu&m3w8wZfuBe zxX_6WyKFqchBX&hw6kIVXk){^sqVstjZJkp&x3tg?TvJ zm$P!^8jz8>Wm)@;zJA$|DCO^!j?S8zulM_}BR<}+)yfHye`vFC_|1r#m*(*w7s=`k zD4NymQ>8I__hd^ysiZ(P?riLm!*<7JOws7J;S)Xi#C!a3f$LlAq6-a(Dz6?Le=zlF z%UxwOa!um=+}t1zH}*yHFjNK2S*Mj<7sTKCMeE5j3M=}-QnNpYc>Xf2DZMCvR@$r? z4fk9tuSku)c)4*xUi#mh7kmDdEl^=stV!BwSe~U?{M35Pl9xr!whA$pwVrB8dh;k_ zw)+Ulw~JftxpPRH2CF%G&nJuk_XT!dJ`(13{0;qNikBSMX<@{B(7osfxp|Cfi_3=c)utsmvl=PhvC(UPmL1D7JGkQhEnTbox;J+2lFQ$CD(TaVa|LHZ z^W?tFZ8DCEI=o-{UfA4ud8>l*N=)?k-jA6vz9l-$yw>WKcCbmyn5wx~s&F=JEA>mV zhz;vBxO{+Ok0y0ZTPj=k0h literal 0 HcmV?d00001 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.hoodie_partition_metadata new file mode 100644 index 00000000000..1c2b491e52c --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/.hoodie/metadata/files/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Fri Oct 14 13:00:12 UTC 2022 +commitTime=00000000000000 +partitionDepth=1 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/.hoodie_partition_metadata new file mode 100644 index 00000000000..bb4d3eb6387 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Fri Oct 14 13:00:16 UTC 2022 +commitTime=20221014130009989 +partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow_column_stats/2018/08/31/7196b21b-34c1-4e78-bf20-72dceb524e9e-0_0-29-29_20221014130009989.parquet new file mode 100644 index 0000000000000000000000000000000000000000..606e147c6e2cef50005325c5c7e62d526a30042e GIT binary patch literal 443915 zcmeEP1%MS*+g`exg#~F4kX&F%36~|MToeHTX_i`+?i7)3P(V@;#80|GN?Kw;Is~N= z1(EJ=Zp_R%GpFaw+`9|<|KC0F#`8XJ%y4(rk3#Dg3ydFFAh3Szz_@|&0|K&-O&s9) zD_xrwErN>&7cU+X98xT#RB-UhwQE<#5BTE1z~DedA}}B@ePEeFfn(w+`UwLQX9-H< z5-t{8ETmY8;Nal0Wy_RB9AyZ|&m^aK(4n+$*P_MER_)@q3P^C~ZtA2S?>oW+A0>4A z4{-Z06_7uJ2|@GV=4HzU1)|P_As4+)t(u-SIC0gTch{6W(EJXw z$IrMqXi&v@^Hwz9{(4@cGz0!Qx9L>>sEd!EeV(*Q&h+&cq@Pf^VAB!deTO{wqt)3$ zJEs)d)4kEGYpatdOYud*(Bbn#kLGBfIwW!C`0nkZ(BE=&m=sbYbDk7a64o59Z0EQC zC?tR8^eM(9tUCO=&||J`8N}c1Gl!&@m9Wn6#i7UNcQ_j2+7?0l{Z+!S;R|*iZP7kB zBq($86s~P`)8CI@bnAEJv~6ptPZ(AJqoWg3CgwAK$3Sg!944Yp+Ul^LE^= zJL2|!zUJw)ch7cfeQ-jaN~woML_(}$-DlXbmaK0p3j~?xOM*g z#7;%8M@N4*uwRkS=MI@(>DN|kIv!8;y2;r4?fx0mv`6!dTYnuizR&K=-{m^B{Mh5_ zw@?35^5bjEk2UW(xAOP1%O?LQ!?&xJW_@qxgknRIEts74#lomDf4sAA{>*jJ%XWVI zFlg4*PVZc+(Ju8j8}c=dO#b*nxdeedy7mm)adKBkw#VbX`?hI=sG5D+WpC8-tD%3K z|0~|4C5am}oVsyam-pkQ7<%w_oloNzs=Bex#nlCJb^3nW)B>qz&&gF|e~Q&hQYItP*_@AWC)az!^!(+cG7&)}x zq?Ow%#EJ9$vO*QNH*WG<(H)b@B&`s-w0GLz#Nqii7X7|j@wg?Yq-oc*&B>XsHa2-L z&a@U4H|=@4<%jqS@}*omE_d?8_3Gr>@xlEfNhg0=XUsdTb8K6;C1u&I=T2Sw;K%pl z>^ssREPe3a(Y5b<&~#n)QD2sd_iRe?_*M6RP$k7Lc{|>Cf_MwsZ{IUtws=vXfRv$Ei^<&oAdQ&wsl5aFH$7!>Vr_xa!v%p`k;n zWE_^X`;#j*t4_K6+s?M(eJhn%(EaeCNAIQ2yE{Q(z0t>R_uV<*i?%E0onBETeA4Ki z*J`9YTq9)J|04c~xL9@h$?^r}-F$HB_vKeRwb{94+RgY4dTk5nyZzDh4J!k?E=V*j zQG<(JlWb@m|9E(#>;=bVDG|J8%h94$ehB<{{K;}(l&sq@=^tlDe;F}$V6DczKfTiW z`Rl|5!$0ZVr`wUCH9HTevaZmpIH?+aQgUNzrK92pW(llfdJbG^<$eyVJz9F@(bA>j z2P`PQG9smC1REXzNnL$cG`LLB62*dmwKDR79e4w^A!5kK51bJv@TWK?NGX0fmSSj* zKFK{){@=_N_|r3!|8)l*WWntW{JY(8yQB8(#6V_YE`^gtHLEhS{ExE(zbM&!mZjB z4=!A~c$?O3Ta_#x(za~d!okgh3l}e|{GT#e43E7cU0ZhV)~jvLAZeBq3Z>bShl-ZI zfWV_cfqRp9Cv;-hJ3``L$0tNq)VrXqos@ z)#`GodsCV{NU}cOleWLS*ScQnE&py9wLNUxh2JkHeBPzZ+6GO>rpf*9(y-a@ZoicF z(yA_dq6a2A@Xdj;pRRgRvBBDWqbuAjwEJ${d!P55eKu#pm5KhnyRXuO&0X@1Dg2;l zf(o5ZoZnpi?>zNNJP&`g>%Gf+Lb|Q{toD*SDV{wa{^ZK+`(w&Y-Bc;r>skM_ivPUA zk@u@zDDhv`-`|~(rq88!=O!I}>ECa=z8AW;?6T_7V@}>I`uCd1XJhu49yuYw^?wpR zopmg7N%Yn;w^#LP{d)AB4F8quJ?2iEdk>o>399xuasNZ*>%0j6Vppnl7i+yJ)GyJo zOwXbQ*Le|f=l$Ikvj6gG_LF`u>UAIT>H0Etx0T*~EnUWX7m}S?_*17__1=3H_h!fF z)ETqA*RoXaqK`JGY<6>cfqzmR-B4rmnXg{eyxteMGQUav@7q1pEDFLgZ8f5hwM<0>BR_A2@2 zQ=48Ne^556>4Or98r~W{b@Q108*0C3@@(=4FKEFSB3x zY)FAL^Hw(B(JwS8+2I%c%Ixdc_)xMjL*k^L^3J;aN+BvEZ*QW&6Uu}s0Kfe9nshM9 zj^N~bR)=M{8F$!?fuH4jp6b(qg{H5lyubYE`X7GNZ|a7Y-`sjIXwkDh=OUhFADN=e z-+5Y%%XejGzNo!5pSQeFHg4}0D|4?tnqge3Lp=(0$X9Srl zUd0Jq6xaLy?CK5W{WTr*;!>`7*>b_bLEvF<*>WMJl|!gIT9^FvX4jrm%jd}bTi*Fc zTXr888v5z9@u4%P9~-}OSj6zLU)3M}XPnR_UyVyZdaRw4IZuYLw@3 zaPcbffBI*Q~d z;^mpZ-@o2bv0qTFFEcF9_V>27SIeGW{G@!D#&hNu8S&0f)w3oI`QTi`0a?R-TpXQl z-t~?NpF}N>D*9929p#ek`R8_Yf=Rmzb{?}OdHQyrZJBcN!=!iC7h6(vewwoL=jTp- zq1d2X85*V?_s^5FA5?Dt*M$+Qe{J$2>SAQX`ghJI=@H%W&Zh-#7TMBpRGYask`~Kd z_RNmOv!|X-eR)LB(L?vVn|$S;<<1?;Q9d|+-K78i_;1UGT}vhY?!wV8c6Gen{7}am zkJ^=b)nZCyn ziAoF|+Tw%!K`BS=4qX)xoOwd^IG0mb$&+qOX!xq8!Q;mztUS4MnK?;!-ETa!@;j49 zlt`Moc&?AW49P#@!z9OdU42@+ZtmR)M%QYQhA<@DSKt|sM-a-shwrzt$`U9Y`7U-ZhVoMl{%j6zwEtLCw?#V zzjeQ^JYVQRk}JLcSFX!%TdF?ldv3+YN$-61?eP`i15<4K^74Yx$(2r>u)I&3Rh}YC z92jhRBN(vz$c})30v`-&r2HAy^=H$Bfkgm1g5qCJq&#!n|EoN2fE`3l>zhhBcX*O# zNj#6I2|d`wc^I=Hrt1E+4ka21?7<$efM^+iMAN{Yti7mN9tbP9rfGO@d+;uS@>c05l`-S?E8B9J0qV*A3D5#*0DLWH&qI$ z*Yfl6BVU#%kmu4bFaDP_OXgu$3hXVHs8e+38oS1x&UY#M>z!3TUzT%b^nqtJlJ&cO zbJX#Ydv+DMl<;CyaPG;EmOh)8@~49PDn!msvgVJ^CQoV`nYeJ~$qiEM9-HB#dCNM@ zKXx=l^w3YUmJd%65q|TNwkwm)DYh=^&(_yYT&Xd%Z^C@{dw<)lYQIGl+Ad3TV8Yg6 za~tLDpYHju5!uIPo&G%6*?7<6op^kubulm2oG zq?IN#tuy*!(N-5TBv`tnaI?ja3Jq>}D)o@g$6nv)8Ro?E>KFG>A4!P&7VYh0`z zZ}!54ReK)D_Yx$$KBU|@B+dM%?*%RB6xBu^- z;2D!v9y~Dp^7*vKIweWjq0{@Fb{$W$BmI-QKc9YaDs;=CYq{(0$$Kwel?@@!u78;6 z;*4x78vXI^#V6hW`tR(A$Bsu2T3k9;&eK0+J96W8rDxsq4{aP>sdKKb~)D_Ood5x6>fPr}*dS6`c5Y+Z)s8+M(}xAoh$ za|d3GD7$-Nsevn-ExWKG|HSgmlO4QxJJW`TlP;}F+rIOf`oo_b{QORrL0fBH?2ztZ zhdy0X?agsy&R+vccW765+=eCd)`xV-yJE<%i-u%-JY>ntirbDRjf~us@SVMPmp9tl zdiv-WMM`&FzO3chhDGir`rm;Pz5aUr$HFc{GIpKV?dSB@X0DGjVA{jw!S}yuQ-4F_ ziiLuHzWyTjZ(+@vCpjKfEbCVl8&|mZbVaJ!4?nHEMp|pka0P)GQt{ymOb$@0Q6pW7Nl2y6i7rbXCr=_g5dP zHM{PTv{S|(uGA&jtANsn#=W!ZQlD&tUjO{h5Q^Jyqb}0 z%jUVsx(?YIRIh!(j9V+*>vd^t;m=wmE!8u6aQ6ds5>)zp%!;U8M^j~Q-#hYgj@(Oj;nNSdii(y_u9MlSlMiQ z8zee&>5r*>cb2?7tX6`qjdu3EF=E#7PW4ybnV6_~?WSweHtaNQ(AF+{x1Ov#`&R$! zkIVn^XP)KNl0~0BGk-+ZkF(w1@pJbJAMcyF_TKw{UyW*ze$Vy2Vag_u`p=CjHNyP?wIqdd0U_V zH~m=B7v-NlKmPmUWSKUc9lmJlrV6LlKYej`MTVExYi>Jq?^XAo2Na&$_e9vhu~jk- zNci++p+iqfe>Q4Kp6Y#%ADrEL>5Z0KhmLqzF-6N>XU{#nvgXs(U83IYvi9Zlua^|e z74^Tc{ZAk6`Epr`I2T5LbL+*AX>%7Wm9Wm1WQ}f2`TR-Aso5h+KcAjx_lD`0E0x$@ z^xu{f2fZ#<{ZPR4ZAHcg&T2NH=+~PvCT)JU%nx5zxSe|I!^DHP{_oE@@#-#ZlAvVm zdnFH+{o!EWq`h8!pFXNmy6C>!raa91d9Hf-G9;Uq^KzYLNe<85aHwn1WiPL9nfJ?s z<3-|KNqhf?+vDobEIp#k!gYN^v$kInw(b2}(b2bO4%zzUza!vZhYl!^W);iK(m#^fAwd2Q{n@zT96 z)c^ZVGp9eP_E(wa1E$`bUb8}xPgdOx$=>g1f?Lts@8A00!DX}N*1GcQ=zj-im8rfy zO_C;+3JfY*tj3UakHdOh`k+U;*As7*t={?9Wq+3V{^Ps1#((#1wvneNtnYlO?6t+I zlD4^%>)7G_n`*bcpv-_X|L!mS^zx-bRSF)dbgloSnwK)&UKO3X&#P${?nd4^n&Qx) zUJX{w99%!k>Mhx}e=}=t>Dk?*ufF`J>!|F9o@}d9^|KEyZvN%Jx{a!;- zpLzG%>z3u?_y6xnBKMz20s=2r4xFZbt?B)Pf#BDgT`mMCKa}F!qKr2a#69y*y+GQE0PG7nf zy}IwyQ5{Q+dtH6(!E9LvUih!6@_lBH@qO|r-)A1%cI@%KoA+NS-)Am8wxL|qg8D~h zXUIRc!kH5j%I~aO>~)69x$FHqansT-qOuHccdX*Xgi-5zw6DKBa$4>=q0Q>n8*?t` zPQRWh3ta!>=2t`ix36Eh8_M^Y^-s1)^8VOVf4jcV-11klNlTJ8Xg+oGwmbb@-)CO> z_PfN5e(d_~*0w7~jr(v=z7-#IO*wjgjz0UZ7r8oV%LkjQhAz3;cYNo(VSgWK{&mIm z%O6cU5ccfvCI_e2O#ge{1|KAv9G+-!hvqLfRgQOP__UW-k|b^)+4kN)xzDB977#Wi z|9|n9l&i8MN0oVhZ2Bbgdx?Lm5jvyWss<5X^;lm&Ui+N`yN^AXs#}s=^|G|jR6Qv5 z#aW*W9d^CYm`SDTWGlVrgYKRBWbM(f*PMgHKIs#8LbD&gD$`dfR!@Gt)c=cNxzjf2*t+e@78^6{4_}bto%?0VwmP$a!v3|b z&#&x}B4^>cSMvr>h`hBq_s#D*?ykHnSFJ=lYi1q#^~Ne6H_tYF-me=DHJZF8-udxo zDz_drJZ{4G)--6dC1=a;sxEqQzs8K`PaiyQRcmhkZrx@*>3`+MnEdVj8C|OB?@`Ss zefLqykfeV+`>^wSYdha9Q1aoD;A|CN)lHbSdcJEZ7vwFRV{`Xi{aW=tS!T+BfP>w0 zw4E1k(!d&3HkAC}{-AlydLJG+xBaGdzuu_2bI9(Dp`NcZXI(zEGcvsA_TYux5C8J$ zz4EocOjcxW=7p#4jd>jZ_8(%n_mcee*T6Vkw@sM7VRhiq-{So`yx`r9U!E`f#jX55MQ5!xA=8YryKXjY zbz;n>z3sB@%)F#v;J<%Y`DIPj!S1g!_f2@6rqtyhYV_%LU}(*bp06_(e%mR+d3HRguM+xuw6+Vfq-K1?$$tsgSX>l`tv)f%49sDraxXL@f zj;u3rVfv(ne=poK`-1<){bk0|Zu1YGxiE2c;#rYhiv3Wt+dG?QW}5TIi8PaMo;W-z z&&M;Hcix+(=7|m;#Vhx9p`goKE2J9m`sWUpZYbYD&AyTH)4Km`8W5bU!MO(gvUfh$ z_^&H3cYk(aN|{d|>^a$EVcL32!#^HSeASxA7n_uRHUE!;$^Lp89RK&-qq81)nQg|d zk;e_cIS`_n@^cWJZrVynN_ z^eGq;w0ZQqKlNFZVfk`=`>fHEu5s=){C4u?fn`rr zirmqn)#oGrZry9gfROcnU09X+;h+s)_lt8ZU$#8k&U_YsB!M*J1vnASadF=THITmIsU+(O-*B>>y z-l{~p$G3AXJzb=K#xY&W{n;U1`D~}|&kTCqG4=Ti=d;h-v}E+a=P#aYyFal()%V(- zh-{L2M8i+>_F4Dj-zA?dem<@Flj$puB$#t^)s$yt`k!6#OTKLnk5_M9?bwE-)jRKt zJ~Q&+(W|$MHCEpAk0$(cYqbZTjX%A1)tn6nf9{p|=a-cauF03>mx2L>YcD;Q^KOm_ zGix7tx}0N_z^zU_Xd#xww|EsX?y}@g*93C|0@y#&<&pyw8C0mz2em_~SeAKc9gFCdmH=xAJ zAEFjloKyGgx53q_e>Jbxksn4rU9on}sm0B9KWx6~L3G5W?M)_4dtCd}wH4)pFW0H_ zYl*YP1}y7eE4p2>d(FD!*}gS=X2WbZ%70p`!tspTQ^k39|5B@GA1+&RxX0ALCRW+@ z{;y9?Cd|`kYoozAN)El&di&0J8m?fIz`k;RSvx|+NG!Y*e9W*ZdIes974 zC!Z!;ed&4kr*nE_zLjrJkq!5+KQ10Vr2dOcYmQD_lJRBB-4VA-E_(OGvg(gYc1g2! zOq+QJgYRq$>9Oowjkz`7xmPL6of3_`k9y_%@o9*mFx2?Y@BYI-bPCNH*HD<+FsRw= z)r<3&3!J@U#-gc}7yK7qBiWe;`}fs)Kk385>F1p268LZKmf<<_|IqEws3tu(w<%d< zW&UD=+kV?>{kR&VZpOb-y=S+&S&N028uE9+C-K&|tQ^wu(%gN&SK6@Q@kQm8e{PRU z7gvvLa^vydmV;M3%AF=d8+uys;BF>_^;8&A5Ixr=Tx;jD@y0?{`Y`~L$dc>k$z&yvQyjb zT$g%7#i746ukd_ZqfB`|&p+~2=jJKKe>J#GskP@{m6+c3YJ!AKUv=*}Vd=}7B|f~h zFG=1I(F%9fzfx< z4B7ni)!)MtM-8ce3v%!_M*5Th2HVRiNwLt%v6>8op`w zrUegFUpGSZH;MU*W?;Srl zc<-{%Jk?UA==#$SFCYCW!+v^r4YqgtOoUv_!;_+ANb_kCS= z^3S)^{yFtvxeZ_3eE-E*#50 zXvLlzrz`cWcH@If*UWIGUk6JMx&%6qU8rRBwquAD~*Jchc zTVq6``(NxTaq7cM56Wz9xwb>}tB91>b2RLEqt>Ogc`Cg-JNxj#YquS$GvPtWc*Fkt z--@&I79AhCyWFwxux9Unyx{%mL0x7J8`$7~hkDAdA?A9{ZG(*wifd^09w z@ur_1ud25#UEZg2+Z`Nm6Zl zQIU}ydMcNZZQoVC*mT{cH{A7T+fDg%sP{6hdfB~g>!|K+ns;j3Prb^ZUMW8c?b)Gc zho~;iySD7vUcFAEUbpMexoz_;M6?eGbS>j`j_TtPiKih`1lqLh=?>2&lAv|x zs2++lw`Nq=wq3mPZCbh_$uK5frYxCWq{*8&OJMR$CF3Vfm;#*;Wko)AV}Rmc*U)RPJKxb)ai@-QWJ z-d7crJSc6?_7f;D50mO@R$Y@9G&-rSV$}S-wNkr?(*V%ucb~iCbrp?khH{A7@3HoohD zrCFVpI`5M~s#WNZ3ND)68HG+En#WCuqfru1dH8v(x~odLQp(Eoo?$Ml4Eu-2Q=WqX znK}oh^Zo)fO@aKEdVblik!z-lU!YTb?&Ted-Uwyp-V)rW`%OKiE6DFRgT)UZP&6_ z=bpLBxkLsP!r61b)JQR?zX(Y&2fsW?F{!<9S+eGNNz+W^>Kvso4-nk5hkobH3vlas zl_N>G-tP4K4qbYJ#@+9G`mNc%>lIMHZxhw4Rp+*#0cARou9Z2XUz@GF-XBT#?pI1( z!0g`DwxoNd|MY9sf3raWL23GQ@6fYt_ac!|ZQ6D&;_98C_^u8M2+HbxgKyP2Dk`$M zl4C`zxvM{vcltJMgOXNhTsO2%&9LU51_uPa6V*MUNXxD*m98w(zE_(LMchgrouFwy zW-jXbt50ZXXyuT~puM?x7Q*macHO9Glu2cm3n9BY zE+KW3l67i%EgL@`>B=X!RB_ZiLp;4Rc7{-KLo|1znGqo7|Jcv;Obdo-ZT$x z1U9$0=8UxPBwy(jI&N7FMS{b1M+Af^1gwsJVM+_SL39NOG+G!EvARKf7Ak5COI8uS z)%ODil#Vu)vTd+%1q!CJO+0K0>C1S*VI8cu#%5XFKxf|1GI!6`5XXZA*yFzX1NP?&&m^LTJa zpk<9m(@t5Ac$~oQ9*BAWOs~Cv}j;Y8YN@J3vn`ovi z-OK18e~HBs5iw91RgON%MSE`Y4A8x(NySLIV|MSvgNX(P!hlYffgO}Z%;SMH=ul=X zpQ!<0@K~@9@npe{ELA>g8B!<Pe;TcQk7HuA?^nqhpU4>^~-@f4vQcmP&_rDXUtb9#?y zD_$_(qBJI*=Gk?RuV?zyl$8=$c39$NltE;`4^c7(!zHE_A(xil3)8FFkr0%+0_(^? zHg^L9-D@yqAsWD_1uGU*Idi4pB@ic%7g|K-nL%b!F9fru?#mmRq%b(CRbdeLRYhX# zCq7Bu8$D0-7;Pu2D#Pzh(O~ifM6+f2Ua%r9`}G*$qQbu98oQed7Lf`sz4#(0YQvWX zlu~TM9^UR`FrefqR^-N0Z$KzKvqFESBD4&Z%tX4OClOYe%s3)sz$(M1&L`QL2(=?=bzwOZumLs%unx-( z#4N)HXphCeq`8QK4xLe(S_g}0TuCT(AKWVr{M@! zF2*NcA6oEcL6tW$d9F-!6QDrDksv4YDl9Ytf=G&-bjO2Ok<7>%D|SYKzX-dsSc_S08eaOD)D=0Vl6fnQt?DF&~0>}Kb`Sri8t^! z7(`KfFO89ry|GKPrjG1SzxCTEKbaDS+R|fWUQuQYvmLJ>m@6dD$U|N)YEzT-$aIo* zg>OLuG$X7?T;v$$fsFh};#> z5<_F-?rnv8?&5PfA=J6-bQ)5@S*)_raNC37vy+V}OuB#AAzeNuN?0jBksiTt5y+@d-LM-omlcPbjnS8ID5X5eJ(G zclHJXdXlkrux(MqKm*29SI{kMv?w=>x)xL!Jur#kiWN{6uUiJ=fFJJ?O^gSGDQnQ{ zoN|jSG324y@f&(#NRrwIhGV}aN5zW;7*hhl2EcPcWrN2UN*3Lns-%;hM)hUnLfu1( z>Y*Bej8E}_DZ~34Yr}Vq?mN%y$wpx$H#3uM^Anj#jUe>4dmwU{2$qmY$SHE8RuE%? zpfQUy$bINYnB1srRCp5twxLdJ%!m`F85_(Z6sEzV`%t={Q$yGF-3s}77aO|DZ(~rO z5)3&dY6{i?kO7Pv_?OslLb2)G zk?h$+nB1af#0b~EuH-3Dm`$i?y`qApt2tt*VO3VZ6^C`z3-k%c@4X`BiF5~9_HLK} z;&Nm#Nbrry9H5O16S(JJ%C`t$VB`bPaA2u|5~xxK(8Y8YD!YCe2Ns;9T7Vmy(#xU@ zF<~XPwN3-sR#oz~NB~S@l+!RVR`jo5DjvvMZU}HXk*F9%kb2e+^(4Z!sxwmT1aeIM zFd~HG^1B^CAh47KQ@%wMFxwMB41=KqQJ`bl8IdAvtYIwk3#-j~0nQjqtzcT*>mg>O z7z`7pMmBz%UI}2(;EtRgP|{!5GAQF`L3ZN(m0hd2@U26$jA0lvW*}?)>4elpHa>94 z2~$R&XfBm3xSvayBx!O!TJThzR9-B{>tAR%!tv8e;E2FEQU4zn8W#qr#+Ok&>(s*4 zMHS=FMswyr3u-icP_Yj!1D}%J@`3b9S2pFw0McxgDqvQ5d!jJF^O&+qn4ytr=9r){ z3AsF(1Wh$Y!y|qllZ(^Dn>Y>pnndD|34}GIftO~8p93ZhIBX^rMj`bzn^D~V#gM7A zk3SYdu+L@|Ww#8n7fjl0QYN*Sbl}@W5}0DkF|s%bR@vUxyjE3V)d?FUf_JG`oNots z-=r$%FAoGs{6_F#rQMLCmgdB4$V(5xJL1T!{`SQG8DMTqW)Gfr80QP$ z*R3eEU=2*L(FaO-4Rip{$ZI~f$+j@JB2O_T5D_&D0+wSV5j=(EFjSUsL}X$~7$Twj zi4N!Mi5%3%%9mp`=2+qBbV^iVp>-${OxF?-G5Pbckj!wUK97CO%Uk!(Fknl~Sbmpt zCU#~ovPe?w;Gc4~kY@YytuZr+G6XSiX?Rl2PBA9Tf_`@*s|x)tFh2+6c|;4;jm&4e zi6gz{rPYNrLng2+2jZj3of^jRX=_ec!n8|qCgO$6Q zGJfrvt@PxQNDdI|%LUb$XC@gS&#jDJ|j)C6Ly zl4^lpfIKn;ljniKu8fpak#XmRkp$H~qj<~^Q%guk3!ms@V#8-_F5Qyu3zLkbS@qxe)v!5CE-vNrr5c2KEXgm3>-CgpvVZ#^@zm)uBQ}jy9Aa z_KcxgC8VDjrwA3^+9s-+s)(~8O!_*ZoR)M&mK8&KSj-S)h_tCcKqct&T@iT18&V6< zR;Kap!5ij629=yzlT+yw3Y+}cmVEnI(~F&@Y*=J$;#M=OcA9qJR|{wu`YTl1>eR1N z!45`m0(Kon$&YDr_O&+6Gw$9uP43MB)IT_c=5qvQHcPRq6%aI-xk8=Dm{cgjeN8V4 z6DoecIr9x*8R>V9Lg6ZBglN~}!1-p7;IXn8+kAB_0jD>&@+|4a!WE0P;<2W!Lw@7_ z6K2wg132{IQY;)qp>i5x7q_F)2!!jQiOnPSzz86$Uisg#pkO)XtsN}UEY3VKye`=-BPE60Io#Jg^&%kyNS+LbK;!VB)2+zwD$>tTb-&hD z4zWuF2~bazv+vS*1YFf(>`qsUM?^^fiH&{yI$>F8?l}R>7e2Vk%N~HnqV;%Tm^U2b z*ZIChbEVUbj!z0s4qnhG-=5mY06Os;4$=L>VE`){GYf~@Gxo71V%c)?vAJ_!@F!3t z2r`D4bW1KBd67iK&UX%{=u;EH%_5;62v@2*@2CFEHe5KkrOgeyGGVk!!{G!aESa-w ztO0hlVpKTNX<-3JCP}E=G%#k{IyUni;Y93T;79(A%BD0EsCnsqhb$83zD~4SFDkCUa;4iuNvv64D)= z2cJsFhfcC+LyoasXRSEFq%c@w5CJXPvo^EqG%Pa$T%@EbQFZuQ73CXsrdlU`0;36A zUnKn)eFp@Ok3*|0+xg+E0z6{28$BGaU%Y!=6f%uDhGe`OmL@>yk>5km8pMCXR8?V0 zy((<_6zw*XA^$wkE5RB~4n*gIDl;3jLe-{$#n^O;EgJHw({Srvggo;pv4hcJ*|9>Q z<``ObKtkAbge_T(Oa}{fBL~|^QB(#I6!IQ!apL($@3FbUz5dC!qN@*1Ja%ixac z#4NzWq*<0;yk?Ka6Pqd$c4LMP-|IUG**2z921`0Xrj>*!nEcoxq>Wbgtl5G?z$nN& zxHg*NiHu~&XxCSJEoHs}@Yb&w1ESMcR8I`0jaodN=z-yE=!DLKfnWnMJdWTkY{L~G z){c!yP)rlzwAZo6y3<1z&H_0?A5%jH$8n2&4{|U-M4nYxo-2fa&w2vMt%GDC{gVz= zDoHXi;Lb`IR&@Q3n##Tj$5`XIXOj8p@}TkfR3RI<5LG4e^PTK@3K zkp*dRwW_gT(bWtr4Pn6rPF1ZlsHV}vtaYK9L|Y-Bnli}h6f(;Y3?wgOqagQ%TM_84 z1eu5yF+|ppN=GfSFU+mhoK|WQW1D&m`xGiMpkqR<-X+* zWw0hJB4a>#oY#aRl4n;qG!{h&GmC-)GKg8i>Xo~I{U?41i82jQ1;p6KOl+a-GbsuP zLBk!|2pDs#gh6zX(+)ZmWU#8Bo@jUu>On+e{Ib*v$TG%^5090Z0u_mPKBdPg+n{WF zK-R=r1B}?Vv>|YL0owphoJi6W)>6o_qIC5t)UV>PTGC`0e3l$Ir%cE>6eYBA4j#WJ zA)u6^B9O!n5T&6w@ZhEZLFtN~p_@`hsFYcOc1#GvXiprYP;n5Dv0Zua(omD%xR9hl zhenpzuvw5I1GZq;pbAB4xO=n;^J}UdjIv9PRTg7H?AQp{W5wNh zhQNIS&l{8hiXl`4GMj3_;Mlmuz%hL%ejRXNeHxUSb{*v?jKN2fIpw77T?^=|x2Gr(UZq zukQW$(=tU8Q_q-fOyR>MD?(O76U(BQ77U(c=k{YuGqZU2$VkjRyHiNglw$=FJp*>p zWfN%y~!5}-253glut`c$!b7@$@he|l=jN(KH93`jyr3Bm{05x$KKnWt*qbO--y zeNET21AhN(ZASA;zt`)|Ad9^)$T_*?r;!aEKsFS%P5!FVzRYJs4N5ul+~90S5-3hn zTCAdangQAZDgckLB%dTYXsCR1FDY<5c!mZ8dJ}I4dBb4br02>J|6|Hz#-|6iXU(V4 zIq2NC4J%&w4pQW6S5S)HxWc!yL}L0Bw5c-th&&!(E}@#W2_~CaJL`uH0h1K_gB%eM z%srDoRmgu5r6RKSDG(3*HjJRx8X`qoO)l)ovRQruVgT%*go<4tkyM*T0W~kibMIb^ zEsQb!*onw=Z>|+3PF>g-v7*SVt6RaWjtjR_?oSnXz-*wSg%bL7OY!8yVJ*-5y_pMc3GW5cbq=CFs17+TfjG3@7@OUR@* zLehhcbSD9UJQLiHI7e%i3!OpOfjab`EZdF5vi-{Bp#c?i1cxl7h}+mw3f4jDZ|8V` zAo1WD7b^3)b@*sko}lcqj`qoU>LYN5$!XYf#t@6dIf*cgc>)pA`i04*tB3WDCcxsn zqDEmD9zLpW&ZuOIGo?tjQ$Bm##bGt3Fp~k+R-@J&U*w>)p<2N!DQhCjkaHV@G)1Bk zZ`CI;I+Iri4tzdsRA#kUbwpOl)=c{4!WEq!D;6SD@>nTJ7o_qe+P>V&UJDA+ytM3!B-nU-g_jSuvnm6`Vdsi7jg|@mL}DsqB`tsYaOII>317 zYdvD75Y0(6pU|E9uY^tqKSpnM2-6O{0@wV$#59;vEM_|9X%P~@!xOP5+2;VeBp=f# zc!bFLj9ZaL0mIm{iyMqPe?PX54y%P>WqRUSXSd4Jw|IXx-pEWH&6d?<+DL;e)0oPj zF++Bb=FMDMiX6tj`CoxU_Grg#eZZ(0ra2lwoVX-ulI~QawAV$B$AW6|?WaF5fNmQJ z$gFj<1ORVMAO;#VQ;;b_O#{@Kjr4N4gCepIln0aLZeb10?1(mjSwlx99d*WPk6F(3 zh~No=<&nZlCo@Qkf_{d_q25qQ(=#T;F(;Xh6tNnrg-@L&LpMkvjSS32j?5E_429)Z zrAqx|(czJ?%Pt)QXKm9B(nqTu31YzIXCjhWFtjj3u(&RlP>cLv81VXmhPmYg{s z#N|#gW(1qk&yY+nBPLaEXtHJu1~Gv^2J9wnTwj<3Z90fy<%LVSP3#!W8Z+$P$1nR^h~u2vWgNC#qO$F=yH`gXy9P#5tYloq08Nz#Kg&7a zpL?`6&=wfVM2>}-;RK`91yj`0>W4yxIc~L9D2kG=ZHQN1AZC)JM9*0|fKPL1Kv4lp zK!vh2Tcjw&l51Dop4i2+p;HkC%v}A{!Ig|46u4T!+t8MHHYAY88NVq!(*cZD0z$Ri zV8ed0bB7=6M}e_OY8nr~S0^|#QVIYzslTY%V%+FV+XHDf#jgq3ztB6*``q%(9$!LY zd_|MQ4Uh+eMNd5_1Zm?K5D&2k$&joaViI5S9h!_Z-S)Z*3n9K?JXR<6P1m#1}h9oul z)xc~KU&rjRSxBu*Rq0;kgR_TC0m2|m*IG2l8yPGhzq#|emDopGAnc2(@ORv1Ffex{ z!)9HQwBugOX3Sc%&k5 zILDyDg{-D9m}TE`l3^`c23Wjh$6)fUh;s~p=>yli2ylI2x&Cm5c>zk2*Jh)ROIw*^ z)vhFYe{04!4A_=Ou~!5t3FJc))c9D z)o2}OM^6L!l0twfZova@&~}8(k4QXs{J#8617Fc{k=p4;#I~Muphe_{Y_w*{k=nf? zvBE&OV7P>>yMzHCceYFdlTmHfA~6sT-3c%mB&yhTsn!C`K3Aq`#_0D(SPr|V0W{>! zLMhNZS-+PGP9JqTF!!0)3ocIv!gOVyuWUy~;~z&Eni+WmR1F-JQnYW(DJ?*Zntp6B zz(=H>57h)-07U@m z3~vE(L!=xbPdtKC%14H#%Stw@9DvwWm{4)55JgzP>S&Pwo@2sNCBuLfb8L{3vh`c$ z1ZqtRAHfH$R!v+_cT2EmT-%zz3RAA|qjxVY7sX2gqNl2OwrW^&D%Yt;}(1}WvCTJ(N` zK)|&@OvaYiyoodn2xO5W4>1N@$0%?dq!TQ&U7`HbHRV(&!iq*8YZA~{8ETP1#BJMy z$deKVP>7h&5N$YRfM@m^Zbr;of9y4+rPquDcOn#&YK%7$P##>X_Pn3$0UHzrwpD>| zLdfb&OnIZ?N`z7&5nKvnoDSR)a%a)I&I3;ounrq*;8?t{$e2;s$EVpB?!-K&aFU2~ zJ#C0f^pPchph;4ph@}Ix?uTnqz*xA?I%10)R|8xw*<67p88j&5fz~{0DWRCn znoro?G`(;jVc$daz=1%S1B}fljDtN!P~cb^9jjZFc1N>huQLO8++$6p1pkSBjyP5y zcx=RUO(d?YRKPJ2q~0fEtS(fZj zHc9?z4>uxAbw=giV98{_r|}!Wfmp0J2WT7_c6HF-+qG2&6plg4t(qh0yNkmqXJ{b4 z^^V3QWMZYMTLQXi#iuraCic`C(vT*R`nZeC45 z4EyBC0T>!{L4r!;-}!|l#dng%8f*sMp}@Hqp-Z7OU1uVi#EQl_GM>W)0Pl>lu08T? z)hj4i6h=2z&G@RB(SD_538qM-SYQjl&IWvdb-!5h@kQ?eI4Cgfcu!QTjbW>zJKYbt zjqpfCJ;QpWIpLB7S%Wp@@5oGG{J8>@Un>B6=%r<L=YVyX`}`eQ`VGggS72i1~@=$3Y^M3zlH;=qJzXAd zjNFO`!cz*vF4D}|Rs13k0X!0x1g?x7BEg${6L3BV(a>F++up!{VN16tM}pA8W(U>H zM=|!qQa1Id&3V=De7=$ZPP{B>l?6AK!WB)y=X5#@K+hgCGve8z#b(X2glCPZUv+wD z*kjW@A!@>v1P1K2&I-^}1+0oBISJKCGH>aQ4V%`TVg8XObI3Aq2`Ar3qH%AA>)Z(= zG5!SEIdWi6%#T4>jkq(tg%G9bmu7pyRqJtvtPkWN9h}c3e1n&*UbNZC&JIgTS!CEf zIwQ$On7PEIqw|nQrI59(Smn@hI1sPPafFS2Y&4DLOy=nu@#te~J!!Oh3&J(~q}K+u zZOu`bq8Av-idAcU?uI4i7>p=_$fzk&P?d3Q9V4To7$e4C!xBmbWJjRpL3L-=elbX` z5$_ivuxCpyTp6>29lIvU-&~riXTfGbV5$+ij&Nl_XWAlY=D|iRaN?y2iOmNVIPxG< zqOoj4i&1W`Xl5N>Aj-#t0U0ZcqREoO2lCQw2JScU^K`o_47T}Z4w&VHn{dCHFy;vD zGl9?o91E_nM5px%|2Gcz!kP}oB;ao6I9!4>PbK#r_h6p4HvY;l> z08i=`@GWSU#ugM3+6T=9TFY?C;K!iBx%H$&tfaVH;UeLy2R>;MK=H;WS+(y@N;x6n z;WC31wsL5_Evna2TB99foQdtB!)VJ(C}=~ zw(*hj*!GRAIt^vTfM&=yu`zg6!*_{-YX(&&5$_8KG}wrKAW0?{C?OszTE?WmEb*W~ zMd84iSj-w_QV*u zY#W%yfHX)Ux3oaC@z z?I>m2gEBermm)+rG5XXLL`*&4f9GY5l)pIrr4U#njB#@ODT#3*iUHOhh+^uoN=rHy z#tCZQ$U)};;SX_y*2SOFp3Qj8w>Pnu0b{KbBNgs6|EN6)O+=)}u}iL`IflwHNA|xJ z_~OM5YA$GFB-Y>S7?U6SNyRue@)yJQ3UAbg*CVHjD$y$z*^{B0M)2nErRZnBJ`!fj zubmA#&_`m~D`R&r#0HEXm zoH$(h4!Z*a^}i)Bg_yd9Nk|Qo<*Ba9;y!UTgOg`I*Giu&Bxdt61!eGv!lO}ux`fVb z%L3$uv;hj0B<8C-BS9u=Wey#gwoHVXkeehd9dLJ>B|LU@jwK2Z%Et21MQqKr?&j_4 zjc35lN)u~hN&oG3f61(*VMOE7&OG`Hl~^43L5fwebb%t9=@CcEB0DPbn-Tez%?ndN zkSRv5DE1&bezoDJK^lA+WzCoqX_(&{(irg8xr5~nKWpR-8!aThg)^0VX6bc@ar=&t%R| zFv^A@YYsD4C)T0q!q_=XX;OYZCtR;$+;Yqr$bp+NQ_yTdN|&-72M0^CgM(P-GVTQp z*0xwta(QJCWzycu2ONv67A!#*ez3Em!^u(e>N8|U#8z`^wCDIf#s}ayW3ktvcUZVT(8GqK zT2KbUwBIfS<`Qc!>ni$6$py4=EH|j=FwQwdF&4y*QN(cM&+HU1$eJ_22>PXfZip@c zIFY*KBuk@*`t*{Dl59)^g%-nxh|U(;h|npKBnoQXp>xF{r;D+C>mh+FIZi=~Hm;FL zBc|B{hCze}wPbS)uCYZ3rVq&ugK>%o=)x$9cgb0XM#HW>$uUaazs|5Ja|kA!07A*i zpc!Z0_8D&?6NkhkXgm;BG#XO(Q0%S^zFAZirccN&q=CPI#Qbn0qXzvhEL^_f&QxII z1gR03EKxOZ0@AA==g4&~4>rk5jJgvp0U%1O4daEZIbpQx#Ujwm%oTxos$yJ%RSm8I z(%vp*Fo5Bo-L#S&Olm1MVqsdvh@kLf%e6KDBxFIeg-6OfCGaNT8ws#sgZAvq1!1}0 z>;dX8RvBu^U!bCJkn0>B9QqVx-7n8z`0&n2z~)xt^KMli=gt2nRjeFA;r2%*>*X6) zDR1_<{rOY*B@by;iu;pefZ!3}r-kInN~GI$>1eeh!&V6;LS;^q94nf;s?HLm`L*lI zbz}wrub3salFj96KEc?0V~$BJh9Q=dM74!*l(II3t2#(%arXV_T(H352w3@owfJ^g z<|)pMxz+TH?w|{bQKRnLy&ML#97yaUV*yx2N}nxs2^?XOY1f zjFEYe@}RK5cn`I^o-ntSfugXsTWMa==!r z1IiH&yC#H2>%De8%E@4H_9dRnd!InZDiNw?%79s8TN8+Ri-@Xkf7j9{S@SlaGDNLT z2A{x>y-uZ1#+WlEd{Bvokt$qPDCn|m3xkrcNO|Xy?2xw=3`sCPm<`vG!Nve{Fab{F z680DaC{iHedEu=`TjTW&7&$Y|gB?$_t;E@79DdFUrH8aO0f z6 zWD%lu_azHWR!0sVg|!DU-Y($=Q`^V_fXTtqAQ*+98X-2?6*5Sg67?Qti7IZBfZW*B zwqm!cYA}SyZj`~8Z6eN>Cq5Kb#uUuR{$|z3BVfMK>wt+s`;h5Cn6c}E4)-)NX*wD| zUMVaK6D>%xus>@wVjq7rvmQlUuvoJ^HQ}Pf)JdiT8c@D_k9i)HeD?iAT2GsOk=mRn zkVx2-k2(niL|WU#xEeLn^ND~D=292bOGZEAT3H3AIOC@d7Y!G3q%c1gq!~L8(J6}o z!0jXjYuiwt*e4ju0Ges9WEi#xkv31sjd>)P1@7YuaC5aB6UZ+duCa1gyVWSGLau>CwngEz1vWgL2| z9b5ELoEU%;wy`NcgfL^)6}C8#VffJlf%sD4aAhO8QVz9>_aPojDboVLMJV&!5>lB! zU|W100_)1#5zAek8x~9MqJWd-Xq~WN^qCB;TKt_UIInCpl53L|v4}@EE}CIB@YK<) z3(#`Q=8#KDm>kUu?oXi#dwOJ=H1mSU1@V&E1&1ImNOlvFAj_#qcG!?4TRtgL)~SRr z#iWg-8W0WH*4hzHt3nEKzC)y$Lm=vkGUaPMbC{M9*~^m+fZY*OW{G8195BE_d^q-7 za56xsH^`#Qx7ObXm|Y%=)ae~nU91b5~j+rNYGiQxYr`q7Q%s= z72%?WcV=nA2_*=Erj1u-zWoO#ag>^2N=Uwi?7DJ<0L%#rVL>SQ=Cm@IQPpI@U~=jf zVr5=MNVd#s&0xiotZ-`d_C1Svfm=Yr>&n2D;1t-%_v z1*5!b&Ug;_WZj6Blb9AOCQdn(51bn99;YYs98qI3%{yYt`J}s^_*k#JX~i1ylILnC zlnm+&g4VSeF58U+t|i^8R=KNh7-R1l8n8Xtzv zAYMkuD;T@(M936LNg$^rbHE+0JCNs+$pc_mUm7XW>(jbzE(>5ua4Gtwuvj$4WLOA| zDUTBUS%6AYWeNp0q6en0U6+*cV2)#yx1ssZr?hBU<1f zVr@9{dBBy4Kha-1Jz*^@U$ta>hYp8qp%q7kI zTvBB9|DF>gQD8uP&|si%z)q)1LZhlC7}MkndP$FdU*JLb^rl16(3GUyGkWH( zObwj4re?y%1^2XYVS)OK298VEY>aQ<$|2ib6BcosBe8Z8jwAw=KGd1fH!?VehGwzt z@rCpN!f>17%ok&`f^N9h3=DADV;5#8mfa?kq?#wI)S3cdFvyvGrg#{H!qhaYu`}q% z5>Z{tvgdm{ro{Umk_ZAH)B*vf@5==FxFI0sl{bEa5@tLg`11EF~Uny$+4p0pijw`^jl+<={KUQ$~;eL7(7pTiNZtZ38VsH#pWYC zNGjhj$#|S1^dsv>#1I}9e8UIJ z$t9*&l^`Ayz}R-6GXzG4J_RLgT}!M(2ArgtBi%8W_6re~()XV@O+$&mAxmVsx{r%m z2l582-XN~s1q-~aiM5iLmYhoBX*nq#>GRzJBqy14&X)y3q@+yT2;V5+lNLwCp=mM6 zz-ZubN)WY(2VO!;z00!w>3}tsanNNFT(e44C=mp#X_O^I^8FT_G0) zYaFHsR&=idOzFzDtvI68F!-oXHUF#&k=PKn)#_yE5+CQ-p~NtiJ4mFYM1UrJvSx}y zEf*6)hq7bx>jwu+#Q6rNe?r3+k#9&Mdqq&F)Vew)Y@dk?a~Xl(pv8)kRSx0t?Gcgz zLn4qeh(pm1QOgox($9{nay~GSM5>2~9+eqx%x()tqGboE<_1cHyhJfpI-?i_@(thc z&mLr|%`9Safv->e4Pe|*QnJff^#t1GXGpd3_ETrTkU0I~?TWwvQ%?}|nXjZ3Ii~HI zfjA&2EmjX!$>!(;I93!cD44!A9)klD8G%s&DdhN}R4QxgWKrpGCQ`>s)0wc)Ud$Ed zKCycs*=-*ktm{;2@wyCR-t1_8+-fu-m;JYY#Q+9;SI-(bz8PC~1&-+|En2XVHCgj8 z1LO%yQ9XjmH}}f*6xT9Br@>k7tv87&G~8Q@4`(}>Q|lafvWg3wo^wBzFs0YWSY;*4 z8wP${<8N$dfZsLTTR_e{bdl@d_ITbpfEX~AFq}L{6x{Gv1*i?LV@g5z@ksTigE4t9 z{iL0UL7?GDrlC8eADK0#c#~`n!tZ|@En4=fCHEoumX81p97&DOAwEoFZQ<$@cTU1c zzZ12Jj|cQ=v~G+QloePrrfg7J{!geJq`YSH+Fb%|e4RK~0ZA2WD z&oznh<-wy*~S4l#g`aay~HVK4_+mbP@X`Kt2$+tbWJG7@T5^qxpbJk~nx z!qyafy9?(7&NEJm7J;wtf&7t60T4uBCQc%RJn}zGiIf8m`MrSJPOag>aLTLE4>_Ag zX=rcd38C}>Eq>g+$t}6$Ajr_h=3Zno?+3L?;m<;aKy>RH$e}_Lh67vy=!Sf{Fo>{kHT1I3AzjPCs+p#mpqp_Bdl2#)q)WCkmS!HSiab76sgIRIF# z(38z-$+8@lGhbq4@Qp8K*|jIzo+)KoD6R$KflcUe2BJ*v>^vYqCArlYAPI@zAy~2} zse<97k}e*tN7H6Xt-?cm509NhqQ11WW4O@nQ4TOVDA70eOt9QNBaXX}AD@jNT$V0Bj1Z_CPoFpK*@ascdqUhBl*ikmJLB}lU z92+Y3g5t>TidOC8vP*(T>qe5Ul{--`VV|s4REA(}X*;>4XhC>KU1JF_W;|&HiTpx3xY0OW+Wxnff8g4(tnVU0sA*j6FSoYIXc5F zT(E>`X3&diTk;TqS@6TyYRdojwEAI`G)QWBvZRZgaGIyL_i`9eGbYCxw-uH`QH{pv zf)RZySA$1=N+E=&0C=*60*pH2@vrEMLvUq)&X~A;RB+fc@p2H?#DWpKkik@aM%ReE zMTnGwIC)yM*52kAQO=ZZ&SB%SCj^l5kA{fyHija! z7YJ9#4@SvEKOdJotZ)*J0Sp}j{R5qA8+UjXC^pffS+=lnp%ygE9tyJqO2Tc(ng!}& zv8t8_OpbY4jX@UUMJ1SWt%}L{q_#1;8`7#s{n?;dq7VVp0n1Kdl zPh`-x<7;6S97k%uLn~9wesn|+Ug+eV?snCa5jE z62`qz5w#c5LD>;I^R0I=%mgi5?cp*>Fon4?5{C1LQMc(>Xu?@}MA-Vk_Zqj=2Qm>B z7=cxxs_-U9C`U+``k^G75M13l?}`tNSK8Q6yiW1*lA^*pP;V+z#*6cXtRl1TlM)j) zt7@NM1xS!7PD!fZor(cGvvrNFL`p!Ex{`>Q0z}l4F#yMzX9Z3yM=^33wTMI0$R8#g zB6^IDZO2StNP!UufgY zQckoi9mN|i!8V=!m?VZ8=P5M3^qFIJArMewwhswv;c^6p1IM3xoFxb%X~W@?__T$5 zc#LYJih~)K8FoLAQ|v~|5LQCk1v{G>InsPPjXcClh@FBYq_KArho86Vul8#dDba1x%(^Jh}9xgys#`(-yeM7L7BR??C%O(KX zrYQ$_P?o4SG+2Udr!hne$y=QHfAqLd*d>^k!mq ztWn_vftoX4E(jdpU?I7|+0m-Ra%(a)BN7=d4Evr4c#n48C>fwpq~$=%iabEldR7Ce zW5X4fE@8upMy3reHIsxwhEu12q~Y`ld0Hfv7Fw*@!jd^y*;r>hJmK+JYy%@G7_x<( z04_oiVt@!Z!B|2a%jO1b9G>K(u({9NMr3oqp9L&F(F=UFi(0f(Pj&+A#L8@2atbV# zySWn?49W`uY+PmrV~Ze6qg~TM0>$Pci7hE`$Khh$F7}-yig{cTT3kZ4I21A*trs_Q zrNY`~$jpY!s(y2Vj}e|c3ShjbP0<+mH2|V@BcXuFc=hrYHbHiPzcqx27i7AjF+?}Q zW&wv&_(AY%@I>j{ zbfl)rohPO{a!@i_ehHu<_b#)Ey zPKIAUu)#bSi1ks~#gbm+v79q$Y10{#W|G~IGf85uFFJ7lo;bw*pa&;cf;j0pUHP;~ zIk8F;hq5WcfW*3&7L>^l5u{kV5lxx^IDcRtJ``WVedtn@HK5(Kj6&yOUzQtrxkMtO zP)WJw0EhHW)yLHpc6NxlLF0g?6&U`|r`%Pa z;+;>7m3V0ZJeR5(qpF0RCLRIG9DuKV5FiD<8s*zh76y#9!J^~0GT6$1YZkyS-u50N zPSMBi$7c@-fK?}!J{UYsghldUO05q$z_uDG0M-K~4;M{>lvAX0G`$FaNNt4N+oLyy0cd9095m2FiYF&M z^UX7xX?UZ|b59)opTP%+%P9ymZ-Qm)f}_zY zXu}K!eIn01Q|izllvDciOde7I1U&bFi?>S}1J?3QvmRsa>4nP>id|eH${TUS0MHLa zw29~d7MmUjwu~?W9f-2BDKY?r{T={kn5APy9%1@QQOZ2Pgk|N}>Dse`m8`2*wXBig z;zrz-TT=%`2LC49I@W!xWGW*`7?XHiDCii6fuqI4J_v+4W6g_M&Jx>Za~4mOy!ifU zMyw$TDI(e@5vWlkY=LB5+OLrkoft1Ed}<*Z^^ALFL8BtLr+@)P)wL6ogtJGPqTsZt zhO2&RK16uelvQhNpXRFfIJY+irs*xhoMMC=w7oH~@e4>nG3pE;P@0nhm~md34D$QEz9U7QkiNS8`OiOf_w)f5TUsow2I1?i>Y7qi0{ zB_(yBxBz}N^H6<(Hpy#za=dzK7RhE=@36jO(0qxg#3Iqi#jZ&sPkE}`g?SV7~ zq}PKqWXy&HOG%Afu#uSI`Bu=O=mc{hiJLH4QJSAgE`;9KEVW=PfCe5cXje$vG;%pi zA?^G45Fo|JNI2bd@*d9mlMhG!wE$SZs*RQLH->+gOf(I9MQ~WV|LMSVCF%&6(>tfPWUy6NlIJ#uG*`j|`cV zU8Dm4A+|{PP07&!Sj2~q?FTs+U@HJ3_=38l*Noc);J_WSA9a;4%R_FGZ6lR2$<$)8 zMZn9UTq;ROfm$|;;Tzg4Yx=E!mxwRR5dl%ok@G=?N&FogIUK-NfI0!%7FfdKOgq1B zQ_>%+n4+^!2*o;(AJ8$WkTEKnI_+K)bYMZ8VRra;DtPA}18gigPBa@V8B)lUk{X21 zDZ-LZl8bi=AViH2HC&Ogj42yD7MC+HA!JDvn!kUdqX2j(2Y^-&MZR~5St4k8N&?IY zLD;z~cQw)&6e(Fu2sBq1HC*aU;#E}#%o>mhe17mZkO6<$h=uP5==IU5uN;77kuOH9 zR56P%yhN!zAfGpeNk9<6JjTp0<UUT7%zJ(a>GgD1ef8B>RXxvZXH~|jV~=%PJ*MnhUWP}jCEiRDQkd9|Pz4fouL}e9Hi%wp@E2=fwP5nwKHZFXnMaeHW zIBo5xT0(gh$D^`kLCX+u?vZg4$>%-lQUJ1vW zn{PQ3LCm%Zp}i-%zM~cq3D^@C|JS)djgSN-8Jw4CT0x$4^csIVYk% zca4n9vA+QE*Da0?-0@-e+!|OWYp^55mBmED>vWM_D6e4;HHBj9W1LeQZ7K&tn{-w588endUa; zf7)y-rF1-y@(J&D8#f0pX?hN!_5rapT0KPd&OZ1OEWB%nv~bNpr);x`k!?V{@C}04C~F?l@w5x{U~FWg4L1HV%N+`~$i08SwV?Y3d3=3bdvI zfb_qjB_(8D{P{1TlLS5=8(cT#J=-7&&S3%_M?;svUU})VNUS{)f=9X-zh3u=_BBMd zT_Q}=QWm68s3{`hEF(hzG#q)fR~5bsXccAJm;)_udF#6#LLIm_h<3-{BUj%o!>T{9 zPFScOxpuz}BI&w#SfZBw;*C-~S`bg#2Y8!QrM7EizDevMPr4cfdBY^cF)FZzKbmXb zi|J0Jsm9BmuQemq&o+LJnH8EGZmEp5zPkW+z*Z5Me9Q=umL7*mO_Jo3U1bp9=2s#_ za|4!1yd?1u9M}p+m(NuJ!<%IF-S5IWzg9n)6331-%!CX-Sm^`k@+Php(a_2GneKrg z!8V^l%6b0z(>BDhsLkAp#uBv$M#gWyIjxmSk>9lMhaG|t;}%RjKEdYYUk|kc9%R}u zynX|)kMeWV@1XG;lF_km2y`gBZi2wJHMdVIcXQy)9d)X2PM7>UJ+6xJT-Mn#tNxmZ zdGOb^ovd=;miGfoD3-*|Y!{wuB71x>suc1`K-n%ThI=>0bFb#%dHU~ApuE<;f(}2# zy`vzNl1eSacd8P%r#aqN%~8c)19;ti?vFI1=31qch7D9ADKA=@?wB>pDdF$WNL11> zrAnJam}uo9N=FDj#7s(K!Q=z)r&taA&rpax*Z_$FP74-|44JvLu8L`K+{}34=<_a&o~7-VLX1pRfR?;hx*@0GAKMp}F#{ zg!19Exq**>Xp|&i^U|b^IVxNqGu#>4BP0xML>oX`kGsxef`BShv zjZdm5W3Ru{&4^EX?GLU_*90xObeTH(dTA@zhv(g2O}=sZaj4#?8JRBasrpet z(Qgg!#;&tp&@3MN$XEoSA6v4|T7nLCPZfM*Ha@|*C`#M z!xZ4%06__`c3&_Sep>w&cwMsyDdA=dSxN@nH=YRng~}Kuper58?tjH)a+?_lRh`nV z%Y^!(2bFrqYPGx@Don}78c4CRMJ%qI5 zMW};ML-kwFmyj?KD~ZnVG_9yUZRdKLTxPLhCmB>qWUmJmy(GNKZ!!)+!>87NPxcBK zoYKVy9IJe?k2^6xU1S)L73_g1EY;p*iTtA zu?vTNK63Y^?1j>j+p$C(k(Lmne3yyzknG>1C^K>LJH~yQf>hy?Vbk?^X~66e2j3`pp6&R6eAMRtmTvGfFzom=FMsYMH0OtVjLdd*Ike7aMRGi2x z8{)XV!xddEe$vKk%ZRvMnT*}gV_Ibi6}}P@6wsy_x)}hZ#cvi<_M%;7DrR5wA~HEv zMc#$0>~S6w_}wM*?_@>h-6`e@E4eyukv{{@z7qd67_n%^X;x9t8CwlKp_}bUHrHjk zoyl6AAYNe;Z;HDR$^T%N%mk1z*=rhwbL%;8AHTIYt z!9BHHl&hrS-bDBb01;2iwqpU7yrdmwA-kCmQ!+8Hv9%S`@-S3Nw4+zSRz(>v!FmOG z6xTt}6b^OVRg7CAqr!`f z-s2F{kcp+1n9*RxOJrjxI32H%XT0oT-pKkn&%vnNj9UCp`Jyen+aL41n(+FUZ>U&w zjr%8?)MCw>=9G9eE|*!z2A~mCMbMx7I+ zy)5aLC0MTC2nn11N6N;~k)EoJwsGUgbZ?~Y0bFZq?m|}<7qCeIpyL~R{U9qpsHfGj zJMR^blDVd6nQ*>m=6(el)|L70z?PSoOi(9v4ztM&X!dT+9%u!pM3zE}$_-*M%%aZ= zFosOZh{CV@5t*2RfF0)K2-yLdN!Ra~Zd8G+3l(t(`Y}IyBo|E`dQ;p4hKe}QQ%9Xz zp6+@zV&y>M&pDF@6`XWqy50K#qXc&MJqu+jKIqx*ViJ&H&sphLN0(a+lEbp+lV&kv zX2o378+q(^9E7~lEQRL%mWS{x8oax$^ zpC{T;ZFY97jW6KJ#ui|hO*Rc~Wt-7TN?#@CCs4U z!6LY=rSoFV)0Br`1u_m`h&x5|ARgeLPc{X}3EP8jCu_ZXr+&!7N^X@P2-sNa70+c6 zNV$OdfaKN8=g22Q3rfze1?l4Jq`MSNHu~Ij1gs)aOmP(ijS8HEUk^Y_Cu~T=}M`b!Q%cYo{Fq2iN z!r?PKl!)eM6JXhhF&~fz9-{;T-KXx6ydly{DzE?VA}ptOwA^Zii{-$m(z|Q4g>YUA zdlG4V?I>sJSrvAtdG2KX8fi$bBSis-38{$=uQbliS%&pZ7vFifh>=w-5$?6|iML=I zr!$9-b)0T0S;0GJW1c*9@XzOPh7OP;1hcv?xvpB^-C+pmDgwT1!*t@G7n#J>K za*13uz_V?kdGV-4>tMe>Mt(__b;NTW(X_U_%2v0bM6Xhlcw$}6CMUpT&k7%vpyFCF9p13#fz5_M=+|KiG0rB8}Qi4au{Q(lp8ipPSuveNbSH> zHWd(L^Z{+YQ2l&mYhmC)+?v%Pe$xol8JLGc^-8YeekeOrxy%wPG=kqjTx927Qt&-V zQ|u+kCo^L2i#-#0Y>Oqt z*Md{}i1jZWxo5qQ-Kd~c^&C;Cc;MWq5qR|KIP&m~-lJHXD%T^S(G@%Cc!puzv|$E| z4UeIk1kvX7D_$MR)q$-vk(VoxGXZ1uw0;qq>Ja+qIM943_`ljCzwsC@3svQkY3tNk zhx7#6sA#K=lRVYFy&Y<9y{n#PR7X!wo7ZG_H>2kOx5eL?=cnA0UI=4XhkJt~)O2nR zqL-nGXZVNfjUc7FIsF-^ya|ctw7re(P*R7U(=hA#H$*@ve`s$7+k640$g)}_sC@UG z>XKDOnoTw_AWEND>6T8S)zskMfSQxbf|t@{hq>PgJ(XFV5OpuAzk2Y|PyhVx-QC;! zm*2kq?&bUYuiw4Dd-<39A3plYvu9ua_T{Hn*H0f^J$>}}`r*}0V*BbpneA6!{OOOW z?e*1zkACyFSMUG+{rx{5ef$35zwY1v{dbNA-y zo12@*pT2%_|K`b)$B*x>o?Jb-x&P(W*Eg>}yZP+u_5JI+>tB5QzaRhX!2|98AAJ1Z MA3gZ;+3&Ca5B6R5DF6Tf literal 0 HcmV?d00001